apache / arrow-rs

Official Rust implementation of Apache Arrow
https://arrow.apache.org/
Apache License 2.0
2.62k stars 802 forks source link

object_store: Retry on connection duration timeouts? #6287

Open flokli opened 3 months ago

flokli commented 3 months ago

**Is your feature request related to a problem or challenge? I'm using object_store to stream large(r) files to in this case, the body of a HTTP .

I essentially do a store.get(path).await?.into_stream() to get the data stream.

When using it with the Azure backend, I noticed that Azure reliably closes the connection after 30 seconds. Other providers (S3) also explicitly inject errors, but keep most of the error handling in their SDKs.

I know there's some retry logic for some error cases in object_store, but the "connection closed while getting data as a stream" part doesn't seem to be covered. I think there should be a high-level function that retries receiving the remaining data in these error cases (verifying the etag is still the same).

Describe alternatives you've considered Manually dealing with error handling and retries in all object_store consumers

cbrewster commented 3 months ago

The golang GCS client supports automatic retrying here via keeping track of how many bytes the client has already seen and creating a new ranged read request to resume the read where the client left off:

https://github.com/googleapis/google-cloud-go/blob/9afb797d75499807b29c372ec375668be4d2995e/storage/http_client.go#L1268-L1275

flokli commented 3 months ago

So apparently object_store::buffered::BufReader might be the answer to this, at least it prevents long-running connections to the backend.

I feel like the docstring of into_stream() should be extended to point it out that you might want to use object_store::buffered::BufReader instead.

I'm not that happy yet object_store::buffered::BufReader doesn't do any "readahead", meaning we'd have to wait the summed up round-trip times whenever the buffer gets empty, but that's probably also tricky to do in the general case.

I'm also not sure if the calls to get_range it does are actually retried, in case of temporary connectivity issues / rate-limiting.

itsjunetime commented 2 months ago

take

erratic-pattern commented 1 month ago

take

erratic-pattern commented 1 month ago

I've made a PR https://github.com/apache/arrow-rs/pull/6519 similar to the closed PR https://github.com/apache/arrow-rs/pull/5383, but only permits reqwest::Error::Decode to be retried, since this is the error type that we see associated with this issue (see https://github.com/apache/arrow-rs/issues/5882)

EDIT: I am not sure this is exactly the same issue that @flokli is seeing, but it is the one being seen in the #5882 issue so maybe I should associate this PR with that issue instead?

alamb commented 1 month ago

Given the subtle nature of errors and retries in the context of streaming requests, I think the only practical way forward will be to create an example/ test case that shows the problem.

The example would likely look something like:

  1. A mock http server that pretends to be an S3 endpoint
  2. Some settings in the mock server that can inject errors (like abort after some bytes have already been returned)
  3. Testing the object store client then with errors / how it would automatically retry.

This test would also let us explore the various issues / corner cases that could be encountered

It would be likely that writing the test harness would be a significant undertaking, but that would be the best way to inform a proposal for API changes

tustvold commented 1 month ago

We already have all of this setup as part of the retry tests - https://github.com/apache/arrow-rs/blob/master/object_store/src/client/retry.rs#L505. It would be a relatively straightforward enhancement to inject a failure mid-stream.

Edit: As for what the API would look like, given the only streaming request is get, and the corresponding logic is already extracted into GetClientExt which is shared across all implementations. This would be the obvious place to put any retry logic for streaming requests.

alamb commented 1 month ago

Perfect! So maybe @erratic-pattern you can make a PR with the failure scenario ? It would be valuable to document in code the current behavior, even if we decide we don't have the time to fix it.