Skip to content

[core] Creating exponential backoff for retryable grpcs #56568

Merged
jjyao merged 24 commits intoray-project:masterfrom
Sparks0219:joshlee/disconnect-raylet-client-pool-on-node-death
Oct 16, 2025
Merged

[core] Creating exponential backoff for retryable grpcs #56568
jjyao merged 24 commits intoray-project:masterfrom
Sparks0219:joshlee/disconnect-raylet-client-pool-on-node-death

Conversation

@Sparks0219
Copy link
Copy Markdown
Contributor

@Sparks0219 Sparks0219 commented Sep 16, 2025

Why are these changes needed?

Creating an exponential backoff in retryable grpcs and build up to 60 seconds to detect node death faster.

Related issue number

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

@Sparks0219 Sparks0219 added the go add ONLY when ready to merge, run all tests label Sep 16, 2025
@Sparks0219 Sparks0219 changed the title [core] Creating exponential backoff + disconnect raylets in pool on node death [core] Creating exponential backoff Sep 16, 2025
@Sparks0219 Sparks0219 changed the title [core] Creating exponential backoff [core] Creating exponential backoff for retryable grpcs Sep 16, 2025
Signed-off-by: joshlee <joshlee@anyscale.com>
@Sparks0219 Sparks0219 requested review from dayshah and jjyao September 16, 2025 05:57
@Sparks0219 Sparks0219 marked this pull request as ready for review September 16, 2025 05:57
@Sparks0219 Sparks0219 requested a review from a team as a code owner September 16, 2025 05:57
@Sparks0219
Copy link
Copy Markdown
Contributor Author

Where's my AI comments :(

