From 815c1da53ff699396a09daff5bf4ec724a51d4e4 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Mon, 10 Jun 2019 15:03:01 +0800 Subject: [PATCH 01/46] Add fields for fbs --- src/ray/gcs/format/gcs.fbs | 4 ++++ src/ray/raylet/task_spec.cc | 17 +++++++++++++++-- src/ray/raylet/task_spec.h | 10 +++++++++- 3 files changed, 28 insertions(+), 3 deletions(-) diff --git a/src/ray/gcs/format/gcs.fbs b/src/ray/gcs/format/gcs.fbs index 614c80b27672..ff5676a3460e 100644 --- a/src/ray/gcs/format/gcs.fbs +++ b/src/ray/gcs/format/gcs.fbs @@ -106,6 +106,10 @@ table TaskInfo { // For a Python function, it should be: [module_name, class_name, function_name] // For a Java function, it should be: [class_name, method_name, type_descriptor] function_descriptor: [string]; + // The prefix string for worker starting. This is only used for actor creating tasks. + worker_starting_prefix: string; + // The suffix string for worker starting. + worker_starting_suffix: string; } table ResourcePair { diff --git a/src/ray/raylet/task_spec.cc b/src/ray/raylet/task_spec.cc index eeab29272126..e766678af974 100644 --- a/src/ray/raylet/task_spec.cc +++ b/src/ray/raylet/task_spec.cc @@ -80,7 +80,8 @@ TaskSpecification::TaskSpecification( const std::vector> &task_arguments, int64_t num_returns, const std::unordered_map &required_resources, const std::unordered_map &required_placement_resources, - const Language &language, const std::vector &function_descriptor) + const Language &language, const std::vector &function_descriptor, + const std::string &worker_starting_prefix, const std::string &worker_starting_suffix) : spec_() { flatbuffers::FlatBufferBuilder fbb; @@ -101,7 +102,9 @@ TaskSpecification::TaskSpecification( ids_to_flatbuf(fbb, new_actor_handles), fbb.CreateVector(arguments), num_returns, map_to_flatbuf(fbb, required_resources), map_to_flatbuf(fbb, required_placement_resources), language, - string_vec_to_flatbuf(fbb, function_descriptor)); + string_vec_to_flatbuf(fbb, function_descriptor), + fbb.CreateString(worker_starting_prefix.c_str(), worker_starting_prefix.size()), + fbb.CreateString(worker_starting_suffix.c_str(), worker_starting_suffix.size())); fbb.Finish(spec); AssignSpecification(fbb.GetBufferPointer(), fbb.GetSize()); } @@ -258,6 +261,16 @@ std::vector TaskSpecification::NewActorHandles() const { return ids_from_flatbuf(*message->new_actor_handles()); } +std::string TaskSpecification::WorkerStartingPrefix() const { + auto message = flatbuffers::GetRoot(spec_.data()); + return string_from_flatbuf(*message->worker_starting_prefix()); +} + +std::string TaskSpecification::WorkerStartingSuffix() const { + auto message = flatbuffers::GetRoot(spec_.data()); + return string_from_flatbuf(*message->worker_starting_suffix()); +} + } // namespace raylet } // namespace ray diff --git a/src/ray/raylet/task_spec.h b/src/ray/raylet/task_spec.h index d557c188ae68..004c0a754b2b 100644 --- a/src/ray/raylet/task_spec.h +++ b/src/ray/raylet/task_spec.h @@ -128,6 +128,8 @@ class TaskSpecification { /// will default to be equal to the required_resources argument. /// \param language The language of the worker that must execute the function. /// \param function_descriptor The function descriptor. + /// \param worker_starting_prefix The prefix string for starting an actor worker. + /// \param worker_starting_suffix The suffix string for starting an actor worker. TaskSpecification( const DriverID &driver_id, const TaskID &parent_task_id, int64_t parent_counter, const ActorID &actor_creation_id, const ObjectID &actor_creation_dummy_object_id, @@ -138,7 +140,9 @@ class TaskSpecification { int64_t num_returns, const std::unordered_map &required_resources, const std::unordered_map &required_placement_resources, - const Language &language, const std::vector &function_descriptor); + const Language &language, const std::vector &function_descriptor, + const std::string &worker_starting_prefix = "", + const std::string &worker_starting_suffix = ""); /// Deserialize a task specification from a string. /// @@ -214,6 +218,10 @@ class TaskSpecification { ObjectID ActorDummyObject() const; std::vector NewActorHandles() const; + // Methods specific to actor creating tasks. + std::string WorkerStartingPrefix() const; + std::string WorkerStartingSuffix() const; + private: /// Assign the specification data from a pointer. void AssignSpecification(const uint8_t *spec, size_t spec_size); From 21ffb937840d60d8fe2e07257703a88455686066 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Tue, 11 Jun 2019 17:22:56 +0800 Subject: [PATCH 02/46] WIP --- .../ray/api/options/ActorCreationOptions.java | 25 +++++++++++++++++-- src/ray/raylet/node_manager.cc | 6 ++++- src/ray/raylet/worker_pool.cc | 20 ++++++++++++++- src/ray/raylet/worker_pool.h | 2 ++ 4 files changed, 49 insertions(+), 4 deletions(-) diff --git a/java/api/src/main/java/org/ray/api/options/ActorCreationOptions.java b/java/api/src/main/java/org/ray/api/options/ActorCreationOptions.java index d1e92f7bb9e9..a0d11659eade 100644 --- a/java/api/src/main/java/org/ray/api/options/ActorCreationOptions.java +++ b/java/api/src/main/java/org/ray/api/options/ActorCreationOptions.java @@ -13,9 +13,18 @@ public class ActorCreationOptions extends BaseTaskOptions { public final int maxReconstructions; - private ActorCreationOptions(Map resources, int maxReconstructions) { + public final String workerStartingPrefix; + + public final String workerStartingSuffix; + + private ActorCreationOptions(Map resources, + int maxReconstructions, + String workerStartingPrefix, + String workerStartingSuffix) { super(resources); this.maxReconstructions = maxReconstructions; + this.workerStartingPrefix = workerStartingPrefix; + this.workerStartingSuffix = workerStartingSuffix; } /** @@ -25,6 +34,8 @@ public static class Builder { private Map resources = new HashMap<>(); private int maxReconstructions = NO_RECONSTRUCTION; + private String workerStartingPrefix = null; + private String workerStartingSuffix = null; public Builder setResources(Map resources) { this.resources = resources; @@ -36,8 +47,18 @@ public Builder setMaxReconstructions(int maxReconstructions) { return this; } + public Builder setWorkerStartingPrefix(String workerStartingPrefix) { + this.workerStartingPrefix = workerStartingPrefix; + return this; + } + + public Builder setWorkerStartingSuffix(String workerStartingSuffix) { + this.workerStartingSuffix = workerStartingSuffix; + return this; + } + public ActorCreationOptions createActorCreationOptions() { - return new ActorCreationOptions(resources, maxReconstructions); + return new ActorCreationOptions(resources, maxReconstructions, workerStartingPrefix, workerStartingSuffix); } } diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index a0bde1ff0655..d121d89fe5eb 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1723,7 +1723,11 @@ bool NodeManager::AssignTask(const Task &task) { std::shared_ptr worker = worker_pool_.PopWorker(spec); if (worker == nullptr) { // There are no workers that can execute this task. - if (!spec.IsActorTask()) { + const bool need_start_new = + (!spec.WorkerStartingPrefix().empty() || !spec.WorkerStartingSuffix().empty()); + if (spec.IsActorCreationTask() && need_start_new) { + worker_pool_.StartWorkerProcess(); + } else if (!spec.IsActorTask()) { // There are no more non-actor workers available to execute this task. // Start a new worker. worker_pool_.StartWorkerProcess(spec.GetLanguage()); diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index d4ac4cf4ecce..556cd6683fde 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -218,13 +218,31 @@ void WorkerPool::PushWorker(const std::shared_ptr &worker) { std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec) { auto &state = GetStateForLanguage(task_spec.GetLanguage()); const auto &actor_id = task_spec.ActorId(); + std::shared_ptr worker = nullptr; - if (actor_id.IsNil()) { + if (task_spec.IsActroCreationTask()) { + // code path of actor creation task. + if (task_spec.WorkerStartingPrefix().empty() && task_spec.WorkerStartingSuffix().empty()) { + // There is no prefix and suffix of command. Pop a worker from idle worker pool of non-actor. + if (!state.idle.empty()) { + worker = std::move(*state.idle.begin()); + state.idle.erase(state.idle.begin()); + } + } else { + auto it = state.waiting_creating_actor_workers.find(task_spec.TaskID()); + if (it != state.waiting_creating_actor_workers.end()) { + worker = std::move(*it); + state.waiting_creating_actor_workers.erase(it); + } + } + } else if (!task_spec.IsActorTask()) { + // code path of normal task. if (!state.idle.empty()) { worker = std::move(*state.idle.begin()); state.idle.erase(state.idle.begin()); } } else { + // code path of actor task. auto actor_entry = state.idle_actor.find(actor_id); if (actor_entry != state.idle_actor.end()) { worker = std::move(actor_entry->second); diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index 03443447cf58..aacacb518945 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -144,6 +144,8 @@ class WorkerPool { struct State { /// The commands and arguments used to start the worker process std::vector worker_command; + /// The pool of actor creation task workers with prefix or suffix. + std::unordered_map> waiting_creating_actor_workers; /// The pool of idle non-actor workers. std::unordered_set> idle; /// The pool of idle actor workers. From f1e24694a08fb5045ac7180e3481aac3ff4559d9 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Tue, 11 Jun 2019 20:04:11 +0800 Subject: [PATCH 03/46] Fix complition errors --- .../api/test/testWorkerStartingSuffix.java | 24 +++++++++++ src/ray/raylet/node_manager.cc | 2 +- src/ray/raylet/worker_pool.cc | 40 ++++++++++++++++--- src/ray/raylet/worker_pool.h | 7 +++- 4 files changed, 66 insertions(+), 7 deletions(-) create mode 100644 java/test/src/main/java/org/ray/api/test/testWorkerStartingSuffix.java diff --git a/java/test/src/main/java/org/ray/api/test/testWorkerStartingSuffix.java b/java/test/src/main/java/org/ray/api/test/testWorkerStartingSuffix.java new file mode 100644 index 000000000000..9810ed4ab77c --- /dev/null +++ b/java/test/src/main/java/org/ray/api/test/testWorkerStartingSuffix.java @@ -0,0 +1,24 @@ +package org.ray.api.test; + +import org.ray.api.Ray; +import org.ray.api.RayActor; +import org.ray.api.annotation.RayRemote; +import org.ray.api.options.ActorCreationOptions; +import org.testng.annotations.Test; + +public class testWorkerStartingSuffix { + + @RayRemote + public static class Echo { + + } + + @Test + public void testSuffix() { + ActorCreationOptions options = new ActorCreationOptions.Builder() + .setWorkerStartingSuffix("-Xss16m") + .createActorCreationOptions(); + RayActor actor = Ray.createActor(Echo::new, options); + + } +} diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index d121d89fe5eb..61310c75d7b1 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1726,7 +1726,7 @@ bool NodeManager::AssignTask(const Task &task) { const bool need_start_new = (!spec.WorkerStartingPrefix().empty() || !spec.WorkerStartingSuffix().empty()); if (spec.IsActorCreationTask() && need_start_new) { - worker_pool_.StartWorkerProcess(); + worker_pool_.StartWorkerProcess(spec.GetLanguage(), &spec); } else if (!spec.IsActorTask()) { // There are no more non-actor workers available to execute this task. // Start a new worker. diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 556cd6683fde..33be2fcb45ad 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -98,7 +98,8 @@ uint32_t WorkerPool::Size(const Language &language) const { } } -void WorkerPool::StartWorkerProcess(const Language &language) { +void WorkerPool::StartWorkerProcess(const Language &language, + const TaskSpecification *task_spec) { auto &state = GetStateForLanguage(language); // If we are already starting up too many workers, then return without starting // more. @@ -115,11 +116,25 @@ void WorkerPool::StartWorkerProcess(const Language &language) { << state.idle_actor.size() << " actor workers, and " << state.idle.size() << " non-actor workers"; + + std::string prefix = ""; + std::string suffix = ""; + if (task_spec != nullptr && task_spec->IsActorCreationTask()) { + prefix = task_spec->WorkerStartingPrefix(); + suffix = task_spec->WorkerStartingSuffix(); + } + // Extract pointers from the worker command to pass into execvp. std::vector worker_command_args; + if (!prefix.empty()) { + worker_command_args.push_back(prefix.c_str()); + } for (auto const &token : state.worker_command) { worker_command_args.push_back(token.c_str()); } + if (!suffix.empty()) { + worker_command_args.push_back(suffix.c_str()); + } worker_command_args.push_back(nullptr); pid_t pid = StartProcess(worker_command_args); @@ -132,6 +147,9 @@ void WorkerPool::StartWorkerProcess(const Language &language) { RAY_LOG(DEBUG) << "Started worker process with pid " << pid; state.starting_worker_processes.emplace( std::make_pair(pid, num_workers_per_process_)); + if (!prefix.empty() || !suffix.empty()) { + state.workers_to_task_id_cache[pid] = task_spec->TaskId(); + } return; } } @@ -158,7 +176,7 @@ pid_t WorkerPool::StartProcess(const std::vector &worker_command_a } void WorkerPool::RegisterWorker(const std::shared_ptr &worker) { - auto pid = worker->Pid(); + const auto pid = worker->Pid(); RAY_LOG(DEBUG) << "Registering worker with pid " << pid; auto &state = GetStateForLanguage(worker->GetLanguage()); state.registered_workers.insert(std::move(worker)); @@ -207,6 +225,18 @@ void WorkerPool::PushWorker(const std::shared_ptr &worker) { RAY_CHECK(worker->GetAssignedTaskId().IsNil()) << "Idle workers cannot have an assigned task ID"; auto &state = GetStateForLanguage(worker->GetLanguage()); + + auto it = state.workers_to_task_id_cache.find(worker->Pid()); + if (it != state.workers_to_task_id_cache.end()) { + // The worker is used for the specific actor creation task. + const auto task_id = it->second; + state.workers_to_task_id_cache.erase(it); + state.waiting_creating_actor_workers[task_id] = std::move(worker); + + // Return to not put this worker to idle pool. + return; + } + // Add the worker to the idle pool. if (worker->GetActorId().IsNil()) { state.idle.insert(std::move(worker)); @@ -220,7 +250,7 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec const auto &actor_id = task_spec.ActorId(); std::shared_ptr worker = nullptr; - if (task_spec.IsActroCreationTask()) { + if (task_spec.IsActorCreationTask()) { // code path of actor creation task. if (task_spec.WorkerStartingPrefix().empty() && task_spec.WorkerStartingSuffix().empty()) { // There is no prefix and suffix of command. Pop a worker from idle worker pool of non-actor. @@ -229,9 +259,9 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec state.idle.erase(state.idle.begin()); } } else { - auto it = state.waiting_creating_actor_workers.find(task_spec.TaskID()); + auto it = state.waiting_creating_actor_workers.find(task_spec.TaskId()); if (it != state.waiting_creating_actor_workers.end()) { - worker = std::move(*it); + worker = std::move(it->second); state.waiting_creating_actor_workers.erase(it); } } diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index aacacb518945..b96d4bfb972b 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -51,7 +51,9 @@ class WorkerPool { /// any workers. /// /// \param language Which language this worker process should be. - void StartWorkerProcess(const Language &language); + /// \param task_spec The task specification that we provide for get more information. + void StartWorkerProcess(const Language &language, + const TaskSpecification *task_spec = nullptr); /// Register a new worker. The Worker should be added by the caller to the /// pool after it becomes idle (e.g., requests a work assignment). @@ -145,6 +147,7 @@ class WorkerPool { /// The commands and arguments used to start the worker process std::vector worker_command; /// The pool of actor creation task workers with prefix or suffix. + // rename std::unordered_map> waiting_creating_actor_workers; /// The pool of idle non-actor workers. std::unordered_set> idle; @@ -158,6 +161,8 @@ class WorkerPool { /// A map from the pids of starting worker processes /// to the number of their unregistered workers. std::unordered_map starting_worker_processes; + /// A cache map for looking up the task id of actor creation task by the pid of worker. + std::unordered_map workers_to_task_id_cache; }; /// The number of workers per process. From 644df89cec346ff0f20625e0f31e390dc54f4485 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Wed, 12 Jun 2019 19:51:14 +0800 Subject: [PATCH 04/46] Add java part --- .../java/org/ray/runtime/AbstractRayRuntime.java | 8 +++++++- .../org/ray/runtime/raylet/RayletClientImpl.java | 10 ++++++++-- .../src/main/java/org/ray/runtime/task/TaskSpec.java | 12 +++++++++++- 3 files changed, 26 insertions(+), 4 deletions(-) diff --git a/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java b/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java index fbd03bf10483..568133db792f 100644 --- a/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java +++ b/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java @@ -363,8 +363,12 @@ private TaskSpec createTaskSpec(RayFunc func, PyFunctionDescriptor pyFunctionDes } int maxActorReconstruction = 0; + String workerStartingPrefix = null; + String workerStartingSuffix = null; if (taskOptions instanceof ActorCreationOptions) { maxActorReconstruction = ((ActorCreationOptions) taskOptions).maxReconstructions; + workerStartingPrefix = ((ActorCreationOptions) taskOptions).workerStartingPrefix; + workerStartingSuffix = ((ActorCreationOptions) taskOptions).workerStartingSuffix; } TaskLanguage language; @@ -393,7 +397,9 @@ private TaskSpec createTaskSpec(RayFunc func, PyFunctionDescriptor pyFunctionDes numReturns, resources, language, - functionDescriptor + functionDescriptor, + workerStartingPrefix, + workerStartingSuffix ); } diff --git a/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java b/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java index 01b9e4675016..7a13c5ec85ba 100644 --- a/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java +++ b/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java @@ -192,7 +192,8 @@ private static TaskSpec parseTaskSpecFromFlatbuffer(ByteBuffer bb) { ); return new TaskSpec(driverId, taskId, parentTaskId, parentCounter, actorCreationId, maxActorReconstructions, actorId, actorHandleId, actorCounter, newActorHandles, - args, numReturns, resources, TaskLanguage.JAVA, functionDescriptor); + args, numReturns, resources, TaskLanguage.JAVA, functionDescriptor, + info.workerStartingPrefix(), info.workerStartingSuffix()); } private static ByteBuffer convertTaskSpecToFlatbuffer(TaskSpec task) { @@ -275,6 +276,9 @@ private static ByteBuffer convertTaskSpecToFlatbuffer(TaskSpec task) { functionDescriptorOffset = fbb.createVectorOfTables(functionDescriptorOffsets); } + final int workerStartingPrefixOffset = fbb.createString(task.workerStartingPrefix); + final int workerStartingSuffixOffset = fbb.createString(task.workerStartingSuffix); + int root = TaskInfo.createTaskInfo( fbb, driverIdOffset, @@ -293,7 +297,9 @@ private static ByteBuffer convertTaskSpecToFlatbuffer(TaskSpec task) { requiredResourcesOffset, requiredPlacementResourcesOffset, language, - functionDescriptorOffset); + functionDescriptorOffset, + workerStartingPrefixOffset, + workerStartingSuffixOffset); fbb.finish(root); ByteBuffer buffer = fbb.dataBuffer(); diff --git a/java/runtime/src/main/java/org/ray/runtime/task/TaskSpec.java b/java/runtime/src/main/java/org/ray/runtime/task/TaskSpec.java index 3473a9bdb3cc..73f6122ba86b 100644 --- a/java/runtime/src/main/java/org/ray/runtime/task/TaskSpec.java +++ b/java/runtime/src/main/java/org/ray/runtime/task/TaskSpec.java @@ -63,6 +63,10 @@ public class TaskSpec { // Language of this task. public final TaskLanguage language; + public final String workerStartingPrefix; + + public final String workerStartingSuffix; + // Descriptor of the remote function. // Note, if task language is Java, the type is JavaFunctionDescriptor. If the task language // is Python, the type is PyFunctionDescriptor. @@ -93,7 +97,9 @@ public TaskSpec( int numReturns, Map resources, TaskLanguage language, - FunctionDescriptor functionDescriptor) { + FunctionDescriptor functionDescriptor, + String workerStartingPrefix, + String workerStartingSuffix) { this.driverId = driverId; this.taskId = taskId; this.parentTaskId = parentTaskId; @@ -106,6 +112,8 @@ public TaskSpec( this.newActorHandles = newActorHandles; this.args = args; this.numReturns = numReturns; + this.workerStartingPrefix = workerStartingPrefix; + this.workerStartingSuffix = workerStartingSuffix; returnIds = new ObjectId[numReturns]; for (int i = 0; i < numReturns; ++i) { returnIds[i] = IdUtil.computeReturnId(taskId, i + 1); @@ -157,6 +165,8 @@ public String toString() { ", resources=" + resources + ", language=" + language + ", functionDescriptor=" + functionDescriptor + + ", workerStartingPrefix=" + workerStartingPrefix + + ", workerStartingSuffix=" + workerStartingSuffix + ", executionDependencies=" + executionDependencies + '}'; } From a01412001b979a04948cdabeda8ad7ea403dac95 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Wed, 12 Jun 2019 22:00:07 +0800 Subject: [PATCH 05/46] FIx --- .../org/ray/api/options/ActorCreationOptions.java | 4 ++-- .../main/java/org/ray/runtime/AbstractRayRuntime.java | 4 ++-- .../org/ray/api/test/testWorkerStartingSuffix.java | 11 ++++++++--- src/ray/raylet/node_manager.cc | 4 +++- src/ray/raylet/worker_pool.cc | 11 +++++++++++ src/ray/raylet/worker_pool.h | 4 +++- 6 files changed, 29 insertions(+), 9 deletions(-) diff --git a/java/api/src/main/java/org/ray/api/options/ActorCreationOptions.java b/java/api/src/main/java/org/ray/api/options/ActorCreationOptions.java index a0d11659eade..f5891ff1457a 100644 --- a/java/api/src/main/java/org/ray/api/options/ActorCreationOptions.java +++ b/java/api/src/main/java/org/ray/api/options/ActorCreationOptions.java @@ -34,8 +34,8 @@ public static class Builder { private Map resources = new HashMap<>(); private int maxReconstructions = NO_RECONSTRUCTION; - private String workerStartingPrefix = null; - private String workerStartingSuffix = null; + private String workerStartingPrefix = ""; + private String workerStartingSuffix = ""; public Builder setResources(Map resources) { this.resources = resources; diff --git a/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java b/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java index 568133db792f..9ced5069c03f 100644 --- a/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java +++ b/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java @@ -363,8 +363,8 @@ private TaskSpec createTaskSpec(RayFunc func, PyFunctionDescriptor pyFunctionDes } int maxActorReconstruction = 0; - String workerStartingPrefix = null; - String workerStartingSuffix = null; + String workerStartingPrefix = ""; + String workerStartingSuffix = ""; if (taskOptions instanceof ActorCreationOptions) { maxActorReconstruction = ((ActorCreationOptions) taskOptions).maxReconstructions; workerStartingPrefix = ((ActorCreationOptions) taskOptions).workerStartingPrefix; diff --git a/java/test/src/main/java/org/ray/api/test/testWorkerStartingSuffix.java b/java/test/src/main/java/org/ray/api/test/testWorkerStartingSuffix.java index 9810ed4ab77c..5f8fd9159bb1 100644 --- a/java/test/src/main/java/org/ray/api/test/testWorkerStartingSuffix.java +++ b/java/test/src/main/java/org/ray/api/test/testWorkerStartingSuffix.java @@ -2,15 +2,19 @@ import org.ray.api.Ray; import org.ray.api.RayActor; +import org.ray.api.RayObject; import org.ray.api.annotation.RayRemote; import org.ray.api.options.ActorCreationOptions; +import org.testng.Assert; import org.testng.annotations.Test; -public class testWorkerStartingSuffix { +public class testWorkerStartingSuffix extends BaseTest { @RayRemote public static class Echo { - + String sayHi() { + return "hi"; + } } @Test @@ -19,6 +23,7 @@ public void testSuffix() { .setWorkerStartingSuffix("-Xss16m") .createActorCreationOptions(); RayActor actor = Ray.createActor(Echo::new, options); - + RayObject obj = Ray.call(Echo::sayHi, actor); + Assert.assertEquals(obj.get(), "hi"); } } diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 61310c75d7b1..311c969790e2 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1726,7 +1726,9 @@ bool NodeManager::AssignTask(const Task &task) { const bool need_start_new = (!spec.WorkerStartingPrefix().empty() || !spec.WorkerStartingSuffix().empty()); if (spec.IsActorCreationTask() && need_start_new) { - worker_pool_.StartWorkerProcess(spec.GetLanguage(), &spec); + if (!worker_pool_.HasWorkerStartingForTask(spec.GetLanguage(), spec.TaskId())) { + worker_pool_.StartWorkerProcess(spec.GetLanguage(), &spec); + } } else if (!spec.IsActorTask()) { // There are no more non-actor workers available to execute this task. // Start a new worker. diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 33be2fcb45ad..9a181a2a4fa9 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -344,6 +344,17 @@ std::string WorkerPool::WarningAboutSize() { return warning_message.str(); } +bool WorkerPool::HasWorkerStartingForTask(const Language &language, const TaskID &task_id) { + auto &state = GetStateForLanguage(language); + for (const auto &item : state.workers_to_task_id_cache) { + if (item.second == task_id) { + return true; + } + } + + return false; +} + std::string WorkerPool::DebugString() const { std::stringstream result; result << "WorkerPool:"; diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index b96d4bfb972b..f7c81346ce5a 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -120,6 +120,8 @@ class WorkerPool { std::vector> GetWorkersRunningTasksForDriver( const DriverID &driver_id) const; + bool HasWorkerStartingForTask(const Language &language, const TaskID &task_id); + /// Returns debug string for class. /// /// \return string. @@ -173,7 +175,7 @@ class WorkerPool { private: /// A helper function that returns the reference of the pool state /// for a given language. - inline State &GetStateForLanguage(const Language &language); + State &GetStateForLanguage(const Language &language); /// We'll push a warning to the user every time a multiple of this many /// workers has been started. From 5006871eeed81494b15b566bd10597fba78d7bd0 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Wed, 12 Jun 2019 23:24:22 +0800 Subject: [PATCH 06/46] Fix --- .../ray/api/test/testWorkerStartingSuffix.java | 15 ++++++++++++--- src/ray/raylet/worker_pool.cc | 7 +++++-- 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/java/test/src/main/java/org/ray/api/test/testWorkerStartingSuffix.java b/java/test/src/main/java/org/ray/api/test/testWorkerStartingSuffix.java index 5f8fd9159bb1..64679e9cfc63 100644 --- a/java/test/src/main/java/org/ray/api/test/testWorkerStartingSuffix.java +++ b/java/test/src/main/java/org/ray/api/test/testWorkerStartingSuffix.java @@ -15,15 +15,24 @@ public static class Echo { String sayHi() { return "hi"; } + + String getSuffix() { + return System.getProperty("test.suffix"); + } } @Test public void testSuffix() { ActorCreationOptions options = new ActorCreationOptions.Builder() - .setWorkerStartingSuffix("-Xss16m") + .setWorkerStartingSuffix("-Dtest.suffix=suffix") .createActorCreationOptions(); RayActor actor = Ray.createActor(Echo::new, options); - RayObject obj = Ray.call(Echo::sayHi, actor); - Assert.assertEquals(obj.get(), "hi"); + RayObject obj = Ray.call(Echo::getSuffix, actor); + Assert.assertEquals(obj.get(), "suffix"); + + // test without suffix + RayActor actor1 = Ray.createActor(Echo::new, options); + RayObject obj1 = Ray.call(Echo::sayHi, actor); + Assert.assertEquals(obj1.get(), "hi"); } } diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 9a181a2a4fa9..e7f21c7c9b73 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -132,8 +132,11 @@ void WorkerPool::StartWorkerProcess(const Language &language, for (auto const &token : state.worker_command) { worker_command_args.push_back(token.c_str()); } - if (!suffix.empty()) { - worker_command_args.push_back(suffix.c_str()); + // Note that this is used for Java worker only currently. + if (!suffix.empty() && task_spec->GetLanguage() == Language::JAVA) { + RAY_CHECK(worker_command_args > 2) << "At least 2 elements of the Java worker command."; + const auto pos_to_insert = worker_command_args.size() - 1; + worker_command_args.insert(worker_command_args.begin() + pos_to_insert, suffix.c_str()); } worker_command_args.push_back(nullptr); From 04cabf883f374a4a43547dffd5ae0eb71ba9b905 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Wed, 12 Jun 2019 23:57:24 +0800 Subject: [PATCH 07/46] Fix --- .../org/ray/api/options/ActorCreationOptions.java | 3 ++- ...rtingSuffix.java => WorkerStartingSuffixTest.java} | 11 +---------- src/ray/raylet/worker_pool.cc | 2 +- 3 files changed, 4 insertions(+), 12 deletions(-) rename java/test/src/main/java/org/ray/api/test/{testWorkerStartingSuffix.java => WorkerStartingSuffixTest.java} (72%) diff --git a/java/api/src/main/java/org/ray/api/options/ActorCreationOptions.java b/java/api/src/main/java/org/ray/api/options/ActorCreationOptions.java index f5891ff1457a..808194b39d18 100644 --- a/java/api/src/main/java/org/ray/api/options/ActorCreationOptions.java +++ b/java/api/src/main/java/org/ray/api/options/ActorCreationOptions.java @@ -58,7 +58,8 @@ public Builder setWorkerStartingSuffix(String workerStartingSuffix) { } public ActorCreationOptions createActorCreationOptions() { - return new ActorCreationOptions(resources, maxReconstructions, workerStartingPrefix, workerStartingSuffix); + return new ActorCreationOptions(resources, maxReconstructions, + workerStartingPrefix, workerStartingSuffix); } } diff --git a/java/test/src/main/java/org/ray/api/test/testWorkerStartingSuffix.java b/java/test/src/main/java/org/ray/api/test/WorkerStartingSuffixTest.java similarity index 72% rename from java/test/src/main/java/org/ray/api/test/testWorkerStartingSuffix.java rename to java/test/src/main/java/org/ray/api/test/WorkerStartingSuffixTest.java index 64679e9cfc63..ec7569093432 100644 --- a/java/test/src/main/java/org/ray/api/test/testWorkerStartingSuffix.java +++ b/java/test/src/main/java/org/ray/api/test/WorkerStartingSuffixTest.java @@ -8,14 +8,10 @@ import org.testng.Assert; import org.testng.annotations.Test; -public class testWorkerStartingSuffix extends BaseTest { +public class WorkerStartingSuffixTest extends BaseTest { @RayRemote public static class Echo { - String sayHi() { - return "hi"; - } - String getSuffix() { return System.getProperty("test.suffix"); } @@ -29,10 +25,5 @@ public void testSuffix() { RayActor actor = Ray.createActor(Echo::new, options); RayObject obj = Ray.call(Echo::getSuffix, actor); Assert.assertEquals(obj.get(), "suffix"); - - // test without suffix - RayActor actor1 = Ray.createActor(Echo::new, options); - RayObject obj1 = Ray.call(Echo::sayHi, actor); - Assert.assertEquals(obj1.get(), "hi"); } } diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index e7f21c7c9b73..5753efb1f2ab 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -134,7 +134,7 @@ void WorkerPool::StartWorkerProcess(const Language &language, } // Note that this is used for Java worker only currently. if (!suffix.empty() && task_spec->GetLanguage() == Language::JAVA) { - RAY_CHECK(worker_command_args > 2) << "At least 2 elements of the Java worker command."; + RAY_CHECK(worker_command_args.size() > 2) << "At least 2 elements of the Java worker command."; const auto pos_to_insert = worker_command_args.size() - 1; worker_command_args.insert(worker_command_args.begin() + pos_to_insert, suffix.c_str()); } From a67d53060d696d7844543767fa1203f619849006 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Thu, 13 Jun 2019 10:32:28 +0800 Subject: [PATCH 08/46] Fix lint --- .../api/test/WorkerStartingSuffixTest.java | 2 ++ src/ray/raylet/worker_pool.cc | 20 ++++++++++--------- src/ray/raylet/worker_pool.h | 8 ++++---- 3 files changed, 17 insertions(+), 13 deletions(-) diff --git a/java/test/src/main/java/org/ray/api/test/WorkerStartingSuffixTest.java b/java/test/src/main/java/org/ray/api/test/WorkerStartingSuffixTest.java index ec7569093432..39cd921db57a 100644 --- a/java/test/src/main/java/org/ray/api/test/WorkerStartingSuffixTest.java +++ b/java/test/src/main/java/org/ray/api/test/WorkerStartingSuffixTest.java @@ -3,6 +3,7 @@ import org.ray.api.Ray; import org.ray.api.RayActor; import org.ray.api.RayObject; +import org.ray.api.TestUtils; import org.ray.api.annotation.RayRemote; import org.ray.api.options.ActorCreationOptions; import org.testng.Assert; @@ -19,6 +20,7 @@ String getSuffix() { @Test public void testSuffix() { + TestUtils.skipTestUnderSingleProcess(); ActorCreationOptions options = new ActorCreationOptions.Builder() .setWorkerStartingSuffix("-Dtest.suffix=suffix") .createActorCreationOptions(); diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 5753efb1f2ab..b87725a83e70 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -116,7 +116,6 @@ void WorkerPool::StartWorkerProcess(const Language &language, << state.idle_actor.size() << " actor workers, and " << state.idle.size() << " non-actor workers"; - std::string prefix = ""; std::string suffix = ""; if (task_spec != nullptr && task_spec->IsActorCreationTask()) { @@ -134,7 +133,8 @@ void WorkerPool::StartWorkerProcess(const Language &language, } // Note that this is used for Java worker only currently. if (!suffix.empty() && task_spec->GetLanguage() == Language::JAVA) { - RAY_CHECK(worker_command_args.size() > 2) << "At least 2 elements of the Java worker command."; + RAY_CHECK(worker_command_args.size() > 2) + << "At least 2 elements of the Java worker command."; const auto pos_to_insert = worker_command_args.size() - 1; worker_command_args.insert(worker_command_args.begin() + pos_to_insert, suffix.c_str()); } @@ -151,7 +151,7 @@ void WorkerPool::StartWorkerProcess(const Language &language, state.starting_worker_processes.emplace( std::make_pair(pid, num_workers_per_process_)); if (!prefix.empty() || !suffix.empty()) { - state.workers_to_task_id_cache[pid] = task_spec->TaskId(); + state.worker_to_task_id_cache[pid] = task_spec->TaskId(); } return; } @@ -229,11 +229,11 @@ void WorkerPool::PushWorker(const std::shared_ptr &worker) { << "Idle workers cannot have an assigned task ID"; auto &state = GetStateForLanguage(worker->GetLanguage()); - auto it = state.workers_to_task_id_cache.find(worker->Pid()); - if (it != state.workers_to_task_id_cache.end()) { + auto it = state.worker_to_task_id_cache.find(worker->Pid()); + if (it != state.worker_to_task_id_cache.end()) { // The worker is used for the specific actor creation task. const auto task_id = it->second; - state.workers_to_task_id_cache.erase(it); + state.worker_to_task_id_cache.erase(it); state.waiting_creating_actor_workers[task_id] = std::move(worker); // Return to not put this worker to idle pool. @@ -255,8 +255,10 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec std::shared_ptr worker = nullptr; if (task_spec.IsActorCreationTask()) { // code path of actor creation task. - if (task_spec.WorkerStartingPrefix().empty() && task_spec.WorkerStartingSuffix().empty()) { - // There is no prefix and suffix of command. Pop a worker from idle worker pool of non-actor. + if (task_spec.WorkerStartingPrefix().empty() && + task_spec.WorkerStartingSuffix().empty()) { + // There is no prefix and suffix of command. Pop a worker from idle worker pool of + // non-actor. if (!state.idle.empty()) { worker = std::move(*state.idle.begin()); state.idle.erase(state.idle.begin()); @@ -349,7 +351,7 @@ std::string WorkerPool::WarningAboutSize() { bool WorkerPool::HasWorkerStartingForTask(const Language &language, const TaskID &task_id) { auto &state = GetStateForLanguage(language); - for (const auto &item : state.workers_to_task_id_cache) { + for (const auto &item : state.worker_to_task_id_cache) { if (item.second == task_id) { return true; } diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index f7c81346ce5a..79d394f2019a 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -148,8 +148,7 @@ class WorkerPool { struct State { /// The commands and arguments used to start the worker process std::vector worker_command; - /// The pool of actor creation task workers with prefix or suffix. - // rename + /// The worker pool of actor creation tasks with prefix or suffix. std::unordered_map> waiting_creating_actor_workers; /// The pool of idle non-actor workers. std::unordered_set> idle; @@ -163,8 +162,9 @@ class WorkerPool { /// A map from the pids of starting worker processes /// to the number of their unregistered workers. std::unordered_map starting_worker_processes; - /// A cache map for looking up the task id of actor creation task by the pid of worker. - std::unordered_map workers_to_task_id_cache; + /// A cache map for looking up the task id of actor creation task by the pid of + /// worker. + std::unordered_map worker_to_task_id_cache; }; /// The number of workers per process. From f3816f7fbc7b5b38d25beabf806efdc9e950182d Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Fri, 14 Jun 2019 12:16:20 +0800 Subject: [PATCH 09/46] Refine API --- .../ray/api/options/ActorCreationOptions.java | 25 ++++++------------- .../org/ray/runtime/AbstractRayRuntime.java | 6 ++--- ...fixTest.java => WorkerJvmOptionsTest.java} | 4 +-- 3 files changed, 11 insertions(+), 24 deletions(-) rename java/test/src/main/java/org/ray/api/test/{WorkerStartingSuffixTest.java => WorkerJvmOptionsTest.java} (87%) diff --git a/java/api/src/main/java/org/ray/api/options/ActorCreationOptions.java b/java/api/src/main/java/org/ray/api/options/ActorCreationOptions.java index 808194b39d18..2e14ca8584dd 100644 --- a/java/api/src/main/java/org/ray/api/options/ActorCreationOptions.java +++ b/java/api/src/main/java/org/ray/api/options/ActorCreationOptions.java @@ -13,18 +13,14 @@ public class ActorCreationOptions extends BaseTaskOptions { public final int maxReconstructions; - public final String workerStartingPrefix; - - public final String workerStartingSuffix; + public final String jvmOptions; private ActorCreationOptions(Map resources, int maxReconstructions, - String workerStartingPrefix, - String workerStartingSuffix) { + String jvmOptions) { super(resources); this.maxReconstructions = maxReconstructions; - this.workerStartingPrefix = workerStartingPrefix; - this.workerStartingSuffix = workerStartingSuffix; + this.jvmOptions = jvmOptions; } /** @@ -34,8 +30,7 @@ public static class Builder { private Map resources = new HashMap<>(); private int maxReconstructions = NO_RECONSTRUCTION; - private String workerStartingPrefix = ""; - private String workerStartingSuffix = ""; + private String jvmOptions = ""; public Builder setResources(Map resources) { this.resources = resources; @@ -47,19 +42,13 @@ public Builder setMaxReconstructions(int maxReconstructions) { return this; } - public Builder setWorkerStartingPrefix(String workerStartingPrefix) { - this.workerStartingPrefix = workerStartingPrefix; - return this; - } - - public Builder setWorkerStartingSuffix(String workerStartingSuffix) { - this.workerStartingSuffix = workerStartingSuffix; + public Builder setJvmOptions(String jvmOptions) { + this.jvmOptions = jvmOptions; return this; } public ActorCreationOptions createActorCreationOptions() { - return new ActorCreationOptions(resources, maxReconstructions, - workerStartingPrefix, workerStartingSuffix); + return new ActorCreationOptions(resources, maxReconstructions, jvmOptions); } } diff --git a/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java b/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java index 9ced5069c03f..bfd9d44cadc2 100644 --- a/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java +++ b/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java @@ -363,12 +363,10 @@ private TaskSpec createTaskSpec(RayFunc func, PyFunctionDescriptor pyFunctionDes } int maxActorReconstruction = 0; - String workerStartingPrefix = ""; String workerStartingSuffix = ""; if (taskOptions instanceof ActorCreationOptions) { maxActorReconstruction = ((ActorCreationOptions) taskOptions).maxReconstructions; - workerStartingPrefix = ((ActorCreationOptions) taskOptions).workerStartingPrefix; - workerStartingSuffix = ((ActorCreationOptions) taskOptions).workerStartingSuffix; + workerStartingSuffix = ((ActorCreationOptions) taskOptions).jvmOptions; } TaskLanguage language; @@ -398,7 +396,7 @@ private TaskSpec createTaskSpec(RayFunc func, PyFunctionDescriptor pyFunctionDes resources, language, functionDescriptor, - workerStartingPrefix, + "", workerStartingSuffix ); } diff --git a/java/test/src/main/java/org/ray/api/test/WorkerStartingSuffixTest.java b/java/test/src/main/java/org/ray/api/test/WorkerJvmOptionsTest.java similarity index 87% rename from java/test/src/main/java/org/ray/api/test/WorkerStartingSuffixTest.java rename to java/test/src/main/java/org/ray/api/test/WorkerJvmOptionsTest.java index 39cd921db57a..5154a97122ed 100644 --- a/java/test/src/main/java/org/ray/api/test/WorkerStartingSuffixTest.java +++ b/java/test/src/main/java/org/ray/api/test/WorkerJvmOptionsTest.java @@ -9,7 +9,7 @@ import org.testng.Assert; import org.testng.annotations.Test; -public class WorkerStartingSuffixTest extends BaseTest { +public class WorkerJvmOptionsTest extends BaseTest { @RayRemote public static class Echo { @@ -22,7 +22,7 @@ String getSuffix() { public void testSuffix() { TestUtils.skipTestUnderSingleProcess(); ActorCreationOptions options = new ActorCreationOptions.Builder() - .setWorkerStartingSuffix("-Dtest.suffix=suffix") + .setJvmOptions("-Dtest.suffix=suffix") .createActorCreationOptions(); RayActor actor = Ray.createActor(Echo::new, options); RayObject obj = Ray.call(Echo::getSuffix, actor); From 28b1b05be6ee0c07c707b6f49c9e1c3f05fba2f5 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Fri, 14 Jun 2019 18:02:20 +0800 Subject: [PATCH 10/46] address comments and add test --- .../org/ray/runtime/runner/RunManager.java | 8 +++ python/ray/ray_constants.py | 7 +++ python/ray/services.py | 8 ++- src/ray/common/constants.h | 5 ++ src/ray/raylet/node_manager.cc | 2 +- src/ray/raylet/worker_pool.cc | 45 ++++++++-------- src/ray/raylet/worker_pool.h | 18 ++++--- src/ray/raylet/worker_pool_test.cc | 54 ++++++++++++++++--- 8 files changed, 112 insertions(+), 35 deletions(-) diff --git a/java/runtime/src/main/java/org/ray/runtime/runner/RunManager.java b/java/runtime/src/main/java/org/ray/runtime/runner/RunManager.java index 15240e43e234..a4e451d641b4 100644 --- a/java/runtime/src/main/java/org/ray/runtime/runner/RunManager.java +++ b/java/runtime/src/main/java/org/ray/runtime/runner/RunManager.java @@ -38,6 +38,10 @@ public class RunManager { private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern("Y-M-d_H-m-s"); + private static final String PREFIX_PLACEHOLDER = "__PREFIX_PLACEHOLDER__"; + + private static final String SUFFIX_PLACEHOLDER = "__SUFFIX_PLACEHOLDER__"; + private static final String WORKER_CLASS = "org.ray.runtime.runner.worker.DefaultWorker"; private RayConfig rayConfig; @@ -281,6 +285,7 @@ private String concatPath(Stream stream) { private String buildWorkerCommandRaylet() { List cmd = new ArrayList<>(); + cmd.add(PREFIX_PLACEHOLDER); cmd.add("java"); cmd.add("-classpath"); @@ -319,6 +324,9 @@ private String buildWorkerCommandRaylet() { cmd.addAll(rayConfig.jvmParameters); + // jvm options + cmd.add(SUFFIX_PLACEHOLDER); + // Main class cmd.add(WORKER_CLASS); String command = Joiner.on(" ").join(cmd); diff --git a/python/ray/ray_constants.py b/python/ray/ray_constants.py index 6fd7d5b643e6..63d3c6313419 100644 --- a/python/ray/ray_constants.py +++ b/python/ray/ray_constants.py @@ -125,3 +125,10 @@ def env_integer(key, default): # A constant used as object metadata to indicate the object is raw binary. RAW_BUFFER_METADATA = b"RAW" + +# These should have the same value as `kPrefixPlaceholder` +# and `kSuffixPlaceholder` in `constants.h` +# A constant used as a placeholder of the prefix for worker command. +PREFIX_PLACEHOLDER = "__PREFIX_PLACEHOLDER__" +# A constant used as a placeholder of the suffix for worker command. +SUFFIX_PLACEHOLDER = "__SUFFIX_PLACEHOLDER__" diff --git a/python/ray/services.py b/python/ray/services.py index 2c843f7bbbc7..878668246af8 100644 --- a/python/ray/services.py +++ b/python/ray/services.py @@ -1232,7 +1232,9 @@ def build_java_worker_command( """ assert java_worker_options is not None - command = "java " + # Add the prefix placeholder. + command = "{} java ".format(ray_constants.PREFIX_PLACEHOLDER) + if redis_address is not None: command += "-Dray.redis.address={} ".format(redis_address) @@ -1253,6 +1255,10 @@ def build_java_worker_command( # Put `java_worker_options` in the last, so it can overwrite the # above options. command += java_worker_options + " " + + # Add the suffix placeholder as jvm options. + command += "{} ".format(ray_constants.SUFFIX_PLACEHOLDER) + command += "org.ray.runtime.runner.worker.DefaultWorker" return command diff --git a/src/ray/common/constants.h b/src/ray/common/constants.h index c92e6a74aa5d..779084bc981e 100644 --- a/src/ray/common/constants.h +++ b/src/ray/common/constants.h @@ -36,4 +36,9 @@ constexpr char kObjectTablePrefix[] = "ObjectTable"; /// Prefix for the task table keys in redis. constexpr char kTaskTablePrefix[] = "TaskTable"; +/// The constant used as a placeholder of the prefix for worker command. +constexpr char kPrefixPlaceholder[] = "__PREFIX_PLACEHOLDER__"; +/// The constant used as a placeholder of the suffix for worker command. +constexpr char kSuffixPlaceholder[] = "__SUFFIX_PLACEHOLDER__"; + #endif // RAY_CONSTANTS_H_ diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 311c969790e2..3aeed2f30f21 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1726,7 +1726,7 @@ bool NodeManager::AssignTask(const Task &task) { const bool need_start_new = (!spec.WorkerStartingPrefix().empty() || !spec.WorkerStartingSuffix().empty()); if (spec.IsActorCreationTask() && need_start_new) { - if (!worker_pool_.HasWorkerStartingForTask(spec.GetLanguage(), spec.TaskId())) { + if (!worker_pool_.IsWorkerStartingForTask(spec.GetLanguage(), spec.TaskId())) { worker_pool_.StartWorkerProcess(spec.GetLanguage(), &spec); } } else if (!spec.IsActorTask()) { diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index b87725a83e70..37bdae786ee0 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -9,6 +9,7 @@ #include "ray/common/status.h" #include "ray/stats/stats.h" #include "ray/util/logging.h" +#include "ray/common/constants.h" namespace { @@ -125,18 +126,18 @@ void WorkerPool::StartWorkerProcess(const Language &language, // Extract pointers from the worker command to pass into execvp. std::vector worker_command_args; - if (!prefix.empty()) { - worker_command_args.push_back(prefix.c_str()); - } for (auto const &token : state.worker_command) { - worker_command_args.push_back(token.c_str()); - } - // Note that this is used for Java worker only currently. - if (!suffix.empty() && task_spec->GetLanguage() == Language::JAVA) { - RAY_CHECK(worker_command_args.size() > 2) - << "At least 2 elements of the Java worker command."; - const auto pos_to_insert = worker_command_args.size() - 1; - worker_command_args.insert(worker_command_args.begin() + pos_to_insert, suffix.c_str()); + if (token == kPrefixPlaceholder) { + if (!prefix.empty()) { + worker_command_args.push_back(prefix.c_str()); + } + } else if (token == kSuffixPlaceholder) { + if (!suffix.empty()) { + worker_command_args.push_back(suffix.c_str()); + } + }else { + worker_command_args.push_back(token.c_str()); + } } worker_command_args.push_back(nullptr); @@ -151,7 +152,9 @@ void WorkerPool::StartWorkerProcess(const Language &language, state.starting_worker_processes.emplace( std::make_pair(pid, num_workers_per_process_)); if (!prefix.empty() || !suffix.empty()) { - state.worker_to_task_id_cache[pid] = task_spec->TaskId(); + RAY_CHECK(task_spec != nullptr) << "task_spec should not be nullptr " + "since we specified prefix or suffix for worker command"; + state.starting_lazy_worker_processes[pid] = task_spec->TaskId(); } return; } @@ -229,12 +232,12 @@ void WorkerPool::PushWorker(const std::shared_ptr &worker) { << "Idle workers cannot have an assigned task ID"; auto &state = GetStateForLanguage(worker->GetLanguage()); - auto it = state.worker_to_task_id_cache.find(worker->Pid()); - if (it != state.worker_to_task_id_cache.end()) { + auto it = state.starting_lazy_worker_processes.find(worker->Pid()); + if (it != state.starting_lazy_worker_processes.end()) { // The worker is used for the specific actor creation task. const auto task_id = it->second; - state.worker_to_task_id_cache.erase(it); - state.waiting_creating_actor_workers[task_id] = std::move(worker); + state.starting_lazy_worker_processes.erase(it); + state.idle_lazy_workers[task_id] = std::move(worker); // Return to not put this worker to idle pool. return; @@ -264,10 +267,10 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec state.idle.erase(state.idle.begin()); } } else { - auto it = state.waiting_creating_actor_workers.find(task_spec.TaskId()); - if (it != state.waiting_creating_actor_workers.end()) { + auto it = state.idle_lazy_workers.find(task_spec.TaskId()); + if (it != state.idle_lazy_workers.end()) { worker = std::move(it->second); - state.waiting_creating_actor_workers.erase(it); + state.idle_lazy_workers.erase(it); } } } else if (!task_spec.IsActorTask()) { @@ -349,9 +352,9 @@ std::string WorkerPool::WarningAboutSize() { return warning_message.str(); } -bool WorkerPool::HasWorkerStartingForTask(const Language &language, const TaskID &task_id) { +bool WorkerPool::IsWorkerStartingForTask(const Language &language, const TaskID &task_id) { auto &state = GetStateForLanguage(language); - for (const auto &item : state.worker_to_task_id_cache) { + for (const auto &item : state.starting_lazy_worker_processes) { if (item.second == task_id) { return true; } diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index 79d394f2019a..a18dee828fb0 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -120,7 +120,12 @@ class WorkerPool { std::vector> GetWorkersRunningTasksForDriver( const DriverID &driver_id) const; - bool HasWorkerStartingForTask(const Language &language, const TaskID &task_id); + /// Whether we're starting a worker for the given task. + /// Note that, this is only used for actor creation task. + /// + /// \param language The required language. + /// \param task_id The task that we'll query by. + bool IsWorkerStartingForTask(const Language &language, const TaskID &task_id); /// Returns debug string for class. /// @@ -148,8 +153,9 @@ class WorkerPool { struct State { /// The commands and arguments used to start the worker process std::vector worker_command; - /// The worker pool of actor creation tasks with prefix or suffix. - std::unordered_map> waiting_creating_actor_workers; + /// The pool of lazy workers for actor creation tasks + /// with prefix or suffix worker command. + std::unordered_map> idle_lazy_workers; /// The pool of idle non-actor workers. std::unordered_set> idle; /// The pool of idle actor workers. @@ -162,9 +168,9 @@ class WorkerPool { /// A map from the pids of starting worker processes /// to the number of their unregistered workers. std::unordered_map starting_worker_processes; - /// A cache map for looking up the task id of actor creation task by the pid of - /// worker. - std::unordered_map worker_to_task_id_cache; + /// A map for looking up the task id of actor creation task by the pid of + /// worker. Note that this is used for the lazy worker processes. + std::unordered_map starting_lazy_worker_processes; }; /// The number of workers per process. diff --git a/src/ray/raylet/worker_pool_test.cc b/src/ray/raylet/worker_pool_test.cc index 143ffd57dda6..5d8721847cbc 100644 --- a/src/ray/raylet/worker_pool_test.cc +++ b/src/ray/raylet/worker_pool_test.cc @@ -3,6 +3,7 @@ #include "ray/raylet/node_manager.h" #include "ray/raylet/worker_pool.h" +#include "ray/common/constants.h" namespace ray { @@ -14,21 +15,38 @@ int MAXIMUM_STARTUP_CONCURRENCY = 5; class WorkerPoolMock : public WorkerPool { public: WorkerPoolMock() - : WorkerPool(0, NUM_WORKERS_PER_PROCESS, MAXIMUM_STARTUP_CONCURRENCY, - {{Language::PYTHON, {"dummy_py_worker_command"}}, - {Language::JAVA, {"dummy_java_worker_command"}}}), + : WorkerPoolMock({{Language::PYTHON, {"dummy_py_worker_command"}}, + {Language::JAVA, {"dummy_java_worker_command"}}}) {} + + explicit WorkerPoolMock(const std::unordered_map> &worker_commands) + : WorkerPool(0, NUM_WORKERS_PER_PROCESS, MAXIMUM_STARTUP_CONCURRENCY, worker_commands), last_worker_pid_(0) {} + ~WorkerPoolMock() { // Avoid killing real processes states_by_lang_.clear(); } pid_t StartProcess(const std::vector &worker_command_args) override { - return ++last_worker_pid_; + last_worker_pid_ += 1; + std::vector local_worker_commands_args; + for (auto item : worker_command_args) { + if (item == nullptr) { + break; + } + local_worker_commands_args.push_back(std::string(item)); + } + worker_commands_by_pid[last_worker_pid_] = std::move(local_worker_commands_args); + return last_worker_pid_; } pid_t LastStartedWorkerProcess() const { return last_worker_pid_; } + const std::vector &GetWorkerCommand(int pid) { + RAY_CHECK(worker_commands_by_pid.count(pid) != 0) << "Invalid argument."; + return worker_commands_by_pid[pid]; + } + int NumWorkerProcessesStarting() const { int total = 0; for (auto &entry : states_by_lang_) { @@ -39,6 +57,8 @@ class WorkerPoolMock : public WorkerPool { private: int last_worker_pid_; + // The worker commands by pid. + std::unordered_map> worker_commands_by_pid; }; class WorkerPoolTest : public ::testing::Test { @@ -61,6 +81,11 @@ class WorkerPoolTest : public ::testing::Test { return std::shared_ptr(new Worker(pid, language, client)); } + void SetWorkerCommands(const std::unordered_map> &worker_commands) { + WorkerPoolMock worker_pool(worker_commands); + this->worker_pool_ = std::move(worker_pool); + } + protected: WorkerPoolMock worker_pool_; boost::asio::io_service io_service_; @@ -73,9 +98,10 @@ class WorkerPoolTest : public ::testing::Test { static inline TaskSpecification ExampleTaskSpec( const ActorID actor_id = ActorID::Nil(), - const Language &language = Language::PYTHON) { + const Language &language = Language::PYTHON, + const ActorID actor_creation_id = ActorID::Nil()) { std::vector function_descriptor(3); - return TaskSpecification(DriverID::Nil(), TaskID::Nil(), 0, ActorID::Nil(), + return TaskSpecification(DriverID::Nil(), TaskID::Nil(), 0, actor_creation_id, ObjectID::Nil(), 0, actor_id, ActorHandleID::Nil(), 0, {}, {}, 0, {}, {}, language, function_descriptor); } @@ -186,6 +212,22 @@ TEST_F(WorkerPoolTest, PopWorkersOfMultipleLanguages) { ASSERT_NE(worker_pool_.PopWorker(java_task_spec), nullptr); } +TEST_F(WorkerPoolTest, StartWorkerWithPrefixAndSuffix) { + SetWorkerCommands({ + {Language::PYTHON, {"dummy_py_worker_command"}}, + {Language::JAVA, {kPrefixPlaceholder, "dummy_java_worker_command", kSuffixPlaceholder}}}); + + TaskSpecification task_spec(DriverID::Nil(), TaskID::Nil(), 0, ActorID::FromRandom(), + ObjectID::Nil(), 0, ActorID::Nil(), ActorHandleID::Nil(), 0, {}, {}, + 0, {}, {}, Language::JAVA, {"", "", ""}, "test_prefix", "test_suffix"); + worker_pool_.StartWorkerProcess(Language::JAVA, &task_spec); + const auto real_command = worker_pool_.GetWorkerCommand(worker_pool_.LastStartedWorkerProcess()); + ASSERT_EQ(3, real_command.size()); + ASSERT_EQ(std::string("test_prefix"), std::string(real_command[0])); + ASSERT_EQ("dummy_java_worker_command", real_command[1]); + ASSERT_EQ("test_suffix", real_command[2]); +} + } // namespace raylet } // namespace ray From 9564ce78676f5dd9b1ce5411d3b329984957efcd Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Fri, 14 Jun 2019 18:11:32 +0800 Subject: [PATCH 11/46] Fix --- .../main/java/org/ray/api/test/WorkerJvmOptionsTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/java/test/src/main/java/org/ray/api/test/WorkerJvmOptionsTest.java b/java/test/src/main/java/org/ray/api/test/WorkerJvmOptionsTest.java index 5154a97122ed..90a2817a8366 100644 --- a/java/test/src/main/java/org/ray/api/test/WorkerJvmOptionsTest.java +++ b/java/test/src/main/java/org/ray/api/test/WorkerJvmOptionsTest.java @@ -13,19 +13,19 @@ public class WorkerJvmOptionsTest extends BaseTest { @RayRemote public static class Echo { - String getSuffix() { + String getOptions() { return System.getProperty("test.suffix"); } } @Test - public void testSuffix() { + public void testJvmOptions() { TestUtils.skipTestUnderSingleProcess(); ActorCreationOptions options = new ActorCreationOptions.Builder() .setJvmOptions("-Dtest.suffix=suffix") .createActorCreationOptions(); RayActor actor = Ray.createActor(Echo::new, options); - RayObject obj = Ray.call(Echo::getSuffix, actor); + RayObject obj = Ray.call(Echo::getOptions, actor); Assert.assertEquals(obj.get(), "suffix"); } } From f85cbb985b6efc130b8a6102dd834e6c93b05bc2 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Sat, 15 Jun 2019 00:23:25 +0800 Subject: [PATCH 12/46] Address comment. --- src/ray/raylet/node_manager.cc | 6 +++--- src/ray/raylet/worker_pool.cc | 20 ++++++++++---------- src/ray/raylet/worker_pool.h | 12 ++++++------ 3 files changed, 19 insertions(+), 19 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 3aeed2f30f21..d69b45f9a912 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1723,10 +1723,10 @@ bool NodeManager::AssignTask(const Task &task) { std::shared_ptr worker = worker_pool_.PopWorker(spec); if (worker == nullptr) { // There are no workers that can execute this task. - const bool need_start_new = + const bool should_start_new = (!spec.WorkerStartingPrefix().empty() || !spec.WorkerStartingSuffix().empty()); - if (spec.IsActorCreationTask() && need_start_new) { - if (!worker_pool_.IsWorkerStartingForTask(spec.GetLanguage(), spec.TaskId())) { + if (spec.IsActorCreationTask() && should_start_new) { + if (!worker_pool_.PendingRegistrationForTask(spec.GetLanguage(), spec.TaskId())) { worker_pool_.StartWorkerProcess(spec.GetLanguage(), &spec); } } else if (!spec.IsActorTask()) { diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 37bdae786ee0..431e29ba7c02 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -154,7 +154,7 @@ void WorkerPool::StartWorkerProcess(const Language &language, if (!prefix.empty() || !suffix.empty()) { RAY_CHECK(task_spec != nullptr) << "task_spec should not be nullptr " "since we specified prefix or suffix for worker command"; - state.starting_lazy_worker_processes[pid] = task_spec->TaskId(); + state.starting_dedicated_worker_processes[pid] = task_spec->TaskId(); } return; } @@ -232,12 +232,12 @@ void WorkerPool::PushWorker(const std::shared_ptr &worker) { << "Idle workers cannot have an assigned task ID"; auto &state = GetStateForLanguage(worker->GetLanguage()); - auto it = state.starting_lazy_worker_processes.find(worker->Pid()); - if (it != state.starting_lazy_worker_processes.end()) { + auto it = state.starting_dedicated_worker_processes.find(worker->Pid()); + if (it != state.starting_dedicated_worker_processes.end()) { // The worker is used for the specific actor creation task. const auto task_id = it->second; - state.starting_lazy_worker_processes.erase(it); - state.idle_lazy_workers[task_id] = std::move(worker); + state.starting_dedicated_worker_processes.erase(it); + state.idle_dedicated_workers[task_id] = std::move(worker); // Return to not put this worker to idle pool. return; @@ -267,10 +267,10 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec state.idle.erase(state.idle.begin()); } } else { - auto it = state.idle_lazy_workers.find(task_spec.TaskId()); - if (it != state.idle_lazy_workers.end()) { + auto it = state.idle_dedicated_workers.find(task_spec.TaskId()); + if (it != state.idle_dedicated_workers.end()) { worker = std::move(it->second); - state.idle_lazy_workers.erase(it); + state.idle_dedicated_workers.erase(it); } } } else if (!task_spec.IsActorTask()) { @@ -352,9 +352,9 @@ std::string WorkerPool::WarningAboutSize() { return warning_message.str(); } -bool WorkerPool::IsWorkerStartingForTask(const Language &language, const TaskID &task_id) { +bool WorkerPool::PendingRegistrationForTask(const Language &language, const TaskID &task_id) { auto &state = GetStateForLanguage(language); - for (const auto &item : state.starting_lazy_worker_processes) { + for (const auto &item : state.starting_dedicated_worker_processes) { if (item.second == task_id) { return true; } diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index a18dee828fb0..e459644c4bb3 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -120,12 +120,12 @@ class WorkerPool { std::vector> GetWorkersRunningTasksForDriver( const DriverID &driver_id) const; - /// Whether we're starting a worker for the given task. + /// Whether we are pending a registration from a worker for the given task. /// Note that, this is only used for actor creation task. /// /// \param language The required language. /// \param task_id The task that we'll query by. - bool IsWorkerStartingForTask(const Language &language, const TaskID &task_id); + bool PendingRegistrationForTask(const Language &language, const TaskID &task_id); /// Returns debug string for class. /// @@ -153,9 +153,9 @@ class WorkerPool { struct State { /// The commands and arguments used to start the worker process std::vector worker_command; - /// The pool of lazy workers for actor creation tasks + /// The pool of dedicated workers for actor creation tasks /// with prefix or suffix worker command. - std::unordered_map> idle_lazy_workers; + std::unordered_map> idle_dedicated_workers; /// The pool of idle non-actor workers. std::unordered_set> idle; /// The pool of idle actor workers. @@ -169,8 +169,8 @@ class WorkerPool { /// to the number of their unregistered workers. std::unordered_map starting_worker_processes; /// A map for looking up the task id of actor creation task by the pid of - /// worker. Note that this is used for the lazy worker processes. - std::unordered_map starting_lazy_worker_processes; + /// worker. Note that this is used for the dedicated worker processes. + std::unordered_map starting_dedicated_worker_processes; }; /// The number of workers per process. From afe1e33d6dc605102aafc5b94138f9c6240ddd8b Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Sun, 16 Jun 2019 00:13:08 +0800 Subject: [PATCH 13/46] Address comments. --- .../org/ray/runtime/raylet/RayletClientImpl.java | 10 +++++----- .../main/java/org/ray/runtime/task/TaskSpec.java | 16 ++++++++-------- src/ray/gcs/format/gcs.fbs | 4 ++-- src/ray/raylet/node_manager.cc | 2 +- src/ray/raylet/task_spec.cc | 14 +++++++------- src/ray/raylet/task_spec.h | 12 ++++++------ src/ray/raylet/worker_pool.cc | 8 ++++---- 7 files changed, 33 insertions(+), 33 deletions(-) diff --git a/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java b/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java index 7a13c5ec85ba..633a6a7b68bf 100644 --- a/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java +++ b/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java @@ -193,7 +193,7 @@ private static TaskSpec parseTaskSpecFromFlatbuffer(ByteBuffer bb) { return new TaskSpec(driverId, taskId, parentTaskId, parentCounter, actorCreationId, maxActorReconstructions, actorId, actorHandleId, actorCounter, newActorHandles, args, numReturns, resources, TaskLanguage.JAVA, functionDescriptor, - info.workerStartingPrefix(), info.workerStartingSuffix()); + info.workerCommandPrefix(), info.workerCommandSuffix()); } private static ByteBuffer convertTaskSpecToFlatbuffer(TaskSpec task) { @@ -276,8 +276,8 @@ private static ByteBuffer convertTaskSpecToFlatbuffer(TaskSpec task) { functionDescriptorOffset = fbb.createVectorOfTables(functionDescriptorOffsets); } - final int workerStartingPrefixOffset = fbb.createString(task.workerStartingPrefix); - final int workerStartingSuffixOffset = fbb.createString(task.workerStartingSuffix); + final int workerCommandPrefixOffset = fbb.createString(task.workerCommandPrefix); + final int workerCommandSuffixOffset = fbb.createString(task.workerCommandSuffix); int root = TaskInfo.createTaskInfo( fbb, @@ -298,8 +298,8 @@ private static ByteBuffer convertTaskSpecToFlatbuffer(TaskSpec task) { requiredPlacementResourcesOffset, language, functionDescriptorOffset, - workerStartingPrefixOffset, - workerStartingSuffixOffset); + workerCommandPrefixOffset, + workerCommandSuffixOffset); fbb.finish(root); ByteBuffer buffer = fbb.dataBuffer(); diff --git a/java/runtime/src/main/java/org/ray/runtime/task/TaskSpec.java b/java/runtime/src/main/java/org/ray/runtime/task/TaskSpec.java index 73f6122ba86b..ffe6795d69f6 100644 --- a/java/runtime/src/main/java/org/ray/runtime/task/TaskSpec.java +++ b/java/runtime/src/main/java/org/ray/runtime/task/TaskSpec.java @@ -63,9 +63,9 @@ public class TaskSpec { // Language of this task. public final TaskLanguage language; - public final String workerStartingPrefix; + public final String workerCommandPrefix; - public final String workerStartingSuffix; + public final String workerCommandSuffix; // Descriptor of the remote function. // Note, if task language is Java, the type is JavaFunctionDescriptor. If the task language @@ -98,8 +98,8 @@ public TaskSpec( Map resources, TaskLanguage language, FunctionDescriptor functionDescriptor, - String workerStartingPrefix, - String workerStartingSuffix) { + String workerCommandPrefix, + String workerCommandSuffix) { this.driverId = driverId; this.taskId = taskId; this.parentTaskId = parentTaskId; @@ -112,8 +112,8 @@ public TaskSpec( this.newActorHandles = newActorHandles; this.args = args; this.numReturns = numReturns; - this.workerStartingPrefix = workerStartingPrefix; - this.workerStartingSuffix = workerStartingSuffix; + this.workerCommandPrefix = workerCommandPrefix; + this.workerCommandSuffix = workerCommandSuffix; returnIds = new ObjectId[numReturns]; for (int i = 0; i < numReturns; ++i) { returnIds[i] = IdUtil.computeReturnId(taskId, i + 1); @@ -165,8 +165,8 @@ public String toString() { ", resources=" + resources + ", language=" + language + ", functionDescriptor=" + functionDescriptor + - ", workerStartingPrefix=" + workerStartingPrefix + - ", workerStartingSuffix=" + workerStartingSuffix + + ", workerCommandPrefix=" + workerCommandPrefix + + ", workerCommandSuffix=" + workerCommandSuffix + ", executionDependencies=" + executionDependencies + '}'; } diff --git a/src/ray/gcs/format/gcs.fbs b/src/ray/gcs/format/gcs.fbs index ff5676a3460e..00c6e2dc507a 100644 --- a/src/ray/gcs/format/gcs.fbs +++ b/src/ray/gcs/format/gcs.fbs @@ -107,9 +107,9 @@ table TaskInfo { // For a Java function, it should be: [class_name, method_name, type_descriptor] function_descriptor: [string]; // The prefix string for worker starting. This is only used for actor creating tasks. - worker_starting_prefix: string; + worker_command_prefix: string; // The suffix string for worker starting. - worker_starting_suffix: string; + worker_command_suffix: string; } table ResourcePair { diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index d69b45f9a912..26fabda71dc2 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1724,7 +1724,7 @@ bool NodeManager::AssignTask(const Task &task) { if (worker == nullptr) { // There are no workers that can execute this task. const bool should_start_new = - (!spec.WorkerStartingPrefix().empty() || !spec.WorkerStartingSuffix().empty()); + (!spec.WorkerCommandPrefix().empty() || !spec.WorkerCommandSuffix().empty()); if (spec.IsActorCreationTask() && should_start_new) { if (!worker_pool_.PendingRegistrationForTask(spec.GetLanguage(), spec.TaskId())) { worker_pool_.StartWorkerProcess(spec.GetLanguage(), &spec); diff --git a/src/ray/raylet/task_spec.cc b/src/ray/raylet/task_spec.cc index e766678af974..bfcddf6fc81f 100644 --- a/src/ray/raylet/task_spec.cc +++ b/src/ray/raylet/task_spec.cc @@ -81,7 +81,7 @@ TaskSpecification::TaskSpecification( const std::unordered_map &required_resources, const std::unordered_map &required_placement_resources, const Language &language, const std::vector &function_descriptor, - const std::string &worker_starting_prefix, const std::string &worker_starting_suffix) + const std::string &worker_command_prefix, const std::string &worker_command_suffix) : spec_() { flatbuffers::FlatBufferBuilder fbb; @@ -103,8 +103,8 @@ TaskSpecification::TaskSpecification( map_to_flatbuf(fbb, required_resources), map_to_flatbuf(fbb, required_placement_resources), language, string_vec_to_flatbuf(fbb, function_descriptor), - fbb.CreateString(worker_starting_prefix.c_str(), worker_starting_prefix.size()), - fbb.CreateString(worker_starting_suffix.c_str(), worker_starting_suffix.size())); + fbb.CreateString(worker_command_prefix.c_str(), worker_command_prefix.size()), + fbb.CreateString(worker_command_suffix.c_str(), worker_command_suffix.size())); fbb.Finish(spec); AssignSpecification(fbb.GetBufferPointer(), fbb.GetSize()); } @@ -261,14 +261,14 @@ std::vector TaskSpecification::NewActorHandles() const { return ids_from_flatbuf(*message->new_actor_handles()); } -std::string TaskSpecification::WorkerStartingPrefix() const { +std::string TaskSpecification::WorkerCommandPrefix() const { auto message = flatbuffers::GetRoot(spec_.data()); - return string_from_flatbuf(*message->worker_starting_prefix()); + return string_from_flatbuf(*message->worker_command_prefix()); } -std::string TaskSpecification::WorkerStartingSuffix() const { +std::string TaskSpecification::WorkerCommandSuffix() const { auto message = flatbuffers::GetRoot(spec_.data()); - return string_from_flatbuf(*message->worker_starting_suffix()); + return string_from_flatbuf(*message->worker_command_suffix()); } } // namespace raylet diff --git a/src/ray/raylet/task_spec.h b/src/ray/raylet/task_spec.h index 004c0a754b2b..ffdf78d9a2e8 100644 --- a/src/ray/raylet/task_spec.h +++ b/src/ray/raylet/task_spec.h @@ -128,8 +128,8 @@ class TaskSpecification { /// will default to be equal to the required_resources argument. /// \param language The language of the worker that must execute the function. /// \param function_descriptor The function descriptor. - /// \param worker_starting_prefix The prefix string for starting an actor worker. - /// \param worker_starting_suffix The suffix string for starting an actor worker. + /// \param worker_command_prefix The prefix string for starting an actor worker. + /// \param worker_command_suffix The suffix string for starting an actor worker. TaskSpecification( const DriverID &driver_id, const TaskID &parent_task_id, int64_t parent_counter, const ActorID &actor_creation_id, const ObjectID &actor_creation_dummy_object_id, @@ -141,8 +141,8 @@ class TaskSpecification { const std::unordered_map &required_resources, const std::unordered_map &required_placement_resources, const Language &language, const std::vector &function_descriptor, - const std::string &worker_starting_prefix = "", - const std::string &worker_starting_suffix = ""); + const std::string &worker_command_prefix = "", + const std::string &worker_command_suffix = ""); /// Deserialize a task specification from a string. /// @@ -219,8 +219,8 @@ class TaskSpecification { std::vector NewActorHandles() const; // Methods specific to actor creating tasks. - std::string WorkerStartingPrefix() const; - std::string WorkerStartingSuffix() const; + std::string WorkerCommandPrefix() const; + std::string WorkerCommandSuffix() const; private: /// Assign the specification data from a pointer. diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 431e29ba7c02..1bc1418613b6 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -120,8 +120,8 @@ void WorkerPool::StartWorkerProcess(const Language &language, std::string prefix = ""; std::string suffix = ""; if (task_spec != nullptr && task_spec->IsActorCreationTask()) { - prefix = task_spec->WorkerStartingPrefix(); - suffix = task_spec->WorkerStartingSuffix(); + prefix = task_spec->WorkerCommandPrefix(); + suffix = task_spec->WorkerCommandSuffix(); } // Extract pointers from the worker command to pass into execvp. @@ -258,8 +258,8 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec std::shared_ptr worker = nullptr; if (task_spec.IsActorCreationTask()) { // code path of actor creation task. - if (task_spec.WorkerStartingPrefix().empty() && - task_spec.WorkerStartingSuffix().empty()) { + if (task_spec.WorkerCommandPrefix().empty() && + task_spec.WorkerCommandSuffix().empty()) { // There is no prefix and suffix of command. Pop a worker from idle worker pool of // non-actor. if (!state.idle.empty()) { From 569c23664b8811541463fd1348944132f591513a Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Sun, 16 Jun 2019 00:23:34 +0800 Subject: [PATCH 14/46] Fix linting --- src/ray/raylet/worker_pool.cc | 12 +++++++----- src/ray/raylet/worker_pool_test.cc | 29 +++++++++++++++++------------ 2 files changed, 24 insertions(+), 17 deletions(-) diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 1bc1418613b6..21892c8db75c 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -5,11 +5,11 @@ #include #include +#include "ray/common/constants.h" #include "ray/common/ray_config.h" #include "ray/common/status.h" #include "ray/stats/stats.h" #include "ray/util/logging.h" -#include "ray/common/constants.h" namespace { @@ -135,7 +135,7 @@ void WorkerPool::StartWorkerProcess(const Language &language, if (!suffix.empty()) { worker_command_args.push_back(suffix.c_str()); } - }else { + } else { worker_command_args.push_back(token.c_str()); } } @@ -152,8 +152,9 @@ void WorkerPool::StartWorkerProcess(const Language &language, state.starting_worker_processes.emplace( std::make_pair(pid, num_workers_per_process_)); if (!prefix.empty() || !suffix.empty()) { - RAY_CHECK(task_spec != nullptr) << "task_spec should not be nullptr " - "since we specified prefix or suffix for worker command"; + RAY_CHECK(task_spec != nullptr) + << "task_spec should not be nullptr " + "since we specified prefix or suffix for worker command"; state.starting_dedicated_worker_processes[pid] = task_spec->TaskId(); } return; @@ -352,7 +353,8 @@ std::string WorkerPool::WarningAboutSize() { return warning_message.str(); } -bool WorkerPool::PendingRegistrationForTask(const Language &language, const TaskID &task_id) { +bool WorkerPool::PendingRegistrationForTask(const Language &language, + const TaskID &task_id) { auto &state = GetStateForLanguage(language); for (const auto &item : state.starting_dedicated_worker_processes) { if (item.second == task_id) { diff --git a/src/ray/raylet/worker_pool_test.cc b/src/ray/raylet/worker_pool_test.cc index 5d8721847cbc..594fda0540d7 100644 --- a/src/ray/raylet/worker_pool_test.cc +++ b/src/ray/raylet/worker_pool_test.cc @@ -1,9 +1,9 @@ #include "gmock/gmock.h" #include "gtest/gtest.h" +#include "ray/common/constants.h" #include "ray/raylet/node_manager.h" #include "ray/raylet/worker_pool.h" -#include "ray/common/constants.h" namespace ray { @@ -18,8 +18,10 @@ class WorkerPoolMock : public WorkerPool { : WorkerPoolMock({{Language::PYTHON, {"dummy_py_worker_command"}}, {Language::JAVA, {"dummy_java_worker_command"}}}) {} - explicit WorkerPoolMock(const std::unordered_map> &worker_commands) - : WorkerPool(0, NUM_WORKERS_PER_PROCESS, MAXIMUM_STARTUP_CONCURRENCY, worker_commands), + explicit WorkerPoolMock( + const std::unordered_map> &worker_commands) + : WorkerPool(0, NUM_WORKERS_PER_PROCESS, MAXIMUM_STARTUP_CONCURRENCY, + worker_commands), last_worker_pid_(0) {} ~WorkerPoolMock() { @@ -81,7 +83,8 @@ class WorkerPoolTest : public ::testing::Test { return std::shared_ptr(new Worker(pid, language, client)); } - void SetWorkerCommands(const std::unordered_map> &worker_commands) { + void SetWorkerCommands( + const std::unordered_map> &worker_commands) { WorkerPoolMock worker_pool(worker_commands); this->worker_pool_ = std::move(worker_pool); } @@ -97,8 +100,7 @@ class WorkerPoolTest : public ::testing::Test { }; static inline TaskSpecification ExampleTaskSpec( - const ActorID actor_id = ActorID::Nil(), - const Language &language = Language::PYTHON, + const ActorID actor_id = ActorID::Nil(), const Language &language = Language::PYTHON, const ActorID actor_creation_id = ActorID::Nil()) { std::vector function_descriptor(3); return TaskSpecification(DriverID::Nil(), TaskID::Nil(), 0, actor_creation_id, @@ -213,15 +215,18 @@ TEST_F(WorkerPoolTest, PopWorkersOfMultipleLanguages) { } TEST_F(WorkerPoolTest, StartWorkerWithPrefixAndSuffix) { - SetWorkerCommands({ - {Language::PYTHON, {"dummy_py_worker_command"}}, - {Language::JAVA, {kPrefixPlaceholder, "dummy_java_worker_command", kSuffixPlaceholder}}}); + SetWorkerCommands( + {{Language::PYTHON, {"dummy_py_worker_command"}}, + {Language::JAVA, + {kPrefixPlaceholder, "dummy_java_worker_command", kSuffixPlaceholder}}}); TaskSpecification task_spec(DriverID::Nil(), TaskID::Nil(), 0, ActorID::FromRandom(), - ObjectID::Nil(), 0, ActorID::Nil(), ActorHandleID::Nil(), 0, {}, {}, - 0, {}, {}, Language::JAVA, {"", "", ""}, "test_prefix", "test_suffix"); + ObjectID::Nil(), 0, ActorID::Nil(), ActorHandleID::Nil(), 0, + {}, {}, 0, {}, {}, Language::JAVA, {"", "", ""}, + "test_prefix", "test_suffix"); worker_pool_.StartWorkerProcess(Language::JAVA, &task_spec); - const auto real_command = worker_pool_.GetWorkerCommand(worker_pool_.LastStartedWorkerProcess()); + const auto real_command = + worker_pool_.GetWorkerCommand(worker_pool_.LastStartedWorkerProcess()); ASSERT_EQ(3, real_command.size()); ASSERT_EQ(std::string("test_prefix"), std::string(real_command[0])); ASSERT_EQ("dummy_java_worker_command", real_command[1]); From 3a5ee31d1a084eedaa711cbe422564544373890d Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Sun, 16 Jun 2019 00:31:09 +0800 Subject: [PATCH 15/46] Refine --- src/ray/raylet/worker_pool_test.cc | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/ray/raylet/worker_pool_test.cc b/src/ray/raylet/worker_pool_test.cc index 594fda0540d7..6cde8bcef4c2 100644 --- a/src/ray/raylet/worker_pool_test.cc +++ b/src/ray/raylet/worker_pool_test.cc @@ -215,10 +215,11 @@ TEST_F(WorkerPoolTest, PopWorkersOfMultipleLanguages) { } TEST_F(WorkerPoolTest, StartWorkerWithPrefixAndSuffix) { + const std::vector java_worker_command + = {kPrefixPlaceholder, "dummy_java_worker_command", kSuffixPlaceholder}; SetWorkerCommands( {{Language::PYTHON, {"dummy_py_worker_command"}}, - {Language::JAVA, - {kPrefixPlaceholder, "dummy_java_worker_command", kSuffixPlaceholder}}}); + {Language::JAVA, java_worker_command}}); TaskSpecification task_spec(DriverID::Nil(), TaskID::Nil(), 0, ActorID::FromRandom(), ObjectID::Nil(), 0, ActorID::Nil(), ActorHandleID::Nil(), 0, @@ -228,7 +229,7 @@ TEST_F(WorkerPoolTest, StartWorkerWithPrefixAndSuffix) { const auto real_command = worker_pool_.GetWorkerCommand(worker_pool_.LastStartedWorkerProcess()); ASSERT_EQ(3, real_command.size()); - ASSERT_EQ(std::string("test_prefix"), std::string(real_command[0])); + ASSERT_EQ("test_prefix", real_command[0]); ASSERT_EQ("dummy_java_worker_command", real_command[1]); ASSERT_EQ("test_suffix", real_command[2]); } From 8f8326c2bd962b1755b2d960453166aab441d882 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Sun, 16 Jun 2019 10:30:22 +0800 Subject: [PATCH 16/46] Fix lint --- src/ray/raylet/worker_pool_test.cc | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/ray/raylet/worker_pool_test.cc b/src/ray/raylet/worker_pool_test.cc index 6cde8bcef4c2..966f126a23a3 100644 --- a/src/ray/raylet/worker_pool_test.cc +++ b/src/ray/raylet/worker_pool_test.cc @@ -215,11 +215,10 @@ TEST_F(WorkerPoolTest, PopWorkersOfMultipleLanguages) { } TEST_F(WorkerPoolTest, StartWorkerWithPrefixAndSuffix) { - const std::vector java_worker_command - = {kPrefixPlaceholder, "dummy_java_worker_command", kSuffixPlaceholder}; - SetWorkerCommands( - {{Language::PYTHON, {"dummy_py_worker_command"}}, - {Language::JAVA, java_worker_command}}); + const std::vector java_worker_command = { + kPrefixPlaceholder, "dummy_java_worker_command", kSuffixPlaceholder}; + SetWorkerCommands({{Language::PYTHON, {"dummy_py_worker_command"}}, + {Language::JAVA, java_worker_command}}); TaskSpecification task_spec(DriverID::Nil(), TaskID::Nil(), 0, ActorID::FromRandom(), ObjectID::Nil(), 0, ActorID::Nil(), ActorHandleID::Nil(), 0, From 23fede1e56b8c780196cfdd0107f61311befd48a Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Tue, 18 Jun 2019 11:56:27 +0800 Subject: [PATCH 17/46] WIP: address comment. --- src/ray/common/constants.h | 5 +---- src/ray/gcs/format/gcs.fbs | 8 ++++---- src/ray/raylet/node_manager.cc | 5 ++--- src/ray/raylet/task_spec.cc | 23 +++++++++++----------- src/ray/raylet/task_spec.h | 10 +++------- src/ray/raylet/worker_pool.cc | 36 +++++++++++++++++----------------- 6 files changed, 40 insertions(+), 47 deletions(-) diff --git a/src/ray/common/constants.h b/src/ray/common/constants.h index 779084bc981e..88c7f4749653 100644 --- a/src/ray/common/constants.h +++ b/src/ray/common/constants.h @@ -36,9 +36,6 @@ constexpr char kObjectTablePrefix[] = "ObjectTable"; /// Prefix for the task table keys in redis. constexpr char kTaskTablePrefix[] = "TaskTable"; -/// The constant used as a placeholder of the prefix for worker command. -constexpr char kPrefixPlaceholder[] = "__PREFIX_PLACEHOLDER__"; -/// The constant used as a placeholder of the suffix for worker command. -constexpr char kSuffixPlaceholder[] = "__SUFFIX_PLACEHOLDER__"; +constexpr char kWorkerOptionPlaceHolderPrefix[] = "RAY_WORKER_OPTION_"; #endif // RAY_CONSTANTS_H_ diff --git a/src/ray/gcs/format/gcs.fbs b/src/ray/gcs/format/gcs.fbs index 00c6e2dc507a..a2ecc0e95860 100644 --- a/src/ray/gcs/format/gcs.fbs +++ b/src/ray/gcs/format/gcs.fbs @@ -106,10 +106,10 @@ table TaskInfo { // For a Python function, it should be: [module_name, class_name, function_name] // For a Java function, it should be: [class_name, method_name, type_descriptor] function_descriptor: [string]; - // The prefix string for worker starting. This is only used for actor creating tasks. - worker_command_prefix: string; - // The suffix string for worker starting. - worker_command_suffix: string; + // The dynamic options for worker command. + // It means we can specify the options for the worker when creating an actor. + // Note that this is used for actor creating task only. + dynamic_worker_options: [string]; } table ResourcePair { diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 26fabda71dc2..7a154ebba5e1 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1723,9 +1723,8 @@ bool NodeManager::AssignTask(const Task &task) { std::shared_ptr worker = worker_pool_.PopWorker(spec); if (worker == nullptr) { // There are no workers that can execute this task. - const bool should_start_new = - (!spec.WorkerCommandPrefix().empty() || !spec.WorkerCommandSuffix().empty()); - if (spec.IsActorCreationTask() && should_start_new) { + const bool should_start_new = spec.IsActorCreationTask() && !spec.DynamicWorkerOptions().empty(); + if (should_start_new) { if (!worker_pool_.PendingRegistrationForTask(spec.GetLanguage(), spec.TaskId())) { worker_pool_.StartWorkerProcess(spec.GetLanguage(), &spec); } diff --git a/src/ray/raylet/task_spec.cc b/src/ray/raylet/task_spec.cc index bfcddf6fc81f..a4e3253c894c 100644 --- a/src/ray/raylet/task_spec.cc +++ b/src/ray/raylet/task_spec.cc @@ -81,12 +81,17 @@ TaskSpecification::TaskSpecification( const std::unordered_map &required_resources, const std::unordered_map &required_placement_resources, const Language &language, const std::vector &function_descriptor, - const std::string &worker_command_prefix, const std::string &worker_command_suffix) + const Language &language, const std::vector &dynamic_worker_options) : spec_() { + // check + if (actor_creation_id.Nil()) { + RAY_CHECK(dynamic_worker_options.size() == 0) + << "`dynamic_worker_options` only can be specified for actor creating tasks."; + } + flatbuffers::FlatBufferBuilder fbb; TaskID task_id = GenerateTaskId(driver_id, parent_task_id, parent_counter); - // Add argument object IDs. std::vector> arguments; for (auto &argument : task_arguments) { @@ -103,8 +108,7 @@ TaskSpecification::TaskSpecification( map_to_flatbuf(fbb, required_resources), map_to_flatbuf(fbb, required_placement_resources), language, string_vec_to_flatbuf(fbb, function_descriptor), - fbb.CreateString(worker_command_prefix.c_str(), worker_command_prefix.size()), - fbb.CreateString(worker_command_suffix.c_str(), worker_command_suffix.size())); + string_vec_to_flatbuf(fbb, dynamic_worker_options), fbb.Finish(spec); AssignSpecification(fbb.GetBufferPointer(), fbb.GetSize()); } @@ -261,14 +265,11 @@ std::vector TaskSpecification::NewActorHandles() const { return ids_from_flatbuf(*message->new_actor_handles()); } -std::string TaskSpecification::WorkerCommandPrefix() const { - auto message = flatbuffers::GetRoot(spec_.data()); - return string_from_flatbuf(*message->worker_command_prefix()); -} - -std::string TaskSpecification::WorkerCommandSuffix() const { +std::vector TaskSpecification::DynamicWorkerOptions() const { + RAY_CHECK(IsActorCreationTask()) + << "This method only can be called for actor creating tasks."; auto message = flatbuffers::GetRoot(spec_.data()); - return string_from_flatbuf(*message->worker_command_suffix()); + return string_vec_from_flatbuf(*message->dynamic_worker_options()); } } // namespace raylet diff --git a/src/ray/raylet/task_spec.h b/src/ray/raylet/task_spec.h index ffdf78d9a2e8..8a08e9974ef2 100644 --- a/src/ray/raylet/task_spec.h +++ b/src/ray/raylet/task_spec.h @@ -128,8 +128,7 @@ class TaskSpecification { /// will default to be equal to the required_resources argument. /// \param language The language of the worker that must execute the function. /// \param function_descriptor The function descriptor. - /// \param worker_command_prefix The prefix string for starting an actor worker. - /// \param worker_command_suffix The suffix string for starting an actor worker. + /// \param dynamic_worker_options The dynamic options for starting an actor worker. TaskSpecification( const DriverID &driver_id, const TaskID &parent_task_id, int64_t parent_counter, const ActorID &actor_creation_id, const ObjectID &actor_creation_dummy_object_id, @@ -141,8 +140,7 @@ class TaskSpecification { const std::unordered_map &required_resources, const std::unordered_map &required_placement_resources, const Language &language, const std::vector &function_descriptor, - const std::string &worker_command_prefix = "", - const std::string &worker_command_suffix = ""); + const std::vector &dynamic_worker_options = {}); /// Deserialize a task specification from a string. /// @@ -218,9 +216,7 @@ class TaskSpecification { ObjectID ActorDummyObject() const; std::vector NewActorHandles() const; - // Methods specific to actor creating tasks. - std::string WorkerCommandPrefix() const; - std::string WorkerCommandSuffix() const; + std::vector DynamicWorkerOptions() const; private: /// Assign the specification data from a pointer. diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 21892c8db75c..c79567eedf91 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -117,24 +117,21 @@ void WorkerPool::StartWorkerProcess(const Language &language, << state.idle_actor.size() << " actor workers, and " << state.idle.size() << " non-actor workers"; - std::string prefix = ""; - std::string suffix = ""; + std::vector dynamic_worker_options; if (task_spec != nullptr && task_spec->IsActorCreationTask()) { - prefix = task_spec->WorkerCommandPrefix(); - suffix = task_spec->WorkerCommandSuffix(); + dynamic_worker_options = task_spec.DynamicWorkerOptions(); } // Extract pointers from the worker command to pass into execvp. std::vector worker_command_args; + size_t dynamic_option_index = 0; for (auto const &token : state.worker_command) { - if (token == kPrefixPlaceholder) { - if (!prefix.empty()) { - worker_command_args.push_back(prefix.c_str()); - } - } else if (token == kSuffixPlaceholder) { - if (!suffix.empty()) { - worker_command_args.push_back(suffix.c_str()); - } + const auto option_placeholder = + kWorkerOptionPlaceHolderPrefix + std::to_string(dynamic_option_index); + + if (token == option_placeholder) { + RAY_CHECK(dynamic_option_index < dynamic_worker_options.size()); + worker_command_args.push_back(dynamic_worker_options[dynamic_option_index].c_str()); } else { worker_command_args.push_back(token.c_str()); } @@ -151,10 +148,10 @@ void WorkerPool::StartWorkerProcess(const Language &language, RAY_LOG(DEBUG) << "Started worker process with pid " << pid; state.starting_worker_processes.emplace( std::make_pair(pid, num_workers_per_process_)); - if (!prefix.empty() || !suffix.empty()) { + if (!dynamic_worker_options.empty()) { RAY_CHECK(task_spec != nullptr) << "task_spec should not be nullptr " - "since we specified prefix or suffix for worker command"; + "because we specified dynamic worker options for this task."; state.starting_dedicated_worker_processes[pid] = task_spec->TaskId(); } return; @@ -259,15 +256,18 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec std::shared_ptr worker = nullptr; if (task_spec.IsActorCreationTask()) { // code path of actor creation task. - if (task_spec.WorkerCommandPrefix().empty() && - task_spec.WorkerCommandSuffix().empty()) { - // There is no prefix and suffix of command. Pop a worker from idle worker pool of - // non-actor. + if (task_spec.DynamicWorkerOptions().empty()) { + // There is no dynamic worker options of this task. Pop a worker + // from idle worker pool of non-actor. if (!state.idle.empty()) { worker = std::move(*state.idle.begin()); state.idle.erase(state.idle.begin()); + } else { + RAY_LOG(DEBUG) << "There is no idle worker for the task " + << task_spec.TaskId() << " ."; } } else { + // Dynamic worker options is not empty. Try to pop it from idle dedicated pool. auto it = state.idle_dedicated_workers.find(task_spec.TaskId()); if (it != state.idle_dedicated_workers.end()) { worker = std::move(it->second); From b6dba59858109b6432917fead2b56278bac7c443 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Tue, 18 Jun 2019 13:35:47 +0800 Subject: [PATCH 18/46] Fix java --- .../org/ray/runtime/AbstractRayRuntime.java | 11 ++++++---- .../ray/runtime/raylet/RayletClientImpl.java | 22 ++++++++++++++----- .../org/ray/runtime/runner/RunManager.java | 7 +----- .../java/org/ray/runtime/task/TaskSpec.java | 19 ++++++++-------- src/ray/raylet/task_spec.cc | 6 ++--- src/ray/raylet/worker_pool.cc | 8 ++++--- 6 files changed, 42 insertions(+), 31 deletions(-) diff --git a/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java b/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java index bfd9d44cadc2..030f77271e34 100644 --- a/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java +++ b/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java @@ -35,6 +35,7 @@ import org.ray.runtime.task.TaskLanguage; import org.ray.runtime.task.TaskSpec; import org.ray.runtime.util.IdUtil; +import org.ray.runtime.util.StringUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -363,10 +364,13 @@ private TaskSpec createTaskSpec(RayFunc func, PyFunctionDescriptor pyFunctionDes } int maxActorReconstruction = 0; - String workerStartingSuffix = ""; + List dynamicWorkerOptions = null; if (taskOptions instanceof ActorCreationOptions) { maxActorReconstruction = ((ActorCreationOptions) taskOptions).maxReconstructions; - workerStartingSuffix = ((ActorCreationOptions) taskOptions).jvmOptions; + String jvmOptions = ((ActorCreationOptions) taskOptions).jvmOptions; + if (!StringUtil.isNullOrEmpty(jvmOptions)) { + dynamicWorkerOptions = ImmutableList.of(((ActorCreationOptions) taskOptions).jvmOptions); + } } TaskLanguage language; @@ -396,8 +400,7 @@ private TaskSpec createTaskSpec(RayFunc func, PyFunctionDescriptor pyFunctionDes resources, language, functionDescriptor, - "", - workerStartingSuffix + dynamicWorkerOptions ); } diff --git a/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java b/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java index 633a6a7b68bf..7d0ecb2b1646 100644 --- a/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java +++ b/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java @@ -190,10 +190,16 @@ private static TaskSpec parseTaskSpecFromFlatbuffer(ByteBuffer bb) { JavaFunctionDescriptor functionDescriptor = new JavaFunctionDescriptor( info.functionDescriptor(0), info.functionDescriptor(1), info.functionDescriptor(2) ); + + // Deserialize dynamic worker options. + List dynamicWorkerOptions = new ArrayList<>(); + for (int i = 0; i < info.dynamicWorkerOptionsLength(); ++i) { + dynamicWorkerOptions.add(info.dynamicWorkerOptions(i)); + } + return new TaskSpec(driverId, taskId, parentTaskId, parentCounter, actorCreationId, maxActorReconstructions, actorId, actorHandleId, actorCounter, newActorHandles, - args, numReturns, resources, TaskLanguage.JAVA, functionDescriptor, - info.workerCommandPrefix(), info.workerCommandSuffix()); + args, numReturns, resources, TaskLanguage.JAVA, functionDescriptor, dynamicWorkerOptions); } private static ByteBuffer convertTaskSpecToFlatbuffer(TaskSpec task) { @@ -276,8 +282,13 @@ private static ByteBuffer convertTaskSpecToFlatbuffer(TaskSpec task) { functionDescriptorOffset = fbb.createVectorOfTables(functionDescriptorOffsets); } - final int workerCommandPrefixOffset = fbb.createString(task.workerCommandPrefix); - final int workerCommandSuffixOffset = fbb.createString(task.workerCommandSuffix); + // TODO(qwang): + + int [] dynamicWorkerOptionsOffsets = new int[task.dynamicWorkerOptions.size()]; + for (int index = 0; index < task.dynamicWorkerOptions.size(); ++index) { + dynamicWorkerOptionsOffsets[index] = fbb.createString(task.dynamicWorkerOptions.get(index)); + } + int dynamicWorkerOptionsOffset = fbb.createVectorOfTables(dynamicWorkerOptionsOffsets); int root = TaskInfo.createTaskInfo( fbb, @@ -298,8 +309,7 @@ private static ByteBuffer convertTaskSpecToFlatbuffer(TaskSpec task) { requiredPlacementResourcesOffset, language, functionDescriptorOffset, - workerCommandPrefixOffset, - workerCommandSuffixOffset); + dynamicWorkerOptionsOffset); fbb.finish(root); ByteBuffer buffer = fbb.dataBuffer(); diff --git a/java/runtime/src/main/java/org/ray/runtime/runner/RunManager.java b/java/runtime/src/main/java/org/ray/runtime/runner/RunManager.java index a4e451d641b4..773499fcf5cf 100644 --- a/java/runtime/src/main/java/org/ray/runtime/runner/RunManager.java +++ b/java/runtime/src/main/java/org/ray/runtime/runner/RunManager.java @@ -38,10 +38,6 @@ public class RunManager { private static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern("Y-M-d_H-m-s"); - private static final String PREFIX_PLACEHOLDER = "__PREFIX_PLACEHOLDER__"; - - private static final String SUFFIX_PLACEHOLDER = "__SUFFIX_PLACEHOLDER__"; - private static final String WORKER_CLASS = "org.ray.runtime.runner.worker.DefaultWorker"; private RayConfig rayConfig; @@ -285,7 +281,6 @@ private String concatPath(Stream stream) { private String buildWorkerCommandRaylet() { List cmd = new ArrayList<>(); - cmd.add(PREFIX_PLACEHOLDER); cmd.add("java"); cmd.add("-classpath"); @@ -325,7 +320,7 @@ private String buildWorkerCommandRaylet() { cmd.addAll(rayConfig.jvmParameters); // jvm options - cmd.add(SUFFIX_PLACEHOLDER); + cmd.add("RAY_WORKER_OPTION_0"); // Main class cmd.add(WORKER_CLASS); diff --git a/java/runtime/src/main/java/org/ray/runtime/task/TaskSpec.java b/java/runtime/src/main/java/org/ray/runtime/task/TaskSpec.java index ffe6795d69f6..b737cf8da45a 100644 --- a/java/runtime/src/main/java/org/ray/runtime/task/TaskSpec.java +++ b/java/runtime/src/main/java/org/ray/runtime/task/TaskSpec.java @@ -63,9 +63,7 @@ public class TaskSpec { // Language of this task. public final TaskLanguage language; - public final String workerCommandPrefix; - - public final String workerCommandSuffix; + public final List dynamicWorkerOptions; // Descriptor of the remote function. // Note, if task language is Java, the type is JavaFunctionDescriptor. If the task language @@ -98,8 +96,7 @@ public TaskSpec( Map resources, TaskLanguage language, FunctionDescriptor functionDescriptor, - String workerCommandPrefix, - String workerCommandSuffix) { + List dynamicWorkerOptions) { this.driverId = driverId; this.taskId = taskId; this.parentTaskId = parentTaskId; @@ -112,8 +109,13 @@ public TaskSpec( this.newActorHandles = newActorHandles; this.args = args; this.numReturns = numReturns; - this.workerCommandPrefix = workerCommandPrefix; - this.workerCommandSuffix = workerCommandSuffix; + + if (dynamicWorkerOptions == null) { + this.dynamicWorkerOptions = new ArrayList<>(); + } else { + this.dynamicWorkerOptions =dynamicWorkerOptions; + } + returnIds = new ObjectId[numReturns]; for (int i = 0; i < numReturns; ++i) { returnIds[i] = IdUtil.computeReturnId(taskId, i + 1); @@ -165,8 +167,7 @@ public String toString() { ", resources=" + resources + ", language=" + language + ", functionDescriptor=" + functionDescriptor + - ", workerCommandPrefix=" + workerCommandPrefix + - ", workerCommandSuffix=" + workerCommandSuffix + + ", dynamicWorkerOptions=" + dynamicWorkerOptions + ", executionDependencies=" + executionDependencies + '}'; } diff --git a/src/ray/raylet/task_spec.cc b/src/ray/raylet/task_spec.cc index a4e3253c894c..46d4e2ced23e 100644 --- a/src/ray/raylet/task_spec.cc +++ b/src/ray/raylet/task_spec.cc @@ -81,10 +81,10 @@ TaskSpecification::TaskSpecification( const std::unordered_map &required_resources, const std::unordered_map &required_placement_resources, const Language &language, const std::vector &function_descriptor, - const Language &language, const std::vector &dynamic_worker_options) + const std::vector &dynamic_worker_options) : spec_() { // check - if (actor_creation_id.Nil()) { + if (actor_creation_id.IsNil()) { RAY_CHECK(dynamic_worker_options.size() == 0) << "`dynamic_worker_options` only can be specified for actor creating tasks."; } @@ -108,7 +108,7 @@ TaskSpecification::TaskSpecification( map_to_flatbuf(fbb, required_resources), map_to_flatbuf(fbb, required_placement_resources), language, string_vec_to_flatbuf(fbb, function_descriptor), - string_vec_to_flatbuf(fbb, dynamic_worker_options), + string_vec_to_flatbuf(fbb, dynamic_worker_options)); fbb.Finish(spec); AssignSpecification(fbb.GetBufferPointer(), fbb.GetSize()); } diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index c79567eedf91..3d6df00b02e3 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -119,7 +119,7 @@ void WorkerPool::StartWorkerProcess(const Language &language, std::vector dynamic_worker_options; if (task_spec != nullptr && task_spec->IsActorCreationTask()) { - dynamic_worker_options = task_spec.DynamicWorkerOptions(); + dynamic_worker_options = task_spec->DynamicWorkerOptions(); } // Extract pointers from the worker command to pass into execvp. @@ -130,8 +130,10 @@ void WorkerPool::StartWorkerProcess(const Language &language, kWorkerOptionPlaceHolderPrefix + std::to_string(dynamic_option_index); if (token == option_placeholder) { - RAY_CHECK(dynamic_option_index < dynamic_worker_options.size()); - worker_command_args.push_back(dynamic_worker_options[dynamic_option_index].c_str()); + if (!dynamic_worker_options.empty()) { + RAY_CHECK(dynamic_option_index < dynamic_worker_options.size()); + worker_command_args.push_back(dynamic_worker_options[dynamic_option_index].c_str()); + } } else { worker_command_args.push_back(token.c_str()); } From 43bfcf2e9de04a04b9943a842c9f2626973bb7f0 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Tue, 18 Jun 2019 15:17:23 +0800 Subject: [PATCH 19/46] Fix py --- python/ray/services.py | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/python/ray/services.py b/python/ray/services.py index 878668246af8..5a85058796a8 100644 --- a/python/ray/services.py +++ b/python/ray/services.py @@ -1232,8 +1232,7 @@ def build_java_worker_command( """ assert java_worker_options is not None - # Add the prefix placeholder. - command = "{} java ".format(ray_constants.PREFIX_PLACEHOLDER) + command = "java " if redis_address is not None: command += "-Dray.redis.address={} ".format(redis_address) @@ -1256,10 +1255,7 @@ def build_java_worker_command( # above options. command += java_worker_options + " " - # Add the suffix placeholder as jvm options. - command += "{} ".format(ray_constants.SUFFIX_PLACEHOLDER) - - command += "org.ray.runtime.runner.worker.DefaultWorker" + command += "RAY_WORKER_OPTION_0 org.ray.runtime.runner.worker.DefaultWorker" return command From e58c7e59917dd3e82f6aa9e419ddedaa3d94c05f Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Wed, 19 Jun 2019 20:02:15 +0800 Subject: [PATCH 20/46] Refin --- src/ray/raylet/task_spec.cc | 8 -------- src/ray/raylet/worker_pool_test.cc | 1 - 2 files changed, 9 deletions(-) diff --git a/src/ray/raylet/task_spec.cc b/src/ray/raylet/task_spec.cc index 46d4e2ced23e..1d722de18f73 100644 --- a/src/ray/raylet/task_spec.cc +++ b/src/ray/raylet/task_spec.cc @@ -83,12 +83,6 @@ TaskSpecification::TaskSpecification( const Language &language, const std::vector &function_descriptor, const std::vector &dynamic_worker_options) : spec_() { - // check - if (actor_creation_id.IsNil()) { - RAY_CHECK(dynamic_worker_options.size() == 0) - << "`dynamic_worker_options` only can be specified for actor creating tasks."; - } - flatbuffers::FlatBufferBuilder fbb; TaskID task_id = GenerateTaskId(driver_id, parent_task_id, parent_counter); @@ -266,8 +260,6 @@ std::vector TaskSpecification::NewActorHandles() const { } std::vector TaskSpecification::DynamicWorkerOptions() const { - RAY_CHECK(IsActorCreationTask()) - << "This method only can be called for actor creating tasks."; auto message = flatbuffers::GetRoot(spec_.data()); return string_vec_from_flatbuf(*message->dynamic_worker_options()); } diff --git a/src/ray/raylet/worker_pool_test.cc b/src/ray/raylet/worker_pool_test.cc index 966f126a23a3..dc98f31048bb 100644 --- a/src/ray/raylet/worker_pool_test.cc +++ b/src/ray/raylet/worker_pool_test.cc @@ -45,7 +45,6 @@ class WorkerPoolMock : public WorkerPool { pid_t LastStartedWorkerProcess() const { return last_worker_pid_; } const std::vector &GetWorkerCommand(int pid) { - RAY_CHECK(worker_commands_by_pid.count(pid) != 0) << "Invalid argument."; return worker_commands_by_pid[pid]; } From c165a382190e70aa59c84d773631b4854040a1cd Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Wed, 19 Jun 2019 20:30:21 +0800 Subject: [PATCH 21/46] Fix --- src/ray/raylet/worker_pool.cc | 3 ++- src/ray/raylet/worker_pool_test.cc | 10 +++++----- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 3d6df00b02e3..d717066d41d4 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -133,6 +133,7 @@ void WorkerPool::StartWorkerProcess(const Language &language, if (!dynamic_worker_options.empty()) { RAY_CHECK(dynamic_option_index < dynamic_worker_options.size()); worker_command_args.push_back(dynamic_worker_options[dynamic_option_index].c_str()); + ++dynamic_option_index; } } else { worker_command_args.push_back(token.c_str()); @@ -239,7 +240,7 @@ void WorkerPool::PushWorker(const std::shared_ptr &worker) { state.starting_dedicated_worker_processes.erase(it); state.idle_dedicated_workers[task_id] = std::move(worker); - // Return to not put this worker to idle pool. + // Return to do not put this worker to idle pool. return; } diff --git a/src/ray/raylet/worker_pool_test.cc b/src/ray/raylet/worker_pool_test.cc index dc98f31048bb..81460fbc36c0 100644 --- a/src/ray/raylet/worker_pool_test.cc +++ b/src/ray/raylet/worker_pool_test.cc @@ -213,23 +213,23 @@ TEST_F(WorkerPoolTest, PopWorkersOfMultipleLanguages) { ASSERT_NE(worker_pool_.PopWorker(java_task_spec), nullptr); } -TEST_F(WorkerPoolTest, StartWorkerWithPrefixAndSuffix) { +TEST_F(WorkerPoolTest, StartWorkerWithDynamicOptionsCommand) { const std::vector java_worker_command = { - kPrefixPlaceholder, "dummy_java_worker_command", kSuffixPlaceholder}; + "RAY_WORKER_OPTION_0", "dummy_java_worker_command", "RAY_WORKER_OPTION_1"}; SetWorkerCommands({{Language::PYTHON, {"dummy_py_worker_command"}}, {Language::JAVA, java_worker_command}}); TaskSpecification task_spec(DriverID::Nil(), TaskID::Nil(), 0, ActorID::FromRandom(), ObjectID::Nil(), 0, ActorID::Nil(), ActorHandleID::Nil(), 0, {}, {}, 0, {}, {}, Language::JAVA, {"", "", ""}, - "test_prefix", "test_suffix"); + {"test_op_0", "test_op_1"}); worker_pool_.StartWorkerProcess(Language::JAVA, &task_spec); const auto real_command = worker_pool_.GetWorkerCommand(worker_pool_.LastStartedWorkerProcess()); ASSERT_EQ(3, real_command.size()); - ASSERT_EQ("test_prefix", real_command[0]); + ASSERT_EQ("test_op_0", real_command[0]); ASSERT_EQ("dummy_java_worker_command", real_command[1]); - ASSERT_EQ("test_suffix", real_command[2]); + ASSERT_EQ("test_op_1", real_command[2]); } } // namespace raylet From fcd2ea721d71a0c1376b0385385201cb7eee9e72 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Wed, 19 Jun 2019 21:24:28 +0800 Subject: [PATCH 22/46] Fix --- .../main/java/org/ray/runtime/raylet/RayletClientImpl.java | 2 -- python/ray/ray_constants.py | 7 ------- 2 files changed, 9 deletions(-) diff --git a/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java b/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java index 7d0ecb2b1646..c369e6f2cab8 100644 --- a/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java +++ b/java/runtime/src/main/java/org/ray/runtime/raylet/RayletClientImpl.java @@ -282,8 +282,6 @@ private static ByteBuffer convertTaskSpecToFlatbuffer(TaskSpec task) { functionDescriptorOffset = fbb.createVectorOfTables(functionDescriptorOffsets); } - // TODO(qwang): - int [] dynamicWorkerOptionsOffsets = new int[task.dynamicWorkerOptions.size()]; for (int index = 0; index < task.dynamicWorkerOptions.size(); ++index) { dynamicWorkerOptionsOffsets[index] = fbb.createString(task.dynamicWorkerOptions.get(index)); diff --git a/python/ray/ray_constants.py b/python/ray/ray_constants.py index 63d3c6313419..6fd7d5b643e6 100644 --- a/python/ray/ray_constants.py +++ b/python/ray/ray_constants.py @@ -125,10 +125,3 @@ def env_integer(key, default): # A constant used as object metadata to indicate the object is raw binary. RAW_BUFFER_METADATA = b"RAW" - -# These should have the same value as `kPrefixPlaceholder` -# and `kSuffixPlaceholder` in `constants.h` -# A constant used as a placeholder of the prefix for worker command. -PREFIX_PLACEHOLDER = "__PREFIX_PLACEHOLDER__" -# A constant used as a placeholder of the suffix for worker command. -SUFFIX_PLACEHOLDER = "__SUFFIX_PLACEHOLDER__" From 54fa36c6b260fd93bb90d4a3ce6481095d0a494e Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Thu, 20 Jun 2019 10:44:48 +0800 Subject: [PATCH 23/46] Fix linting --- src/ray/raylet/node_manager.cc | 3 ++- src/ray/raylet/worker_pool.cc | 6 +++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 7a154ebba5e1..dcfbdb316041 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1723,7 +1723,8 @@ bool NodeManager::AssignTask(const Task &task) { std::shared_ptr worker = worker_pool_.PopWorker(spec); if (worker == nullptr) { // There are no workers that can execute this task. - const bool should_start_new = spec.IsActorCreationTask() && !spec.DynamicWorkerOptions().empty(); + const bool should_start_new = + spec.IsActorCreationTask() && !spec.DynamicWorkerOptions().empty(); if (should_start_new) { if (!worker_pool_.PendingRegistrationForTask(spec.GetLanguage(), spec.TaskId())) { worker_pool_.StartWorkerProcess(spec.GetLanguage(), &spec); diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index d717066d41d4..134d96bb66c3 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -132,7 +132,8 @@ void WorkerPool::StartWorkerProcess(const Language &language, if (token == option_placeholder) { if (!dynamic_worker_options.empty()) { RAY_CHECK(dynamic_option_index < dynamic_worker_options.size()); - worker_command_args.push_back(dynamic_worker_options[dynamic_option_index].c_str()); + worker_command_args.push_back( + dynamic_worker_options[dynamic_option_index].c_str()); ++dynamic_option_index; } } else { @@ -266,8 +267,7 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec worker = std::move(*state.idle.begin()); state.idle.erase(state.idle.begin()); } else { - RAY_LOG(DEBUG) << "There is no idle worker for the task " - << task_spec.TaskId() << " ."; + RAY_LOG(DEBUG) << "There is no idle worker for the task " << task_spec.TaskId(); } } else { // Dynamic worker options is not empty. Try to pop it from idle dedicated pool. From 34fb3b87561e716a3273b1ea4704aaad3b284e24 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Thu, 20 Jun 2019 15:59:10 +0800 Subject: [PATCH 24/46] Fix lint --- python/ray/services.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/ray/services.py b/python/ray/services.py index 5a85058796a8..14e13620eea2 100644 --- a/python/ray/services.py +++ b/python/ray/services.py @@ -1255,7 +1255,8 @@ def build_java_worker_command( # above options. command += java_worker_options + " " - command += "RAY_WORKER_OPTION_0 org.ray.runtime.runner.worker.DefaultWorker" + command += "RAY_WORKER_OPTION_0 " + command += "org.ray.runtime.runner.worker.DefaultWorker" return command From 34ab434a6b692ed9b10d642cd82d998a2331ad7c Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Thu, 20 Jun 2019 16:05:02 +0800 Subject: [PATCH 25/46] Address comments --- .../src/main/java/org/ray/runtime/AbstractRayRuntime.java | 2 +- .../src/main/java/org/ray/runtime/task/TaskSpec.java | 7 +------ src/ray/common/constants.h | 2 +- src/ray/raylet/worker_pool.cc | 2 +- 4 files changed, 4 insertions(+), 9 deletions(-) diff --git a/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java b/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java index 030f77271e34..4c559a499839 100644 --- a/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java +++ b/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java @@ -364,7 +364,7 @@ private TaskSpec createTaskSpec(RayFunc func, PyFunctionDescriptor pyFunctionDes } int maxActorReconstruction = 0; - List dynamicWorkerOptions = null; + List dynamicWorkerOptions = new ArrayList<>(); if (taskOptions instanceof ActorCreationOptions) { maxActorReconstruction = ((ActorCreationOptions) taskOptions).maxReconstructions; String jvmOptions = ((ActorCreationOptions) taskOptions).jvmOptions; diff --git a/java/runtime/src/main/java/org/ray/runtime/task/TaskSpec.java b/java/runtime/src/main/java/org/ray/runtime/task/TaskSpec.java index b737cf8da45a..060ca6fff4c3 100644 --- a/java/runtime/src/main/java/org/ray/runtime/task/TaskSpec.java +++ b/java/runtime/src/main/java/org/ray/runtime/task/TaskSpec.java @@ -109,12 +109,7 @@ public TaskSpec( this.newActorHandles = newActorHandles; this.args = args; this.numReturns = numReturns; - - if (dynamicWorkerOptions == null) { - this.dynamicWorkerOptions = new ArrayList<>(); - } else { - this.dynamicWorkerOptions =dynamicWorkerOptions; - } + this.dynamicWorkerOptions = dynamicWorkerOptions; returnIds = new ObjectId[numReturns]; for (int i = 0; i < numReturns; ++i) { diff --git a/src/ray/common/constants.h b/src/ray/common/constants.h index 88c7f4749653..1f50b8025d57 100644 --- a/src/ray/common/constants.h +++ b/src/ray/common/constants.h @@ -36,6 +36,6 @@ constexpr char kObjectTablePrefix[] = "ObjectTable"; /// Prefix for the task table keys in redis. constexpr char kTaskTablePrefix[] = "TaskTable"; -constexpr char kWorkerOptionPlaceHolderPrefix[] = "RAY_WORKER_OPTION_"; +constexpr char kWorkerDynamicOptionPlaceholderPrefix[] = "RAY_WORKER_OPTION_"; #endif // RAY_CONSTANTS_H_ diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 134d96bb66c3..94f8219f4af7 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -127,7 +127,7 @@ void WorkerPool::StartWorkerProcess(const Language &language, size_t dynamic_option_index = 0; for (auto const &token : state.worker_command) { const auto option_placeholder = - kWorkerOptionPlaceHolderPrefix + std::to_string(dynamic_option_index); + kWorkerDynamicOptionPlaceholderPrefix + std::to_string(dynamic_option_index); if (token == option_placeholder) { if (!dynamic_worker_options.empty()) { From 8948f9deb1f14c0570b1453a7b3252075865850e Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Thu, 20 Jun 2019 19:48:53 +0800 Subject: [PATCH 26/46] WIP --- src/ray/raylet/node_manager.cc | 24 ++++---------- src/ray/raylet/worker_pool.cc | 58 ++++++++++++++++++++++------------ src/ray/raylet/worker_pool.h | 6 ++++ 3 files changed, 49 insertions(+), 39 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index dcfbdb316041..315ec43aca17 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1723,24 +1723,6 @@ bool NodeManager::AssignTask(const Task &task) { std::shared_ptr worker = worker_pool_.PopWorker(spec); if (worker == nullptr) { // There are no workers that can execute this task. - const bool should_start_new = - spec.IsActorCreationTask() && !spec.DynamicWorkerOptions().empty(); - if (should_start_new) { - if (!worker_pool_.PendingRegistrationForTask(spec.GetLanguage(), spec.TaskId())) { - worker_pool_.StartWorkerProcess(spec.GetLanguage(), &spec); - } - } else if (!spec.IsActorTask()) { - // There are no more non-actor workers available to execute this task. - // Start a new worker. - worker_pool_.StartWorkerProcess(spec.GetLanguage()); - // Push an error message to the user if the worker pool tells us that it is - // getting too big. - const std::string warning_message = worker_pool_.WarningAboutSize(); - if (warning_message != "") { - RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( - DriverID::Nil(), "worker_pool_large", warning_message, current_time_ms())); - } - } // We couldn't assign this task, as no worker available. return false; } @@ -2211,6 +2193,12 @@ void NodeManager::ForwardTask( const auto &spec = task.GetTaskSpecification(); auto task_id = spec.TaskId(); + if (worker_pool_.HasWorkerForTask(task_id)) { + RAY_LOG(INFO) << "There is a worker being starting for this task," + << "so we shouldn't forward this task to another node."; + return ; + } + // Get and serialize the task's unforwarded, uncommitted lineage. Lineage uncommitted_lineage; if (lineage_cache_.ContainsTask(task_id)) { diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 94f8219f4af7..44243e1d7f37 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -158,7 +158,6 @@ void WorkerPool::StartWorkerProcess(const Language &language, "because we specified dynamic worker options for this task."; state.starting_dedicated_worker_processes[pid] = task_spec->TaskId(); } - return; } } @@ -258,39 +257,45 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec const auto &actor_id = task_spec.ActorId(); std::shared_ptr worker = nullptr; - if (task_spec.IsActorCreationTask()) { - // code path of actor creation task. - if (task_spec.DynamicWorkerOptions().empty()) { - // There is no dynamic worker options of this task. Pop a worker - // from idle worker pool of non-actor. - if (!state.idle.empty()) { - worker = std::move(*state.idle.begin()); - state.idle.erase(state.idle.begin()); - } else { - RAY_LOG(DEBUG) << "There is no idle worker for the task " << task_spec.TaskId(); - } - } else { - // Dynamic worker options is not empty. Try to pop it from idle dedicated pool. - auto it = state.idle_dedicated_workers.find(task_spec.TaskId()); - if (it != state.idle_dedicated_workers.end()) { - worker = std::move(it->second); - state.idle_dedicated_workers.erase(it); - } + if (task_spec.IsActorCreationTask() && !task_spec.DynamicWorkerOptions().empty()) { + // Code path of actor creation task with dynamic worker options. + // Try to pop it from idle dedicated pool. + auto it = state.idle_dedicated_workers.find(task_spec.TaskId()); + if (it != state.idle_dedicated_workers.end()) { + // There is a idle dedicated worker for this task. + worker = std::move(it->second); + state.idle_dedicated_workers.erase(it); + } else if (!PendingRegistrationForTask(task_spec.GetLanguage(), task_spec.TaskId())) { + // We are not pending a registration from a worker for this task, + // so start a new worker process for this task. + StartWorkerProcess(task_spec.Getlanguage(), &task_spec); } } else if (!task_spec.IsActorTask()) { - // code path of normal task. + // Code path of normal task or actor creation task without dynamic worker options. if (!state.idle.empty()) { worker = std::move(*state.idle.begin()); state.idle.erase(state.idle.begin()); + } else { + // There are no more non-actor workers available to execute this task. + // Start a new worker process. + StartWorkerProcess(spec.GetLanguage()); + // Push an error message to the user if the worker pool tells us that it is + // getting too big. + const std::string warning_message = worker_pool_.WarningAboutSize(); + if (warning_message != "") { + RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( + DriverID::Nil(), "worker_pool_large", warning_message, current_time_ms())); + } } } else { - // code path of actor task. + // Code path of actor task. auto actor_entry = state.idle_actor.find(actor_id); if (actor_entry != state.idle_actor.end()) { worker = std::move(actor_entry->second); state.idle_actor.erase(actor_entry); } } + return worker; } @@ -334,6 +339,17 @@ std::vector> WorkerPool::GetWorkersRunningTasksForDriver return workers; } +bool WorkerPool::HasWorkerForTask(const TaskID &task_id) const { + for (const auto &it : starting_dedicated_worker_processes) { + if (it.second == task_id) { + return true; + } + } + + auto it = idle_dedicated_workers.find(task_id); + return (it != idle_dedicated_workers.end()); +} + std::string WorkerPool::WarningAboutSize() { int64_t num_workers_started_or_registered = 0; for (const auto &entry : states_by_lang_) { diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index e459644c4bb3..863b95dc5a38 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -120,6 +120,12 @@ class WorkerPool { std::vector> GetWorkersRunningTasksForDriver( const DriverID &driver_id) const; + /// Whether we're starting a worker for this task. + /// + /// \param task_id The task that we want to query. + /// \return True if there is a worker being starting, otherwise false. + bool HasWorkerForTask(const TaskID &task_id) const; + /// Whether we are pending a registration from a worker for the given task. /// Note that, this is only used for actor creation task. /// From d5657a3177552cb39ccc33e2b49a7b54e0513bf6 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Thu, 20 Jun 2019 20:25:31 +0800 Subject: [PATCH 27/46] Fix --- src/ray/raylet/node_manager.cc | 10 +++++++++- src/ray/raylet/worker_pool.cc | 22 +++++++++------------- src/ray/raylet/worker_pool.h | 3 ++- 3 files changed, 20 insertions(+), 15 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 315ec43aca17..bf1d83add19b 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1722,6 +1722,14 @@ bool NodeManager::AssignTask(const Task &task) { // Try to get an idle worker that can execute this task. std::shared_ptr worker = worker_pool_.PopWorker(spec); if (worker == nullptr) { + // Push an error message to the user if the worker pool tells us that it is + // getting too big. + const std::string warning_message = worker_pool_.WarningAboutSize(); + if (warning_message != "") { + RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( + DriverID::Nil(), "worker_pool_large", warning_message, current_time_ms())); + } + // There are no workers that can execute this task. // We couldn't assign this task, as no worker available. return false; @@ -2193,7 +2201,7 @@ void NodeManager::ForwardTask( const auto &spec = task.GetTaskSpecification(); auto task_id = spec.TaskId(); - if (worker_pool_.HasWorkerForTask(task_id)) { + if (worker_pool_.HasWorkerForTask(spec.GetLanguage(), task_id)) { RAY_LOG(INFO) << "There is a worker being starting for this task," << "so we shouldn't forward this task to another node."; return ; diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 44243e1d7f37..5f8f6ed37473 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -10,6 +10,7 @@ #include "ray/common/status.h" #include "ray/stats/stats.h" #include "ray/util/logging.h" +#include "ray/util/util.h" namespace { @@ -158,6 +159,7 @@ void WorkerPool::StartWorkerProcess(const Language &language, "because we specified dynamic worker options for this task."; state.starting_dedicated_worker_processes[pid] = task_spec->TaskId(); } + return; } } @@ -268,7 +270,7 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec } else if (!PendingRegistrationForTask(task_spec.GetLanguage(), task_spec.TaskId())) { // We are not pending a registration from a worker for this task, // so start a new worker process for this task. - StartWorkerProcess(task_spec.Getlanguage(), &task_spec); + StartWorkerProcess(task_spec.GetLanguage(), &task_spec); } } else if (!task_spec.IsActorTask()) { // Code path of normal task or actor creation task without dynamic worker options. @@ -278,14 +280,7 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec } else { // There are no more non-actor workers available to execute this task. // Start a new worker process. - StartWorkerProcess(spec.GetLanguage()); - // Push an error message to the user if the worker pool tells us that it is - // getting too big. - const std::string warning_message = worker_pool_.WarningAboutSize(); - if (warning_message != "") { - RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( - DriverID::Nil(), "worker_pool_large", warning_message, current_time_ms())); - } + StartWorkerProcess(task_spec.GetLanguage()); } } else { // Code path of actor task. @@ -339,15 +334,16 @@ std::vector> WorkerPool::GetWorkersRunningTasksForDriver return workers; } -bool WorkerPool::HasWorkerForTask(const TaskID &task_id) const { - for (const auto &it : starting_dedicated_worker_processes) { +bool WorkerPool::HasWorkerForTask(const Language &language, const TaskID &task_id) { + auto &state = GetStateForLanguage(language); + for (const auto &it : state.starting_dedicated_worker_processes) { if (it.second == task_id) { return true; } } - auto it = idle_dedicated_workers.find(task_id); - return (it != idle_dedicated_workers.end()); + auto it = state.idle_dedicated_workers.find(task_id); + return it != state.idle_dedicated_workers.end(); } std::string WorkerPool::WarningAboutSize() { diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index 863b95dc5a38..de721d0dce97 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -122,9 +122,10 @@ class WorkerPool { /// Whether we're starting a worker for this task. /// + /// \param language The required language. /// \param task_id The task that we want to query. /// \return True if there is a worker being starting, otherwise false. - bool HasWorkerForTask(const TaskID &task_id) const; + bool HasWorkerForTask(const Language &language, const TaskID &task_id); /// Whether we are pending a registration from a worker for the given task. /// Note that, this is only used for actor creation task. From b8117b64285c0e96d2e5f8212b09c6f8d30a7ceb Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Thu, 20 Jun 2019 20:27:22 +0800 Subject: [PATCH 28/46] Fix --- src/ray/raylet/worker_pool.cc | 8 +++---- src/ray/raylet/worker_pool.h | 39 ++++++++++++++++++----------------- 2 files changed, 23 insertions(+), 24 deletions(-) diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 5f8f6ed37473..ce1df9b4a80c 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -335,13 +335,11 @@ std::vector> WorkerPool::GetWorkersRunningTasksForDriver } bool WorkerPool::HasWorkerForTask(const Language &language, const TaskID &task_id) { - auto &state = GetStateForLanguage(language); - for (const auto &it : state.starting_dedicated_worker_processes) { - if (it.second == task_id) { - return true; - } + if (PendingRegistrationForTask(language, task_id)) { + return true; } + auto &state = GetStateForLanguage(language); auto it = state.idle_dedicated_workers.find(task_id); return it != state.idle_dedicated_workers.end(); } diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index de721d0dce97..b4d26a5bbe88 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -43,18 +43,6 @@ class WorkerPool { /// Destructor responsible for freeing a set of workers owned by this class. virtual ~WorkerPool(); - /// Asynchronously start a new worker process. Once the worker process has - /// registered with an external server, the process should create and - /// register num_workers_per_process_ workers, then add them to the pool. - /// Failure to start the worker process is a fatal error. If too many workers - /// are already being started, then this function will return without starting - /// any workers. - /// - /// \param language Which language this worker process should be. - /// \param task_spec The task specification that we provide for get more information. - void StartWorkerProcess(const Language &language, - const TaskSpecification *task_spec = nullptr); - /// Register a new worker. The Worker should be added by the caller to the /// pool after it becomes idle (e.g., requests a work assignment). /// @@ -127,13 +115,6 @@ class WorkerPool { /// \return True if there is a worker being starting, otherwise false. bool HasWorkerForTask(const Language &language, const TaskID &task_id); - /// Whether we are pending a registration from a worker for the given task. - /// Note that, this is only used for actor creation task. - /// - /// \param language The required language. - /// \param task_id The task that we'll query by. - bool PendingRegistrationForTask(const Language &language, const TaskID &task_id); - /// Returns debug string for class. /// /// \return string. @@ -190,6 +171,26 @@ class WorkerPool { /// for a given language. State &GetStateForLanguage(const Language &language); + /// Whether we are pending a registration from a worker for the given task. + /// Note that, this is only used for actor creation task. + /// + /// \param language The required language. + /// \param task_id The task that we'll query by. + bool PendingRegistrationForTask(const Language &language, const TaskID &task_id); + + /// Asynchronously start a new worker process. Once the worker process has + /// registered with an external server, the process should create and + /// register num_workers_per_process_ workers, then add them to the pool. + /// Failure to start the worker process is a fatal error. If too many workers + /// are already being started, then this function will return without starting + /// any workers. + /// + /// \param language Which language this worker process should be. + /// \param task_spec The task specification that we provide for get more information. + void StartWorkerProcess(const Language &language, + const TaskSpecification *task_spec = nullptr); + + private: /// We'll push a warning to the user every time a multiple of this many /// workers has been started. int multiple_for_warning_; From 68b900250069cca3b404044146c356781e36506c Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Thu, 20 Jun 2019 20:35:40 +0800 Subject: [PATCH 29/46] minor refine --- src/ray/raylet/node_manager.cc | 4 ++-- src/ray/raylet/worker_pool.cc | 2 +- src/ray/raylet/worker_pool.h | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index bf1d83add19b..5d7eebacdd54 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -2202,8 +2202,8 @@ void NodeManager::ForwardTask( auto task_id = spec.TaskId(); if (worker_pool_.HasWorkerForTask(spec.GetLanguage(), task_id)) { - RAY_LOG(INFO) << "There is a worker being starting for this task," - << "so we shouldn't forward this task to another node."; + RAY_LOG(DEBUG) << "There is a worker being starting for this task," + << "so we shouldn't forward this task to another node."; return ; } diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index ce1df9b4a80c..a42dc1871eb7 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -264,7 +264,7 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec // Try to pop it from idle dedicated pool. auto it = state.idle_dedicated_workers.find(task_spec.TaskId()); if (it != state.idle_dedicated_workers.end()) { - // There is a idle dedicated worker for this task. + // There is an idle dedicated worker for this task. worker = std::move(it->second); state.idle_dedicated_workers.erase(it); } else if (!PendingRegistrationForTask(task_spec.GetLanguage(), task_spec.TaskId())) { diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index b4d26a5bbe88..54e4580fed80 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -175,7 +175,7 @@ class WorkerPool { /// Note that, this is only used for actor creation task. /// /// \param language The required language. - /// \param task_id The task that we'll query by. + /// \param task_id The task that we want to query. bool PendingRegistrationForTask(const Language &language, const TaskID &task_id); /// Asynchronously start a new worker process. Once the worker process has From c096b2a6f3a831862ec10f33e9bbdb047f09cea1 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Thu, 20 Jun 2019 20:38:10 +0800 Subject: [PATCH 30/46] Fix lint --- src/ray/raylet/node_manager.cc | 4 ++-- src/ray/raylet/worker_pool.cc | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 5d7eebacdd54..39eecd7b9cae 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -1727,7 +1727,7 @@ bool NodeManager::AssignTask(const Task &task) { const std::string warning_message = worker_pool_.WarningAboutSize(); if (warning_message != "") { RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( - DriverID::Nil(), "worker_pool_large", warning_message, current_time_ms())); + DriverID::Nil(), "worker_pool_large", warning_message, current_time_ms())); } // There are no workers that can execute this task. @@ -2204,7 +2204,7 @@ void NodeManager::ForwardTask( if (worker_pool_.HasWorkerForTask(spec.GetLanguage(), task_id)) { RAY_LOG(DEBUG) << "There is a worker being starting for this task," << "so we shouldn't forward this task to another node."; - return ; + return; } // Get and serialize the task's unforwarded, uncommitted lineage. diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index a42dc1871eb7..1906e9fecdac 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -128,7 +128,7 @@ void WorkerPool::StartWorkerProcess(const Language &language, size_t dynamic_option_index = 0; for (auto const &token : state.worker_command) { const auto option_placeholder = - kWorkerDynamicOptionPlaceholderPrefix + std::to_string(dynamic_option_index); + kWorkerDynamicOptionPlaceholderPrefix + std::to_string(dynamic_option_index); if (token == option_placeholder) { if (!dynamic_worker_options.empty()) { From 7755912bedb369909dc0beb8291820a7c357d3a5 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Thu, 20 Jun 2019 20:48:49 +0800 Subject: [PATCH 31/46] Fix raylet test. --- src/ray/raylet/worker_pool.h | 24 ++++++++++++------------ src/ray/raylet/worker_pool_test.cc | 5 +++++ 2 files changed, 17 insertions(+), 12 deletions(-) diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index 54e4580fed80..4e205e833b20 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -131,6 +131,18 @@ class WorkerPool { std::string WarningAboutSize(); protected: + /// Asynchronously start a new worker process. Once the worker process has + /// registered with an external server, the process should create and + /// register num_workers_per_process_ workers, then add them to the pool. + /// Failure to start the worker process is a fatal error. If too many workers + /// are already being started, then this function will return without starting + /// any workers. + /// + /// \param language Which language this worker process should be. + /// \param task_spec The task specification that we provide for get more information. + void StartWorkerProcess(const Language &language, + const TaskSpecification *task_spec = nullptr); + /// The implementation of how to start a new worker process with command arguments. /// /// \param worker_command_args The command arguments of new worker process. @@ -178,18 +190,6 @@ class WorkerPool { /// \param task_id The task that we want to query. bool PendingRegistrationForTask(const Language &language, const TaskID &task_id); - /// Asynchronously start a new worker process. Once the worker process has - /// registered with an external server, the process should create and - /// register num_workers_per_process_ workers, then add them to the pool. - /// Failure to start the worker process is a fatal error. If too many workers - /// are already being started, then this function will return without starting - /// any workers. - /// - /// \param language Which language this worker process should be. - /// \param task_spec The task specification that we provide for get more information. - void StartWorkerProcess(const Language &language, - const TaskSpecification *task_spec = nullptr); - private: /// We'll push a warning to the user every time a multiple of this many /// workers has been started. diff --git a/src/ray/raylet/worker_pool_test.cc b/src/ray/raylet/worker_pool_test.cc index 81460fbc36c0..6c7476a660a4 100644 --- a/src/ray/raylet/worker_pool_test.cc +++ b/src/ray/raylet/worker_pool_test.cc @@ -29,6 +29,11 @@ class WorkerPoolMock : public WorkerPool { states_by_lang_.clear(); } + void StartWorkerProcess(const Language &language, + const TaskSpecification *task_spec = nullptr) { + WorkerPool::StartWorkerProcess(language, task_spec); + } + pid_t StartProcess(const std::vector &worker_command_args) override { last_worker_pid_ += 1; std::vector local_worker_commands_args; From 3de7d6a3c864974ecfa5425f20d87c182163d05f Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Fri, 21 Jun 2019 09:56:24 +0800 Subject: [PATCH 32/46] Fix lint --- src/ray/raylet/worker_pool_test.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/ray/raylet/worker_pool_test.cc b/src/ray/raylet/worker_pool_test.cc index 6c7476a660a4..1e4067032fbe 100644 --- a/src/ray/raylet/worker_pool_test.cc +++ b/src/ray/raylet/worker_pool_test.cc @@ -30,7 +30,7 @@ class WorkerPoolMock : public WorkerPool { } void StartWorkerProcess(const Language &language, - const TaskSpecification *task_spec = nullptr) { + const TaskSpecification *task_spec = nullptr) { WorkerPool::StartWorkerProcess(language, task_spec); } From ce94f8197762fa67b6b48a870e6c442a91ec00a0 Mon Sep 17 00:00:00 2001 From: Qing Wang Date: Fri, 21 Jun 2019 15:36:00 +0800 Subject: [PATCH 33/46] Update src/ray/raylet/worker_pool.h Co-Authored-By: Hao Chen --- src/ray/raylet/worker_pool.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index 4e205e833b20..4abaed975113 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -183,7 +183,7 @@ class WorkerPool { /// for a given language. State &GetStateForLanguage(const Language &language); - /// Whether we are pending a registration from a worker for the given task. + /// Whether there is a pending registration from a worker for the given task. /// Note that, this is only used for actor creation task. /// /// \param language The required language. From c10777d058f8e524dd148f08a9c81b15ba4b8f18 Mon Sep 17 00:00:00 2001 From: Qing Wang Date: Fri, 21 Jun 2019 15:36:17 +0800 Subject: [PATCH 34/46] Update java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java Co-Authored-By: Hao Chen --- .../src/main/java/org/ray/runtime/AbstractRayRuntime.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java b/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java index 4c559a499839..26a8d6e541ba 100644 --- a/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java +++ b/java/runtime/src/main/java/org/ray/runtime/AbstractRayRuntime.java @@ -364,7 +364,7 @@ private TaskSpec createTaskSpec(RayFunc func, PyFunctionDescriptor pyFunctionDes } int maxActorReconstruction = 0; - List dynamicWorkerOptions = new ArrayList<>(); + List dynamicWorkerOptions = ImmutableList.of(); if (taskOptions instanceof ActorCreationOptions) { maxActorReconstruction = ((ActorCreationOptions) taskOptions).maxReconstructions; String jvmOptions = ((ActorCreationOptions) taskOptions).jvmOptions; From 01794c3b1b7fb2498154777a04221829c8f17030 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Fri, 21 Jun 2019 16:35:20 +0800 Subject: [PATCH 35/46] Address comments. --- src/ray/raylet/node_manager.cc | 10 +------ src/ray/raylet/worker_pool.cc | 54 ++++++++++++++++++---------------- src/ray/raylet/worker_pool.h | 29 ++++++++++-------- 3 files changed, 46 insertions(+), 47 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 39eecd7b9cae..bc4f5893b4cf 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -83,7 +83,7 @@ NodeManager::NodeManager(boost::asio::io_service &io_service, initial_config_(config), local_available_resources_(config.resource_config), worker_pool_(config.num_initial_workers, config.num_workers_per_process, - config.maximum_startup_concurrency, config.worker_commands), + config.maximum_startup_concurrency, gcs_client_, config.worker_commands), scheduling_policy_(local_queues_), reconstruction_policy_( io_service_, @@ -1722,14 +1722,6 @@ bool NodeManager::AssignTask(const Task &task) { // Try to get an idle worker that can execute this task. std::shared_ptr worker = worker_pool_.PopWorker(spec); if (worker == nullptr) { - // Push an error message to the user if the worker pool tells us that it is - // getting too big. - const std::string warning_message = worker_pool_.WarningAboutSize(); - if (warning_message != "") { - RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( - DriverID::Nil(), "worker_pool_large", warning_message, current_time_ms())); - } - // There are no workers that can execute this task. // We couldn't assign this task, as no worker available. return false; diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 1906e9fecdac..b9b862e505e2 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -43,11 +43,12 @@ namespace raylet { /// (num_worker_processes * num_workers_per_process) workers for each language. WorkerPool::WorkerPool( int num_worker_processes, int num_workers_per_process, - int maximum_startup_concurrency, + int maximum_startup_concurrency, std::shared_ptr gcs_client, const std::unordered_map> &worker_commands) : num_workers_per_process_(num_workers_per_process), multiple_for_warning_(std::max(num_worker_processes, maximum_startup_concurrency)), maximum_startup_concurrency_(maximum_startup_concurrency), + gcs_client_(std::move(gcs_client)), last_warning_multiple_(0) { RAY_CHECK(num_workers_per_process > 0) << "num_workers_per_process must be positive."; RAY_CHECK(maximum_startup_concurrency > 0); @@ -100,8 +101,8 @@ uint32_t WorkerPool::Size(const Language &language) const { } } -void WorkerPool::StartWorkerProcess(const Language &language, - const TaskSpecification *task_spec) { +int WorkerPool::StartWorkerProcess(const Language &language, + const std::vector dynamic_options) { auto &state = GetStateForLanguage(language); // If we are already starting up too many workers, then return without starting // more. @@ -111,18 +112,13 @@ void WorkerPool::StartWorkerProcess(const Language &language, RAY_LOG(DEBUG) << "Worker not started, " << state.starting_worker_processes.size() << " worker processes of language type " << static_cast(language) << " pending registration"; - return; + return -1; } // Either there are no workers pending registration or the worker start is being forced. RAY_LOG(DEBUG) << "Starting new worker process, current pool has " << state.idle_actor.size() << " actor workers, and " << state.idle.size() << " non-actor workers"; - std::vector dynamic_worker_options; - if (task_spec != nullptr && task_spec->IsActorCreationTask()) { - dynamic_worker_options = task_spec->DynamicWorkerOptions(); - } - // Extract pointers from the worker command to pass into execvp. std::vector worker_command_args; size_t dynamic_option_index = 0; @@ -131,10 +127,9 @@ void WorkerPool::StartWorkerProcess(const Language &language, kWorkerDynamicOptionPlaceholderPrefix + std::to_string(dynamic_option_index); if (token == option_placeholder) { - if (!dynamic_worker_options.empty()) { - RAY_CHECK(dynamic_option_index < dynamic_worker_options.size()); - worker_command_args.push_back( - dynamic_worker_options[dynamic_option_index].c_str()); + if (!dynamic_options.empty()) { + RAY_CHECK(dynamic_option_index < dynamic_options.size()); + worker_command_args.push_back(dynamic_options[dynamic_option_index].c_str()); ++dynamic_option_index; } } else { @@ -147,19 +142,13 @@ void WorkerPool::StartWorkerProcess(const Language &language, if (pid < 0) { // Failure case. RAY_LOG(FATAL) << "Failed to fork worker process: " << strerror(errno); - return; + return -1; } else if (pid > 0) { // Parent process case. RAY_LOG(DEBUG) << "Started worker process with pid " << pid; state.starting_worker_processes.emplace( std::make_pair(pid, num_workers_per_process_)); - if (!dynamic_worker_options.empty()) { - RAY_CHECK(task_spec != nullptr) - << "task_spec should not be nullptr " - "because we specified dynamic worker options for this task."; - state.starting_dedicated_worker_processes[pid] = task_spec->TaskId(); - } - return; + return pid; } } @@ -267,10 +256,13 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec // There is an idle dedicated worker for this task. worker = std::move(it->second); state.idle_dedicated_workers.erase(it); - } else if (!PendingRegistrationForTask(task_spec.GetLanguage(), task_spec.TaskId())) { + } else if (!HasPendingRegistrationForTask(task_spec.GetLanguage(), task_spec.TaskId())) { // We are not pending a registration from a worker for this task, // so start a new worker process for this task. - StartWorkerProcess(task_spec.GetLanguage(), &task_spec); + auto pid = StartWorkerProcess(task_spec.GetLanguage(), task_spec.DynamicWorkerOptions()); + if (pid > 0) { + state.starting_dedicated_worker_processes[pid] = task_spec.TaskId(); + } } } else if (!task_spec.IsActorTask()) { // Code path of normal task or actor creation task without dynamic worker options. @@ -291,6 +283,16 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec } } + if (worker == nullptr) { + // Push an error message to the user if the worker pool tells us that it is + // getting too big. + const std::string warning_message = WarningAboutSize(); + if (warning_message != "") { + RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( + DriverID::Nil(), "worker_pool_large", warning_message, current_time_ms())); + } + } + return worker; } @@ -335,7 +337,7 @@ std::vector> WorkerPool::GetWorkersRunningTasksForDriver } bool WorkerPool::HasWorkerForTask(const Language &language, const TaskID &task_id) { - if (PendingRegistrationForTask(language, task_id)) { + if (HasPendingRegistrationForTask(language, task_id)) { return true; } @@ -366,8 +368,8 @@ std::string WorkerPool::WarningAboutSize() { return warning_message.str(); } -bool WorkerPool::PendingRegistrationForTask(const Language &language, - const TaskID &task_id) { +bool WorkerPool::HasPendingRegistrationForTask(const Language &language, + const TaskID &task_id) { auto &state = GetStateForLanguage(language); for (const auto &item : state.starting_dedicated_worker_processes) { if (item.second == task_id) { diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index 4abaed975113..7abd07d20cf4 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -10,6 +10,7 @@ #include "ray/gcs/format/util.h" #include "ray/raylet/task.h" #include "ray/raylet/worker.h" +#include "ray/gcs/client.h" namespace ray { @@ -37,7 +38,7 @@ class WorkerPool { /// language. WorkerPool( int num_worker_processes, int num_workers_per_process, - int maximum_startup_concurrency, + int maximum_startup_concurrency, std::shared_ptr gcs_client, const std::unordered_map> &worker_commands); /// Destructor responsible for freeing a set of workers owned by this class. @@ -123,13 +124,6 @@ class WorkerPool { /// Record metrics. void RecordMetrics() const; - /// Generate a warning about the number of workers that have registered or - /// started if appropriate. - /// - /// \return An empty string if no warning should be generated and otherwise a - /// string with a warning message. - std::string WarningAboutSize(); - protected: /// Asynchronously start a new worker process. Once the worker process has /// registered with an external server, the process should create and @@ -139,9 +133,11 @@ class WorkerPool { /// any workers. /// /// \param language Which language this worker process should be. - /// \param task_spec The task specification that we provide for get more information. - void StartWorkerProcess(const Language &language, - const TaskSpecification *task_spec = nullptr); + /// \param dynamic_options The dynamic options that we should add for worker command. + /// \return The id of the process that we started if it's positive, + /// otherwise it means we didn't start a process. + int StartWorkerProcess(const Language &language, + const std::vector dynamic_options = {}); /// The implementation of how to start a new worker process with command arguments. /// @@ -188,7 +184,14 @@ class WorkerPool { /// /// \param language The required language. /// \param task_id The task that we want to query. - bool PendingRegistrationForTask(const Language &language, const TaskID &task_id); + bool HasPendingRegistrationForTask(const Language &language, const TaskID &task_id); + + /// Generate a warning about the number of workers that have registered or + /// started if appropriate. + /// + /// \return An empty string if no warning should be generated and otherwise a + /// string with a warning message. + std::string WarningAboutSize(); private: /// We'll push a warning to the user every time a multiple of this many @@ -199,6 +202,8 @@ class WorkerPool { /// The last size at which a warning about the number of registered workers /// was generated. int64_t last_warning_multiple_; + /// A client connection to the GCS. + std::shared_ptr gcs_client_; }; } // namespace raylet From 1b5ec6c1828a4dcb92336bc5979095027a01b884 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Fri, 21 Jun 2019 18:38:51 +0800 Subject: [PATCH 36/46] Address comments. --- src/ray/raylet/node_manager.cc | 2 +- src/ray/raylet/worker_pool.cc | 33 ++++++++++++--------------------- src/ray/raylet/worker_pool.h | 19 +++++++------------ 3 files changed, 20 insertions(+), 34 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index bc4f5893b4cf..ef07c19d4f39 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -2193,7 +2193,7 @@ void NodeManager::ForwardTask( const auto &spec = task.GetTaskSpecification(); auto task_id = spec.TaskId(); - if (worker_pool_.HasWorkerForTask(spec.GetLanguage(), task_id)) { + if (worker_pool_.HasPendingWorkerForTask(spec.GetLanguage(), task_id)) { RAY_LOG(DEBUG) << "There is a worker being starting for this task," << "so we shouldn't forward this task to another node."; return; diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index b9b862e505e2..9117e5b9fc05 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -224,11 +224,10 @@ void WorkerPool::PushWorker(const std::shared_ptr &worker) { << "Idle workers cannot have an assigned task ID"; auto &state = GetStateForLanguage(worker->GetLanguage()); - auto it = state.starting_dedicated_worker_processes.find(worker->Pid()); - if (it != state.starting_dedicated_worker_processes.end()) { - // The worker is used for the specific actor creation task. + auto it = state.dedicated_workers_to_tasks.find(worker->Pid()); + if (it != state.dedicated_workers_to_tasks.end()) { + // The worker is used for the specific actor creation task with dynamic options. const auto task_id = it->second; - state.starting_dedicated_worker_processes.erase(it); state.idle_dedicated_workers[task_id] = std::move(worker); // Return to do not put this worker to idle pool. @@ -256,12 +255,15 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec // There is an idle dedicated worker for this task. worker = std::move(it->second); state.idle_dedicated_workers.erase(it); - } else if (!HasPendingRegistrationForTask(task_spec.GetLanguage(), task_spec.TaskId())) { + // Because we found a worker that can perform this task, + // we can remove it from dedicated_workers_to_tasks. + state.dedicated_workers_to_tasks.erase(worker->Pid()); + } else if (!HasPendingWorkerForTask(task_spec.GetLanguage(), task_spec.TaskId())) { // We are not pending a registration from a worker for this task, // so start a new worker process for this task. auto pid = StartWorkerProcess(task_spec.GetLanguage(), task_spec.DynamicWorkerOptions()); if (pid > 0) { - state.starting_dedicated_worker_processes[pid] = task_spec.TaskId(); + state.dedicated_workers_to_tasks[pid] = task_spec.TaskId(); } } } else if (!task_spec.IsActorTask()) { @@ -336,16 +338,6 @@ std::vector> WorkerPool::GetWorkersRunningTasksForDriver return workers; } -bool WorkerPool::HasWorkerForTask(const Language &language, const TaskID &task_id) { - if (HasPendingRegistrationForTask(language, task_id)) { - return true; - } - - auto &state = GetStateForLanguage(language); - auto it = state.idle_dedicated_workers.find(task_id); - return it != state.idle_dedicated_workers.end(); -} - std::string WorkerPool::WarningAboutSize() { int64_t num_workers_started_or_registered = 0; for (const auto &entry : states_by_lang_) { @@ -368,15 +360,14 @@ std::string WorkerPool::WarningAboutSize() { return warning_message.str(); } -bool WorkerPool::HasPendingRegistrationForTask(const Language &language, - const TaskID &task_id) { +bool WorkerPool::HasPendingWorkerForTask(const Language &language, + const TaskID &task_id) { auto &state = GetStateForLanguage(language); - for (const auto &item : state.starting_dedicated_worker_processes) { - if (item.second == task_id) { + for (const auto &it : state.dedicated_workers_to_tasks) { + if (it.second == task_id) { return true; } } - return false; } diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index 7abd07d20cf4..187d470fa2e5 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -109,12 +109,14 @@ class WorkerPool { std::vector> GetWorkersRunningTasksForDriver( const DriverID &driver_id) const; - /// Whether we're starting a worker for this task. + /// Whether there is a pending worker for the given task. + /// Note that, this is only used for actor creation task with dynamic options. + /// And if the worker registered but isn't assigned a task, + /// the worker also is on pending state, and this'll return true. /// /// \param language The required language. /// \param task_id The task that we want to query. - /// \return True if there is a worker being starting, otherwise false. - bool HasWorkerForTask(const Language &language, const TaskID &task_id); + bool HasPendingWorkerForTask(const Language &language, const TaskID &task_id); /// Returns debug string for class. /// @@ -164,9 +166,9 @@ class WorkerPool { /// A map from the pids of starting worker processes /// to the number of their unregistered workers. std::unordered_map starting_worker_processes; - /// A map for looking up the task id of actor creation task by the pid of + /// A map for looking up the task with dynamic options by the pid of /// worker. Note that this is used for the dedicated worker processes. - std::unordered_map starting_dedicated_worker_processes; + std::unordered_map dedicated_workers_to_tasks; }; /// The number of workers per process. @@ -179,13 +181,6 @@ class WorkerPool { /// for a given language. State &GetStateForLanguage(const Language &language); - /// Whether there is a pending registration from a worker for the given task. - /// Note that, this is only used for actor creation task. - /// - /// \param language The required language. - /// \param task_id The task that we want to query. - bool HasPendingRegistrationForTask(const Language &language, const TaskID &task_id); - /// Generate a warning about the number of workers that have registered or /// started if appropriate. /// From 2010a4128b3dde9875fb9d5d7bbd27729e9d725f Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Fri, 21 Jun 2019 19:12:01 +0800 Subject: [PATCH 37/46] Fix test. --- src/ray/raylet/worker_pool.cc | 8 ++++++-- src/ray/raylet/worker_pool_test.cc | 14 ++++++-------- 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 9117e5b9fc05..5812c6a77e9c 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -290,8 +290,12 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec // getting too big. const std::string warning_message = WarningAboutSize(); if (warning_message != "") { - RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( - DriverID::Nil(), "worker_pool_large", warning_message, current_time_ms())); + if (gcs_client_ != nullptr) { + RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( + DriverID::Nil(), "worker_pool_large", warning_message, current_time_ms())); + } else { + RAY_LOG(WARNING) << "Failed to push error message to user since gcs client is null."; + } } } diff --git a/src/ray/raylet/worker_pool_test.cc b/src/ray/raylet/worker_pool_test.cc index 1e4067032fbe..ac50da1f8250 100644 --- a/src/ray/raylet/worker_pool_test.cc +++ b/src/ray/raylet/worker_pool_test.cc @@ -21,7 +21,7 @@ class WorkerPoolMock : public WorkerPool { explicit WorkerPoolMock( const std::unordered_map> &worker_commands) : WorkerPool(0, NUM_WORKERS_PER_PROCESS, MAXIMUM_STARTUP_CONCURRENCY, - worker_commands), + nullptr, worker_commands), last_worker_pid_(0) {} ~WorkerPoolMock() { @@ -30,8 +30,8 @@ class WorkerPoolMock : public WorkerPool { } void StartWorkerProcess(const Language &language, - const TaskSpecification *task_spec = nullptr) { - WorkerPool::StartWorkerProcess(language, task_spec); + const std::vector &dynamic_options = {}) { + WorkerPool::StartWorkerProcess(language, dynamic_options); } pid_t StartProcess(const std::vector &worker_command_args) override { @@ -228,13 +228,11 @@ TEST_F(WorkerPoolTest, StartWorkerWithDynamicOptionsCommand) { ObjectID::Nil(), 0, ActorID::Nil(), ActorHandleID::Nil(), 0, {}, {}, 0, {}, {}, Language::JAVA, {"", "", ""}, {"test_op_0", "test_op_1"}); - worker_pool_.StartWorkerProcess(Language::JAVA, &task_spec); + worker_pool_.StartWorkerProcess(Language::JAVA, task_spec.DynamicWorkerOptions()); const auto real_command = worker_pool_.GetWorkerCommand(worker_pool_.LastStartedWorkerProcess()); - ASSERT_EQ(3, real_command.size()); - ASSERT_EQ("test_op_0", real_command[0]); - ASSERT_EQ("dummy_java_worker_command", real_command[1]); - ASSERT_EQ("test_op_1", real_command[2]); + ASSERT_EQ(real_command, + std::vector({"test_op_0", "dummy_java_worker_command", "test_op_1"})); } } // namespace raylet From 28d7902e7935afb5bdbb365722e51dce5472de6a Mon Sep 17 00:00:00 2001 From: Qing Wang Date: Fri, 21 Jun 2019 21:58:12 +0800 Subject: [PATCH 38/46] Update src/ray/raylet/worker_pool.h Co-Authored-By: Hao Chen --- src/ray/raylet/worker_pool.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index 187d470fa2e5..d7959e8d7819 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -112,7 +112,7 @@ class WorkerPool { /// Whether there is a pending worker for the given task. /// Note that, this is only used for actor creation task with dynamic options. /// And if the worker registered but isn't assigned a task, - /// the worker also is on pending state, and this'll return true. + /// the worker also is in pending state, and this'll return true. /// /// \param language The required language. /// \param task_id The task that we want to query. From ab178ca140cc8283cb4803fb650a48b07a75fbe7 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Fri, 21 Jun 2019 23:00:41 +0800 Subject: [PATCH 39/46] Address comments. --- src/ray/gcs/format/gcs.fbs | 7 ++++--- src/ray/raylet/node_manager.cc | 4 ++-- src/ray/raylet/worker_pool.cc | 21 ++++++++++----------- src/ray/raylet/worker_pool.h | 2 +- 4 files changed, 17 insertions(+), 17 deletions(-) diff --git a/src/ray/gcs/format/gcs.fbs b/src/ray/gcs/format/gcs.fbs index a2ecc0e95860..90476da73425 100644 --- a/src/ray/gcs/format/gcs.fbs +++ b/src/ray/gcs/format/gcs.fbs @@ -106,9 +106,10 @@ table TaskInfo { // For a Python function, it should be: [module_name, class_name, function_name] // For a Java function, it should be: [class_name, method_name, type_descriptor] function_descriptor: [string]; - // The dynamic options for worker command. - // It means we can specify the options for the worker when creating an actor. - // Note that this is used for actor creating task only. + // The dynamic options used in the worker command when starting the worker process for + // an actor creation task. If the list isn't empty, the options will be used to replace + // the placeholder strings (`RAY_WORKER_OPTION_0`, `RAY_WORKER_OPTION_1`, etc) in the + // worker command. dynamic_worker_options: [string]; } diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index ef07c19d4f39..19b429f0697c 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -2194,8 +2194,8 @@ void NodeManager::ForwardTask( auto task_id = spec.TaskId(); if (worker_pool_.HasPendingWorkerForTask(spec.GetLanguage(), task_id)) { - RAY_LOG(DEBUG) << "There is a worker being starting for this task," - << "so we shouldn't forward this task to another node."; + // There is a worker being starting for this task, + // so we shouldn't forward this task to another node. return; } diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 5812c6a77e9c..980bf57ffffd 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -102,7 +102,7 @@ uint32_t WorkerPool::Size(const Language &language) const { } int WorkerPool::StartWorkerProcess(const Language &language, - const std::vector dynamic_options) { + const std::vector &dynamic_options) { auto &state = GetStateForLanguage(language); // If we are already starting up too many workers, then return without starting // more. @@ -226,19 +226,18 @@ void WorkerPool::PushWorker(const std::shared_ptr &worker) { auto it = state.dedicated_workers_to_tasks.find(worker->Pid()); if (it != state.dedicated_workers_to_tasks.end()) { - // The worker is used for the specific actor creation task with dynamic options. + // The worker is used for the actor creation task with dynamic options. + // Put it into idle dedicated worker pool. const auto task_id = it->second; state.idle_dedicated_workers[task_id] = std::move(worker); - - // Return to do not put this worker to idle pool. - return; - } - - // Add the worker to the idle pool. - if (worker->GetActorId().IsNil()) { - state.idle.insert(std::move(worker)); } else { - state.idle_actor[worker->GetActorId()] = std::move(worker); + // The worker is not used for the actor creation task without dynamic options. + // Put the worker to the corresponding idle pool. + if (worker->GetActorId().IsNil()) { + state.idle.insert(std::move(worker)); + } else { + state.idle_actor[worker->GetActorId()] = std::move(worker); + } } } diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index d7959e8d7819..90dee34445c1 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -139,7 +139,7 @@ class WorkerPool { /// \return The id of the process that we started if it's positive, /// otherwise it means we didn't start a process. int StartWorkerProcess(const Language &language, - const std::vector dynamic_options = {}); + const std::vector &dynamic_options = {}); /// The implementation of how to start a new worker process with command arguments. /// From 5c302300e534a87959ae5bd5269afcae755b973f Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Fri, 21 Jun 2019 23:11:35 +0800 Subject: [PATCH 40/46] Address comments. --- src/ray/raylet/worker_pool.cc | 26 ++++++++++++-------------- src/ray/raylet/worker_pool.h | 9 ++------- 2 files changed, 14 insertions(+), 21 deletions(-) diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 980bf57ffffd..07e464555d87 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -284,18 +284,8 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec } } - if (worker == nullptr) { - // Push an error message to the user if the worker pool tells us that it is - // getting too big. - const std::string warning_message = WarningAboutSize(); - if (warning_message != "") { - if (gcs_client_ != nullptr) { - RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( - DriverID::Nil(), "worker_pool_large", warning_message, current_time_ms())); - } else { - RAY_LOG(WARNING) << "Failed to push error message to user since gcs client is null."; - } - } + if (worker == nullptr && pid > 0) { + WarnAboutSize(); } return worker; @@ -341,7 +331,7 @@ std::vector> WorkerPool::GetWorkersRunningTasksForDriver return workers; } -std::string WorkerPool::WarningAboutSize() { +void WorkerPool::WarnAboutSize() { int64_t num_workers_started_or_registered = 0; for (const auto &entry : states_by_lang_) { num_workers_started_or_registered += @@ -352,6 +342,8 @@ std::string WorkerPool::WarningAboutSize() { int64_t multiple = num_workers_started_or_registered / multiple_for_warning_; std::stringstream warning_message; if (multiple >= 3 && multiple > last_warning_multiple_) { + // Push an error message to the user if the worker pool tells us that it is + // getting too big. last_warning_multiple_ = multiple; warning_message << "WARNING: " << num_workers_started_or_registered << " workers have been started. This could be a result of using " @@ -359,8 +351,14 @@ std::string WorkerPool::WarningAboutSize() { << "using nested tasks " << "(see https://github.com/ray-project/ray/issues/3644) for " << "some a discussion of workarounds."; + + if (gcs_client_ != nullptr) { + RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( + DriverID::Nil(), "worker_pool_large", warning_message.str(), current_time_ms())); + } else { + RAY_LOG(WARNING) << "Failed to push error message to user since gcs client is null."; + } } - return warning_message.str(); } bool WorkerPool::HasPendingWorkerForTask(const Language &language, diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index 90dee34445c1..89034f0e8cff 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -181,14 +181,9 @@ class WorkerPool { /// for a given language. State &GetStateForLanguage(const Language &language); - /// Generate a warning about the number of workers that have registered or - /// started if appropriate. - /// - /// \return An empty string if no warning should be generated and otherwise a - /// string with a warning message. - std::string WarningAboutSize(); + /// Push an warning message to user if worker pool is getting to big. + void WarnAboutSize(); - private: /// We'll push a warning to the user every time a multiple of this many /// workers has been started. int multiple_for_warning_; From 597d7c281499a8f8e4cc4e17bf75c3d0b83eeda3 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Fri, 21 Jun 2019 23:29:52 +0800 Subject: [PATCH 41/46] Fix --- src/ray/raylet/worker_pool.cc | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 07e464555d87..9210e873c277 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -142,7 +142,6 @@ int WorkerPool::StartWorkerProcess(const Language &language, if (pid < 0) { // Failure case. RAY_LOG(FATAL) << "Failed to fork worker process: " << strerror(errno); - return -1; } else if (pid > 0) { // Parent process case. RAY_LOG(DEBUG) << "Started worker process with pid " << pid; @@ -150,6 +149,7 @@ int WorkerPool::StartWorkerProcess(const Language &language, std::make_pair(pid, num_workers_per_process_)); return pid; } + return -1; } pid_t WorkerPool::StartProcess(const std::vector &worker_command_args) { @@ -246,6 +246,7 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec const auto &actor_id = task_spec.ActorId(); std::shared_ptr worker = nullptr; + int pid = -1; if (task_spec.IsActorCreationTask() && !task_spec.DynamicWorkerOptions().empty()) { // Code path of actor creation task with dynamic worker options. // Try to pop it from idle dedicated pool. @@ -260,7 +261,7 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec } else if (!HasPendingWorkerForTask(task_spec.GetLanguage(), task_spec.TaskId())) { // We are not pending a registration from a worker for this task, // so start a new worker process for this task. - auto pid = StartWorkerProcess(task_spec.GetLanguage(), task_spec.DynamicWorkerOptions()); + pid = StartWorkerProcess(task_spec.GetLanguage(), task_spec.DynamicWorkerOptions()); if (pid > 0) { state.dedicated_workers_to_tasks[pid] = task_spec.TaskId(); } @@ -273,7 +274,7 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec } else { // There are no more non-actor workers available to execute this task. // Start a new worker process. - StartWorkerProcess(task_spec.GetLanguage()); + pid = StartWorkerProcess(task_spec.GetLanguage()); } } else { // Code path of actor task. From c65e12c0a5f3dcc0eec465196071e005a9535078 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Sat, 22 Jun 2019 10:13:11 +0800 Subject: [PATCH 42/46] Fix lint --- src/ray/raylet/node_manager.cc | 3 ++- src/ray/raylet/worker_pool.cc | 16 ++++++++-------- src/ray/raylet/worker_pool.h | 4 +++- src/ray/raylet/worker_pool_test.cc | 8 ++++---- 4 files changed, 17 insertions(+), 14 deletions(-) diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 19b429f0697c..fc364539ccce 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -83,7 +83,8 @@ NodeManager::NodeManager(boost::asio::io_service &io_service, initial_config_(config), local_available_resources_(config.resource_config), worker_pool_(config.num_initial_workers, config.num_workers_per_process, - config.maximum_startup_concurrency, gcs_client_, config.worker_commands), + config.maximum_startup_concurrency, gcs_client_, + config.worker_commands), scheduling_policy_(local_queues_), reconstruction_policy_( io_service_, diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 9210e873c277..fe02b0c6c352 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -258,12 +258,14 @@ std::shared_ptr WorkerPool::PopWorker(const TaskSpecification &task_spec // Because we found a worker that can perform this task, // we can remove it from dedicated_workers_to_tasks. state.dedicated_workers_to_tasks.erase(worker->Pid()); + state.tasks_to_dedicated_workers.erase(task_spec.TaskId()); } else if (!HasPendingWorkerForTask(task_spec.GetLanguage(), task_spec.TaskId())) { // We are not pending a registration from a worker for this task, // so start a new worker process for this task. pid = StartWorkerProcess(task_spec.GetLanguage(), task_spec.DynamicWorkerOptions()); if (pid > 0) { state.dedicated_workers_to_tasks[pid] = task_spec.TaskId(); + state.tasks_to_dedicated_workers[task_spec.TaskId()] = pid; } } } else if (!task_spec.IsActorTask()) { @@ -355,9 +357,11 @@ void WorkerPool::WarnAboutSize() { if (gcs_client_ != nullptr) { RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( - DriverID::Nil(), "worker_pool_large", warning_message.str(), current_time_ms())); + DriverID::Nil(), "worker_pool_large", warning_message.str(), + current_time_ms())); } else { - RAY_LOG(WARNING) << "Failed to push error message to user since gcs client is null."; + RAY_LOG(WARNING) + << "Failed to push error message to user since gcs client is null."; } } } @@ -365,12 +369,8 @@ void WorkerPool::WarnAboutSize() { bool WorkerPool::HasPendingWorkerForTask(const Language &language, const TaskID &task_id) { auto &state = GetStateForLanguage(language); - for (const auto &it : state.dedicated_workers_to_tasks) { - if (it.second == task_id) { - return true; - } - } - return false; + auto it = state.tasks_to_dedicated_workers.find(task_id); + return it != state.tasks_to_dedicated_workers.end(); } std::string WorkerPool::DebugString() const { diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index 89034f0e8cff..959db73fb62a 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -7,10 +7,10 @@ #include #include "ray/common/client_connection.h" +#include "ray/gcs/client.h" #include "ray/gcs/format/util.h" #include "ray/raylet/task.h" #include "ray/raylet/worker.h" -#include "ray/gcs/client.h" namespace ray { @@ -169,6 +169,8 @@ class WorkerPool { /// A map for looking up the task with dynamic options by the pid of /// worker. Note that this is used for the dedicated worker processes. std::unordered_map dedicated_workers_to_tasks; + /// A map for speeding up looking up the pending worker for the given task. + std::unordered_map tasks_to_dedicated_workers; }; /// The number of workers per process. diff --git a/src/ray/raylet/worker_pool_test.cc b/src/ray/raylet/worker_pool_test.cc index ac50da1f8250..ee1095108d44 100644 --- a/src/ray/raylet/worker_pool_test.cc +++ b/src/ray/raylet/worker_pool_test.cc @@ -20,8 +20,8 @@ class WorkerPoolMock : public WorkerPool { explicit WorkerPoolMock( const std::unordered_map> &worker_commands) - : WorkerPool(0, NUM_WORKERS_PER_PROCESS, MAXIMUM_STARTUP_CONCURRENCY, - nullptr, worker_commands), + : WorkerPool(0, NUM_WORKERS_PER_PROCESS, MAXIMUM_STARTUP_CONCURRENCY, nullptr, + worker_commands), last_worker_pid_(0) {} ~WorkerPoolMock() { @@ -231,8 +231,8 @@ TEST_F(WorkerPoolTest, StartWorkerWithDynamicOptionsCommand) { worker_pool_.StartWorkerProcess(Language::JAVA, task_spec.DynamicWorkerOptions()); const auto real_command = worker_pool_.GetWorkerCommand(worker_pool_.LastStartedWorkerProcess()); - ASSERT_EQ(real_command, - std::vector({"test_op_0", "dummy_java_worker_command", "test_op_1"})); + ASSERT_EQ(real_command, std::vector( + {"test_op_0", "dummy_java_worker_command", "test_op_1"})); } } // namespace raylet From 36429e5a91618e1e319b428dd87167f3e52d79f1 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Sat, 22 Jun 2019 12:28:56 +0800 Subject: [PATCH 43/46] Fix lint --- src/ray/raylet/worker_pool_test.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/ray/raylet/worker_pool_test.cc b/src/ray/raylet/worker_pool_test.cc index ee1095108d44..eeb181683b85 100644 --- a/src/ray/raylet/worker_pool_test.cc +++ b/src/ray/raylet/worker_pool_test.cc @@ -232,7 +232,7 @@ TEST_F(WorkerPoolTest, StartWorkerWithDynamicOptionsCommand) { const auto real_command = worker_pool_.GetWorkerCommand(worker_pool_.LastStartedWorkerProcess()); ASSERT_EQ(real_command, std::vector( - {"test_op_0", "dummy_java_worker_command", "test_op_1"})); + {"test_op_0", "dummy_java_worker_command", "test_op_1"})); } } // namespace raylet From 79e6358f9ab9144934b4fd6863a60780d9555f20 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Sat, 22 Jun 2019 16:56:40 +0800 Subject: [PATCH 44/46] Fix --- src/ray/raylet/worker_pool_test.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/ray/raylet/worker_pool_test.cc b/src/ray/raylet/worker_pool_test.cc index eeb181683b85..3e9a4449df60 100644 --- a/src/ray/raylet/worker_pool_test.cc +++ b/src/ray/raylet/worker_pool_test.cc @@ -232,7 +232,7 @@ TEST_F(WorkerPoolTest, StartWorkerWithDynamicOptionsCommand) { const auto real_command = worker_pool_.GetWorkerCommand(worker_pool_.LastStartedWorkerProcess()); ASSERT_EQ(real_command, std::vector( - {"test_op_0", "dummy_java_worker_command", "test_op_1"})); + {"test_op_0", "dummy_java_worker_command", "test_op_1"})); } } // namespace raylet From cc925273ef443493570dda6260c2cac4470e1079 Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Sat, 22 Jun 2019 17:27:48 +0800 Subject: [PATCH 45/46] Address comments. --- src/ray/raylet/worker_pool.cc | 12 +++--------- src/ray/raylet/worker_pool.h | 6 +++--- src/ray/raylet/worker_pool_test.cc | 2 ++ 3 files changed, 8 insertions(+), 12 deletions(-) diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index fe02b0c6c352..82337e17a43e 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -354,15 +354,9 @@ void WorkerPool::WarnAboutSize() { << "using nested tasks " << "(see https://github.com/ray-project/ray/issues/3644) for " << "some a discussion of workarounds."; - - if (gcs_client_ != nullptr) { - RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( - DriverID::Nil(), "worker_pool_large", warning_message.str(), - current_time_ms())); - } else { - RAY_LOG(WARNING) - << "Failed to push error message to user since gcs client is null."; - } + RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( + DriverID::Nil(), "worker_pool_large", warning_message.str(), + current_time_ms())); } } diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index 959db73fb62a..e1e726268093 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -147,6 +147,9 @@ class WorkerPool { /// \return The process ID of started worker process. virtual pid_t StartProcess(const std::vector &worker_command_args); + /// Push an warning message to user if worker pool is getting to big. + virtual void WarnAboutSize(); + /// An internal data structure that maintains the pool state per language. struct State { /// The commands and arguments used to start the worker process @@ -183,9 +186,6 @@ class WorkerPool { /// for a given language. State &GetStateForLanguage(const Language &language); - /// Push an warning message to user if worker pool is getting to big. - void WarnAboutSize(); - /// We'll push a warning to the user every time a multiple of this many /// workers has been started. int multiple_for_warning_; diff --git a/src/ray/raylet/worker_pool_test.cc b/src/ray/raylet/worker_pool_test.cc index 3e9a4449df60..15a5fb0471e0 100644 --- a/src/ray/raylet/worker_pool_test.cc +++ b/src/ray/raylet/worker_pool_test.cc @@ -47,6 +47,8 @@ class WorkerPoolMock : public WorkerPool { return last_worker_pid_; } + void WarnAboutSize() override {} + pid_t LastStartedWorkerProcess() const { return last_worker_pid_; } const std::vector &GetWorkerCommand(int pid) { From 313e1dfd14cdbaac21454e9c7a92171d87936ddf Mon Sep 17 00:00:00 2001 From: jovany-wang Date: Sat, 22 Jun 2019 22:35:15 +0800 Subject: [PATCH 46/46] Fix linting --- src/ray/raylet/worker_pool.cc | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index 82337e17a43e..719378216fb7 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -355,8 +355,7 @@ void WorkerPool::WarnAboutSize() { << "(see https://github.com/ray-project/ray/issues/3644) for " << "some a discussion of workarounds."; RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver( - DriverID::Nil(), "worker_pool_large", warning_message.str(), - current_time_ms())); + DriverID::Nil(), "worker_pool_large", warning_message.str(), current_time_ms())); } }