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 "vijayant soni (Jira)" <ji...@apache.org> on 2020/03/26 13:58:00 UTC
[jira] [Comment Edited] (HADOOP-16942) S3A creating folder level
delete markers
[ https://issues.apache.org/jira/browse/HADOOP-16942?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17067692#comment-17067692 ]
vijayant soni edited comment on HADOOP-16942 at 3/26/20, 1:57 PM:
------------------------------------------------------------------
Error when there are too many delete markers.
The stack trace is from Spark 2.3.2 and Hadoop 2.8.3
{code:java}
org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on <s3-directory>: com.amazonaws.SdkClientException: Failed to parse XML document with handler class com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler: Failed to parse XML document with handler class com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:128)
at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1638)
at org.apache.hadoop.fs.s3a.S3AFileSystem.innerMkdirs(S3AFileSystem.java:1518)
at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1482)
at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:1961)
at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.setupJob(FileOutputCommitter.java:339)
at org.apache.spark.internal.io.HadoopMapReduceCommitProtocol.setupJob(HadoopMapReduceCommitProtocol.scala:162)
at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:176)
at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:154)
at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult$lzycompute(commands.scala:104)
at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult(commands.scala:102)
at org.apache.spark.sql.execution.command.DataWritingCommandExec.doExecute(commands.scala:122)
at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:80)
at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:80)
at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:654)
at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:654)
at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:77)
at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:654)
at org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:273)
at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:267)
at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:225)
at org.apache.spark.sql.DataFrameWriter.parquet(DataFrameWriter.scala:547)
at com.poshmark.spark.streaming.S3Basin$$anonfun$s3basinFunction$1.apply(S3Basin.scala:62)
at com.poshmark.spark.streaming.S3Basin$$anonfun$s3basinFunction$1.apply(S3Basin.scala:19)
at org.apache.spark.streaming.dstream.DStream$$anonfun$foreachRDD$1$$anonfun$apply$mcV$sp$3.apply(DStream.scala:628)
at org.apache.spark.streaming.dstream.DStream$$anonfun$foreachRDD$1$$anonfun$apply$mcV$sp$3.apply(DStream.scala:628)
at org.apache.spark.streaming.dstream.ForEachDStream$$anonfun$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(ForEachDStream.scala:51)
at org.apache.spark.streaming.dstream.ForEachDStream$$anonfun$1$$anonfun$apply$mcV$sp$1.apply(ForEachDStream.scala:51)
at org.apache.spark.streaming.dstream.ForEachDStream$$anonfun$1$$anonfun$apply$mcV$sp$1.apply(ForEachDStream.scala:51)
at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
at org.apache.spark.streaming.dstream.ForEachDStream$$anonfun$1.apply$mcV$sp(ForEachDStream.scala:50)
at org.apache.spark.streaming.dstream.ForEachDStream$$anonfun$1.apply(ForEachDStream.scala:50)
at org.apache.spark.streaming.dstream.ForEachDStream$$anonfun$1.apply(ForEachDStream.scala:50)
at scala.util.Try$.apply(Try.scala:192)
at org.apache.spark.streaming.scheduler.Job.run(Job.scala:39)
at org.apache.spark.streaming.scheduler.JobScheduler$JobHandler$$anonfun$run$1.apply$mcV$sp(JobScheduler.scala:257)
at org.apache.spark.streaming.scheduler.JobScheduler$JobHandler$$anonfun$run$1.apply(JobScheduler.scala:257)
at org.apache.spark.streaming.scheduler.JobScheduler$JobHandler$$anonfun$run$1.apply(JobScheduler.scala:257)
at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
at org.apache.spark.streaming.scheduler.JobScheduler$JobHandler.run(JobScheduler.scala:256)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
Caused by: com.amazonaws.SdkClientException: Failed to parse XML document with handler class com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler
at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser.parseXmlInputStream(XmlResponsesSaxParser.java:161)
at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser.parseListBucketObjectsResponse(XmlResponsesSaxParser.java:317)
at com.amazonaws.services.s3.model.transform.Unmarshallers$ListObjectsUnmarshaller.unmarshall(Unmarshallers.java:70)
at com.amazonaws.services.s3.model.transform.Unmarshallers$ListObjectsUnmarshaller.unmarshall(Unmarshallers.java:59)
at com.amazonaws.services.s3.internal.S3XmlResponseHandler.handle(S3XmlResponseHandler.java:62)
at com.amazonaws.services.s3.internal.S3XmlResponseHandler.handle(S3XmlResponseHandler.java:31)
at com.amazonaws.http.response.AwsResponseHandlerAdapter.handle(AwsResponseHandlerAdapter.java:70)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleResponse(AmazonHttpClient.java:1545)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1270)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1056)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4330)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4277)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4271)
at com.amazonaws.services.s3.AmazonS3Client.listObjects(AmazonS3Client.java:835)
at org.apache.hadoop.fs.s3a.S3AFileSystem.listObjects(S3AFileSystem.java:918)
at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1611)
... 47 more
Caused by: org.xml.sax.SAXParseException; lineNumber: 1; columnNumber: 5; XML document structures must start and end within the same entity.
at org.apache.xerces.util.ErrorHandlerWrapper.createSAXParseException(Unknown Source)
at org.apache.xerces.util.ErrorHandlerWrapper.fatalError(Unknown Source)
at org.apache.xerces.impl.XMLErrorReporter.reportError(Unknown Source)
at org.apache.xerces.impl.XMLErrorReporter.reportError(Unknown Source)
at org.apache.xerces.impl.XMLErrorReporter.reportError(Unknown Source)
at org.apache.xerces.impl.XMLScanner.reportFatalError(Unknown Source)
at org.apache.xerces.impl.XMLDocumentFragmentScannerImpl.endEntity(Unknown Source)
at org.apache.xerces.impl.XMLDocumentScannerImpl.endEntity(Unknown Source)
at org.apache.xerces.impl.XMLEntityManager.endEntity(Unknown Source)
at org.apache.xerces.impl.XMLEntityScanner.load(Unknown Source)
at org.apache.xerces.impl.XMLEntityScanner.skipSpaces(Unknown Source)
at org.apache.xerces.impl.XMLScanner.scanPIData(Unknown Source)
at org.apache.xerces.impl.XMLDocumentFragmentScannerImpl.scanPIData(Unknown Source)
at org.apache.xerces.impl.XMLScanner.scanPI(Unknown Source)
at org.apache.xerces.impl.XMLDocumentScannerImpl$PrologDispatcher.dispatch(Unknown Source)
at org.apache.xerces.impl.XMLDocumentFragmentScannerImpl.scanDocument(Unknown Source)
at org.apache.xerces.parsers.XML11Configuration.parse(Unknown Source)
at org.apache.xerces.parsers.XML11Configuration.parse(Unknown Source)
at org.apache.xerces.parsers.XMLParser.parse(Unknown Source)
at org.apache.xerces.parsers.AbstractSAXParser.parse(Unknown Source)
at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser.parseXmlInputStream(XmlResponsesSaxParser.java:147)
... 68 more
{code}
was (Author: vijayant):
Error when there are too many delete markers.
{code:java}
org.apache.hadoop.fs.s3a.AWSClientIOException: getFileStatus on <s3-directory>: com.amazonaws.SdkClientException: Failed to parse XML document with handler class com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler: Failed to parse XML document with handler class com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler
at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:128)
at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1638)
at org.apache.hadoop.fs.s3a.S3AFileSystem.innerMkdirs(S3AFileSystem.java:1518)
at org.apache.hadoop.fs.s3a.S3AFileSystem.mkdirs(S3AFileSystem.java:1482)
at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:1961)
at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.setupJob(FileOutputCommitter.java:339)
at org.apache.spark.internal.io.HadoopMapReduceCommitProtocol.setupJob(HadoopMapReduceCommitProtocol.scala:162)
at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:176)
at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:154)
at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult$lzycompute(commands.scala:104)
at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult(commands.scala:102)
at org.apache.spark.sql.execution.command.DataWritingCommandExec.doExecute(commands.scala:122)
at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:80)
at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:80)
at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:654)
at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:654)
at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:77)
at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:654)
at org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:273)
at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:267)
at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:225)
at org.apache.spark.sql.DataFrameWriter.parquet(DataFrameWriter.scala:547)
at com.poshmark.spark.streaming.S3Basin$$anonfun$s3basinFunction$1.apply(S3Basin.scala:62)
at com.poshmark.spark.streaming.S3Basin$$anonfun$s3basinFunction$1.apply(S3Basin.scala:19)
at org.apache.spark.streaming.dstream.DStream$$anonfun$foreachRDD$1$$anonfun$apply$mcV$sp$3.apply(DStream.scala:628)
at org.apache.spark.streaming.dstream.DStream$$anonfun$foreachRDD$1$$anonfun$apply$mcV$sp$3.apply(DStream.scala:628)
at org.apache.spark.streaming.dstream.ForEachDStream$$anonfun$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(ForEachDStream.scala:51)
at org.apache.spark.streaming.dstream.ForEachDStream$$anonfun$1$$anonfun$apply$mcV$sp$1.apply(ForEachDStream.scala:51)
at org.apache.spark.streaming.dstream.ForEachDStream$$anonfun$1$$anonfun$apply$mcV$sp$1.apply(ForEachDStream.scala:51)
at org.apache.spark.streaming.dstream.DStream.createRDDWithLocalProperties(DStream.scala:416)
at org.apache.spark.streaming.dstream.ForEachDStream$$anonfun$1.apply$mcV$sp(ForEachDStream.scala:50)
at org.apache.spark.streaming.dstream.ForEachDStream$$anonfun$1.apply(ForEachDStream.scala:50)
at org.apache.spark.streaming.dstream.ForEachDStream$$anonfun$1.apply(ForEachDStream.scala:50)
at scala.util.Try$.apply(Try.scala:192)
at org.apache.spark.streaming.scheduler.Job.run(Job.scala:39)
at org.apache.spark.streaming.scheduler.JobScheduler$JobHandler$$anonfun$run$1.apply$mcV$sp(JobScheduler.scala:257)
at org.apache.spark.streaming.scheduler.JobScheduler$JobHandler$$anonfun$run$1.apply(JobScheduler.scala:257)
at org.apache.spark.streaming.scheduler.JobScheduler$JobHandler$$anonfun$run$1.apply(JobScheduler.scala:257)
at scala.util.DynamicVariable.withValue(DynamicVariable.scala:58)
at org.apache.spark.streaming.scheduler.JobScheduler$JobHandler.run(JobScheduler.scala:256)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
Caused by: com.amazonaws.SdkClientException: Failed to parse XML document with handler class com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser$ListBucketHandler
at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser.parseXmlInputStream(XmlResponsesSaxParser.java:161)
at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser.parseListBucketObjectsResponse(XmlResponsesSaxParser.java:317)
at com.amazonaws.services.s3.model.transform.Unmarshallers$ListObjectsUnmarshaller.unmarshall(Unmarshallers.java:70)
at com.amazonaws.services.s3.model.transform.Unmarshallers$ListObjectsUnmarshaller.unmarshall(Unmarshallers.java:59)
at com.amazonaws.services.s3.internal.S3XmlResponseHandler.handle(S3XmlResponseHandler.java:62)
at com.amazonaws.services.s3.internal.S3XmlResponseHandler.handle(S3XmlResponseHandler.java:31)
at com.amazonaws.http.response.AwsResponseHandlerAdapter.handle(AwsResponseHandlerAdapter.java:70)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleResponse(AmazonHttpClient.java:1545)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1270)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1056)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699)
at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667)
at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:513)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4330)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4277)
at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:4271)
at com.amazonaws.services.s3.AmazonS3Client.listObjects(AmazonS3Client.java:835)
at org.apache.hadoop.fs.s3a.S3AFileSystem.listObjects(S3AFileSystem.java:918)
at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:1611)
... 47 more
Caused by: org.xml.sax.SAXParseException; lineNumber: 1; columnNumber: 5; XML document structures must start and end within the same entity.
at org.apache.xerces.util.ErrorHandlerWrapper.createSAXParseException(Unknown Source)
at org.apache.xerces.util.ErrorHandlerWrapper.fatalError(Unknown Source)
at org.apache.xerces.impl.XMLErrorReporter.reportError(Unknown Source)
at org.apache.xerces.impl.XMLErrorReporter.reportError(Unknown Source)
at org.apache.xerces.impl.XMLErrorReporter.reportError(Unknown Source)
at org.apache.xerces.impl.XMLScanner.reportFatalError(Unknown Source)
at org.apache.xerces.impl.XMLDocumentFragmentScannerImpl.endEntity(Unknown Source)
at org.apache.xerces.impl.XMLDocumentScannerImpl.endEntity(Unknown Source)
at org.apache.xerces.impl.XMLEntityManager.endEntity(Unknown Source)
at org.apache.xerces.impl.XMLEntityScanner.load(Unknown Source)
at org.apache.xerces.impl.XMLEntityScanner.skipSpaces(Unknown Source)
at org.apache.xerces.impl.XMLScanner.scanPIData(Unknown Source)
at org.apache.xerces.impl.XMLDocumentFragmentScannerImpl.scanPIData(Unknown Source)
at org.apache.xerces.impl.XMLScanner.scanPI(Unknown Source)
at org.apache.xerces.impl.XMLDocumentScannerImpl$PrologDispatcher.dispatch(Unknown Source)
at org.apache.xerces.impl.XMLDocumentFragmentScannerImpl.scanDocument(Unknown Source)
at org.apache.xerces.parsers.XML11Configuration.parse(Unknown Source)
at org.apache.xerces.parsers.XML11Configuration.parse(Unknown Source)
at org.apache.xerces.parsers.XMLParser.parse(Unknown Source)
at org.apache.xerces.parsers.AbstractSAXParser.parse(Unknown Source)
at com.amazonaws.services.s3.model.transform.XmlResponsesSaxParser.parseXmlInputStream(XmlResponsesSaxParser.java:147)
... 68 more
{code}
> S3A creating folder level delete markers
> ----------------------------------------
>
> Key: HADOOP-16942
> URL: https://issues.apache.org/jira/browse/HADOOP-16942
> Project: Hadoop Common
> Issue Type: Task
> Components: fs/s3
> Affects Versions: 3.2.1
> Reporter: vijayant soni
> Priority: Minor
>
> Using S3A URL scheme while writing out data from Spark to S3 is creating many folder level delete markers.
> Writing the same with S3 URL scheme, does not create any delete markers at all.
>
> Spark - 2.4.4
> Hadoop - 3.2.1
> EMR version - 6.0.0
> {code:scala}
> spark-shell
> Welcome to
> ____ __
> / __/__ ___ _____/ /__
> _\ \/ _ \/ _ `/ __/ '_/
> /___/ .__/\_,_/_/ /_/\_\ version 2.4.4
> /_/
>
> Using Scala version 2.12.10 (OpenJDK 64-Bit Server VM, Java 1.8.0_242)
> Type in expressions to have them evaluated.
> Type :help for more information.
> scala> val df = spark.sql("select 1 as a")
> df: org.apache.spark.sql.DataFrame = [a: int]
> scala> df.show(false)
> +---+
> |a |
> +---+
> |1 |
> +---+
> scala> // Writing to S3 using s3
> scala> df.write.mode(org.apache.spark.sql.SaveMode.Overwrite).save("s3://stage-dwh/tmp/vijayant/s3/")
>
> scala> // Writing to S3 using s3a
> scala> df.write.mode(org.apache.spark.sql.SaveMode.Overwrite).save("s3a://stage-dwh/tmp/vijayant/s3a/")
>
> scala>
> {code}
> Getting delete markers from `s3` write
> {code:bash}
> aws s3api list-object-versions --bucket stage-dwh --prefix tmp/vijayant/s3
> {
> "Versions": [
> {
> "LastModified": "2020-03-26T12:57:54.000Z",
> "VersionId": "h7_SIsHYoC.1il2s4qporAFnVbLgiLN5",
> "ETag": "\"d41d8cd98f00b204e9800998ecf8427e\"",
> "StorageClass": "STANDARD",
> "Key": "tmp/vijayant/s3/_SUCCESS",
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": true,
> "Size": 0
> },
> {
> "LastModified": "2020-03-26T12:57:54.000Z",
> "VersionId": "pOALzyzpBR7glCEk3cqPOR.u8QCIcLnC",
> "ETag": "\"26e70a1e26c709e3e8498acd49cfaaa3-1\"",
> "StorageClass": "STANDARD",
> "Key": "tmp/vijayant/s3/part-00000-9af16781-7944-497d-9b19-f31ab1e5f850-c000.snappy.parquet",
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": true,
> "Size": 384
> }
> ]
> }
> {code}
> Getting delete markers from `s3a` write
> {code:bash}
> aws s3api list-object-versions --bucket stage-dwh --prefix tmp/vijayant/s3a
> {
> "DeleteMarkers": [
> {
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": true,
> "VersionId": "Jd8PHlUK3TbVJY2RWQxi74a6.2Gp2mUL",
> "Key": "tmp/vijayant/s3a/",
> "LastModified": "2020-03-26T13:00:14.000Z"
> },
> {
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": false,
> "VersionId": "j2llpkiXzEaomJr5xLhQ9xTmfoq_8dOy",
> "Key": "tmp/vijayant/s3a/",
> "LastModified": "2020-03-26T13:00:13.000Z"
> },
> {
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": false,
> "VersionId": "ry6BqTUGvyY3U.eqFfgg2hJ2BBMxVcwH",
> "Key": "tmp/vijayant/s3a/",
> "LastModified": "2020-03-26T13:00:13.000Z"
> },
> {
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": false,
> "VersionId": "xG2oNiCpovqWCSZxaWiqtL.E7znE7AmR",
> "Key": "tmp/vijayant/s3a/",
> "LastModified": "2020-03-26T13:00:12.000Z"
> },
> {
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": false,
> "VersionId": "OedjkEU1VeWl0ZZouur.13dufhYa7JXm",
> "Key": "tmp/vijayant/s3a/",
> "LastModified": "2020-03-26T13:00:01.000Z"
> },
> {
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": true,
> "VersionId": "Nn8O947GGwqCePelc9VL9O2sWsmSsy2i",
> "Key": "tmp/vijayant/s3a/_temporary/",
> "LastModified": "2020-03-26T13:00:13.000Z"
> },
> {
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": false,
> "VersionId": "7DcZ4b3pmoIi_TuzoRsykdtzyUGDLUo9",
> "Key": "tmp/vijayant/s3a/_temporary/",
> "LastModified": "2020-03-26T13:00:12.000Z"
> },
> {
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": false,
> "VersionId": "eFahQrYnglWeRHZHTod6IszSoNE3jPCH",
> "Key": "tmp/vijayant/s3a/_temporary/",
> "LastModified": "2020-03-26T13:00:01.000Z"
> },
> {
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": true,
> "VersionId": "m4mGr.QA3sO0pQb_tuZEZX6OVIeprgwl",
> "Key": "tmp/vijayant/s3a/_temporary/0/",
> "LastModified": "2020-03-26T13:00:13.000Z"
> },
> {
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": false,
> "VersionId": "2TBn8RzdMKzEqn6cP8O_CI9OdZkhvv53",
> "Key": "tmp/vijayant/s3a/_temporary/0/",
> "LastModified": "2020-03-26T13:00:12.000Z"
> },
> {
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": true,
> "VersionId": "NXVubIX_eu9RYLDWpD4JH91VK08OmHwu",
> "Key": "tmp/vijayant/s3a/_temporary/0/_temporary/",
> "LastModified": "2020-03-26T13:00:13.000Z"
> },
> {
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": false,
> "VersionId": "FtubGslxkfMiT5uxuuEorWsg0OIvXmzY",
> "Key": "tmp/vijayant/s3a/_temporary/0/_temporary/",
> "LastModified": "2020-03-26T13:00:12.000Z"
> },
> {
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": true,
> "VersionId": "0tv9I0s1mvurxP4KX_Zgqr7P8OQ5bIs7",
> "Key": "tmp/vijayant/s3a/_temporary/0/_temporary/attempt_20200326130000_0002_m_000000_2/",
> "LastModified": "2020-03-26T13:00:14.000Z"
> },
> {
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": false,
> "VersionId": "y058RX4xXC.a_ltup_OxdI7S5o288h38",
> "Key": "tmp/vijayant/s3a/_temporary/0/_temporary/attempt_20200326130000_0002_m_000000_2/",
> "LastModified": "2020-03-26T13:00:12.000Z"
> },
> {
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": true,
> "VersionId": "T54GB8P1SVmWUu_9lXXogf970cYZOszE",
> "Key": "tmp/vijayant/s3a/_temporary/0/_temporary/attempt_20200326130000_0002_m_000000_2/part-00000-8ecb77ed-8279-4256-9ef1-5ea352318c1a-c000.snappy.parquet",
> "LastModified": "2020-03-26T13:00:13.000Z"
> }
> ],
> "Versions": [
> {
> "LastModified": "2020-03-26T13:00:14.000Z",
> "VersionId": "3HSpCqBQyrVoh9X1tTfskNEiQIet7f_0",
> "ETag": "\"d41d8cd98f00b204e9800998ecf8427e\"",
> "StorageClass": "STANDARD",
> "Key": "tmp/vijayant/s3a/_SUCCESS",
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": true,
> "Size": 0
> },
> {
> "LastModified": "2020-03-26T13:00:00.000Z",
> "VersionId": "zB.ELKr2RcK9RgdSgx5wwj55YPlZTWD0",
> "ETag": "\"d41d8cd98f00b204e9800998ecf8427e\"",
> "StorageClass": "STANDARD",
> "Key": "tmp/vijayant/s3a/_temporary/0/",
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": false,
> "Size": 0
> },
> {
> "LastModified": "2020-03-26T13:00:13.000Z",
> "VersionId": "XoIDfWRP0Y6DySn_FVkh3z.LCSCv1H4x",
> "ETag": "\"d41d8cd98f00b204e9800998ecf8427e\"",
> "StorageClass": "STANDARD",
> "Key": "tmp/vijayant/s3a/_temporary/0/_temporary/attempt_20200326130000_0002_m_000000_2/",
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": false,
> "Size": 0
> },
> {
> "LastModified": "2020-03-26T13:00:12.000Z",
> "VersionId": "OL24nTI4C0DJFur6ZfXeWFH1N_eo.SIl",
> "ETag": "\"1c1179f44b770f1d661f06b9324c27da\"",
> "StorageClass": "STANDARD",
> "Key": "tmp/vijayant/s3a/_temporary/0/_temporary/attempt_20200326130000_0002_m_000000_2/part-00000-8ecb77ed-8279-4256-9ef1-5ea352318c1a-c000.snappy.parquet",
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": false,
> "Size": 384
> },
> {
> "LastModified": "2020-03-26T13:00:13.000Z",
> "VersionId": "EscpHQeMrYBhDGdmnH5TPEDLpLUwZzBS",
> "ETag": "\"1c1179f44b770f1d661f06b9324c27da\"",
> "StorageClass": "STANDARD",
> "Key": "tmp/vijayant/s3a/part-00000-8ecb77ed-8279-4256-9ef1-5ea352318c1a-c000.snappy.parquet",
> "Owner": {
> "DisplayName": "<display-name>",
> "ID": "<owner-id>"
> },
> "IsLatest": true,
> "Size": 384
> }
> ]
> }
> {code}
> This in turn makes listing objects slow and we have even noticed timeouts due to too many delete markers.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org