- 
                Notifications
    
You must be signed in to change notification settings  - Fork 6.9k
 
Description
What happened + What you expected to happen
I run a simple hyperparameters optimisation (backtesting) using Ray Tune and Optuna. Tasks are very short (1-3 seconds long, 1-5 trials per iteration with early termination, 300-1500 iterations per backtest). The simplified code is attached below, but the problem is not with the code but with how Ray handles its own internal errors. Everything was working very stable until about Ray v. 2.4, and since then I observe this problematic behaviour on each version, including current nightly wheels. I used to stay on v 2.3 because of this, but with recent upgrade to Python 3.11 I cannot do that any longer, hence this report.
During the trials occasionally (like once in 10k trials) a raylet crashes with unexplained error related to Ray serialization mechanisms, but other raylets continue and all trials eventually end, moving on to the next backtest.
Once in a while however, main Ray process gets stuck with the following error, i.e. instead of exiting Ray hangs until I notice it and press Ctrl-C, which blocks the whole pipeline. I found no setting to let Ray simply quit in such case. In the error message  below Ray offers to set max_retries=0 to avoid resubmitting, but I could not find a way to pass this parameter to a worker defined as a function instead of a class (i.e. cannot use a @ray.remote() decorator). As a last resort, I tried modifying Ray source code directly to change DEFAULT_TASK_MAX_RETRIES to 0: 
ray/python/ray/_private/ray_constants.py
Line 425 in 27a88ba
| DEFAULT_TASK_MAX_RETRIES = 3 | 
I guess my main question is not how to avoid this error but rather how to recover from it and move on or at least exit main process even while sacrificing some trials?
Ray is run on powerful baremetal servers w/o Docker, i.e. there's enough ram/cpu/disk space without external constraints. I run the code with same Ray/Python versions on multiple machines via separate ray instances or in cluster mode, and still can encounter this problematic behaviour regardless of execution mode, i.e. networking problems are out of the question also.
(raylet) [2023-11-29 07:14:39,972 C 3822814 3822814] (raylet) node_manager.cc:1413:  Check failed: worker
(raylet) *** StackTrace Information ***
(raylet) /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xb7c11a) [0x562863b0011a] ray::operator<<()
(raylet) /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xb7d8d7) [0x562863b018d7] ray::SpdLogMessage::Flush()
(raylet) /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xb7dd77) [0x562863b01d77] ray::RayLog::~RayLog()
(raylet) /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x2ca733) [0x56286324e733] ray::raylet::NodeManager::HandleWorkerAvailable()
(raylet) /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x2f1b0b) [0x562863275b0b] ray::raylet::NodeManager::ProcessClientMessage()
(raylet) /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x30ce11) [0x562863290e11] std::_Function_handler<>::_M_invoke()
(raylet) /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x59e97d) [0x56286352297d] ray::ClientConnection::ProcessMessage()
(raylet) /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x5ae6ae) [0x5628635326ae] EventTracker::RecordExecution()
(raylet) /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x595652) [0x562863519652] boost::asio::detail::binder2<>::operator()()
(raylet) /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x595da8) [0x562863519da8] boost::asio::detail::reactive_socket_recv_op<>::do_complete()
(raylet) /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xc5d99b) [0x562863be199b] boost::asio::detail::scheduler::do_run_one()
(raylet) /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xc5ff29) [0x562863be3f29] boost::asio::detail::scheduler::run()
(raylet) /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xc60442) [0x562863be4442] boost::asio::io_context::run()
(raylet) /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x1ce9ba) [0x5628631529ba] main
(raylet) /lib/x86_64-linux-gnu/libc.so.6(__libc_start_main+0x80) [0x7f299d229e40] __libc_start_main
(backtest_rungs pid=3824218) [2023-11-29 07:14:40,215 C 3824218 3824218] core_worker.cc:2666:  Check failed: _s.ok() Bad status: IOError: Broken pipe
(backtest_rungs pid=3824218) /home/ubuntu/.local/lib/python3.11/site-packages/ray/_raylet.so(_ZN3ray4core10CoreWorker11ExecuteTaskERKNS_17TaskSpecificationERKSt10shared_ptrISt13unordered_mapISsSt6vectorISt4pairIldESaIS9_EESt4hashISsESt8equal_toISsESaIS8_IKSsSB_EEEEPS7_IS8_INS_8ObjectIDES5_INS_9RayObjectEEESaISQ_EEST_PS7_IS8_ISN_bESaISU_EEPN6google8protobuf16RepeatedPtrFieldINS_3rpc20ObjectReferenceCountEEEPbPSs+0x6cb) [0x7fc41a176fab] ray::core::CoreWorker::ExecuteTask()
(backtest_rungs pid=3824218) /home/ubuntu/.local/lib/python3.11/site-packages/ray/_raylet.so(+0x7b575e) [0x7fc41a1b575e] ray::core::InboundRequest::Accept()
(backtest_rungs pid=3824218) /home/ubuntu/.local/lib/python3.11/site-packages/ray/_raylet.so(_ZN3ray4core20ActorSchedulingQueue31AcceptRequestOrRejectIfCanceledENS_6TaskIDERNS0_14InboundRequestE+0x114) [0x7fc41a1b6774] ray::core::ActorSchedulingQueue::AcceptRequestOrRejectIfCanceled()
(backtest_rungs pid=3824218) /home/ubuntu/.local/lib/python3.11/site-packages/ray/_raylet.so(+0x7b939b) [0x7fc41a1b939b] ray::core::ActorSchedulingQueue::ScheduleRequests()
(backtest_rungs pid=3824218) /home/ubuntu/.local/lib/python3.11/site-packages/ray/_raylet.so(_ZN3ray4core20ActorSchedulingQueue3AddEllSt8functionIFvS2_IFvNS_6StatusES2_IFvvEES5_EEEES2_IFvRKS3_S7_EES7_RKSsRKSt10shared_ptrINS_27FunctionDescriptorInterfaceEENS_6TaskIDERKSt6vectorINS_3rpc15ObjectReferenceESaISO_EE+0x400) [0x7fc41a1baeb0] ray::core::ActorSchedulingQueue::Add()
(backtest_rungs pid=3824218) /home/ubuntu/.local/lib/python3.11/site-packages/ray/_raylet.so(_ZN3ray4core28CoreWorkerDirectTaskReceiver10HandleTaskERKNS_3rpc15PushTaskRequestEPNS2_13PushTaskReplyESt8functionIFvNS_6StatusES8_IFvvEESB_EE+0x119c) [0x7fc41a19d3cc] ray::core::CoreWorkerDirectTaskReceiver::HandleTask()
(backtest_rungs pid=3824218) /home/ubuntu/.local/lib/python3.11/site-packages/ray/_raylet.so(+0xa033f6) [0x7fc41a4033f6] boost::asio::detail::completion_handler<>::do_complete()
(backtest_rungs pid=3824218) /home/ubuntu/.local/lib/python3.11/site-packages/ray/_raylet.so(_ZN3ray4core10CoreWorker20RunTaskExecutionLoopEv+0xcd) [0x7fc41a14026d] ray::core::CoreWorker::RunTaskExecutionLoop()
(backtest_rungs pid=3824218) /home/ubuntu/.local/lib/python3.11/site-packages/ray/_raylet.so(_ZN3ray4core21CoreWorkerProcessImpl26RunWorkerTaskExecutionLoopEv+0x8c) [0x7fc41a1824fc] ray::core::CoreWorkerProcessImpl::RunWorkerTaskExecutionLoop()
(backtest_rungs pid=3824218) /home/ubuntu/.local/lib/python3.11/site-packages/ray/_raylet.so(_ZN3ray4core17CoreWorkerProcess20RunTaskExecutionLoopEv+0x1d) [0x7fc41a1826ad] ray::core::CoreWorkerProcess::RunTaskExecutionLoop()
(backtest_rungs pid=3824218) ray::ImplicitFunc() [0x56337d]
(backtest_rungs pid=3824218) ray::ImplicitFunc(PyObject_Vectorcall+0x35) [0x547c35] PyObject_Vectorcall
(backtest_rungs pid=3824218) ray::ImplicitFunc(_PyEval_EvalFrameDefault+0x6d8) [0x53a5a8] _PyEval_EvalFrameDefault
(backtest_rungs pid=3824218) ray::ImplicitFunc() [0x60ee54]
(backtest_rungs pid=3824218) ray::ImplicitFunc(PyEval_EvalCode+0x97) [0x60e4b7] PyEval_EvalCode
(backtest_rungs pid=3824218) ray::ImplicitFunc() [0x62f61b]
(backtest_rungs pid=3824218) ray::ImplicitFunc() [0x62bca4]
(backtest_rungs pid=3824218) ray::ImplicitFunc() [0x640005]
(backtest_rungs pid=3824218) ray::ImplicitFunc(_PyRun_SimpleFileObject+0x194) [0x63f644] _PyRun_SimpleFileObject
(backtest_rungs pid=3824218) ray::ImplicitFunc(_PyRun_AnyFileObject+0x47) [0x63f457] _PyRun_AnyFileObject
(backtest_rungs pid=3824218) ray::ImplicitFunc(Py_RunMain+0x2c9) [0x639d69] Py_RunMain
(backtest_rungs pid=3824218) ray::ImplicitFunc(Py_BytesMain+0x2d) [0x5fe3ed] Py_BytesMain
(backtest_rungs pid=3824218) ray::ImplicitFunc(_start+0x25) [0x5fe275] _start
(raylet) Raylet is terminated. Termination is unexpected. Possible reasons include: (1) SIGKILL by the user or system OOM killer, (2) Invalid memory access from Raylet causing SIGSEGV or SIGBUS, (3) Other termination signals. Last 20 lines of the Raylet logs:
    *** StackTrace Information ***
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xb7c11a) [0x562863b0011a] ray::operator<<()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xb7d8d7) [0x562863b018d7] ray::SpdLogMessage::Flush()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xb7dd77) [0x562863b01d77] ray::RayLog::~RayLog()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x2ca733) [0x56286324e733] ray::raylet::NodeManager::HandleWorkerAvailable()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x2ca7bd) [0x56286324e7bd] ray::raylet::NodeManager::HandleWorkerAvailable()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x2f1b0b) [0x562863275b0b] ray::raylet::NodeManager::ProcessClientMessage()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x30ce11) [0x562863290e11] std::_Function_handler<>::_M_invoke()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x59e97d) [0x56286352297d] ray::ClientConnection::ProcessMessage()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x5ae6ae) [0x5628635326ae] EventTracker::RecordExecution()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x595652) [0x562863519652] boost::asio::detail::binder2<>::operator()()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x595da8) [0x562863519da8] boost::asio::detail::reactive_socket_recv_op<>::do_complete()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xc5d99b) [0x562863be199b] boost::asio::detail::scheduler::do_run_one()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xc5ff29) [0x562863be3f29] boost::asio::detail::scheduler::run()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xc60442) [0x562863be4442] boost::asio::io_context::run()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x1ce9ba) [0x5628631529ba] main
    /lib/x86_64-linux-gnu/libc.so.6(+0x29d90) [0x7f299d229d90]
    /lib/x86_64-linux-gnu/libc.so.6(__libc_start_main+0x80) [0x7f299d229e40] __libc_start_main
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x221d67) [0x5628631a5d67]
Traceback (most recent call last):
  File "/home/ubuntu/jesse-bot/backtest_optuna.py", line 855, in <module>
    ])), param_space=hyperparams, tune_config=tune_config, run_config=run_config).fit()
                                                                                  ^^^^^
  File "/home/ubuntu/.local/lib/python3.11/site-packages/ray/tune/tuner.py", line 381, in fit
    return self._local_tuner.fit()
           ^^^^^^^^^^^^^^^^^^^^^^^
  File "/home/ubuntu/.local/lib/python3.11/site-packages/ray/tune/impl/tuner_internal.py", line 509, in fit
    analysis = self._fit_internal(trainable, param_space)
               ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/home/ubuntu/.local/lib/python3.11/site-packages/ray/tune/impl/tuner_internal.py", line 628, in _fit_internal
    analysis = run(
               ^^^^
  File "/home/ubuntu/.local/lib/python3.11/site-packages/ray/tune/tune.py", line 1007, in run
    _report_progress(runner, progress_reporter)
  File "/home/ubuntu/.local/lib/python3.11/site-packages/ray/tune/tune.py", line 168, in _report_progress
    reporter.report(trials, done, sched_debug_str, used_resources_str)
  File "/home/ubuntu/.local/lib/python3.11/site-packages/ray/tune/progress_reporter.py", line 702, in report
    self._print(self._progress_str(trials, done, *sys_info))
                ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/home/ubuntu/.local/lib/python3.11/site-packages/ray/tune/progress_reporter.py", line 357, in _progress_str
    current_best_trial, metric = self._current_best_trial(trials)
                                 ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/home/ubuntu/.local/lib/python3.11/site-packages/ray/tune/progress_reporter.py", line 421, in _current_best_trial
    if not t.last_result:
           ^^^^^^^^^^^^^
  File "/home/ubuntu/.local/lib/python3.11/site-packages/ray/tune/experiment/trial.py", line 496, in last_result
    self._get_default_result_or_future()
  File "/home/ubuntu/.local/lib/python3.11/site-packages/ray/tune/experiment/trial.py", line 464, in _get_default_result_or_future
    self._default_result_or_future = ray.get(self._default_result_or_future)
                                     ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/home/ubuntu/.local/lib/python3.11/site-packages/ray/_private/auto_init_hook.py", line 24, in auto_init_wrapper
    return fn(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^
  File "/home/ubuntu/.local/lib/python3.11/site-packages/ray/_private/client_mode_hook.py", line 103, in wrapper
    return func(*args, **kwargs)
           ^^^^^^^^^^^^^^^^^^^^^
  File "/home/ubuntu/.local/lib/python3.11/site-packages/ray/_private/worker.py", line 2589, in get
    values, debugger_breakpoint = worker.get_objects(object_refs, timeout=timeout)
                                  ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "/home/ubuntu/.local/lib/python3.11/site-packages/ray/_private/worker.py", line 794, in get_objects
    data_metadata_pairs = self.core_worker.get_objects(
                          ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
  File "python/ray/_raylet.pyx", line 3222, in ray._raylet.CoreWorker.get_objects
  File "python/ray/_raylet.pyx", line 469, in ray._raylet.check_status
ray.exceptions.RaySystemError: System error: Broken pipe
(raylet) The node with node id: aed161029c52714812867ce5d16b0e6949e216a40860a47b3dbabf90 and address: 213.239.194.52 and node name: 213.239.194.52 has been marked dead because the detector has missed too many heartbeats from it. This can happen when a      (1) raylet crashes unexpectedly (OOM, preempted node, etc.)
        (2) raylet has lagging heartbeats due to slow network or busy workload.
(raylet) Raylet is terminated. Termination is unexpected. Possible reasons include: (1) SIGKILL by the user or system OOM killer, (2) Invalid memory access from Raylet causing SIGSEGV or SIGBUS, (3) Other termination signals. Last 20 lines of the Raylet logs:
    *** StackTrace Information ***
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xb7c11a) [0x562863b0011a] ray::operator<<()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xb7d8d7) [0x562863b018d7] ray::SpdLogMessage::Flush()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xb7dd77) [0x562863b01d77] ray::RayLog::~RayLog()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x2ca733) [0x56286324e733] ray::raylet::NodeManager::HandleWorkerAvailable()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x2ca7bd) [0x56286324e7bd] ray::raylet::NodeManager::HandleWorkerAvailable()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x2f1b0b) [0x562863275b0b] ray::raylet::NodeManager::ProcessClientMessage()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x30ce11) [0x562863290e11] std::_Function_handler<>::_M_invoke()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x59e97d) [0x56286352297d] ray::ClientConnection::ProcessMessage()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x5ae6ae) [0x5628635326ae] EventTracker::RecordExecution()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x595652) [0x562863519652] boost::asio::detail::binder2<>::operator()()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x595da8) [0x562863519da8] boost::asio::detail::reactive_socket_recv_op<>::do_complete()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xc5d99b) [0x562863be199b] boost::asio::detail::scheduler::do_run_one()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xc5ff29) [0x562863be3f29] boost::asio::detail::scheduler::run()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0xc60442) [0x562863be4442] boost::asio::io_context::run()
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x1ce9ba) [0x5628631529ba] main
    /lib/x86_64-linux-gnu/libc.so.6(+0x29d90) [0x7f299d229d90]
    /lib/x86_64-linux-gnu/libc.so.6(__libc_start_main+0x80) [0x7f299d229e40] __libc_start_main
    /home/ubuntu/.local/lib/python3.11/site-packages/ray/core/src/ray/raylet/raylet(+0x221d67) [0x5628631a5d67]
