Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HADOOP-19295. S3A: large uploads can timeout over slow links #7089

Merged

Conversation

steveloughran
Copy link
Contributor

@steveloughran steveloughran commented Oct 1, 2024

Long term fix; #7087 is the quick one

This sets a different timeout for put/post calls to all other requests, so that they don't time out uploads.

option fs.s3a.connection.part.upload.timeout
default 15m (just a guess..we could make bigger)

Although itests show the option is being applied to put/part uploads, and interpreted by the SDK, a full command line tests is showing a failure at 60s, as before.

fMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz.COPYING:
Retried 1: org.apache.hadoop.fs.s3a.AWSApiCallTimeoutException:
upload part #3
on hadoop-3.4.1.tar.gz.COPYING:
software.amazon.awssdk.core.exception.ApiCallAttemptTimeoutException:
HTTP request execution did not complete before the specified timeout
configuration: 60000 millis

This actually validates that the upload recovery is good, which makes me happy

2024-10-01 18:30:50,287 [s3a-transfer-stevel-london-bounded-pool1-t1] INFO impl.UploadContentProviders
(UploadContentProviders.java:newStream(278)) -
Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0001-751923718162888182.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=7, currentStream=null}

even though the upload doesn't actually take.

I've modified the different 60s timeouts to help identify which is causing http
request timeouts. hopefully it is something we are actually setting ourselves.

How was this patch tested?

new ITest and manual upload of a gigabyte file

For code changes:

  • Does the title or this PR starts with the corresponding JIRA issue id (e.g. 'HADOOP-17799. Your PR title ...')?
  • Object storage: have the integration tests been executed and the endpoint declared according to the connector-specific documentation?
  • If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under ASF 2.0?
  • If applicable, have you updated the LICENSE, LICENSE-binary, NOTICE-binary files?

This sets a different timeout for put/post calls to
all other requests, so that

option fs.s3a.connection.part.upload.timeout
default 15m

Although itests show the option is being applied to put/part uploads,
and interpreted by the SDK, a full command line tests is showing a
failure at 60s, as before.

fMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_:
 Retried 1: org.apache.hadoop.fs.s3a.AWSApiCallTimeoutException:
 upload part apache#3
 on hadoop-3.4.1.tar.gz._COPYING_:
 software.amazon.awssdk.core.exception.ApiCallAttemptTimeoutException:
 HTTP request execution did not complete before the specified timeout
 configuration: 60000 millis

This actually validates that the upload recovery is good, which makes
me happy

2024-10-01 18:30:50,287 [s3a-transfer-stevel-london-bounded-pool1-t1] INFO  impl.UploadContentProviders
 (UploadContentProviders.java:newStream(278)) -
  Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0001-751923718162888182.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=7, currentStream=null}

Change-Id: I84e594eae55746a85f58b05ad376173ddbbc3ad1
@steveloughran
Copy link
Contributor Author

