You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2020/05/16 16:27:50 UTC
[flink] 01/13: [FLINK-17674][state] Type OperatorCoordinator state
in checkpoints strictly to ByteStreamStateHandle
This is an automated email from the ASF dual-hosted git repository.
sewen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
commit e3413106ccd030e44d5e4690430c841370a1f988
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Wed May 13 13:31:58 2020 +0200
[FLINK-17674][state] Type OperatorCoordinator state in checkpoints strictly to ByteStreamStateHandle
State restore on the master side happens in the JobManager's main thread and must hence not do any
potentially blocking I/O operations.
Typing the state to ByteStreamStateHandle makes sure that we can retrieve the data directly without I/O
as a strict contract.
If state restoring becomes an asynchronous operation we can relax this restriction.
---
.../checkpoint/OperatorCoordinatorCheckpoints.java | 9 ++----
.../flink/runtime/checkpoint/OperatorState.java | 8 +++---
.../runtime/checkpoint/PendingCheckpoint.java | 4 +--
.../metadata/MetadataV2V3SerializerBase.java | 14 ++++++++++
.../checkpoint/metadata/MetadataV3Serializer.java | 2 +-
.../runtime/checkpoint/PendingCheckpointTest.java | 4 +--
.../checkpoint/metadata/CheckpointTestUtils.java | 8 ++++--
.../runtime/state/TestingStreamStateHandle.java | 32 ++++------------------
8 files changed, 36 insertions(+), 45 deletions(-)
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpoints.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpoints.java
index cfacec8..2423bcb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpoints.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorCoordinatorCheckpoints.java
@@ -20,7 +20,6 @@ package org.apache.flink.runtime.checkpoint;
import org.apache.flink.runtime.concurrent.FutureUtils;
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
-import org.apache.flink.runtime.state.StreamStateHandle;
import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
import java.util.ArrayList;
@@ -130,13 +129,9 @@ final class OperatorCoordinatorCheckpoints {
static final class CoordinatorSnapshot {
final OperatorCoordinatorCheckpointContext coordinator;
- final StreamStateHandle state;
-
- CoordinatorSnapshot(OperatorCoordinatorCheckpointContext coordinator, StreamStateHandle state) {
- // if this is not true any more, we need more elaborate dispose/cleanup handling
- // see comment above the class.
- assert state instanceof ByteStreamStateHandle;
+ final ByteStreamStateHandle state;
+ CoordinatorSnapshot(OperatorCoordinatorCheckpointContext coordinator, ByteStreamStateHandle state) {
this.coordinator = coordinator;
this.state = state;
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java
index 998a3bd..a4994c6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.checkpoint;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.state.CompositeStateHandle;
import org.apache.flink.runtime.state.SharedStateRegistry;
-import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
import org.apache.flink.util.Preconditions;
import javax.annotation.Nullable;
@@ -50,7 +50,7 @@ public class OperatorState implements CompositeStateHandle {
/** The state of the operator coordinator. Null, if no such state exists. */
@Nullable
- private StreamStateHandle coordinatorState;
+ private ByteStreamStateHandle coordinatorState;
/** The parallelism of the operator when it was checkpointed. */
private final int parallelism;
@@ -96,13 +96,13 @@ public class OperatorState implements CompositeStateHandle {
}
}
- public void setCoordinatorState(@Nullable StreamStateHandle coordinatorState) {
+ public void setCoordinatorState(@Nullable ByteStreamStateHandle coordinatorState) {
checkState(this.coordinatorState == null, "coordinator state already set");
this.coordinatorState = coordinatorState;
}
@Nullable
- public StreamStateHandle getCoordinatorState() {
+ public ByteStreamStateHandle getCoordinatorState() {
return coordinatorState;
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
index c69c1cc..2a0eba7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java
@@ -28,7 +28,7 @@ import org.apache.flink.runtime.state.CheckpointMetadataOutputStream;
import org.apache.flink.runtime.state.CheckpointStorageLocation;
import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation;
import org.apache.flink.runtime.state.StateUtil;
-import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.Preconditions;
@@ -432,7 +432,7 @@ public class PendingCheckpoint {
public TaskAcknowledgeResult acknowledgeCoordinatorState(
OperatorCoordinatorCheckpointContext coordinatorInfo,
- @Nullable StreamStateHandle stateHandle) {
+ @Nullable ByteStreamStateHandle stateHandle) {
synchronized (lock) {
if (discarded) {
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java
index fe8343a..44ad464 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java
@@ -468,6 +468,7 @@ public abstract class MetadataV2V3SerializerBase {
dos.flush();
}
+ @Nullable
static StreamStateHandle deserializeStreamStateHandle(
DataInputStream dis,
@Nullable DeserializationContext context) throws IOException {
@@ -498,6 +499,19 @@ public abstract class MetadataV2V3SerializerBase {
}
}
+ @Nullable
+ static ByteStreamStateHandle deserializeAndCheckByteStreamStateHandle(
+ DataInputStream dis,
+ @Nullable DeserializationContext context) throws IOException {
+
+ final StreamStateHandle handle = deserializeStreamStateHandle(dis, context);
+ if (handle == null || handle instanceof ByteStreamStateHandle) {
+ return (ByteStreamStateHandle) handle;
+ } else {
+ throw new IOException("Expected a ByteStreamStateHandle but found a " + handle.getClass().getName());
+ }
+ }
+
// ------------------------------------------------------------------------
// utilities
// ------------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java
index 6bf98c9..1563abb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV3Serializer.java
@@ -122,7 +122,7 @@ public class MetadataV3Serializer extends MetadataV2V3SerializerBase implements
final OperatorState operatorState = new OperatorState(jobVertexId, parallelism, maxParallelism);
// Coordinator state
- operatorState.setCoordinatorState(deserializeStreamStateHandle(dis, context));
+ operatorState.setCoordinatorState(deserializeAndCheckByteStreamStateHandle(dis, context));
// Sub task states
final int numSubTaskStates = dis.readInt();
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
index d5cec9b..ee292b0 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java
@@ -35,9 +35,9 @@ import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinator;
import org.apache.flink.runtime.state.CheckpointStorageLocationReference;
import org.apache.flink.runtime.state.SharedStateRegistry;
-import org.apache.flink.runtime.state.StreamStateHandle;
import org.apache.flink.runtime.state.TestingStreamStateHandle;
import org.apache.flink.runtime.state.filesystem.FsCheckpointStorageLocation;
+import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
import org.hamcrest.Matchers;
import org.junit.Assert;
@@ -519,7 +519,7 @@ public class PendingCheckpointTest {
return checkpoint;
}
- private PendingCheckpoint createPendingCheckpointWithAcknowledgedCoordinators(StreamStateHandle... handles) throws IOException {
+ private PendingCheckpoint createPendingCheckpointWithAcknowledgedCoordinators(ByteStreamStateHandle... handles) throws IOException {
OperatorCoordinatorCheckpointContext[] coords = new OperatorCoordinatorCheckpointContext[handles.length];
for (int i = 0; i < handles.length; i++) {
coords[i] = createOperatorCoordinator();
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/CheckpointTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/CheckpointTestUtils.java
index 64cfafd..be4ee3f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/CheckpointTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/metadata/CheckpointTestUtils.java
@@ -84,7 +84,7 @@ public class CheckpointTestUtils {
final boolean hasCoordinatorState = random.nextBoolean();
if (hasCoordinatorState) {
- final StreamStateHandle stateHandle = createDummyStreamStateHandle(random, basePath);
+ final ByteStreamStateHandle stateHandle = createDummyByteStreamStreamStateHandle(random);
taskState.setCoordinatorState(stateHandle);
}
@@ -222,7 +222,11 @@ public class CheckpointTestUtils {
createDummyStreamStateHandle(rnd, basePath));
}
- public static StreamStateHandle createDummyStreamStateHandle(Random rnd, String basePath) {
+ public static ByteStreamStateHandle createDummyByteStreamStreamStateHandle(Random rnd) {
+ return (ByteStreamStateHandle) createDummyStreamStateHandle(rnd, null);
+ }
+
+ public static StreamStateHandle createDummyStreamStateHandle(Random rnd, @Nullable String basePath) {
if (!isSavepoint(basePath)) {
return new ByteStreamStateHandle(
String.valueOf(createRandomUUID(rnd)),
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestingStreamStateHandle.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestingStreamStateHandle.java
index 7a968d2..2471fd4 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestingStreamStateHandle.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestingStreamStateHandle.java
@@ -18,52 +18,30 @@
package org.apache.flink.runtime.state;
-import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
-import javax.annotation.Nullable;
+import java.util.UUID;
/**
* A simple test mock for a {@link StreamStateHandle}.
*/
-public class TestingStreamStateHandle implements StreamStateHandle {
+public class TestingStreamStateHandle extends ByteStreamStateHandle {
private static final long serialVersionUID = 1L;
- @Nullable
- private final FSDataInputStream inputStream;
-
- private final long size;
-
private boolean disposed;
public TestingStreamStateHandle() {
- this(null, 0L);
- }
-
- public TestingStreamStateHandle(@Nullable FSDataInputStream inputStream, long size) {
- this.inputStream = inputStream;
- this.size = size;
+ super(UUID.randomUUID().toString(), new byte[0]);
}
// ------------------------------------------------------------------------
@Override
- public FSDataInputStream openInputStream() {
- if (inputStream == null) {
- throw new UnsupportedOperationException("no input stream provided");
- }
- return inputStream;
- }
-
- @Override
public void discardState() {
+ super.discardState();
disposed = true;
}
- @Override
- public long getStateSize() {
- return size;
- }
-
// ------------------------------------------------------------------------
public boolean isDisposed() {