[ https://issues.apache.org/jira/browse/CRUNCH-597?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Stan Rosenberg updated CRUNCH-597:
----------------------------------
Description:
Current version of parquet-hadoop results in the following stack trace while attempting to
read from parquet file,
{code}
java.lang.Exception: java.lang.ClassCastException: org.apache.hadoop.mapreduce.lib.input.FileSplit
cannot be cast to parquet.hadoop.ParquetInputSplit
at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:406)
Caused by: java.lang.ClassCastException: org.apache.hadoop.mapreduce.lib.input.FileSplit cannot
be cast to parquet.hadoop.ParquetInputSplit
at parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:107)
at org.apache.crunch.impl.mr.run.CrunchRecordReader.initialize(CrunchRecordReader.java:140)
at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.initialize(MapTask.java:478)
at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:671)
at org.apache.hadoop.mapred.MapTask.run(MapTask.java:330)
at org.apache.hadoop.mapred.LocalJobRunner$Job$MapTaskRunnable.run(LocalJobRunner.java:268)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
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:745)
{code}
Here is the relevant code snippet,
{code}
Pipeline pipeline = new MRPipeline(Crunch.class, conf);
PCollection<Pair<String, Observation>> observations =
pipeline.read(AvroParquetFileSource.builder(record).build(new Path(args[0])))
.parallelDo(new TranslateFn(), Avros.tableOf(Avros.strings(), Avros.specifics(Observation.class)));
for (Pair<String, Observation> pair : observations.materialize()) {
System.out.println(pair.second());
}
// observations.write(new AvroFileTarget(args[1]), Target.WriteMode.OVERWRITE);
// observations.write(new AvroPathPartitionTarget(
// new Path(args[1]),
// AvroPathPartitionTarget.AvroPathPartitionOutputFormat.class,
// SequentialFileNamingScheme.getInstance()),
// Target.WriteMode.OVERWRITE);
PipelineResult result = pipeline.done();
{code}
was:
Current version of parquet-hadoop results in the following stack trace while attempting to
read from parquet file,
{code}
java.lang.Exception: java.lang.ClassCastException: org.apache.hadoop.mapreduce.lib.input.FileSplit
cannot be cast to parquet.hadoop.ParquetInputSplit
at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:406)
Caused by: java.lang.ClassCastException: org.apache.hadoop.mapreduce.lib.input.FileSplit cannot
be cast to parquet.hadoop.ParquetInputSplit
at parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:107)
at org.apache.crunch.impl.mr.run.CrunchRecordReader.initialize(CrunchRecordReader.java:140)
at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.initialize(MapTask.java:478)
at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:671)
at org.apache.hadoop.mapred.MapTask.run(MapTask.java:330)
at org.apache.hadoop.mapred.LocalJobRunner$Job$MapTaskRunnable.run(LocalJobRunner.java:268)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
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:745)
{code}
> Unable to process parquet files using Hadoop
> --------------------------------------------
>
> Key: CRUNCH-597
> URL: https://issues.apache.org/jira/browse/CRUNCH-597
> Project: Crunch
> Issue Type: Bug
> Components: Core, IO
> Affects Versions: 0.13.0
> Reporter: Stan Rosenberg
> Assignee: Josh Wills
>
> Current version of parquet-hadoop results in the following stack trace while attempting
to read from parquet file,
> {code}
> java.lang.Exception: java.lang.ClassCastException: org.apache.hadoop.mapreduce.lib.input.FileSplit
cannot be cast to parquet.hadoop.ParquetInputSplit
> at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:406)
> Caused by: java.lang.ClassCastException: org.apache.hadoop.mapreduce.lib.input.FileSplit
cannot be cast to parquet.hadoop.ParquetInputSplit
> at parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:107)
> at org.apache.crunch.impl.mr.run.CrunchRecordReader.initialize(CrunchRecordReader.java:140)
> at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.initialize(MapTask.java:478)
> at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:671)
> at org.apache.hadoop.mapred.MapTask.run(MapTask.java:330)
> at org.apache.hadoop.mapred.LocalJobRunner$Job$MapTaskRunnable.run(LocalJobRunner.java:268)
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> 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:745)
> {code}
> Here is the relevant code snippet,
> {code}
> Pipeline pipeline = new MRPipeline(Crunch.class, conf);
> PCollection<Pair<String, Observation>> observations =
> pipeline.read(AvroParquetFileSource.builder(record).build(new Path(args[0])))
> .parallelDo(new TranslateFn(), Avros.tableOf(Avros.strings(),
Avros.specifics(Observation.class)));
> for (Pair<String, Observation> pair : observations.materialize()) {
> System.out.println(pair.second());
> }
> // observations.write(new AvroFileTarget(args[1]), Target.WriteMode.OVERWRITE);
> // observations.write(new AvroPathPartitionTarget(
> // new Path(args[1]),
> // AvroPathPartitionTarget.AvroPathPartitionOutputFormat.class,
> // SequentialFileNamingScheme.getInstance()),
> // Target.WriteMode.OVERWRITE);
> PipelineResult result = pipeline.done();
> {code}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)
|