You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by "Chris Albright (JIRA)" <ji...@apache.org> on 2014/10/31 05:34:33 UTC

[jira] [Created] (PARQUET-124) parquet.hadoop.ParquetOutputCommitter.commitJob() throws parquet.io.ParquetEncodingException

Chris Albright created PARQUET-124:
--------------------------------------

             Summary: parquet.hadoop.ParquetOutputCommitter.commitJob() throws parquet.io.ParquetEncodingException
                 Key: PARQUET-124
                 URL: https://issues.apache.org/jira/browse/PARQUET-124
             Project: Parquet
          Issue Type: Bug
          Components: parquet-mr
    Affects Versions: parquet-mr_1.6.0, 1.6.0rc2
            Reporter: Chris Albright
            Priority: Minor


I'm running an example combining Avro, Spark and Parquet (https://github.com/massie/spark-parquet-example), and in the process of updating the library versions, am getting the warning below.

The version of Parquet-Hadoop in the original example is 1.0.0. I am using 1.6.0rc3

The ParquetFileWriter.mergeFooters(Path, List<Footer>) method is performing a check to ensure the footers are all for files in the output directory. The output directory is supplied by ParquetFileWriter.writeMetadataFile; in 1.0.0, the output path was converted to a fully qualified output path before the call to mergeFooters, but in 1.6.0rc[2,3] that conversion happens after the call to mergeFooters. Because of this, the check within merge footers is failing (the URI for the footers starts with file:, but not the URI for the root path does not)

Here is the warning message and stacktrace.

Oct 30, 2014 9:11:31 PM WARNING: parquet.hadoop.ParquetOutputCommitter: could not write summary file for /tmp/1414728690018-0/output
parquet.io.ParquetEncodingException: file:/tmp/1414728690018-0/output/part-r-00000.parquet invalid: all the files must be contained in the root /tmp/1414728690018-0/output
	at parquet.hadoop.ParquetFileWriter.mergeFooters(ParquetFileWriter.java:422)
	at parquet.hadoop.ParquetFileWriter.writeMetadataFile(ParquetFileWriter.java:398)
	at parquet.hadoop.ParquetOutputCommitter.commitJob(ParquetOutputCommitter.java:50)
	at org.apache.spark.rdd.PairRDDFunctions.saveAsNewAPIHadoopDataset(PairRDDFunctions.scala:936)
	at org.apache.spark.rdd.PairRDDFunctions.saveAsNewAPIHadoopFile(PairRDDFunctions.scala:832)
	at com.zenfractal.SparkParquetExample$.main(SparkParquetExample.scala:72)
	at com.zenfractal.SparkParquetExample.main(SparkParquetExample.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 com.intellij.rt.execution.application.AppMain.main(AppMain.java:134)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)