Hi,

Thanks for all the information and reporting this.
We've identified this to be an actual issue:
https://issues.apache.org/jira/browse/FLINK-11436.

There's also a PR opened to fix this, and is currently under review:
https://github.com/apache/flink/pull/7580.
I'll make sure that this is fixed for the next bugfix release for 1.7.x
(i.e. 1.7.2).

For the time being if waiting for 1.7.2 isn't an option, to workaround this
for 1.7.1, you would have to have a copy of the AvroSerializer class, under
the same package namespace in your user code, but with serialVersionUID
changed to 1.
Also, you'll need to exclude Flink's original AvroSerializer class from
flink-avro.
You would be able to remove that workaround and extra AvroSerializer class
once you upgrade to 1.7.2.

Thanks,
Gordon

On Thu, Jan 24, 2019 at 8:37 PM pwestermann <no.westerm...@genesys.com>
wrote:

> I ran `mvn dependency:tree` and only see 1.7.1 dependencies for Flink:
>
> [INFO] com.inin.analytics:analytics-flink:jar:0.0.1-SNAPSHOT
> [INFO] +- org.apache.flink:flink-streaming-java_2.11:jar:1.7.1:provided
> [INFO] |  +- org.apache.flink:flink-runtime_2.11:jar:1.7.1:provided
> [INFO] |  |  +-
> org.apache.flink:flink-queryable-state-client-java_2.11:jar:1.7.1:provided
> [INFO] |  |  +- org.apache.flink:flink-hadoop-fs:jar:1.7.1:provided
> [INFO] |  |  +- commons-io:commons-io:jar:2.4:compile
> [INFO] |  |  +-
> org.apache.flink:flink-shaded-netty:jar:4.1.24.Final-5.0:provided
> [INFO] |  |  +- org.apache.flink:flink-shaded-asm:jar:5.0.4-5.0:provided
> [INFO] |  |  +-
> org.apache.flink:flink-shaded-jackson:jar:2.7.9-5.0:provided
> [INFO] |  |  +- org.javassist:javassist:jar:3.19.0-GA:provided
> [INFO] |  |  +- org.scala-lang:scala-library:jar:2.11.12:compile
> [INFO] |  |  +- com.typesafe.akka:akka-actor_2.11:jar:2.4.20:provided
> [INFO] |  |  |  +- com.typesafe:config:jar:1.3.0:provided
> [INFO] |  |  |  \-
> org.scala-lang.modules:scala-java8-compat_2.11:jar:0.7.0:provided
> [INFO] |  |  +- com.typesafe.akka:akka-stream_2.11:jar:2.4.20:provided
> [INFO] |  |  |  \- com.typesafe:ssl-config-core_2.11:jar:0.2.1:provided
> [INFO] |  |  |     \-
> org.scala-lang.modules:scala-parser-combinators_2.11:jar:1.0.4:provided
> [INFO] |  |  +- com.typesafe.akka:akka-protobuf_2.11:jar:2.4.20:provided
> [INFO] |  |  +- com.typesafe.akka:akka-slf4j_2.11:jar:2.4.20:provided
> [INFO] |  |  +- org.clapper:grizzled-slf4j_2.11:jar:1.3.2:provided
> [INFO] |  |  +- com.github.scopt:scopt_2.11:jar:3.5.0:provided
> [INFO] |  |  +- org.xerial.snappy:snappy-java:jar:1.1.4:compile
> [INFO] |  |  \- com.twitter:chill_2.11:jar:0.7.6:provided
> [INFO] |  |     \- com.twitter:chill-java:jar:0.7.6:provided
> [INFO] |  +- org.apache.flink:flink-shaded-guava:jar:18.0-5.0:provided
> [INFO] |  +- org.apache.commons:commons-math3:jar:3.5:compile
> [INFO] |  \- org.apache.flink:force-shading:jar:1.7.1:compile
> [INFO] +- org.apache.flink:flink-clients_2.11:jar:1.7.1:provided
> [INFO] |  +- org.apache.flink:flink-core:jar:1.7.1:provided
> [INFO] |  |  +- org.apache.flink:flink-annotations:jar:1.7.1:provided
> [INFO] |  |  +- org.apache.flink:flink-metrics-core:jar:1.7.1:provided
> [INFO] |  |  +- com.esotericsoftware.kryo:kryo:jar:2.24.0:provided
> [INFO] |  |  |  +- com.esotericsoftware.minlog:minlog:jar:1.2:provided
> [INFO] |  |  |  \- org.objenesis:objenesis:jar:2.1:provided
> [INFO] |  |  +- commons-collections:commons-collections:jar:3.2.2:provided
> [INFO] |  |  \- org.apache.commons:commons-compress:jar:1.4.1:compile
> [INFO] |  +- org.apache.flink:flink-optimizer_2.11:jar:1.7.1:provided
> [INFO] |  +- org.apache.flink:flink-java:jar:1.7.1:provided
> [INFO] |  \- commons-cli:commons-cli:jar:1.3.1:provided
> [INFO] +- org.apache.flink:flink-avro:jar:1.7.1:compile
> [INFO] |  \- org.apache.avro:avro:jar:1.8.2:compile
> [INFO] |     +- org.codehaus.jackson:jackson-core-asl:jar:1.9.13:compile
> [INFO] |     +- org.codehaus.jackson:jackson-mapper-asl:jar:1.9.13:compile
> [INFO] |     +- com.thoughtworks.paranamer:paranamer:jar:2.7:compile
> [INFO] |     \- org.tukaani:xz:jar:1.5:compile
> [INFO] +-
> org.apache.flink:flink-statebackend-rocksdb_2.11:jar:1.7.1:provided
> [INFO] |  \- org.rocksdb:rocksdbjni:jar:5.7.5:provided
> [INFO] +-
> org.apache.flink:flink-connector-kafka-0.11_2.11:jar:1.7.1:compile
> [INFO] |  +-
> org.apache.flink:flink-connector-kafka-0.10_2.11:jar:1.7.1:compile
> [INFO] |  |  \-
> org.apache.flink:flink-connector-kafka-0.9_2.11:jar:1.7.1:compile
> [INFO] |  |     \-
> org.apache.flink:flink-connector-kafka-base_2.11:jar:1.7.1:compile
> [INFO] |  \- org.apache.kafka:kafka-clients:jar:0.11.0.2:compile
> [INFO] |     \- net.jpountz.lz4:lz4:jar:1.3.0:compile
> [INFO] +- org.apache.flink:flink-s3-fs-presto:jar:1.7.1:provided
>
> I also tried this again with debug logging enabled but didn't see any more
> messages that would explain the failure.
> To me, the error message
> (org.apache.flink.formats.avro.typeutils.AvroSerializer; local class
> incompatible: stream classdesc serialVersionUID = 1, local class
> serialVersionUID = 2) looks like this is caused by only having the 1.7.1
> AvroSerializer class (serialVersionUID = 2) in the classpath but the
> savepoint requires the old one (serialVersionUID = 1).
>
>
>
> --
> Sent from:
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/
>

Reply via email to