You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tz...@apache.org on 2019/07/04 07:10:36 UTC

[flink] 05/11: [FLINK-12963] [state-processor] Introduce OperatorStateSpec wrapper class

This is an automated email from the ASF dual-hosted git repository.

tzulitai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 9f4e69fe25c9563a4e429e3aabc077c6a0962872
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
AuthorDate: Wed Jul 3 16:26:33 2019 +0800

    [FLINK-12963] [state-processor] Introduce OperatorStateSpec wrapper class
    
    This class is a simple container class to represent an operator state
    that is either still defined by a BootstrapTransformation, i.e. new
    state that has not been written out yet, or an existing OperatorState.
    
    Introducing this class improves readability of the code, instead of
    using Eithers and Tuples that would not have clear semantics for the
    user.
---
 .../apache/flink/state/api/WritableSavepoint.java  | 13 ++--
 .../api/runtime/BootstrapTransformationWithID.java | 49 +++++++++++++
 .../metadata/ModifiableSavepointMetadata.java      | 38 +++++-----
 .../api/runtime/metadata/OperatorStateSpec.java    | 83 ++++++++++++++++++++++
 4 files changed, 156 insertions(+), 27 deletions(-)

diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/WritableSavepoint.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/WritableSavepoint.java
index 0f68cce..512edeb 100644
--- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/WritableSavepoint.java
+++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/WritableSavepoint.java
@@ -19,13 +19,12 @@ package org.apache.flink.state.api;
 
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.checkpoint.OperatorState;
-import org.apache.flink.runtime.jobgraph.OperatorID;
 import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.state.api.output.MergeOperatorStates;
 import org.apache.flink.state.api.output.SavepointOutputFormat;
+import org.apache.flink.state.api.runtime.BootstrapTransformationWithID;
 import org.apache.flink.state.api.runtime.metadata.ModifiableSavepointMetadata;
 import org.apache.flink.util.Preconditions;
 
