alamb opened a new issue, #15:
URL: https://github.com/apache/arrow-rs-object-store/issues/15

   # Problem Description
   if a request fails mid-stream (after we begin to read data) it is not 
retried and instead an error is returned. The error message, somewhat 
confusingly, often says `error decoding response body`
   
   Some examples:
   
   > ExternalError(General("ParquetObjectReader::get_byte_ranges error: Generic 
MicrosoftAzure error: error decoding response body"))
   
   >  Generic S3 error: error decoding response body
   
   # Workaround
   You can often work around this error by increasing the network timeout to 
something longer than the 30s default
   
   ### Related Tickets
   - there are many related discussions / report 
https://github.com/apache/arrow-rs/issues/5882
   - we see this in DataFusion too 
https://github.com/apache/datafusion/issues/15067 
   - https://github.com/apache/arrow-rs/issues/6287
   - https://github.com/apache/arrow-rs/pull/6519
   - https://github.com/delta-io/delta-rs/issues/2595
   
   As  @crepererum says on 
https://github.com/apache/arrow-rs/issues/5882#issuecomment-2700954147 :
   
   > So long store short: People agree that this would be a good feature to 
have, but it requires a proper implementation.
   
   # Background
   
   ## Streaming ✅ 
   Some APIs like 
[`ObjectStore::get`](https://docs.rs/object_store/latest/object_store/trait.ObjectStore.html#method.get)
 are "streaming" in the sense that they start returning data as soon as it 
comes back from the network (as opposed to buffering the response before 
returning to the caller)
   
   This is great for performance as response processing can happen immediately 
and limits memory usage for large payloads 🏆 
   
   ## Retries ✅ 
   In order to deal with the intermittent errors that occur processing object 
store requests, most ObjectStore implementations retry the request if they 
encounter error (see 
[retry.rs](https://github.com/apache/arrow-rs/blob/main/object_store/src/client/retry.rs))
   
   ## Retries + Streaming ❌ 
   
   However, there is a problem when streaming is mixed with the existing 
retries. Specifically, if a request fails mid-stream (after some, but not all, 
of the data has been returned to the client), just retrying the entire request 
isn't enough because then the client would be potentially be given the same 
data from the start of the response that it had already been given
   
   # Solution 
   
   **Describe the solution you'd like**
   
   Implementing retries for streaming reads would need something more 
complicated like retrying the request just for the bytes that hadn't been 
already read
   
   
   
   Any solution for this I think needs:
   1. Very good tests / clear documentation
   
   **Describe alternatives you've considered**
   
   @crepererum suggests on 
https://github.com/apache/arrow-rs/issues/5882#issuecomment-2700954147 :
   
   > retrying would need to make a new request with a new range starting after 
the last received byte and ideally also an ETAG/version check to ensure that 
the object that is returned by the retry is the the one that was already "in 
flight". This retry mechanic is obviously chaining/nested, i.e. if the retry 
fails mid-stream, you wanna have yet another retry that picks up the where the 
previous one ended. 
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to