Skip to content

Commit

Permalink
Refactor node manager to eliminate new_scheduler_enabled_ (ray-proj…
Browse files Browse the repository at this point in the history
  • Loading branch information
wumuzi520 authored Jan 17, 2021
1 parent 2cd51ce commit a4ebdbd
Show file tree
Hide file tree
Showing 14 changed files with 1,411 additions and 737 deletions.
599 changes: 72 additions & 527 deletions src/ray/raylet/node_manager.cc

Large diffs are not rendered by default.

135 changes: 122 additions & 13 deletions src/ray/raylet/node_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,8 @@
#include "ray/raylet/scheduling/scheduling_ids.h"
#include "ray/raylet/scheduling/cluster_resource_scheduler.h"
#include "ray/raylet/scheduling/cluster_task_manager.h"
#include "ray/raylet/scheduling/old_cluster_resource_scheduler.h"
#include "ray/raylet/scheduling/cluster_task_manager_interface.h"
#include "ray/raylet/scheduling_policy.h"
#include "ray/raylet/scheduling_queue.h"
#include "ray/raylet/reconstruction_policy.h"
Expand Down Expand Up @@ -110,7 +112,8 @@ struct NodeManagerConfig {
int64_t min_spilling_size;
};

class NodeManager : public rpc::NodeManagerServiceHandler {
class NodeManager : public rpc::NodeManagerServiceHandler,
public ClusterTaskManagerInterface {
public:
/// Create a node manager.
///
Expand Down Expand Up @@ -613,13 +616,6 @@ class NodeManager : public rpc::NodeManagerServiceHandler {
/// Dispatch tasks to available workers.
void DispatchScheduledTasksToWorkers();

/// For the pending task at the head of tasks_to_schedule_, return a node
/// in the system (local or remote) that has enough resources available to
/// run the task, if any such node exist.
/// Repeat the process as long as we can schedule a task.
/// NEW SCHEDULER_FUNCTION
void ScheduleAndDispatch();

/// Whether a task is an actor creation task.
bool IsActorCreationTask(const TaskID &task_id);

Expand All @@ -635,6 +631,122 @@ class NodeManager : public rpc::NodeManagerServiceHandler {
/// \param task Task that is infeasible
void PublishInfeasibleTaskError(const Task &task) const;

std::unordered_map<SchedulingClass, ordered_set<TaskID>> MakeTasksByClass(
const std::vector<Task> &tasks) const;

///////////////////////////////////////////////////////////////////////////////////////
//////////////////// Begin of the override methods of ClusterTaskManager //////////////
// The following methods are defined in node_manager.task.cc instead of node_manager.cc

/// Return the resources that were being used by this worker.
void ReleaseWorkerResources(std::shared_ptr<WorkerInterface> worker) override;

/// When a task is blocked in ray.get or ray.wait, the worker who is executing the task
/// should give up the CPU resources allocated for the running task for the time being
/// and the worker itself should also be marked as blocked.
///
/// \param worker The worker who will give up the CPU resources.
/// \return true if the cpu resources of the specified worker are released successfully,
/// else false.
bool ReleaseCpuResourcesFromUnblockedWorker(
std::shared_ptr<WorkerInterface> worker) override;

/// When a task is no longer blocked in a ray.get or ray.wait, the CPU resources that
/// the worker gave up should be returned to it.
///
/// \param worker The blocked worker.
/// \return true if the cpu resources are returned back to the specified worker, else
/// false.
bool ReturnCpuResourcesToBlockedWorker(
std::shared_ptr<WorkerInterface> worker) override;

// Schedule and dispatch tasks.
void ScheduleAndDispatchTasks() override;

/// Move tasks from waiting to ready for dispatch. Called when a task's
/// dependencies are resolved.
///
/// \param readyIds: The tasks which are now ready to be dispatched.
void TasksUnblocked(const std::vector<TaskID> &ready_ids) override;

/// Populate the relevant parts of the heartbeat table. This is intended for
/// sending raylet <-> gcs heartbeats. In particular, this should fill in
/// resource_load and resource_load_by_shape.
///
/// \param Output parameter. `resource_load` and `resource_load_by_shape` are the only
/// fields used.
void FillResourceUsage(std::shared_ptr<rpc::ResourcesData> data) override;

/// Populate the list of pending or infeasible actor tasks for node stats.
///
/// \param Output parameter.
void FillPendingActorInfo(rpc::GetNodeStatsReply *reply) const override;

/// Return the finished task and relase the worker resources.
/// This method will be removed and can be replaced by `ReleaseWorkerResources` directly
/// once we remove the legacy scheduler.
///
/// \param worker: The worker which was running the task.
/// \param task: Output parameter.
void TaskFinished(std::shared_ptr<WorkerInterface> worker, Task *task) override;

/// Return worker resources.
/// This method will be removed and can be replaced by `ReleaseWorkerResources` directly
/// once we remove the legacy scheduler.
///
/// \param worker: The worker which was running the task.
void ReturnWorkerResources(std::shared_ptr<WorkerInterface> worker) override;

/// Attempt to cancel an already queued task.
///
/// \param task_id: The id of the task to remove.
///
/// \return True if task was successfully removed. This function will return
/// false if the task is already running.
bool CancelTask(const TaskID &task_id) override;

/// Queue task and schedule. This hanppens when processing the worker lease request.
///
/// \param task: The incoming task to be queued and scheduled.
/// \param reply: The reply of the lease request.
/// \param send_reply_callback: The function used during dispatching.
void QueueAndScheduleTask(const Task &task, rpc::RequestWorkerLeaseReply *reply,
rpc::SendReplyCallback send_reply_callback) override;

/// Schedule infeasible tasks.
void ScheduleInfeasibleTasks() override;

/// Return if any tasks are pending resource acquisition.
///
/// \param[in] exemplar An example task that is deadlocking.
/// \param[in] num_pending_actor_creation Number of pending actor creation tasks.
/// \param[in] num_pending_tasks Number of pending tasks.
/// \param[in] any_pending True if there's any pending exemplar.
/// \return True if any progress is any tasks are pending.
bool AnyPendingTasks(Task *exemplar, bool *any_pending, int *num_pending_actor_creation,
int *num_pending_tasks) const override;

/// Handle the resource usage updated event of the specified node.
///
/// \param node_id ID of the node which resources are updated.
/// \param resource_data The node resources.
void OnNodeResourceUsageUpdated(const NodeID &node_id,
const rpc::ResourcesData &resource_data) override;

/// Handle the object missing event.
///
/// \param object_id ID of the missing object.
/// \param waiting_task_ids IDs of tasks that are waitting for the specified missing
/// object.
void OnObjectMissing(const ObjectID &object_id,
const std::vector<TaskID> &waiting_task_ids) override;

/// The helper to dump the debug state of the cluster task manater.
std::string DebugStr() const override;

//////////////////// End of the Override of ClusterTaskManager //////////////////////
///////////////////////////////////////////////////////////////////////////////////////

/// ID of this node.
NodeID self_node_id_;
boost::asio::io_service &io_service_;
Expand Down Expand Up @@ -732,9 +844,6 @@ class NodeManager : public rpc::NodeManagerServiceHandler {
/// lease on.
absl::flat_hash_map<WorkerID, std::vector<WorkerID>> leased_workers_by_owner_;

/// Whether new schedule is enabled.
const bool new_scheduler_enabled_;

/// Whether to report the worker's backlog size in the GCS heartbeat.
const bool report_worker_backlog_;

Expand All @@ -760,8 +869,8 @@ class NodeManager : public rpc::NodeManagerServiceHandler {
/// responsible for maintaining a view of the cluster state w.r.t resource
/// usage. ClusterTaskManager is responsible for queuing, spilling back, and
/// dispatching tasks.
std::shared_ptr<ClusterResourceScheduler> new_resource_scheduler_;
std::shared_ptr<ClusterTaskManager> cluster_task_manager_;
std::shared_ptr<ClusterResourceSchedulerInterface> cluster_resource_scheduler_;
std::shared_ptr<ClusterTaskManagerInterface> cluster_task_manager_;

absl::flat_hash_map<ObjectID, std::unique_ptr<RayObject>> pinned_objects_;

Expand Down
Loading

0 comments on commit a4ebdbd

Please sign in to comment.