[ https://issues.apache.org/jira/browse/FLINK-10490?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16674854#comment-16674854 ]
ASF GitHub Bot commented on FLINK-10490: ---------------------------------------- asfgit closed pull request #6910: [FLINK-10490][tests] OperatorSnapshotUtil should use SavepointV2Seria… URL: https://github.com/apache/flink/pull/6910 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java index faee588c48d..fa8407729d9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.checkpoint.savepoint; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.checkpoint.MasterState; import org.apache.flink.runtime.checkpoint.OperatorState; @@ -67,7 +69,9 @@ * +--------------+---------------------+---------+------+---------------+ * </pre> */ -class SavepointV2Serializer implements SavepointSerializer<SavepointV2> { +@Internal +@VisibleForTesting +public class SavepointV2Serializer implements SavepointSerializer<SavepointV2> { /** Random magic number for consistency checks */ private static final int MASTER_STATE_MAGIC_NUMBER = 0xc96b1696; @@ -320,7 +324,8 @@ private static OperatorSubtaskState deserializeSubtaskState(DataInputStream dis) keyedStateStream); } - private static void serializeKeyedStateHandle( + @VisibleForTesting + public static void serializeKeyedStateHandle( KeyedStateHandle stateHandle, DataOutputStream dos) throws IOException { if (stateHandle == null) { @@ -380,7 +385,8 @@ private static void serializeStreamStateHandleMap( return result; } - private static KeyedStateHandle deserializeKeyedStateHandle(DataInputStream dis) throws IOException { + @VisibleForTesting + public static KeyedStateHandle deserializeKeyedStateHandle(DataInputStream dis) throws IOException { final int type = dis.readByte(); if (NULL_HANDLE == type) { @@ -433,7 +439,8 @@ private static KeyedStateHandle deserializeKeyedStateHandle(DataInputStream dis) } } - private static void serializeOperatorStateHandle( + @VisibleForTesting + public static void serializeOperatorStateHandle( OperatorStateHandle stateHandle, DataOutputStream dos) throws IOException { if (stateHandle != null) { @@ -461,7 +468,8 @@ private static void serializeOperatorStateHandle( } } - private static OperatorStateHandle deserializeOperatorStateHandle( + @VisibleForTesting + public static OperatorStateHandle deserializeOperatorStateHandle( DataInputStream dis) throws IOException { final int type = dis.readByte(); @@ -492,7 +500,8 @@ private static OperatorStateHandle deserializeOperatorStateHandle( } } - private static void serializeStreamStateHandle( + @VisibleForTesting + public static void serializeStreamStateHandle( StreamStateHandle stateHandle, DataOutputStream dos) throws IOException { if (stateHandle == null) { @@ -518,7 +527,7 @@ private static void serializeStreamStateHandle( dos.flush(); } - private static StreamStateHandle deserializeStreamStateHandle(DataInputStream dis) throws IOException { + public static StreamStateHandle deserializeStreamStateHandle(DataInputStream dis) throws IOException { final int type = dis.read(); if (NULL_HANDLE == type) { return null; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OperatorSnapshotUtil.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OperatorSnapshotUtil.java index 1b5113da2cb..53627d5ad49 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OperatorSnapshotUtil.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OperatorSnapshotUtil.java @@ -20,7 +20,7 @@ import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.checkpoint.StateObjectCollection; -import org.apache.flink.runtime.checkpoint.savepoint.SavepointV1Serializer; +import org.apache.flink.runtime.checkpoint.savepoint.SavepointV2Serializer; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; @@ -55,13 +55,13 @@ public static void writeStateHandle(OperatorSubtaskState state, String path) thr dos.writeInt(0); // still required for compatibility - SavepointV1Serializer.serializeStreamStateHandle(null, dos); + SavepointV2Serializer.serializeStreamStateHandle(null, dos); Collection<OperatorStateHandle> rawOperatorState = state.getRawOperatorState(); if (rawOperatorState != null) { dos.writeInt(rawOperatorState.size()); for (OperatorStateHandle operatorStateHandle : rawOperatorState) { - SavepointV1Serializer.serializeOperatorStateHandle(operatorStateHandle, dos); + SavepointV2Serializer.serializeOperatorStateHandle(operatorStateHandle, dos); } } else { // this means no states, not even an empty list @@ -72,7 +72,7 @@ public static void writeStateHandle(OperatorSubtaskState state, String path) thr if (managedOperatorState != null) { dos.writeInt(managedOperatorState.size()); for (OperatorStateHandle operatorStateHandle : managedOperatorState) { - SavepointV1Serializer.serializeOperatorStateHandle(operatorStateHandle, dos); + SavepointV2Serializer.serializeOperatorStateHandle(operatorStateHandle, dos); } } else { // this means no states, not even an empty list @@ -83,7 +83,7 @@ public static void writeStateHandle(OperatorSubtaskState state, String path) thr if (rawKeyedState != null) { dos.writeInt(rawKeyedState.size()); for (KeyedStateHandle keyedStateHandle : rawKeyedState) { - SavepointV1Serializer.serializeKeyedStateHandle(keyedStateHandle, dos); + SavepointV2Serializer.serializeKeyedStateHandle(keyedStateHandle, dos); } } else { // this means no operator states, not even an empty list @@ -94,7 +94,7 @@ public static void writeStateHandle(OperatorSubtaskState state, String path) thr if (managedKeyedState != null) { dos.writeInt(managedKeyedState.size()); for (KeyedStateHandle keyedStateHandle : managedKeyedState) { - SavepointV1Serializer.serializeKeyedStateHandle(keyedStateHandle, dos); + SavepointV2Serializer.serializeKeyedStateHandle(keyedStateHandle, dos); } } else { // this means no operator states, not even an empty list @@ -113,14 +113,14 @@ public static OperatorSubtaskState readStateHandle(String path) throws IOExcepti dis.readInt(); // still required for compatibility to consume the bytes. - SavepointV1Serializer.deserializeStreamStateHandle(dis); + SavepointV2Serializer.deserializeStreamStateHandle(dis); List<OperatorStateHandle> rawOperatorState = null; int numRawOperatorStates = dis.readInt(); if (numRawOperatorStates >= 0) { rawOperatorState = new ArrayList<>(); for (int i = 0; i < numRawOperatorStates; i++) { - OperatorStateHandle operatorState = SavepointV1Serializer.deserializeOperatorStateHandle( + OperatorStateHandle operatorState = SavepointV2Serializer.deserializeOperatorStateHandle( dis); rawOperatorState.add(operatorState); } @@ -131,7 +131,7 @@ public static OperatorSubtaskState readStateHandle(String path) throws IOExcepti if (numManagedOperatorStates >= 0) { managedOperatorState = new ArrayList<>(); for (int i = 0; i < numManagedOperatorStates; i++) { - OperatorStateHandle operatorState = SavepointV1Serializer.deserializeOperatorStateHandle( + OperatorStateHandle operatorState = SavepointV2Serializer.deserializeOperatorStateHandle( dis); managedOperatorState.add(operatorState); } @@ -142,7 +142,7 @@ public static OperatorSubtaskState readStateHandle(String path) throws IOExcepti if (numRawKeyedStates >= 0) { rawKeyedState = new ArrayList<>(); for (int i = 0; i < numRawKeyedStates; i++) { - KeyedStateHandle keyedState = SavepointV1Serializer.deserializeKeyedStateHandle( + KeyedStateHandle keyedState = SavepointV2Serializer.deserializeKeyedStateHandle( dis); rawKeyedState.add(keyedState); } @@ -153,7 +153,7 @@ public static OperatorSubtaskState readStateHandle(String path) throws IOExcepti if (numManagedKeyedStates >= 0) { managedKeyedState = new ArrayList<>(); for (int i = 0; i < numManagedKeyedStates; i++) { - KeyedStateHandle keyedState = SavepointV1Serializer.deserializeKeyedStateHandle( + KeyedStateHandle keyedState = SavepointV2Serializer.deserializeKeyedStateHandle( dis); managedKeyedState.add(keyedState); } ---------------------------------------------------------------- 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: us...@infra.apache.org > OperatorSnapshotUtil should probably use SavepointV2Serializer > -------------------------------------------------------------- > > Key: FLINK-10490 > URL: https://issues.apache.org/jira/browse/FLINK-10490 > Project: Flink > Issue Type: Test > Components: Tests > Affects Versions: 1.7.0 > Reporter: Stefan Richter > Assignee: Stefan Richter > Priority: Major > Labels: pull-request-available > > {{OperatorSnapshotUtil}} is used for testing savepoint migration. This > utility internally still uses {{SavepointV1Serializer}} and I would assume > that it should use {{SavepointV2Serializer}}. I wonder if that means that > some newer cases are actually not covered in the migration tests. -- This message was sent by Atlassian JIRA (v7.6.3#76005)