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

Maximilian Michels commented on FLINK-3565:
-------------------------------------------

Problem: flink-parent contains the Scala version property which assumedly was 
substituted by the shading plugin. However, as state above, this is not always 
the case. This may result in the wrong Scala version being pulled in from 
flink-parent.

Two options for now:

1) Suffix all modules with the Scala version (because they all depend on 
flink-parent).
2) Remove the Scala version property and explicitly set the Scala version in 
all modules

I'd go with option 2. This would be transparent to the user. Plus, developers 
already had to use the {{change_scala_version}} script to set a different Scala 
version. This will stay the same, just the property will go.

> FlinkKafkaConsumer does not work with Scala 2.11 
> -------------------------------------------------
>
>                 Key: FLINK-3565
>                 URL: https://issues.apache.org/jira/browse/FLINK-3565
>             Project: Flink
>          Issue Type: Bug
>          Components: Streaming
>    Affects Versions: 1.0.0
>            Reporter: Aljoscha Krettek
>            Assignee: Maximilian Michels
>            Priority: Blocker
>
> Running a program built against Flink_2.11 dependencies fails on a Flink_2.11 
> cluster with this exception:
> java.lang.NoClassDefFoundError: scala/collection/GenTraversableOnce$class
>       at kafka.utils.Pool.<init>(Pool.scala:28)
>       at 
> kafka.consumer.FetchRequestAndResponseStatsRegistry$.<init>(FetchRequestAndResponseStats.scala:60)
>       at 
> kafka.consumer.FetchRequestAndResponseStatsRegistry$.<clinit>(FetchRequestAndResponseStats.scala)
>       at kafka.consumer.SimpleConsumer.<init>(SimpleConsumer.scala:39)
>       at kafka.javaapi.consumer.SimpleConsumer.<init>(SimpleConsumer.scala:34)
>       at 
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08.getPartitionsForTopic(FlinkKafkaConsumer08.java:518)
>       at 
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08.<init>(FlinkKafkaConsumer08.java:218)
>       at 
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08.<init>(FlinkKafkaConsumer08.java:193)
>       at 
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08.<init>(FlinkKafkaConsumer08.java:160)
>       at com.dataartisans.querywindow.WindowJob.main(WindowJob.java:93)
>       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.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:505)
>       at 
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:403)
>       at org.apache.flink.client.program.Client.runBlocking(Client.java:248)
>       at 
> org.apache.flink.client.CliFrontend.executeProgramBlocking(CliFrontend.java:866)
>       at org.apache.flink.client.CliFrontend.run(CliFrontend.java:333)
>       at 
> org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1189)
>       at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1239)
> Caused by: java.lang.ClassNotFoundException: 
> scala.collection.GenTraversableOnce$class
>       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 java.lang.ClassLoader.loadClass(ClassLoader.java:358)
>       ... 21 more



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to