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

Steve Loughran edited comment on HADOOP-19091 at 12/6/24 3:02 PM:
------------------------------------------------------------------


I've been working on this again, and I'm down to a couple of ITest failures 
that I'd like second opinions on. The second error I've discussed with Steve 
and I don't think is particularly significant. Anybody wit

 

Error 1:

{code}

[ERROR] 
testObjectUploadTimeouts(org.apache.hadoop.fs.s3a.impl.ITestConnectionTimeouts) 
Time elapsed: 20.317 s <<< FAILURE!}
java.lang.AssertionError: }
[Duration of write] }
Expecting:}
 <PT17.068S>}
to be less than:}
 <PT10S> }
 at 
org.apache.hadoop.fs.s3a.impl.ITestConnectionTimeouts.testObjectUploadTimeouts(ITestConnectionTimeouts.java:242)}
 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)}
{code}

Based on what I see of the code, I don't believe this has anything to do with 
my changes to the magic committer. It might be something to do with my 
network/changes on S3's side.
 
Another error is:
{code}

[ERROR] 
testAutomaticProxyPortSelection(org.apache.hadoop.fs.s3a.ITestS3AConfiguration) 
Time elapsed: 0.035 s <<< ERROR!}
java.io.FileNotFoundException: listObjects() on s3a://test-aws-s3a-vnarayanan/: 
software.amazon.awssdk.services.s3.model.S3Exception: null (Service: S3, Status 
Code: 404, Request ID: null):null}
 at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:283)}
 at org.apache.hadoop.fs.s3a.Invoker.onceInTheFuture(Invoker.java:190)}
 at 
org.apache.hadoop.fs.s3a.Listing$ObjectListingIterator.next(Listing.java:653)}
 at 
org.apache.hadoop.fs.s3a.Listing$FileStatusListingIterator.requestNextBatch(Listing.java:429)}
 at 
org.apache.hadoop.fs.s3a.Listing$FileStatusListingIterator.<init>(Listing.java:371)}
 at 
org.apache.hadoop.fs.s3a.Listing.createFileStatusListingIterator(Listing.java:142)}
 at org.apache.hadoop.fs.s3a.Listing.getListFilesAssumingDir(Listing.java:210)}
 at 
org.apache.hadoop.fs.s3a.S3AFileSystem.innerListFiles(S3AFileSystem.java:5279)}
 at 
org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$listFiles$36(S3AFileSystem.java:5221)}
 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.trackDuration(IOStatisticsBinding.java:449)}
 at 
org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2905)}
 at 
org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2924)}
 at org.apache.hadoop.fs.s3a.S3AFileSystem.listFiles(S3AFileSystem.java:5220)}
 at 
org.apache.hadoop.fs.s3a.ITestS3AConfiguration.lambda$expectFSCreateFailure$0(ITestS3AConfiguration.java:194)}
 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.ITestS3AConfiguration.expectFSCreateFailure(ITestS3AConfiguration.java:191)}
 at 
org.apache.hadoop.fs.s3a.ITestS3AConfiguration.testAutomaticProxyPortSelection(ITestS3AConfiguration.java:220)}
 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.rules.ExternalResource$1.evaluate(ExternalResource.java:54)}
 at 
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:299)}
 at 
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:293)}
 at java.util.concurrent.FutureTask.run(FutureTask.java:266)}
 at java.lang.Thread.run(Thread.java:750)}
Caused by: software.amazon.awssdk.services.s3.model.S3Exception: null (Service: 
S3, Status Code: 404, Request ID: null)}
 at 
software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handleErrorResponse(AwsXmlPredicatedResponseHandler.java:156)}
 at 
software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handleResponse(AwsXmlPredicatedResponseHandler.java:108)}
 at 
software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handle(AwsXmlPredicatedResponseHandler.java:85)}
 at 
software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handle(AwsXmlPredicatedResponseHandler.java:43)}
 at 
software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler$Crc32ValidationResponseHandler.handle(AwsSyncClientHandler.java:93)}
 at 
software.amazon.awssdk.core.internal.handler.BaseClientHandler.lambda$successTransformationResponseHandler$7(BaseClientHandler.java:279)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:50)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:38)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:72)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:55)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:39)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:81)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:36)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)}
 at 
software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:56)}
 at 
software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:36)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.executeWithTimer(ApiCallTimeoutTrackingStage.java:80)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:60)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:42)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:50)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:32)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:37)}
 at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:26)}
 at 
software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient$RequestExecutionBuilderImpl.execute(AmazonSyncHttpClient.java:224)}
 at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.invoke(BaseSyncClientHandler.java:103)}
 at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.doExecute(BaseSyncClientHandler.java:173)}
 at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$1(BaseSyncClientHandler.java:80)}
 at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:182)}
 at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:74)}
 at 
software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:45)}
 at 
software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:53)}
 at 
software.amazon.awssdk.services.s3.DefaultS3Client.listObjectsV2(DefaultS3Client.java:7356)}
 at 
