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

API, AWS: Retry S3InputStream reads #10433

Merged

Conversation

amogh-jahagirdar
Copy link
Contributor

@amogh-jahagirdar amogh-jahagirdar commented Jun 3, 2024

Fixes #10340

This is an alternative approach to https://github.com/apache/iceberg/pull/4912/files and https://github.com/apache/iceberg/pull/8221/files#diff-0b632866a3b10fac55c442b08178ec0ac72b3b600878243e15d788a8bd031054

for retrying failures encountered when retrying on the reading of input streams.

This approach defines a RetryableInputStream class which will wrap underlying input streams returned by object store APIs.
Upon failures a new stream will be created. Custom exceptions can be passed in, but the default retries are on SocketTimeoutException and SSLException. This change integrates this input stream implementation with S3InputStream, but RetryableINputStream should be able to be used for the other input streams implementations that are provided by Iceberg.

This change relies on the Failsafe dependency.

Comment on lines 57 to 59
public void testReadWithFuzzyStreamRetrySucceed(IOException exception) throws Exception {
testRead(
fuzzyStreamClient(new AtomicInteger(3), exception), new S3FileIOProperties(), DATA_SIZE);
}
Copy link
Contributor Author

@amogh-jahagirdar amogh-jahagirdar Jun 3, 2024

Choose a reason for hiding this comment

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

This test takes way too long. That's primarily because it internally tests retrying on just read() (the non-buffered reads) which means every byte read will fail and be retried 2 times with a 500 ms in between. So essentially that's a second per byte.

I think what we can do is modularize further and only do buffered read tests + with a much smaller data size test the per byte read to exercise that code path in the tests. The buffered read tests are pretty fast.

import software.amazon.awssdk.services.s3.model.PutObjectRequest;
import software.amazon.awssdk.services.s3.model.PutObjectResponse;

