Skip to content

Commit 7f19859

Browse files
committed
Code review
1 parent d786a88 commit 7f19859

File tree

3 files changed

+35
-16
lines changed

3 files changed

+35
-16
lines changed

python/ray/tests/test_runtime_env.py

Lines changed: 26 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -9,6 +9,7 @@
99
import pytest
1010

1111
import ray
12+
from ray.exceptions import RuntimeEnvSetupError
1213
from ray.runtime_env import RuntimeEnv, RuntimeEnvConfig
1314

1415

@@ -167,5 +168,30 @@ def run(runtime_env):
167168
run(runtime_env)
168169

169170

171+
def test_large_runtime_env_fails_fast(start_cluster_shared):
172+
"""
173+
Tests that a task with a runtime_env that is too large fails quickly
174+
instead of hanging. This is a regression test for #47432.
175+
"""
176+
cluster, address = start_cluster_shared
177+
ray.init(address)
178+
179+
# Create a runtime_env with a very large environment variable to trigger
180+
# the E2BIG error. The exact limit for process arguments depends on the OS,
181+
# but ~400KB should exceed it on most systems.
182+
large_env_vars = {"MY_HUGE_VAR": "X" * 4096 * 100}
183+
runtime_env = {"env_vars": large_env_vars}
184+
185+
@ray.remote
186+
def f():
187+
# This code should not be reached.
188+
return 1
189+
190+
# The PR ensures the E2BIG error from the raylet is propagated to the
191+
# driver, which should raise a RuntimeEnvSetupError.
192+
with pytest.raises(RuntimeEnvSetupError):
193+
ray.get(f.options(runtime_env=runtime_env).remote())
194+
195+
170196
if __name__ == "__main__":
171197
sys.exit(pytest.main(["-sv", __file__]))

src/ray/raylet/worker_pool.cc

Lines changed: 7 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -530,14 +530,10 @@ std::tuple<Process, StartupToken> WorkerPool::StartWorkerProcess(
530530
// Start a process and measure the startup time.
531531
Process proc = StartProcess(worker_command_args, env, ec);
532532
if (ec) {
533-
if (ec.value() == E2BIG) {
534-
RAY_LOG(ERROR) << "E2BIG error occurred when starting worker process. Worker "
533+
RAY_CHECK(ec.value() == E2BIG);
534+
RAY_LOG(WARNING) << "E2BIG error occurred when starting worker process. Worker "
535535
"command arguments likely too long.";
536-
*status = PopWorkerStatus::ArgumentListTooLong;
537-
return {Process(), (StartupToken)-1};
538-
}
539-
RAY_LOG(ERROR) << "Failed to start worker process: " << ec.message();
540-
*status = PopWorkerStatus::RuntimeEnvCreationFailed;
536+
*status = PopWorkerStatus::ArgumentListTooLong;
541537
return {Process(), (StartupToken)-1};
542538
}
543539
stats::NumWorkersStarted.Record(1);
@@ -684,8 +680,10 @@ Process WorkerPool::StartProcess(const std::vector<std::string> &worker_command_
684680

685681
Process child(argv.data(), io_service_, ec, /*decouple=*/false, env);
686682
if (!child.IsValid() || ec) {
687-
// errorcode 24: Too many files. This is caused by ulimit.
688-
if (ec.value() == 24) {
683+
if (ec.value() == E2BIG) {
684+
// Do nothing here; the error code `ec` will be propagated to the caller.
685+
} else if (ec.value() == 24) {
686+
// errorcode 24: Too many files. This is caused by ulimit.
689687
RAY_LOG(FATAL) << "Too many workers, failed to create a file. Try setting "
690688
<< "`ulimit -n <num_files>` then restart Ray.";
691689
} else {

src/ray/raylet/worker_pool.h

Lines changed: 2 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -51,30 +51,25 @@ using WorkerCommandMap =
5151

5252
enum PopWorkerStatus {
5353
// OK.
54-
// A registered worker will be returned with callback.
5554
OK = 0,
5655
// Job config is not found.
57-
// A nullptr worker will be returned with callback.
5856
JobConfigMissing = 1,
5957
// Worker process startup rate is limited.
60-
// A nullptr worker will be returned with callback.
6158
TooManyStartingWorkerProcesses = 2,
6259
// Worker process has been started, but the worker did not register at the raylet within
6360
// the timeout.
64-
// A nullptr worker will be returned with callback.
6561
WorkerPendingRegistration = 3,
6662
// Any fails of runtime env creation.
67-
// A nullptr worker will be returned with callback.
6863
RuntimeEnvCreationFailed = 4,
6964
// The task's job has finished.
70-
// A nullptr worker will be returned with callback.
7165
JobFinished = 5,
7266
// The worker process failed to launch because the OS returned an `E2BIG`
7367
// (Argument list too long) error. This typically occurs when a `runtime_env`
7468
// is so large that its serialized context exceeds the kernel's command-line
7569
// argument size limit.
76-
// A nullptr worker will be returned with callback.
7770
ArgumentListTooLong = 6,
71+
// The worker process failed to launch.
72+
WorkerLaunchFailed = 7,
7873
};
7974

8075
/// \param[in] worker The started worker instance. Nullptr if worker is not started.

0 commit comments

Comments
 (0)