[core] Creating exponential backoff for retryable grpcs #56568
[core] Creating exponential backoff for retryable grpcs #56568jjyao merged 24 commits intoray-project:masterfrom
Conversation
Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: joshlee <joshlee@anyscale.com>
…t-raylet-client-pool-on-node-death
|
Where's my AI comments :( |
…t-raylet-client-pool-on-node-death
src/ray/rpc/retryable_grpc_client.cc
Outdated
| RAY_LOG(WARNING) << server_name_ << " has been unavailable for more than " | ||
| << server_unavailable_timeout_seconds_ << " seconds"; | ||
| << (attempt_number > 0 ? ExponentialBackoff::GetBackoffMs( | ||
| attempt_number - 1, |
There was a problem hiding this comment.
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:
ray/src/ray/rpc/retryable_grpc_client.cc
Line 158 in 19bfc16
Otherwise we set it here:
ray/src/ray/rpc/retryable_grpc_client.cc
Line 87 in 19bfc16
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.
src/ray/rpc/retryable_grpc_client.h
Outdated
|
|
||
| uint32_t GetAttemptNumber() const { return attempt_number_; } | ||
|
|
||
| void SetAttemptNumber(uint32_t attempt_number) { attempt_number_ = attempt_number; } |
There was a problem hiding this comment.
isn't this just accessed from within the class so we don't need getter/setter
There was a problem hiding this comment.
This is accessed with the parent class RetryableGrpcClient, so we still need the getters/setters to access members within the nested class RetryableGrpcRequest.
There was a problem hiding this comment.
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>
src/ray/rpc/retryable_grpc_client.cc
Outdated
| if (server_name_ == "GCS") { | ||
| server_unavailable_timeout_time_ = | ||
| now + absl::Seconds(server_unavailable_base_timeout_seconds_); | ||
| SetupCheckTimer(); | ||
| } else { | ||
| server_unavailable_timeout_time_ = now; | ||
| CheckChannelStatus(); |
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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:
ray/src/ray/gcs_client/rpc_client.h
Lines 191 to 215 in b25b7e2
There was a problem hiding this comment.
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
There was a problem hiding this comment.
+1. We can just pass in exponential backoff configs explicitly: initial delay, exponential factor, max delay.
There was a problem hiding this comment.
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
src/ray/rpc/retryable_grpc_client.cc
Outdated
| ? ExponentialBackoff::GetBackoffMs( | ||
| attempt_number - 1, | ||
| server_unavailable_base_timeout_seconds_ * 1000) / | ||
| 1000.0 |
There was a problem hiding this comment.
this should be capped, else the timeout will approach infinity quickly
There was a problem hiding this comment.
gotcha, I'll add a cap of 60 seconds
Signed-off-by: joshlee <joshlee@anyscale.com>
Signed-off-by: joshlee <joshlee@anyscale.com>
src/ray/rpc/retryable_grpc_client.h
Outdated
| 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, |
There was a problem hiding this comment.
I think we just need three configs:
server_unavailable_timeout_initial_seconds, server_unavailable_timeout_max_seconds, and server_unavailable_timeout_exponential_factor?
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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
There was a problem hiding this comment.
Yea good point... I ended end just keep server_unavailable_max_timeout_seconds since
server_unavailable_timeout_initial_seconds is not needed after: #56613
src/ray/rpc/retryable_grpc_client.cc
Outdated
| SetupCheckTimer(); | ||
| if (call_first_unavailable_timeout_callback_immediately_) { | ||
| server_unavailable_timeout_time_ = now; | ||
| CheckChannelStatus(); |
There was a problem hiding this comment.
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
There was a problem hiding this comment.
Yea good point, I ended up just removing the default parameter since its literally only used in one place
src/ray/rpc/retryable_grpc_client.h
Outdated
|
|
||
| uint32_t GetAttemptNumber() const { return attempt_number_; } | ||
|
|
||
| void SetAttemptNumber(uint32_t attempt_number) { attempt_number_ = attempt_number; } |
There was a problem hiding this comment.
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
src/ray/rpc/retryable_grpc_client.h
Outdated
| 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, |
There was a problem hiding this comment.
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
…t-raylet-client-pool-on-node-death
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>
edoakes
left a comment
There was a problem hiding this comment.
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.
…t-raylet-client-pool-on-node-death
src/ray/gcs_rpc_client/rpc_client.h
Outdated
| /*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(), |
There was a problem hiding this comment.
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.
|
test failure |
looks like its unrelated windows wheel errors, lemme try running them again |
…t-raylet-client-pool-on-node-death
| 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) |
There was a problem hiding this comment.
Check to see if user depends on this config
There was a problem hiding this comment.
(this is why we need proper Ray config w/ stabilities)
Signed-off-by: joshlee <joshlee@anyscale.com>
src/ray/gcs_rpc_client/rpc_client.h
Outdated
| /*server_unavailable_timeout_seconds=*/ | ||
| /*client_reconnect_timeout_base_seconds=*/ | ||
| ::RayConfig::instance().gcs_rpc_server_reconnect_timeout_s(), | ||
| /*client_reconnect_timeout_max_seconds=*/ |
There was a problem hiding this comment.
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.
Signed-off-by: joshlee <joshlee@anyscale.com>
| /*server_unavailable_timeout_seconds=*/ | ||
| /*server_reconnect_timeout_base_seconds=*/ | ||
| ::RayConfig::instance().gcs_rpc_server_reconnect_timeout_s(), | ||
| /*server_reconnect_timeout_max_seconds=*/ |
There was a problem hiding this comment.
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.
|
Reverted name changes, should be good now |
…t-raylet-client-pool-on-node-death
| /*server_unavailable_timeout_seconds=*/ | ||
| /*server_reconnect_timeout_base_seconds=*/ | ||
| ::RayConfig::instance().gcs_rpc_server_reconnect_timeout_s(), | ||
| /*server_reconnect_timeout_max_seconds=*/ |
There was a problem hiding this comment.
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.
…56568) Signed-off-by: joshlee <joshlee@anyscale.com>
…56568) Signed-off-by: joshlee <joshlee@anyscale.com> Signed-off-by: xgui <xgui@anyscale.com>
Signed-off-by: joshlee <joshlee@anyscale.com> Signed-off-by: elliot-barn <elliot.barnwell@anyscale.com>
…56568) Signed-off-by: joshlee <joshlee@anyscale.com>
…56568) Signed-off-by: joshlee <joshlee@anyscale.com> Signed-off-by: Aydin Abiar <aydin@anyscale.com>
…56568) Signed-off-by: joshlee <joshlee@anyscale.com> Signed-off-by: Future-Outlier <eric901201@gmail.com>
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
git commit -s) in this PR.scripts/format.shto lint the changes in this PR.method in Tune, I've added it in
doc/source/tune/api/under thecorresponding
.rstfile.