My spark is 0.9.0-SNAPSHOT, built from wherever master was at the time
(like a week or two ago).

If you're referring to the cloneRecords parameter, it appears to default to
true, but even when I add it explicitly, I get the same error.


On Wed, Feb 5, 2014 at 7:17 PM, Frank Austin Nothaft
<[email protected]>wrote:

> Uri,
>
> Which version of Spark are you running? If it is >0.9.0, you need to add
> an optional true argument at the end of the sc.newApiHadoopFile(…) call to
> read Parquet data.
>
> Regards,
>
> Frank Austin Nothaft
> [email protected]
> [email protected]
> 202-340-0466
>
> On Feb 5, 2014, at 7:14 PM, Uri Laserson <[email protected]> wrote:
>
> I am cross-posting on the parquet mailing list.  Short recap: I am trying
> to read Parquet data from the spark interactive shell.
>
> I have added all the necessary parquet jars to SPARK_CLASSPATH:
>
> export
> SPARK_CLASSPATH="/Users/laserson/repos/parquet-mr/parquet-avro/target/parquet-avro-1.3.3-SNAPSHOT.jar:/Users/laserson/repos/parquet-mr/parquet-hadoop/target/parquet-hadoop-1.3.3-SNAPSHOT.jar:/Users/laserson/repos/parquet-mr/parquet-common/target/parquet-common-1.3.3-SNAPSHOT.jar:/Users/laserson/repos/parquet-mr/parquet-column/target/parquet-column-1.3.3-SNAPSHOT.jar:/Users/laserson/repos/parquet-format/target/parquet-format-2.0.1-SNAPSHOT.jar:/Users/laserson/repos/parquet-mr/parquet-jackson/target/parquet-jackson-1.3.3-SNAPSHOT.jar"
>
> From the spark-shell, I run:
>
> val job = new Job(sc.hadoopConfiguration)
> ParquetInputFormat.setReadSupportClass(job,
> classOf[AvroReadSupport[GenericRecord]])
> val records1 =
> sc.newAPIHadoopFile("/Users/laserson/temp/test-parquet/alltypeuri",
> classOf[ParquetInputFormat[GenericRecord]], classOf[Void],
> classOf[GenericRecord], job.getConfiguration)
>
> Then I try
>
> records1.count
>
> Which gives the following error:
>
> 14/02/05 18:42:22 ERROR Executor: Exception in task ID 1
> java.lang.NoSuchMethodError:
> org.apache.avro.Schema$Field.<init>(Ljava/lang/String;Lorg/apache/avro/Schema;Ljava/lang/String;Lparquet/org/codehaus/jackson/JsonNode;)V
>  at
> parquet.avro.AvroSchemaConverter.convertFields(AvroSchemaConverter.java:191)
> at parquet.avro.AvroSchemaConverter.convert(AvroSchemaConverter.java:177)
>  at parquet.avro.AvroReadSupport.prepareForRead(AvroReadSupport.java:86)
> at
> parquet.hadoop.InternalParquetRecordReader.initialize(InternalParquetRecordReader.java:142)
>  at
> parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:118)
> at
> parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:107)
>  at
> org.apache.spark.rdd.NewHadoopRDD$$anon$1.<init>(NewHadoopRDD.scala:106)
> at org.apache.spark.rdd.NewHadoopRDD.compute(NewHadoopRDD.scala:94)
>  at org.apache.spark.rdd.NewHadoopRDD.compute(NewHadoopRDD.scala:57)
> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:241)
>  at org.apache.spark.rdd.RDD.iterator(RDD.scala:232)
> at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:109)
>  at org.apache.spark.scheduler.Task.run(Task.scala:53)
> at
> org.apache.spark.executor.Executor$TaskRunner$$anonfun$run$1.apply$mcV$sp(Executor.scala:213)
>  at
> org.apache.spark.deploy.SparkHadoopUtil.runAsUser(SparkHadoopUtil.scala:49)
> at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:178)
>  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:744)
>
>
> My hypothesis is that this a shading problem.  It appears that the code is
> trying to call a constructor that looks like this:
>
> String.Field(String, Schema, String, *parquet*
> .org.codehaus.jackson.JsonNode)
>
> but the signature from the spark-assembly jar is
>
> public org.apache.avro.Schema$Field(java.lang.String,
> org.apache.avro.Schema, java.lang.String, org.codehaus.jackson.JsonNode);
>
> Where do I go from here?
>
> Uri
>
>
>
>
>
>
>
> On Wed, Feb 5, 2014 at 5:02 PM, Uri Laserson <[email protected]>wrote:
>
>> Yep, I did not include that jar in the class path.  Now I've got some
>> "real" errors to try to work through.  Thanks!
>>
>>
>>  On Wed, Feb 5, 2014 at 3:52 PM, Jey Kottalam <[email protected]>wrote:
>>
>>> Hi Uri,
>>>
>>> Could you try adding the parquet-jackson JAR to your classpath? There
>>> may possibly be other parquet-avro dependencies that are missing too.
>>>
>>> http://mvnrepository.com/artifact/com.twitter/parquet-jackson/1.3.2
>>>
>>> -Jey
>>>
>>> On Wed, Feb 5, 2014 at 3:02 PM, Uri Laserson <[email protected]>
>>> wrote:
>>> > Yes, of course.  That class is a jackson class, and I'm not sure why
>>> it's
>>> > being referred to as
>>> parquet.org.codehaus.jackson.JsonGenerationException.
>>> >
>>> > org.codehaus.jackson.JsonGenerationException is on the classpath.  But
>>> not
>>> > when it's prefixed by parquet.
>>> >
>>> >
>>> > On Wed, Feb 5, 2014 at 12:06 PM, Andrew Ash <[email protected]>
>>> wrote:
>>> >>
>>> >> I'm assuming you checked all the jars in SPARK_CLASSPATH to confirm
>>> that
>>> >> parquet/org/codehaus/jackson/JsonGenerationException.class exists in
>>> one of
>>> >> them?
>>> >>
>>> >>
>>> >> On Wed, Feb 5, 2014 at 12:02 PM, Uri Laserson <[email protected]>
>>> >> wrote:
>>> >>>
>>> >>> Has anyone tried this?  I'd like to read a bunch of Avro
>>> GenericRecords
>>> >>> from a Parquet file. I'm having a bit of trouble with respect to
>>> >>> dependencies.  My latest attempt looks like this:
>>> >>>
>>> >>> export
>>> >>>
>>> SPARK_CLASSPATH="/Users/laserson/repos/parquet-mr/parquet-avro/target/parquet-avro-1.3.3-SNAPSHOT.jar:/Users/laserson/repos/parquet-mr/parquet-hadoop/target/parquet-hadoop-1.3.3-SNAPSHOT.jar:/Users/laserson/repos/parquet-mr/parquet-common/target/parquet-common-1.3.3-SNAPSHOT.jar:/Users/laserson/repos/parquet-mr/parquet-column/target/parquet-column-1.3.3-SNAPSHOT.jar:/Users/laserson/repos/parquet-format/target/parquet-format-2.0.1-SNAPSHOT.jar"
>>> >>>
>>> >>> MASTER=local ~/repos/incubator-spark/bin/spark-shell
>>> >>>
>>> >>> Then in the shell:
>>> >>>
>>> >>> val records1 =
>>> >>> sc.newAPIHadoopFile("/Users/laserson/temp/test-parquet/alltypeuri",
>>> >>> classOf[AvroParquetInputFormat], classOf[Void],
>>> classOf[IndexedRecord],
>>> >>> sc.hadoopConfiguration)
>>> >>> records1.collect
>>> >>>
>>> >>> At which point it barfs:
>>> >>>
>>> >>> 14/02/05 12:02:32 INFO FileInputFormat: Total input paths to process
>>> : 3
>>> >>> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
>>> >>> SLF4J: Defaulting to no-operation (NOP) logger implementation
>>> >>> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for
>>> further
>>> >>> details.
>>> >>> java.io.IOException: Could not read footer:
>>> >>> java.lang.NoClassDefFoundError:
>>> >>> parquet/org/codehaus/jackson/JsonGenerationException
>>> >>> at
>>> >>>
>>> parquet.hadoop.ParquetFileReader.readAllFootersInParallel(ParquetFileReader.java:189)
>>> >>> at
>>> >>>
>>> parquet.hadoop.ParquetFileReader.readAllFootersInParallelUsingSummaryFiles(ParquetFileReader.java:145)
>>> >>> at
>>> >>>
>>> parquet.hadoop.ParquetInputFormat.getFooters(ParquetInputFormat.java:354)
>>> >>> at
>>> >>>
>>> parquet.hadoop.ParquetInputFormat.getFooters(ParquetInputFormat.java:339)
>>> >>> at
>>> >>>
>>> parquet.hadoop.ParquetInputFormat.getSplits(ParquetInputFormat.java:246)
>>> >>> at
>>> org.apache.spark.rdd.NewHadoopRDD.getPartitions(NewHadoopRDD.scala:85)
>>> >>> at
>>> org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:207)
>>> >>> at
>>> org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:205)
>>> >>> at scala.Option.getOrElse(Option.scala:120)
>>> >>> at org.apache.spark.rdd.RDD.partitions(RDD.scala:205)
>>> >>> at org.apache.spark.SparkContext.runJob(SparkContext.scala:863)
>>> >>> at org.apache.spark.rdd.RDD.collect(RDD.scala:602)
>>> >>> at $iwC$$iwC$$iwC$$iwC.<init>(<console>:20)
>>> >>> at $iwC$$iwC$$iwC.<init>(<console>:25)
>>> >>> at $iwC$$iwC.<init>(<console>:27)
>>> >>> at $iwC.<init>(<console>:29)
>>> >>> at <init>(<console>:31)
>>> >>> at .<init>(<console>:35)
>>> >>> at .<clinit>(<console>)
>>> >>> at .<init>(<console>:7)
>>> >>> at .<clinit>(<console>)
>>> >>> at $print(<console>)
>>> >>> 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.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:772)
>>> >>> at
>>> >>>
>>> org.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1040)
>>> >>> at
>>> org.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:609)
>>> >>> at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:640)
>>> >>> at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:604)
>>> >>> at
>>> >>>
>>> org.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:788)
>>> >>> at
>>> >>>
>>> org.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:833)
>>> >>> at org.apache.spark.repl.SparkILoop.command(SparkILoop.scala:745)
>>> >>> at
>>> org.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:593)
>>> >>> at org.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:600)
>>> >>> at org.apache.spark.repl.SparkILoop.loop(SparkILoop.scala:603)
>>> >>> at
>>> >>>
>>> org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply$mcZ$sp(SparkILoop.scala:926)
>>> >>> at
>>> >>>
>>> org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply(SparkILoop.scala:876)
>>> >>> at
>>> >>>
>>> org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply(SparkILoop.scala:876)
>>> >>> at
>>> >>>
>>> scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135)
>>> >>> at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:876)
>>> >>> at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:968)
>>> >>> at org.apache.spark.repl.Main$.main(Main.scala:31)
>>> >>> at org.apache.spark.repl.Main.main(Main.scala)
>>> >>> Caused by: java.lang.NoClassDefFoundError:
>>> >>> parquet/org/codehaus/jackson/JsonGenerationException
>>> >>> at
>>> >>>
>>> parquet.format.converter.ParquetMetadataConverter.fromParquetMetadata(ParquetMetadataConverter.java:359)
>>> >>> at
>>> >>>
>>> parquet.format.converter.ParquetMetadataConverter.readParquetMetadata(ParquetMetadataConverter.java:312)
>>> >>> at
>>> >>>
>>> parquet.hadoop.ParquetFileReader.readFooter(ParquetFileReader.java:295)
>>> >>> at
>>> parquet.hadoop.ParquetFileReader$2.call(ParquetFileReader.java:179)
>>> >>> at
>>> parquet.hadoop.ParquetFileReader$2.call(ParquetFileReader.java:175)
>>> >>> at java.util.concurrent.FutureTask.run(FutureTask.java:262)
>>> >>> 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:744)
>>> >>> Caused by: java.lang.ClassNotFoundException:
>>> >>> parquet.org.codehaus.jackson.JsonGenerationException
>>> >>> at java.net.URLClassLoader$1.run(URLClassLoader.java:366)
>>> >>> at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
>>> >>> at java.security.AccessController.doPrivileged(Native Method)
>>> >>> at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
>>> >>> at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
>>> >>> at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308)
>>> >>> at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
>>> >>> ... 9 more
>>> >>>
>>> >>>
>>> >>> --
>>> >>> Uri Laserson, PhD
>>> >>> Data Scientist, Cloudera
>>> >>> Twitter/GitHub: @laserson
>>> >>> +1 617 910 0447
>>> >>> [email protected]
>>> >>
>>> >>
>>> >
>>> >
>>> >
>>> > --
>>> > Uri Laserson, PhD
>>> > Data Scientist, Cloudera
>>> > Twitter/GitHub: @laserson
>>> > +1 617 910 0447
>>> > [email protected]
>>>
>>
>>
>>
>> --
>> Uri Laserson, PhD
>> Data Scientist, Cloudera
>> Twitter/GitHub: @laserson
>> +1 617 910 0447
>> [email protected]
>>
>
>
>
> --
> Uri Laserson, PhD
> Data Scientist, Cloudera
> Twitter/GitHub: @laserson
> +1 617 910 0447
> [email protected]
>
>
>  --
> http://parquet.github.com/
> ---
> You received this message because you are subscribed to the Google Groups
> "Parquet" group.
> To post to this group, send email to [email protected].
>



-- 
Uri Laserson, PhD
Data Scientist, Cloudera
Twitter/GitHub: @laserson
+1 617 910 0447
[email protected]

Reply via email to