[ https://issues.apache.org/jira/browse/HADOOP-19479?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17951211#comment-17951211 ]
Sarunas Valaskevicius edited comment on HADOOP-19479 at 5/13/25 3:14 PM: ------------------------------------------------------------------------- if you're talking about per request retries vs per connection retry - would new requests, however small the retry settings, not incur the same error once connection timeout kicks in? context - in our application we expect new rows to be added to the parquet output at all times and very frequently - essentially it's a never ending flow of data to rotating parquet files it is ok to fail to export to s3 if network fails, but we mustn't stop the host service as it's primary purpose is other than just exporting this data was (Author: rakatan): if you're talking about per request retries vs per connection retry - would new requests, however small the retry settings, not incur the same error once connection timeout kicks in? context - in our application we expect new rows to be added to the parquet output at all times and very frequently - essentially it's a never ending flow of data to rotating parquet files > S3A: Deadlock in multipart upload > --------------------------------- > > Key: HADOOP-19479 > URL: https://issues.apache.org/jira/browse/HADOOP-19479 > Project: Hadoop Common > Issue Type: Bug > Components: fs/s3 > Affects Versions: 3.4.1 > Reporter: Sarunas Valaskevicius > Priority: Major > > Reproduced while testing system resilience and turning S3 network off > (introduced a network partition to the list of IP addresses S3 uses) - but > given it's seemingly timers related stack traces, I'd guess it could happen > any time? > {code:java} > Found one Java-level deadlock: > ============================= > "sdk-ScheduledExecutor-2-3": > waiting to lock monitor 0x00007f5c880a8630 (object 0x0000000315523c78, a > java.lang.Object), > which is held by "sdk-ScheduledExecutor-2-4" > "sdk-ScheduledExecutor-2-4": > waiting to lock monitor 0x00007f5c7c016700 (object 0x0000000327800000, a > org.apache.hadoop.fs.s3a.S3ABlockOutputStream), > which is held by "io-compute-blocker-15" > "io-compute-blocker-15": > waiting to lock monitor 0x00007f5c642ae900 (object 0x00000003af0001d8, a > java.lang.Object), > which is held by "sdk-ScheduledExecutor-2-3" > Java stack information for the threads listed above: > =================================================== > "sdk-ScheduledExecutor-2-3": > at java.lang.Thread.interrupt(java.base@21/Thread.java:1717) > - waiting to lock <0x0000000315523c78> (a java.lang.Object) > at > software.amazon.awssdk.core.internal.http.timers.SyncTimeoutTask.run(SyncTimeoutTask.java:60) > - locked <0x00000003af0001d8> (a java.lang.Object) > at > java.util.concurrent.Executors$RunnableAdapter.call(java.base@21/Executors.java:572) > at > java.util.concurrent.FutureTask.run(java.base@21/FutureTask.java:317) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(java.base@21/ScheduledThreadPoolExecutor.java:304) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@21/ThreadPoolExecutor.java:1144) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@21/ThreadPoolExecutor.java:642) > at java.lang.Thread.runWith(java.base@21/Thread.java:1596) > at java.lang.Thread.run(java.base@21/Thread.java:1583) > "sdk-ScheduledExecutor-2-4": > at > org.apache.hadoop.fs.s3a.S3ABlockOutputStream.getActiveBlock(S3ABlockOutputStream.java:304) > - waiting to lock <0x0000000327800000> (a > org.apache.hadoop.fs.s3a.S3ABlockOutputStream) > at > org.apache.hadoop.fs.s3a.S3ABlockOutputStream.close(S3ABlockOutputStream.java:485) > at > org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:77) > at > org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106) > at > org.apache.parquet.hadoop.util.HadoopPositionOutputStream.close(HadoopPositionOutputStream.java:66) > at > java.nio.channels.Channels$WritableByteChannelImpl.implCloseChannel(java.base@21/Channels.java:404) > at > java.nio.channels.spi.AbstractInterruptibleChannel$1.interrupt(java.base@21/AbstractInterruptibleChannel.java:163) > - locked <0x00000003af0002a0> (a java.lang.Object) > at java.lang.Thread.interrupt(java.base@21/Thread.java:1722) > - locked <0x0000000315523c78> (a java.lang.Object) > at > software.amazon.awssdk.core.internal.http.timers.SyncTimeoutTask.run(SyncTimeoutTask.java:60) > - locked <0x00000003af0002e0> (a java.lang.Object) > at > java.util.concurrent.Executors$RunnableAdapter.call(java.base@21/Executors.java:572) > at > java.util.concurrent.FutureTask.run(java.base@21/FutureTask.java:317) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(java.base@21/ScheduledThreadPoolExecutor.java:304) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@21/ThreadPoolExecutor.java:1144) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@21/ThreadPoolExecutor.java:642) > at java.lang.Thread.runWith(java.base@21/Thread.java:1596) > at java.lang.Thread.run(java.base@21/Thread.java:1583) > "io-compute-blocker-15": > at > software.amazon.awssdk.core.internal.http.timers.SyncTimeoutTask.cancel(SyncTimeoutTask.java:74) > - waiting to lock <0x00000003af0001d8> (a java.lang.Object) > at > software.amazon.awssdk.core.internal.http.timers.ApiCallTimeoutTracker.cancel(ApiCallTimeoutTracker.java:53) > at > software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:77) > 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$$Lambda/0x00007f5d2cb20ca8.get(Unknown > Source) > 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.createMultipartUpload(DefaultS3Client.java:1463) > at > software.amazon.awssdk.services.s3.DelegatingS3Client.lambda$createMultipartUpload$4(DelegatingS3Client.java:1232) > at > software.amazon.awssdk.services.s3.DelegatingS3Client$$Lambda/0x00007f5d2d316118.apply(Unknown > Source) > at > software.amazon.awssdk.services.s3.internal.crossregion.S3CrossRegionSyncClient.invokeOperation(S3CrossRegionSyncClient.java:67) > at > software.amazon.awssdk.services.s3.DelegatingS3Client.createMultipartUpload(DelegatingS3Client.java:1232) > at > org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$initiateMultipartUpload$30(S3AFileSystem.java:4705) > at > org.apache.hadoop.fs.s3a.S3AFileSystem$$Lambda/0x00007f5d2d315ef8.get(Unknown > Source) > at > org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:651) > at > org.apache.hadoop.fs.s3a.S3AFileSystem.initiateMultipartUpload(S3AFileSystem.java:4703) > at > org.apache.hadoop.fs.s3a.WriteOperationHelper.lambda$initiateMultiPartUpload$0(WriteOperationHelper.java:283) > at > org.apache.hadoop.fs.s3a.WriteOperationHelper$$Lambda/0x00007f5d2d30e230.apply(Unknown > Source) > 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$$Lambda/0x00007f5d2d2dd6a0.apply(Unknown > Source) > 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.initiateMultiPartUpload(WriteOperationHelper.java:278) > at > org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.lambda$new$0(S3ABlockOutputStream.java:904) > at > org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload$$Lambda/0x00007f5d2d30e000.apply(Unknown > Source) > at > org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration(IOStatisticsBinding.java:547) > at > org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.lambda$trackDurationOfOperation$5(IOStatisticsBinding.java:528) > at > org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding$$Lambda/0x00007f5d2ca3c918.apply(Unknown > Source) > at > org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration(IOStatisticsBinding.java:449) > at > org.apache.hadoop.fs.s3a.S3ABlockOutputStream$MultiPartUpload.<init>(S3ABlockOutputStream.java:902) > at > org.apache.hadoop.fs.s3a.S3ABlockOutputStream.initMultipartUpload(S3ABlockOutputStream.java:462) > at > org.apache.hadoop.fs.s3a.S3ABlockOutputStream.uploadCurrentBlock(S3ABlockOutputStream.java:439) > - locked <0x0000000327800000> (a > org.apache.hadoop.fs.s3a.S3ABlockOutputStream) > at > org.apache.hadoop.fs.s3a.S3ABlockOutputStream.write(S3ABlockOutputStream.java:413) > - locked <0x0000000327800000> (a > org.apache.hadoop.fs.s3a.S3ABlockOutputStream) > at > org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:62) > at > java.io.DataOutputStream.write(java.base@21/DataOutputStream.java:115) > - locked <0x0000000327800208> (a > org.apache.hadoop.fs.FSDataOutputStream) > at > org.apache.parquet.hadoop.util.HadoopPositionOutputStream.write(HadoopPositionOutputStream.java:50) > at > java.nio.channels.Channels$WritableByteChannelImpl.write(java.base@21/Channels.java:392) > - locked <0x00000003afab3da8> (a java.lang.Object) > at > org.apache.parquet.bytes.ConcatenatingByteBufferCollector.writeAllTo(ConcatenatingByteBufferCollector.java:77) > at > org.apache.parquet.hadoop.ParquetFileWriter.writeColumnChunk(ParquetFileWriter.java:1338) > at > org.apache.parquet.hadoop.ParquetFileWriter.writeColumnChunk(ParquetFileWriter.java:1259) > at > org.apache.parquet.hadoop.ColumnChunkPageWriteStore$ColumnChunkPageWriter.writeToFileWriter(ColumnChunkPageWriteStore.java:408) > at > org.apache.parquet.hadoop.ColumnChunkPageWriteStore.flushToFileWriter(ColumnChunkPageWriteStore.java:675) > at > org.apache.parquet.hadoop.InternalParquetRecordWriter.flushRowGroupToStore(InternalParquetRecordWriter.java:210) > at > org.apache.parquet.hadoop.InternalParquetRecordWriter.checkBlockSizeReached(InternalParquetRecordWriter.java:178) > at > org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:154) > at > org.apache.parquet.hadoop.ParquetWriter.write(ParquetWriter.java:428) > Found 1 deadlock. > {code} -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org