[jira] [Commented] (BEAM-5164) ParquetIOIT fails on Spark and Flink

2019-08-14 Thread Lukasz Gajowy (JIRA)


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

Lukasz Gajowy commented on BEAM-5164:
-

I'm fine with adding the docs as you suggested. I left comments in your PR.

> 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
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> 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.(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.(Lorg/apache/parquet/io/OutputFile;)V{code}
>  
>  



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


[jira] [Commented] (BEAM-5164) ParquetIOIT fails on Spark and Flink

2019-08-14 Thread Ryan Skraba (JIRA)


[ 
https://issues.apache.org/jira/browse/BEAM-5164?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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.(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.(Lorg/apache/parquet/io/OutputFile;)V{code}
>  
>  



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


[jira] [Commented] (BEAM-5164) ParquetIOIT fails on Spark and Flink

2019-08-13 Thread Luke Cwik (JIRA)


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

Luke Cwik commented on BEAM-5164:
-

In this specific case I think we could shade the parquet library.

 

Your reasoning listed above is correct:

 _(1) we should shade to prevent transitive dependency collisions in runners 
when necessary, but (2) don't shade systematically by default "just in case", 
and (3) once a dependency has reached a certain threshold, like the extremely 
common guava and grpc jars, vendor them for reuse._

 

The downside to shading/vendoring is that it makes it more difficult for users 
to force a dependency version change without having the Apache Beam folks 
perform a release and also getting the shading/vendoring done correctly is 
quite annoying and very error prone. Vendoring requires two releases (the 
vendored artifact, and then core Beam projects that are updated to consume it) 
while shading only needs one but vendoring is much easier to reason 
about/builds faster/...

 

The best option is typically to try and get all parts aligned to use the same 
version but this is not possible always (such as in the case where you are 
trying to use multiple versions of Spark and Spark itself is incompatible with 
the newer version of a library) then your forced to shade/vendor.

 

 

> 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.(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.(Lorg/apache/parquet/io/OutputFile;)V{code}
>  
>  



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


[jira] [Commented] (BEAM-5164) ParquetIOIT fails on Spark and Flink

2019-08-13 Thread Ryan Skraba (JIRA)


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

Ryan Skraba commented on BEAM-5164:
---

I am not confident on the overall strategy with respect to shading/relocating 
or vendoring, so waiting for advice from [~lcwik] sounds good.  It seems like 
the right thing to do for older, supported versions of Spark.

I checked with the flink distributions on docker from 1.5 to 1.8 and there 
aren't existing parquet artifacts in the image, or inside 
{{flink-dist_2.11_1.X.X.jar}}, so I'm also unsure why the IT test failed on 
your remote cluster!

> 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.(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.(Lorg/apache/parquet/io/OutputFile;)V{code}
>  
>  



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


[jira] [Commented] (BEAM-5164) ParquetIOIT fails on Spark and Flink

2019-08-13 Thread Lukasz Gajowy (JIRA)


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

Lukasz Gajowy commented on BEAM-5164:
-

Do you know what is the situation on Flink runner? Should we still 
shadow/vendor this dependency in your opinion?

> 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.(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.(Lorg/apache/parquet/io/OutputFile;)V{code}
>  
>  



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


[jira] [Commented] (BEAM-5164) ParquetIOIT fails on Spark and Flink

2019-08-13 Thread Ryan Skraba (JIRA)


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

Ryan Skraba commented on BEAM-5164:
---

I checked with a spark local run on Spark 2.4.3, there is no issue (expected, 
since it includes the "right" parquet jars in the spark-supplied classpath).

The workaround that I proposed in the stack overflow question works for Spark 
2.2, 2.3 and 2.4, but is only a workaround... I added the following relocations 
to the "fat jar" build proposed in the Spark runner instructions:

```

  org.apache.parquet
  shaded.org.apache.parquet



  shaded.parquet
  reshaded.parquet


  org.apache.avro
  shaded.org.apache.avro

 ```

> 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.(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.(Lorg/apache/parquet/io/OutputFile;)V{code}
>  
>  



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


[jira] [Commented] (BEAM-5164) ParquetIOIT fails on Spark and Flink

2019-08-13 Thread Lukasz Gajowy (JIRA)


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

Lukasz Gajowy commented on BEAM-5164:
-

> Were you running with your own spark installation?

At the time of reporting this issue i used Spark 2.3.2 prebuilt from Hadoop 2.7 
as this was the latest supported version in beam back then. Downloaded from 
here: [https://spark.apache.org/downloads.html]. It is possible that this issue 
doesn't concern future versions.

> 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.(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.(Lorg/apache/parquet/io/OutputFile;)V{code}
>  
>  



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


[jira] [Commented] (BEAM-5164) ParquetIOIT fails on Spark and Flink

2019-08-13 Thread Ryan Skraba (JIRA)


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

Ryan Skraba commented on BEAM-5164:
---

Thanks for the link for the context!  Is it possible that de-shading of parquet 
was a mistake?

>From the discussion, it sounds like (1) we should shade to prevent transitive 
>dependency collisions in runners when necessary, but (2) don't shade 
>systematically by default "just in case", and (3) once a dependency has 
>reached a certain threshold, like the extremely common guava and grpc jars, 
>vendor them for reuse.

Is that about right?

Specifically for Spark, it looks like this is reported at least since 2.12.0 
for versions of Spark < 2.4 -- it looks like ParquetIOIT should be OK as-is 
with 2.4.3.  I couldn't find any references to older versions of spark in the 
code.  [~ŁukaszG] Were you running with your own spark installation?





> 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.(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.(Lorg/apache/parquet/io/OutputFile;)V{code}
>  
>  



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


[jira] [Commented] (BEAM-5164) ParquetIOIT fails on Spark and Flink

2019-08-12 Thread Lukasz Gajowy (JIRA)


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

Lukasz Gajowy commented on BEAM-5164:
-

I think so but I'm not very confident when it comes to these decisions. [After 
removing shading by 
default|https://lists.apache.org/thread.html/00202ff6a1e6eb7c2ce8c54cafd2afe7eb5524c63f29ed3bd695860e@%3Cdev.beam.apache.org%3E]
 and switching to vendoring some dependencies in general I'm not sure if we 
should just use shadow when declaring the dependency or vendor the jars in that 
case same as we vendor e.g. guava. Adding [~lcwik] so that he could share his 
opinion. 

> 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.(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.(Lorg/apache/parquet/io/OutputFile;)V{code}
>  
>  



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


[jira] [Commented] (BEAM-5164) ParquetIOIT fails on Spark and Flink

2019-08-09 Thread Ryan Skraba (JIRA)


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

Ryan Skraba commented on BEAM-5164:
---

See question and possible workarounds on [stack 
overflow|https://stackoverflow.com/questions/57395634/apache-beam-java-sdk-sparkrunner-write-to-parquet-error/57399273].

Parquet relies on some new methods in Parquet *1.10.x*, while Spark 2.2 and 
Spark 2.3 embed *Parquet 1.8.2*.   In this case, should the parquet jars be 
shaded/vendored to work with all supported versions of spark?

> 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.(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.(Lorg/apache/parquet/io/OutputFile;)V{code}
>  
>  



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