[ https://issues.apache.org/jira/browse/HADOOP-18695?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17711424#comment-17711424 ]
Steve Loughran commented on HADOOP-18695: ----------------------------------------- auditor will raise access denied exception, ``` [ERROR] Tests run: 10, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 21.882 s <<< FAILURE! - in org.apache.hadoop.fs.s3a.scale.ITestS3AHugeFilesSinglePut [ERROR] test_100_renameHugeFile(org.apache.hadoop.fs.s3a.scale.ITestS3AHugeFilesSinglePut) Time elapsed: 1.363 s <<< ERROR! java.nio.file.AccessDeniedException: tests3ascale/disk/hugefile: org.apache.hadoop.fs.s3a.audit.AuditFailureException: Multipart IO request com.amazonaws.services.s3.model.EncryptedInitiateMultipartUploadRequest@6781cfbe rejected https://audit.example.org/hadoop/1/op_rename/458f1c6e-aa1f-46da-a46b-9c03c30d2c7f-00000049/?op=op_rename&p1=s3a://stevel-london/tests3ascale/disk/hugefile&pr=stevel&p2=s3a://stevel-london/tests3ascale/disk/hugefileRenamed&ps=7d5e3e98-6b66-49de-a7d2-e6b9281fa410&id=458f1c6e-aa1f-46da-a46b-9c03c30d2c7f-00000049&t0=94&fs=458f1c6e-aa1f-46da-a46b-9c03c30d2c7f&t1=102&ts=1681306681816 at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:211) at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:124) at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$4(Invoker.java:376) at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:468) at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:372) at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:347) at org.apache.hadoop.fs.s3a.S3AFileSystem.copyFile(S3AFileSystem.java:4254) at org.apache.hadoop.fs.s3a.S3AFileSystem.access$1300(S3AFileSystem.java:266) at org.apache.hadoop.fs.s3a.S3AFileSystem$OperationCallbacksImpl.copyFile(S3AFileSystem.java:2275) at org.apache.hadoop.fs.s3a.impl.RenameOperation.copySource(RenameOperation.java:562) at org.apache.hadoop.fs.s3a.impl.RenameOperation.renameFileToDest(RenameOperation.java:313) at org.apache.hadoop.fs.s3a.impl.RenameOperation.execute(RenameOperation.java:267) at org.apache.hadoop.fs.s3a.S3AFileSystem.innerRename(S3AFileSystem.java:2194) at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$rename$8(S3AFileSystem.java:2045) 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:2470) at org.apache.hadoop.fs.s3a.S3AFileSystem.rename(S3AFileSystem.java:2043) at org.apache.hadoop.fs.s3a.scale.AbstractSTestS3AHugeFiles.test_100_renameHugeFile(AbstractSTestS3AHugeFiles.java:552) 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: org.apache.hadoop.fs.s3a.audit.AuditFailureException: Multipart IO request com.amazonaws.services.s3.model.EncryptedInitiateMultipartUploadRequest@6781cfbe rejected https://audit.example.org/hadoop/1/op_rename/458f1c6e-aa1f-46da-a46b-9c03c30d2c7f-00000049/?op=op_rename&p1=s3a://stevel-london/tests3ascale/disk/hugefile&pr=stevel&p2=s3a://stevel-london/tests3ascale/disk/hugefileRenamed&ps=7d5e3e98-6b66-49de-a7d2-e6b9281fa410&id=458f1c6e-aa1f-46da-a46b-9c03c30d2c7f-00000049&t0=94&fs=458f1c6e-aa1f-46da-a46b-9c03c30d2c7f&t1=102&ts=1681306681816 at org.apache.hadoop.fs.s3a.audit.impl.LoggingAuditor$LoggingAuditSpan.beforeExecution(LoggingAuditor.java:380) at org.apache.hadoop.fs.s3a.audit.impl.ActiveAuditManagerS3A$WrappingAuditSpan.beforeExecution(ActiveAuditManagerS3A.java:784) at org.apache.hadoop.fs.s3a.audit.impl.ActiveAuditManagerS3A.beforeExecution(ActiveAuditManagerS3A.java:473) at org.apache.hadoop.fs.s3a.audit.impl.ActiveAuditManagerS3A$SdkRequestHandler.beforeExecution(ActiveAuditManagerS3A.java:619) at com.amazonaws.AmazonWebServiceClient.beforeClientExecution(AmazonWebServiceClient.java:787) at com.amazonaws.services.s3.AmazonS3Client.initiateMultipartUpload(AmazonS3Client.java:3662) at com.amazonaws.services.s3.transfer.internal.CopyCallable.initiateMultipartUpload(CopyCallable.java:250) at com.amazonaws.services.s3.transfer.internal.CopyCallable.copyInParts(CopyCallable.java:168) at com.amazonaws.services.s3.transfer.internal.CopyCallable.call(CopyCallable.java:130) at com.amazonaws.services.s3.transfer.internal.CopyMonitor.call(CopyMonitor.java:132) at com.amazonaws.services.s3.transfer.internal.CopyMonitor.call(CopyMonitor.java:43) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ... 1 more ``` i'm going to add a new exception for unsupported io, so it is more meaningful. with tests > s3a fs to reject multipart copy-in-rename when multipart api disabled > --------------------------------------------------------------------- > > Key: HADOOP-18695 > URL: https://issues.apache.org/jira/browse/HADOOP-18695 > Project: Hadoop Common > Issue Type: Improvement > Components: fs/s3 > Affects Versions: 3.4.0 > Reporter: Steve Loughran > Assignee: Steve Loughran > Priority: Minor > > follow-on to HADOOP-18637 and support for huge file uploads with stores which > don't support MPU. > * prevent use of API against any s3 store when disabled, using logging > auditor to reject it > * tests to verify rename of huge files still works (by setting large part > size) -- 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