[ 
https://issues.apache.org/jira/browse/HADOOP-19221?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17867791#comment-17867791
 ] 

Steve Loughran commented on HADOOP-19221:
-----------------------------------------

I have replicated this in a test. it only affects the staging committers, which 
upload a single file in individual parts as range-limited input streams. 

{code}
 RequestBody.fromInputStream(fileStream, size);
{code}

The range limiting wraps them in a new stream and mark/reset doesn't seem, to 
work. 
the javadocs imply that the mark/reset only works for 128k, but in the test to 
regenerate the issue it seems independent of length.

{code}
     * To support resetting via {@link ContentStreamProvider}, this uses {@link 
InputStream#reset()} and uses a read limit of
     * 128 KiB. If you need more control, use {@link 
#fromContentProvider(ContentStreamProvider, long, String)} or
     * {@link #fromContentProvider(ContentStreamProvider, String)}.

{code}

org.apache.hadoop.fs.s3a.AWSBadRequestException: upload part #1 upload ID 
KylDNXbmiZlZE5JI2aKVcVeA66lyRaUZWmKv5K3QfGcjgqjpMAsnbRkJRug8jf3kdjXuL_zVt09813gtOMS9CSxwTDbfGR4xOLPUmZZ9suEJGh.iolwJgMnSXk5KFxnG
 on test/testCommitOperations[array-wrap-true]: 
software.amazon.awssdk.services.s3.model.S3Exception: Your socket connection to 
the server was not read from or written to within the timeout period. Idle 
connections will be closed. (Service: S3, Status Code: 400, Request ID: 
TT17CRYF6HJH2G0Y, Extended Request ID: 
d7VKx+iXtShSdp/uVQhrDAhC7dOGbZiZE6ul1I/k19RRDc6wRHyKSkBJU82ta22ns+J7QbbL5XY=):RequestTimeout:
 Your socket connection to the server was not read from or written to within 
the timeout period. Idle connections will be closed. (Service: S3, Status Code: 
400, Request ID: TT17CRYF6HJH2G0Y, Extended Request ID: 
d7VKx+iXtShSdp/uVQhrDAhC7dOGbZiZE6ul1I/k19RRDc6wRHyKSkBJU82ta22ns+J7QbbL5XY=)

        at 
org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:260)
        at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:124)
        at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$4(Invoker.java:376)
        at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:468)
        at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:372)
        at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:347)
        at 
org.apache.hadoop.fs.s3a.WriteOperationHelper.retry(WriteOperationHelper.java:205)
        at 
org.apache.hadoop.fs.s3a.WriteOperationHelper.uploadPart(WriteOperationHelper.java:589)
        at 
org.apache.hadoop.fs.s3a.commit.impl.CommitOperations.uploadFileDataAsInputStream(CommitOperations.java:686)
        at 
org.apache.hadoop.fs.s3a.commit.impl.CommitOperations.uploadFileToPendingCommit(CommitOperations.java:586)
        at 
org.apache.hadoop.fs.s3a.commit.magic.ITestUploadRecovery.testCommitOperations(ITestUploadRecovery.java:280)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at 
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
        at 
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
        at 
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
        at 
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
        at 
org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
        at 
org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
        at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:61)
        at 
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:299)
        at 
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:293)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.lang.Thread.run(Thread.java:750)
Caused by: software.amazon.awssdk.services.s3.model.S3Exception: Your socket 
connection to the server was not read from or written to within the timeout 
period. Idle connections will be closed. (Service: S3, Status Code: 400, 
Request ID: TT17CRYF6HJH2G0Y, Extended Request ID: 
d7VKx+iXtShSdp/uVQhrDAhC7dOGbZiZE6ul1I/k19RRDc6wRHyKSkBJU82ta22ns+J7QbbL5XY=)
        at 
software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handleErrorResponse(AwsXmlPredicatedResponseHandler.java:156)
        at 
software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handleResponse(AwsXmlPredicatedResponseHandler.java:108)
        at 
software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handle(AwsXmlPredicatedResponseHandler.java:85)
        at 
