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

Steve Loughran commented on HADOOP-19080:
-----------------------------------------

stack

{code}
py4j.protocol.Py4JJavaError: An error occurred while calling o121.save. : 
org.apache.hadoop.fs.s3a.AWSBadRequestException: PUT 0-byte object  on 
datalake/showroom_variants: com.amazonaws.services.s3.model.AmazonS3Exception: 
Content-MD5 OR x-amz-checksum- HTTP header is required for Put Object requests 
with Object Lock parameters (Service: Amazon S3; Status Code: 400; Error Code: 
InvalidRequest; Request ID: <request_id>; S3 Extended Request ID: <request_id>; 
Proxy: null), S3 Extended Request ID: <some_id>:InvalidRequest: Content-MD5 OR 
x-amz-checksum- HTTP header is required for Put Object requests with Object 
Lock parameters (Service: Amazon S3; Status Code: 400; Error Code: 
InvalidRequest; Request ID: <request_id>; S3 Extended Request ID: 
<extended_request_id>; Proxy: null)
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:249)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:119)
at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$4(Invoker.java:322)
at org.apache.hadoop.fs.s3a.Invoker.retryUntranslated(Invoker.java:414)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:318)
at org.apache.hadoop.fs.s3a.Invoker.retry(Invoker.java:293)
at 
org.apache.hadoop.fs.s3a.S3AFileSystem.createEmptyObject(S3AFileSystem.java:4532)
at org.apache.hadoop.fs.s3a.S3AFileSystem.access$1900(S3AFileSystem.java:259)
at 
org.apache.hadoop.fs.s3a.S3AFileSystem$MkdirOperationCallbacksImpl.createFakeDirectory(S3AFileSystem.java:3461)
at org.apache.hadoop.fs.s3a.impl.MkdirOperation.execute(MkdirOperation.java:121)
at org.apache.hadoop.fs.s3a.impl.MkdirOperation.execute(MkdirOperation.java:45)
at 
org.apache.hadoop.fs.s3a.impl.ExecutingStoreOperation.apply(ExecutingStoreOperation.java:76)
at 
org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.lambda$trackDurationOfOperation$5(IOStatisticsBinding.java:499)
at 
org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration(IOStatisticsBinding.java:444)
at 
org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2337)
at 
org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2356)
at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:3428)
at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:2388)
at 
org.apache.hudi.common.table.HoodieTableMetaClient.initTableAndGetMetaClient(HoodieTableMetaClient.java:481)
at 
org.apache.hudi.common.table.HoodieTableMetaClient$PropertyBuilder.initTable(HoodieTableMetaClient.java:1201)
at 
org.apache.hudi.HoodieSparkSqlWriterInternal.writeInternal(HoodieSparkSqlWriter.scala:323)
at 
org.apache.hudi.HoodieSparkSqlWriterInternal.write(HoodieSparkSqlWriter.scala:204)
at org.apache.hudi.HoodieSparkSqlWriter$.write(HoodieSparkSqlWriter.scala:121)
at org.apache.hudi.DefaultSource.createRelation(DefaultSource.scala:150)
at 
org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run(SaveIntoDataSourceCommand.scala:47)
at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:75)
at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:73)
at 
org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:84)
at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:98)
at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:118)
at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:195)
at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:103)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:827)
at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:65)
at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:98)
at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:94)
at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:512)
at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:104)
at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:512)
at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:31)
at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:31)
at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:31)
at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:488)
at 
org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:94)
at 
org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:81)
at 
org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:79)
at 
org.apache.spark.sql.execution.QueryExecution.assertCommandExecuted(QueryExecution.scala:133)
at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:856)
at 
org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:387)
at org.apache.spark.sql.DataFrameWriter.saveInternal(DataFrameWriter.scala:360)
at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:239)
at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native 
Method)
at 
java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.base/java.lang.reflect.Method.invoke(Method.java:566)
at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:374)
at py4j.Gateway.invoke(Gateway.java:282)
at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
at py4j.commands.CallCommand.execute(CallCommand.java:79)
at py4j.ClientServerConnection.waitForCommands(ClientServerConnection.java:182)
at py4j.ClientServerConnection.run(ClientServerConnection.java:106)
at java.base/java.lang.Thread.run(Thread.java:829) Caused by: 
com.amazonaws.services.s3.model.AmazonS3Exception: Content-MD5 OR 
x-amz-checksum- HTTP header is required for Put Object requests with Object 
Lock parameters (Service: Amazon S3; Status Code: 400; Error Code: 
InvalidRequest; Request ID: DKTYYJHRJGJZFQ1X; S3 Extended Request ID: 
37QbiLXXUA+I4SaRhZZvqSNQz5P2zPfykuw9nzmgmRhBpYPN1c9GdN76pzZec7Rpd7b0ek4IdXs9GQcN99eUUfkwd3+Xofv2lq9At781zwI=;
 Proxy: null), S3 Extended Request ID: 
37QbiLXXUA+I4SaRhZZvqSNQz5P2zPfykuw9nzmgmRhBpYPN1c9GdN76pzZec7Rpd7b0ek4IdXs9GQcN99eUUfkwd3+Xofv2lq9At781zwI=
at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1879)
at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1418)
at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1387)
at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1157)
at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:814)
at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:781)
at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:755)
at 
com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:715)
at 
com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:697)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:561)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:541)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5470)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5417)
at com.amazonaws.services.s3.AmazonS3Client.access$300(AmazonS3Client.java:422)
at 
com.amazonaws.services.s3.AmazonS3Client$PutObjectStrategy.invokeServiceCall(AmazonS3Client.java:6551)
at 
com.amazonaws.services.s3.AmazonS3Client.uploadObject(AmazonS3Client.java:1862)
at com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1822)
at 
org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$putObjectDirect$17(S3AFileSystem.java:2877)
at 
org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier(IOStatisticsBinding.java:604)
at 
org.apache.hadoop.fs.s3a.S3AFileSystem.putObjectDirect(S3AFileSystem.java:2874)
at 
org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$createEmptyObject$32(S3AFileSystem.java:4534)
at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:117)         ..

{code}


> S3A createFakeDirectory/put fails on object lock bucket if the path exists
> --------------------------------------------------------------------------
>
>                 Key: HADOOP-19080
>                 URL: https://issues.apache.org/jira/browse/HADOOP-19080
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: fs/s3
>    Affects Versions: 3.3.6
>            Reporter: Steve Loughran
>            Priority: Major
>
> s3 bucket with object lock enabled fails in createFakeDirectory (reported on 
> S.O)
> error implies that we need to calculate and include the md5 checksum on the 
> PUT, which gets complex once you include CSE into the mix: the checksum of 
> the encrypted data is what'd be required.



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