jackye1995 commented on code in PR #4912:
URL: https://github.com/apache/iceberg/pull/4912#discussion_r889726156


##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3InputStream.java:
##########
@@ -178,25 +281,56 @@ private void positionStream() throws IOException {
   }
 
   private void openStream() throws IOException {
-    GetObjectRequest.Builder requestBuilder = GetObjectRequest.builder()
-        .bucket(location.bucket())
-        .key(location.key())
-        .range(String.format("bytes=%s-", pos));
-
-    S3RequestUtil.configureEncryption(awsProperties, requestBuilder);
-
     closeStream();
-    stream = s3.getObject(requestBuilder.build(), 
ResponseTransformer.toInputStream());
+    stream = readRange(String.format("bytes=%s-", pos));
   }
 
   private void closeStream() throws IOException {
-    if (stream != null) {
-      stream.close();
+    closeServerSideStream(stream);
+    stream = null;
+  }
+
+  private static void closeServerSideStream(InputStream streamToClose) {
+    if (streamToClose != null) {
+      try {
+        if (streamToClose instanceof ResponseInputStream) {
+          // Stated in the ResponseInputStream javadoc:
+          // If it is not desired to read remaining data from the stream,
+          // you can explicitly abort the connection via abort().
+          ((ResponseInputStream<?>) streamToClose).abort();
+        } else {
+          streamToClose.close();
+        }
+      } catch (IOException | AbortedException e) {
+        // ignore failure to abort or close stream
+      }
     }
   }
 
-  public void setSkipSize(int skipSize) {
-    this.skipSize = skipSize;
+  private static boolean shouldRetry(Exception exception) {
+    if (exception instanceof UncheckedIOException) {
+      if (exception.getCause() instanceof EOFException) {
+        return false;
+      }
+    }
+
+    if (exception instanceof AwsServiceException) {
+      switch (((AwsServiceException) exception).statusCode()) {
+        case HttpURLConnection.HTTP_FORBIDDEN:
+        case HttpURLConnection.HTTP_BAD_REQUEST:
+          return false;
+      }
+    }
+
+    if (exception instanceof S3Exception) {
+      switch (((S3Exception) exception).statusCode()) {
+        case HttpURLConnection.HTTP_NOT_FOUND:
+        case 416: // range not satisfied
+          return false;
+      }
+    }
+
+    return true;

Review Comment:
   It's not. It's closer to the ones in Presto and Trino. Basically it retires 
almost all IO exceptions except for EOF, because they are most likely network 
issues. For AWS side exceptions, this logic seems sufficient to me if they are 
proven sufficient in Presto and Trino. I am not sure if we need to list every 
single possible exception class like S3A did. 



-- 
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]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to