Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Core] Retryable grpc client #47981

Draft
wants to merge 8 commits into
base: master
Choose a base branch
from
Draft

Conversation

jjyao
Copy link
Collaborator

@jjyao jjyao commented Oct 10, 2024

Why are these changes needed?

Currently gcs_rpc_client has retries for gcs rpc calls and this PR moves the retry functionality to RetryableGrpcClient so that it can be used by non-gcs rpc client (e.g. core worker client).

Also enable retry for ReportGeneratorItemReturns rpc since it's idempotent.

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 :(

Signed-off-by: Jiajun Yao <[email protected]>
@jjyao jjyao added the go add ONLY when ready to merge, run all tests label Oct 11, 2024
Signed-off-by: Jiajun Yao <[email protected]>
Signed-off-by: Jiajun Yao <[email protected]>
Signed-off-by: Jiajun Yao <[email protected]>
Signed-off-by: Jiajun Yao <[email protected]>
const int64_t timeout_ms) {
auto executor = new Executor(
[callback](const ray::Status &status) { callback(status, Reply()); });
std::weak_ptr<RetryableGrpcClient> weak_self = shared_from_this();
Copy link
Contributor

Choose a reason for hiding this comment

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

This weak_ptr handles the case when RetryableGrpcClient is already destructed, after which the
operation_callback is called. However we don't have same treatments to &grpc_client which faces similar issues. We need to think about the lifetimes of a single call, a retryable call, &grpc_client and RetryableGrpcClient.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Remove weak_ptr and rely on shutdown_.

Signed-off-by: Jiajun Yao <[email protected]>
src/ray/rpc/retryable_grpc_client.h Show resolved Hide resolved
executor->Abort(ray::Status::TimedOut(absl::StrFormat(
"Timed out while waiting for %s to become available.", server_name_)));
pending_requests_bytes_ -= request_bytes;
delete executor;
Copy link
Contributor

Choose a reason for hiding this comment

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

how can we make sure the executor::Execute's callback is never called by grpc?

src/ray/rpc/retryable_grpc_client.h Outdated Show resolved Hide resolved
<< "limit. Blocking the current thread until network is recovered";
while (self->server_is_unavailable_ && !self->shutdown_) {
self->CheckChannelStatus(false);
std::this_thread::sleep_for(std::chrono::milliseconds(
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm thinking, is this harmful to sleep the thread since it may be on an asio event loop? can we make it a post?

src/ray/rpc/worker/core_worker_client.h Outdated Show resolved Hide resolved
src/ray/rpc/worker/core_worker_client.h Outdated Show resolved Hide resolved
/*server_unavailable_timeout_seconds=*/
std::numeric_limits<uint64_t>::max(),
/*server_unavailable_timeout_callback=*/
[]() { RAY_LOG(FATAL) << "Server unavailable should never timeout"; },
Copy link
Contributor

Choose a reason for hiding this comment

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

are there other possible handler for server unavailable? if not we can build the FATAL in RetryableGrpcClient

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
go add ONLY when ready to merge, run all tests
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants