You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "Steve Loughran (JIRA)" <ji...@apache.org> on 2017/02/22 12:14:44 UTC

[jira] [Commented] (HADOOP-14101) MultiObjectDeleteException thrown when writing directly to s3a

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

Steve Loughran commented on HADOOP-14101:
-----------------------------------------

[~Thomas Demoor] what do you think? I think we're seeing a partially deleted message due to concurrent deletes, but that this isn't actually harmful. That is, provided the final message just entries which weren't there when the request was made, in which case, given we want an idempotent call and all, all is well

I don't want to try and be clever about reissuing delete requests, precisely because of race conditions if a another caller is creating things. Its bad enough with a blob-by-blob delete as it is. Instead I think I'll catch and build up a report to WARN on.

Luke, while I look at this, can I highlight that you should not be using S3 as a direct destination of spark work, not unless you've got something like s3mper or s3guard (HADOOP-13345) providing a consistency layer. Because lists are not consistent with the actual state of the store, there's a risk that the final commit-by-rename protocol won't actually enumerate all files to move into the right place. 

> MultiObjectDeleteException thrown when writing directly to s3a
> --------------------------------------------------------------
>
>                 Key: HADOOP-14101
>                 URL: https://issues.apache.org/jira/browse/HADOOP-14101
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: fs/s3
>    Affects Versions: 2.7.0, 2.7.1, 2.7.2, 2.7.3
>            Reporter: Luke Miner
>              Labels: s3
>
> When I write large jobs to s3a in spark, I get non-deterministic {{MultiObjectDeleteException}}. This seems to happen at the end of the job when the temporary files are being cleaned up. It is fixed when {{fs.s3a.multiobjectdelete.enable}} is set to {{false}}.
> {code}
> Exception in thread "main" com.amazonaws.services.s3.model.MultiObjectDeleteException: Status Code: 0, AWS Service: null, AWS Request ID: null, AWS Error Code: null, AWS Error Message: One or more objects could not be deleted, S3 Extended Request ID: null
> 	at com.amazonaws.services.s3.AmazonS3Client.deleteObjects(AmazonS3Client.java:1745)
> 	at org.apache.hadoop.fs.s3a.S3AFileSystem.delete(S3AFileSystem.java:674)
> 	at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:90)
> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:60)
> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:58)
> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.doExecute(commands.scala:74)
> 	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115)
> 	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115)
> 	at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:136)
> 	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
> 	at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:133)
> 	at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:114)
> 	at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:86)
> 	at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:86)
> 	at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:510)
> 	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:211)
> 	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:194)
> 	at org.apache.spark.sql.DataFrameWriter.parquet(DataFrameWriter.scala:488)
> 	at Json2Pq$.main(json2pq.scala:179)
> 	at Json2Pq.main(json2pq.scala)
> 	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> 	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 	at java.lang.reflect.Method.invoke(Method.java:606)
> 	at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:736)
> 	at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:185)
> 	at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:210)
> 	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:124)
> 	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org