flink-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From tzuli...@apache.org
Subject [4/4] flink git commit: [FLINK-6493] Fix ineffective null check in equality of RegisteredKeyedBackendStateMetaInfo snapshots
Date Fri, 04 Aug 2017 02:33:38 GMT
[FLINK-6493] Fix ineffective null check in equality of RegisteredKeyedBackendStateMetaInfo
snapshots

This closes #4328.
This closes #4408.


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

Branch: refs/heads/master
Commit: 875a1369c2ecfd721e3797f59f7ae29e7c522840
Parents: 2248103
Author: Tzu-Li (Gordon) Tai <tzulitai@apache.org>
Authored: Thu Aug 3 23:14:34 2017 +0800
Committer: Tzu-Li (Gordon) Tai <tzulitai@apache.org>
Committed: Thu Aug 3 23:14:34 2017 +0800

----------------------------------------------------------------------
 .../state/RegisteredKeyedBackendStateMetaInfo.java   | 15 ++++++---------
 .../RegisteredOperatorBackendStateMetaInfo.java      |  7 ++++---
 2 files changed, 10 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/875a1369/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyedBackendStateMetaInfo.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyedBackendStateMetaInfo.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyedBackendStateMetaInfo.java
index e1a7e06..342bc7f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyedBackendStateMetaInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyedBackendStateMetaInfo.java
@@ -23,6 +23,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.util.Preconditions;
 
+import java.util.Objects;
+
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
@@ -220,15 +222,10 @@ public class RegisteredKeyedBackendStateMetaInfo<N, S> {
 			}
 
 			// need to check for nulls because serializer and config snapshots may be null on restore
-			return
-				((getStateSerializer() == null && that.getStateSerializer() == null)
-					|| getStateSerializer().equals(that.getStateSerializer()))
-				&& ((getNamespaceSerializer() == null && that.getNamespaceSerializer()
== null)
-					|| getNamespaceSerializer().equals(that.getNamespaceSerializer()))
-				&& ((getNamespaceSerializerConfigSnapshot() == null && that.getNamespaceSerializerConfigSnapshot()
== null)
-					|| getNamespaceSerializerConfigSnapshot().equals(that.getNamespaceSerializerConfigSnapshot()))
-				&& ((getStateSerializerConfigSnapshot() == null && that.getStateSerializerConfigSnapshot()
== null)
-					|| getStateSerializerConfigSnapshot().equals(that.getStateSerializerConfigSnapshot()));
+			return Objects.equals(getStateSerializer(), that.getStateSerializer())
+				&& Objects.equals(getNamespaceSerializer(), that.getNamespaceSerializer())
+				&& Objects.equals(getNamespaceSerializerConfigSnapshot(), that.getNamespaceSerializerConfigSnapshot())
+				&& Objects.equals(getStateSerializerConfigSnapshot(), that.getStateSerializerConfigSnapshot());
 		}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/875a1369/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorBackendStateMetaInfo.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorBackendStateMetaInfo.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorBackendStateMetaInfo.java
index 7c56851..af289f9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorBackendStateMetaInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorBackendStateMetaInfo.java
@@ -18,11 +18,12 @@
 
 package org.apache.flink.runtime.state;
 
-import java.util.Objects;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.util.Preconditions;
 
+import java.util.Objects;
+
 /**
  * Compound meta information for a registered state in an operator state backend.
  * This contains the state name, assignment mode, and state partition serializer.
@@ -185,8 +186,8 @@ public class RegisteredOperatorBackendStateMetaInfo<S> {
 			// need to check for nulls because serializer and config snapshots may be null on restore
 			return name.equals(snapshot.getName())
 				&& assignmentMode.equals(snapshot.getAssignmentMode())
-				&& (Objects.equals(partitionStateSerializer, snapshot.getPartitionStateSerializer()))
-				&& (Objects.equals(partitionStateSerializerConfigSnapshot, snapshot.getPartitionStateSerializerConfigSnapshot()));
+				&& Objects.equals(partitionStateSerializer, snapshot.getPartitionStateSerializer())
+				&& Objects.equals(partitionStateSerializerConfigSnapshot, snapshot.getPartitionStateSerializerConfigSnapshot());
 		}
 
 		@Override


Mime
View raw message