mirror of
https://github.com/vale981/ray
synced 2025-03-05 10:01:43 -05:00
[Scheduler] separate scheduler code to its own build target (#23124)
* wip * comments * fix build * fix-test * fix format
This commit is contained in:
parent
35c7275bfc
commit
5a2ebc281c
13 changed files with 225 additions and 87 deletions
54
BUILD.bazel
54
BUILD.bazel
|
@ -649,6 +649,53 @@ cc_library(
|
|||
],
|
||||
)
|
||||
|
||||
cc_library(
|
||||
name = "scheduler",
|
||||
srcs = glob(
|
||||
[
|
||||
"src/ray/raylet/scheduling/**/*.cc",
|
||||
],
|
||||
exclude = [
|
||||
"src/ray/raylet/scheduling/**/*_test.cc",
|
||||
],
|
||||
),
|
||||
hdrs = glob(
|
||||
[
|
||||
"src/ray/raylet/scheduling/**/*.h",
|
||||
"src/ray/core_worker/common.h",
|
||||
],
|
||||
),
|
||||
copts = COPTS,
|
||||
linkopts = select({
|
||||
"@bazel_tools//src/conditions:windows": [
|
||||
],
|
||||
"//conditions:default": [
|
||||
"-lpthread",
|
||||
],
|
||||
}),
|
||||
strip_include_prefix = "src",
|
||||
visibility = ["//visibility:public"],
|
||||
deps = [
|
||||
":gcs_client_lib",
|
||||
":ray_common",
|
||||
":ray_util",
|
||||
":stats_lib",
|
||||
"//src/ray/protobuf:common_cc_proto",
|
||||
"@boost//:asio",
|
||||
"@boost//:filesystem",
|
||||
"@boost//:system",
|
||||
"@com_github_jupp0r_prometheus_cpp//pull",
|
||||
"@com_google_absl//absl/base:core_headers",
|
||||
"@com_google_absl//absl/container:flat_hash_set",
|
||||
"@com_google_absl//absl/memory",
|
||||
"@com_google_absl//absl/strings",
|
||||
"@com_google_googletest//:gtest",
|
||||
"@io_opencensus_cpp//opencensus/exporters/stats/prometheus:prometheus_exporter",
|
||||
"@io_opencensus_cpp//opencensus/stats",
|
||||
"@io_opencensus_cpp//opencensus/tags",
|
||||
],
|
||||
)
|
||||
|
||||
cc_library(
|
||||
name = "raylet_lib",
|
||||
srcs = glob(
|
||||
|
@ -657,13 +704,17 @@ cc_library(
|
|||
],
|
||||
exclude = [
|
||||
"src/ray/raylet/**/*_test.cc",
|
||||
"src/ray/raylet/scheduling/**/*.cc",
|
||||
"src/ray/raylet/main.cc",
|
||||
],
|
||||
),
|
||||
hdrs = glob(
|
||||
[
|
||||
"src/ray/raylet/**/*.h",
|
||||
"src/ray/core_worker/common.h",
|
||||
],
|
||||
exclude = [
|
||||
"src/ray/raylet/scheduling/**/*.h",
|
||||
"src/ray/raylet/main.cc",
|
||||
],
|
||||
),
|
||||
copts = COPTS,
|
||||
|
@ -688,6 +739,7 @@ cc_library(
|
|||
":ray_common",
|
||||
":ray_util",
|
||||
":runtime_env_rpc",
|
||||
":scheduler",
|
||||
":stats_lib",
|
||||
":worker_rpc",
|
||||
"//src/ray/protobuf:common_cc_proto",
|
||||
|
|
|
@ -12,7 +12,7 @@
|
|||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#include "ray/raylet/scheduling/local_task_manager.h"
|
||||
#include "ray/raylet/local_task_manager.h"
|
||||
|
||||
#include <google/protobuf/map.h>
|
||||
|
||||
|
@ -1050,5 +1050,61 @@ uint64_t LocalTaskManager::MaxRunningTasksPerSchedulingClass(
|
|||
return static_cast<uint64_t>(std::round(total_cpus / cpu_req));
|
||||
}
|
||||
|
||||
void LocalTaskManager::RecordMetrics() const {
|
||||
ray::stats::STATS_scheduler_tasks.Record(executing_task_args_.size(), "Executing");
|
||||
ray::stats::STATS_scheduler_tasks.Record(waiting_tasks_index_.size(), "Waiting");
|
||||
}
|
||||
|
||||
void LocalTaskManager::DebugStr(std::stringstream &buffer) const {
|
||||
buffer << "Waiting tasks size: " << waiting_tasks_index_.size() << "\n";
|
||||
buffer << "Number of executing tasks: " << executing_task_args_.size() << "\n";
|
||||
buffer << "Number of pinned task arguments: " << pinned_task_arguments_.size() << "\n";
|
||||
buffer << "Resource usage {\n";
|
||||
|
||||
// Calculates how much resources are occupied by tasks or actors.
|
||||
// Only iterate upto this number to avoid excessive CPU usage.
|
||||
auto max_iteration = RayConfig::instance().worker_max_resource_analysis_iteration();
|
||||
uint32_t iteration = 0;
|
||||
for (const auto &worker : worker_pool_.GetAllRegisteredWorkers(
|
||||
/*filter_dead_workers*/ true)) {
|
||||
if (max_iteration < iteration++) {
|
||||
break;
|
||||
}
|
||||
if (worker->IsDead() // worker is dead
|
||||
|| worker->IsBlocked() // worker is blocked by blocking Ray API
|
||||
|| (worker->GetAssignedTaskId().IsNil() &&
|
||||
worker->GetActorId().IsNil())) { // Tasks or actors not assigned
|
||||
// Then this shouldn't have allocated resources.
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto &task_or_actor_name = worker->GetAssignedTask()
|
||||
.GetTaskSpecification()
|
||||
.FunctionDescriptor()
|
||||
->CallString();
|
||||
buffer << " - ("
|
||||
<< "language="
|
||||
<< rpc::Language_descriptor()->FindValueByNumber(worker->GetLanguage())->name()
|
||||
<< " "
|
||||
<< "actor_or_task=" << task_or_actor_name << " "
|
||||
<< "pid=" << worker->GetProcess().GetId() << "): "
|
||||
<< worker->GetAssignedTask()
|
||||
.GetTaskSpecification()
|
||||
.GetRequiredResources()
|
||||
.ToString()
|
||||
<< "\n";
|
||||
}
|
||||
buffer << "}\n";
|
||||
buffer << "Running tasks by scheduling class:\n";
|
||||
|
||||
for (const auto &pair : info_by_sched_cls_) {
|
||||
const auto &sched_cls = pair.first;
|
||||
const auto &info = pair.second;
|
||||
const auto &descriptor = TaskSpecification::GetSchedulingClassDescriptor(sched_cls);
|
||||
buffer << " - " << descriptor.DebugString() << ": " << info.running_tasks.size()
|
||||
<< "/" << info.capacity << "\n";
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace raylet
|
||||
} // namespace ray
|
|
@ -23,6 +23,7 @@
|
|||
#include "ray/raylet/scheduling/cluster_resource_scheduler.h"
|
||||
#include "ray/raylet/scheduling/cluster_task_manager_interface.h"
|
||||
#include "ray/raylet/scheduling/internal.h"
|
||||
#include "ray/raylet/scheduling/local_task_manager_interface.h"
|
||||
#include "ray/raylet/worker.h"
|
||||
#include "ray/raylet/worker_pool.h"
|
||||
#include "ray/rpc/grpc_client.h"
|
||||
|
@ -54,7 +55,7 @@ namespace raylet {
|
|||
/// as it should return the request to the distributed scheduler if
|
||||
/// resource accusition failed, or a task has arguments pending resolution for too long
|
||||
/// time.
|
||||
class LocalTaskManager {
|
||||
class LocalTaskManager : public ILocalTaskManager {
|
||||
public:
|
||||
/// \param self_node_id: ID of local node.
|
||||
/// \param cluster_resource_scheduler: The resource scheduler which contains
|
||||
|
@ -91,10 +92,10 @@ class LocalTaskManager {
|
|||
RayConfig::instance().worker_cap_initial_backoff_delay_ms());
|
||||
|
||||
/// Queue task and schedule.
|
||||
void QueueAndScheduleTask(std::shared_ptr<internal::Work> work);
|
||||
void QueueAndScheduleTask(std::shared_ptr<internal::Work> work) override;
|
||||
|
||||
// Schedule and dispatch tasks.
|
||||
void ScheduleAndDispatchTasks();
|
||||
void ScheduleAndDispatchTasks() override;
|
||||
|
||||
/// Move tasks from waiting to ready for dispatch. Called when a task's
|
||||
/// dependencies are resolved.
|
||||
|
@ -120,7 +121,7 @@ class LocalTaskManager {
|
|||
bool CancelTask(const TaskID &task_id,
|
||||
rpc::RequestWorkerLeaseReply::SchedulingFailureType failure_type =
|
||||
rpc::RequestWorkerLeaseReply::SCHEDULING_CANCELLED_INTENDED,
|
||||
const std::string &scheduling_failure_message = "");
|
||||
const std::string &scheduling_failure_message = "") override;
|
||||
|
||||
/// Return if any tasks are pending resource acquisition.
|
||||
///
|
||||
|
@ -132,7 +133,7 @@ class LocalTaskManager {
|
|||
bool AnyPendingTasksForResourceAcquisition(RayTask *example,
|
||||
bool *any_pending,
|
||||
int *num_pending_actor_creation,
|
||||
int *num_pending_tasks) const;
|
||||
int *num_pending_tasks) const override;
|
||||
|
||||
/// Call once a task finishes (i.e. a worker is returned).
|
||||
///
|
||||
|
@ -167,6 +168,22 @@ class LocalTaskManager {
|
|||
|
||||
void ClearWorkerBacklog(const WorkerID &worker_id);
|
||||
|
||||
const absl::flat_hash_map<SchedulingClass, std::deque<std::shared_ptr<internal::Work>>>
|
||||
&GetTaskToDispatch() const override {
|
||||
return tasks_to_dispatch_;
|
||||
}
|
||||
|
||||
const absl::flat_hash_map<SchedulingClass, absl::flat_hash_map<WorkerID, int64_t>>
|
||||
&GetBackLogTracker() const override {
|
||||
return backlog_tracker_;
|
||||
}
|
||||
|
||||
void RecordMetrics() const override;
|
||||
|
||||
void DebugStr(std::stringstream &buffer) const override;
|
||||
|
||||
size_t GetNumTaskSpilled() const override { return num_task_spilled_; }
|
||||
|
||||
private:
|
||||
struct SchedulingClassInfo;
|
||||
|
|
@ -34,6 +34,7 @@
|
|||
#include "ray/raylet/scheduling/cluster_task_manager.h"
|
||||
#include "ray/raylet/scheduling/cluster_task_manager_interface.h"
|
||||
#include "ray/raylet/dependency_manager.h"
|
||||
#include "ray/raylet/local_task_manager.h"
|
||||
#include "ray/raylet/wait_manager.h"
|
||||
#include "ray/raylet/worker_pool.h"
|
||||
#include "ray/rpc/worker/core_worker_client_pool.h"
|
||||
|
|
|
@ -29,7 +29,7 @@ ClusterTaskManager::ClusterTaskManager(
|
|||
std::shared_ptr<ClusterResourceScheduler> cluster_resource_scheduler,
|
||||
internal::NodeInfoGetter get_node_info,
|
||||
std::function<void(const RayTask &)> announce_infeasible_task,
|
||||
std::shared_ptr<LocalTaskManager> local_task_manager,
|
||||
std::shared_ptr<ILocalTaskManager> local_task_manager,
|
||||
std::function<int64_t(void)> get_time_ms)
|
||||
: self_node_id_(self_node_id),
|
||||
cluster_resource_scheduler_(cluster_resource_scheduler),
|
||||
|
|
|
@ -19,18 +19,12 @@
|
|||
#include "ray/common/ray_object.h"
|
||||
#include "ray/common/task/task.h"
|
||||
#include "ray/common/task/task_common.h"
|
||||
#include "ray/raylet/dependency_manager.h"
|
||||
#include "ray/raylet/scheduling/cluster_resource_scheduler.h"
|
||||
#include "ray/raylet/scheduling/cluster_task_manager_interface.h"
|
||||
#include "ray/raylet/scheduling/internal.h"
|
||||
#include "ray/raylet/scheduling/local_task_manager.h"
|
||||
#include "ray/raylet/scheduling/local_task_manager_interface.h"
|
||||
#include "ray/raylet/scheduling/scheduler_resource_reporter.h"
|
||||
#include "ray/raylet/scheduling/scheduler_stats.h"
|
||||
#include "ray/raylet/worker.h"
|
||||
#include "ray/raylet/worker_pool.h"
|
||||
#include "ray/rpc/grpc_client.h"
|
||||
#include "ray/rpc/node_manager/node_manager_client.h"
|
||||
#include "ray/rpc/node_manager/node_manager_server.h"
|
||||
|
||||
namespace ray {
|
||||
namespace raylet {
|
||||
|
@ -59,7 +53,7 @@ class ClusterTaskManager : public ClusterTaskManagerInterface {
|
|||
std::shared_ptr<ClusterResourceScheduler> cluster_resource_scheduler,
|
||||
internal::NodeInfoGetter get_node_info,
|
||||
std::function<void(const RayTask &)> announce_infeasible_task,
|
||||
std::shared_ptr<LocalTaskManager> local_task_manager,
|
||||
std::shared_ptr<ILocalTaskManager> local_task_manager,
|
||||
std::function<int64_t(void)> get_time_ms = []() {
|
||||
return (int64_t)(absl::GetCurrentTimeNanos() / 1e6);
|
||||
});
|
||||
|
@ -152,7 +146,7 @@ class ClusterTaskManager : public ClusterTaskManagerInterface {
|
|||
/// Function to announce infeasible task to GCS.
|
||||
std::function<void(const RayTask &)> announce_infeasible_task_;
|
||||
|
||||
std::shared_ptr<LocalTaskManager> local_task_manager_;
|
||||
std::shared_ptr<ILocalTaskManager> local_task_manager_;
|
||||
|
||||
/// TODO(swang): Add index from TaskID -> Work to avoid having to iterate
|
||||
/// through queues to cancel tasks, etc.
|
||||
|
|
|
@ -14,7 +14,6 @@
|
|||
|
||||
#pragma once
|
||||
|
||||
#include "ray/raylet/worker.h"
|
||||
#include "ray/rpc/server_call.h"
|
||||
#include "src/ray/protobuf/node_manager.pb.h"
|
||||
|
||||
|
|
|
@ -28,6 +28,7 @@
|
|||
#include "ray/common/test_util.h"
|
||||
#include "ray/raylet/scheduling/cluster_resource_scheduler.h"
|
||||
#include "ray/raylet/scheduling/scheduling_ids.h"
|
||||
#include "ray/raylet/local_task_manager.h"
|
||||
#include "ray/raylet/test/util.h"
|
||||
#include "mock/ray/gcs/gcs_client/gcs_client.h"
|
||||
|
||||
|
|
73
src/ray/raylet/scheduling/local_task_manager_interface.h
Normal file
73
src/ray/raylet/scheduling/local_task_manager_interface.h
Normal file
|
@ -0,0 +1,73 @@
|
|||
// Copyright 2020-2021 The Ray Authors.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
#pragma once
|
||||
|
||||
#include "absl/container/flat_hash_map.h"
|
||||
#include "absl/container/flat_hash_set.h"
|
||||
#include "ray/common/ray_object.h"
|
||||
#include "ray/common/task/task.h"
|
||||
#include "ray/common/task/task_common.h"
|
||||
#include "ray/raylet/scheduling/internal.h"
|
||||
|
||||
namespace ray {
|
||||
namespace raylet {
|
||||
|
||||
/// Manages the lifetime of a task on the local node. It receives request from
|
||||
/// cluster_task_manager and tries to execute the task locally.
|
||||
/// Read raylet/local_task_manager.h for more information.
|
||||
class ILocalTaskManager {
|
||||
public:
|
||||
virtual ~ILocalTaskManager() = default;
|
||||
|
||||
/// Queue task and schedule.
|
||||
virtual void QueueAndScheduleTask(std::shared_ptr<internal::Work> work) = 0;
|
||||
|
||||
// Schedule and dispatch tasks.
|
||||
virtual void ScheduleAndDispatchTasks() = 0;
|
||||
|
||||
/// Attempt to cancel an already queued task.
|
||||
///
|
||||
/// \param task_id: The id of the task to remove.
|
||||
/// \param failure_type: The failure type.
|
||||
///
|
||||
/// \return True if task was successfully removed. This function will return
|
||||
/// false if the task is already running.
|
||||
virtual bool CancelTask(
|
||||
const TaskID &task_id,
|
||||
rpc::RequestWorkerLeaseReply::SchedulingFailureType failure_type =
|
||||
rpc::RequestWorkerLeaseReply::SCHEDULING_CANCELLED_INTENDED,
|
||||
const std::string &scheduling_failure_message = "") = 0;
|
||||
|
||||
virtual const absl::flat_hash_map<SchedulingClass,
|
||||
std::deque<std::shared_ptr<internal::Work>>>
|
||||
&GetTaskToDispatch() const = 0;
|
||||
|
||||
virtual const absl::flat_hash_map<SchedulingClass,
|
||||
absl::flat_hash_map<WorkerID, int64_t>>
|
||||
&GetBackLogTracker() const = 0;
|
||||
|
||||
virtual bool AnyPendingTasksForResourceAcquisition(RayTask *example,
|
||||
bool *any_pending,
|
||||
int *num_pending_actor_creation,
|
||||
int *num_pending_tasks) const = 0;
|
||||
|
||||
virtual void RecordMetrics() const = 0;
|
||||
|
||||
virtual void DebugStr(std::stringstream &buffer) const = 0;
|
||||
|
||||
virtual size_t GetNumTaskSpilled() const = 0;
|
||||
};
|
||||
} // namespace raylet
|
||||
} // namespace ray
|
|
@ -34,13 +34,13 @@ SchedulerResourceReporter::SchedulerResourceReporter(
|
|||
const absl::flat_hash_map<SchedulingClass,
|
||||
std::deque<std::shared_ptr<internal::Work>>>
|
||||
&infeasible_tasks,
|
||||
const LocalTaskManager &local_task_manager)
|
||||
const ILocalTaskManager &local_task_manager)
|
||||
: max_resource_shapes_per_load_report_(
|
||||
RayConfig::instance().max_resource_shapes_per_load_report()),
|
||||
tasks_to_schedule_(tasks_to_schedule),
|
||||
tasks_to_dispatch_(local_task_manager.tasks_to_dispatch_),
|
||||
tasks_to_dispatch_(local_task_manager.GetTaskToDispatch()),
|
||||
infeasible_tasks_(infeasible_tasks),
|
||||
backlog_tracker_(local_task_manager.backlog_tracker_) {}
|
||||
backlog_tracker_(local_task_manager.GetBackLogTracker()) {}
|
||||
|
||||
int64_t SchedulerResourceReporter::TotalBacklogSize(
|
||||
SchedulingClass scheduling_class) const {
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
#include "ray/common/ray_config.h"
|
||||
#include "ray/common/task/task_spec.h"
|
||||
#include "ray/raylet/scheduling/internal.h"
|
||||
#include "ray/raylet/scheduling/local_task_manager.h"
|
||||
#include "ray/raylet/scheduling/local_task_manager_interface.h"
|
||||
|
||||
namespace ray {
|
||||
namespace raylet {
|
||||
|
@ -33,7 +33,7 @@ class SchedulerResourceReporter {
|
|||
const absl::flat_hash_map<SchedulingClass,
|
||||
std::deque<std::shared_ptr<internal::Work>>>
|
||||
&infeasible_tasks,
|
||||
const LocalTaskManager &local_task_manager);
|
||||
const ILocalTaskManager &local_task_manager);
|
||||
|
||||
/// Populate the relevant parts of the heartbeat table. This is intended for
|
||||
/// sending resource usage of raylet to gcs. In particular, this should fill in
|
||||
|
|
|
@ -19,7 +19,7 @@ namespace ray {
|
|||
namespace raylet {
|
||||
|
||||
SchedulerStats::SchedulerStats(const ClusterTaskManager &cluster_task_manager,
|
||||
const LocalTaskManager &local_task_manager)
|
||||
const ILocalTaskManager &local_task_manager)
|
||||
: cluster_task_manager_(cluster_task_manager),
|
||||
local_task_manager_(local_task_manager) {}
|
||||
|
||||
|
@ -95,8 +95,8 @@ void SchedulerStats::ComputeStats() {
|
|||
(size_t)0,
|
||||
per_work_accumulator);
|
||||
size_t num_tasks_to_dispatch =
|
||||
std::accumulate(local_task_manager_.tasks_to_dispatch_.begin(),
|
||||
local_task_manager_.tasks_to_dispatch_.end(),
|
||||
std::accumulate(local_task_manager_.GetTaskToDispatch().begin(),
|
||||
local_task_manager_.GetTaskToDispatch().end(),
|
||||
(size_t)0,
|
||||
per_work_accumulator);
|
||||
|
||||
|
@ -122,10 +122,10 @@ void SchedulerStats::RecordMetrics() const {
|
|||
/// that function is expensive. ComputeStats is called by ComputeAndReportDebugStr
|
||||
/// method and they are always periodically called by node manager.
|
||||
stats::NumSpilledTasks.Record(metric_tasks_spilled_ +
|
||||
local_task_manager_.num_task_spilled_);
|
||||
local_task_manager_.GetNumTaskSpilled());
|
||||
local_task_manager_.RecordMetrics();
|
||||
stats::NumInfeasibleSchedulingClasses.Record(
|
||||
cluster_task_manager_.infeasible_tasks_.size());
|
||||
|
||||
/// Worker startup failure
|
||||
ray::stats::STATS_scheduler_failed_worker_startup_total.Record(
|
||||
num_worker_not_started_by_job_config_not_exist_, "JobConfigMissing");
|
||||
|
@ -136,10 +136,6 @@ void SchedulerStats::RecordMetrics() const {
|
|||
|
||||
/// Queued tasks.
|
||||
ray::stats::STATS_scheduler_tasks.Record(num_cancelled_tasks_, "Cancelled");
|
||||
ray::stats::STATS_scheduler_tasks.Record(
|
||||
local_task_manager_.executing_task_args_.size(), "Executing");
|
||||
ray::stats::STATS_scheduler_tasks.Record(
|
||||
local_task_manager_.waiting_tasks_index_.size(), "Waiting");
|
||||
ray::stats::STATS_scheduler_tasks.Record(num_tasks_to_dispatch_, "Dispatched");
|
||||
ray::stats::STATS_scheduler_tasks.Record(num_tasks_to_schedule_, "Received");
|
||||
|
||||
|
@ -181,59 +177,9 @@ std::string SchedulerStats::ComputeAndReportDebugStr() {
|
|||
<< num_worker_not_started_by_process_rate_limit_ << "\n";
|
||||
buffer << "num_tasks_waiting_for_workers: " << num_tasks_waiting_for_workers_ << "\n";
|
||||
buffer << "num_cancelled_tasks: " << num_cancelled_tasks_ << "\n";
|
||||
buffer << "Waiting tasks size: " << local_task_manager_.waiting_tasks_index_.size()
|
||||
<< "\n";
|
||||
buffer << "Number of executing tasks: "
|
||||
<< local_task_manager_.executing_task_args_.size() << "\n";
|
||||
buffer << "Number of pinned task arguments: "
|
||||
<< local_task_manager_.pinned_task_arguments_.size() << "\n";
|
||||
buffer << "cluster_resource_scheduler state: "
|
||||
<< cluster_task_manager_.cluster_resource_scheduler_->DebugString() << "\n";
|
||||
buffer << "Resource usage {\n";
|
||||
|
||||
// Calculates how much resources are occupied by tasks or actors.
|
||||
// Only iterate upto this number to avoid excessive CPU usage.
|
||||
auto max_iteration = RayConfig::instance().worker_max_resource_analysis_iteration();
|
||||
uint32_t iteration = 0;
|
||||
for (const auto &worker : local_task_manager_.worker_pool_.GetAllRegisteredWorkers(
|
||||
/*filter_dead_workers*/ true)) {
|
||||
if (max_iteration < iteration++) {
|
||||
break;
|
||||
}
|
||||
if (worker->IsDead() // worker is dead
|
||||
|| worker->IsBlocked() // worker is blocked by blocking Ray API
|
||||
|| (worker->GetAssignedTaskId().IsNil() &&
|
||||
worker->GetActorId().IsNil())) { // Tasks or actors not assigned
|
||||
// Then this shouldn't have allocated resources.
|
||||
continue;
|
||||
}
|
||||
|
||||
const auto &task_or_actor_name = worker->GetAssignedTask()
|
||||
.GetTaskSpecification()
|
||||
.FunctionDescriptor()
|
||||
->CallString();
|
||||
buffer << " - ("
|
||||
<< "language="
|
||||
<< rpc::Language_descriptor()->FindValueByNumber(worker->GetLanguage())->name()
|
||||
<< " "
|
||||
<< "actor_or_task=" << task_or_actor_name << " "
|
||||
<< "pid=" << worker->GetProcess().GetId() << "): "
|
||||
<< worker->GetAssignedTask()
|
||||
.GetTaskSpecification()
|
||||
.GetRequiredResources()
|
||||
.ToString()
|
||||
<< "\n";
|
||||
}
|
||||
buffer << "}\n";
|
||||
buffer << "Running tasks by scheduling class:\n";
|
||||
|
||||
for (const auto &pair : local_task_manager_.info_by_sched_cls_) {
|
||||
const auto &sched_cls = pair.first;
|
||||
const auto &info = pair.second;
|
||||
const auto &descriptor = TaskSpecification::GetSchedulingClassDescriptor(sched_cls);
|
||||
buffer << " - " << descriptor.DebugString() << ": " << info.running_tasks.size()
|
||||
<< "/" << info.capacity << "\n";
|
||||
}
|
||||
local_task_manager_.DebugStr(buffer);
|
||||
|
||||
buffer << "==================================================\n";
|
||||
return buffer.str();
|
||||
|
|
|
@ -18,8 +18,7 @@
|
|||
#include "ray/common/ray_config.h"
|
||||
#include "ray/common/task/task_spec.h"
|
||||
#include "ray/raylet/scheduling/internal.h"
|
||||
#include "ray/raylet/scheduling/local_task_manager.h"
|
||||
#include "ray/raylet/worker_pool.h"
|
||||
#include "ray/raylet/scheduling/local_task_manager_interface.h"
|
||||
|
||||
namespace ray {
|
||||
namespace raylet {
|
||||
|
@ -30,7 +29,7 @@ class ClusterTaskManager;
|
|||
class SchedulerStats {
|
||||
public:
|
||||
explicit SchedulerStats(const ClusterTaskManager &cluster_task_manager,
|
||||
const LocalTaskManager &local_task_manager);
|
||||
const ILocalTaskManager &local_task_manager);
|
||||
|
||||
// Report metrics doesn't recompute the stats.
|
||||
void RecordMetrics() const;
|
||||
|
@ -46,7 +45,7 @@ class SchedulerStats {
|
|||
void ComputeStats();
|
||||
|
||||
const ClusterTaskManager &cluster_task_manager_;
|
||||
const LocalTaskManager &local_task_manager_;
|
||||
const ILocalTaskManager &local_task_manager_;
|
||||
|
||||
/// Number of tasks that are spilled to other
|
||||
/// nodes because it cannot be scheduled locally.
|
||||
|
|
Loading…
Add table
Reference in a new issue