[2023-11-29 07:14:52,933 E 3822625 3822899] core_worker.cc:593: :info_message: Attempting to recover 15 lost objects by resubmitting their tasks. To disable object reconstruction, set @ray.remote(max_retries=0).
(hangs indefinitely)  🫤Versions / Dependencies
As I've been experiencing this behaviour with current 2.8.0 release (and all releases after 2.3.0), so I tried installing Ray from nightly wheels via
 pip install -U "ray @ https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-3.0.0.dev0-cp311-cp311-manylinux2014_x86_64.whl"
The result is the same unfortunately.
✗ ray --version
ray, version 3.0.0.dev0
✗ python --version
Python 3.11.6
✗ uname -a
Linux backtest 6.5.0-1008-oem #8-Ubuntu SMP PREEMPT_DYNAMIC Fri Nov 10 13:08:33 UTC 2023 x86_64 x86_64 x86_64 GNU/Linux
Reproduction script
(not a full reproduction unfortunatelly, as it calls external storage to fetch candles and run backtest function. Once again, it works fine 99% of the time, but when it hangs, I cannot find a way to recover from the hang.
Please note that I've disabled checkpoints, but with checkpoints the behaviour was the same.
hyperparams =  {
        'ma_period': tune.qrandint(45, 205, 20),
        'min_slope': tune.quniform(0.1, 0.6, 0.05),
        'period': tune.qrandint(5, 205, 20),
        'lookback': tune.qrandint(2, 5, 1),
        'tp_pc': tune.quniform(0.6, 1.2, 0.3),
        'sl_pc': tune.quniform(3, 6, 1),
        'tf': tune.choice(timeframes)
    }
optuna = OptunaSearch(
    metric="avg_smart_sharpe",
    mode="max",
    sampler=optuna.samplers.QMCSampler(seed=42, warn_independent_sampling = False),
    )
def backtest_rungs(params):
    try:
        for candles_ref in random.sample(test_candles_refs, trial_length):
            result = backtest(config=ray.get(config_ref),  
                            routes=ray.get(routes_refs[params["tf"]]))
            train.report(result['metrics'])
    except Exception as e:
        print("Exception in worker:", e)
        ray.shutdown()
tune_config = tune.TuneConfig(num_samples=iterations, reuse_actors=True, search_alg=optuna, metric="avg_smart_sharpe", mode="max")
run_config = train.RunConfig(
    name=spreadsheet_title,
    storage_path="/mnt/air/",
    stop=terminator,
    callbacks=[google_sheets_logger(symbol, strategy, timeframes, exchange, pre_candles, live_candles, realtime_candles, spreadsheet_title, test_dates, trial_length, extra, strategy_name, ft_note)],
    failure_config=train.FailureConfig(fail_fast=False,max_failures=0)
)
results = tune.Tuner(backtest_rungs, param_space=hyperparams, tune_config=tune_config, run_config=run_config).fit()Issue Severity
High: It blocks me from completing my task.