log of a failure with this PR it is timing out after 60s, just doing it differently from 3.4.1 as we log at info recoveries of the content provider (whose log message I'm now tuning)

This is complex enough that I think #7087 should be the 3.4.1 solution; this will be the stable one.

 time bin/hadoop fs -put -f /Users/stevel/Projects/Misc/client-validator/downloads/hadoop-3.4.1-RC2/hadoop-3.4.1.tar.gz s3a://stevel-london/hadoop-3.4.1.tar.gz
SLF4J: Class path contains multiple SLF4J bindings.
SLF4J: Found binding in [jar:file:/Users/stevel/Projects/Releases/hadoop-3.5.0-SNAPSHOT/share/hadoop/common/lib/bundle-2.25.53.jar!/software/amazon/awssdk/thirdparty/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in [jar:file:/Users/stevel/Projects/Releases/hadoop-3.5.0-SNAPSHOT/share/hadoop/tools/lib/bundle-2.25.53.jar!/software/amazon/awssdk/thirdparty/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation.
SLF4J: Actual binding is of type [software.amazon.awssdk.thirdparty.org.slf4j.impl.internal.ILoggerFactoryAdapter]
2024-10-01 18:25:48,092 [s3a-transfer-stevel-london-bounded-pool1-t2] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0002-4939124672357233248.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=2, currentStream=null}
2024-10-01 18:25:48,153 [s3a-transfer-stevel-london-bounded-pool1-t1] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0001-751923718162888182.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=2, currentStream=null}
2024-10-01 18:25:48,201 [s3a-transfer-stevel-london-bounded-pool1-t4] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0004-2287376701501989244.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=2, currentStream=null}
2024-10-01 18:25:48,242 [s3a-transfer-stevel-london-bounded-pool1-t3] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0003-7101091016371104860.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=2, currentStream=null}
2024-10-01 18:26:48,277 [s3a-transfer-stevel-london-bounded-pool1-t1] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0001-751923718162888182.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=3, currentStream=null}
2024-10-01 18:26:48,303 [s3a-transfer-stevel-london-bounded-pool1-t4] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0004-2287376701501989244.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=3, currentStream=null}
2024-10-01 18:26:48,343 [s3a-transfer-stevel-london-bounded-pool1-t2] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0002-4939124672357233248.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=3, currentStream=null}
2024-10-01 18:26:48,368 [s3a-transfer-stevel-london-bounded-pool1-t3] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0003-7101091016371104860.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=3, currentStream=null}
2024-10-01 18:27:48,293 [s3a-transfer-stevel-london-bounded-pool1-t1] INFO  s3a.WriteOperationHelper (WriteOperationHelper.java:operationRetried(184)) - upload part #1 upload ID POVKsWsu77JaQ36V7YyB9xtdy8O_ZMIBfl9lSjaFgNAHdpsRFQs4XBi3w_dnucA05vJhaHPLh_xsEZi.d0ptSXfMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_: Retried 0: org.apache.hadoop.fs.s3a.AWSApiCallTimeoutException: upload part #1 upload ID POVKsWsu77JaQ36V7YyB9xtdy8O_ZMIBfl9lSjaFgNAHdpsRFQs4XBi3w_dnucA05vJhaHPLh_xsEZi.d0ptSXfMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_: software.amazon.awssdk.core.exception.ApiCallAttemptTimeoutException: HTTP request execution did not complete before the specified timeout configuration: 60000 millis
2024-10-01 18:27:48,327 [s3a-transfer-stevel-london-bounded-pool1-t4] INFO  s3a.WriteOperationHelper (WriteOperationHelper.java:operationRetried(184)) - upload part #4 upload ID POVKsWsu77JaQ36V7YyB9xtdy8O_ZMIBfl9lSjaFgNAHdpsRFQs4XBi3w_dnucA05vJhaHPLh_xsEZi.d0ptSXfMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_: Retried 0: org.apache.hadoop.fs.s3a.AWSApiCallTimeoutException: upload part #4 upload ID POVKsWsu77JaQ36V7YyB9xtdy8O_ZMIBfl9lSjaFgNAHdpsRFQs4XBi3w_dnucA05vJhaHPLh_xsEZi.d0ptSXfMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_: software.amazon.awssdk.core.exception.ApiCallAttemptTimeoutException: HTTP request execution did not complete before the specified timeout configuration: 60000 millis
2024-10-01 18:27:48,358 [s3a-transfer-stevel-london-bounded-pool1-t2] INFO  s3a.WriteOperationHelper (WriteOperationHelper.java:operationRetried(184)) - upload part #2 upload ID POVKsWsu77JaQ36V7YyB9xtdy8O_ZMIBfl9lSjaFgNAHdpsRFQs4XBi3w_dnucA05vJhaHPLh_xsEZi.d0ptSXfMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_: Retried 0: org.apache.hadoop.fs.s3a.AWSApiCallTimeoutException: upload part #2 upload ID POVKsWsu77JaQ36V7YyB9xtdy8O_ZMIBfl9lSjaFgNAHdpsRFQs4XBi3w_dnucA05vJhaHPLh_xsEZi.d0ptSXfMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_: software.amazon.awssdk.core.exception.ApiCallAttemptTimeoutException: HTTP request execution did not complete before the specified timeout configuration: 60000 millis
2024-10-01 18:27:48,381 [s3a-transfer-stevel-london-bounded-pool1-t3] INFO  s3a.WriteOperationHelper (WriteOperationHelper.java:operationRetried(184)) - upload part #3 upload ID POVKsWsu77JaQ36V7YyB9xtdy8O_ZMIBfl9lSjaFgNAHdpsRFQs4XBi3w_dnucA05vJhaHPLh_xsEZi.d0ptSXfMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_: Retried 0: org.apache.hadoop.fs.s3a.AWSApiCallTimeoutException: upload part #3 upload ID POVKsWsu77JaQ36V7YyB9xtdy8O_ZMIBfl9lSjaFgNAHdpsRFQs4XBi3w_dnucA05vJhaHPLh_xsEZi.d0ptSXfMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_: software.amazon.awssdk.core.exception.ApiCallAttemptTimeoutException: HTTP request execution did not complete before the specified timeout configuration: 60000 millis
2024-10-01 18:27:48,804 [s3a-transfer-stevel-london-bounded-pool1-t1] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0001-751923718162888182.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=4, currentStream=null}
2024-10-01 18:27:48,935 [s3a-transfer-stevel-london-bounded-pool1-t2] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0002-4939124672357233248.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=4, currentStream=null}
2024-10-01 18:27:49,330 [s3a-transfer-stevel-london-bounded-pool1-t4] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0004-2287376701501989244.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=4, currentStream=null}
2024-10-01 18:27:49,743 [s3a-transfer-stevel-london-bounded-pool1-t3] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0003-7101091016371104860.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=4, currentStream=null}
2024-10-01 18:28:48,874 [s3a-transfer-stevel-london-bounded-pool1-t1] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0001-751923718162888182.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=5, currentStream=null}
2024-10-01 18:28:49,056 [s3a-transfer-stevel-london-bounded-pool1-t2] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0002-4939124672357233248.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=5, currentStream=null}
2024-10-01 18:28:49,384 [s3a-transfer-stevel-london-bounded-pool1-t4] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0004-2287376701501989244.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=5, currentStream=null}
2024-10-01 18:28:49,787 [s3a-transfer-stevel-london-bounded-pool1-t3] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0003-7101091016371104860.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=5, currentStream=null}
2024-10-01 18:29:49,014 [s3a-transfer-stevel-london-bounded-pool1-t1] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0001-751923718162888182.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=6, currentStream=null}
2024-10-01 18:29:49,198 [s3a-transfer-stevel-london-bounded-pool1-t2] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0002-4939124672357233248.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=6, currentStream=null}
2024-10-01 18:29:49,592 [s3a-transfer-stevel-london-bounded-pool1-t4] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0004-2287376701501989244.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=6, currentStream=null}
2024-10-01 18:29:50,030 [s3a-transfer-stevel-london-bounded-pool1-t3] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0003-7101091016371104860.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=6, currentStream=null}
2024-10-01 18:30:49,030 [s3a-transfer-stevel-london-bounded-pool1-t1] INFO  s3a.WriteOperationHelper (WriteOperationHelper.java:operationRetried(184)) - upload part #1 upload ID POVKsWsu77JaQ36V7YyB9xtdy8O_ZMIBfl9lSjaFgNAHdpsRFQs4XBi3w_dnucA05vJhaHPLh_xsEZi.d0ptSXfMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_: Retried 1: org.apache.hadoop.fs.s3a.AWSApiCallTimeoutException: upload part #1 upload ID POVKsWsu77JaQ36V7YyB9xtdy8O_ZMIBfl9lSjaFgNAHdpsRFQs4XBi3w_dnucA05vJhaHPLh_xsEZi.d0ptSXfMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_: software.amazon.awssdk.core.exception.ApiCallAttemptTimeoutException: HTTP request execution did not complete before the specified timeout configuration: 60000 millis
2024-10-01 18:30:49,213 [s3a-transfer-stevel-london-bounded-pool1-t2] INFO  s3a.WriteOperationHelper (WriteOperationHelper.java:operationRetried(184)) - upload part #2 upload ID POVKsWsu77JaQ36V7YyB9xtdy8O_ZMIBfl9lSjaFgNAHdpsRFQs4XBi3w_dnucA05vJhaHPLh_xsEZi.d0ptSXfMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_: Retried 1: org.apache.hadoop.fs.s3a.AWSApiCallTimeoutException: upload part #2 upload ID POVKsWsu77JaQ36V7YyB9xtdy8O_ZMIBfl9lSjaFgNAHdpsRFQs4XBi3w_dnucA05vJhaHPLh_xsEZi.d0ptSXfMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_: software.amazon.awssdk.core.exception.ApiCallAttemptTimeoutException: HTTP request execution did not complete before the specified timeout configuration: 60000 millis
2024-10-01 18:30:49,629 [s3a-transfer-stevel-london-bounded-pool1-t4] INFO  s3a.WriteOperationHelper (WriteOperationHelper.java:operationRetried(184)) - upload part #4 upload ID POVKsWsu77JaQ36V7YyB9xtdy8O_ZMIBfl9lSjaFgNAHdpsRFQs4XBi3w_dnucA05vJhaHPLh_xsEZi.d0ptSXfMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_: Retried 1: org.apache.hadoop.fs.s3a.AWSApiCallTimeoutException: upload part #4 upload ID POVKsWsu77JaQ36V7YyB9xtdy8O_ZMIBfl9lSjaFgNAHdpsRFQs4XBi3w_dnucA05vJhaHPLh_xsEZi.d0ptSXfMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_: software.amazon.awssdk.core.exception.ApiCallAttemptTimeoutException: HTTP request execution did not complete before the specified timeout configuration: 60000 millis
2024-10-01 18:30:50,037 [s3a-transfer-stevel-london-bounded-pool1-t3] INFO  s3a.WriteOperationHelper (WriteOperationHelper.java:operationRetried(184)) - upload part #3 upload ID POVKsWsu77JaQ36V7YyB9xtdy8O_ZMIBfl9lSjaFgNAHdpsRFQs4XBi3w_dnucA05vJhaHPLh_xsEZi.d0ptSXfMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_: Retried 1: org.apache.hadoop.fs.s3a.AWSApiCallTimeoutException: upload part #3 upload ID POVKsWsu77JaQ36V7YyB9xtdy8O_ZMIBfl9lSjaFgNAHdpsRFQs4XBi3w_dnucA05vJhaHPLh_xsEZi.d0ptSXfMrSmr7TgYCqPw1C2tI_tM0A_TzaYJWPfwVxnE9MyC on hadoop-3.4.1.tar.gz._COPYING_: software.amazon.awssdk.core.exception.ApiCallAttemptTimeoutException: HTTP request execution did not complete before the specified timeout configuration: 60000 millis
2024-10-01 18:30:50,287 [s3a-transfer-stevel-london-bounded-pool1-t1] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0001-751923718162888182.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=7, currentStream=null}
2024-10-01 18:30:50,814 [s3a-transfer-stevel-london-bounded-pool1-t4] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0004-2287376701501989244.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=7, currentStream=null}
2024-10-01 18:30:51,459 [s3a-transfer-stevel-london-bounded-pool1-t2] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0002-4939124672357233248.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=7, currentStream=null}
2024-10-01 18:30:51,687 [s3a-transfer-stevel-london-bounded-pool1-t3] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(278)) - Stream created more than once: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0003-7101091016371104860.tmp, offset=0} BaseContentProvider{size=67108864, streamCreationCount=7, currentStream=null}
^C^C2024-10-01 18:31:23,798 [s3a-transfer-stevel-london-bounded-pool1-t4] WARN  s3a.S3ABlockOutputStream (S3ABlockOutputStream.java:progressChanged(1295)) - Transfer failure of block FileBlock{index=4, destFile=/tmp/hadoop-stevel/s3a/s3ablock-0004-2287376701501989244.tmp, state=Upload, dataSize=67108864, limit=67108864}
2024-10-01 18:31:23,802 [main] ERROR util.BlockingThreadPoolExecutorService (BlockingThreadPoolExecutorService.java:rejectedExecution(141)) - Could not submit task to executor java.util.concurrent.ThreadPoolExecutor@62891fc8[Shutting down, pool size = 3, active threads = 3, queued tasks = 0, completed tasks = 1]
2024-10-01 18:31:23,829 [s3a-transfer-stevel-london-bounded-pool1-t2] WARN  s3a.S3ABlockOutputStream (S3ABlockOutputStream.java:progressChanged(1295)) - Transfer failure of block FileBlock{index=2, destFile=/tmp/hadoop-stevel/s3a/s3ablock-0002-4939124672357233248.tmp, state=Upload, dataSize=67108864, limit=67108864}
2024-10-01 18:31:23,846 [s3a-transfer-stevel-london-bounded-pool1-t3] WARN  s3a.S3ABlockOutputStream (S3ABlockOutputStream.java:progressChanged(1295)) - Transfer failure of block FileBlock{index=3, destFile=/tmp/hadoop-stevel/s3a/s3ablock-0003-7101091016371104860.tmp, state=Upload, dataSize=67108864, limit=67108864}
2024-10-01 18:31:23,873 [s3a-transfer-stevel-london-bounded-pool1-t1] WARN  s3a.S3ABlockOutputStream (S3ABlockOutputStream.java:progressChanged(1295)) - Transfer failure of block FileBlock{index=1, destFile=/tmp/hadoop-stevel/s3a/s3ablock-0001-751923718162888182.tmp, state=Upload, dataSize=67108864, limit=67108864}
2024-10-01 18:31:23,883 [shutdown-hook-0] INFO  statistics.IOStatisticsLogging (IOStatisticsLogging.java:logIOStatisticsAtLevel(269)) - IOStatistics: counters=((action_http_head_request=2)
(audit_request_execution=17)
(audit_span_creation=6)
(files_created=1)
(filesystem_close=1)
(filesystem_initialization=1)
(ignored_errors=8)
(object_list_request=2)
(object_metadata_request=2)
(object_multipart_initiated=1)
(op_create=1)
(op_exists=1)
(op_get_file_status=2)
(op_get_file_status.failures=2)
(op_glob_status=1)
(store_client_creation=1)
(store_io_request=29)
(store_io_retry=24));

