flink-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From aljos...@apache.org
Subject [12/15] flink git commit: [hotfix] Fix compatibility check in RegisteredBackendStateMetaInfo
Date Tue, 20 Dec 2016 15:09:27 GMT
[hotfix] Fix compatibility check in RegisteredBackendStateMetaInfo

This was to strict. RocksDB initializes with null namespace Serializer
and when we have the actual namespace serializer the check fails.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d1eaa1ee
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d1eaa1ee
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d1eaa1ee

Branch: refs/heads/master
Commit: d1eaa1ee41728e6d788f1e914cb0568a874a6f32
Parents: 434013a
Author: Aljoscha Krettek <aljoscha.krettek@gmail.com>
Authored: Mon Dec 19 19:48:18 2016 +0100
Committer: Aljoscha Krettek <aljoscha.krettek@gmail.com>
Committed: Tue Dec 20 15:42:54 2016 +0100

----------------------------------------------------------------------
 .../state/RegisteredBackendStateMetaInfo.java   | 21 ++++++++++++++++----
 1 file changed, 17 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d1eaa1ee/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBackendStateMetaInfo.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBackendStateMetaInfo.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBackendStateMetaInfo.java
index 62418c3..0c50486 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBackendStateMetaInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBackendStateMetaInfo.java
@@ -20,7 +20,8 @@ package org.apache.flink.runtime.state;
 
 import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.util.Preconditions;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * Compound meta information for a registered state in a keyed state backend. This combines
all serializers and the
@@ -49,8 +50,8 @@ public class RegisteredBackendStateMetaInfo<N, S> {
 			TypeSerializer<N> namespaceSerializer,
 			TypeSerializer<S> stateSerializer) {
 
-		this.stateType = Preconditions.checkNotNull(stateType);
-		this.name = Preconditions.checkNotNull(name);
+		this.stateType = checkNotNull(stateType);
+		this.name = checkNotNull(name);
 		this.namespaceSerializer = namespaceSerializer;
 		this.stateSerializer = stateSerializer;
 	}
@@ -91,7 +92,9 @@ public class RegisteredBackendStateMetaInfo<N, S> {
 			return false;
 		}
 
-		return namespaceSerializer.isCompatibleWith(other.namespaceSerializer)
+		return ((namespaceSerializer == null && other.namespaceSerializer == null)
+					|| namespaceSerializer == null || other.namespaceSerializer == null
+					|| namespaceSerializer.isCompatibleWith(other.namespaceSerializer))
 				&& stateSerializer.isCompatibleWith(other.stateSerializer);
 	}
 
@@ -122,6 +125,16 @@ public class RegisteredBackendStateMetaInfo<N, S> {
 	}
 
 	@Override
+	public String toString() {
+		return "RegisteredBackendStateMetaInfo{" +
+				"stateType=" + stateType +
+				", name='" + name + '\'' +
+				", namespaceSerializer=" + namespaceSerializer +
+				", stateSerializer=" + stateSerializer +
+				'}';
+	}
+
+	@Override
 	public int hashCode() {
 		int result = getName().hashCode();
 		result = 31 * result + getStateType().hashCode();


Mime
View raw message