amogh-jahagirdar commented on code in PR #10433:
URL: https://github.com/apache/iceberg/pull/10433#discussion_r1764030252


##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java:
##########
@@ -195,14 +230,20 @@ private void openStream() throws IOException {
     }
   }
 
-  private void closeStream() throws IOException {
+  private void closeStream(boolean closeQuietly) throws IOException {
     if (stream != null) {
       // if we aren't at the end of the stream, and the stream is abortable, 
then
       // call abort() so we don't read the remaining data with the Apache HTTP 
client
       abortStream();
       try {
         stream.close();
       } catch (IOException e) {
+        if (closeQuietly) {
+          stream = null;
+          LOG.warn("An error occurred while closing the stream", e);
+          return;
+        }

Review Comment:
   This is needed so that IOException failures on close during the retry don't 
count against the retry on the read. It can just be a best effort close.



##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java:
##########
@@ -56,7 +63,15 @@ class S3InputStream extends SeekableInputStream implements 
RangeReadable {
   private final Counter readBytes;
   private final Counter readOperations;
 
-  private int skipSize = 1024 * 1024;
+  private int skipSize = 1024 * 1024;;
+  private RetryPolicy<Object> retryPolicy =
+      RetryPolicy.builder()
+          .handle(
+              ImmutableList.of(
+                  SSLException.class, SocketTimeoutException.class, 
SocketException.class))
+          .onFailure(failure -> openStream(true))
+          .withMaxRetries(3)
+          .build();

Review Comment:
   I ended up removing RetryableInputStream for now; GCS is channel based and 
we should probably verify the Azure SDK doesn't already retry internally. We 
can always add the abstraction later if it ends up being needed but there's 
arguably not a compelling case for adding it up front when inlining Failsafe is 
simple. cc @danielcweeks 



-- 
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: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to