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

Steve Loughran commented on HADOOP-19295:
-----------------------------------------

{code}
Write block 383org.apache.hadoop.fs.s3a.AWSApiCallTimeoutException: upload part 
#2 upload ID 
uV3wPWbQTgAFPMXuTxDK2ePOEAZEf3wmkS76wLds14u.HUOx_2P0pLaB7sHCnfuxRIEmEG9qLoQ98JHAKqUPFhQRLuJq5uBb16rFh9Le6xzX60uY8l6CeVEtgAlAUr3F
 on testfile: software.amazon.awssdk.core.exception.ApiCallTimeoutException: 
Client execution did not complete before the specified timeout configuration: 
60000 millis
        at 
org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:223)
        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:592)
        at 
org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.lambda$uploadBlockAsync$1(S3ABlockOutputStream.java:911)
        at 
org.apache.hadoop.thirdparty.com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:131)
        at 
org.apache.hadoop.thirdparty.com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:75)
        at 
org.apache.hadoop.thirdparty.com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:82)
        at 
org.apache.hadoop.util.SemaphoredDelegatingExecutor$RunnableWithPermitRelease.run(SemaphoredDelegatingExecutor.java:225)
        at 
org.apache.hadoop.util.SemaphoredDelegatingExecutor$RunnableWithPermitRelease.run(SemaphoredDelegatingExecutor.java:225)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:750)
Caused by: software.amazon.awssdk.core.exception.ApiCallTimeoutException: 
Client execution did not complete before the specified timeout configuration: 
60000 millis
        at 
software.amazon.awssdk.core.exception.ApiCallTimeoutException$BuilderImpl.build(ApiCallTimeoutException.java:97)
        at 
software.amazon.awssdk.core.exception.ApiCallTimeoutException.create(ApiCallTimeoutException.java:38)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.generateApiCallTimeoutException(ApiCallTimeoutTrackingStage.java:151)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.handleInterruptedException(ApiCallTimeoutTrackingStage.java:139)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.translatePipelineException(ApiCallTimeoutTrackingStage.java:107)
        at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:62)
        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:11565)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$uploadPart$16(S3AFileSystem.java:3367)
        at 
org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:651)
        at 
org.apache.hadoop.fs.s3a.S3AFileSystem.uploadPart(S3AFileSystem.java:3364)
        at 
org.apache.hadoop.fs.s3a.WriteOperationHelper.lambda$uploadPart$9(WriteOperationHelper.java:597)
        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)
        ... 15 more
{code}


> S3A: fs.s3a.connection.request.timeout too low for large uploads over slow 
> links
> --------------------------------------------------------------------------------
>
>                 Key: HADOOP-19295
>                 URL: https://issues.apache.org/jira/browse/HADOOP-19295
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/s3
>    Affects Versions: 3.4.0, 3.4.1
>            Reporter: Steve Loughran
>            Assignee: Steve Loughran
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 3.5.0, 3.4.1
>
>
> The value of {{fs.s3a.connection.request.timeout}} (default = 60s} is too low 
> for large uploads over slow connections.
> I suspect something changed between the v1 and v2 SDK versions so that put 
> was exempt from the normal timeouts, It is not and now surfaces in failures 
> to upload 1+ GB files over slower network connections. Smailer (for example 
> 128 MB) files work.
> The parallel queuing of writes in the S3ABlockOutputStream is helping create 
> this problem as it queues multiple blocks at the same time, so per-block 
> bandwidth becomes available/blocks ; four blocks cuts the capacity down by a 
> quarter.
> The fix is straightforward: use a much bigger timeout. I'm going to propose 
> 15 minutes. We need to strike a balance between upload time allocation and 
> other requests timing out.
> I do worry about other consequences; we've found that timeout exception happy 
> to hide the underlying causes of retry failures -so in fact this may be 
> better for all but a server hanging after the HTTP request is initiated.
> too bad we can't alter the timeout for different requests



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