Trouble migrating state from 1.6.3 to 1.7.1

classic Classic list List threaded Threaded
11 messages Options
Reply | Threaded
Open this post in threaded view
|

Trouble migrating state from 1.6.3 to 1.7.1

pwestermann
This post was updated on .
I am trying to migrate from Flink 1.6.3 to 1.7.1 but am not able to restore
the job from a savepoint taken in 1.6.3.

We are using an AsyncFunction to publish Avro records to SQS. The state for
the AsyncWaitOperator cannot be restored because of serializer changes in
flink-avro from 1.6.3 to 1.7.1.

java.lang.Exception: Exception while creating StreamOperatorStateContext.
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:195)
	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:250)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:738)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:289)
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:704)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.flink.util.FlinkException: Could not restore operator state backend for AsyncWaitOperator_37b091deda97c699be10c0d72db7d5a5_(2/6) from any of the 1 provided restore options.
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:137)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.operatorStateBackend(StreamTaskStateInitializerImpl.java:245)
	at org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:143)
	... 5 more
Caused by: java.lang.IllegalStateException: Could not Java-deserialize TypeSerializer while restoring checkpoint metadata for serializer snapshot 'org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer$StreamElementSerializerConfigSnapshot'. Please update to the TypeSerializerSnapshot interface that removes Java Serialization to avoid this problem in the future.
	at org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot.restoreSerializer(TypeSerializerConfigSnapshot.java:138)
	at org.apache.flink.runtime.state.StateSerializerProvider$RestoredStateSerializerProvider.previousSchemaSerializer(StateSerializerProvider.java:212)
	at org.apache.flink.runtime.state.StateSerializerProvider$RestoredStateSerializerProvider.currentSchemaSerializer(StateSerializerProvider.java:188)
	at org.apache.flink.runtime.state.RegisteredOperatorStateBackendMetaInfo.getPartitionStateSerializer(RegisteredOperatorStateBackendMetaInfo.java:113)
	at org.apache.flink.runtime.state.DefaultOperatorStateBackend.restore(DefaultOperatorStateBackend.java:307)
	at org.apache.flink.runtime.state.DefaultOperatorStateBackend.restore(DefaultOperatorStateBackend.java:62)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.attemptCreateAndRestore(BackendRestorerProcedure.java:151)
	at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:123)
	... 7 more
Caused by: java.io.InvalidClassException: org.apache.flink.formats.avro.typeutils.AvroSerializer; local class incompatible: stream classdesc serialVersionUID = 1, local class serialVersionUID = 2
	at java.io.ObjectStreamClass.initNonProxy(ObjectStreamClass.java:699)
	at java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1884)


Any idea how to avoid this problem? Maybe start the job with flink-avro
1.6.3 or will that break other parts?

Thanks,
Peter



--
Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/
Reply | Threaded
Open this post in threaded view
|

Re: Trouble migrating state from 1.6.3 to 1.7.1

Tzu-Li (Gordon) Tai
Hi,

Thanks for reporting this.

Could you provide more details (error message, exception stack trace) that you are getting?
This is unexpected, as the changes to flink-avro serializers in 1.7.x should be backwards compatible.
More details on how the restore failed will be helpful here.

Cheers,
Gordon


On Wed, Jan 23, 2019 at 2:54 PM pwestermann <[hidden email]> wrote:
I am trying to migrate from Flink 1.6.3 to 1.7.1 but am not able to restore
the job from a savepoint taken in 1.6.3.

We are using an AsyncFunction to publish Avro records to SQS. The state for
the AsyncWaitOperator cannot be restored because of serializer changes in
flink-avro from 1.6.3 to 1.7.1.

Any idea how to avoid this problem? Maybe start the job with flink-avro
1.6.3 or will that break other parts?

Thanks,
Peter



--
Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/
Reply | Threaded
Open this post in threaded view
|

Re: Trouble migrating state from 1.6.3 to 1.7.1

pwestermann
This post was updated on .
There is not much in the log as this immediately happens when I start the
job. I attached one of the taskmanager logs. The first error message I see
is  "Could not read a requested serializer. Replaced with a
UnloadableDummyTypeSerializer."
<http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/file/t1547/taskmanager.log




--
Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/
Reply | Threaded
Open this post in threaded view
|

Re: Trouble migrating state from 1.6.3 to 1.7.1

Tzu-Li (Gordon) Tai
Thanks for the logs.

Is the job restore actually failing? If yes, there should be an exception for the exact cause of the failure.

Otherwise, the AvroSerializer warnings in the taskmanager logs is actually expected behaviour when restoring from savepoint versions before 1.7.x, and shouldn't cause job failures (unless something unexpected is happening).
Shortly put, to describe the cause of that warning:
Previously in 1.6.x, the AvroSerializer was Java-serialized into savepoints.
In 1.7.x, when restoring from previous version savepoints, that serializer will still be attempted to be read using Java serialization (which explains the InvalidClassException in the WARN log).
However, starting from 1.7 we no longer rely on serializers being written directly into savepoints, so whether or not reading that serializer was successful should not matter and the restore should proceed normally.

