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

Ryan Skraba commented on BEAM-5164:
-----------------------------------

OK -- one huge complication :/  Relocating the parquet library *also* requires 
relocating avro...  It uses an Avro method that is not API-compatible with 
avro-1.7.7 delivered with Spark 2.2 and 2.3 (specifically logical types in the 
Avro write support).

At the same time, pretty much anything using Avro with Beam SQL is also very 
likely to fail on avro-1.7.7 (therefore Spark 2.2 and 2.3) due to logical type 
conversions.

Amusingly enough, *AvroIO* looks like it's fine on Spark 2.2, 2.3.

How does this sound: instead of trying to correctly satisfy everyone with 
Parquet shading, I'll create the equivalent doc for the workaround (similar to 
what [hcatalog already 
does|https://beam.apache.org/documentation/io/built-in/hcatalog/]) and link it 
to the ParquetIO javadoc and built-in sources documentation.

In any case, this documentation should exist for users of Spark 2.2, 2.3 with 
Beam 2.12.0-2.15.0..

At the same time I can start investigating what it would take to vendor/shade 
Avro overall and find/create a JIRA for that.

What do you think?



> ParquetIOIT fails on Spark and Flink
> ------------------------------------
>
>                 Key: BEAM-5164
>                 URL: https://issues.apache.org/jira/browse/BEAM-5164
>             Project: Beam
>          Issue Type: Bug
>          Components: testing
>            Reporter: Lukasz Gajowy
>            Priority: Minor
>
> When run on Spark or Flink remote cluster, ParquetIOIT fails with the 
> following stacktrace: 
> {code:java}
> org.apache.beam.sdk.io.parquet.ParquetIOIT > writeThenReadAll FAILED
> org.apache.beam.sdk.Pipeline$PipelineExecutionException: 
> java.lang.NoSuchMethodError: 
> org.apache.parquet.hadoop.ParquetWriter$Builder.<init>(Lorg/apache/parquet/io/OutputFile;)V
> at 
> org.apache.beam.runners.spark.SparkPipelineResult.beamExceptionFrom(SparkPipelineResult.java:66)
> at 
> org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish(SparkPipelineResult.java:99)
> at 
> org.apache.beam.runners.spark.SparkPipelineResult.waitUntilFinish(SparkPipelineResult.java:87)
> at org.apache.beam.runners.spark.TestSparkRunner.run(TestSparkRunner.java:116)
> at org.apache.beam.runners.spark.TestSparkRunner.run(TestSparkRunner.java:61)
> at org.apache.beam.sdk.Pipeline.run(Pipeline.java:313)
> at org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:350)
> at org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:331)
> at 
> org.apache.beam.sdk.io.parquet.ParquetIOIT.writeThenReadAll(ParquetIOIT.java:133)
> Caused by:
> java.lang.NoSuchMethodError: 
> org.apache.parquet.hadoop.ParquetWriter$Builder.<init>(Lorg/apache/parquet/io/OutputFile;)V{code}
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to