software.amazon.awssdk.services.s3.DelegatingS3Client.lambda$listObjectsV2$63(DelegatingS3Client.java:5885)}
 at 
software.amazon.awssdk.services.s3.internal.crossregion.S3CrossRegionSyncClient.invokeOperation(S3CrossRegionSyncClient.java:67)}
 at 
software.amazon.awssdk.services.s3.DelegatingS3Client.listObjectsV2(DelegatingS3Client.java:5885)}
 at 
org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$listObjects$12(S3AFileSystem.java:3128)}
 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.s3a.Invoker.retryUntranslated(Invoker.java:468)}
 at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:431)}
 at org.apache.hadoop.fs.s3a.S3AFileSystem.listObjects(S3AFileSystem.java:3119)}
 at 
org.apache.hadoop.fs.s3a.S3AFileSystem$ListingOperationCallbacksImpl.lambda$listObjectsAsync$0(S3AFileSystem.java:2759)}
 at 
org.apache.hadoop.fs.s3a.impl.CallableSupplier.get(CallableSupplier.java:88)}
 at 
java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1604)}
 at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)}
 at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)}
 ... 1 more}
{code}

This second error is likely to be S3 changing its behavior (again, it has 
nothing to do with the magic committer), because this failure happens past the 
point where the expected Exception would have been thrown.


was (Author: vnarayanan7):
I've been working on this again, and I'm down to a couple of ITest failures 
that I'd like second opinions on. The second error I've discussed with Steve 
and I don't think is particularly significant. Anybody with more experience 
with hadoop-aws weighing in on the first in particular would be appreciated, so 
I can more confidently put the PR up. I've had to tweak some tests to use an 
alternate CredentialsProvider to actually run the tests with my account, which 
I'm happy to put up as a separate PR.

 

Error 1:

 
{{[ERROR] 
testObjectUploadTimeouts(org.apache.hadoop.fs.s3a.impl.ITestConnectionTimeouts) 
Time elapsed: 20.317 s <<< FAILURE!}}
{{java.lang.AssertionError: }}
{{[Duration of write] }}
{{Expecting:}}
{{ <PT17.068S>}}
{{to be less than:}}
{{ <PT10S> }}
{{ at 
org.apache.hadoop.fs.s3a.impl.ITestConnectionTimeouts.testObjectUploadTimeouts(ITestConnectionTimeouts.java:242)}}
{{ 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)}}
 
Based on what I see of the code, I don't believe this has anything to do with 
my changes to the magic committer. It might be something to do with my 
network/changes on S3's side.
 
Another error is:
 