software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handle(AwsXmlPredicatedResponseHandler.java:43)
        at 
software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler$Crc32ValidationResponseHandler.handle(AwsSyncClientHandler.java:93)
        at 
software.amazon.awssdk.core.internal.handler.BaseClientHandler.lambda$successTransformationResponseHandler$7(BaseClientHandler.java:279)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:50)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:38)
        at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:72)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:55)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:39)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:81)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:36)
        at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
        at 
software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:56)
        at 
software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:36)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.executeWithTimer(ApiCallTimeoutTrackingStage.java:80)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:60)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:42)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:50)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:32)
        at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
        at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:37)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:26)
        at 
software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient$RequestExecutionBuilderImpl.execute(AmazonSyncHttpClient.java:224)
        at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.invoke(BaseSyncClientHandler.java:103)
        at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.doExecute(BaseSyncClientHandler.java:173)
        at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$1(BaseSyncClientHandler.java:80)
        at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:182)
        at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:74)
        at 
software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:45)
        at 
software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:53)
        at 
software.amazon.awssdk.services.s3.DefaultS3Client.uploadPart(DefaultS3Client.java:11515)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$uploadPart$16(S3AFileSystem.java:3345)
        at 
org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:651)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.uploadPart(S3AFileSystem.java:3342)
        at 
org.apache.hadoop.fs.s3a.WriteOperationHelper.lambda$uploadPart$9(WriteOperationHelper.java:594)
        at 
org.apache.hadoop.fs.store.audit.AuditingFunctions.lambda$withinAuditSpan$0(AuditingFunctions.java:62)
        at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:122)
        ... 24 more
        Suppressed: software.amazon.awssdk.core.exception.SdkClientException: 
Request attempt 1 failure: null (Service: S3, Status Code: 500, Request ID: 
TZ0558DJES2Z0HF3, Extended Request ID: 
NZRAAV8C5lcYmt1knZyj3jQbUxvEQFgOdOI+Od/xwx+jZVTLOTZyXb87acVfI6SoE5IybofBiDY=)
        Suppressed: software.amazon.awssdk.core.exception.SdkClientException: 
Request attempt 2 failure: Your socket connection to the server was not read 
from or written to within the timeout period. Idle connections will be closed. 
(Service: S3, Status Code: 400, Request ID: FRGQYV48RNTQDADX, Extended Request 
ID: 
r7drlnxjjdIbZCfmCg5i5Y/a622hxSMO+JK7idhL49zJOvLvCwwDUPxbgW8rbxAWTvoN76cCnp4=)


{code}


> S3A: Unable to recover from failure of multipart block upload attempt
> ---------------------------------------------------------------------
>
>                 Key: HADOOP-19221
>                 URL: https://issues.apache.org/jira/browse/HADOOP-19221
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/s3
>    Affects Versions: 3.4.0
>            Reporter: Steve Loughran
>            Assignee: Steve Loughran
>            Priority: Major
>              Labels: pull-request-available
>
> If a multipart PUT request fails for some reason (e.g. networrk error) then 
> all subsequent retry attempts fail with a 400 Response and ErrorCode 
> RequestTimeout .
> {code}
> Your socket connection to the server was not read from or written to within 
> the timeout period. Idle connections will be closed. (Service: Amazon S3; 
> Status Code: 400; Error Code: RequestTimeout; Request ID:; S3 Extended 
> Request ID:
> {code}
> The list of supporessed exceptions contains the root cause (the initial 
> failure was a 500); all retries failed to upload properly from the source 
> input stream {{RequestBody.fromInputStream(fileStream, size)}}.
> Hypothesis: the mark/reset stuff doesn't work for input streams. On the v1 
> sdk we would build a multipart block upload request passing in (file, offset, 
> length), the way we are now doing this doesn't recover.
> probably fixable by providing our own {{ContentStreamProvider}} 
> implementations for
> # file + offset + length
> # bytebuffer
> # byte array
> The sdk does have explicit support for the memory ones, but they copy the 
> data blocks first. we don't want that as it would double the memory 
> requirements of active blocks.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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

Reply via email to