gauges=();

minimums=((action_http_head_request.min=42)
(filesystem_close.min=450)
(filesystem_initialization.min=507)
(object_list_request.min=40)
(object_multipart_initiated.min=110)
(op_create.min=15)
(op_exists.min=0)
(op_get_file_status.failures.min=82)
(op_glob_status.min=545)
(store_client_creation.min=449));

maximums=((action_http_head_request.max=445)
(filesystem_close.max=450)
(filesystem_initialization.max=507)
(object_list_request.max=70)
(object_multipart_initiated.max=110)
(op_create.max=15)
(op_exists.max=0)
(op_get_file_status.failures.max=528)
(op_glob_status.max=545)
(store_client_creation.max=449));

means=((action_http_head_request.mean=(samples=2, sum=487, mean=243.5000))
(filesystem_close.mean=(samples=1, sum=450, mean=450.0000))
(filesystem_initialization.mean=(samples=1, sum=507, mean=507.0000))
(object_list_request.mean=(samples=2, sum=110, mean=55.0000))
(object_multipart_initiated.mean=(samples=1, sum=110, mean=110.0000))
(op_create.mean=(samples=1, sum=15, mean=15.0000))
(op_exists.mean=(samples=1, sum=0, mean=0.0000))
(op_get_file_status.failures.mean=(samples=2, sum=610, mean=305.0000))
(op_glob_status.mean=(samples=1, sum=545, mean=545.0000))
(store_client_creation.mean=(samples=1, sum=449, mean=449.0000)));

