[
https://issues.apache.org/jira/browse/FLINK-6883?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16046565#comment-16046565
]
ASF GitHub Bot commented on FLINK-6883:
---------------------------------------
Github user tzulitai commented on a diff in the pull request:
https://github.com/apache/flink/pull/4103#discussion_r121393465
--- Diff:
flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompatibilityResult.java
---
@@ -60,10 +62,10 @@
*
* @return a result that signals migration is necessary, also providing
a convert deserializer.
*/
- public static <T> CompatibilityResult<T>
requiresMigration(TypeDeserializer<T> convertDeserializer) {
+ public static <T> CompatibilityResult<T> requiresMigration(@Nonnull
TypeDeserializer<T> convertDeserializer) {
Preconditions.checkNotNull(convertDeserializer, "Convert
deserializer cannot be null.");
- return new CompatibilityResult<>(true,
Preconditions.checkNotNull(convertDeserializer));
+ return new CompatibilityResult<>(true, convertDeserializer);
--- End diff --
I've removed this `Preconditions.checkNotNull` because it was already done
in the factory method (see L66). So this one is just redundant, the removal is
not related to the `Nonnull` annotation.
> Serializer for collection of Scala case classes are generated with different
> anonymous class names in 1.3
> ---------------------------------------------------------------------------------------------------------
>
> Key: FLINK-6883
> URL: https://issues.apache.org/jira/browse/FLINK-6883
> Project: Flink
> Issue Type: Bug
> Components: Scala API, Type Serialization System
> Affects Versions: 1.3.0
> Reporter: Tzu-Li (Gordon) Tai
> Assignee: Tzu-Li (Gordon) Tai
> Priority: Blocker
> Labels: flink-rel-1.3.1-blockers
> Fix For: 1.3.1
>
>
> In the Scala API, serializers are generated using Scala macros (via the
> {{org.apache.flink.streaming.api.scala.createTypeInformation(..)}} util).
> The generated serializers are inner anonymous classes, therefore classnames
> will differ depending on when / order that the serializers are generated.
> From 1.1 / 1.2 to Flink 1.3, the generated classnames for a serializer for a
> collections of case classes (e.g. {{List[SomeUserCaseClass]}}) will be
> different. In other words, the exact same user code written in the Scala API,
> compiling it with 1.1 / 1.2 and with 1.3 will result in different classnames.
> This is problematic for restoring older savepoints that have Scala case class
> collections in their state, because the old serializer cannot be recovered
> (due to the generated classname change).
> For now, I've managed to identify that the root cause for this is that in 1.3
> the {{TypeSerializer}} base class additionally extends the
> {{TypeDeserializer}} interface. Removing this extending resolves the problem.
> The actual reason for why this affects the generated classname is still being
> investigated.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)