You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "Tao Li (Jira)" <ji...@apache.org> on 2021/01/30 01:06:00 UTC
[jira] [Comment Edited] (BEAM-11721) Cannot read array values with
ParquetIO
[ https://issues.apache.org/jira/browse/BEAM-11721?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17275432#comment-17275432 ]
Tao Li edited comment on BEAM-11721 at 1/30/21, 1:05 AM:
---------------------------------------------------------
Thanks [~chamikara].
cc [~aromanenko]
Just to share some more info regarding my test, the parquet files were created using spark 2.3 code. I am not sure if there is any compatibility issue between the parquet format and ParquetIO.
I also did an extensive testing including:
1) use spark 2.4 to create the parquet files and use the same beam code to read it. The error is the same.
2) I tried both avro 1.8.2 and 1.9.2 for my beam app. The error is the same.
3) Besides the direct runner I also tried spark runner. Error is the same:
{noformat}
java.lang.ClassCastException: shaded.org.apache.avro.generic.GenericData$Record cannot be cast to java.lang.Number
at shaded.org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion(GenericDatumWriter.java:130)
at shaded.org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:75)
at shaded.org.apache.avro.generic.GenericDatumWriter.writeArray(GenericDatumWriter.java:192)
at shaded.org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion(GenericDatumWriter.java:120)
at shaded.org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:75)
at shaded.org.apache.avro.generic.GenericDatumWriter.writeField(GenericDatumWriter.java:166)
at shaded.org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:156)
at shaded.org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion(GenericDatumWriter.java:118)
at shaded.org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:75)
at shaded.org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:62)
at org.apache.beam.sdk.coders.AvroCoder.encode(AvroCoder.java:317)
at org.apache.beam.sdk.coders.Coder.encode(Coder.java:136)
at org.apache.beam.sdk.coders.KvCoder.encode(KvCoder.java:73)
at org.apache.beam.sdk.coders.KvCoder.encode(KvCoder.java:37)
at org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder.encode(WindowedValue.java:591)
at org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder.encode(WindowedValue.java:582)
at org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder.encode(WindowedValue.java:542)
at org.apache.beam.runners.spark.coders.CoderHelpers.toByteArray(CoderHelpers.java:55)
at
{noformat}
was (Author: sekiforever):
Thanks [~chamikara].
BTW the parquet files were created using spark 2.3 code. I am not sure if there is any compatibility issue between the parquet format and ParquetIO. I also did an extensive testing by using spark 2.4 to create the parquet files. The error is the same. I also tried both avro 1.8.2 and 1.9.2 and I am seeing the same error regardless of the avro versions.
cc [~aromanenko]
> Cannot read array values with ParquetIO
> ---------------------------------------
>
> Key: BEAM-11721
> URL: https://issues.apache.org/jira/browse/BEAM-11721
> Project: Beam
> Issue Type: Bug
> Components: io-java-parquet
> Affects Versions: 2.25.0
> Reporter: Tao Li
> Priority: P0
> Attachments: beam-test-schema.avsc, beam-test.snappy.parquet
>
>
> Hi Beam community,
> I am seeing an error when reading an array field using ParquetIO. I was using beam 2.25. Both direct runner and spark runner testing is seeing this issue. This is a blocker issue to me for the beam adoption, so a prompt help would be appreciated.
> Below is the schema tree as a quick visualization. The array field name is "list" and the element type is int.
>
> root
> |-- list: array (nullable = true)
> | |-- element: integer (containsNull = true)
>
> The beam code is very simple: pipeline.apply(ParquetIO.read(avroSchema).from(parquetPath));
>
> Here is the error when running that code:
>
> {noformat}
> Exception in thread "main" org.apache.beam.sdk.Pipeline$PipelineExecutionException: java.lang.ClassCastException: org.apache.avro.generic.GenericData$Record cannot be cast to java.lang.Number
> at org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:353)
> at org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:321)
> at org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:216)
> at org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:67)
> at org.apache.beam.sdk.Pipeline.run(Pipeline.java:317)
> at org.apache.beam.sdk.Pipeline.run(Pipeline.java:303)
> Caused by: java.lang.ClassCastException: org.apache.avro.generic.GenericData$Record cannot be cast to java.lang.Number
> at org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion(GenericDatumWriter.java:156)
> at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:82)
> at org.apache.avro.generic.GenericDatumWriter.writeArray(GenericDatumWriter.java:234)
> at org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion(GenericDatumWriter.java:136)
> at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:82)
> at org.apache.avro.generic.GenericDatumWriter.writeField(GenericDatumWriter.java:206)
> at org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:195)
> at org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion(GenericDatumWriter.java:130)
> at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:82)
> at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:72)
> at org.apache.beam.sdk.coders.AvroCoder.encode(AvroCoder.java:317)
> at org.apache.beam.sdk.coders.Coder.encode(Coder.java:136)
> at org.apache.beam.sdk.util.CoderUtils.encodeToSafeStream(CoderUtils.java:82)
> at org.apache.beam.sdk.util.CoderUtils.encodeToByteArray(CoderUtils.java:66)
> at org.apache.beam.sdk.util.CoderUtils.encodeToByteArray(CoderUtils.java:51)
> at org.apache.beam.sdk.util.CoderUtils.clone(CoderUtils.java:141)
> at org.apache.beam.sdk.util.MutationDetectors$CodedValueMutationDetector.<init>(MutationDetectors.java:115)
> at org.apache.beam.sdk.util.MutationDetectors.forValueWithCoder(MutationDetectors.java:46)
> at org.apache.beam.runners.direct.ImmutabilityCheckingBundleFactory$ImmutabilityEnforcingBundle.add(ImmutabilityCheckingBundleFactory.java:112)
> at org.apache.beam.runners.direct.ParDoEvaluator$BundleOutputManager.output(ParDoEvaluator.java:301)
> at org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner.outputWindowedValue(SimpleDoFnRunner.java:267)
> at org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner.access$900(SimpleDoFnRunner.java:79)
> at org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner$DoFnProcessContext.output(SimpleDoFnRunner.java:413)
> at org.apache.beam.repackaged.direct_java.runners.core.SimpleDoFnRunner$DoFnProcessContext.output(SimpleDoFnRunner.java:401)
> at org.apache.beam.sdk.io.parquet.ParquetIO$ReadFiles$ReadFn.processElement(ParquetIO.java:646)
> {noformat}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)