You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by "Emil Kleszcz (Jira)" <ji...@apache.org> on 2022/11/15 11:35:00 UTC

[jira] [Commented] (PARQUET-632) Parquet file in invalid state while writing to S3 from EMR

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

Emil Kleszcz commented on PARQUET-632:
--------------------------------------

The same issue was observed in spark 3.2.1 when writing a parquet file to HDFS using parquet v. 1.12.2. 

Some of the logs from the server side in HDFS, NN:
{code:java}
<2022-11-14T08:12:31.941+0100> <INFO> <org.apache.hadoop.hdfs.StateChange>: <BLOCK* allocate blk_5112663052_4039631919, replicas=xxx:1004, xxx:1004, xxx:1004 for /project//part-00003-a1dc7884-5736-437f-a573-35c8d0ea922f.c000.snappy.parquet>
<2022-11-14T08:13:43.029+0100> <INFO> <org.apache.hadoop.hdfs.StateChange>: <BLOCK* allocate blk_5112664776_4039633643, replicas=xxx:1004, xxx:1004, xxx:1004 for /project/437f-a573-35c8d0ea922f.c000.snappy.parquet>
<2022-11-14T08:14:51.025+0100> <INFO> <org.apache.hadoop.ipc.Server>: <IPC Server handler 124 on default port 8020, call Call#64349 Retry#0 org.apache.hadoop.hdfs.protocol.ClientProtocol.addBlock from xxx:45140: java.io.FileNotFoundException: File does not exist: /project/part-00003-a1dc7884-5736-437f-a573-35c8d0ea922f.c000.snappy.parquet (inode 7450217277) Holder DFSClient_NONMAPREDUCE_-885269500_210 does not have any open files.>{code}

DN logs for one of the file blocks:
 
{code:java}
<2022-11-14T08:12:31.945+0100> <INFO> <org.apache.hadoop.hdfs.server.datanode.DataNode>: <Receiving BP-1877419787-xxx-1507645851253:blk_5112663052_4039631919 src: /xxx:43856 dest: /xxx:1004> <2022-11-14T08:13:43.028+0100> <INFO> <org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace>: <src: /xxx:43856, dest: /xxx:1004, bytes: 268435456, op: HDFS_WRITE, cliID: DFSClient_NONMAPREDUCE_-885269500_210, offset: 0, srvID: e20a1db9-5297-41b3-a49a-a6dc6a2169b6, blockid: BP-1877419787-xxx-1507645851253:blk_5112663052_4039631919, duration(ns): 71075002589> <2022-11-14T08:13:43.028+0100> <INFO> <org.apache.hadoop.hdfs.server.datanode.DataNode>: <PacketResponder: BP-1877419787-xxx-1507645851253:blk_5112663052_4039631919, type=HAS_DOWNSTREAM_IN_PIPELINE, downstreams=2:[xxx:1004, xxx:1004] terminating> <2022-11-14T08:15:01.095+0100> <INFO> <org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetAsyncDiskService>: <Scheduling blk_5112663052_4039631919 replica FinalizedReplica, blk_5112663052_4039631919, FINALIZED <2022-11-14T08:15:01.157+0100> <INFO> <org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetAsyncDiskService>: <Deleted BP-1877419787-xxx-1507645851253 blk_5112663052_4039631919 URI file:/data18/hdfs/data/current/BP-1877419787-xxx-1507645851253/current/finalized/subdir29/subdir12/blk_5112663052>{code}

> Parquet file in invalid state while writing to S3 from EMR
> ----------------------------------------------------------
>
>                 Key: PARQUET-632
>                 URL: https://issues.apache.org/jira/browse/PARQUET-632
>             Project: Parquet
>          Issue Type: Bug
>    Affects Versions: 1.7.0
>            Reporter: Peter Halliday
>            Priority: Blocker
>
> I'm writing parquet to S3 from Spark 1.6.1 on EMR.  And when it got to the last few files to write to S3, I received this stacktrace in the log with no other errors before or after it.  It's very consistent.  This particular batch keeps erroring the same way.
> {noformat}
> 2016-06-10 01:46:05,282] WARN org.apache.spark.scheduler.TaskSetManager [task-result-getter-2hread] - Lost task 3737.0 in stage 2.0 (TID 10585, ip-172-16-96-32.ec2.internal): org.apache.spark.SparkException: Task failed while writing rows.
> 	at org.apache.spark.sql.execution.datasources.DynamicPartitionWriterContainer.writeRows(WriterContainer.scala:414)
> 	at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation$$anonfun$run$1$$anonfun$apply$mcV$sp$3.apply(InsertIntoHadoopFsRelation.scala:150)
> 	at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation$$anonfun$run$1$$anonfun$apply$mcV$sp$3.apply(InsertIntoHadoopFsRelation.scala:150)
> 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
> 	at org.apache.spark.scheduler.Task.run(Task.scala:89)
> 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> 	at java.lang.Thread.run(Thread.java:745)
> Caused by: java.io.IOException: The file being written is in an invalid state. Probably caused by an error thrown previously. Current state: COLUMN
> 	at org.apache.parquet.hadoop.ParquetFileWriter$STATE.error(ParquetFileWriter.java:146)
> 	at org.apache.parquet.hadoop.ParquetFileWriter$STATE.startBlock(ParquetFileWriter.java:138)
> 	at org.apache.parquet.hadoop.ParquetFileWriter.startBlock(ParquetFileWriter.java:195)
> 	at org.apache.parquet.hadoop.InternalParquetRecordWriter.flushRowGroupToStore(InternalParquetRecordWriter.java:153)
> 	at org.apache.parquet.hadoop.InternalParquetRecordWriter.close(InternalParquetRecordWriter.java:113)
> 	at org.apache.parquet.hadoop.ParquetRecordWriter.close(ParquetRecordWriter.java:112)
> 	at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.close(ParquetRelation.scala:101)
> 	at org.apache.spark.sql.execution.datasources.DynamicPartitionWriterContainer.writeRows(WriterContainer.scala:405)
> 	... 8 more
> {noformat}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)