You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2019/02/28 08:42:05 UTC

[GitHub] tzulitai commented on a change in pull request #7849: [FLINK-11772] [DataStream] InternalTimerServiceSerializationProxy should be using the new serialization compatibility abstractions

tzulitai commented on a change in pull request #7849: [FLINK-11772] [DataStream] InternalTimerServiceSerializationProxy should be using the new serialization compatibility abstractions
URL: https://github.com/apache/flink/pull/7849#discussion_r261092510
 
 

 ##########
 File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java
 ##########
 @@ -142,26 +141,31 @@ public void startTimerService(
 
 			// the following is the case where we restore
 			if (restoredTimersSnapshot != null) {
-				CompatibilityResult<K> keySerializerCompatibility = CompatibilityUtil.resolveCompatibilityResult(
-					this.keyDeserializer,
-					null,
-					restoredTimersSnapshot.getKeySerializerSnapshot(),
-					keySerializer);
-
-				CompatibilityResult<N> namespaceSerializerCompatibility = CompatibilityUtil.resolveCompatibilityResult(
-					this.namespaceDeserializer,
-					null,
-					restoredTimersSnapshot.getNamespaceSerializerSnapshot(),
-					namespaceSerializer);
-
-				if (keySerializerCompatibility.isRequiresMigration() || namespaceSerializerCompatibility.isRequiresMigration()) {
-					throw new IllegalStateException("Tried to initialize restored TimerService " +
-						"with incompatible serializers than those used to snapshot its state.");
+				TypeSerializerSchemaCompatibility<K> keySerializerCompatibility =
+					restoredTimersSnapshot.getKeySerializerSnapshot().resolveSchemaCompatibility(keySerializer);
+
+				if (keySerializerCompatibility.isIncompatible() || keySerializerCompatibility.isCompatibleAfterMigration()) {
+					throw new IllegalStateException(
+						"Tried to initialize restored TimerService with new key serializer that requires migration or is incompatible.");
 				}
+
+				TypeSerializerSchemaCompatibility<N> namespaceSerializerCompatibility =
+					restoredTimersSnapshot.getNamespaceSerializerSnapshot().resolveSchemaCompatibility(namespaceSerializer);
+
+				if (namespaceSerializerCompatibility.isIncompatible() || namespaceSerializerCompatibility.isCompatibleAfterMigration()) {
+					throw new IllegalStateException(
+						"Tried to initialize restored TimerService with new namespace serializer that requires migration or is incompatible.");
+				}
+
+				this.keySerializer = keySerializerCompatibility.isCompatibleAsIs()
+					? keySerializer : keySerializerCompatibility.getReconfiguredSerializer();
+				this.namespaceSerializer = keySerializerCompatibility.isCompatibleAsIs()
 
 Review comment:
   this should be `namespaceSerializerCompatibility`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services