@ray-gardener ray-gardener bot added the core Issues that should be addressed in Ray Core label Sep 16, 2025
RAY_LOG(WARNING) << server_name_ << " has been unavailable for more than "
<< server_unavailable_timeout_seconds_ << " seconds";
<< (attempt_number > 0 ? ExponentialBackoff::GetBackoffMs(
attempt_number - 1,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why the -1 and +1 after?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah this one got a little complicated because the first retry for a request can be set in two spots:

If the retryable grpc queue is empty/idle we set it here:

// First request to retry.

Otherwise we set it here:
server_unavailable_timeout_time_ =

Hence I need to bump the attempt number before I push it back into the queue otherwise there's no way for me to tell whether this should be bumped to 1 or its the first retry and I should use 0. So since I bump it before I push it in the queue, the prior time is attempt_number - 1, the current one is attempt_number, and the next one is attempt_number + 1.


uint32_t GetAttemptNumber() const { return attempt_number_; }

void SetAttemptNumber(uint32_t attempt_number) { attempt_number_ = attempt_number; }
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

isn't this just accessed from within the class so we don't need getter/setter

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is accessed with the parent class RetryableGrpcClient, so we still need the getters/setters to access members within the nested class RetryableGrpcRequest.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

wait the whole RetryableGrpcRequest class is private inside RetryableGrpcClient, i feel like there's no point to the encapsulation

But nvm that, I think attempt number should live on the client instead of on the request. I think we talked about this before but the "attempt #" we base the backoff on is technically the client-wide

Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: joshlee <joshlee@anyscale.com>
Comment on lines +167 to +173
if (server_name_ == "GCS") {
server_unavailable_timeout_time_ =
now + absl::Seconds(server_unavailable_base_timeout_seconds_);
SetupCheckTimer();
} else {
server_unavailable_timeout_time_ = now;
CheckChannelStatus();
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this looks really hacky. i'm probably missing some context. why is the behavior different for GCS and why are we matching against a magic string to implement it?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yea I agree this is hacky, but I think the behavior has to be different for the gcs vs the raylet/core worker client pool.
The motivation behind this PR is to remove all the individual timeouts for the raylet/core worker client pool. I had to reduce the timeout to sub 1s for a couple of tests, hence I had the raylet/core worker client pool server timeouts be initially 100ms. After talking to @dayshah we decided it would be cleaner to put it as 1s, but for the first retry trigger the default_unavailable_timeout_callback immediately to detect node death ASAP and stop retries. I can't trigger this for the gcs default unavailable timeout callback though, because it doesn't have any retries and just kills the process without waiting for gcs to potentially restart:

retryable_grpc_client_ = RetryableGrpcClient::Create(
channel_,
client_call_manager.GetMainService(),
/*max_pending_requests_bytes=*/
::RayConfig::instance().gcs_grpc_max_request_queued_max_bytes(),
/*check_channel_status_interval_milliseconds=*/
::RayConfig::instance()
.grpc_client_check_connection_status_interval_milliseconds(),
/*server_unavailable_timeout_seconds=*/
::RayConfig::instance().gcs_rpc_server_reconnect_timeout_s(),
/*server_unavailable_timeout_callback=*/
[]() {
RAY_LOG(ERROR) << "Failed to connect to GCS within "
<< ::RayConfig::instance().gcs_rpc_server_reconnect_timeout_s()
<< " seconds. "
<< "GCS may have been killed. It's either GCS is terminated by "
"`ray stop` or "
<< "is killed unexpectedly. If it is killed unexpectedly, "
<< "see the log file gcs_server.out. "
<< "https://docs.ray.io/en/master/ray-observability/user-guides/"
"configure-logging.html#logging-directory-structure. "
<< "The program will terminate.";
std::_Exit(EXIT_FAILURE);
},
/*server_name=*/"GCS");

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok I get the motivation now. instead of hard coding a match against the server name string, let's make this an explicit param to the retryable client that is passed by all of the components, like: first_server_unavailable_timeout=0 or call_first_unavailable_callback_immediately=true

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1. We can just pass in exponential backoff configs explicitly: initial delay, exponential factor, max delay.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sounds good, added:
call_first_unavailable_timeout_callback_immediately_
exponential_factor_
max_backoff_seconds_
as explicit parameters to the RetryableGrpcClient. Initial delay should be covered by the *_server_reconnect_base_timeout_s already

Comment on lines +88 to +91
? ExponentialBackoff::GetBackoffMs(
attempt_number - 1,
server_unavailable_base_timeout_seconds_ * 1000) /
1000.0
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this should be capped, else the timeout will approach infinity quickly

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

gotcha, I'll add a cap of 60 seconds

Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: joshlee <joshlee@anyscale.com>
uint64_t max_pending_requests_bytes,
uint64_t check_channel_status_interval_milliseconds,
uint64_t server_unavailable_timeout_seconds,
uint64_t server_unavailable_base_timeout_seconds,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we just need three configs:

server_unavailable_timeout_initial_seconds, server_unavailable_timeout_max_seconds, and server_unavailable_timeout_exponential_factor?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How I thought of call_first_unavailable_timeout_callback_immediately to be is to add an extra iteration of 0 in front of the sequence, so we'll start off at:
0 and trigger the channel check-> server_unavailable_base_timeout_seconds -> 2 * server_unavailable_base_timeout_seconds...

Otherwise it it's false then we have the normal sequence of:
server_unavailable_base_timeout_seconds -> 2 * server_unavailable_base_timeout_seconds -> ...

If we just set server_unavailable_base_timeout_seconds to 0 then its a question of what is getting multiplied by the exponential factor. I could hardcode it to 1s, but think that would be confusing.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

server_unavailable_timeout_exponential_factor is always going to be the same, if it doesn't need to be a param don't make it a param imo, saves an extra hop when reading the code. And we're adding an extra default param that's never set to GetBackoffMilliseconds now

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yea good point... I ended end just keep server_unavailable_max_timeout_seconds since
server_unavailable_timeout_initial_seconds is not needed after: #56613

SetupCheckTimer();
if (call_first_unavailable_timeout_callback_immediately_) {
server_unavailable_timeout_time_ = now;
CheckChannelStatus();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit but can you make reset_timer not true as a default parameter, imo easier to read if you don't have to lookup the not obvious default param

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yea good point, I ended up just removing the default parameter since its literally only used in one place


uint32_t GetAttemptNumber() const { return attempt_number_; }

void SetAttemptNumber(uint32_t attempt_number) { attempt_number_ = attempt_number; }
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

wait the whole RetryableGrpcRequest class is private inside RetryableGrpcClient, i feel like there's no point to the encapsulation

But nvm that, I think attempt number should live on the client instead of on the request. I think we talked about this before but the "attempt #" we base the backoff on is technically the client-wide

uint64_t max_pending_requests_bytes,
uint64_t check_channel_status_interval_milliseconds,
uint64_t server_unavailable_timeout_seconds,
uint64_t server_unavailable_base_timeout_seconds,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

server_unavailable_timeout_exponential_factor is always going to be the same, if it doesn't need to be a param don't make it a param imo, saves an extra hop when reading the code. And we're adding an extra default param that's never set to GetBackoffMilliseconds now

@Sparks0219
Copy link
Copy Markdown
Contributor Author

But nvm that, I think attempt number should live on the client instead of on the request. I think we talked about this before but the "attempt #" we base the backoff on is technically the client-wide

Definitely agree with you on this, moved it toe the client class and reset back to 0 when the channel is back online so its shared among different request types, only unique among nodes which it should be.

Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: joshlee <joshlee@anyscale.com>
@Sparks0219 Sparks0219 requested review from dayshah and jjyao September 18, 2025 08:03
Copy link
Copy Markdown
Collaborator

@edoakes edoakes left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

One question that is only now coming to mind -- what client implementations actually rely on the server_unavailable_callback_ being called multiple times for a given channel? And how do they handle subsequent callbacks after the first? I would expect that one notification per state change would be sufficient.

Signed-off-by: joshlee <joshlee@anyscale.com>
/*client_reconnect_timeout_base_seconds=*/
::RayConfig::instance().gcs_rpc_client_reconnect_timeout_s(),
/*client_reconnect_timeout_max_seconds=*/
::RayConfig::instance().gcs_rpc_client_reconnect_timeout_s(),
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Bug: GCS Client Retry Timeout Configuration Issue

The GCS client's RetryableGrpcClient is configured with identical values for client_reconnect_timeout_base_seconds and client_reconnect_timeout_max_seconds. This disables exponential backoff, causing retries to use a fixed timeout instead of an exponentially increasing one, unlike other clients like CoreWorker and Raylet.

Fix in Cursor Fix in Web

@edoakes
Copy link
Copy Markdown
Collaborator

edoakes commented Oct 13, 2025

test failure

@Sparks0219
Copy link
Copy Markdown
Contributor Author

test failure

looks like its unrelated windows wheel errors, lemme try running them again

@edoakes edoakes enabled auto-merge (squash) October 14, 2025 16:40
RAY_CONFIG(int64_t, core_worker_internal_heartbeat_ms, 1000)

/// Timeout for core worker grpc server reconnection in seconds.
RAY_CONFIG(uint32_t, core_worker_rpc_server_reconnect_timeout_s, 60)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Check to see if user depends on this config

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(this is why we need proper Ray config w/ stabilities)

@jjyao jjyao disabled auto-merge October 15, 2025 03:41
Signed-off-by: joshlee <joshlee@anyscale.com>
/*server_unavailable_timeout_seconds=*/
/*client_reconnect_timeout_base_seconds=*/
::RayConfig::instance().gcs_rpc_server_reconnect_timeout_s(),
/*client_reconnect_timeout_max_seconds=*/
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Bug: GCS Client Retry Timeout Configuration Issue

The GCS client's RetryableGrpcClient sets both client_reconnect_timeout_base_seconds and client_reconnect_timeout_max_seconds to the same config value. This prevents exponential backoff from working, resulting in constant retry intervals. This behavior is inconsistent with other clients like raylet and core worker, which use separate base and max timeouts.

Fix in Cursor Fix in Web

@jjyao jjyao enabled auto-merge (squash) October 15, 2025 20:55
Signed-off-by: joshlee <joshlee@anyscale.com>
@github-actions github-actions bot disabled auto-merge October 15, 2025 20:56
/*server_unavailable_timeout_seconds=*/
/*server_reconnect_timeout_base_seconds=*/
::RayConfig::instance().gcs_rpc_server_reconnect_timeout_s(),
/*server_reconnect_timeout_max_seconds=*/
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Bug: GCS Client Retry Timeout Configuration Issue

The GCS client is configured with the same value for both server_reconnect_timeout_base_seconds and server_reconnect_timeout_max_seconds (gcs_rpc_server_reconnect_timeout_s()). This effectively disables exponential backoff, causing retry intervals to remain constant (e.g., 60 seconds) instead of increasing, which is inconsistent with how core worker and raylet clients are set up.

Fix in Cursor Fix in Web

@Sparks0219
Copy link
Copy Markdown
Contributor Author

Reverted name changes, should be good now

/*server_unavailable_timeout_seconds=*/
/*server_reconnect_timeout_base_seconds=*/
::RayConfig::instance().gcs_rpc_server_reconnect_timeout_s(),
/*server_reconnect_timeout_max_seconds=*/
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Bug: GCS Client Configuration Prevents Exponential Backoff

The GCS client passes the same gcs_rpc_server_reconnect_timeout_s config value to both server_reconnect_timeout_base_seconds and server_reconnect_timeout_max_seconds. This prevents exponential backoff from functioning, causing reconnection attempts to use a constant timeout instead of an increasing one.

Fix in Cursor Fix in Web

@jjyao jjyao merged commit 8fa387a into ray-project:master Oct 16, 2025
6 checks passed
justinyeh1995 pushed a commit to justinyeh1995/ray that referenced this pull request Oct 20, 2025
xinyuangui2 pushed a commit to xinyuangui2/ray that referenced this pull request Oct 22, 2025
…56568)

Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: xgui <xgui@anyscale.com>
elliot-barn pushed a commit that referenced this pull request Oct 23, 2025
Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: elliot-barn <elliot.barnwell@anyscale.com>
landscapepainter pushed a commit to landscapepainter/ray that referenced this pull request Nov 17, 2025
Aydin-ab pushed a commit to Aydin-ab/ray-aydin that referenced this pull request Nov 19, 2025
…56568)

Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: Aydin Abiar <aydin@anyscale.com>
Future-Outlier pushed a commit to Future-Outlier/ray that referenced this pull request Dec 7, 2025
…56568)

Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: Future-Outlier <eric901201@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

core Issues that should be addressed in Ray Core go add ONLY when ready to merge, run all tests

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants