[
https://issues.apache.org/jira/browse/FLINK-13910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17000590#comment-17000590
]
Yun Tang commented on FLINK-13910:
----------------------------------
[~dwysakowicz] [~twalthr], thanks for your suggestions. Since you have provided
concern of changing too many serializable classes without serialVersionUID, and
I agree to split this issue to several sub-tasks to follow up. Below is my
thoughts:
# Stop to fix this issue in Flink-1.10.0 but should fix in Flink-1.10.1 to
ensure the compatibility of 1.10.0 with 1.10.1.
# Analyze what modules would be effected. I have counted the number of
serializable classes without serialVersionUID for different modules at master
[5249249|https://github.com/apache/flink/commit/5249249d58814691fbd1fe93e56dc821e0c1f069]:
[^serializable-classes-without-uid-5249249]. I think we could then discussed
how many sub-tasks should we split.
By the way, there exist a quick way to detect which serializable classes
without serialVersionUID by leveraging intellij IDEA's future of [java
inspection
analyze|https://www.jetbrains.com/help/idea/list-of-java-inspections.html#serialization-issues].
And there also exist a way to add uid which equals to JVM's result by
following my PR's description with the help of my customized
[IDEA.|https://github.com/Myasuka/intellij-community/tree/add-SerialVersionUID]
What do you think of my proposal?
> Many serializable classes have no explicit 'serialVersionUID'
> -------------------------------------------------------------
>
> Key: FLINK-13910
> URL: https://issues.apache.org/jira/browse/FLINK-13910
> Project: Flink
> Issue Type: Bug
> Components: API / Type Serialization System
> Reporter: Yun Tang
> Assignee: Yun Tang
> Priority: Critical
> Labels: pull-request-available
> Fix For: 1.9.2, 1.10.0
>
> Attachments: SerializableNoSerialVersionUIDField,
> serializable-classes-without-uid-5249249
>
> Time Spent: 20m
> Remaining Estimate: 0h
>
> Currently, many serializable classes in Flink have no explicit
> 'serialVersionUID'. As [official
> doc|https://flink.apache.org/contributing/code-style-and-quality-java.html#java-serialization]
> said, {{Serializable classes must define a Serial Version UID}}.
> No 'serialVersionUID' would cause compatibility problem. Take
> {{TwoPhaseCommitSinkFunction}} for example, since no explicit
> 'serialVersionUID' defined, after
> [FLINK-10455|https://github.com/apache/flink/commit/489be82a6d93057ed4a3f9bf38ef50d01d11d96b]
> introduced, its default 'serialVersionUID' has changed from
> "4584405056408828651" to "4064406918549730832". In other words, if we submit
> a job from Flink-1.6.3 local home to remote Flink-1.6.2 cluster with the
> usage of {{TwoPhaseCommitSinkFunction}}, we would get exception like:
> {code:java}
> org.apache.flink.streaming.runtime.tasks.StreamTaskException: Cannot
> instantiate user function.
> at
> org.apache.flink.streaming.api.graph.StreamConfig.getStreamOperator(StreamConfig.java:239)
> at
> org.apache.flink.streaming.runtime.tasks.OperatorChain.<init>(OperatorChain.java:104)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:267)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:711)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: java.io.InvalidClassException:
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
> local class incompatible: stream classdesc serialVersionUID =
> 4584405056408828651, local class serialVersionUID = 4064406918549730832
> at java.io.ObjectStreamClass.initNonProxy(ObjectStreamClass.java:699)
> at
> java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1885)
> at
> java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1751)
> at
> java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1885)
> at
> java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1751)
> at
> java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2042)
> at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1573)
> at
> java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2287)
> at
> java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2211)
> at
> java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2069)
> at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1573)
> at java.io.ObjectInputStream.readObject(ObjectInputStream.java:431)
> at
> org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:537)
> at
> org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:524)
> at
> org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:512)
> at
> org.apache.flink.util.InstantiationUtil.readObjectFromConfig(InstantiationUtil.java:473)
> at
> org.apache.flink.streaming.api.graph.StreamConfig.getStreamOperator(StreamConfig.java:224)
> ... 4 more
> {code}
> Similar problems existed in
> {{org.apache.flink.streaming.api.operators.SimpleOperatorFactory}} which has
> different 'serialVersionUID' from release-1.9 and current master branch.
> IMO, we might have two options to fix this bug:
> # Add explicit serialVersionUID for those classes which is identical to
> latest Flink-1.9.0 release code.
> # Use similar mechanism like {{FailureTolerantObjectInputStream}} in
> {{InstantiationUtil}} to ignore serialVersionUID mismatch.
> I have collected all production classes without serialVersionUID from latest
> master branch in the attachment, which counts to 639 classes.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)