Please do let me know if the job is actually failing, then we should investigate further. If it is failing, there should be an exception in the JM logs identifying the cause of job failure.
CC'ing Igal, as he worked on the AvroSerializer for 1.7.x and might have more info.

Cheers,
Gordon

On Wed, Jan 23, 2019 at 7:42 PM pwestermann <[hidden email]> wrote:
There is not much in the log as this immediately happens when I start the
job. I attached one of the taskmanager logs. The first error message I see
is  /Could not read a requested serializer. Replaced with a
UnloadableDummyTypeSerializer./ and the exception is


taskmanager.log
<http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/file/t1547/taskmanager.log




--
Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/
Reply | Threaded
Open this post in threaded view
|

Re: Trouble migrating state from 1.6.3 to 1.7.1

pwestermann
This post was updated on .
Thanks Gordon,

I get the same exception in the JM logs and that looks like it's causing the
job failure.

{"t":"2019-01-23T12:03:00.399Z","message":"conversation-modeler -> (Map, Map, flow-observation-splitter, session-observation-splitter) (3/6) (3ad01aeac4fccaeaac8d974d8ab42c42) switched from DEPLOYING to RUNNING.","logger_name":"org.apache.flink.runtime.executiongraph.ExecutionGraph","level":"INFO"}
{"t":"2019-01-23T12:03:00.613Z","message":"observations-dynamo-sink -> Sink: Unnamed (6/6) (0c05c6b77cd8a46cbaa72ca23c33d289) switched from RUNNING to FAILED.","logger_name":"org.apache.flink.runtime.executiongraph.ExecutionGraph","level":"INFO","stack_trace":"j.i.InvalidClassException: org.apache.flink.formats.avro.typeutils.AvroSerializer; local class incompatible: stream classdesc serialVersionUID = 1, local class serialVersionUID = 2\n\tat j.i.ObjectStreamClass.initNonProxy(ObjectStreamClass.java:699)\n\tat j.i.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1884)\n\tat j.i.ObjectInputStream.readClassDesc(ObjectInputStream.java:1750)\n\tat j.i.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2041)\n\tat j.i.ObjectInputStream.readObject0(ObjectInputStream.java:1572)\n\tat j.i.ObjectInputStream.readArray(ObjectInputStream.java:1974)\n\tat j.i.ObjectInputStream.readObject0(ObjectInputStream.java:1566)\n\tat j.i.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2286)\n\tat j.i.ObjectInputStream.readSerialData(ObjectInputStream.java:2210)\n\tat j.i.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2068)\n\tat j.i.ObjectInputStream.readObject0(ObjectInputStream.java:1572)\n\tat j.i.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2286)\n\tat j.i.ObjectInputStream.readSerialData(ObjectInputStream.java:2210)\n\tat j.i.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2068)\n\tat j.i.ObjectInputStream.readObject0(ObjectInputStream.java:1572)\n\tat j.i.ObjectInputStream.readObject(ObjectInputStream.java:430)\n\tat o.a.f.a.c.t.TypeSerializerSerializationUtil$TypeSerializerSerializationProxy.read(TypeSerializerSerializationUtil.java:289)\n\tat o.a.f.a.c.t.TypeSerializerSerializationUtil.tryReadSerializer(TypeSerializerSerializationUtil.java:116)\n\tat o.a.f.a.c.t.TypeSerializerSerializationUtil.readSerializersAndConfigsWithResilience(TypeSerializerSerializationUtil.java:214)\n\tat o.a.f.r.s.m.StateMetaInfoSnapshotReadersWriters$V5ReaderImpl.readStateMetaInfoSnapshot(StateMetaInfoSnapshotReadersWriters.java:262)\n\tat o.a.f.r.s.OperatorBackendSerializationProxy.read(OperatorBackendSerializationProxy.java:119)\n\tat o.a.f.r.s.DefaultOperatorStateBackend.restore(DefaultOperatorStateBackend.java:296)\n\t... 10 common frames omitted\nWrapped by: j.l.IllegalStateExce..."}
{"t":"2019-01-23T12:03:00.616Z","message":"Job analytics-flink-v1 (aea6ba71bea6f35dd4d873cfe2e2d20b) switched from state RUNNING to FAILING.","logger_name":"org.apache.flink.runtime.executiongraph.ExecutionGraph","level":"INFO","stack_trace":"j.i.InvalidClassException: org.apache.flink.formats.avro.typeutils.AvroSerializer; local class incompatible: stream classdesc serialVersionUID = 1, local class serialVersionUID = 2\n\tat j.i.ObjectStreamClass.initNonProxy(ObjectStreamClass.java:699)\n\tat j.i.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1884)\n\tat j.i.ObjectInputStream.readClassDesc(ObjectInputStream.java:1750)\n\tat j.i.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2041)\n\tat j.i.ObjectInputStream.readObject0(ObjectInputStream.java:1572)\n\tat j.i.ObjectInputStream.readArray(ObjectInputStream.java:1974)\n\tat j.i.ObjectInputStream.readObject0(ObjectInputStream.java:1566)\n\tat j.i.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2286)\n\tat j.i.ObjectInputStream.readSerialData(ObjectInputStream.java:2210)\n\tat j.i.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2068)\n\tat j.i.ObjectInputStream.readObject0(ObjectInputStream.java:1572)\n\tat j.i.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2286)\n\tat j.i.ObjectInputStream.readSerialData(ObjectInputStream.java:2210)\n\tat j.i.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2068)\n\tat j.i.ObjectInputStream.readObject0(ObjectInputStream.java:1572)\n\tat j.i.ObjectInputStream.readObject(ObjectInputStream.java:430)\n\tat o.a.f.a.c.t.TypeSerializerSerializationUtil$TypeSerializerSerializationProxy.read(TypeSerializerSerializationUtil.java:289)\n\tat o.a.f.a.c.t.TypeSerializerSerializationUtil.tryReadSerializer(TypeSerializerSerializationUtil.java:116)\n\tat o.a.f.a.c.t.TypeSerializerSerializationUtil.readSerializersAndConfigsWithResilience(TypeSerializerSerializationUtil.java:214)\n\tat o.a.f.r.s.m.StateMetaInfoSnapshotReadersWriters$V5ReaderImpl.readStateMetaInfoSnapshot(StateMetaInfoSnapshotReadersWriters.java:262)\n\tat o.a.f.r.s.OperatorBackendSerializationProxy.read(OperatorBackendSerializationProxy.java:119)\n\tat o.a.f.r.s.DefaultOperatorStateBackend.restore(DefaultOperatorStateBackend.java:296)\n\t... 10 common frames omitted\nWrapped by: j.l.IllegalStateExce..."}