@@ -80,7 +79,7 @@ public abstract class WritableSavepoint<F extends WritableSavepoint> {
 	public final void write(String path) {
 		final Path savepointPath = new Path(path);
 
-		List<Tuple2<OperatorID, BootstrapTransformation<?>>> newOperatorTransformations = metadata.getNewOperatorTransformations();
+		List<BootstrapTransformationWithID<?>> newOperatorTransformations = metadata.getNewOperators();
 		DataSet<OperatorState> newOperatorStates = writeOperatorStates(newOperatorTransformations, savepointPath);
 
 		List<OperatorState> existingOperators = metadata.getExistingOperators();
@@ -109,11 +108,13 @@ public abstract class WritableSavepoint<F extends WritableSavepoint> {
 	}
 
 	private DataSet<OperatorState> writeOperatorStates(
-			List<Tuple2<OperatorID, BootstrapTransformation<?>>> newOperatorTransformations,
+			List<BootstrapTransformationWithID<?>> newOperatorStates,
 			Path savepointWritePath) {
-		return newOperatorTransformations
+		return newOperatorStates
 			.stream()
-			.map(transformation -> transformation.f1.writeOperatorState(transformation.f0, stateBackend, metadata, savepointWritePath))
+			.map(newOperatorState -> newOperatorState
+				.getBootstrapTransformation()
+				.writeOperatorState(newOperatorState.getOperatorID(), stateBackend, metadata, savepointWritePath))
 			.reduce(DataSet::union)
 			.orElseThrow(() -> new IllegalStateException("Savepoint's must contain at least one operator"));
 	}
diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/BootstrapTransformationWithID.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/BootstrapTransformationWithID.java
new file mode 100644
index 0000000..cb5b5af
--- /dev/null
+++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/BootstrapTransformationWithID.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.state.api.runtime;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.state.api.BootstrapTransformation;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * A simple container class that represents a newly bootstrapped operator state within savepoints.
+ * It wraps the target {@link OperatorID} for the bootstrapped operator, as well as the {@link BootstrapTransformation}
+ * that defines how the state is bootstrapped.
+ */
+@Internal
+public class BootstrapTransformationWithID<T> {
+
+	private final OperatorID operatorID;
+	private final BootstrapTransformation<T> bootstrapTransformation;
+
+	public BootstrapTransformationWithID(OperatorID operatorID, BootstrapTransformation<T> bootstrapTransformation) {
+		this.operatorID = Preconditions.checkNotNull(operatorID);
+		this.bootstrapTransformation = Preconditions.checkNotNull(bootstrapTransformation);
+	}
+
+	public OperatorID getOperatorID() {
+		return operatorID;
+	}
+
+	public BootstrapTransformation<T> getBootstrapTransformation() {
+		return bootstrapTransformation;
+	}
+}
diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/metadata/ModifiableSavepointMetadata.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/metadata/ModifiableSavepointMetadata.java
index 8df8723..3484a59 100644
--- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/metadata/ModifiableSavepointMetadata.java
+++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/metadata/ModifiableSavepointMetadata.java
@@ -1,13 +1,12 @@
 package org.apache.flink.state.api.runtime.metadata;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.checkpoint.MasterState;
 import org.apache.flink.runtime.checkpoint.OperatorState;
 import org.apache.flink.runtime.jobgraph.OperatorID;
 import org.apache.flink.state.api.BootstrapTransformation;
+import org.apache.flink.state.api.runtime.BootstrapTransformationWithID;
 import org.apache.flink.state.api.runtime.OperatorIDGenerator;
-import org.apache.flink.types.Either;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -15,7 +14,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
-import java.util.stream.Stream;
 
 /**
  * Savepoint metadata that can be modified.
@@ -23,16 +21,15 @@ import java.util.stream.Stream;
 @Internal
 public class ModifiableSavepointMetadata extends SavepointMetadata {
 
-	private transient Map<OperatorID, Either<OperatorState, BootstrapTransformation<?>>> operatorStateIndex;
+	private transient Map<OperatorID, OperatorStateSpec> operatorStateIndex;
 
 	public ModifiableSavepointMetadata(int maxParallelism, Collection<MasterState> masterStates, Collection<OperatorState> initialStates) {
 		super(maxParallelism, masterStates);
 
 		this.operatorStateIndex = new HashMap<>(initialStates.size());
-
-		for (OperatorState operatorState : initialStates) {
-			operatorStateIndex.put(operatorState.getOperatorID(), Either.Left(operatorState));
-		}
+		initialStates.forEach(existingState -> operatorStateIndex.put(
+			existingState.getOperatorID(),
+			OperatorStateSpec.existing(existingState)));
 	}
 
 	/**
@@ -43,12 +40,12 @@ public class ModifiableSavepointMetadata extends SavepointMetadata {
 	public OperatorState getOperatorState(String uid) throws IOException {
 		OperatorID operatorID = OperatorIDGenerator.fromUid(uid);
 
-		Either<OperatorState, BootstrapTransformation<?>> operatorState = operatorStateIndex.get(operatorID);
-		if (operatorState == null || operatorState.isRight()) {
+		OperatorStateSpec operatorState = operatorStateIndex.get(operatorID);
+		if (operatorState == null || operatorState.isNewStateTransformation()) {
 			throw new IOException("Savepoint does not contain state with operator uid " + uid);
 		}
 
-		return operatorState.left();
+		return operatorState.asExistingState();
 	}
 
 	public void removeOperator(String uid) {
@@ -62,7 +59,7 @@ public class ModifiableSavepointMetadata extends SavepointMetadata {
 			throw new IllegalArgumentException("The savepoint already contains uid " + uid + ". All uid's must be unique");
 		}
 
-		operatorStateIndex.put(id, Either.Right(transformation));
+		operatorStateIndex.put(id, OperatorStateSpec.newWithTransformation(new BootstrapTransformationWithID<>(id, transformation)));
 	}
 
 	/**
@@ -72,21 +69,20 @@ public class ModifiableSavepointMetadata extends SavepointMetadata {
 		return operatorStateIndex
 			.values()
 			.stream()
-			.filter(Either::isLeft)
-			.map(Either::left)
+			.filter(OperatorStateSpec::isExistingState)
+			.map(OperatorStateSpec::asExistingState)
 			.collect(Collectors.toList());
 	}
 
 	/**
 	 * @return List of new operator states for the savepoint, represented by their target {@link OperatorID} and {@link BootstrapTransformation}.
 	 */
-	public List<Tuple2<OperatorID, BootstrapTransformation<?>>> getNewOperatorTransformations() {
-		Stream<Tuple2<OperatorID, BootstrapTransformation<?>>> transformations = operatorStateIndex
-			.entrySet()
+	public List<BootstrapTransformationWithID<?>> getNewOperators() {
+		return operatorStateIndex
+			.values()
 			.stream()
-			.filter(entry -> entry.getValue().isRight())
-			.map(entry -> Tuple2.of(entry.getKey(), entry.getValue().right()));
-
-		return transformations.collect(Collectors.toList());
+			.filter(OperatorStateSpec::isNewStateTransformation)
+			.map(OperatorStateSpec::asNewStateTransformation)
+			.collect(Collectors.toList());
 	}
 }
diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/metadata/OperatorStateSpec.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/metadata/OperatorStateSpec.java
new file mode 100644
index 0000000..cb87e4c
--- /dev/null
+++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/metadata/OperatorStateSpec.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.state.api.runtime.metadata;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.OperatorState;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.state.api.runtime.BootstrapTransformationWithID;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+/**
+ * This class specifies an operator state maintained by {@link SavepointMetadata}.
+ * An operator state is either represented as an existing {@link OperatorState}, or a
+ * {@link org.apache.flink.state.api.BootstrapTransformation} that will be used to create it.
+ */
+@Internal
+class OperatorStateSpec {
+
+	private final OperatorID id;
+
+	@Nullable
+	private final OperatorState existingState;
+
+	@Nullable
+	private final BootstrapTransformationWithID<?> newOperatorStateTransformation;
+
+	static OperatorStateSpec existing(OperatorState existingState) {
+		return new OperatorStateSpec(Preconditions.checkNotNull(existingState));
+	}
+
+	static OperatorStateSpec newWithTransformation(BootstrapTransformationWithID<?> transformation) {
+		return new OperatorStateSpec(Preconditions.checkNotNull(transformation));
+	}
+
+	private OperatorStateSpec(OperatorState existingState) {
+		this.id = existingState.getOperatorID();
+		this.existingState = existingState;
+		this.newOperatorStateTransformation = null;
+	}
+
+	private OperatorStateSpec(BootstrapTransformationWithID<?> transformation) {
+		this.id = transformation.getOperatorID();
+		this.newOperatorStateTransformation = transformation;
+		this.existingState = null;
+	}
+
+	boolean isExistingState() {
+		return existingState != null;
+	}
+
+	boolean isNewStateTransformation() {
+		return !isExistingState();
+	}
+
+	OperatorState asExistingState() {
+		Preconditions.checkState(isExistingState(), "OperatorState %s is not an existing state.", id);
+		return existingState;
+	}
+
+	@SuppressWarnings("unchecked")
+	<T> BootstrapTransformationWithID<T> asNewStateTransformation() {
+		Preconditions.checkState(isNewStateTransformation(), "OperatorState %s is not a new state defined with BootstrapTransformation", id);
+		return (BootstrapTransformationWithID<T>) newOperatorStateTransformation;
+	}
+}