public class TestFuzzyS3InputStream extends TestS3InputStream {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I copied a lot of the test logic from https://github.com/apache/iceberg/pull/4912/files, will mark @jackye1995 as coauthor

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I saw @xiaoxuandev also had the same tests in #8221 so I'm marking her as co-author here as well

return s3.getObject(requestBuilder.build(), ResponseTransformer.toInputStream());
stream =
RetryableInputStream.builderFor(
() -> s3.getObject(requestBuilder.build(), ResponseTransformer.toInputStream()))
Copy link
Contributor Author

Choose a reason for hiding this comment

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

any failures that occur on the retry during the getObject request (not the stream read) should just be handled by the SDK. Don't think we need to add anything custom for that since the S3 client is already pluggable.

Copy link
Contributor

Choose a reason for hiding this comment

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

I feel like there's an issue here. When the stream gets recreated, the stream will reset back to the original position and we continue from there as if we're at the right place in the stream.

The pos won't reflect the new position of the stream, if I'm reading this correctly. I would think the following retry would need to start from next to reflect where the next read should start. There's a small problem with the single byte read method because we increment the positions prior to read, so that would likely need to be adjusted to be after the read like in the range read method.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'll double check this, I think you're right although I'm not sure why the first test case wouldn't surface that (since the content would be different). We may need to seek properly on the input stream initialization during retry.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I revisited this PR, and took a look with fresh eyes and yes the current logic is definitely incorrect for the case where a non range read (readFully, readTail perform range reads) is performed.

For the range-reads we don't really care about the current position for the purpose of tracking in the retryable input stream. But for normal seeking based reads we definitely do!

I think the way to solve this is to pass a supplier of the current position to the retryable input stream. That supplier would have a reference to this and the stream provider would be a function which accepts a position. Upon retries the stream provider would open a new connection and open a stream that begins with the position that the position supplier (which is guaranteed to be the correct position to start the stream from) returns.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I updated this, the RetryableInputStream offers two builder APIs, one for specifying just a new stream initialization and one for a stream initialization plus a position supplier. The stream initialization function takes in a position (the position can be null to handle the range based requests since for range reads with explicit begin/end we don't care about the current position in the stream). cc @danielcweeks

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Discussed with @danielcweeks , for the range read cases since we're using IOUtil readFully/readRemaining which will read the range in a buffered manner. On retries we would read from the beginning position but the internal stream tracking in readFully/readRemaining would not reset to the right position in the buffer to read so that's still an issue.

What we can do is to just not retry the RangeReadable methods for now since they're not actually exercised anywhere. Down the line, we could just use FailSafe and retry on the whole method.

@@ -37,6 +37,7 @@ delta-standalone = "3.1.0"
delta-spark = "3.2.0"
esotericsoftware-kryo = "4.0.3"
errorprone-annotations = "2.27.0"
failsafe = "3.3.2"
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This dependency is quite nice in that it's 0 dependency itself, has Apache licensing and I think there's more use cases in Iceberg to leverage it. For example, I think a lot of the complex logic in Tasks can be simplified.

Furthermore, there's some custom retry logic in JDBC connector which we couldn't use tasks for, but now we could use Failsafe. Wonder what others think

Copy link
Contributor

Choose a reason for hiding this comment

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

I like this a lot

@SandeepSinghGahir
Copy link

When will this merged? I'm getting this issue while reading iceberg tables in glue.

Copy link
Contributor Author

@amogh-jahagirdar amogh-jahagirdar left a comment

Choose a reason for hiding this comment

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

When will this merged? I'm getting this issue while reading iceberg tables in glue.

Sorry about that @SandeepSinghGahir , I drafted this PR and never followed through. I just took a pass and determined next steps so that we can take this to completion since I know a few folks are hitting this issue and I think it's reasonable that Iceberg S3 input stream has some level of retries when reading from the stream.

I'll also mention, it's really important for changes like this which are on the critical path that they are well tested so I'm also thinking through the test cases as well.

return s3.getObject(requestBuilder.build(), ResponseTransformer.toInputStream());
stream =
RetryableInputStream.builderFor(
() -> s3.getObject(requestBuilder.build(), ResponseTransformer.toInputStream()))
Copy link
Contributor Author

Choose a reason for hiding this comment

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

I revisited this PR, and took a look with fresh eyes and yes the current logic is definitely incorrect for the case where a non range read (readFully, readTail perform range reads) is performed.

For the range-reads we don't really care about the current position for the purpose of tracking in the retryable input stream. But for normal seeking based reads we definitely do!

I think the way to solve this is to pass a supplier of the current position to the retryable input stream. That supplier would have a reference to this and the stream provider would be a function which accepts a position. Upon retries the stream provider would open a new connection and open a stream that begins with the position that the position supplier (which is guaranteed to be the correct position to start the stream from) returns.

@amogh-jahagirdar amogh-jahagirdar force-pushed the retry-reading-input-stream branch from f4788e7 to deae6eb Compare August 21, 2024 02:27
@github-actions github-actions bot added core and removed API labels Aug 21, 2024
@amogh-jahagirdar amogh-jahagirdar force-pushed the retry-reading-input-stream branch 9 times, most recently from 28513fc to 8ba83e3 Compare August 22, 2024 00:13
@amogh-jahagirdar amogh-jahagirdar marked this pull request as ready for review August 22, 2024 00:13
@amogh-jahagirdar amogh-jahagirdar force-pushed the retry-reading-input-stream branch 2 times, most recently from 9707998 to 3b916fe Compare September 17, 2024 18:47
@amogh-jahagirdar amogh-jahagirdar force-pushed the retry-reading-input-stream branch 2 times, most recently from d77f22d to 56cef08 Compare September 17, 2024 19:59
@amogh-jahagirdar amogh-jahagirdar force-pushed the retry-reading-input-stream branch 3 times, most recently from 95fb440 to f65a26d Compare September 17, 2024 21:13
@amogh-jahagirdar amogh-jahagirdar force-pushed the retry-reading-input-stream branch from f65a26d to ffb1274 Compare September 17, 2024 21:21
@amogh-jahagirdar amogh-jahagirdar changed the title API, AWS: Add RetryableInputStream and use that in S3InputStream API, AWS: Retry S3InputStream reads Sep 17, 2024
@SandeepSinghGahir
Copy link

@amogh-jahagirdar Any tentative timeline on merging of this PR?

@amogh-jahagirdar amogh-jahagirdar force-pushed the retry-reading-input-stream branch 5 times, most recently from 3dbdf1e to 9d81f18 Compare September 23, 2024 16:46
@amogh-jahagirdar amogh-jahagirdar force-pushed the retry-reading-input-stream branch from 9d81f18 to 3421040 Compare September 23, 2024 17:29
Copy link
Contributor

@danielcweeks danielcweeks left a comment

Choose a reason for hiding this comment

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

Thanks for all the work and revisions @amogh-jahagirdar!

@amogh-jahagirdar
Copy link
Contributor Author

Thanks for the reviews @danielcweeks! Merging.

@amogh-jahagirdar amogh-jahagirdar merged commit c0d73f4 into apache:main Sep 24, 2024
50 checks passed
RetryPolicy.builder()
.handle(
ImmutableList.of(
SSLException.class, SocketTimeoutException.class, SocketException.class))
Copy link

Choose a reason for hiding this comment

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

Should software.amazon.awssdk.core.exception.SdkClientException be included in the exception list ?
It indicates issues with the client-side networking stack, such as network timeouts.

Copy link
Contributor

Choose a reason for hiding this comment

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

Judging by the comment in the SdkClientException class, it should not be retryable as there might be multiple reasons for it

Base type for all client exceptions thrown by the SDK. This exception is thrown when service could not be contacted for a response, or when client is unable to parse the response from service.
Exceptions that extend SdkClientException are assumed to be not retryable, with a few exceptions:
RetryableException - usable when calls should explicitly be retried
Exceptions mentioned as a retryable exception in SdkDefaultRetrySetting
See Also:
SdkServiceException

Is there a more specific one that you were thinking?

Copy link

Choose a reason for hiding this comment

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

ImmutableList.of(
SSLException.class, SocketTimeoutException.class, SocketException.class))
.onFailure(failure -> openStream(true))
.withMaxRetries(3)
Copy link

Choose a reason for hiding this comment

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

It seems we can use jitter so that simultaneous retries don't overwhelm the system:

.withBackoff(1, 10, TimeUnit.SECONDS, BackoffJitter.random())

Copy link
Contributor

@ookumuso ookumuso Oct 3, 2024

Choose a reason for hiding this comment

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

EqualJitterBackoffStrategy adds the jitter internallly on every retry: software.amazon.awssdk.core.retry.backoff.EqualJitterBackoffStrategy

Scratch my response above, it won't apply here since this happens outside of SDK

zachdisc pushed a commit to zachdisc/iceberg that referenced this pull request Dec 23, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

javax.net.ssl.SSLException: Connection reset on S3 w/ S3FileIO and Apache HTTP client
9 participants