2024-10-01 18:31:23,887 [main] WARN  s3a.S3AInstrumentation (S3AInstrumentation.java:close(1667)) - Closing output stream statistics while data is still marked as pending upload in OutputStreamStatistics{counters=((stream_write_exceptions=4) (op_hflush=0) (action_executor_acquired=5) (op_abort=0) (multipart_upload_completed=0) (stream_write_total_time=0) (object_multipart_initiated=1) (multipart_upload_part_put.failures=12) (op_abort.failures=0) (stream_write_total_data=268435456) (multipart_upload_part_put=12) (stream_write_bytes=402653184) (object_multipart_aborted=1) (stream_write_exceptions_completing_upload=0) (multipart_upload_completed.failures=0) (object_put_request=0) (op_hsync=0) (object_multipart_initiated.failures=0) (action_executor_acquired.failures=0) (stream_write_queue_duration=0) (stream_write_block_uploads=5) (object_put_request.failures=0) (object_multipart_aborted.failures=1) (committer_magic_marker_put.failures=0) (committer_magic_marker_put=0));
gauges=((stream_write_block_uploads_pending=1) (stream_write_block_uploads_data_pending=67108864) (stream_write_block_uploads_active=0));
minimums=((multipart_upload_part_put.min=-1) (op_abort.min=-1) (object_multipart_initiated.failures.min=-1) (action_executor_acquired.failures.min=-1) (committer_magic_marker_put.min=-1) (object_put_request.failures.min=-1) (object_multipart_aborted.failures.min=7) (op_abort.failures.min=-1) (committer_magic_marker_put.failures.min=-1) (multipart_upload_completed.failures.min=-1) (object_multipart_initiated.min=144) (multipart_upload_completed.min=-1) (object_put_request.min=-1) (object_multipart_aborted.min=-1) (multipart_upload_part_put.failures.min=32162) (action_executor_acquired.min=0));
maximums=((committer_magic_marker_put.failures.max=-1) (op_abort.max=-1) (object_put_request.max=-1) (multipart_upload_completed.max=-1) (op_abort.failures.max=-1) (multipart_upload_part_put.max=-1) (committer_magic_marker_put.max=-1) (object_multipart_aborted.failures.max=7) (object_multipart_aborted.max=-1) (object_multipart_initiated.failures.max=-1) (multipart_upload_part_put.failures.max=180307) (multipart_upload_completed.failures.max=-1) (action_executor_acquired.max=395664) (object_put_request.failures.max=-1) (object_multipart_initiated.max=144) (action_executor_acquired.failures.max=-1));
means=((multipart_upload_completed.failures.mean=(samples=0, sum=0, mean=0.0000)) (object_multipart_aborted.mean=(samples=0, sum=0, mean=0.0000)) (multipart_upload_part_put.failures.mean=(samples=12, sum=1573293, mean=131107.7500)) (action_executor_acquired.mean=(samples=9, sum=395672, mean=43963.5556)) (object_multipart_initiated.failures.mean=(samples=0, sum=0, mean=0.0000)) (committer_magic_marker_put.failures.mean=(samples=0, sum=0, mean=0.0000)) (object_put_request.mean=(samples=0, sum=0, mean=0.0000)) (action_executor_acquired.failures.mean=(samples=0, sum=0, mean=0.0000)) (committer_magic_marker_put.mean=(samples=0, sum=0, mean=0.0000)) (multipart_upload_completed.mean=(samples=0, sum=0, mean=0.0000)) (object_put_request.failures.mean=(samples=0, sum=0, mean=0.0000)) (object_multipart_aborted.failures.mean=(samples=1, sum=7, mean=7.0000)) (multipart_upload_part_put.mean=(samples=0, sum=0, mean=0.0000)) (object_multipart_initiated.mean=(samples=1, sum=144, mean=144.0000)) (op_abort.mean=(samples=0, sum=0, mean=0.0000)) (op_abort.failures.mean=(samples=0, sum=0, mean=0.0000)));
, blocksActive=0, blockUploadsCompleted=4, blocksAllocated=6, blocksReleased=5, blocksActivelyAllocated=1, transferDuration=1583052 ms, totalUploadDuration=1583060 ms, effectiveBandwidth=169567.45543441185 bytes/s}
2024-10-01 18:31:23,890 [main] DEBUG shell.Command (Command.java:displayError(481)) - put failure
org.apache.hadoop.fs.PathIOException: `s3a://stevel-london': FileSystem is closed!
        at org.apache.hadoop.fs.s3a.S3AFileSystem.checkNotClosed(S3AFileSystem.java:4473)
        at org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2868)
        at org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2890)
        at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:3988)
        at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:462)
        at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1862)
        at org.apache.hadoop.fs.FileSystem.deleteOnExit(FileSystem.java:1805)
        at org.apache.hadoop.fs.shell.CommandWithDestination$TargetFileSystem.writeStreamToFile(CommandWithDestination.java:508)
        at org.apache.hadoop.fs.shell.CommandWithDestination.copyStreamToTarget(CommandWithDestination.java:427)
        at org.apache.hadoop.fs.shell.CommandWithDestination.copyFileToTarget(CommandWithDestination.java:362)
        at org.apache.hadoop.fs.shell.CopyCommandWithMultiThread.copyFileToTarget(CopyCommandWithMultiThread.java:144)
        at org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:293)
        at org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:278)
        at org.apache.hadoop.fs.shell.Command.processPathInternal(Command.java:383)
        at org.apache.hadoop.fs.shell.Command.processPaths(Command.java:346)
        at org.apache.hadoop.fs.shell.Command.processPathArgument(Command.java:319)
        at org.apache.hadoop.fs.shell.CommandWithDestination.processPathArgument(CommandWithDestination.java:273)
        at org.apache.hadoop.fs.shell.Command.processArgument(Command.java:301)
        at org.apache.hadoop.fs.shell.Command.processArguments(Command.java:285)
        at org.apache.hadoop.fs.shell.CommandWithDestination.processArguments(CommandWithDestination.java:244)
        at org.apache.hadoop.fs.shell.CopyCommandWithMultiThread.processArguments(CopyCommandWithMultiThread.java:89)
        at org.apache.hadoop.fs.shell.CopyCommands$Put.processArguments(CopyCommands.java:314)
        at org.apache.hadoop.fs.shell.FsCommand.processRawArguments(FsCommand.java:121)
        at org.apache.hadoop.fs.shell.Command.run(Command.java:192)
        at org.apache.hadoop.fs.FsShell.run(FsShell.java:327)
        at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:82)
        at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:97)
        at org.apache.hadoop.fs.FsShell.main(FsShell.java:390)

________________________________________________________
Executed in  398.43 secs    fish           external
   usr time   27.14 secs   49.00 micros   27.14 secs
   sys time    5.30 secs  775.00 micros    5.29 secs

@hadoop-yetus
Copy link

🎊 +1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 21s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+0 🆗 detsecrets 0m 0s detect-secrets was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 5 new or modified test files.
_ trunk Compile Tests _
+1 💚 mvninstall 32m 35s trunk passed
+1 💚 compile 0m 27s trunk passed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04
+1 💚 compile 0m 24s trunk passed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05
+1 💚 checkstyle 0m 20s trunk passed
+1 💚 mvnsite 0m 30s trunk passed
+1 💚 javadoc 0m 30s trunk passed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04
+1 💚 javadoc 0m 21s trunk passed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05
+1 💚 spotbugs 0m 46s trunk passed
+1 💚 shadedclient 21m 26s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+1 💚 mvninstall 0m 17s the patch passed
+1 💚 compile 0m 20s the patch passed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04
+1 💚 javac 0m 20s the patch passed
+1 💚 compile 0m 16s the patch passed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05
+1 💚 javac 0m 16s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
+1 💚 checkstyle 0m 12s the patch passed
+1 💚 mvnsite 0m 20s the patch passed
+1 💚 javadoc 0m 16s the patch passed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04
+1 💚 javadoc 0m 17s the patch passed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05
+1 💚 spotbugs 0m 42s the patch passed
+1 💚 shadedclient 21m 5s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 1m 57s hadoop-aws in the patch passed.
+1 💚 asflicense 0m 23s The patch does not generate ASF License warnings.
84m 24s
Subsystem Report/Notes
Docker ClientAPI=1.47 ServerAPI=1.47 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7089/1/artifact/out/Dockerfile
GITHUB PR #7089
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets
uname Linux 44a62865b77f 5.15.0-117-generic #127-Ubuntu SMP Fri Jul 5 20:13:28 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 7159fc5
Default Java Private Build-1.8.0_422-8u422-b05-1~20.04-b05
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_422-8u422-b05-1~20.04-b05
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7089/1/testReport/
Max. process+thread count 552 (vs. ulimit of 5500)
modules C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7089/1/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@steveloughran
Copy link
Contributor Author

Changing the different timeouts from 60s shows that the request timeout is still the timeout problem; and that on trunk recovery is different from 3.4.1 as strem recreation is logged. going to improve diags there (start time, and at debug stack)

@steveloughran
Copy link
Contributor Author

next iteration will improve tostring of upload with start time.

2024-10-02 14:32:40,447 [s3a-transfer-stevel-london-bounded-pool1-t3] INFO  impl.UploadContentProviders (UploadContentProviders.java:newStream(286)) - Stream recreated: FileWithOffsetContentProvider{file=/tmp/hadoop-stevel/s3a/s3ablock-0003-8089030384052356560.tmp, offset=0} BaseContentProvider{size=67108864, initiated at 2024-10-02T14:30:18.416, streamCreationCount=10, currentStream=null}

and if debug log enabled, full stack is generated

2024-10-02 14:32:39,124 [s3a-transfer-stevel-london-bounded-pool1-t4] DEBUG impl.UploadContentProviders (UploadContentProviders.java:newStream(288)) - Stream creation stack
java.lang.Exception: here
        at org.apache.hadoop.fs.s3a.impl.UploadContentProviders$BaseContentProvider.newStream(UploadContentProviders.java:288)
        at software.amazon.awssdk.core.internal.handler.BaseClientHandler.lambda$getBody$3(BaseClientHandler.java:139)
        at software.amazon.awssdk.core.internal.http.StreamManagingStage$ClosingStreamProvider.newStream(StreamManagingStage.java:78)
        at java.util.Optional.map(Optional.java:215)
        at software.amazon.awssdk.http.apache.internal.RepeatableInputStreamRequestEntity.getContent(RepeatableInputStreamRequestEntity.java:123)
        at software.amazon.awssdk.http.apache.internal.RepeatableInputStreamRequestEntity.<init>(RepeatableInputStreamRequestEntity.java:98)
        at software.amazon.awssdk.http.apache.internal.impl.ApacheHttpRequestFactory.wrapEntity(ApacheHttpRequestFactory.java:153)
        at software.amazon.awssdk.http.apache.internal.impl.ApacheHttpRequestFactory.createApacheRequest(ApacheHttpRequestFactory.java:133)
        at software.amazon.awssdk.http.apache.internal.impl.ApacheHttpRequestFactory.create(ApacheHttpRequestFactory.java:55)
        at software.amazon.awssdk.http.apache.ApacheHttpClient.toApacheRequest(ApacheHttpClient.java:262)
        at software.amazon.awssdk.http.apache.ApacheHttpClient.prepareRequest(ApacheHttpClient.java:227)
        at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.executeHttpRequest(MakeHttpRequestStage.java:69)
        at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.execute(MakeHttpRequestStage.java:57)
        at software.amazon.awssdk.core.internal.http.pipeline.stages.MakeHttpRequestStage.execute(MakeHttpRequestStage.java:40)
        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.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.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.impl.S3AStoreImpl.lambda$uploadPart$4(S3AStoreImpl.java:610)
        at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:651)
        at org.apache.hadoop.fs.s3a.impl.S3AStoreImpl.uploadPart(S3AStoreImpl.java:607)
        at org.apache.hadoop.fs.s3a.S3AFileSystem$WriteOperationHelperCallbacksImpl.uploadPart(S3AFileSystem.java:1988)
        at org.apache.hadoop.fs.s3a.WriteOperationHelper.lambda$uploadPart$9(WriteOperationHelper.java:595)
        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)
        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:207)
        at org.apache.hadoop.fs.s3a.WriteOperationHelper.uploadPart(WriteOperationHelper.java:590)
        at org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.lambda$uploadBlockAsync$1(S3ABlockOutputStream.java:1048)
        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)
2

This sets a different timeout for put/post calls to
all other requests.

This commit sets apiCallAttemptTimeout() as well as the apiCallTimeout();
both need to be set for the extended timeouts to get picked up.

Change-Id: I30b3832c5240ba3d655c5bfd550aab18c5767b4f
@steveloughran steveloughran marked this pull request as ready for review October 2, 2024 13:49
@steveloughran
Copy link
Contributor Author

working test run. note that even this was (accidentally) through a VPN, upload performance is closer to v1; will retest without the VPN after running the itests

 time bin/hadoop fs -put -f /Users/stevel/Projects/Misc/client-validator/downloads/hadoop-3.4.1-RC2/hadoop-3.4.1.tar.gz s3a://stevel-london/hadoop-3.4.1.tar.gz
2024-10-02 14:34:38,431 [main] DEBUG s3a.WriteOperationHelper (WriteOperationHelper.java:initiateMultiPartUpload(276)) - Initiating Multipart upload to hadoop-3.4.1.tar.gz._COPYING_
2024-10-02 14:40:15,594 [main] DEBUG s3a.WriteOperationHelper (WriteOperationHelper.java:completeMPUwithRetries(360)) - Completing multipart upload pLAE7Y.7OpABhkA.LioNF4QlHktGRd8eO8Pf61RNB5jHMyo41ZanXDgTRGBS8EBtbDX0aZ1gpGKlAww_Umd_UREH2vA5.Cgd_NTIqOqjC1zBvy68adGcosBCnW_dcdcx with 15 parts
2024-10-02 14:40:20,870 [shutdown-hook-0] INFO  statistics.IOStatisticsLogging (IOStatisticsLogging.java:logIOStatisticsAtLevel(269)) - IOStatistics: counters=((action_executor_acquired=15)
(action_http_head_request=7)
(audit_request_execution=46)
(audit_span_creation=8)
(files_copied=1)
(files_copied_bytes=973970699)
(files_created=1)
(files_deleted=2)
(filesystem_close=1)
(filesystem_initialization=1)
(multipart_upload_completed=1)
(multipart_upload_part_put=15)
(object_copy_requests=1)
(object_delete_objects=2)
(object_delete_request=2)
(object_list_request=2)
(object_metadata_request=7)
(object_multipart_initiated=2)
(object_put_bytes=973970699)
(object_put_request_completed=15)
(op_create=1)
(op_delete=1)
(op_get_file_status=3)
(op_get_file_status.failures=1)
(op_glob_status=1)
(op_rename=1)
(store_client_creation=3)
(store_io_request=46)
(stream_write_block_uploads=30)
(stream_write_bytes=973970699)
(stream_write_queue_duration=301720)
(stream_write_total_data=1947941398)
(stream_write_total_time=1602051));

gauges=();

minimums=((action_executor_acquired.min=0)
(action_http_head_request.min=60)
(filesystem_close.min=2063)
(filesystem_initialization.min=515)
(object_delete_request.min=67)
(object_list_request.min=62)
(object_multipart_initiated.min=165)
(op_create.min=15)
(op_delete.min=75)
(op_get_file_status.failures.min=213)
(op_get_file_status.min=93)
(op_glob_status.min=689)
(op_rename.min=2754)
(store_client_creation.min=6)
(store_io_rate_limited_duration.min=0));

maximums=((action_executor_acquired.max=110972)
(action_http_head_request.max=665)
(filesystem_close.max=2063)
(filesystem_initialization.max=515)
(object_delete_request.max=70)
(object_list_request.max=142)
(object_multipart_initiated.max=165)
(op_create.max=15)
(op_delete.max=75)
(op_get_file_status.failures.max=213)
(op_get_file_status.max=673)
(op_glob_status.max=689)
(op_rename.max=2754)
(store_client_creation.max=467)
(store_io_rate_limited_duration.max=0));

means=((action_executor_acquired.mean=(samples=30, sum=603425, mean=20114.1667))
(action_http_head_request.mean=(samples=7, sum=1086, mean=155.1429))
(filesystem_close.mean=(samples=1, sum=2063, mean=2063.0000))
(filesystem_initialization.mean=(samples=1, sum=515, mean=515.0000))
(object_delete_request.mean=(samples=2, sum=137, mean=68.5000))
(object_list_request.mean=(samples=2, sum=204, mean=102.0000))
(object_multipart_initiated.mean=(samples=2, sum=279, mean=139.5000))
(op_create.mean=(samples=1, sum=15, mean=15.0000))
(op_delete.mean=(samples=1, sum=75, mean=75.0000))
(op_get_file_status.failures.mean=(samples=1, sum=213, mean=213.0000))
(op_get_file_status.mean=(samples=2, sum=766, mean=383.0000))
(op_glob_status.mean=(samples=1, sum=689, mean=689.0000))
(op_rename.mean=(samples=1, sum=2754, mean=2754.0000))
(store_client_creation.mean=(samples=3, sum=574, mean=191.3333))
(store_io_rate_limited_duration.mean=(samples=2, sum=0, mean=0.0000)));


________________________________________________________
Executed in  345.26 secs    fish           external
   usr time   28.34 secs    0.09 millis   28.34 secs
   sys time    6.55 secs    1.70 millis    6.54 secs

@hadoop-yetus
Copy link

💔 -1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 8m 6s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 1s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+0 🆗 detsecrets 0m 0s detect-secrets was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 5 new or modified test files.
_ trunk Compile Tests _
-1 ❌ mvninstall 0m 22s /branch-mvninstall-root.txt root in trunk failed.
-1 ❌ compile 0m 22s /branch-compile-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04.txt hadoop-aws in trunk failed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04.
-1 ❌ compile 0m 21s /branch-compile-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_422-8u422-b05-1~20.04-b05.txt hadoop-aws in trunk failed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05.
-0 ⚠️ checkstyle 0m 20s /buildtool-branch-checkstyle-hadoop-tools_hadoop-aws.txt The patch fails to run checkstyle in hadoop-aws
-1 ❌ mvnsite 0m 22s /branch-mvnsite-hadoop-tools_hadoop-aws.txt hadoop-aws in trunk failed.
-1 ❌ javadoc 0m 19s /branch-javadoc-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04.txt hadoop-aws in trunk failed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04.
-1 ❌ javadoc 0m 22s /branch-javadoc-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_422-8u422-b05-1~20.04-b05.txt hadoop-aws in trunk failed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05.
-1 ❌ spotbugs 0m 22s /branch-spotbugs-hadoop-tools_hadoop-aws.txt hadoop-aws in trunk failed.
+1 💚 shadedclient 2m 29s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
-1 ❌ mvninstall 0m 22s /patch-mvninstall-hadoop-tools_hadoop-aws.txt hadoop-aws in the patch failed.
-1 ❌ compile 0m 22s /patch-compile-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04.txt hadoop-aws in the patch failed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04.
-1 ❌ javac 0m 22s /patch-compile-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04.txt hadoop-aws in the patch failed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04.
-1 ❌ compile 0m 22s /patch-compile-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_422-8u422-b05-1~20.04-b05.txt hadoop-aws in the patch failed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05.
-1 ❌ javac 0m 22s /patch-compile-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_422-8u422-b05-1~20.04-b05.txt hadoop-aws in the patch failed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05.
+1 💚 blanks 0m 0s The patch has no blanks issues.
-0 ⚠️ checkstyle 0m 20s /buildtool-patch-checkstyle-hadoop-tools_hadoop-aws.txt The patch fails to run checkstyle in hadoop-aws
-1 ❌ mvnsite 0m 22s /patch-mvnsite-hadoop-tools_hadoop-aws.txt hadoop-aws in the patch failed.
-1 ❌ javadoc 4m 50s /patch-javadoc-hadoop-tools_hadoop-aws-jdkUbuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04.txt hadoop-aws in the patch failed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04.
-1 ❌ javadoc 0m 21s /patch-javadoc-hadoop-tools_hadoop-aws-jdkPrivateBuild-1.8.0_422-8u422-b05-1~20.04-b05.txt hadoop-aws in the patch failed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05.
-1 ❌ spotbugs 0m 22s /patch-spotbugs-hadoop-tools_hadoop-aws.txt hadoop-aws in the patch failed.
-1 ❌ shadedclient 8m 32s patch has errors when building and testing our client artifacts.
_ Other Tests _
-1 ❌ unit 0m 22s /patch-unit-hadoop-tools_hadoop-aws.txt hadoop-aws in the patch failed.
+1 💚 asflicense 0m 43s The patch does not generate ASF License warnings.
23m 32s
Subsystem Report/Notes
Docker ClientAPI=1.47 ServerAPI=1.47 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7089/2/artifact/out/Dockerfile
GITHUB PR #7089
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets
uname Linux 9f779eb6c260 5.15.0-117-generic #127-Ubuntu SMP Fri Jul 5 20:13:28 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / b5833bb
Default Java Private Build-1.8.0_422-8u422-b05-1~20.04-b05
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_422-8u422-b05-1~20.04-b05
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7089/2/testReport/
Max. process+thread count 51 (vs. ulimit of 5500)
modules C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7089/2/console
versions git=2.25.1 maven=3.6.3
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@steveloughran
Copy link
Contributor Author

testing s3 london; -Dparallel-tests -DtestsThreadCount=8 -Dscale

the unrelated failure from an assert we should cut

[ERROR] testReadLargeFileFully(org.apache.hadoop.fs.s3a.ITestS3APrefetchingInputStream)  Time elapsed: 2.182 s  <<< FAILURE!
java.lang.AssertionError: 
[Maxiumum named action_executor_acquired.max] 
Expecting:
 <0L>
to be greater than:
 <0L> 
        at org.apache.hadoop.fs.s3a.ITestS3APrefetchingInputStream.testReadLargeFileFully(ITestS3APrefetchingInputStream.java:136)

@@ -1295,6 +1302,7 @@ protected RequestFactory createRequestFactory() {
.withContentEncoding(contentEncoding)
.withStorageClass(storageClass)
.withMultipartUploadEnabled(isMultipartUploadEnabled)
.withPartUploadTimeout(partUploadTimeout)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How is this part upload timeout different that multipart upload timeout?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

its my new option, same value for simple PUT as multipart; we patch the individual requests

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

anyway

  1. will cut
  2. will modify info log to only print once per stream, to keep that log noise down.

* This will be set on data put/post operations only.
* A zero value means "no custom timeout"
*/
private Duration partUploadTimeout = Duration.ZERO;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I believe you have tested this explicitly. Hopefully zero doesn't mean infinite.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, I will set to the default...even though its set up properly in production, there are some test cases which didn't

LOG.info("Stream created more than once: {}", this);
LOG.info("Stream recreated: {}", this);
if (LOG.isDebugEnabled()) {
LOG.debug("Stream creation stack", new Exception("here"));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this is for some testing,

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

good q. should we delete it? it's actually pretty handy for troubleshooting networking issues

* default timeout set in builder
* tune logging of content provider on recovery
* new tests to verify timeout propagation
* discovered a new wrapping of failures in read(),
  so relaxed intercept exception class more

Change-Id: I43e2822e4dbd684d2c0469650b07369b731a2e7c
@steveloughran steveloughran force-pushed the s3/HADOOP-19295-request-timeout branch from 7298e92 to 04a18d3 Compare October 4, 2024 15:55
@steveloughran
Copy link
Contributor Author

found that sometimes the timeout is wrapped in an UncheckedIOException; so relaxed the exception intercepted.

java.io.UncheckedIOException: software.amazon.awssdk.thirdparty.org.apache.http.ConnectionClosedException: Premature end of Content-Length delimited message body (expected: 1,024; received: 0)
        at software.amazon.awssdk.utils.FunctionalUtils.asRuntimeException(FunctionalUtils.java:180)
        at software.amazon.awssdk.utils.FunctionalUtils.lambda$safeRunnable$5(FunctionalUtils.java:126)
        at software.amazon.awssdk.utils.FunctionalUtils.invokeSafely(FunctionalUtils.java:140)
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.lambda$handleInterruptedException$0(ApiCallAttemptTimeoutTrackingStage.java:132)
        at java.util.Optional.ifPresent(Optional.java:159)
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.handleInterruptedException(ApiCallAttemptTimeoutTrackingStage.java:132)
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.translatePipelineException(ApiCallAttemptTimeoutTrackingStage.java:105)
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:89)
        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$0(BaseSyncClientHandler.java:66)
        at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:182)
        at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:60)
        at software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:52)
        at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:60)
        at software.amazon.awssdk.services.s3.DefaultS3Client.getObject(DefaultS3Client.java:5203)
        at software.amazon.awssdk.services.s3.S3Client.getObject(S3Client.java:9063)
        at org.apache.hadoop.fs.s3a.S3AFileSystem$InputStreamCallbacksImpl.getObject(S3AFileSystem.java:1948)
        at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$reopen$0(S3AInputStream.java:310)
        at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration(IOStatisticsBinding.java:547)
        at org.apache.hadoop.fs.s3a.Invoker.onceTrackingDuration(Invoker.java:147)
        at org.apache.hadoop.fs.s3a.S3AInputStream.reopen(S3AInputStream.java:308)
        at org.apache.hadoop.fs.s3a.S3AInputStream.lambda$lazySeek$1(S3AInputStream.java:459)
        at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$2(Invoker.java:257)
        at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:122)
        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:255)
        at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:304)
        at org.apache.hadoop.fs.s3a.S3AInputStream.lazySeek(S3AInputStream.java:452)
        at org.apache.hadoop.fs.s3a.S3AInputStream.read(S3AInputStream.java:581)
        at org.apache.hadoop.fs.s3a.S3AInputStream.readFully(S3AInputStream.java:868)
        at org.apache.hadoop.fs.FSDataInputStream.readFully(FSDataInputStream.java:128)
        at org.apache.hadoop.fs.contract.ContractTestUtils.readUTF8(ContractTestUtils.java:830)
        at org.apache.hadoop.fs.s3a.impl.ITestConnectionTimeouts.lambda$testObjectUploadTimeouts$3(ITestConnectionTimeouts.java:252)
        at org.apache.hadoop.test.LambdaTestUtils.intercept(LambdaTestUtils.java:500)
        at org.apache.hadoop.test.LambdaTestUtils.intercept(LambdaTestUtils.java:386)
        at org.apache.hadoop.fs.s3a.impl.ITestConnectionTimeouts.testObjectUploadTimeouts(ITestConnectionTimeouts.java:251)
        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.thirdparty.org.apache.http.ConnectionClosedException: Premature end of Content-Length delimited message body (expected: 1,024; received: 0)
        at software.amazon.awssdk.thirdparty.org.apache.http.impl.io.ContentLengthInputStream.read(ContentLengthInputStream.java:178)
        at software.amazon.awssdk.thirdparty.org.apache.http.impl.io.ContentLengthInputStream.read(ContentLengthInputStream.java:198)
        at software.amazon.awssdk.thirdparty.org.apache.http.impl.io.ContentLengthInputStream.close(ContentLengthInputStream.java:101)
        at software.amazon.awssdk.thirdparty.org.apache.http.impl.execchain.ResponseEntityProxy.streamClosed(ResponseEntityProxy.java:142)
        at software.amazon.awssdk.thirdparty.org.apache.http.conn.EofSensorInputStream.checkClose(EofSensorInputStream.java:228)
        at software.amazon.awssdk.thirdparty.org.apache.http.conn.EofSensorInputStream.close(EofSensorInputStream.java:172)
        at java.io.FilterInputStream.close(FilterInputStream.java:181)
        at software.amazon.awssdk.utils.FunctionalUtils.lambda$safeRunnable$5(FunctionalUtils.java:124)
        ... 72 more

@hadoop-yetus
Copy link

🎊 +1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 22s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 0s No case conflicting files found.
+0 🆗 codespell 0m 1s codespell was not available.
+0 🆗 detsecrets 0m 1s detect-secrets was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 8 new or modified test files.
_ trunk Compile Tests _
+1 💚 mvninstall 33m 48s trunk passed
+1 💚 compile 0m 25s trunk passed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04
+1 💚 compile 0m 19s trunk passed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05
+1 💚 checkstyle 0m 19s trunk passed
+1 💚 mvnsite 0m 26s trunk passed
+1 💚 javadoc 0m 25s trunk passed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04
+1 💚 javadoc 0m 20s trunk passed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05
+1 💚 spotbugs 0m 42s trunk passed
+1 💚 shadedclient 24m 34s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+1 💚 mvninstall 0m 17s the patch passed
+1 💚 compile 0m 20s the patch passed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04
+1 💚 javac 0m 20s the patch passed
+1 💚 compile 0m 15s the patch passed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05
+1 💚 javac 0m 15s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
+1 💚 checkstyle 0m 11s the patch passed
+1 💚 mvnsite 0m 19s the patch passed
+1 💚 javadoc 0m 17s the patch passed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04
+1 💚 javadoc 0m 15s the patch passed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05
+1 💚 spotbugs 0m 39s the patch passed
+1 💚 shadedclient 22m 20s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 1m 50s hadoop-aws in the patch passed.
+1 💚 asflicense 0m 23s The patch does not generate ASF License warnings.
89m 33s
Subsystem Report/Notes
Docker ClientAPI=1.47 ServerAPI=1.47 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7089/3/artifact/out/Dockerfile
GITHUB PR #7089
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets
uname Linux c000e4583f82 5.15.0-117-generic #127-Ubuntu SMP Fri Jul 5 20:13:28 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 7298e92
Default Java Private Build-1.8.0_422-8u422-b05-1~20.04-b05
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_422-8u422-b05-1~20.04-b05
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7089/3/testReport/
Max. process+thread count 551 (vs. ulimit of 5500)
modules C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7089/3/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@hadoop-yetus
Copy link

🎊 +1 overall

Vote Subsystem Runtime Logfile Comment
+0 🆗 reexec 0m 20s Docker mode activated.
_ Prechecks _
+1 💚 dupname 0m 1s No case conflicting files found.
+0 🆗 codespell 0m 0s codespell was not available.
+0 🆗 detsecrets 0m 0s detect-secrets was not available.
+1 💚 @author 0m 0s The patch does not contain any @author tags.
+1 💚 test4tests 0m 0s The patch appears to include 8 new or modified test files.
_ trunk Compile Tests _
+1 💚 mvninstall 36m 36s trunk passed
+1 💚 compile 0m 25s trunk passed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04
+1 💚 compile 0m 21s trunk passed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05
+1 💚 checkstyle 0m 19s trunk passed
+1 💚 mvnsite 0m 26s trunk passed
+1 💚 javadoc 0m 25s trunk passed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04
+1 💚 javadoc 0m 20s trunk passed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05
+1 💚 spotbugs 0m 44s trunk passed
+1 💚 shadedclient 23m 8s branch has no errors when building and testing our client artifacts.
_ Patch Compile Tests _
+1 💚 mvninstall 0m 17s the patch passed
+1 💚 compile 0m 20s the patch passed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04
+1 💚 javac 0m 20s the patch passed
+1 💚 compile 0m 16s the patch passed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05
+1 💚 javac 0m 16s the patch passed
+1 💚 blanks 0m 0s The patch has no blanks issues.
+1 💚 checkstyle 0m 12s the patch passed
+1 💚 mvnsite 0m 18s the patch passed
+1 💚 javadoc 0m 16s the patch passed with JDK Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04
+1 💚 javadoc 0m 16s the patch passed with JDK Private Build-1.8.0_422-8u422-b05-1~20.04-b05
+1 💚 spotbugs 0m 40s the patch passed
+1 💚 shadedclient 22m 1s patch has no errors when building and testing our client artifacts.
_ Other Tests _
+1 💚 unit 2m 6s hadoop-aws in the patch passed.
+1 💚 asflicense 0m 27s The patch does not generate ASF License warnings.
90m 58s
Subsystem Report/Notes
Docker ClientAPI=1.47 ServerAPI=1.47 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7089/4/artifact/out/Dockerfile
GITHUB PR #7089
Optional Tests dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell detsecrets
uname Linux 5a34083ac19e 5.15.0-117-generic #127-Ubuntu SMP Fri Jul 5 20:13:28 UTC 2024 x86_64 x86_64 x86_64 GNU/Linux
Build tool maven
Personality dev-support/bin/hadoop.sh
git revision trunk / 04a18d3
Default Java Private Build-1.8.0_422-8u422-b05-1~20.04-b05
Multi-JDK versions /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.24+8-post-Ubuntu-1ubuntu320.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_422-8u422-b05-1~20.04-b05
Test Results https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7089/4/testReport/
Max. process+thread count 555 (vs. ulimit of 5500)
modules C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws
Console output https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-7089/4/console
versions git=2.25.1 maven=3.6.3 spotbugs=4.2.2
Powered by Apache Yetus 0.14.0 https://yetus.apache.org

This message was automatically generated.

@mukund-thakur mukund-thakur changed the title HADOOP-19295. S3A: fs.s3a.connection.request.timeout too low HADOOP-19295. S3A: fs.s3a.connection.request.timeout too low for large file uploads Oct 7, 2024
Copy link
Contributor

@mukund-thakur mukund-thakur left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 LGTM other than one question below
15 mins seems too much to me. Any specific reason for selecting this high value as opposed to 5 mins ?

@steveloughran
Copy link
Contributor Author

15 mins seems too much to me. Any specific reason for selecting this high value as opposed to 5 mins ?

no. just something to cope with a slow block upload with maybe a transient network error. we still have that retry count limit so repeated network errors will fail in less than 15 min.

@steveloughran steveloughran merged commit dc56fc3 into apache:trunk Oct 7, 2024
4 checks passed
@steveloughran steveloughran changed the title HADOOP-19295. S3A: fs.s3a.connection.request.timeout too low for large file uploads HADOOP-19295. S3A: large uploads can timeout over slow links Oct 7, 2024
steveloughran added a commit to steveloughran/hadoop that referenced this pull request Oct 7, 2024
…7089)


This sets a different timeout for data upload PUT/POST calls to all
other requests, so that slow block uploads do not trigger timeouts
as rapidly as normal requests. This was always the behavior
in the V1 AWS SDK; for V2 we have to explicitly set it on the operations
we want to give extended timeouts. 

Option:  fs.s3a.connection.part.upload.timeout
Default: 15m

Contributed by Steve Loughran
steveloughran added a commit that referenced this pull request Oct 8, 2024
…7100)


This sets a different timeout for data upload PUT/POST calls to all
other requests, so that slow block uploads do not trigger timeouts
as rapidly as normal requests. This was always the behavior
in the V1 AWS SDK; for V2 we have to explicitly set it on the operations
we want to give extended timeouts. 

Option:  fs.s3a.connection.part.upload.timeout
Default: 15m

Contributed by Steve Loughran
asfgit pushed a commit that referenced this pull request Oct 9, 2024
…7100)

This sets a different timeout for data upload PUT/POST calls to all
other requests, so that slow block uploads do not trigger timeouts
as rapidly as normal requests. This was always the behavior
in the V1 AWS SDK; for V2 we have to explicitly set it on the operations
we want to give extended timeouts.

Option:  fs.s3a.connection.part.upload.timeout
Default: 15m

Contributed by Steve Loughran
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants