You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/07/15 05:10:06 UTC

[GitHub] [iceberg] jfz opened a new pull request, #5282: Fix #2796: avoid S3 io error of "Resetting to invalid mark" by re-creating input stream on retries

jfz opened a new pull request, #5282:
URL: https://github.com/apache/iceberg/pull/5282

   This PR fixes [issue #2796](https://github.com/apache/iceberg/issues/2796). It avoids invalid mark reset on retries by providing the input stream factory instead of original input stream to S3 API.
   
   Root cause:
   
   > Currently, the underlying InputStream created from staging files is passed to S3 RequestBody and is reused by S3 client for every retry attempts; and S3 `mark` the underlying input stream with hard-coded of `read limit` of 128K at the beginning for once and `reset` it on every retry. In the case of a first attempt failed after reading more than 128K, the `mark` was invalidated because more than `read limit` amount of data has been read, and when then second attempt try to `reset` the invalidated mark, it breaks the mark/reset contract and throws exception with "Resetting to invalid mark". See java doc of `InputStream` for detail of mark/reset contract.
   
   Steps to reproduce:
   
   1. Get code of [s3mock](https://github.com/adobe/S3Mock) and run server with below change so that it fails the first request with 503 but succeeds on subsequent requests.
   
   > FilestoreController.putObject:
   > \+      LOG.info(" ######## " + count.incrementAndGet());
   > \+      if(count.get() < 2) {
   > \+        return ResponseEntity.status(503).build();
   > \+      }
   >        return ResponseEntity
   >            .ok()
   >            .eTag("\"" + s3Object.getEtag() + "\"")
   > @@ -966,6 +970,7 @@ public class FileStoreController {
   >            "Error persisting object.");
   >      }
   >    }
   > \+  private static AtomicInteger count = new AtomicInteger(0);
   
   2. Update unit test `TestS3OutputStream` as below and run Update unit test as below and run `testInvalidMark`:
   
   > \-  @ClassRule
   > \-  public static final S3MockRule S3_MOCK_RULE = S3MockRule.builder().silent().build();
   > \-
   > \-  private final S3Client s3 = S3_MOCK_RULE.createS3ClientV2();
   > \+//  @ClassRule
   > \+//  public static final S3MockRule S3_MOCK_RULE = S3MockRule.builder().silent().build();
   > \+//
   > \+//  private final S3Client s3 = S3_MOCK_RULE.createS3ClientV2();
   > \+  private final S3Client s3 = S3Client.builder()
   > \+      .overrideConfiguration(ClientOverrideConfiguration.builder().retryPolicy(RetryPolicy.builder().numRetries(3).build()).build())
   > \+      .region(Region.of("us\-east\-1"))
   > \+      .credentialsProvider(
   > \+              StaticCredentialsProvider.create(AwsBasicCredentials.create("foo", "bar")))
   > \+      .endpointOverride(URI.create("http://localhost:" \+ 9090))
   > \+      .httpClient(UrlConnectionHttpClient.builder().buildWithDefaults(AttributeMap.builder().build()))
   > \+      .build();
   > \+  @ Test()
   > \+  public void testInvalidMark() {
   > \+    byte[] data = randomData(129 * 1024);
   > \+    writeAndVerify(s3mock, randomURI(), data, false);
   > \+    ArgumentCaptor<PutObjectRequest> putObjectRequestArgumentCaptor =
   > \+            ArgumentCaptor.forClass(PutObjectRequest.class);
   > \+    verify(s3mock, times(1)).putObject(putObjectRequestArgumentCaptor.capture(),
   > \+            (RequestBody) any());
   > \+    checkPutObjectRequestContent(data, putObjectRequestArgumentCaptor);
   > \+    checkTags(putObjectRequestArgumentCaptor);
   > \+    reset(s3mock);
   > \+  }
   
   


-- 
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-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] rdblue merged pull request #5282: AWS: Fix #2796 - avoid S3 error of "Resetting to invalid mark" by re-creating input stream on retries

Posted by GitBox <gi...@apache.org>.
rdblue merged PR #5282:
URL: https://github.com/apache/iceberg/pull/5282


-- 
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-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] rdblue commented on pull request #5282: AWS: Fix #2796 - avoid S3 error of "Resetting to invalid mark" by re-creating input stream on retries

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #5282:
URL: https://github.com/apache/iceberg/pull/5282#issuecomment-1192793240

   Thanks, @jfz!


-- 
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-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] jfz commented on a diff in pull request #5282: AWS: Fix #2796 - avoid S3 error of "Resetting to invalid mark" by re-creating input stream on retries

Posted by GitBox <gi...@apache.org>.
jfz commented on code in PR #5282:
URL: https://github.com/apache/iceberg/pull/5282#discussion_r927851995


##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java:
##########
@@ -389,7 +391,9 @@ private void completeUploads() {
       S3RequestUtil.configureEncryption(awsProperties, requestBuilder);
       S3RequestUtil.configurePermission(awsProperties, requestBuilder);
 
-      s3.putObject(requestBuilder.build(), RequestBody.fromInputStream(contentStream, contentLength));
+      s3.putObject(
+          requestBuilder.build(),
+          RequestBody.fromContentProvider(contentProvider, contentLength, Mimetype.MIMETYPE_OCTET_STREAM));

Review Comment:
   The input stream here is a composition from multiple `stagingFiles`, is size of `stagingFiles` always 1 when it's not multipart? It's possible if that's the case, but I think the current way is a bit more elegant and flexible without that assumption.



-- 
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-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] puneetzaroo commented on pull request #5282: AWS: Fix #2796 - avoid S3 error of "Resetting to invalid mark" by re-creating input stream on retries