--
Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/
Reply | Threaded
Open this post in threaded view
|

Re: Trouble migrating state from 1.6.3 to 1.7.1

Tzu-Li (Gordon) Tai
Hi!

We've double checked the code, and the only plausible cause of this is that you may be using flink-avro 1.6.x with Flink 1.7.x.
Could you double check that all Flink dependencies, including flink-avro, are 1.7.1?
You can verify this by doing `mvn dependency:tree` on your job, and check that flink-avro 1.6.x isn't in there.

A more detailed explanation of why we suspect this:
In Flink 1.7.x, the job will only fail if a previous Java-serialized serializer, that couldn't be deserialized in the restore, was attempted to be used.
In flink-avro 1.7.x, we've made sure that the previous serialized AvroSerializer instance (which is expected to no longer be deserializable in 1.7.1) is never accessed. This isn't the case for flink-avro 1.6.x, which still attempts to access the serializer AvroSerializer instance.

Please update us on your verifications here. And thanks for the effort!

Cheers,
Gordon

On Wed, Jan 23, 2019 at 8:41 PM pwestermann <[hidden email]> wrote:
Thanks Gordon,

I get the same exception in the JM logs and that looks like it's causing the
job failure.





--
Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/
Reply | Threaded
Open this post in threaded view
|

Re: Trouble migrating state from 1.6.3 to 1.7.1

pwestermann
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 | Threaded
Open this post in threaded view
|

Re: Trouble migrating state from 1.6.3 to 1.7.1

Tzu-Li (Gordon) Tai
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 <[hidden email]> 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 | Threaded
Open this post in threaded view
|

Re: Trouble migrating state from 1.6.3 to 1.7.1

pwestermann
Hi Gordon,

We should be able to wait for 1.7.2 but I will also test the workaround and
post if I run into further issues.

Thanks a lot!
Peter



--
Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/
Reply | Threaded
Open this post in threaded view
|

Re: Trouble migrating state from 1.6.3 to 1.7.1

Tzu-Li (Gordon) Tai
Thanks Peter! Yes, it would also be great if you try the patch in  https://github.com/apache/flink/pull/7580 out and see if that works for you.

On Mon, Jan 28, 2019 at 7:47 PM pwestermann <[hidden email]> wrote:
Hi Gordon,

We should be able to wait for 1.7.2 but I will also test the workaround and
post if I run into further issues.

Thanks a lot!
Peter



--
Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/
Reply | Threaded
Open this post in threaded view
|

Re: Trouble migrating state from 1.6.3 to 1.7.1

pwestermann
Just using a copy of AvroSerializer with the serialVersionUID set to 1 did
not work. There was a NullPointerException on the next checkpoint, probably
because previousSchema doesn't exist in the old serializer.

However, the version from the PR with serialVersionUID set to 1 worked. (I
didn't want to have to change classes in flink-core, that's why I just
changed the serialVersionUID).

Thanks again Gordon!

Peter





--
Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/