ray-project / ray

Ray is an AI compute engine. Ray consists of a core distributed runtime and a set of AI Libraries for accelerating ML workloads.
https://ray.io
Apache License 2.0
33.92k stars 5.77k forks source link

[Ray-Tune] Overhead slowdown possibly because of worker errors #42201

Open JosephRRB opened 10 months ago

JosephRRB commented 10 months ago

What happened + What you expected to happen

Initially, we were testing the execution time of OptunaSearch in ray-tune vs directly using optuna. We found that there seems to be a large overhead time from ray-tune. We think it was because of multiple worker errors during tuner.fit().

To isolate the errors from external packages, we used BasicVariantGenerator instead to generate the errors. We also notice that increasing the number of cpu resources (the maximum: 4, in our case) through tune.with_resources seem to remove these errors.

The error logs below was generated using the Reproduction script provided:

2024-01-05 17:28:52,112 INFO worker.py:1673 -- Started a local Ray instance.
2024-01-05 17:28:52,984 INFO tune.py:220 -- Initializing Ray automatically. For cluster usage or custom Ray initialization, call `ray.init(...)` before `Tuner(...)`.
2024-01-05 17:28:52,986 INFO tune.py:595 -- [output] This will use the new output engine with verbosity 1. To disable the new output and use the legacy output engine, set the environment variable RAY_AIR_NEW_OUTPUT=0. For more information, please see https://github.com/ray-project/ray/issues/36949
(bundle_reservation_check_func pid=119476) Traceback (most recent call last):
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 1649, in ray._raylet.execute_task
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 1651, in ray._raylet.execute_task
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/site-packages/ray/_private/worker.py", line 740, in deserialize_objects
(bundle_reservation_check_func pid=119476)     context = self.get_serialization_context()
(bundle_reservation_check_func pid=119476)               ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/site-packages/ray/_private/worker.py", line 628, in get_serialization_context
(bundle_reservation_check_func pid=119476)     context_map[job_id] = serialization.SerializationContext(self)
(bundle_reservation_check_func pid=119476)                           ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/site-packages/ray/_private/serialization.py", line 153, in __init__
(bundle_reservation_check_func pid=119476)     serialization_addons.apply(self)
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/site-packages/ray/util/serialization_addons.py", line 82, in apply
(bundle_reservation_check_func pid=119476)     from ray._private.arrow_serialization import (
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/site-packages/ray/_private/arrow_serialization.py", line 216, in <module>
(bundle_reservation_check_func pid=119476)     @dataclass
(bundle_reservation_check_func pid=119476)      ^^^^^^^^^
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/dataclasses.py", line 1230, in dataclass
(bundle_reservation_check_func pid=119476)     return wrap(cls)
(bundle_reservation_check_func pid=119476)            ^^^^^^^^^
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/dataclasses.py", line 1220, in wrap
(bundle_reservation_check_func pid=119476)     return _process_class(cls, init, repr, eq, order, unsafe_hash,
(bundle_reservation_check_func pid=119476)            ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/dataclasses.py", line 1056, in _process_class
(bundle_reservation_check_func pid=119476)     _cmp_fn('__eq__', '==',
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/dataclasses.py", line 630, in _cmp_fn
(bundle_reservation_check_func pid=119476)     return _create_fn(name,
(bundle_reservation_check_func pid=119476)            ^^^^^^^^^^^^^^^^
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/dataclasses.py", line 433, in _create_fn
(bundle_reservation_check_func pid=119476)     exec(txt, globals, ns)
(bundle_reservation_check_func pid=119476)   File "<string>", line 0, in <module>
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/site-packages/ray/_private/worker.py", line 791, in sigterm_handler
(bundle_reservation_check_func pid=119476)     raise_sys_exit_with_custom_error_message(
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 725, in ray._raylet.raise_sys_exit_with_custom_error_message
(bundle_reservation_check_func pid=119476) SystemExit: 1
(bundle_reservation_check_func pid=119476) 
(bundle_reservation_check_func pid=119476) During handling of the above exception, another exception occurred:
(bundle_reservation_check_func pid=119476) 
(bundle_reservation_check_func pid=119476) Traceback (most recent call last):
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 1960, in ray._raylet.execute_task_with_cancellation_handler
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 1617, in ray._raylet.execute_task
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 1618, in ray._raylet.execute_task
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 1621, in ray._raylet.execute_task
(bundle_reservation_check_func pid=119476)   File "python/ray/includes/libcoreworker.pxi", line 33, in ray._raylet.ProfileEvent.__exit__
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/traceback.py", line 184, in format_exc
(bundle_reservation_check_func pid=119476)     return "".join(format_exception(*sys.exc_info(), limit=limit, chain=chain))
(bundle_reservation_check_func pid=119476)                
(bundle_reservation_check_func pid=119476)     ^^^^^^^^^^^^^^^^^^^^^^^^^^^^
(bundle_reservation_check_func pid=119476) ^^^^^^^^^^^^^^^
(bundle_reservation_check_func pid=119476) ^^^^^^^^^^^^^^^^
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/traceback.py", line 139, in format_exception
(bundle_reservation_check_func pid=119476)     te = TracebackException(type(value), value, tb, limit=limit, compact=True)
(bundle_reservation_check_func pid=119476)          ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/traceback.py", line 728, in __init__
(bundle_reservation_check_func pid=119476)     self.stack = StackSummary._extract_from_extended_frame_gen(
(bundle_reservation_check_func pid=119476)                  ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/traceback.py", line 433, in _extract_from_extended_frame_gen
(bundle_reservation_check_func pid=119476)     f.line
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/traceback.py", line 318, in line
(bundle_reservation_check_func pid=119476)     self._line = linecache.getline(self.filename, self.lineno)
(bundle_reservation_check_func pid=119476)                  ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/linecache.py", line 30, in getline
(bundle_reservation_check_func pid=119476)     lines = getlines(filename, module_globals)
(bundle_reservation_check_func pid=119476)             ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/linecache.py", line 46, in getlines
(bundle_reservation_check_func pid=119476)     return updatecache(filename, module_globals)
(bundle_reservation_check_func pid=119476)            ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/linecache.py", line 93, in updatecache
(bundle_reservation_check_func pid=119476)     stat = os.stat(fullname)
(bundle_reservation_check_func pid=119476)            ^^^^^^^^^^^^^^^^^
(bundle_reservation_check_func pid=119476) KeyboardInterrupt
(bundle_reservation_check_func pid=119476) 
(bundle_reservation_check_func pid=119476) During handling of the above exception, another exception occurred:
(bundle_reservation_check_func pid=119476) 
(bundle_reservation_check_func pid=119476) Traceback (most recent call last):
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 2064, in ray._raylet.task_execution_handler
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 1995, in ray._raylet.execute_task_with_cancellation_handler
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 953, in ray._raylet.store_task_errors
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/site-packages/ray/_private/utils.py", line 178, in push_error_to_driver
(bundle_reservation_check_func pid=119476)     worker.core_worker.push_error(job_id, error_type, message, time.time())
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 4527, in ray._raylet.CoreWorker.push_error
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 468, in ray._raylet.check_status
(bundle_reservation_check_func pid=119476) ray.exceptions.RaySystemError: System error: Broken pipe
(bundle_reservation_check_func pid=119476) 
(bundle_reservation_check_func pid=119476) During handling of the above exception, another exception occurred:
(bundle_reservation_check_func pid=119476) 
(bundle_reservation_check_func pid=119476) Traceback (most recent call last):
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 2103, in ray._raylet.task_execution_handler
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/site-packages/ray/_private/utils.py", line 178, in push_error_to_driver
(bundle_reservation_check_func pid=119476)     worker.core_worker.push_error(job_id, error_type, message, time.time())
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 4527, in ray._raylet.CoreWorker.push_error
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 468, in ray._raylet.check_status
(bundle_reservation_check_func pid=119476) ray.exceptions.RaySystemError: System error: Broken pipe
(bundle_reservation_check_func pid=119476) Exception ignored in: 'ray._raylet.task_execution_handler'
(bundle_reservation_check_func pid=119476) Traceback (most recent call last):
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 2103, in ray._raylet.task_execution_handler
(bundle_reservation_check_func pid=119476)   File "/home/ubuntu/miniconda3/envs/raytune_test/lib/python3.11/site-packages/ray/_private/utils.py", line 178, in push_error_to_driver
(bundle_reservation_check_func pid=119476)     worker.core_worker.push_error(job_id, error_type, message, time.time())
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 4527, in ray._raylet.CoreWorker.push_error
(bundle_reservation_check_func pid=119476)   File "python/ray/_raylet.pyx", line 468, in ray._raylet.check_status
(bundle_reservation_check_func pid=119476) ray.exceptions.RaySystemError: System error: Broken pipe

Versions / Dependencies

python==3.11.7 ray==2.8.0

Reproduction script

from ray import tune
from ray.tune.search.basic_variant import BasicVariantGenerator

params = {
    "x1": tune.uniform(-5, 5),
    "x2": tune.uniform(-5, 5),
}

def objective_fn(config):
    fn = (config["x1"] - 2)**2 + (config["x2"] + 3)**2
    return {"fn": fn}

searcher = BasicVariantGenerator()
tune_config = tune.TuneConfig(
    metric="fn",
    mode="min",
    search_alg=searcher,
    num_samples=200,
)

tuner = tune.Tuner(
    objective_fn, 
    tune_config=tune_config, 
    param_space=params
)
results = tuner.fit()

Issue Severity

Medium: It is a significant difficulty but I can work around it.

justinvyu commented 10 months ago

@JosephRRB What's your cluster setup (instance type, number of nodes, etc.)? I am not able to reproduce these errors on a m5.8xlarge.

What's the order of magnitude of the slowdown when moving from non-Ray to Ray Tune?

justinvyu commented 10 months ago

Could you try running with tuner.Tuner(..., tune_config=tune.TuneConfig(reuse_actors=False))?

JosephRRB commented 10 months ago

Thank you for your reply!

We don't have a remote cluster setup so we are only using ray-tune in an instance locally. So we're just using 1 node and our instance type is g4dn.xlarge.

Indeed, using reuse_actors=False removes the errors but the overall speed is slower. Specifically, this is how I timed it:

start = time.time()

params = {
    "x1": tune.uniform(-5, 5),
    "x2": tune.uniform(-5, 5),
}

def objective_fn(config):
    fn = (config["x1"] - 2)**2 + (config["x2"] + 3)**2
    return {"fn": fn}

searcher = BasicVariantGenerator()
tune_config = tune.TuneConfig(
    metric="fn",
    mode="min",
    search_alg=searcher,
    num_samples=200,
    reuse_actors=False
)

tuner = tune.Tuner(
    objective_fn, 
    tune_config=tune_config, 
    param_space=params
)
results = tuner.fit()
elapsed = time.time() - start
print(f"Elapsed: {elapsed} s")

And elapsed=162.90476822853088 s for reuse_actors=False while elapsed=18.27258825302124 s for reuse_actors=True

JosephRRB commented 10 months ago

Additionally, here are some timing results from comparing optuna with ray-tune vs directly using optuna. I tried it for different number of samples

It seems that the gap becomes smaller when more samples are used.

Screenshot from 2024-01-09 11-45-20 Screenshot from 2024-01-09 11-50-32 Screenshot from 2024-01-09 12-10-41

JosephRRB commented 10 months ago

If needed, here's the code comparing optuna with and without ray-tune

params = {
    "x1": tune.uniform(-10, 10),
    "x2": tune.uniform(-10, 10),
}

def parabolic_surface(x1, x2):
    return (x1 - 3)**2 + (x2 - 5)**2

num_samples = 100
start = time.time()

searcher = OptunaSearch(sampler=op.samplers.TPESampler())
tune_config = tune.TuneConfig(
    metric="f(x1,x2)",
    mode="min",
    search_alg=searcher,
    num_samples=num_samples,
)

tuner = tune.Tuner(
    lambda config: {"f(x1,x2)": parabolic_surface(**config)},
    tune_config=tune_config, 
    param_space=params
)
results = tuner.fit()
elapsed = time.time() - start
start = time.time()

def objective(trial):
    x1 = trial.suggest_float("x1", -10, 10)
    x2 = trial.suggest_float("x2", -10, 10)
    return parabolic_surface(x1=x1, x2=x2)

sampler = op.samplers.TPESampler()
study = op.create_study(sampler=sampler, direction="minimize")
study.optimize(objective, n_trials=num_samples)

elapsed = time.time() - start
JosephRRB commented 10 months ago

We ran it on a Mac with python 3.11 and ray 2.6.3 but there's still some issue with the ray worker. It was generated with reuse_actors=False

2024-01-09 17:49:32,362 ERROR tune_controller.py:911 -- Trial task failed for trial objective_fn_57913_00005
Traceback (most recent call last):
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/air/execution/_internal/event_manager.py", line 110, in resolve_future
    result = ray.get(future)
             ^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/auto_init_hook.py", line 24, in auto_init_wrapper
    return fn(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/client_mode_hook.py", line 103, in wrapper
    return func(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/worker.py", line 2526, in get
    raise value
ray.exceptions.RayActorError: The actor died unexpectedly before finishing this task.
    class_name: ImplicitFunc
    actor_id: 81387472d4b38e5cdee6a0d601000000
    namespace: 9c932ba2-96c7-4f5b-9875-384893a5a2a9
The actor is dead because its owner has died. Owner Id: 01000000ffffffffffffffffffffffffffffffffffffffffffffffff Owner Ip address: 127.0.0.1 Owner worker exit type: SYSTEM_ERROR Worker exit detail: Owner's node has crashed.
The actor never ran - it was cancelled before it started running.
2024-01-09 17:49:32,368 ERROR tune_controller.py:911 -- Trial task failed for trial objective_fn_57913_00006
Traceback (most recent call last):
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/air/execution/_internal/event_manager.py", line 110, in resolve_future
    result = ray.get(future)
             ^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/auto_init_hook.py", line 24, in auto_init_wrapper
    return fn(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/client_mode_hook.py", line 103, in wrapper
    return func(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/worker.py", line 2526, in get
    raise value
ray.exceptions.RayActorError: The actor died unexpectedly before finishing this task.
    class_name: ImplicitFunc
    actor_id: f3d74f7bcce8e9f60db1465e01000000
    namespace: 9c932ba2-96c7-4f5b-9875-384893a5a2a9
The actor is dead because its node has died. Node Id: 5d11b677f340fc9012ce947e1d4bf5e1c79aa76095c93a28c4b5ae53
The actor never ran - it was cancelled before it started running.
2024-01-09 17:49:32,371 ERROR tune_controller.py:911 -- Trial task failed for trial objective_fn_57913_00001
Traceback (most recent call last):
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/air/execution/_internal/event_manager.py", line 110, in resolve_future
    result = ray.get(future)
             ^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/auto_init_hook.py", line 24, in auto_init_wrapper
    return fn(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/client_mode_hook.py", line 103, in wrapper
    return func(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/worker.py", line 2526, in get
    raise value
ray.exceptions.RayActorError: The actor died unexpectedly before finishing this task.
    class_name: ImplicitFunc
    actor_id: 776fa51d17f2438142a34dfc01000000
    namespace: 9c932ba2-96c7-4f5b-9875-384893a5a2a9
The actor is dead because its node has died. Node Id: 5d11b677f340fc9012ce947e1d4bf5e1c79aa76095c93a28c4b5ae53
The actor never ran - it was cancelled before it started running.
2024-01-09 17:49:32,375 ERROR tune_controller.py:911 -- Trial task failed for trial objective_fn_57913_00003
Traceback (most recent call last):
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/air/execution/_internal/event_manager.py", line 110, in resolve_future
    result = ray.get(future)
             ^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/auto_init_hook.py", line 24, in auto_init_wrapper
    return fn(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/client_mode_hook.py", line 103, in wrapper
    return func(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/worker.py", line 2526, in get
    raise value
ray.exceptions.RayActorError: The actor died unexpectedly before finishing this task.
    class_name: ImplicitFunc
    actor_id: a784953ed942f99f31fd311e01000000
    namespace: 9c932ba2-96c7-4f5b-9875-384893a5a2a9
The actor is dead because its node has died. Node Id: 5d11b677f340fc9012ce947e1d4bf5e1c79aa76095c93a28c4b5ae53
The actor never ran - it was cancelled before it started running.
2024-01-09 17:49:32,379 ERROR tune_controller.py:911 -- Trial task failed for trial objective_fn_57913_00002
Traceback (most recent call last):
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/air/execution/_internal/event_manager.py", line 110, in resolve_future
    result = ray.get(future)
             ^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/auto_init_hook.py", line 24, in auto_init_wrapper
    return fn(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/client_mode_hook.py", line 103, in wrapper
    return func(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/worker.py", line 2526, in get
    raise value
ray.exceptions.RayActorError: The actor died unexpectedly before finishing this task.
    class_name: ImplicitFunc
    actor_id: dabfadb2b8f83dcafcd5757301000000
    namespace: 9c932ba2-96c7-4f5b-9875-384893a5a2a9
The actor is dead because its node has died. Node Id: 5d11b677f340fc9012ce947e1d4bf5e1c79aa76095c93a28c4b5ae53
The actor never ran - it was cancelled before it started running.
2024-01-09 17:49:32,383 ERROR tune_controller.py:911 -- Trial task failed for trial objective_fn_57913_00007
Traceback (most recent call last):
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/air/execution/_internal/event_manager.py", line 110, in resolve_future
    result = ray.get(future)
             ^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/auto_init_hook.py", line 24, in auto_init_wrapper
    return fn(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/client_mode_hook.py", line 103, in wrapper
    return func(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/worker.py", line 2526, in get
    raise value
ray.exceptions.RayActorError: The actor died unexpectedly before finishing this task.
    class_name: ImplicitFunc
    actor_id: c22dfa77030b7162e206fb6901000000
    namespace: 9c932ba2-96c7-4f5b-9875-384893a5a2a9
The actor is dead because its node has died. Node Id: 5d11b677f340fc9012ce947e1d4bf5e1c79aa76095c93a28c4b5ae53
The actor never ran - it was cancelled before it started running.
(raylet) [2024-01-09 17:49:32,358 E 64916 10149657] (raylet) agent_manager.cc:135: The raylet exited immediately because the Ray agent failed. The raylet fate shares with the agent. This can happen because the Ray agent was unexpectedly killed or failed. Agent can fail when
(raylet) - The version of `grpcio` doesn't follow Ray's requirement. Agent can segfault with the incorrect `grpcio` version. Check the grpcio version `pip freeze | grep grpcio`.
(raylet) - The agent failed to start because of unexpected error or port conflict. Read the log `cat /tmp/ray/session_latest/logs/dashboard_agent.log`. You can find the log file structure here https://docs.ray.io/en/master/ray-observability/ray-logging.html#logging-directory-structure.
(raylet) - The agent is killed by the OS (e.g., out of memory).
2024-01-09 17:49:32,387 ERROR tune_controller.py:911 -- Trial task failed for trial objective_fn_57913_00004
Traceback (most recent call last):
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/air/execution/_internal/event_manager.py", line 110, in resolve_future
    result = ray.get(future)
             ^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/auto_init_hook.py", line 24, in auto_init_wrapper
    return fn(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/client_mode_hook.py", line 103, in wrapper
    return func(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/worker.py", line 2526, in get
    raise value
ray.exceptions.RayActorError: The actor died unexpectedly before finishing this task.
    class_name: ImplicitFunc
    actor_id: 65ed5c48e50903aba4a2a8c201000000
    namespace: 9c932ba2-96c7-4f5b-9875-384893a5a2a9
The actor is dead because its node has died. Node Id: 5d11b677f340fc9012ce947e1d4bf5e1c79aa76095c93a28c4b5ae53
The actor never ran - it was cancelled before it started running.
2024-01-09 17:49:32,391 ERROR tune_controller.py:911 -- Trial task failed for trial objective_fn_57913_00000
Traceback (most recent call last):
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/air/execution/_internal/event_manager.py", line 110, in resolve_future
    result = ray.get(future)
             ^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/auto_init_hook.py", line 24, in auto_init_wrapper
    return fn(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/client_mode_hook.py", line 103, in wrapper
    return func(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^^^
  File "/Users/james.park/anaconda3/envs/raytune-test/lib/python3.11/site-packages/ray/_private/worker.py", line 2526, in get
    raise value
ray.exceptions.RayActorError: The actor died unexpectedly before finishing this task.
    class_name: ImplicitFunc
    actor_id: 504963e871c28bfa14fd0c6201000000
    namespace: 9c932ba2-96c7-4f5b-9875-384893a5a2a9
The actor is dead because its node has died. Node Id: 5d11b677f340fc9012ce947e1d4bf5e1c79aa76095c93a28c4b5ae53
The actor never ran - it was cancelled before it started running.
2024-01-09 17:49:37,190 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:49:37,694 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:49:38,200 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:49:38,708 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:49:39,214 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:49:42,239 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:49:42,744 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:49:43,249 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:49:43,754 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:49:44,260 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:49:47,280 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:49:47,784 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:49:48,289 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:49:48,793 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:49:49,300 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:49:52,317 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:49:52,824 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:49:53,331 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:49:53,834 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:49:54,340 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:49:57,342 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:49:57,846 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:49:58,349 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:49:58,853 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:49:59,357 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:50:02,386 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:50:02,890 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:50:03,395 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:50:03,897 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:50:04,401 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:50:07,417 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:50:07,921 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:50:08,424 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:50:08,930 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:50:09,434 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:50:12,452 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:50:12,958 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:50:13,464 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:50:13,969 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:50:14,473 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:50:17,480 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:50:17,985 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:50:18,492 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:50:18,997 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:50:19,502 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:50:22,514 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:50:23,019 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:50:23,523 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:50:24,028 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:50:24,532 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:50:27,527 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:50:28,029 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:50:28,537 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:50:29,043 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:50:29,549 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:50:32,573 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:50:33,077 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:50:33,582 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:50:34,089 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:50:34,590 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:50:37,599 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:50:38,104 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:50:38,607 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:50:39,113 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:50:39,618 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:50:42,659 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:50:43,166 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:50:43,677 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:50:44,194 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:50:44,699 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:50:47,706 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:50:48,209 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:50:48,720 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:50:49,224 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:50:49,732 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:50:52,795 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:50:53,300 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:50:53,807 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:50:54,314 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:50:54,818 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:50:57,839 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:50:58,344 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:50:58,850 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:50:59,355 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:50:59,859 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:51:02,886 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:51:03,392 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:51:03,897 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:51:04,400 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:51:04,907 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:51:07,957 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:51:08,465 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:51:08,969 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:51:09,474 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:51:09,982 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:51:13,010 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...
2024-01-09 17:51:13,518 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #3...
2024-01-09 17:51:14,024 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #4...
2024-01-09 17:51:14,528 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #5...
2024-01-09 17:51:15,033 WARNING resource_updater.py:275 -- Cluster resources cannot be detected or are 0. You can resume this experiment by passing in `resume=True` to `run`.
2024-01-09 17:51:18,059 WARNING resource_updater.py:262 -- Cluster resources not detected or are 0. Attempt #2...

Screenshot from 2024-01-10 10-28-50 Screenshot from 2024-01-10 10-29-05