Posted by GitBox <gi...@apache.org>.
puneetzaroo commented on PR #5282:
URL: https://github.com/apache/iceberg/pull/5282#issuecomment-1191043001

   Looks good


-- 
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-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] jfz commented on pull request #5282: AWS: Fix #2796 - avoid S3 error of "Resetting to invalid mark" by re-creating input stream on retries

Posted by GitBox <gi...@apache.org>.
jfz commented on PR #5282:
URL: https://github.com/apache/iceberg/pull/5282#issuecomment-1185848975

   Another issue with `fromInputStream` is that it disables `close` of the original stream, and user is responsible for closing it outside of AWS SDK. In this particular case I think it's not too bad because SequenceInputStream closes the underlying input streams on iteration, only the BufferedInputStream is not closed but I think the buffer is GCed when this completes. 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-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] stevenzwu commented on pull request #5282: AWS: Fix #2796 - avoid S3 error of "Resetting to invalid mark" by re-creating input stream on retries

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on PR #5282:
URL: https://github.com/apache/iceberg/pull/5282#issuecomment-1191045722

   @jfz can you refactor the manual testing in the description to proper unit tests?


-- 
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-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] holdenk commented on pull request #5282: AWS: Fix #2796 - avoid S3 error of "Resetting to invalid mark" by re-creating input stream on retries

Posted by GitBox <gi...@apache.org>.
holdenk commented on PR #5282:
URL: https://github.com/apache/iceberg/pull/5282#issuecomment-1187806334

   Interesting, we've been running into something similar.


-- 
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-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5282: AWS: Fix #2796 - avoid S3 error of "Resetting to invalid mark" by re-creating input stream on retries

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5282:
URL: https://github.com/apache/iceberg/pull/5282#discussion_r927841774


##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java:
##########
@@ -389,7 +391,9 @@ private void completeUploads() {
       S3RequestUtil.configureEncryption(awsProperties, requestBuilder);
       S3RequestUtil.configurePermission(awsProperties, requestBuilder);
 
-      s3.putObject(requestBuilder.build(), RequestBody.fromInputStream(contentStream, contentLength));
+      s3.putObject(
+          requestBuilder.build(),
+          RequestBody.fromContentProvider(contentProvider, contentLength, Mimetype.MIMETYPE_OCTET_STREAM));

Review Comment:
   Could this use `RequestBody.fromFile(f)` instead, like the `uploadParts` method does?



-- 
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-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] jfz commented on pull request #5282: AWS: Fix #2796 - avoid S3 error of "Resetting to invalid mark" by re-creating input stream on retries

Posted by GitBox <gi...@apache.org>.
jfz commented on PR #5282:
URL: https://github.com/apache/iceberg/pull/5282#issuecomment-1191677388

   > @jfz can you refactor the manual testing in the description to proper unit tests?
   
   @stevenzwu Thanks for looking into this. The challenge to test retry is that it requires "S3Mock" code change and still tricky to get only a particular request to fail at first attempt and succeed later. The happen path (upload a s3 object with size over 128k) is well covered in existing tests though, the above example is actually simplified from an existing test.


-- 
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-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] jfz commented on a diff in pull request #5282: AWS: Fix #2796 - avoid S3 error of "Resetting to invalid mark" by re-creating input stream on retries

Posted by GitBox <gi...@apache.org>.
jfz commented on code in PR #5282:
URL: https://github.com/apache/iceberg/pull/5282#discussion_r927851995


##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java:
##########
@@ -389,7 +391,9 @@ private void completeUploads() {
       S3RequestUtil.configureEncryption(awsProperties, requestBuilder);
       S3RequestUtil.configurePermission(awsProperties, requestBuilder);
 
-      s3.putObject(requestBuilder.build(), RequestBody.fromInputStream(contentStream, contentLength));
+      s3.putObject(
+          requestBuilder.build(),
+          RequestBody.fromContentProvider(contentProvider, contentLength, Mimetype.MIMETYPE_OCTET_STREAM));

Review Comment:
   The input stream here is a composition from multiple `stagingFiles`, is size of `stagingFiles` always 1 when it's not multipart? It's possible if that's the case, but I think the current way is a bit more elegant and future proof without that assumption.



-- 
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-unsubscribe@iceberg.apache.org

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


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #5282: AWS: Fix #2796 - avoid S3 error of "Resetting to invalid mark" by re-creating input stream on retries

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5282:
URL: https://github.com/apache/iceberg/pull/5282#discussion_r927861598


##########
aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java:
##########
@@ -389,7 +391,9 @@ private void completeUploads() {
       S3RequestUtil.configureEncryption(awsProperties, requestBuilder);
       S3RequestUtil.configurePermission(awsProperties, requestBuilder);
 
-      s3.putObject(requestBuilder.build(), RequestBody.fromInputStream(contentStream, contentLength));
+      s3.putObject(
+          requestBuilder.build(),
+          RequestBody.fromContentProvider(contentProvider, contentLength, Mimetype.MIMETYPE_OCTET_STREAM));

Review Comment:
   Ah, you're right. We're concatenating them. Sounds good!



-- 
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-unsubscribe@iceberg.apache.org

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


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