{{[ERROR] 
testAutomaticProxyPortSelection(org.apache.hadoop.fs.s3a.ITestS3AConfiguration) 
Time elapsed: 0.035 s <<< ERROR!}}
{{java.io.FileNotFoundException: listObjects() on 
s3a://test-aws-s3a-vnarayanan/: 
software.amazon.awssdk.services.s3.model.S3Exception: null (Service: S3, Status 
Code: 404, Request ID: null):null}}
{{ at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:283)}}
{{ at org.apache.hadoop.fs.s3a.Invoker.onceInTheFuture(Invoker.java:190)}}
{{ at 
org.apache.hadoop.fs.s3a.Listing$ObjectListingIterator.next(Listing.java:653)}}
{{ at 
org.apache.hadoop.fs.s3a.Listing$FileStatusListingIterator.requestNextBatch(Listing.java:429)}}
{{ at 
org.apache.hadoop.fs.s3a.Listing$FileStatusListingIterator.<init>(Listing.java:371)}}
{{ at 
org.apache.hadoop.fs.s3a.Listing.createFileStatusListingIterator(Listing.java:142)}}
{{ at 
org.apache.hadoop.fs.s3a.Listing.getListFilesAssumingDir(Listing.java:210)}}
{{ at 
org.apache.hadoop.fs.s3a.S3AFileSystem.innerListFiles(S3AFileSystem.java:5279)}}
{{ at 
org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$listFiles$36(S3AFileSystem.java:5221)}}
{{ 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.trackDuration(IOStatisticsBinding.java:449)}}
{{ at 
org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2905)}}
{{ at 
org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2924)}}
{{ at 
org.apache.hadoop.fs.s3a.S3AFileSystem.listFiles(S3AFileSystem.java:5220)}}
{{ at 
org.apache.hadoop.fs.s3a.ITestS3AConfiguration.lambda$expectFSCreateFailure$0(ITestS3AConfiguration.java:194)}}
{{ 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.ITestS3AConfiguration.expectFSCreateFailure(ITestS3AConfiguration.java:191)}}
{{ at 
org.apache.hadoop.fs.s3a.ITestS3AConfiguration.testAutomaticProxyPortSelection(ITestS3AConfiguration.java:220)}}
{{ 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.rules.ExternalResource$1.evaluate(ExternalResource.java:54)}}
{{ at 
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:299)}}
{{ at 
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:293)}}
{{ at java.util.concurrent.FutureTask.run(FutureTask.java:266)}}
{{ at java.lang.Thread.run(Thread.java:750)}}
{{Caused by: software.amazon.awssdk.services.s3.model.S3Exception: null 
(Service: S3, Status Code: 404, Request ID: null)}}
{{ at 
software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handleErrorResponse(AwsXmlPredicatedResponseHandler.java:156)}}
{{ at 
software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handleResponse(AwsXmlPredicatedResponseHandler.java:108)}}
{{ at 
software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handle(AwsXmlPredicatedResponseHandler.java:85)}}
{{ at 
software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handle(AwsXmlPredicatedResponseHandler.java:43)}}
{{ at 
software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler$Crc32ValidationResponseHandler.handle(AwsSyncClientHandler.java:93)}}
{{ at 
software.amazon.awssdk.core.internal.handler.BaseClientHandler.lambda$successTransformationResponseHandler$7(BaseClientHandler.java:279)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:50)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:38)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:72)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:55)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:39)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:81)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:36)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)}}
{{ at 
software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:56)}}
{{ at 
software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:36)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.executeWithTimer(ApiCallTimeoutTrackingStage.java:80)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:60)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:42)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:50)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:32)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:37)}}
{{ at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:26)}}
{{ at 
software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient$RequestExecutionBuilderImpl.execute(AmazonSyncHttpClient.java:224)}}
{{ at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.invoke(BaseSyncClientHandler.java:103)}}
{{ at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.doExecute(BaseSyncClientHandler.java:173)}}
{{ at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$1(BaseSyncClientHandler.java:80)}}
{{ at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:182)}}
{{ at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:74)}}
{{ at 
software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:45)}}
{{ at 
software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:53)}}
{{ at 
software.amazon.awssdk.services.s3.DefaultS3Client.listObjectsV2(DefaultS3Client.java:7356)}}
{{ at 
software.amazon.awssdk.services.s3.DelegatingS3Client.lambda$listObjectsV2$63(DelegatingS3Client.java:5885)}}
{{ at 
software.amazon.awssdk.services.s3.internal.crossregion.S3CrossRegionSyncClient.invokeOperation(S3CrossRegionSyncClient.java:67)}}
{{ at 
software.amazon.awssdk.services.s3.DelegatingS3Client.listObjectsV2(DelegatingS3Client.java:5885)}}
{{ at 
org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$listObjects$12(S3AFileSystem.java:3128)}}
{{ 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.s3a.Invoker.retryUntranslated(Invoker.java:468)}}
{{ at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:431)}}
{{ at 
org.apache.hadoop.fs.s3a.S3AFileSystem.listObjects(S3AFileSystem.java:3119)}}
{{ at 
org.apache.hadoop.fs.s3a.S3AFileSystem$ListingOperationCallbacksImpl.lambda$listObjectsAsync$0(S3AFileSystem.java:2759)}}
{{ at 
org.apache.hadoop.fs.s3a.impl.CallableSupplier.get(CallableSupplier.java:88)}}
{{ at 
java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1604)}}
{{ at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)}}
{{ at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)}}
{{ ... 1 more}}
 
This second error is likely to be S3 changing its behavior (again, it has 
nothing to do with the magic committer), because this failure happens past the 
point where the expected Exception would have been thrown.

> Add support for Tez to MagicS3GuardCommitter
> --------------------------------------------
>
>                 Key: HADOOP-19091
>                 URL: https://issues.apache.org/jira/browse/HADOOP-19091
>             Project: Hadoop Common
>          Issue Type: Improvement
>          Components: fs/s3
>    Affects Versions: 3.3.6
>         Environment: Pig 17/Hive 3.1.3 with Hadoop 3.3.3 on AWS EMR 6-12.0
>            Reporter: Venkatasubrahmanian Narayanan
>            Assignee: Venkatasubrahmanian Narayanan
>            Priority: Major
>         Attachments: 0001-AWS-Hive-Changes.patch, 
> 0002-HIVE-27698-Backport-of-HIVE-22398-Remove-legacy-code.patch, 
> HADOOP-19091-HIVE-WIP.patch
>
>
> The MagicS3GuardCommitter assumes that the JobID of the task is the same as 
> that of the job's application master when writing/reading the .pendingset 
> file. This assumption is not valid when running with Tez, which creates 
> slightly different JobIDs for tasks and the application master.
>  
> While the MagicS3GuardCommitter is intended only for MRv2, it mostly works 
> fine with an MRv1 wrapper with Hive/Pig (with some minor changes to Hive) run 
> in MR mode. This issue only crops up when running queries with the Tez 
> execution engine. I can upload a patch to Hive 3.1 to reproduce this error on 
> EMR if needed.
>  
> Fixing this will probably require work from both Tez and Hadoop, wanted to 
> start a discussion here so we can figure out how exactly we go about this.



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