mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
Fix the bug of unregistered workers in worker pool (#7343)
* Fix * Fix * Fix complie * Fix lint * Fix linting * Fix testDeleteObject * Fix linting * Update src/ray/raylet/worker_pool.cc Co-Authored-By: Hao Chen <chenh1024@gmail.com> * Update src/ray/raylet/worker_pool.cc Co-Authored-By: Hao Chen <chenh1024@gmail.com> * Update src/ray/raylet/worker_pool.h Co-Authored-By: Hao Chen <chenh1024@gmail.com> * Update src/ray/raylet/worker_pool.cc Co-Authored-By: Hao Chen <chenh1024@gmail.com> * Address comments. * FIx linting Co-authored-by: Hao Chen <chenh1024@gmail.com>
This commit is contained in:
parent
0792b5cb93
commit
2771af1036
6 changed files with 61 additions and 11 deletions
|
@ -33,6 +33,10 @@ public class TestUtils {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static boolean isDirectActorCallEnabled() {
|
||||||
|
return ActorCreationOptions.DEFAULT_USE_DIRECT_CALL;
|
||||||
|
}
|
||||||
|
|
||||||
public static void skipTestIfDirectActorCallEnabled() {
|
public static void skipTestIfDirectActorCallEnabled() {
|
||||||
skipTestIfDirectActorCallEnabled(true);
|
skipTestIfDirectActorCallEnabled(true);
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,9 +25,14 @@ public class PlasmaFreeTest extends BaseTest {
|
||||||
Ray.internal().free(ImmutableList.of(helloId.getId()), true, false);
|
Ray.internal().free(ImmutableList.of(helloId.getId()), true, false);
|
||||||
|
|
||||||
final boolean result = TestUtils.waitForCondition(() ->
|
final boolean result = TestUtils.waitForCondition(() ->
|
||||||
TestUtils.getRuntime().getObjectStore()
|
!TestUtils.getRuntime().getObjectStore()
|
||||||
.wait(ImmutableList.of(helloId.getId()), 1, 0).get(0) == false, 50);
|
.wait(ImmutableList.of(helloId.getId()), 1, 0).get(0), 50);
|
||||||
Assert.assertTrue(result);
|
if (TestUtils.isDirectActorCallEnabled()) {
|
||||||
|
// Direct call will not delete object from im-memory store.
|
||||||
|
Assert.assertFalse(result);
|
||||||
|
} else {
|
||||||
|
Assert.assertTrue(result);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -90,9 +90,11 @@ NodeManager::NodeManager(boost::asio::io_service &io_service,
|
||||||
object_manager_profile_timer_(io_service),
|
object_manager_profile_timer_(io_service),
|
||||||
initial_config_(config),
|
initial_config_(config),
|
||||||
local_available_resources_(config.resource_config),
|
local_available_resources_(config.resource_config),
|
||||||
worker_pool_(io_service, config.num_initial_workers,
|
worker_pool_(
|
||||||
config.maximum_startup_concurrency, gcs_client_,
|
io_service, config.num_initial_workers, config.maximum_startup_concurrency,
|
||||||
config.worker_commands),
|
gcs_client_, config.worker_commands,
|
||||||
|
/*starting_worker_timeout_callback=*/
|
||||||
|
[this]() { this->DispatchTasks(this->local_queues_.GetReadyTasksByClass()); }),
|
||||||
scheduling_policy_(local_queues_),
|
scheduling_policy_(local_queues_),
|
||||||
reconstruction_policy_(
|
reconstruction_policy_(
|
||||||
io_service_,
|
io_service_,
|
||||||
|
|
|
@ -3,6 +3,7 @@
|
||||||
#include <sys/wait.h>
|
#include <sys/wait.h>
|
||||||
|
|
||||||
#include <algorithm>
|
#include <algorithm>
|
||||||
|
#include <boost/date_time/posix_time/posix_time.hpp>
|
||||||
|
|
||||||
#include "ray/common/constants.h"
|
#include "ray/common/constants.h"
|
||||||
#include "ray/common/ray_config.h"
|
#include "ray/common/ray_config.h"
|
||||||
|
@ -44,10 +45,12 @@ namespace raylet {
|
||||||
WorkerPool::WorkerPool(boost::asio::io_service &io_service, int num_workers,
|
WorkerPool::WorkerPool(boost::asio::io_service &io_service, int num_workers,
|
||||||
int maximum_startup_concurrency,
|
int maximum_startup_concurrency,
|
||||||
std::shared_ptr<gcs::GcsClient> gcs_client,
|
std::shared_ptr<gcs::GcsClient> gcs_client,
|
||||||
const WorkerCommandMap &worker_commands)
|
const WorkerCommandMap &worker_commands,
|
||||||
|
std::function<void()> starting_worker_timeout_callback)
|
||||||
: io_service_(&io_service),
|
: io_service_(&io_service),
|
||||||
maximum_startup_concurrency_(maximum_startup_concurrency),
|
maximum_startup_concurrency_(maximum_startup_concurrency),
|
||||||
gcs_client_(std::move(gcs_client)) {
|
gcs_client_(std::move(gcs_client)),
|
||||||
|
starting_worker_timeout_callback_(starting_worker_timeout_callback) {
|
||||||
RAY_CHECK(maximum_startup_concurrency > 0);
|
RAY_CHECK(maximum_startup_concurrency > 0);
|
||||||
#ifndef _WIN32
|
#ifndef _WIN32
|
||||||
// Ignore SIGCHLD signals. If we don't do this, then worker processes will
|
// Ignore SIGCHLD signals. If we don't do this, then worker processes will
|
||||||
|
@ -190,10 +193,33 @@ Process WorkerPool::StartWorkerProcess(const Language &language,
|
||||||
Process proc = StartProcess(worker_command_args);
|
Process proc = StartProcess(worker_command_args);
|
||||||
RAY_LOG(DEBUG) << "Started worker process of " << workers_to_start
|
RAY_LOG(DEBUG) << "Started worker process of " << workers_to_start
|
||||||
<< " worker(s) with pid " << proc.GetId();
|
<< " worker(s) with pid " << proc.GetId();
|
||||||
|
MonitorStartingWorkerProcess(proc, language);
|
||||||
state.starting_worker_processes.emplace(proc, workers_to_start);
|
state.starting_worker_processes.emplace(proc, workers_to_start);
|
||||||
return proc;
|
return proc;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void WorkerPool::MonitorStartingWorkerProcess(const Process &proc,
|
||||||
|
const Language &language) {
|
||||||
|
constexpr static size_t worker_register_timeout_seconds = 30;
|
||||||
|
auto timer = std::make_shared<boost::asio::deadline_timer>(
|
||||||
|
*io_service_, boost::posix_time::seconds(worker_register_timeout_seconds));
|
||||||
|
// Capture timer in lambda to copy it once, so that it can avoid destructing timer.
|
||||||
|
timer->async_wait(
|
||||||
|
[timer, language, proc, this](const boost::system::error_code e) -> void {
|
||||||
|
// check the error code.
|
||||||
|
auto &state = this->GetStateForLanguage(language);
|
||||||
|
// Since this process times out to start, remove it from starting_worker_processes
|
||||||
|
// to avoid the zombie worker.
|
||||||
|
auto it = state.starting_worker_processes.find(proc);
|
||||||
|
if (it != state.starting_worker_processes.end()) {
|
||||||
|
RAY_LOG(INFO) << "Some workers of the worker process(" << proc.GetId()
|
||||||
|
<< ") have not registered to raylet within timeout.";
|
||||||
|
state.starting_worker_processes.erase(it);
|
||||||
|
starting_worker_timeout_callback_();
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
Process WorkerPool::StartProcess(const std::vector<std::string> &worker_command_args) {
|
Process WorkerPool::StartProcess(const std::vector<std::string> &worker_command_args) {
|
||||||
if (RAY_LOG_ENABLED(DEBUG)) {
|
if (RAY_LOG_ENABLED(DEBUG)) {
|
||||||
std::stringstream stream;
|
std::stringstream stream;
|
||||||
|
|
|
@ -42,9 +42,12 @@ class WorkerPool {
|
||||||
/// resources on the machine).
|
/// resources on the machine).
|
||||||
/// \param worker_commands The commands used to start the worker process, grouped by
|
/// \param worker_commands The commands used to start the worker process, grouped by
|
||||||
/// language.
|
/// language.
|
||||||
|
/// \param starting_worker_timeout_callback The callback that will be triggered once
|
||||||
|
/// it times out to start a worker.
|
||||||
WorkerPool(boost::asio::io_service &io_service, int num_workers,
|
WorkerPool(boost::asio::io_service &io_service, int num_workers,
|
||||||
int maximum_startup_concurrency, std::shared_ptr<gcs::GcsClient> gcs_client,
|
int maximum_startup_concurrency, std::shared_ptr<gcs::GcsClient> gcs_client,
|
||||||
const WorkerCommandMap &worker_commands);
|
const WorkerCommandMap &worker_commands,
|
||||||
|
std::function<void()> starting_worker_timeout_callback);
|
||||||
|
|
||||||
/// Destructor responsible for freeing a set of workers owned by this class.
|
/// Destructor responsible for freeing a set of workers owned by this class.
|
||||||
virtual ~WorkerPool();
|
virtual ~WorkerPool();
|
||||||
|
@ -221,13 +224,22 @@ class WorkerPool {
|
||||||
/// for a given language.
|
/// for a given language.
|
||||||
State &GetStateForLanguage(const Language &language);
|
State &GetStateForLanguage(const Language &language);
|
||||||
|
|
||||||
|
/// Start a timer to monitor the starting worker process.
|
||||||
|
///
|
||||||
|
/// If any workers in this process don't register within the timeout
|
||||||
|
/// (due to worker process crash or any other reasons), remove them
|
||||||
|
/// from `starting_worker_processes`. Otherwise if we'll mistakenly
|
||||||
|
/// think there are unregistered workers, and won't start new workers.
|
||||||
|
void MonitorStartingWorkerProcess(const Process &proc, const Language &language);
|
||||||
|
|
||||||
/// For Process class for managing subprocesses (e.g. reaping zombies).
|
/// For Process class for managing subprocesses (e.g. reaping zombies).
|
||||||
boost::asio::io_service *io_service_;
|
boost::asio::io_service *io_service_;
|
||||||
/// The maximum number of worker processes that can be started concurrently.
|
/// The maximum number of worker processes that can be started concurrently.
|
||||||
int maximum_startup_concurrency_;
|
int maximum_startup_concurrency_;
|
||||||
/// A client connection to the GCS.
|
/// A client connection to the GCS.
|
||||||
std::shared_ptr<gcs::GcsClient> gcs_client_;
|
std::shared_ptr<gcs::GcsClient> gcs_client_;
|
||||||
|
/// The callback that will be triggered once it times out to start a worker.
|
||||||
|
std::function<void()> starting_worker_timeout_callback_;
|
||||||
FRIEND_TEST(WorkerPoolTest, InitialWorkerProcessCount);
|
FRIEND_TEST(WorkerPoolTest, InitialWorkerProcessCount);
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
|
@ -28,7 +28,8 @@ class WorkerPoolMock : public WorkerPool {
|
||||||
|
|
||||||
explicit WorkerPoolMock(boost::asio::io_service &io_service,
|
explicit WorkerPoolMock(boost::asio::io_service &io_service,
|
||||||
const WorkerCommandMap &worker_commands)
|
const WorkerCommandMap &worker_commands)
|
||||||
: WorkerPool(io_service, 0, MAXIMUM_STARTUP_CONCURRENCY, nullptr, worker_commands),
|
: WorkerPool(io_service, 0, MAXIMUM_STARTUP_CONCURRENCY, nullptr, worker_commands,
|
||||||
|
[]() {}),
|
||||||
last_worker_process_() {
|
last_worker_process_() {
|
||||||
for (auto &entry : states_by_lang_) {
|
for (auto &entry : states_by_lang_) {
|
||||||
entry.second.num_workers_per_process = NUM_WORKERS_PER_PROCESS;
|
entry.second.num_workers_per_process = NUM_WORKERS_PER_PROCESS;
|
||||||
|
|
Loading…
Add table
Reference in a new issue