diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java index e76cf13da64f9..81bf0bf2db441 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; @@ -389,34 +390,34 @@ public void testScaleUpAfterScalingDown() throws Exception { final int parallelism3 = 3; final int maxParallelism = Math.max(parallelism1, Math.max(parallelism2, parallelism3)); - List operatorStateHandles = repartitionAndExecute( + List operatorSubtaskState = repartitionAndExecute( topic, Collections.emptyList(), parallelism1, maxParallelism, IntStream.range(0, parallelism1).boxed().iterator()); - operatorStateHandles = repartitionAndExecute( + operatorSubtaskState = repartitionAndExecute( topic, - operatorStateHandles, + operatorSubtaskState, parallelism2, maxParallelism, IntStream.range(parallelism1, parallelism1 + parallelism2).boxed().iterator()); - operatorStateHandles = repartitionAndExecute( + operatorSubtaskState = repartitionAndExecute( topic, - operatorStateHandles, + operatorSubtaskState, parallelism3, maxParallelism, IntStream.range(parallelism1 + parallelism2, parallelism1 + parallelism2 + parallelism3).boxed().iterator()); // After each previous repartitionAndExecute call, we are left with some lingering transactions, that would // not allow us to read all committed messages from the topic. Thus we initialize operators from - // operatorStateHandles once more, but without any new data. This should terminate all ongoing transactions. + // OperatorSubtaskState once more, but without any new data. This should terminate all ongoing transactions. - operatorStateHandles = repartitionAndExecute( + operatorSubtaskState = repartitionAndExecute( topic, - operatorStateHandles, + operatorSubtaskState, 1, maxParallelism, Collections.emptyIterator()); @@ -448,10 +449,10 @@ private List repartitionAndExecute( testHarness.setup(); testHarness.initializeState(new OperatorSubtaskState( - inputStates, - Collections.emptyList(), - Collections.emptyList(), - Collections.emptyList())); + new StateObjectCollection<>(inputStates), + StateObjectCollection.empty(), + StateObjectCollection.empty(), + StateObjectCollection.empty())); testHarness.open(); if (inputData.hasNext()) { @@ -460,9 +461,9 @@ private List repartitionAndExecute( OperatorSubtaskState snapshot = testHarness.snapshot(0, 0); outputStates.addAll(snapshot.getManagedOperatorState()); - checkState(snapshot.getRawOperatorState() == null, "Unexpected raw operator state"); - checkState(snapshot.getManagedKeyedState() == null, "Unexpected managed keyed state"); - checkState(snapshot.getRawKeyedState() == null, "Unexpected raw keyed state"); + checkState(snapshot.getRawOperatorState().isEmpty(), "Unexpected raw operator state"); + checkState(snapshot.getManagedKeyedState().isEmpty(), "Unexpected managed keyed state"); + checkState(snapshot.getRawKeyedState().isEmpty(), "Unexpected raw keyed state"); for (int i = 1; i < FlinkKafkaProducer011.DEFAULT_KAFKA_PRODUCERS_POOL_SIZE - 1; i++) { testHarness.processElement(-nextValue, 0); diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java index c89aff2adfe63..c6af7dd894bcb 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CheckpointingOptions.java @@ -66,6 +66,20 @@ public class CheckpointingOptions { " complete checkpoint state. Some state backends may not support incremental checkpoints and ignore" + " this option."); + /** + * This option configures local recovery for this state backend. + */ + public static final ConfigOption LOCAL_RECOVERY = ConfigOptions + .key("state.backend.local-recovery") + .defaultValue("DISABLED"); + + /** + * The config parameter defining the root directories for storing file-based state for local recovery. + */ + public static final ConfigOption LOCAL_RECOVERY_TASK_MANAGER_STATE_ROOT_DIRS = ConfigOptions + .key("taskmanager.state.local.root-dirs") + .noDefaultValue(); + // ------------------------------------------------------------------------ // Options specific to the file-system-based state backends // ------------------------------------------------------------------------ diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java index e697e6fb2825c..8566a43a3413a 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigurationUtils.java @@ -18,6 +18,8 @@ package org.apache.flink.configuration; +import javax.annotation.Nonnull; + import java.io.File; /** @@ -25,6 +27,8 @@ */ public class ConfigurationUtils { + private static final String[] EMPTY = new String[0]; + /** * Extracts the task manager directories for temporary files as defined by * {@link org.apache.flink.configuration.CoreOptions#TMP_DIRS}. @@ -32,10 +36,30 @@ public class ConfigurationUtils { * @param configuration configuration object * @return array of configured directories (in order) */ + @Nonnull public static String[] parseTempDirectories(Configuration configuration) { - return configuration.getString(CoreOptions.TMP_DIRS).split(",|" + File.pathSeparator); + return splitPaths(configuration.getString(CoreOptions.TMP_DIRS)); + } + + /** + * Extracts the local state directories as defined by + * {@link CheckpointingOptions#LOCAL_RECOVERY_TASK_MANAGER_STATE_ROOT_DIRS}. + * + * @param configuration configuration object + * @return array of configured directories (in order) + */ + @Nonnull + public static String[] parseLocalStateDirectories(Configuration configuration) { + String configValue = configuration.getString(CheckpointingOptions.LOCAL_RECOVERY_TASK_MANAGER_STATE_ROOT_DIRS, ""); + return splitPaths(configValue); + } + + @Nonnull + private static String[] splitPaths(@Nonnull String separatedPaths) { + return separatedPaths.length() > 0 ? separatedPaths.split(",|" + File.pathSeparator) : EMPTY; } // Make sure that we cannot instantiate this class - private ConfigurationUtils() {} + private ConfigurationUtils() { + } } diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java index ee2ecbe20b656..2d7fbd593fbbc 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalFileSystem.java @@ -211,10 +211,12 @@ private boolean delete(final File f) throws IOException { if (f.isDirectory()) { final File[] files = f.listFiles(); - for (File file : files) { - final boolean del = delete(file); - if (!del) { - return false; + if (files != null) { + for (File file : files) { + final boolean del = delete(file); + if (!del) { + return false; + } } } } else { diff --git a/flink-core/src/main/java/org/apache/flink/util/Disposable.java b/flink-core/src/main/java/org/apache/flink/util/Disposable.java new file mode 100644 index 0000000000000..12ef7630bff63 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/util/Disposable.java @@ -0,0 +1,36 @@ +/* + * 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.util; + +/** + * Interface for classes that can be disposed, i.e. that have a dedicated lifecycle step to "destroy" the object. On + * reason for this is for example to release native resources. From this point, the interface fulfills a similar purpose + * as the {@link java.io.Closeable} interface, but sometimes both should be represented as isolated, independent + * lifecycle steps. + */ +public interface Disposable { + + /** + * Disposes the object and releases all resources. After calling this method, calling any methods on the + * object may result in undefined behavior. + * + * @throws Exception if something goes wrong during disposal. + */ + void dispose() throws Exception; +} diff --git a/flink-core/src/test/java/org/apache/flink/util/MethodForwardingTestUtil.java b/flink-core/src/test/java/org/apache/flink/util/MethodForwardingTestUtil.java new file mode 100644 index 0000000000000..8a9e8cad18539 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/util/MethodForwardingTestUtil.java @@ -0,0 +1,179 @@ +/* + * 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.util; + +import org.mockito.Mockito; +import org.mockito.internal.util.MockUtil; + +import java.lang.reflect.Array; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Collections; +import java.util.Set; +import java.util.function.Function; +import java.util.function.Supplier; + +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.spy; + +/** + * Helper class with a method that attempts to automatically test method forwarding between a delegate and a wrapper. + */ +public class MethodForwardingTestUtil { + + /** + * This is a best effort automatic test for method forwarding between a delegate and its wrapper, where the wrapper + * class is a subtype of the delegate. This ignores methods that are inherited from Object. + * + * @param delegateClass the class for the delegate. + * @param wrapperFactory factory that produces a wrapper from a delegate. + * @param type of the delegate + * @param type of the wrapper + */ + public static void testMethodForwarding( + Class delegateClass, + Function wrapperFactory) + throws NoSuchMethodException, IllegalAccessException, InvocationTargetException { + testMethodForwarding(delegateClass, wrapperFactory, () -> spy(delegateClass), Collections.emptySet()); + } + + /** + * This is a best effort automatic test for method forwarding between a delegate and its wrapper, where the wrapper + * class is a subtype of the delegate. This ignores methods that are inherited from Object. + * + * @param delegateClass the class for the delegate. + * @param wrapperFactory factory that produces a wrapper from a delegate. + * @param delegateObjectSupplier supplier for the delegate object passed to the wrapper factory. + * @param type of the delegate + * @param type of the wrapper + * @param type of the object created as delegate, is a subtype of D. + */ + public static void testMethodForwarding( + Class delegateClass, + Function wrapperFactory, + Supplier delegateObjectSupplier) + throws NoSuchMethodException, IllegalAccessException, InvocationTargetException { + testMethodForwarding(delegateClass, wrapperFactory, delegateObjectSupplier, Collections.emptySet()); + } + + /** + * This is a best effort automatic test for method forwarding between a delegate and its wrapper, where the wrapper + * class is a subtype of the delegate. Methods can be remapped in case that the implementation does not call the + * original method. Remapping to null skips the method. This ignores methods that are inherited from Object. + * + * @param delegateClass the class for the delegate. + * @param wrapperFactory factory that produces a wrapper from a delegate. + * @param delegateObjectSupplier supplier for the delegate object passed to the wrapper factory. + * @param skipMethodSet set of methods to ignore. + * @param type of the delegate + * @param type of the wrapper + * @param type of the object created as delegate, is a subtype of D. + */ + public static void testMethodForwarding( + Class delegateClass, + Function wrapperFactory, + Supplier delegateObjectSupplier, + Set skipMethodSet) throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { + + Preconditions.checkNotNull(delegateClass); + Preconditions.checkNotNull(wrapperFactory); + Preconditions.checkNotNull(skipMethodSet); + + I delegate = delegateObjectSupplier.get(); + + //check if we need to wrap the delegate object as a spy, or if it is already testable with Mockito. + MockUtil mockUtil = new MockUtil(); + if (!mockUtil.isSpy(delegate) || !mockUtil.isMock(delegate)) { + delegate = spy(delegate); + } + + W wrapper = wrapperFactory.apply(delegate); + + // ensure that wrapper is a subtype of delegate + Preconditions.checkArgument(delegateClass.isAssignableFrom(wrapper.getClass())); + + for (Method delegateMethod : delegateClass.getMethods()) { + + if (checkSkipMethodForwardCheck(delegateMethod, skipMethodSet)) { + continue; + } + + // find the correct method to substitute the bridge for erased generic types. + // if this doesn't work, the user need to exclude the method and write an additional test. + Method wrapperMethod = wrapper.getClass().getMethod( + delegateMethod.getName(), + delegateMethod.getParameterTypes()); + + // things get a bit fuzzy here, best effort to find a match but this might end up with a wrong method. + if (wrapperMethod.isBridge()) { + for (Method method : wrapper.getClass().getMethods()) { + if (!method.isBridge() + && method.getName().equals(wrapperMethod.getName()) + && method.getParameterCount() == wrapperMethod.getParameterCount()) { + wrapperMethod = method; + break; + } + } + } + + Class[] parameterTypes = wrapperMethod.getParameterTypes(); + Object[] arguments = new Object[parameterTypes.length]; + for (int j = 0; j < arguments.length; j++) { + Class parameterType = parameterTypes[j]; + if (parameterType.isArray()) { + arguments[j] = Array.newInstance(parameterType.getComponentType(), 0); + } else if (parameterType.isPrimitive()) { + if (boolean.class.equals(parameterType)) { + arguments[j] = false; + } else if (char.class.equals(parameterType)) { + arguments[j] = 'a'; + } else { + arguments[j] = (byte) 0; + } + } else { + arguments[j] = Mockito.mock(parameterType); + } + } + + wrapperMethod.invoke(wrapper, arguments); + delegateMethod.invoke(Mockito.verify(delegate, Mockito.times(1)), arguments); + reset(delegate); + } + } + + /** + * Test if this method should be skipped in our check for proper forwarding, e.g. because it is just a bridge. + */ + private static boolean checkSkipMethodForwardCheck(Method delegateMethod, Set skipMethods) { + + if (delegateMethod.isBridge() + || delegateMethod.isDefault() + || skipMethods.contains(delegateMethod)) { + return true; + } + + // skip methods declared in Object (Mockito doesn't like them) + try { + Object.class.getMethod(delegateMethod.getName(), delegateMethod.getParameterTypes()); + return true; + } catch (Exception ignore) { + } + return false; + } +} diff --git a/flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java b/flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java index b3a5def26dcb1..be5f67740c7f9 100644 --- a/flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java +++ b/flink-docs/src/main/java/org/apache/flink/docs/rest/RestAPIDocGenerator.java @@ -120,9 +120,7 @@ private static void createHtmlFile(DocumentingRestEndpoint restEndpoint, Path ou List specs = restEndpoint.getSpecs(); specs.forEach(spec -> html.append(createHtmlEntry(spec))); - if (Files.exists(outputFile)) { - Files.delete(outputFile); - } + Files.deleteIfExists(outputFile); Files.write(outputFile, html.toString().getBytes(StandardCharsets.UTF_8)); } diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala index e69472e7c0d1e..12cc375012a5f 100644 --- a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala +++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala @@ -24,6 +24,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup +import org.apache.flink.runtime.state.TaskExecutorLocalStateStoresManager import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation} @@ -37,6 +38,7 @@ class MesosTaskManager( memoryManager: MemoryManager, ioManager: IOManager, network: NetworkEnvironment, + taskManagerLocalStateStoresManager: TaskExecutorLocalStateStoresManager, numberOfSlots: Int, highAvailabilityServices: HighAvailabilityServices, taskManagerMetricGroup : TaskManagerMetricGroup) @@ -47,6 +49,7 @@ class MesosTaskManager( memoryManager, ioManager, network, + taskManagerLocalStateStoresManager, numberOfSlots, highAvailabilityServices, taskManagerMetricGroup) { diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KVStateRequestSerializerRocksDBTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KVStateRequestSerializerRocksDBTest.java index 07517ab0c0375..dd75dd6ed34fd 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KVStateRequestSerializerRocksDBTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KVStateRequestSerializerRocksDBTest.java @@ -30,6 +30,7 @@ import org.apache.flink.queryablestate.client.VoidNamespaceSerializer; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.TestLocalRecoveryConfig; import org.apache.flink.runtime.state.internal.InternalListState; import org.apache.flink.runtime.state.internal.InternalMapState; @@ -76,7 +77,8 @@ static final class RocksDBKeyedStateBackend2 extends RocksDBKeyedStateBackend super(operatorIdentifier, userCodeClassLoader, instanceBasePath, dbOptions, columnFamilyOptions, kvStateRegistry, keySerializer, - numberOfKeyGroups, keyGroupRange, executionConfig, false); + numberOfKeyGroups, keyGroupRange, executionConfig, false, + TestLocalRecoveryConfig.disabled()); } @Override @@ -152,7 +154,8 @@ public void testMapSerialization() throws Exception { LongSerializer.INSTANCE, 1, new KeyGroupRange(0, 0), new ExecutionConfig(), - false); + false, + TestLocalRecoveryConfig.disabled()); longHeapKeyedStateBackend.restore(null); longHeapKeyedStateBackend.setCurrentKey(key); diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java index d3314abad1a6a..8d1014197f123 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateRequestSerializerTest.java @@ -30,6 +30,7 @@ import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.TestLocalRecoveryConfig; import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend; import org.apache.flink.runtime.state.internal.InternalKvState; import org.apache.flink.runtime.state.internal.InternalListState; @@ -194,7 +195,8 @@ public void testListSerialization() throws Exception { 1, new KeyGroupRange(0, 0), async, - new ExecutionConfig() + new ExecutionConfig(), + TestLocalRecoveryConfig.disabled() ); longHeapKeyedStateBackend.setCurrentKey(key); @@ -290,13 +292,14 @@ public void testMapSerialization() throws Exception { // objects for heap state list serialisation final HeapKeyedStateBackend longHeapKeyedStateBackend = new HeapKeyedStateBackend<>( - mock(TaskKvStateRegistry.class), - LongSerializer.INSTANCE, - ClassLoader.getSystemClassLoader(), - 1, - new KeyGroupRange(0, 0), - async, - new ExecutionConfig() + mock(TaskKvStateRegistry.class), + LongSerializer.INSTANCE, + ClassLoader.getSystemClassLoader(), + 1, + new KeyGroupRange(0, 0), + async, + new ExecutionConfig(), + TestLocalRecoveryConfig.disabled() ); longHeapKeyedStateBackend.setCurrentKey(key); diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java index 58d532b0b5193..450436fc36253 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/history/HistoryServerArchiveFetcher.java @@ -180,9 +180,7 @@ public void run() { // We overwrite existing files since this may be another attempt at fetching this archive. // Existing files may be incomplete/corrupt. - if (Files.exists(targetPath)) { - Files.delete(targetPath); - } + Files.deleteIfExists(targetPath); Files.createFile(target.toPath()); try (FileWriter fw = new FileWriter(target)) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/JobManagerTaskRestore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/JobManagerTaskRestore.java index d5ac3e061d128..2537db1a6625c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/JobManagerTaskRestore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/JobManagerTaskRestore.java @@ -18,6 +18,9 @@ package org.apache.flink.runtime.checkpoint; +import javax.annotation.Nonnegative; +import javax.annotation.Nonnull; + import java.io.Serializable; /** @@ -27,11 +30,13 @@ public class JobManagerTaskRestore implements Serializable { private static final long serialVersionUID = 1L; + /** The id of the checkpoint from which we restore. */ private final long restoreCheckpointId; + /** The state for this task to restore. */ private final TaskStateSnapshot taskStateSnapshot; - public JobManagerTaskRestore(long restoreCheckpointId, TaskStateSnapshot taskStateSnapshot) { + public JobManagerTaskRestore(@Nonnegative long restoreCheckpointId, @Nonnull TaskStateSnapshot taskStateSnapshot) { this.restoreCheckpointId = restoreCheckpointId; this.taskStateSnapshot = taskStateSnapshot; } @@ -40,13 +45,14 @@ public long getRestoreCheckpointId() { return restoreCheckpointId; } + @Nonnull public TaskStateSnapshot getTaskStateSnapshot() { return taskStateSnapshot; } @Override public String toString() { - return "TaskRestore{" + + return "JobManagerTaskRestore{" + "restoreCheckpointId=" + restoreCheckpointId + ", taskStateSnapshot=" + taskStateSnapshot + '}'; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java index 3df9c4fc5248d..c2b8e9aa8420a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java @@ -19,8 +19,8 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.runtime.state.CompositeStateHandle; -import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.StateObject; import org.apache.flink.runtime.state.StateUtil; @@ -30,10 +30,9 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; import java.util.List; /** @@ -50,8 +49,6 @@ * Under normal circumstances, the expected size of each collection is still 0 or 1, except for scale-down. In * scale-down, one operator subtask can become responsible for the state of multiple previous subtasks. The collections * can then store all the state handles that are relevant to build up the new subtask state. - * - *

There is no collection for legacy state because it is not rescalable. */ public class OperatorSubtaskState implements CompositeStateHandle { @@ -63,25 +60,25 @@ public class OperatorSubtaskState implements CompositeStateHandle { * Snapshot from the {@link org.apache.flink.runtime.state.OperatorStateBackend}. */ @Nonnull - private final Collection managedOperatorState; + private final StateObjectCollection managedOperatorState; /** * Snapshot written using {@link org.apache.flink.runtime.state.OperatorStateCheckpointOutputStream}. */ @Nonnull - private final Collection rawOperatorState; + private final StateObjectCollection rawOperatorState; /** * Snapshot from {@link org.apache.flink.runtime.state.KeyedStateBackend}. */ @Nonnull - private final Collection managedKeyedState; + private final StateObjectCollection managedKeyedState; /** * Snapshot written using {@link org.apache.flink.runtime.state.KeyedStateCheckpointOutputStream}. */ @Nonnull - private final Collection rawKeyedState; + private final StateObjectCollection rawKeyedState; /** * The state size. This is also part of the deserialized state handle. @@ -95,43 +92,39 @@ public class OperatorSubtaskState implements CompositeStateHandle { */ public OperatorSubtaskState() { this( - Collections.emptyList(), - Collections.emptyList(), - Collections.emptyList(), - Collections.emptyList()); + StateObjectCollection.empty(), + StateObjectCollection.empty(), + StateObjectCollection.empty(), + StateObjectCollection.empty()); } public OperatorSubtaskState( - Collection managedOperatorState, - Collection rawOperatorState, - Collection managedKeyedState, - Collection rawKeyedState) { + @Nonnull StateObjectCollection managedOperatorState, + @Nonnull StateObjectCollection rawOperatorState, + @Nonnull StateObjectCollection managedKeyedState, + @Nonnull StateObjectCollection rawKeyedState) { this.managedOperatorState = Preconditions.checkNotNull(managedOperatorState); this.rawOperatorState = Preconditions.checkNotNull(rawOperatorState); this.managedKeyedState = Preconditions.checkNotNull(managedKeyedState); this.rawKeyedState = Preconditions.checkNotNull(rawKeyedState); - try { - long calculateStateSize = sumAllSizes(managedOperatorState); - calculateStateSize += sumAllSizes(rawOperatorState); - calculateStateSize += sumAllSizes(managedKeyedState); - calculateStateSize += sumAllSizes(rawKeyedState); - stateSize = calculateStateSize; - } catch (Exception e) { - throw new RuntimeException("Failed to get state size.", e); - } + long calculateStateSize = managedOperatorState.getStateSize(); + calculateStateSize += rawOperatorState.getStateSize(); + calculateStateSize += managedKeyedState.getStateSize(); + calculateStateSize += rawKeyedState.getStateSize(); + stateSize = calculateStateSize; } /** * For convenience because the size of the collections is typically 0 or 1. Null values are translated into empty - * Collections (except for legacy state). + * Collections. */ public OperatorSubtaskState( - OperatorStateHandle managedOperatorState, - OperatorStateHandle rawOperatorState, - KeyedStateHandle managedKeyedState, - KeyedStateHandle rawKeyedState) { + @Nullable OperatorStateHandle managedOperatorState, + @Nullable OperatorStateHandle rawOperatorState, + @Nullable KeyedStateHandle managedKeyedState, + @Nullable KeyedStateHandle rawKeyedState) { this( singletonOrEmptyOnNull(managedOperatorState), @@ -140,21 +133,8 @@ public OperatorSubtaskState( singletonOrEmptyOnNull(rawKeyedState)); } - private static Collection singletonOrEmptyOnNull(T element) { - return element != null ? Collections.singletonList(element) : Collections.emptyList(); - } - - private static long sumAllSizes(Collection stateObject) throws Exception { - long size = 0L; - for (StateObject object : stateObject) { - size += getSizeNullSafe(object); - } - - return size; - } - - private static long getSizeNullSafe(StateObject stateObject) throws Exception { - return stateObject != null ? stateObject.getStateSize() : 0L; + private static StateObjectCollection singletonOrEmptyOnNull(T element) { + return element != null ? StateObjectCollection.singleton(element) : StateObjectCollection.empty(); } // -------------------------------------------------------------------------------------------- @@ -163,7 +143,7 @@ private static long getSizeNullSafe(StateObject stateObject) throws Exception { * Returns a handle to the managed operator state. */ @Nonnull - public Collection getManagedOperatorState() { + public StateObjectCollection getManagedOperatorState() { return managedOperatorState; } @@ -171,7 +151,7 @@ public Collection getManagedOperatorState() { * Returns a handle to the raw operator state. */ @Nonnull - public Collection getRawOperatorState() { + public StateObjectCollection getRawOperatorState() { return rawOperatorState; } @@ -179,7 +159,7 @@ public Collection getRawOperatorState() { * Returns a handle to the managed keyed state. */ @Nonnull - public Collection getManagedKeyedState() { + public StateObjectCollection getManagedKeyedState() { return managedKeyedState; } @@ -187,7 +167,7 @@ public Collection getManagedKeyedState() { * Returns a handle to the raw keyed state. */ @Nonnull - public Collection getRawKeyedState() { + public StateObjectCollection getRawKeyedState() { return rawKeyedState; } @@ -281,18 +261,9 @@ public String toString() { } public boolean hasState() { - return hasState(managedOperatorState) - || hasState(rawOperatorState) - || hasState(managedKeyedState) - || hasState(rawKeyedState); - } - - private boolean hasState(Iterable states) { - for (StateObject state : states) { - if (state != null) { - return true; - } - } - return false; + return managedOperatorState.hasState() + || rawOperatorState.hasState() + || managedKeyedState.hasState() + || rawKeyedState.hasState(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java new file mode 100644 index 0000000000000..f48d3110c53c0 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java @@ -0,0 +1,297 @@ +/* + * 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.runtime.checkpoint; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.StateObject; + +import javax.annotation.Nonnull; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.function.BiFunction; + +/** + * This class is a wrapper over multiple alternative {@link OperatorSubtaskState} that are (partial) substitutes for + * each other and imposes a priority ordering over all alternatives for the different states which define an order in + * which the operator should attempt to restore the state from them. One OperatorSubtaskState is considered as the + * "ground truth" about which state should be represented. Alternatives may be complete or partial substitutes for + * the "ground truth" with a higher priority (if they had a lower alternative, they would not really be alternatives). + * Substitution is determined on a per-sub-state basis. + */ +@Internal +public class PrioritizedOperatorSubtaskState { + + /** Singleton instance for an empty, non-restored operator state. */ + private static final PrioritizedOperatorSubtaskState EMPTY_NON_RESTORED_INSTANCE = + new PrioritizedOperatorSubtaskState.Builder(new OperatorSubtaskState(), Collections.emptyList(), false) + .build(); + + /** List of prioritized snapshot alternatives for managed operator state. */ + private final List> prioritizedManagedOperatorState; + + /** List of prioritized snapshot alternatives for raw operator state. */ + private final List> prioritizedRawOperatorState; + + /** List of prioritized snapshot alternatives for managed keyed state. */ + private final List> prioritizedManagedKeyedState; + + /** List of prioritized snapshot alternatives for raw keyed state. */ + private final List> prioritizedRawKeyedState; + + /** Signal flag if this represents state for a restored operator. */ + private final boolean restored; + + PrioritizedOperatorSubtaskState( + @Nonnull List> prioritizedManagedKeyedState, + @Nonnull List> prioritizedRawKeyedState, + @Nonnull List> prioritizedManagedOperatorState, + @Nonnull List> prioritizedRawOperatorState, + boolean restored) { + + this.prioritizedManagedOperatorState = prioritizedManagedOperatorState; + this.prioritizedRawOperatorState = prioritizedRawOperatorState; + this.prioritizedManagedKeyedState = prioritizedManagedKeyedState; + this.prioritizedRawKeyedState = prioritizedRawKeyedState; + this.restored = restored; + } + + // ----------------------------------------------------------------------------------------------------------------- + + /** + * Returns an iterator over all alternative snapshots to restore the managed operator state, in the order in which + * we should attempt to restore. + */ + @Nonnull + public Iterator> getPrioritizedManagedOperatorState() { + return prioritizedManagedOperatorState.iterator(); + } + + /** + * Returns an iterator over all alternative snapshots to restore the raw operator state, in the order in which we + * should attempt to restore. + */ + @Nonnull + public Iterator> getPrioritizedRawOperatorState() { + return prioritizedRawOperatorState.iterator(); + } + + /** + * Returns an iterator over all alternative snapshots to restore the managed keyed state, in the order in which we + * should attempt to restore. + */ + @Nonnull + public Iterator> getPrioritizedManagedKeyedState() { + return prioritizedManagedKeyedState.iterator(); + } + + /** + * Returns an iterator over all alternative snapshots to restore the raw keyed state, in the order in which we + * should attempt to restore. + */ + @Nonnull + public Iterator> getPrioritizedRawKeyedState() { + return prioritizedRawKeyedState.iterator(); + } + + // ----------------------------------------------------------------------------------------------------------------- + + /** + * Returns the managed operator state from the job manager, which represents the ground truth about what this state + * should represent. This is the alternative with lowest priority. + */ + @Nonnull + public StateObjectCollection getJobManagerManagedOperatorState() { + return lastElement(prioritizedManagedOperatorState); + } + + /** + * Returns the raw operator state from the job manager, which represents the ground truth about what this state + * should represent. This is the alternative with lowest priority. + */ + @Nonnull + public StateObjectCollection getJobManagerRawOperatorState() { + return lastElement(prioritizedRawOperatorState); + } + + /** + * Returns the managed keyed state from the job manager, which represents the ground truth about what this state + * should represent. This is the alternative with lowest priority. + */ + @Nonnull + public StateObjectCollection getJobManagerManagedKeyedState() { + return lastElement(prioritizedManagedKeyedState); + } + + /** + * Returns the raw keyed state from the job manager, which represents the ground truth about what this state + * should represent. This is the alternative with lowest priority. + */ + @Nonnull + public StateObjectCollection getJobManagerRawKeyedState() { + return lastElement(prioritizedRawKeyedState); + } + + // ----------------------------------------------------------------------------------------------------------------- + + /** + * Returns true if this was created for a restored operator, false otherwise. Restored operators are operators that + * participated in a previous checkpoint, even if they did not emit any state snapshots. + */ + public boolean isRestored() { + return restored; + } + + + private static StateObjectCollection lastElement(List> list) { + return list.get(list.size() - 1); + } + + /** + * Returns an empty {@link PrioritizedOperatorSubtaskState} singleton for an empty, not-restored operator state. + */ + public static PrioritizedOperatorSubtaskState emptyNotRestored() { + return EMPTY_NON_RESTORED_INSTANCE; + } + + @Internal + public static class Builder { + + /** Ground truth of state, provided by job manager. */ + @Nonnull + private final OperatorSubtaskState jobManagerState; + + /** (Local) alternatives to the job manager state. */ + @Nonnull + private final List alternativesByPriority; + + /** Flag if the states have been restored. */ + private final boolean restored; + + public Builder( + @Nonnull OperatorSubtaskState jobManagerState, + @Nonnull List alternativesByPriority) { + this(jobManagerState, alternativesByPriority, true); + } + + public Builder( + @Nonnull OperatorSubtaskState jobManagerState, + @Nonnull List alternativesByPriority, + boolean restored) { + + this.jobManagerState = jobManagerState; + this.alternativesByPriority = alternativesByPriority; + this.restored = restored; + } + + public PrioritizedOperatorSubtaskState build() { + int size = alternativesByPriority.size(); + List> managedOperatorAlternatives = new ArrayList<>(size); + List> managedKeyedAlternatives = new ArrayList<>(size); + List> rawOperatorAlternatives = new ArrayList<>(size); + List> rawKeyedAlternatives = new ArrayList<>(size); + + for (OperatorSubtaskState subtaskState : alternativesByPriority) { + + if (subtaskState != null) { + managedKeyedAlternatives.add(subtaskState.getManagedKeyedState()); + rawKeyedAlternatives.add(subtaskState.getRawKeyedState()); + managedOperatorAlternatives.add(subtaskState.getManagedOperatorState()); + rawOperatorAlternatives.add(subtaskState.getRawOperatorState()); + } + } + + // Key-groups should match. + BiFunction keyedStateApprover = + (ref, alt) -> ref.getKeyGroupRange().equals(alt.getKeyGroupRange()); + + // State meta data should match. + BiFunction operatorStateApprover = + (ref, alt) -> ref.getStateNameToPartitionOffsets().equals(alt.getStateNameToPartitionOffsets()); + + return new PrioritizedOperatorSubtaskState( + resolvePrioritizedAlternatives( + jobManagerState.getManagedKeyedState(), + managedKeyedAlternatives, + keyedStateApprover), + resolvePrioritizedAlternatives( + jobManagerState.getRawKeyedState(), + rawKeyedAlternatives, + keyedStateApprover), + resolvePrioritizedAlternatives( + jobManagerState.getManagedOperatorState(), + managedOperatorAlternatives, + operatorStateApprover), + resolvePrioritizedAlternatives( + jobManagerState.getRawOperatorState(), + rawOperatorAlternatives, + operatorStateApprover), + restored); + } + + /** + * This helper method resolves the dependencies between the ground truth of the operator state obtained from the + * job manager and potential alternatives for recovery, e.g. from a task-local source. + */ + protected List> resolvePrioritizedAlternatives( + StateObjectCollection jobManagerState, + List> alternativesByPriority, + BiFunction approveFun) { + + // Nothing to resolve if there are no alternatives, or the ground truth has already no state, or if we can + // assume that a rescaling happened because we find more than one handle in the JM state (this is more a sanity + // check). + if (alternativesByPriority == null + || alternativesByPriority.isEmpty() + || !jobManagerState.hasState() + || jobManagerState.size() != 1) { + + return Collections.singletonList(jobManagerState); + } + + // As we know size is == 1 + T reference = jobManagerState.iterator().next(); + + // This will contain the end result, we initialize it with the potential max. size. + List> approved = + new ArrayList<>(1 + alternativesByPriority.size()); + + for (StateObjectCollection alternative : alternativesByPriority) { + + // We found an alternative to the JM state if it has state, we have a 1:1 relationship, and the + // approve-function signaled true. + if (alternative != null + && alternative.hasState() + && alternative.size() == 1 + && approveFun.apply(reference, alternative.iterator().next())) { + + approved.add(alternative); + } + } + + // Of course we include the ground truth as last alternative. + approved.add(jobManagerState); + return Collections.unmodifiableList(approved); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/RoundRobinOperatorStateRepartitioner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/RoundRobinOperatorStateRepartitioner.java index e09b677c3b0d8..e6fa687fd1469 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/RoundRobinOperatorStateRepartitioner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/RoundRobinOperatorStateRepartitioner.java @@ -20,6 +20,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.util.Preconditions; @@ -80,7 +81,10 @@ private GroupByStateNameResults groupByStateName(List previ new EnumMap<>(OperatorStateHandle.Mode.class); for (OperatorStateHandle.Mode mode : OperatorStateHandle.Mode.values()) { - nameToStateByMode.put(mode, new HashMap<>()); + + nameToStateByMode.put( + mode, + new HashMap<>()); } for (OperatorStateHandle psh : previousParallelSubtaskStates) { @@ -171,8 +175,6 @@ private List> repartition( } // Now start collection the partitions for the parallel instance into this list - List> parallelOperatorState = - new ArrayList<>(); while (numberOfPartitionsToAssign > 0) { Tuple2 handleWithOffsets = @@ -194,10 +196,6 @@ private List> repartition( ++lstIdx; } - parallelOperatorState.add(new Tuple2<>( - handleWithOffsets.f0, - new OperatorStateHandle.StateMetaInfo(offs, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE))); - numberOfPartitionsToAssign -= remaining; // As a last step we merge partitions that use the same StreamStateHandle in a single @@ -205,7 +203,7 @@ private List> repartition( Map mergeMap = mergeMapList.get(parallelOpIdx); OperatorStateHandle operatorStateHandle = mergeMap.get(handleWithOffsets.f0); if (operatorStateHandle == null) { - operatorStateHandle = new OperatorStateHandle(new HashMap<>(), handleWithOffsets.f0); + operatorStateHandle = new OperatorStreamStateHandle(new HashMap<>(), handleWithOffsets.f0); mergeMap.put(handleWithOffsets.f0, operatorStateHandle); } operatorStateHandle.getStateNameToPartitionOffsets().put( @@ -231,7 +229,7 @@ private List> repartition( for (Tuple2 handleWithMetaInfo : e.getValue()) { OperatorStateHandle operatorStateHandle = mergeMap.get(handleWithMetaInfo.f0); if (operatorStateHandle == null) { - operatorStateHandle = new OperatorStateHandle(new HashMap<>(), handleWithMetaInfo.f0); + operatorStateHandle = new OperatorStreamStateHandle(new HashMap<>(), handleWithMetaInfo.f0); mergeMap.put(handleWithMetaInfo.f0, operatorStateHandle); } operatorStateHandle.getStateNameToPartitionOffsets().put(e.getKey(), handleWithMetaInfo.f1); @@ -252,13 +250,13 @@ private List> repartition( uniformBroadcastNameToState.entrySet()) { int oldParallelism = e.getValue().size(); - + Tuple2 handleWithMetaInfo = e.getValue().get(i % oldParallelism); OperatorStateHandle operatorStateHandle = mergeMap.get(handleWithMetaInfo.f0); if (operatorStateHandle == null) { - operatorStateHandle = new OperatorStateHandle(new HashMap<>(), handleWithMetaInfo.f0); + operatorStateHandle = new OperatorStreamStateHandle(new HashMap<>(), handleWithMetaInfo.f0); mergeMap.put(handleWithMetaInfo.f0, operatorStateHandle); } operatorStateHandle.getStateNameToPartitionOffsets().put(e.getKey(), handleWithMetaInfo.f1); @@ -282,4 +280,4 @@ public Map(subManagedOperatorState.getOrDefault(instanceID, Collections.emptyList())), + new StateObjectCollection<>(subRawOperatorState.getOrDefault(instanceID, Collections.emptyList())), + new StateObjectCollection<>(subManagedKeyedState.getOrDefault(instanceID, Collections.emptyList())), + new StateObjectCollection<>(subRawKeyedState.getOrDefault(instanceID, Collections.emptyList()))); } private static boolean isHeadOperator(int opIdx, List operatorIDs) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateObjectCollection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateObjectCollection.java new file mode 100644 index 0000000000000..38e3d15da29a5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateObjectCollection.java @@ -0,0 +1,205 @@ +/* + * 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.runtime.checkpoint; + +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.runtime.state.StateUtil; + +import org.apache.commons.collections.CollectionUtils; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.function.Predicate; + +/** + * This class represents a generic collection for {@link StateObject}s. Being a state object itself, it delegates + * {@link #discardState()} to all contained state objects and computes {@link #getStateSize()} as sum of the state + * sizes of all contained objects. + * + * @param type of the contained state objects. + */ +public class StateObjectCollection implements Collection, StateObject { + + private static final long serialVersionUID = 1L; + + /** The empty StateObjectCollection. */ + private static final StateObjectCollection EMPTY = new StateObjectCollection<>(Collections.emptyList()); + + /** Wrapped collection that contains the state objects. */ + private final Collection stateObjects; + + /** + * Creates a new StateObjectCollection that is backed by an {@link ArrayList}. + */ + public StateObjectCollection() { + this.stateObjects = new ArrayList<>(); + } + + /** + * Creates a new StateObjectCollection wraps the given collection and delegates to it. + * @param stateObjects collection of state objects to wrap. + */ + public StateObjectCollection(Collection stateObjects) { + this.stateObjects = stateObjects != null ? stateObjects : Collections.emptyList(); + } + + @Override + public int size() { + return stateObjects.size(); + } + + @Override + public boolean isEmpty() { + return stateObjects.isEmpty(); + } + + @Override + public boolean contains(Object o) { + return stateObjects.contains(o); + } + + @Override + public Iterator iterator() { + return stateObjects.iterator(); + } + + @Override + public Object[] toArray() { + return stateObjects.toArray(); + } + + @Override + public T1[] toArray(T1[] a) { + return stateObjects.toArray(a); + } + + @Override + public boolean add(T t) { + return stateObjects.add(t); + } + + @Override + public boolean remove(Object o) { + return stateObjects.remove(o); + } + + @Override + public boolean containsAll(Collection c) { + return stateObjects.containsAll(c); + } + + @Override + public boolean addAll(Collection c) { + return stateObjects.addAll(c); + } + + @Override + public boolean removeAll(Collection c) { + return stateObjects.removeAll(c); + } + + @Override + public boolean removeIf(Predicate filter) { + return stateObjects.removeIf(filter); + } + + @Override + public boolean retainAll(Collection c) { + return stateObjects.retainAll(c); + } + + @Override + public void clear() { + stateObjects.clear(); + } + + @Override + public void discardState() throws Exception { + StateUtil.bestEffortDiscardAllStateObjects(stateObjects); + } + + @Override + public long getStateSize() { + return sumAllSizes(stateObjects); + } + + /** + * Returns true if this contains at least one {@link StateObject}. + */ + public boolean hasState() { + for (StateObject state : stateObjects) { + if (state != null) { + return true; + } + } + return false; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + StateObjectCollection that = (StateObjectCollection) o; + + // simple equals can cause troubles here because of how equals works e.g. between lists and sets. + return CollectionUtils.isEqualCollection(stateObjects, that.stateObjects); + } + + @Override + public int hashCode() { + return stateObjects.hashCode(); + } + + @Override + public String toString() { + return "StateObjectCollection{" + stateObjects + '}'; + } + + // ------------------------------------------------------------------------ + // Helper methods. + // ------------------------------------------------------------------------ + + public static StateObjectCollection empty() { + return (StateObjectCollection) EMPTY; + } + + public static StateObjectCollection singleton(T stateObject) { + return new StateObjectCollection<>(Collections.singleton(stateObject)); + } + + private static long sumAllSizes(Collection stateObject) { + long size = 0L; + for (StateObject object : stateObject) { + size += getSizeNullSafe(object); + } + + return size; + } + + private static long getSizeNullSafe(StateObject stateObject) { + return stateObject != null ? stateObject.getStateSize() : 0L; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskState.java index 281693bc90304..5aab33a8c4c80 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SubtaskState.java @@ -20,8 +20,8 @@ import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.CompositeStateHandle; -import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.StateObject; import org.apache.flink.runtime.state.StateUtil; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskStateSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskStateSnapshot.java index 28edc63fafff3..abc96a30cc1f6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskStateSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskStateSnapshot.java @@ -24,6 +24,9 @@ import org.apache.flink.runtime.state.StateUtil; import org.apache.flink.util.Preconditions; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + import java.util.HashMap; import java.util.Map; import java.util.Set; @@ -67,6 +70,7 @@ public TaskStateSnapshot(Map subtaskStatesByOp /** * Returns the subtask state for the given operator id (or null if not contained). */ + @Nullable public OperatorSubtaskState getSubtaskStateByOperatorID(OperatorID operatorID) { return subtaskStatesByOperatorID.get(operatorID); } @@ -75,7 +79,10 @@ public OperatorSubtaskState getSubtaskStateByOperatorID(OperatorID operatorID) { * Maps the given operator id to the given subtask state. Returns the subtask state of a previous mapping, if such * a mapping existed or null otherwise. */ - public OperatorSubtaskState putSubtaskStateByOperatorID(OperatorID operatorID, OperatorSubtaskState state) { + public OperatorSubtaskState putSubtaskStateByOperatorID( + @Nonnull OperatorID operatorID, + @Nonnull OperatorSubtaskState state) { + return subtaskStatesByOperatorID.put(operatorID, Preconditions.checkNotNull(state)); } @@ -86,6 +93,18 @@ public Set> getSubtaskStateMappings( return subtaskStatesByOperatorID.entrySet(); } + /** + * Returns true if at least one {@link OperatorSubtaskState} in subtaskStatesByOperatorID has state. + */ + public boolean hasState() { + for (OperatorSubtaskState operatorSubtaskState : subtaskStatesByOperatorID.values()) { + if (operatorSubtaskState != null && operatorSubtaskState.hasState()) { + return true; + } + } + return false; + } + @Override public void discardState() throws Exception { StateUtil.bestEffortDiscardAllStateObjects(subtaskStatesByOperatorID.values()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/Savepoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/Savepoint.java index d7966e6c175d6..468b12f3d5a88 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/Savepoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/Savepoint.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.checkpoint.MasterState; import org.apache.flink.runtime.checkpoint.OperatorState; import org.apache.flink.runtime.checkpoint.TaskState; +import org.apache.flink.util.Disposable; import java.util.Collection; @@ -36,7 +37,7 @@ * *

Savepoints are serialized via a {@link SavepointSerializer}. */ -public interface Savepoint extends Versioned { +public interface Savepoint extends Disposable, Versioned { /** * Returns the checkpoint ID of the savepoint. @@ -74,9 +75,4 @@ public interface Savepoint extends Versioned { */ Collection getOperatorStates(); - /** - * Disposes the savepoint. - */ - void dispose() throws Exception; - } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java index c26c983fb93a2..b3e6e895a7b6d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV1Serializer.java @@ -25,11 +25,12 @@ import org.apache.flink.runtime.checkpoint.TaskState; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.state.ChainedStateHandle; +import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupRangeOffsets; import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; -import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.filesystem.FileStateHandle; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; @@ -256,7 +257,7 @@ public static KeyedStateHandle deserializeKeyedStateHandle(DataInputStream dis) @VisibleForTesting public static void serializeOperatorStateHandle( - OperatorStateHandle stateHandle, DataOutputStream dos) throws IOException { + OperatorStateHandle stateHandle, DataOutputStream dos) throws IOException { if (stateHandle != null) { dos.writeByte(PARTITIONABLE_OPERATOR_STATE_HANDLE); @@ -309,7 +310,7 @@ public static OperatorStateHandle deserializeOperatorStateHandle( offsetsMap.put(key, metaInfo); } StreamStateHandle stateHandle = deserializeStreamStateHandle(dis); - return new OperatorStateHandle(offsetsMap, stateHandle); + return new OperatorStreamStateHandle(offsetsMap, stateHandle); } else { throw new IllegalStateException("Reading invalid OperatorStateHandle, type: " + type); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2.java index 9e406dfe44c64..bd3bfae41ad4d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2.java @@ -27,8 +27,8 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.ChainedStateHandle; -import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.util.Preconditions; import java.util.Collection; 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 5636a52aba71e..faee588c48d7c 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 @@ -23,12 +23,13 @@ import org.apache.flink.runtime.checkpoint.OperatorState; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.IncrementalKeyedStateHandle; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupRangeOffsets; import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; -import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; import org.apache.flink.runtime.state.StateHandleID; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.filesystem.FileStateHandle; @@ -433,7 +434,7 @@ private static KeyedStateHandle deserializeKeyedStateHandle(DataInputStream dis) } private static void serializeOperatorStateHandle( - OperatorStateHandle stateHandle, DataOutputStream dos) throws IOException { + OperatorStateHandle stateHandle, DataOutputStream dos) throws IOException { if (stateHandle != null) { dos.writeByte(PARTITIONABLE_OPERATOR_STATE_HANDLE); @@ -485,7 +486,7 @@ private static OperatorStateHandle deserializeOperatorStateHandle( offsetsMap.put(key, metaInfo); } StreamStateHandle stateHandle = deserializeStreamStateHandle(dis); - return new OperatorStateHandle(offsetsMap, stateHandle); + return new OperatorStreamStateHandle(offsetsMap, stateHandle); } else { throw new IllegalStateException("Reading invalid OperatorStateHandle, type: " + type); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index a50479948eb05..946f6e42cf241 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -1240,7 +1240,7 @@ private void releaseAssignedResource(@Nullable Throwable cause) { */ @VisibleForTesting public CompletableFuture> calculatePreferredLocations(LocationPreferenceConstraint locationPreferenceConstraint) { - final Collection> preferredLocationFutures = getVertex().getPreferredLocationsBasedOnInputs(); + final Collection> preferredLocationFutures = getVertex().getPreferredLocations(); final CompletableFuture> preferredLocationsFuture; switch(locationPreferenceConstraint) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/InMemorySorter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/InMemorySorter.java index a47041b7a5a1a..6c3577d050907 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/InMemorySorter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/InMemorySorter.java @@ -18,15 +18,16 @@ package org.apache.flink.runtime.operators.sort; -import java.io.IOException; - import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView; +import org.apache.flink.util.Disposable; import org.apache.flink.util.MutableObjectIterator; +import java.io.IOException; + /** * */ -public interface InMemorySorter extends IndexedSortable { +public interface InMemorySorter extends IndexedSortable, Disposable { /** * Resets the sort buffer back to the state where it is empty. All contained data is discarded. @@ -39,11 +40,12 @@ public interface InMemorySorter extends IndexedSortable { * @return True, if no record is contained, false otherwise. */ boolean isEmpty(); - + /** * Disposes the sorter. * This method does not release the memory segments used by the sorter. */ + @Override void dispose(); /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java index 48982922bc50b..123cec2b4341f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/AbstractKeyedStateBackend.java @@ -51,6 +51,7 @@ import java.io.Closeable; import java.io.IOException; +import java.util.Collection; import java.util.HashMap; import java.util.stream.Stream; @@ -62,8 +63,11 @@ * * @param Type of the key by which state is keyed. */ -public abstract class AbstractKeyedStateBackend - implements KeyedStateBackend, Snapshotable, Closeable, CheckpointListener { +public abstract class AbstractKeyedStateBackend implements + KeyedStateBackend, + Snapshotable, Collection>, + Closeable, + CheckpointListener { /** {@link TypeSerializer} for our key. */ protected final TypeSerializer keySerializer; @@ -110,7 +114,7 @@ public AbstractKeyedStateBackend( KeyGroupRange keyGroupRange, ExecutionConfig executionConfig) { - this.kvStateRegistry = kvStateRegistry; //Preconditions.checkNotNull(kvStateRegistry); + this.kvStateRegistry = kvStateRegistry; this.keySerializer = Preconditions.checkNotNull(keySerializer); this.numberOfKeyGroups = Preconditions.checkNotNull(numberOfKeyGroups); this.userCodeClassLoader = Preconditions.checkNotNull(userCodeClassLoader); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamFactory.java index e8997cd591206..3c6794e246559 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamFactory.java @@ -20,6 +20,8 @@ import org.apache.flink.core.fs.FSDataOutputStream; +import javax.annotation.Nullable; + import java.io.IOException; import java.io.OutputStream; @@ -38,9 +40,9 @@ public interface CheckpointStreamFactory { * @param scope The state's scope, whether it is exclusive or shared. * @return An output stream that writes state for the given checkpoint. * - * @throws Exception Exceptions may occur while creating the stream and should be forwarded. + * @throws IOException Exceptions may occur while creating the stream and should be forwarded. */ - CheckpointStateOutputStream createCheckpointStateOutputStream(CheckpointedStateScope scope) throws Exception; + CheckpointStateOutputStream createCheckpointStateOutputStream(CheckpointedStateScope scope) throws IOException; /** * A dedicated output stream that produces a {@link StreamStateHandle} when closed. @@ -69,6 +71,7 @@ abstract class CheckpointStateOutputStream extends FSDataOutputStream { * @return A state handle that can create an input stream producing the data written to this stream. * @throws IOException Thrown, if the stream cannot be closed. */ + @Nullable public abstract StreamStateHandle closeAndGetHandle() throws IOException; /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamWithResultProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamWithResultProvider.java new file mode 100644 index 0000000000000..41c35d4614ab6 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointStreamWithResultProvider.java @@ -0,0 +1,221 @@ +/* + * 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.runtime.state; + +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.state.filesystem.FileBasedStateOutputStream; +import org.apache.flink.util.ExceptionUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnegative; +import javax.annotation.Nonnull; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.util.UUID; + +/** + * Interface that provides access to a CheckpointStateOutputStream and a method to provide the {@link SnapshotResult}. + * This abstracts from different ways that a result is obtained from checkpoint output streams. + */ +public interface CheckpointStreamWithResultProvider extends Closeable { + + Logger LOG = LoggerFactory.getLogger(CheckpointStreamWithResultProvider.class); + + /** + * Closes the stream ans returns a snapshot result with the stream handle(s). + */ + @Nonnull + SnapshotResult closeAndFinalizeCheckpointStreamResult() throws IOException; + + /** + * Returns the encapsulated output stream. + */ + @Nonnull + CheckpointStreamFactory.CheckpointStateOutputStream getCheckpointOutputStream(); + + @Override + default void close() throws IOException { + getCheckpointOutputStream().close(); + } + + /** + * Implementation of {@link CheckpointStreamWithResultProvider} that only creates the + * primary/remote/jm-owned state. + */ + class PrimaryStreamOnly implements CheckpointStreamWithResultProvider { + + @Nonnull + private final CheckpointStreamFactory.CheckpointStateOutputStream outputStream; + + public PrimaryStreamOnly(@Nonnull CheckpointStreamFactory.CheckpointStateOutputStream outputStream) { + this.outputStream = outputStream; + } + + @Nonnull + @Override + public SnapshotResult closeAndFinalizeCheckpointStreamResult() throws IOException { + return SnapshotResult.of(outputStream.closeAndGetHandle()); + } + + @Nonnull + @Override + public CheckpointStreamFactory.CheckpointStateOutputStream getCheckpointOutputStream() { + return outputStream; + } + } + + /** + * Implementation of {@link CheckpointStreamWithResultProvider} that creates both, the + * primary/remote/jm-owned state and the secondary/local/tm-owned state. + */ + class PrimaryAndSecondaryStream implements CheckpointStreamWithResultProvider { + + private static final Logger LOG = LoggerFactory.getLogger(PrimaryAndSecondaryStream.class); + + @Nonnull + private final DuplicatingCheckpointOutputStream outputStream; + + public PrimaryAndSecondaryStream( + @Nonnull CheckpointStreamFactory.CheckpointStateOutputStream primaryOut, + CheckpointStreamFactory.CheckpointStateOutputStream secondaryOut) throws IOException { + this(new DuplicatingCheckpointOutputStream(primaryOut, secondaryOut)); + } + + PrimaryAndSecondaryStream(@Nonnull DuplicatingCheckpointOutputStream outputStream) { + this.outputStream = outputStream; + } + + @Nonnull + @Override + public SnapshotResult closeAndFinalizeCheckpointStreamResult() throws IOException { + + final StreamStateHandle primaryStreamStateHandle; + + try { + primaryStreamStateHandle = outputStream.closeAndGetPrimaryHandle(); + } catch (IOException primaryEx) { + try { + outputStream.close(); + } catch (IOException closeEx) { + primaryEx = ExceptionUtils.firstOrSuppressed(closeEx, primaryEx); + } + throw primaryEx; + } + + StreamStateHandle secondaryStreamStateHandle = null; + + try { + secondaryStreamStateHandle = outputStream.closeAndGetSecondaryHandle(); + } catch (IOException secondaryEx) { + LOG.warn("Exception from secondary/local checkpoint stream.", secondaryEx); + } + + if (primaryStreamStateHandle != null) { + if (secondaryStreamStateHandle != null) { + return SnapshotResult.withLocalState(primaryStreamStateHandle, secondaryStreamStateHandle); + } else { + return SnapshotResult.of(primaryStreamStateHandle); + } + } else { + return SnapshotResult.empty(); + } + } + + @Nonnull + @Override + public DuplicatingCheckpointOutputStream getCheckpointOutputStream() { + return outputStream; + } + } + + @Nonnull + static CheckpointStreamWithResultProvider createSimpleStream( + @Nonnull CheckpointedStateScope checkpointedStateScope, + @Nonnull CheckpointStreamFactory primaryStreamFactory) throws IOException { + + CheckpointStreamFactory.CheckpointStateOutputStream primaryOut = + primaryStreamFactory.createCheckpointStateOutputStream(checkpointedStateScope); + + return new CheckpointStreamWithResultProvider.PrimaryStreamOnly(primaryOut); + } + + @Nonnull + static CheckpointStreamWithResultProvider createDuplicatingStream( + @Nonnegative long checkpointId, + @Nonnull CheckpointedStateScope checkpointedStateScope, + @Nonnull CheckpointStreamFactory primaryStreamFactory, + @Nonnull LocalRecoveryDirectoryProvider secondaryStreamDirProvider) throws IOException { + + CheckpointStreamFactory.CheckpointStateOutputStream primaryOut = + primaryStreamFactory.createCheckpointStateOutputStream(checkpointedStateScope); + + try { + File outFile = new File( + secondaryStreamDirProvider.subtaskSpecificCheckpointDirectory(checkpointId), + String.valueOf(UUID.randomUUID())); + Path outPath = new Path(outFile.toURI()); + + CheckpointStreamFactory.CheckpointStateOutputStream secondaryOut = + new FileBasedStateOutputStream(outPath.getFileSystem(), outPath); + + return new CheckpointStreamWithResultProvider.PrimaryAndSecondaryStream(primaryOut, secondaryOut); + } catch (IOException secondaryEx) { + LOG.warn("Exception when opening secondary/local checkpoint output stream. " + + "Continue only with the primary stream.", secondaryEx); + } + + return new CheckpointStreamWithResultProvider.PrimaryStreamOnly(primaryOut); + } + + + /** + * Helper method that takes a {@link SnapshotResult} and a {@link KeyGroupRangeOffsets} and + * creates a {@link SnapshotResult} by combining the key groups offsets with all the + * present stream state handles. + */ + @Nonnull + static SnapshotResult toKeyedStateHandleSnapshotResult( + @Nonnull SnapshotResult snapshotResult, + @Nonnull KeyGroupRangeOffsets keyGroupRangeOffsets) { + + StreamStateHandle jobManagerOwnedSnapshot = snapshotResult.getJobManagerOwnedSnapshot(); + + if (jobManagerOwnedSnapshot != null) { + + KeyedStateHandle jmKeyedState = new KeyGroupsStateHandle(keyGroupRangeOffsets, jobManagerOwnedSnapshot); + StreamStateHandle taskLocalSnapshot = snapshotResult.getTaskLocalSnapshot(); + + if (taskLocalSnapshot != null) { + + KeyedStateHandle localKeyedState = new KeyGroupsStateHandle(keyGroupRangeOffsets, taskLocalSnapshot); + return SnapshotResult.withLocalState(jmKeyedState, localKeyedState); + } else { + + return SnapshotResult.of(jmKeyedState); + } + } else { + + return SnapshotResult.empty(); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java index 266483f93e940..01a397ae6c1be 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java @@ -20,10 +20,10 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.state.BroadcastState; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.MapStateDescriptor; -import org.apache.flink.api.common.state.BroadcastState; import org.apache.flink.api.common.typeutils.CompatibilityResult; import org.apache.flink.api.common.typeutils.CompatibilityUtil; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -45,6 +45,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; + import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; @@ -133,7 +135,7 @@ public class DefaultOperatorStateBackend implements OperatorStateBackend { public DefaultOperatorStateBackend( ClassLoader userClassLoader, ExecutionConfig executionConfig, - boolean asynchronousSnapshots) throws IOException { + boolean asynchronousSnapshots) { this.closeStreamOnCancelRegistry = new CloseableRegistry(); this.userClassloader = Preconditions.checkNotNull(userClassLoader); @@ -299,7 +301,7 @@ public ListState getSerializableListState(String sta // ------------------------------------------------------------------------------------------- @Override - public RunnableFuture snapshot( + public RunnableFuture> snapshot( final long checkpointId, final long timestamp, final CheckpointStreamFactory streamFactory, @@ -308,7 +310,7 @@ public RunnableFuture snapshot( final long syncStartTime = System.currentTimeMillis(); if (registeredOperatorStates.isEmpty() && registeredBroadcastStates.isEmpty()) { - return DoneFuture.nullValue(); + return DoneFuture.of(SnapshotResult.empty()); } final Map> registeredOperatorStatesDeepCopies = @@ -346,8 +348,8 @@ public RunnableFuture snapshot( } // implementation of the async IO operation, based on FutureTask - final AbstractAsyncCallableWithResources ioCallable = - new AbstractAsyncCallableWithResources() { + final AbstractAsyncCallableWithResources> ioCallable = + new AbstractAsyncCallableWithResources>() { CheckpointStreamFactory.CheckpointStateOutputStream out = null; @@ -357,12 +359,12 @@ protected void acquireResources() throws Exception { } @Override - protected void releaseResources() throws Exception { + protected void releaseResources() { closeOutStream(); } @Override - protected void stopOperation() throws Exception { + protected void stopOperation() { closeOutStream(); } @@ -377,8 +379,9 @@ private void closeOutStream() { } } + @Nonnull @Override - public OperatorStateHandle performOperation() throws Exception { + public SnapshotResult performOperation() throws Exception { long asyncStartTime = System.currentTimeMillis(); CheckpointStreamFactory.CheckpointStateOutputStream localOut = this.out; @@ -444,7 +447,7 @@ public OperatorStateHandle performOperation() throws Exception { StreamStateHandle stateHandle = out.closeAndGetHandle(); if (stateHandle != null) { - retValue = new OperatorStateHandle(writtenStatesMetaData, stateHandle); + retValue = new OperatorStreamStateHandle(writtenStatesMetaData, stateHandle); } } @@ -453,11 +456,12 @@ public OperatorStateHandle performOperation() throws Exception { streamFactory, Thread.currentThread(), (System.currentTimeMillis() - asyncStartTime)); } - return retValue; + return SnapshotResult.of(retValue); } }; - AsyncStoppableTaskWithCallback task = AsyncStoppableTaskWithCallback.from(ioCallable); + AsyncStoppableTaskWithCallback> task = + AsyncStoppableTaskWithCallback.from(ioCallable); if (!asynchronousSnapshots) { task.run(); @@ -469,10 +473,9 @@ public OperatorStateHandle performOperation() throws Exception { return task; } - @Override public void restore(Collection restoreSnapshots) throws Exception { - if (null == restoreSnapshots) { + if (null == restoreSnapshots || restoreSnapshots.isEmpty()) { return; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DirectoryKeyedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DirectoryKeyedStateHandle.java new file mode 100644 index 0000000000000..fc746380c83b6 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DirectoryKeyedStateHandle.java @@ -0,0 +1,109 @@ +/* + * 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.runtime.state; + +import javax.annotation.Nonnull; + +/** + * This class is a keyed state handle based on a directory. It combines a {@link DirectoryStateHandle} and a + * {@link KeyGroupRange}. + */ +public class DirectoryKeyedStateHandle implements KeyedStateHandle { + + private static final long serialVersionUID = 1L; + + /** The directory state handle. */ + @Nonnull + private final DirectoryStateHandle directoryStateHandle; + + /** The key-group range. */ + @Nonnull + private final KeyGroupRange keyGroupRange; + + public DirectoryKeyedStateHandle( + @Nonnull DirectoryStateHandle directoryStateHandle, + @Nonnull KeyGroupRange keyGroupRange) { + + this.directoryStateHandle = directoryStateHandle; + this.keyGroupRange = keyGroupRange; + } + + @Nonnull + public DirectoryStateHandle getDirectoryStateHandle() { + return directoryStateHandle; + } + + @Nonnull + @Override + public KeyGroupRange getKeyGroupRange() { + return keyGroupRange; + } + + @Override + public void discardState() throws Exception { + directoryStateHandle.discardState(); + } + + @Override + public long getStateSize() { + return directoryStateHandle.getStateSize(); + } + + @Override + public KeyedStateHandle getIntersection(KeyGroupRange otherKeyGroupRange) { + return this.keyGroupRange.getIntersection(otherKeyGroupRange).getNumberOfKeyGroups() > 0 ? this : null; + } + + @Override + public void registerSharedStates(SharedStateRegistry stateRegistry) { + // Nothing to do, this is for local use only. + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + DirectoryKeyedStateHandle that = (DirectoryKeyedStateHandle) o; + + if (!getDirectoryStateHandle().equals(that.getDirectoryStateHandle())) { + return false; + } + return getKeyGroupRange().equals(that.getKeyGroupRange()); + } + + @Override + public int hashCode() { + int result = getDirectoryStateHandle().hashCode(); + result = 31 * result + getKeyGroupRange().hashCode(); + return result; + } + + @Override + public String toString() { + return "DirectoryKeyedStateHandle{" + + "directoryStateHandle=" + directoryStateHandle + + ", keyGroupRange=" + keyGroupRange + + '}'; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DirectoryStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DirectoryStateHandle.java new file mode 100644 index 0000000000000..2d087773d10b5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DirectoryStateHandle.java @@ -0,0 +1,87 @@ +/* + * 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.runtime.state; + +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; + +import javax.annotation.Nonnull; + +import java.io.IOException; + +/** + * This state handle represents a directory. This class is, for example, used to represent the directory of RocksDB's + * native checkpoint directories for local recovery. + */ +public class DirectoryStateHandle implements StateObject { + + /** Serial version. */ + private static final long serialVersionUID = 1L; + + /** The path that describes the directory. */ + @Nonnull + private final Path directory; + + public DirectoryStateHandle(@Nonnull Path directory) { + this.directory = directory; + } + + @Override + public void discardState() throws IOException { + FileSystem fileSystem = directory.getFileSystem(); + fileSystem.delete(directory, true); + } + + @Override + public long getStateSize() { + // For now, we will not report any size, but in the future this could (if needed) return the total dir size. + return 0L; // unknown + } + + @Nonnull + public Path getDirectory() { + return directory; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + DirectoryStateHandle that = (DirectoryStateHandle) o; + + return directory.equals(that.directory); + } + + @Override + public int hashCode() { + return directory.hashCode(); + } + + @Override + public String toString() { + return "DirectoryStateHandle{" + + "directory=" + directory + + '}'; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DoneFuture.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DoneFuture.java index d2d808dc66001..556212f6c2140 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DoneFuture.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DoneFuture.java @@ -17,11 +17,10 @@ */ package org.apache.flink.runtime.state; -import java.util.concurrent.ExecutionException; +import javax.annotation.Nullable; import java.util.concurrent.Future; import java.util.concurrent.RunnableFuture; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; /** * A {@link Future} that is always done and will just yield the object that was given at creation @@ -31,11 +30,10 @@ */ public class DoneFuture implements RunnableFuture { - private static final DoneFuture NULL_FUTURE = new DoneFuture(null); - + @Nullable private final T payload; - public DoneFuture(T payload) { + protected DoneFuture(@Nullable T payload) { this.payload = payload; } @@ -55,14 +53,14 @@ public boolean isDone() { } @Override - public T get() throws InterruptedException, ExecutionException { + public T get() { return payload; } @Override public T get( long timeout, - TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + TimeUnit unit) { return get(); } @@ -71,8 +69,8 @@ public void run() { } - @SuppressWarnings("unchecked") - public static DoneFuture nullValue() { - return (DoneFuture) NULL_FUTURE; + + public static DoneFuture of(@Nullable T result) { + return new DoneFuture<>(result); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DuplicatingCheckpointOutputStream.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DuplicatingCheckpointOutputStream.java new file mode 100644 index 0000000000000..259900cbf05c9 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DuplicatingCheckpointOutputStream.java @@ -0,0 +1,283 @@ +/* + * 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.runtime.state; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; + +import java.io.IOException; + +/** + * A CheckpointStateOutputStream that wraps a primary and a secondary CheckpointStateOutputStream and duplicates + * all writes into both streams. This stream applies buffering to reduce the amount of dual-method calling. Furthermore, + * exceptions that happen in interactions with the secondary stream are not exposed, until the user calls + * {@link #closeAndGetSecondaryHandle()}. In contrast to that, exceptions from interactions with the primary stream + * are immediately returned to the user. This class is used to write state for local recovery as a local (secondary) + * copy of the (primary) state snapshot that is written to a (slower but highly-available) remote filesystem. + */ +public class DuplicatingCheckpointOutputStream extends CheckpointStreamFactory.CheckpointStateOutputStream { + + /** Default buffer size of 8KB. */ + private static final int DEFAULT_BUFFER_SIZER = 8 * 1024; + + /** Write buffer. */ + private final byte[] buffer; + + /** Position in the write buffer. */ + private int bufferIdx; + + /** + * Primary stream for writing the checkpoint data. Failures from this stream are forwarded. + */ + private final CheckpointStreamFactory.CheckpointStateOutputStream primaryOutputStream; + + /** + * Primary stream for writing the checkpoint data. Failures from this stream are not forwarded until + * {@link #closeAndGetSecondaryHandle()}. + */ + private final CheckpointStreamFactory.CheckpointStateOutputStream secondaryOutputStream; + + /** + * Stores a potential exception that occurred while interacting with {@link #secondaryOutputStream} + */ + private Exception secondaryStreamException; + + public DuplicatingCheckpointOutputStream( + CheckpointStreamFactory.CheckpointStateOutputStream primaryOutputStream, + CheckpointStreamFactory.CheckpointStateOutputStream secondaryOutputStream) throws IOException { + this(primaryOutputStream, secondaryOutputStream, DEFAULT_BUFFER_SIZER); + } + + public DuplicatingCheckpointOutputStream( + CheckpointStreamFactory.CheckpointStateOutputStream primaryOutputStream, + CheckpointStreamFactory.CheckpointStateOutputStream secondaryOutputStream, + int bufferSize) throws IOException { + + this.primaryOutputStream = Preconditions.checkNotNull(primaryOutputStream); + this.secondaryOutputStream = Preconditions.checkNotNull(secondaryOutputStream); + + this.buffer = new byte[bufferSize]; + this.bufferIdx = 0; + + this.secondaryStreamException = null; + + checkForAlignedStreamPositions(); + } + + @Override + public void write(int b) throws IOException { + + if (buffer.length <= bufferIdx) { + flushInternalBuffer(); + } + + buffer[bufferIdx] = (byte) b; + ++bufferIdx; + } + + @Override + public void write(byte[] b) throws IOException { + + write(b, 0, b.length); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + + if (buffer.length <= len) { + + flushInternalBuffer(); + writeThroughInternal(b, off, len); + } else { + + if (buffer.length < len + bufferIdx) { + flushInternalBuffer(); + } + + System.arraycopy(b, off, buffer, bufferIdx, len); + bufferIdx += len; + } + } + + @Override + public long getPos() throws IOException { + final long referencePos = primaryOutputStream.getPos(); + return referencePos + bufferIdx; + } + + @Override + public void flush() throws IOException { + + flushInternalBuffer(); + primaryOutputStream.flush(); + + if (secondaryStreamException == null) { + try { + secondaryOutputStream.flush(); + } catch (Exception flushEx) { + handleSecondaryStreamOnException(flushEx); + } + } + } + + @Override + public void sync() throws IOException { + + flushInternalBuffer(); + primaryOutputStream.sync(); + + if (secondaryStreamException == null) { + try { + secondaryOutputStream.sync(); + } catch (Exception syncEx) { + handleSecondaryStreamOnException(syncEx); + } + } + } + + @Override + public void close() throws IOException { + + Exception exCollector = null; + + try { + flushInternalBuffer(); + } catch (Exception flushEx) { + exCollector = flushEx; + } + + try { + primaryOutputStream.close(); + } catch (Exception closeEx) { + exCollector = ExceptionUtils.firstOrSuppressed(closeEx, exCollector); + } + + if (secondaryStreamException == null) { + try { + secondaryOutputStream.close(); + } catch (Exception closeEx) { + handleSecondaryStreamOnException(closeEx); + } + } + + if (exCollector != null) { + throw new IOException("Exception while closing duplicating stream.", exCollector); + } + } + + private void checkForAlignedStreamPositions() throws IOException { + + if (secondaryStreamException != null) { + return; + } + + final long primaryPos = primaryOutputStream.getPos(); + + try { + final long secondaryPos = secondaryOutputStream.getPos(); + + if (primaryPos != secondaryPos) { + handleSecondaryStreamOnException( + new IOException("Stream positions are out of sync between primary stream and secondary stream. " + + "Reported positions are " + primaryPos + " (primary) and " + secondaryPos + " (secondary).")); + } + } catch (Exception posEx) { + handleSecondaryStreamOnException(posEx); + } + } + + private void flushInternalBuffer() throws IOException { + + if (bufferIdx > 0) { + writeThroughInternal(buffer, 0, bufferIdx); + bufferIdx = 0; + } + } + + private void writeThroughInternal(byte[] b, int off, int len) throws IOException { + + primaryOutputStream.write(b, off, len); + + if (secondaryStreamException == null) { + try { + secondaryOutputStream.write(b, off, len); + } catch (Exception writeEx) { + handleSecondaryStreamOnException(writeEx); + } + } + } + + private void handleSecondaryStreamOnException(Exception ex) { + + Preconditions.checkState(secondaryStreamException == null, + "Secondary stream already failed from previous exception!"); + + try { + secondaryOutputStream.close(); + } catch (Exception closeEx) { + ex = ExceptionUtils.firstOrSuppressed(closeEx, ex); + } + + secondaryStreamException = Preconditions.checkNotNull(ex); + } + + @Nullable + @Override + public StreamStateHandle closeAndGetHandle() throws IOException { + return closeAndGetPrimaryHandle(); + } + + /** + * Returns the state handle from the {@link #primaryOutputStream}. + */ + public StreamStateHandle closeAndGetPrimaryHandle() throws IOException { + flushInternalBuffer(); + return primaryOutputStream.closeAndGetHandle(); + } + + /** + * Returns the state handle from the {@link #secondaryOutputStream}. Also reports suppressed exceptions from earlier + * interactions with that stream. + */ + public StreamStateHandle closeAndGetSecondaryHandle() throws IOException { + if (secondaryStreamException == null) { + flushInternalBuffer(); + return secondaryOutputStream.closeAndGetHandle(); + } else { + throw new IOException("Secondary stream previously failed exceptionally", secondaryStreamException); + } + } + + public Exception getSecondaryStreamException() { + return secondaryStreamException; + } + + @VisibleForTesting + CheckpointStreamFactory.CheckpointStateOutputStream getPrimaryOutputStream() { + return primaryOutputStream; + } + + @VisibleForTesting + CheckpointStreamFactory.CheckpointStateOutputStream getSecondaryOutputStream() { + return secondaryOutputStream; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java new file mode 100644 index 0000000000000..f80a8ce914d79 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalLocalKeyedStateHandle.java @@ -0,0 +1,145 @@ +/* + * 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.runtime.state; + +import org.apache.flink.util.ExceptionUtils; + +import javax.annotation.Nonnegative; +import javax.annotation.Nonnull; + +import java.util.Set; +import java.util.UUID; + +/** + * State handle for local copies of {@link IncrementalKeyedStateHandle}. Consists of a {@link DirectoryStateHandle} that + * represents the directory of the native RocksDB snapshot, the key groups, and a stream state handle for Flink's state + * meta data file. + */ +public class IncrementalLocalKeyedStateHandle extends DirectoryKeyedStateHandle { + + private static final long serialVersionUID = 1L; + + /** Id of the checkpoint that created this state handle. */ + @Nonnegative + private final long checkpointId; + + /** UUID to identify the backend which created this state handle. */ + @Nonnull + private final UUID backendIdentifier; + + /** Handle to Flink's state meta data. */ + @Nonnull + private final StreamStateHandle metaDataState; + + /** Set with the ids of all shared state handles created by the checkpoint. */ + @Nonnull + private final Set sharedStateHandleIDs; + + public IncrementalLocalKeyedStateHandle( + @Nonnull UUID backendIdentifier, + @Nonnegative long checkpointId, + @Nonnull DirectoryStateHandle directoryStateHandle, + @Nonnull KeyGroupRange keyGroupRange, + @Nonnull StreamStateHandle metaDataState, + @Nonnull Set sharedStateHandleIDs) { + + super(directoryStateHandle, keyGroupRange); + this.backendIdentifier = backendIdentifier; + this.checkpointId = checkpointId; + this.metaDataState = metaDataState; + this.sharedStateHandleIDs = sharedStateHandleIDs; + } + + @Nonnull + public StreamStateHandle getMetaDataState() { + return metaDataState; + } + + public long getCheckpointId() { + return checkpointId; + } + + @Nonnull + public UUID getBackendIdentifier() { + return backendIdentifier; + } + + @Nonnull + public Set getSharedStateHandleIDs() { + return sharedStateHandleIDs; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + + IncrementalLocalKeyedStateHandle that = (IncrementalLocalKeyedStateHandle) o; + + return getMetaDataState().equals(that.getMetaDataState()); + } + + @Override + public void discardState() throws Exception { + + Exception collectedEx = null; + + try { + super.discardState(); + } catch (Exception e) { + collectedEx = e; + } + + try { + metaDataState.discardState(); + } catch (Exception e) { + collectedEx = ExceptionUtils.firstOrSuppressed(e, collectedEx); + } + + if (collectedEx != null) { + throw collectedEx; + } + } + + @Override + public long getStateSize() { + return super.getStateSize() + metaDataState.getStateSize(); + } + + @Override + public int hashCode() { + int result = super.hashCode(); + result = 31 * result + getMetaDataState().hashCode(); + return result; + } + + @Override + public String toString() { + return "IncrementalLocalKeyedStateHandle{" + + "metaDataState=" + metaDataState + + "} " + super.toString(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java index cbe40ee7b1080..3326a814609ff 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedStateBackend.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.state.StateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.state.heap.InternalKeyContext; +import org.apache.flink.util.Disposable; import java.util.stream.Stream; @@ -30,7 +31,7 @@ * * @param The key by which state is keyed. */ -public interface KeyedStateBackend extends InternalKeyContext { +public interface KeyedStateBackend extends InternalKeyContext, Disposable { /** * Sets the current key that is used for partitioned state. @@ -102,8 +103,6 @@ S getPartitionedState( TypeSerializer namespaceSerializer, StateDescriptor stateDescriptor) throws Exception; - /** - * Closes the backend and releases all resources. - */ + @Override void dispose(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalRecoveryConfig.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalRecoveryConfig.java new file mode 100644 index 0000000000000..c97fa0b57d450 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalRecoveryConfig.java @@ -0,0 +1,99 @@ +/* + * 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.runtime.state; + +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; + +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +/** + * This class encapsulates the completed configuration for local recovery, i.e. the root + * directories into which all file-based snapshots can be written and the general mode for the local recover feature. + */ +public class LocalRecoveryConfig { + + /** + * Enum over modes of local recovery: + *

    + *
  • DISABLED: disables local recovery. + *
  • ENABLE_FILE_BASED: enables local recovery in a variant that is based on local files. + *
+ */ + public enum LocalRecoveryMode { + DISABLED, + ENABLE_FILE_BASED; + + /** + * Extracts the {@link LocalRecoveryMode} from the given configuration. Defaults to LocalRecoveryMode.DISABLED + * if no configuration value is specified or parsing the value resulted in an exception. + * + * @param configuration the configuration that specifies the value for the local recovery mode. + * @return the local recovery mode as found in the config, or LocalRecoveryMode.DISABLED if no mode was + * configured or the specified mode could not be parsed. + */ + @Nonnull + public static LocalRecoveryMode fromConfig(@Nonnull Configuration configuration) { + String localRecoveryConfString = configuration.getString(CheckpointingOptions.LOCAL_RECOVERY); + try { + return LocalRecoveryConfig.LocalRecoveryMode.valueOf(localRecoveryConfString); + } catch (IllegalArgumentException ex) { + LoggerFactory.getLogger(LocalRecoveryConfig.class).warn( + "Exception while parsing configuration of local recovery mode. Local recovery will be disabled.", + ex); + return LocalRecoveryConfig.LocalRecoveryMode.DISABLED; + } + } + } + + /** The local recovery mode. */ + @Nonnull + private final LocalRecoveryMode localRecoveryMode; + + /** Encapsulates the root directories and the subtask-specific path. */ + @Nonnull + private final LocalRecoveryDirectoryProvider localStateDirectories; + + public LocalRecoveryConfig( + @Nonnull LocalRecoveryMode localRecoveryMode, + @Nonnull LocalRecoveryDirectoryProvider directoryProvider) { + this.localRecoveryMode = localRecoveryMode; + this.localStateDirectories = directoryProvider; + } + + @Nonnull + public LocalRecoveryMode getLocalRecoveryMode() { + return localRecoveryMode; + } + + @Nonnull + public LocalRecoveryDirectoryProvider getLocalStateDirectoryProvider() { + return localStateDirectories; + } + + @Override + public String toString() { + return "LocalRecoveryConfig{" + + "localRecoveryMode=" + localRecoveryMode + + ", localStateDirectories=" + localStateDirectories + + '}'; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalRecoveryDirectoryProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalRecoveryDirectoryProvider.java new file mode 100644 index 0000000000000..3f7ab5ca3c55d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalRecoveryDirectoryProvider.java @@ -0,0 +1,81 @@ +/* + * 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.runtime.state; + +import java.io.File; +import java.io.Serializable; + +/** + * Provides directories for local recovery. It offers access to the allocation base directories (i.e. the root + * directories for all local state that is created under the same allocation id) and the subtask-specific paths, which + * contain the local state for one subtask. Access by checkpoint id rotates over all root directory indexes, in case + * that there is more than one. Selection methods are provided to pick the directory under a certain index. Directory + * structures are of the following shape: + * + *

+ * |-----allocationBaseDirectory------|
+ * |-----subtaskBaseDirectory--------------------------------------|
+ * |-----subtaskSpecificCheckpointDirectory------------------------------|
+ *
+ * ../local_state_root_1/allocation_id/job_id/vertex_id_subtask_idx/chk_1/(state)
+ * ../local_state_root_2/allocation_id/job_id/vertex_id_subtask_idx/chk_2/(state)
+ *
+ * (...)
+ * 

+ */ +public interface LocalRecoveryDirectoryProvider extends Serializable { + + /** + * Returns the local state allocation base directory for given checkpoint id w.r.t. our rotation + * over all available allocation base directories. + */ + File allocationBaseDirectory(long checkpointId); + + /** + * Returns the local state directory for the owning subtask the given checkpoint id w.r.t. our rotation over all + * available available allocation base directories. This directory is contained in the directory returned by + * {@link #allocationBaseDirectory(long)} for the same checkpoint id. + */ + File subtaskBaseDirectory(long checkpointId); + + /** + * Returns the local state directory for the specific operator subtask and the given checkpoint id w.r.t. our + * rotation over all available root dirs. This directory is contained in the directory returned by + * {@link #subtaskBaseDirectory(long)} for the same checkpoint id. + */ + File subtaskSpecificCheckpointDirectory(long checkpointId); + + /** + * Returns a specific allocation base directory. The index must be between 0 (incl.) and + * {@link #allocationBaseDirsCount()} (excl.). + */ + File selectAllocationBaseDirectory(int idx); + + /** + * Returns a specific subtask base directory. The index must be between 0 (incl.) and + * {@link #allocationBaseDirsCount()} (excl.). This directory is direct a child of + * {@link #selectSubtaskBaseDirectory(int)} given the same index. + */ + File selectSubtaskBaseDirectory(int idx); + + /** + * Returns the total number of allocation base directories. + */ + int allocationBaseDirsCount(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalRecoveryDirectoryProviderImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalRecoveryDirectoryProviderImpl.java new file mode 100644 index 0000000000000..ef50929429dd5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalRecoveryDirectoryProviderImpl.java @@ -0,0 +1,138 @@ +/* + * 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.runtime.state; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnegative; +import javax.annotation.Nonnull; + +import java.io.File; +import java.nio.file.Paths; +import java.util.Arrays; + +/** + * Implementation of {@link LocalRecoveryDirectoryProvider}. + */ +public class LocalRecoveryDirectoryProviderImpl implements LocalRecoveryDirectoryProvider { + + /** Serial version. */ + private static final long serialVersionUID = 1L; + + /** Logger for this class. */ + private static final Logger LOG = LoggerFactory.getLogger(LocalRecoveryDirectoryProviderImpl.class); + + /** All available root directories that this can potentially deliver. */ + @Nonnull + private final File[] allocationBaseDirs; + + /** JobID of the owning job. */ + @Nonnull + private final JobID jobID; + + /** JobVertexID of the owning task. */ + @Nonnull + private final JobVertexID jobVertexID; + + /** Index of the owning subtask. */ + @Nonnegative + private final int subtaskIndex; + + public LocalRecoveryDirectoryProviderImpl( + File allocationBaseDir, + @Nonnull JobID jobID, + @Nonnull JobVertexID jobVertexID, + @Nonnegative int subtaskIndex) { + this(new File[]{allocationBaseDir}, jobID, jobVertexID, subtaskIndex); + } + + public LocalRecoveryDirectoryProviderImpl( + @Nonnull File[] allocationBaseDirs, + @Nonnull JobID jobID, + @Nonnull JobVertexID jobVertexID, + @Nonnegative int subtaskIndex) { + + Preconditions.checkArgument(allocationBaseDirs.length > 0); + this.allocationBaseDirs = allocationBaseDirs; + this.jobID = jobID; + this.jobVertexID = jobVertexID; + this.subtaskIndex = subtaskIndex; + + for (File allocationBaseDir : allocationBaseDirs) { + Preconditions.checkNotNull(allocationBaseDir); + allocationBaseDir.mkdirs(); + } + } + + @Override + public File allocationBaseDirectory(long checkpointId) { + return selectAllocationBaseDirectory((((int) checkpointId) & Integer.MAX_VALUE) % allocationBaseDirs.length); + } + + @Override + public File subtaskBaseDirectory(long checkpointId) { + return new File(allocationBaseDirectory(checkpointId), subtaskDirString()); + } + + @Override + public File subtaskSpecificCheckpointDirectory(long checkpointId) { + return new File(subtaskBaseDirectory(checkpointId), checkpointDirString(checkpointId)); + } + + @Override + public File selectAllocationBaseDirectory(int idx) { + return allocationBaseDirs[idx]; + } + + @Override + public File selectSubtaskBaseDirectory(int idx) { + return new File(selectAllocationBaseDirectory(idx), subtaskDirString()); + } + + @Override + public int allocationBaseDirsCount() { + return allocationBaseDirs.length; + } + + @Override + public String toString() { + return "LocalRecoveryDirectoryProvider{" + + "rootDirectories=" + Arrays.toString(allocationBaseDirs) + + ", jobID=" + jobID + + ", jobVertexID=" + jobVertexID + + ", subtaskIndex=" + subtaskIndex + + '}'; + } + + @VisibleForTesting + String subtaskDirString() { + return Paths.get("jid_" + jobID, "vtx_" + jobVertexID + "_sti_" + subtaskIndex).toString(); + } + + @VisibleForTesting + String checkpointDirString(long checkpointId) { + return "chk_" + checkpointId; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/MultiStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/MultiStreamStateHandle.java deleted file mode 100644 index 1960c1c95f431..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/MultiStreamStateHandle.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * 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.runtime.state; - -import org.apache.flink.core.fs.AbstractMultiFSDataInputStream; -import org.apache.flink.core.fs.FSDataInputStream; -import org.apache.flink.util.Preconditions; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; - -/** - * Wrapper class that takes multiple {@link StreamStateHandle} and makes them look like a single one. This is done by - * providing a contiguous view on all the streams of the inner handles through a wrapper stream and by summing up all - * all the meta data. - */ -public class MultiStreamStateHandle implements StreamStateHandle { - - private static final long serialVersionUID = -4588701089489569707L; - private final List stateHandles; - private final long stateSize; - - public MultiStreamStateHandle(List stateHandles) { - this.stateHandles = Preconditions.checkNotNull(stateHandles); - long calculateSize = 0L; - for(StreamStateHandle stateHandle : stateHandles) { - calculateSize += stateHandle.getStateSize(); - } - this.stateSize = calculateSize; - } - - @Override - public FSDataInputStream openInputStream() throws IOException { - return new MultiFSDataInputStream(stateHandles); - } - - @Override - public void discardState() throws Exception { - StateUtil.bestEffortDiscardAllStateObjects(stateHandles); - } - - @Override - public long getStateSize() { - return stateSize; - } - - @Override - public String toString() { - return "MultiStreamStateHandle{" + - "stateHandles=" + stateHandles + - ", stateSize=" + stateSize + - '}'; - } - - static final class MultiFSDataInputStream extends AbstractMultiFSDataInputStream { - - private final TreeMap stateHandleMap; - - public MultiFSDataInputStream(List stateHandles) throws IOException { - this.stateHandleMap = new TreeMap<>(); - this.totalPos = 0L; - long calculateSize = 0L; - for (StreamStateHandle stateHandle : stateHandles) { - stateHandleMap.put(calculateSize, stateHandle); - calculateSize += stateHandle.getStateSize(); - } - this.totalAvailable = calculateSize; - - if (totalAvailable > 0L) { - StreamStateHandle first = stateHandleMap.firstEntry().getValue(); - delegate = first.openInputStream(); - } - } - - @Override - protected FSDataInputStream getSeekedStreamForOffset(long globalStreamOffset) throws IOException { - Map.Entry handleEntry = stateHandleMap.floorEntry(globalStreamOffset); - if (handleEntry != null) { - FSDataInputStream stream = handleEntry.getValue().openInputStream(); - stream.seek(globalStreamOffset - handleEntry.getKey()); - return stream; - } - return null; - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateBackend.java index aee5226c08397..3cbb35180c050 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateBackend.java @@ -19,19 +19,22 @@ package org.apache.flink.runtime.state; import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.util.Disposable; import java.io.Closeable; +import java.util.Collection; /** * Interface that combines both, the user facing {@link OperatorStateStore} interface and the system interface * {@link Snapshotable} * */ -public interface OperatorStateBackend extends OperatorStateStore, Snapshotable, Closeable { +public interface OperatorStateBackend extends + OperatorStateStore, + Snapshotable, Collection>, + Closeable, + Disposable { - /** - * Disposes the backend and releases all resources. - */ + @Override void dispose(); - } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateCheckpointOutputStream.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateCheckpointOutputStream.java index 036aed014e2e3..ba28631c9b97d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateCheckpointOutputStream.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateCheckpointOutputStream.java @@ -71,14 +71,14 @@ OperatorStateHandle closeAndGetHandle() throws IOException { OperatorStateHandle.StateMetaInfo metaInfo = new OperatorStateHandle.StateMetaInfo( partitionOffsets.toArray(), - OperatorStateHandle.Mode.SPLIT_DISTRIBUTE); + OperatorStateHandle.Mode.SPLIT_DISTRIBUTE); offsetsMap.put(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME, metaInfo); - return new OperatorStateHandle(offsetsMap, streamStateHandle); + return new OperatorStreamStateHandle(offsetsMap, streamStateHandle); } public int getNumberOfPartitions() { return partitionOffsets.size(); } -} \ No newline at end of file +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateHandle.java index f9427ef868491..1ebdaff9c7619 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStateHandle.java @@ -27,107 +27,39 @@ import java.util.Map; /** - * State handle for partitionable operator state. Besides being a {@link StreamStateHandle}, this also provides a - * map that contains the offsets to the partitions of named states in the stream. + * Interface of a state handle for operator state. */ -public class OperatorStateHandle implements StreamStateHandle { +public interface OperatorStateHandle extends StreamStateHandle { /** - * The modes that determine how an {@link OperatorStateHandle} is assigned to tasks during restore. + * Returns a map of meta data for all contained states by their name. */ - public enum Mode { - SPLIT_DISTRIBUTE, // The operator state partitions in the state handle are split and distributed to one task each. - UNION, // The operator state partitions are UNION-ed upon restoring and sent to all tasks. - BROADCAST // The operator states are identical, as the state is produced from a broadcast stream. - } - - private static final long serialVersionUID = 35876522969227335L; + Map getStateNameToPartitionOffsets(); /** - * unique state name -> offsets for available partitions in the handle stream + * Returns an input stream to read the operator state information. */ - private final Map stateNameToPartitionOffsets; - private final StreamStateHandle delegateStateHandle; - - public OperatorStateHandle( - Map stateNameToPartitionOffsets, - StreamStateHandle delegateStateHandle) { - - this.delegateStateHandle = Preconditions.checkNotNull(delegateStateHandle); - this.stateNameToPartitionOffsets = Preconditions.checkNotNull(stateNameToPartitionOffsets); - } - - public Map getStateNameToPartitionOffsets() { - return stateNameToPartitionOffsets; - } - - @Override - public void discardState() throws Exception { - delegateStateHandle.discardState(); - } - - @Override - public long getStateSize() { - return delegateStateHandle.getStateSize(); - } - @Override - public FSDataInputStream openInputStream() throws IOException { - return delegateStateHandle.openInputStream(); - } - - public StreamStateHandle getDelegateStateHandle() { - return delegateStateHandle; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (!(o instanceof OperatorStateHandle)) { - return false; - } - - OperatorStateHandle that = (OperatorStateHandle) o; + FSDataInputStream openInputStream() throws IOException; - if (stateNameToPartitionOffsets.size() != that.stateNameToPartitionOffsets.size()) { - return false; - } - - for (Map.Entry entry : stateNameToPartitionOffsets.entrySet()) { - if (!entry.getValue().equals(that.stateNameToPartitionOffsets.get(entry.getKey()))) { - return false; - } - } - - return delegateStateHandle.equals(that.delegateStateHandle); - } - - @Override - public int hashCode() { - int result = delegateStateHandle.hashCode(); - for (Map.Entry entry : stateNameToPartitionOffsets.entrySet()) { - - int entryHash = entry.getKey().hashCode(); - if (entry.getValue() != null) { - entryHash += entry.getValue().hashCode(); - } - result = 31 * result + entryHash; - } - return result; - } + /** + * Returns the underlying stream state handle that points to the state data. + */ + StreamStateHandle getDelegateStateHandle(); - @Override - public String toString() { - return "OperatorStateHandle{" + - "stateNameToPartitionOffsets=" + stateNameToPartitionOffsets + - ", delegateStateHandle=" + delegateStateHandle + - '}'; + /** + * The modes that determine how an {@link OperatorStreamStateHandle} is assigned to tasks during restore. + */ + enum Mode { + SPLIT_DISTRIBUTE, // The operator state partitions in the state handle are split and distributed to one task each. + UNION, // The operator state partitions are UNION-ed upon restoring and sent to all tasks. + BROADCAST // The operator states are identical, as the state is produced from a broadcast stream. } - public static class StateMetaInfo implements Serializable { + /** + * Meta information about the operator state handle. + */ + class StateMetaInfo implements Serializable { private static final long serialVersionUID = 3593817615858941166L; @@ -158,10 +90,8 @@ public boolean equals(Object o) { StateMetaInfo that = (StateMetaInfo) o; - if (!Arrays.equals(getOffsets(), that.getOffsets())) { - return false; - } - return getDistributionMode() == that.getDistributionMode(); + return Arrays.equals(getOffsets(), that.getOffsets()) + && getDistributionMode() == that.getDistributionMode(); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStreamStateHandle.java new file mode 100644 index 0000000000000..39008345bd8be --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorStreamStateHandle.java @@ -0,0 +1,120 @@ +/* + * 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.runtime.state; + +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.Map; + +/** + * State handle for partitionable operator state. Besides being a {@link StreamStateHandle}, this also provides a + * map that contains the offsets to the partitions of named states in the stream. + */ +public class OperatorStreamStateHandle implements OperatorStateHandle { + + private static final long serialVersionUID = 35876522969227335L; + + /** + * unique state name -> offsets for available partitions in the handle stream + */ + private final Map stateNameToPartitionOffsets; + private final StreamStateHandle delegateStateHandle; + + public OperatorStreamStateHandle( + Map stateNameToPartitionOffsets, + StreamStateHandle delegateStateHandle) { + + this.delegateStateHandle = Preconditions.checkNotNull(delegateStateHandle); + this.stateNameToPartitionOffsets = Preconditions.checkNotNull(stateNameToPartitionOffsets); + } + + @Override + public Map getStateNameToPartitionOffsets() { + return stateNameToPartitionOffsets; + } + + @Override + public void discardState() throws Exception { + delegateStateHandle.discardState(); + } + + @Override + public long getStateSize() { + return delegateStateHandle.getStateSize(); + } + + @Override + public FSDataInputStream openInputStream() throws IOException { + return delegateStateHandle.openInputStream(); + } + + @Override + public StreamStateHandle getDelegateStateHandle() { + return delegateStateHandle; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof OperatorStreamStateHandle)) { + return false; + } + + OperatorStreamStateHandle that = (OperatorStreamStateHandle) o; + + if (stateNameToPartitionOffsets.size() != that.stateNameToPartitionOffsets.size()) { + return false; + } + + for (Map.Entry entry : stateNameToPartitionOffsets.entrySet()) { + if (!entry.getValue().equals(that.stateNameToPartitionOffsets.get(entry.getKey()))) { + return false; + } + } + + return delegateStateHandle.equals(that.delegateStateHandle); + } + + @Override + public int hashCode() { + int result = delegateStateHandle.hashCode(); + for (Map.Entry entry : stateNameToPartitionOffsets.entrySet()) { + + int entryHash = entry.getKey().hashCode(); + if (entry.getValue() != null) { + entryHash += entry.getValue().hashCode(); + } + result = 31 * result + entryHash; + } + return result; + } + + @Override + public String toString() { + return "OperatorStateHandle{" + + "stateNameToPartitionOffsets=" + stateNameToPartitionOffsets + + ", delegateStateHandle=" + delegateStateHandle + + '}'; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnapshotDirectory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnapshotDirectory.java new file mode 100644 index 0000000000000..3c0054f5ba26b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnapshotDirectory.java @@ -0,0 +1,200 @@ +/* + * 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.runtime.state; + +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicReference; + +/** + * This class represents a directory that is the target for a state snapshot. This class provides some method that + * simplify resource management when dealing with such directories, e.g. it can produce a {@link DirectoryStateHandle} + * when the snapshot is completed and disposal considers whether or not a snapshot was already completed. For a + * completed snapshot, the ownership for cleanup is transferred to the created directory state handle. For incomplete + * snapshots, calling {@link #cleanup()} will delete the underlying directory resource. + */ +public abstract class SnapshotDirectory { + + /** + * Lifecycle stages of a snapshot directory. + */ + enum State { + ONGOING, COMPLETED, DELETED + } + + /** This path describes the underlying directory for the snapshot. */ + @Nonnull + protected final Path directory; + + /** The filesystem that contains the snapshot directory. */ + @Nonnull + protected final FileSystem fileSystem; + + /** This reference tracks the lifecycle state of the snapshot directory. */ + @Nonnull + protected AtomicReference state; + + private SnapshotDirectory(@Nonnull Path directory, @Nonnull FileSystem fileSystem) { + this.directory = directory; + this.fileSystem = fileSystem; + this.state = new AtomicReference<>(State.ONGOING); + } + + private SnapshotDirectory(@Nonnull Path directory) throws IOException { + this(directory, directory.getFileSystem()); + } + + @Nonnull + public Path getDirectory() { + return directory; + } + + public boolean mkdirs() throws IOException { + return fileSystem.mkdirs(directory); + } + + @Nonnull + public FileSystem getFileSystem() { + return fileSystem; + } + + public boolean exists() throws IOException { + return fileSystem.exists(directory); + } + + /** + * List the statuses of the files/directories in the snapshot directory. + * + * @return the statuses of the files/directories in the given path. + * @throws IOException if there is a problem creating the file statuses. + */ + public FileStatus[] listStatus() throws IOException { + return fileSystem.listStatus(directory); + } + + /** + * Calling this method will attempt delete the underlying snapshot directory recursively, if the state is + * "ongoing". In this case, the state will be set to "deleted" as a result of this call. + * + * @return true if delete is successful, false otherwise. + * @throws IOException if an exception happens during the delete. + */ + public boolean cleanup() throws IOException { + return !state.compareAndSet(State.ONGOING, State.DELETED) || fileSystem.delete(directory, true); + } + + /** + * Returns true if the snapshot is marked as completed. + */ + public boolean isSnapshotCompleted() { + return State.COMPLETED == state.get(); + } + + /** + * Calling this method completes the snapshot for this snapshot directory, if possible, and creates a corresponding + * {@link DirectoryStateHandle} that points to the snapshot directory. Calling this method can change the + * lifecycle state from ONGOING to COMPLETED if the directory should no longer deleted in {@link #cleanup()}. This + * method can return Can return true if the directory is temporary and should therefore not be + * referenced in a handle. + * + * @return A directory state handle that points to the snapshot directory. Can return true if the + * directory is temporary and should therefore not be referenced in a handle. + * @throws IOException if the state of this snapshot directory object is different from "ongoing". + */ + @Nullable + public abstract DirectoryStateHandle completeSnapshotAndGetHandle() throws IOException; + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + SnapshotDirectory that = (SnapshotDirectory) o; + + return directory.equals(that.directory); + } + + @Override + public int hashCode() { + return directory.hashCode(); + } + + @Override + public String toString() { + return "SnapshotDirectory{" + + "directory=" + directory + + ", state=" + state + + '}'; + } + + /** + * Creates a temporary snapshot directory for the given path. This will always return "null" as result of + * {@link #completeSnapshotAndGetHandle()} and always attempt to delete the underlying directory in + * {@link #cleanup()}. + */ + public static SnapshotDirectory temporary(@Nonnull Path directory) throws IOException { + return new TemporarySnapshotDirectory(directory); + } + + /** + * Creates a permanent snapshot directory for the given path, which will not delete the underlying directory in + * {@link #cleanup()} after {@link #completeSnapshotAndGetHandle()} was called. + */ + public static SnapshotDirectory permanent(@Nonnull Path directory) throws IOException { + return new PermanentSnapshotDirectory(directory); + } + + private static class TemporarySnapshotDirectory extends SnapshotDirectory { + + TemporarySnapshotDirectory(@Nonnull Path directory) throws IOException { + super(directory); + } + + @Override + public DirectoryStateHandle completeSnapshotAndGetHandle() { + return null; // We return null so that directory it is not referenced by a state handle. + } + } + + private static class PermanentSnapshotDirectory extends SnapshotDirectory { + + PermanentSnapshotDirectory(@Nonnull Path directory) throws IOException { + super(directory); + } + + @Override + public DirectoryStateHandle completeSnapshotAndGetHandle() throws IOException { + if (State.COMPLETED == state.get() || state.compareAndSet(State.ONGOING, State.COMPLETED)) { + return new DirectoryStateHandle(directory); + } else { + throw new IOException("Expected state " + State.ONGOING + " but found state " + state.get()); + } + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnapshotResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnapshotResult.java new file mode 100644 index 0000000000000..e292c7374c99d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnapshotResult.java @@ -0,0 +1,120 @@ +/* + * 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.runtime.state; + +import org.apache.flink.util.ExceptionUtils; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +/** + * This class contains the combined results from the snapshot of a state backend: + *

    + *
  • A state object representing the state that will be reported to the Job Manager to acknowledge the checkpoint.
  • + *
  • A state object that represents the state for the {@link TaskLocalStateStoreImpl}.
  • + *
+ * + * Both state objects are optional and can be null, e.g. if there was no state to snapshot in the backend. A local + * state object that is not null also requires a state to report to the job manager that is not null, because the + * Job Manager always owns the ground truth about the checkpointed state. + */ +public class SnapshotResult implements StateObject { + + private static final long serialVersionUID = 1L; + + /** An singleton instance to represent an empty snapshot result. */ + private static final SnapshotResult EMPTY = new SnapshotResult<>(null, null); + + /** This is the state snapshot that will be reported to the Job Manager to acknowledge a checkpoint. */ + private final T jobManagerOwnedSnapshot; + + /** This is the state snapshot that will be reported to the Job Manager to acknowledge a checkpoint. */ + private final T taskLocalSnapshot; + + /** + * Creates a {@link SnapshotResult} for the given jobManagerOwnedSnapshot and taskLocalSnapshot. If the + * jobManagerOwnedSnapshot is null, taskLocalSnapshot must also be null. + * + * @param jobManagerOwnedSnapshot Snapshot for report to job manager. Can be null. + * @param taskLocalSnapshot Snapshot for report to local state manager. This is optional and requires + * jobManagerOwnedSnapshot to be not null if this is not also null. + */ + private SnapshotResult(T jobManagerOwnedSnapshot, T taskLocalSnapshot) { + + if (jobManagerOwnedSnapshot == null && taskLocalSnapshot != null) { + throw new IllegalStateException("Cannot report local state snapshot without corresponding remote state!"); + } + + this.jobManagerOwnedSnapshot = jobManagerOwnedSnapshot; + this.taskLocalSnapshot = taskLocalSnapshot; + } + + public T getJobManagerOwnedSnapshot() { + return jobManagerOwnedSnapshot; + } + + public T getTaskLocalSnapshot() { + return taskLocalSnapshot; + } + + @Override + public void discardState() throws Exception { + + Exception aggregatedExceptions = null; + + if (jobManagerOwnedSnapshot != null) { + try { + jobManagerOwnedSnapshot.discardState(); + } catch (Exception remoteDiscardEx) { + aggregatedExceptions = remoteDiscardEx; + } + } + + if (taskLocalSnapshot != null) { + try { + taskLocalSnapshot.discardState(); + } catch (Exception localDiscardEx) { + aggregatedExceptions = ExceptionUtils.firstOrSuppressed(localDiscardEx, aggregatedExceptions); + } + } + + if (aggregatedExceptions != null) { + throw aggregatedExceptions; + } + } + + @Override + public long getStateSize() { + return jobManagerOwnedSnapshot != null ? jobManagerOwnedSnapshot.getStateSize() : 0L; + } + + @SuppressWarnings("unchecked") + public static SnapshotResult empty() { + return (SnapshotResult) EMPTY; + } + + public static SnapshotResult of(@Nullable T jobManagerState) { + return jobManagerState != null ? new SnapshotResult<>(jobManagerState, null) : empty(); + } + + public static SnapshotResult withLocalState( + @Nonnull T jobManagerState, + @Nonnull T localState) { + return new SnapshotResult<>(jobManagerState, localState); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnapshotStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnapshotStrategy.java new file mode 100644 index 0000000000000..9139fa70ebcea --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnapshotStrategy.java @@ -0,0 +1,51 @@ +/* + * 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.runtime.state; + +import org.apache.flink.runtime.checkpoint.CheckpointOptions; + +import java.util.concurrent.RunnableFuture; + +/** + * Interface for different snapshot approaches in state backends. Implementing classes should ideally be stateless or at + * least threadsafe, i.e. this is a functional interface and is can be called in parallel by multiple checkpoints. + * + * @param type of the returned state object that represents the result of the snapshot operation. + */ +@FunctionalInterface +public interface SnapshotStrategy { + + /** + * Operation that writes a snapshot into a stream that is provided by the given {@link CheckpointStreamFactory} and + * returns a @{@link RunnableFuture} that gives a state handle to the snapshot. It is up to the implementation if + * the operation is performed synchronous or asynchronous. In the later case, the returned Runnable must be executed + * first before obtaining the handle. + * + * @param checkpointId The ID of the checkpoint. + * @param timestamp The timestamp of the checkpoint. + * @param streamFactory The factory that we can use for writing our state to streams. + * @param checkpointOptions Options for how to perform this checkpoint. + * @return A runnable future that will yield a {@link StateObject}. + */ + RunnableFuture performSnapshot( + long checkpointId, + long timestamp, + CheckpointStreamFactory streamFactory, + CheckpointOptions checkpointOptions) throws Exception; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/Snapshotable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/Snapshotable.java index c7e62f09be594..733339f7aee94 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/Snapshotable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/Snapshotable.java @@ -20,15 +20,15 @@ import org.apache.flink.runtime.checkpoint.CheckpointOptions; -import java.util.Collection; import java.util.concurrent.RunnableFuture; /** * Interface for operators that can perform snapshots of their state. * * @param Generic type of the state object that is created as handle to snapshots. + * @param Generic type of the state object that used in restore. */ -public interface Snapshotable { +public interface Snapshotable { /** * Operation that writes a snapshot into a stream that is provided by the given {@link CheckpointStreamFactory} and @@ -54,5 +54,5 @@ RunnableFuture snapshot( * * @param state the old state to restore. */ - void restore(Collection state) throws Exception; + void restore(R state) throws Exception; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSnapshotContextSynchronousImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSnapshotContextSynchronousImpl.java index 427734813b81b..44ca9d8d9bbaf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSnapshotContextSynchronousImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSnapshotContextSynchronousImpl.java @@ -23,6 +23,8 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; +import javax.annotation.Nonnull; + import java.io.Closeable; import java.io.IOException; import java.util.concurrent.RunnableFuture; @@ -31,8 +33,11 @@ * This class is a default implementation for StateSnapshotContext. */ public class StateSnapshotContextSynchronousImpl implements StateSnapshotContext, Closeable { - + + /** Checkpoint id of the snapshot. */ private final long checkpointId; + + /** Checkpoint timestamp of the snapshot. */ private final long checkpointTimestamp; /** Factory for he checkpointing stream */ @@ -47,7 +52,10 @@ public class StateSnapshotContextSynchronousImpl implements StateSnapshotContext */ private final CloseableRegistry closableRegistry; + /** Output stream for the raw keyed state. */ private KeyedStateCheckpointOutputStream keyedStateCheckpointOutputStream; + + /** Output stream for the raw operator state. */ private OperatorStateCheckpointOutputStream operatorStateCheckpointOutputStream; @VisibleForTesting @@ -109,14 +117,23 @@ public OperatorStateCheckpointOutputStream getRawOperatorStateOutput() throws Ex return operatorStateCheckpointOutputStream; } - public RunnableFuture getKeyedStateStreamFuture() throws IOException { - KeyGroupsStateHandle keyGroupsStateHandle = closeAndUnregisterStreamToObtainStateHandle(keyedStateCheckpointOutputStream); - return new DoneFuture(keyGroupsStateHandle); + @Nonnull + public RunnableFuture> getKeyedStateStreamFuture() throws IOException { + KeyedStateHandle keyGroupsStateHandle = + closeAndUnregisterStreamToObtainStateHandle(keyedStateCheckpointOutputStream); + return toDoneFutureOfSnapshotResult(keyGroupsStateHandle); } - public RunnableFuture getOperatorStateStreamFuture() throws IOException { - OperatorStateHandle operatorStateHandle = closeAndUnregisterStreamToObtainStateHandle(operatorStateCheckpointOutputStream); - return new DoneFuture<>(operatorStateHandle); + @Nonnull + public RunnableFuture> getOperatorStateStreamFuture() throws IOException { + OperatorStateHandle operatorStateHandle = + closeAndUnregisterStreamToObtainStateHandle(operatorStateCheckpointOutputStream); + return toDoneFutureOfSnapshotResult(operatorStateHandle); + } + + private RunnableFuture> toDoneFutureOfSnapshotResult(T handle) { + SnapshotResult snapshotResult = SnapshotResult.of(handle); + return DoneFuture.of(snapshotResult); } private T closeAndUnregisterStreamToObtainStateHandle( @@ -130,7 +147,7 @@ private T closeAndUnregisterStreamToObtainStateHan } private void closeAndUnregisterStream( - NonClosingCheckpointOutputStream stream) throws IOException { + NonClosingCheckpointOutputStream stream) throws IOException { Preconditions.checkNotNull(stream); @@ -149,9 +166,7 @@ public void close() throws IOException { try { closeAndUnregisterStream(keyedStateCheckpointOutputStream); } catch (IOException e) { - exception = ExceptionUtils.firstOrSuppressed( - new IOException("Could not close the raw keyed state checkpoint output stream.", e), - exception); + exception = new IOException("Could not close the raw keyed state checkpoint output stream.", e); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManager.java index 326b95ce62a89..a940aefcc7f6d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManager.java @@ -18,81 +18,241 @@ package org.apache.flink.runtime.state; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.util.Preconditions; +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.ShutdownHookUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnegative; import javax.annotation.Nonnull; +import javax.annotation.concurrent.GuardedBy; +import java.io.File; +import java.io.IOException; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.Executor; /** - * This class holds the all {@link TaskLocalStateStore} objects for a task executor (manager). - * - * TODO: this still still work in progress and partially still acts as a placeholder. + * This class holds the all {@link TaskLocalStateStoreImpl} objects for a task executor (manager). */ public class TaskExecutorLocalStateStoresManager { + /** Logger for this class. */ + private static final Logger LOG = LoggerFactory.getLogger(TaskExecutorLocalStateStoresManager.class); + /** * This map holds all local state stores for tasks running on the task manager / executor that own the instance of - * this. + * this. Maps from allocation id to all the subtask's local state stores. */ - private final Map> taskStateManagers; + @GuardedBy("lock") + private final Map> taskStateStoresByAllocationID; + + /** The configured mode for local recovery on this task manager. */ + private final LocalRecoveryConfig.LocalRecoveryMode localRecoveryMode; + + /** This is the root directory for all local state of this task manager / executor. */ + private final File[] localStateRootDirectories; + + /** Executor that runs the discarding of released state objects. */ + private final Executor discardExecutor; + + /** Guarding lock for taskStateStoresByAllocationID and closed-flag. */ + private final Object lock; + + private final Thread shutdownHook; + + @GuardedBy("lock") + private boolean closed; + + public TaskExecutorLocalStateStoresManager( + @Nonnull LocalRecoveryConfig.LocalRecoveryMode localRecoveryMode, + @Nonnull File[] localStateRootDirectories, + @Nonnull Executor discardExecutor) throws IOException { + + this.taskStateStoresByAllocationID = new HashMap<>(); + this.localRecoveryMode = localRecoveryMode; + this.localStateRootDirectories = localStateRootDirectories; + this.discardExecutor = discardExecutor; + this.lock = new Object(); + this.closed = false; + + for (File localStateRecoveryRootDir : localStateRootDirectories) { + if (!localStateRecoveryRootDir.exists()) { + + if (!localStateRecoveryRootDir.mkdirs()) { + throw new IOException("Could not create root directory for local recovery: " + + localStateRecoveryRootDir); + } + } + } - public TaskExecutorLocalStateStoresManager() { - this.taskStateManagers = new HashMap<>(); + // register a shutdown hook + this.shutdownHook = ShutdownHookUtil.addShutdownHook(this::shutdown, getClass().getSimpleName(), LOG); } - public TaskLocalStateStore localStateStoreForTask( - JobID jobId, - JobVertexID jobVertexID, - int subtaskIndex) { + @Nonnull + public TaskLocalStateStore localStateStoreForSubtask( + @Nonnull JobID jobId, + @Nonnull AllocationID allocationID, + @Nonnull JobVertexID jobVertexID, + @Nonnegative int subtaskIndex) { + + synchronized (lock) { + + if (closed) { + throw new IllegalStateException("TaskExecutorLocalStateStoresManager is already closed and cannot " + + "register a new TaskLocalStateStore."); + } + + final Map taskStateManagers = + this.taskStateStoresByAllocationID.computeIfAbsent(allocationID, k -> new HashMap<>()); + + final JobVertexSubtaskKey taskKey = new JobVertexSubtaskKey(jobVertexID, subtaskIndex); + + // create the allocation base dirs, one inside each root dir. + File[] allocationBaseDirectories = allocationBaseDirectories(allocationID); - Preconditions.checkNotNull(jobId); - final JobVertexSubtaskKey taskKey = new JobVertexSubtaskKey(jobVertexID, subtaskIndex); + LocalRecoveryDirectoryProviderImpl directoryProvider = new LocalRecoveryDirectoryProviderImpl( + allocationBaseDirectories, + jobId, + jobVertexID, + subtaskIndex); - final Map taskStateManagers = - this.taskStateManagers.computeIfAbsent(jobId, k -> new HashMap<>()); + LocalRecoveryConfig localRecoveryConfig = new LocalRecoveryConfig( + localRecoveryMode, + directoryProvider); - return taskStateManagers.computeIfAbsent( - taskKey, k -> new TaskLocalStateStore(jobId, jobVertexID, subtaskIndex)); + return taskStateManagers.computeIfAbsent( + taskKey, + k -> new TaskLocalStateStoreImpl( + jobId, + allocationID, + jobVertexID, + subtaskIndex, + localRecoveryConfig, + discardExecutor)); + } } - public void releaseJob(JobID jobID) { + public void releaseLocalStateForAllocationId(@Nonnull AllocationID allocationID) { + + Map cleanupLocalStores; - Map cleanupLocalStores = taskStateManagers.remove(jobID); + synchronized (lock) { + if (closed) { + return; + } + cleanupLocalStores = taskStateStoresByAllocationID.remove(allocationID); + } if (cleanupLocalStores != null) { doRelease(cleanupLocalStores.values()); } + + cleanupAllocationBaseDirs(allocationID); } - public void releaseAll() { + public void shutdown() { + + HashMap> toRelease; + + synchronized (lock) { - for (Map stateStoreMap : taskStateManagers.values()) { - doRelease(stateStoreMap.values()); + if (closed) { + return; + } + + closed = true; + toRelease = new HashMap<>(taskStateStoresByAllocationID); + taskStateStoresByAllocationID.clear(); + } + + ShutdownHookUtil.removeShutdownHook(shutdownHook, getClass().getSimpleName(), LOG); + + LOG.debug("Shutting down TaskExecutorLocalStateStoresManager."); + + for (Map.Entry> entry : + toRelease.entrySet()) { + + doRelease(entry.getValue().values()); + cleanupAllocationBaseDirs(entry.getKey()); } + } + + @VisibleForTesting + public LocalRecoveryConfig.LocalRecoveryMode getLocalRecoveryMode() { + return localRecoveryMode; + } + + @VisibleForTesting + File[] getLocalStateRootDirectories() { + return localStateRootDirectories; + } + + @VisibleForTesting + String allocationSubDirString(AllocationID allocationID) { + return "aid_" + allocationID; + } - taskStateManagers.clear(); + private File[] allocationBaseDirectories(AllocationID allocationID) { + final String allocationSubDirString = allocationSubDirString(allocationID); + final File[] allocationDirectories = new File[localStateRootDirectories.length]; + for (int i = 0; i < localStateRootDirectories.length; ++i) { + allocationDirectories[i] = new File(localStateRootDirectories[i], allocationSubDirString); + } + return allocationDirectories; } - private void doRelease(Iterable toRelease) { + private void doRelease(Iterable toRelease) { + if (toRelease != null) { - for (TaskLocalStateStore stateStore : toRelease) { - stateStore.dispose(); + + for (TaskLocalStateStoreImpl stateStore : toRelease) { + try { + stateStore.dispose(); + } catch (Exception disposeEx) { + LOG.warn("Exception while disposing local state store " + stateStore, disposeEx); + } + } + } + } + + /** + * Deletes the base dirs for this allocation id (recursively). + */ + private void cleanupAllocationBaseDirs(AllocationID allocationID) { + // clear the base dirs for this allocation id. + File[] allocationDirectories = allocationBaseDirectories(allocationID); + for (File directory : allocationDirectories) { + try { + FileUtils.deleteFileOrDirectory(directory); + } catch (IOException e) { + LOG.warn("Exception while deleting local state directory for allocation " + allocationID, e); } } } + /** + * Composite key of {@link JobVertexID} and subtask index that describes the subtask of a job vertex. + */ private static final class JobVertexSubtaskKey { + /** The job vertex id. */ @Nonnull final JobVertexID jobVertexID; + + /** The subtask index. */ + @Nonnegative final int subtaskIndex; - public JobVertexSubtaskKey(@Nonnull JobVertexID jobVertexID, int subtaskIndex) { - this.jobVertexID = Preconditions.checkNotNull(jobVertexID); + JobVertexSubtaskKey(@Nonnull JobVertexID jobVertexID, @Nonnegative int subtaskIndex) { + this.jobVertexID = jobVertexID; this.subtaskIndex = subtaskIndex; } @@ -107,10 +267,7 @@ public boolean equals(Object o) { JobVertexSubtaskKey that = (JobVertexSubtaskKey) o; - if (subtaskIndex != that.subtaskIndex) { - return false; - } - return jobVertexID.equals(that.jobVertexID); + return subtaskIndex == that.subtaskIndex && jobVertexID.equals(that.jobVertexID); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStore.java index f7436308abe52..7089894c1c5af 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStore.java @@ -18,44 +18,48 @@ package org.apache.flink.runtime.state; -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; + +import javax.annotation.Nonnegative; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; /** - * This class will service as a task-manager-level local storage for local checkpointed state. The purpose is to provide - * access to a state that is stored locally for a faster recovery compared to the state that is stored remotely in a - * stable store DFS. For now, this storage is only complementary to the stable storage and local state is typically - * lost in case of machine failures. In such cases (and others), client code of this class must fall back to using the - * slower but highly available store. - * - * TODO this is currently a placeholder / mock that still must be implemented! + * Classes that implement this interface serve as a task-manager-level local storage for local checkpointed state. + * The purpose is to provide access to a state that is stored locally for a faster recovery compared to the state that + * is stored remotely in a stable store DFS. For now, this storage is only complementary to the stable storage and local + * state is typically lost in case of machine failures. In such cases (and others), client code of this class must fall + * back to using the slower but highly available store. */ -public class TaskLocalStateStore { - - /** */ - private final JobID jobID; - - /** */ - private final JobVertexID jobVertexID; - - /** */ - private final int subtaskIndex; - - public TaskLocalStateStore( - JobID jobID, - JobVertexID jobVertexID, - int subtaskIndex) { +public interface TaskLocalStateStore { + /** + * Stores the local state for the given checkpoint id. + * + * @param checkpointId id for the checkpoint that created the local state that will be stored. + * @param localState the local state to store. + */ + void storeLocalState( + @Nonnegative long checkpointId, + @Nullable TaskStateSnapshot localState); - this.jobID = jobID; - this.jobVertexID = jobVertexID; - this.subtaskIndex = subtaskIndex; - } + /** + * Returns the local state that is stored under the given checkpoint id or null if nothing was stored under the id. + * + * @param checkpointID the checkpoint id by which we search for local state. + * @return the local state found for the given checkpoint id. Can be null + */ + @Nullable + TaskStateSnapshot retrieveLocalState(long checkpointID); - public void storeSnapshot(/* TODO */) { - throw new UnsupportedOperationException("TODO!"); - } + /** + * Returns the {@link LocalRecoveryConfig} for this task local state store. + */ + @Nonnull + LocalRecoveryConfig getLocalRecoveryConfig(); - public void dispose() { - throw new UnsupportedOperationException("TODO!"); - } + /** + * Notifies that the checkpoint with the given id was confirmed as complete. This prunes the checkpoint history + * and removes all local states with a checkpoint id that is smaller than the newly confirmed checkpoint id. + */ + void confirmCheckpoint(long confirmedCheckpointId); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStoreImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStoreImpl.java new file mode 100644 index 0000000000000..191c1096aa4d5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStoreImpl.java @@ -0,0 +1,288 @@ +/* + * 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.runtime.state; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.jobgraph.JobVertexID; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnegative; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; + +/** + * Main implementation of a {@link TaskLocalStateStore}. + */ +public class TaskLocalStateStoreImpl implements TaskLocalStateStore { + + /** Logger for this class. */ + private static final Logger LOG = LoggerFactory.getLogger(TaskLocalStateStoreImpl.class); + + /** Dummy value to use instead of null to satisfy {@link ConcurrentHashMap}. */ + private static final TaskStateSnapshot NULL_DUMMY = new TaskStateSnapshot(0); + + /** JobID from the owning subtask. */ + @Nonnull + private final JobID jobID; + + /** AllocationID of the owning slot. */ + @Nonnull + private final AllocationID allocationID; + + /** JobVertexID of the owning subtask. */ + @Nonnull + private final JobVertexID jobVertexID; + + /** Subtask index of the owning subtask. */ + @Nonnegative + private final int subtaskIndex; + + /** The configured mode for local recovery. */ + @Nonnull + private final LocalRecoveryConfig localRecoveryConfig; + + /** Executor that runs the discarding of released state objects. */ + @Nonnull + private final Executor discardExecutor; + + /** Lock for synchronisation on the storage map and the discarded status. */ + @Nonnull + private final Object lock; + + /** Status flag if this store was already discarded. */ + @GuardedBy("lock") + private boolean disposed; + + /** Maps checkpoint ids to local TaskStateSnapshots. */ + @Nonnull + @GuardedBy("lock") + private final SortedMap storedTaskStateByCheckpointID; + + public TaskLocalStateStoreImpl( + @Nonnull JobID jobID, + @Nonnull AllocationID allocationID, + @Nonnull JobVertexID jobVertexID, + @Nonnegative int subtaskIndex, + @Nonnull LocalRecoveryConfig localRecoveryConfig, + @Nonnull Executor discardExecutor) { + + this.jobID = jobID; + this.allocationID = allocationID; + this.jobVertexID = jobVertexID; + this.subtaskIndex = subtaskIndex; + this.discardExecutor = discardExecutor; + this.lock = new Object(); + this.storedTaskStateByCheckpointID = new TreeMap<>(); + this.disposed = false; + this.localRecoveryConfig = localRecoveryConfig; + } + + @Override + public void storeLocalState( + @Nonnegative long checkpointId, + @Nullable TaskStateSnapshot localState) { + + if (localState == null) { + localState = NULL_DUMMY; + } + + LOG.info("Storing local state for checkpoint {}.", checkpointId); + LOG.debug("Local state for checkpoint {} is {}.", checkpointId, localState); + + Map toDiscard = new HashMap<>(16); + + synchronized (lock) { + if (disposed) { + // we ignore late stores and simply discard the state. + toDiscard.put(checkpointId, localState); + } else { + TaskStateSnapshot previous = + storedTaskStateByCheckpointID.put(checkpointId, localState); + + if (previous != null) { + toDiscard.put(checkpointId, previous); + } + } + } + + asyncDiscardLocalStateForCollection(toDiscard.entrySet()); + } + + @Override + @Nullable + public TaskStateSnapshot retrieveLocalState(long checkpointID) { + synchronized (lock) { + TaskStateSnapshot snapshot = storedTaskStateByCheckpointID.get(checkpointID); + return snapshot != NULL_DUMMY ? snapshot : null; + } + } + + @Override + @Nonnull + public LocalRecoveryConfig getLocalRecoveryConfig() { + return localRecoveryConfig; + } + + @Override + public void confirmCheckpoint(long confirmedCheckpointId) { + + LOG.debug("Received confirmation for checkpoint {}. Starting to prune history.", confirmedCheckpointId); + + final List> toRemove = new ArrayList<>(); + + synchronized (lock) { + + Iterator> entryIterator = + storedTaskStateByCheckpointID.entrySet().iterator(); + + // remove entries for outdated checkpoints and discard their state. + while (entryIterator.hasNext()) { + + Map.Entry snapshotEntry = entryIterator.next(); + long entryCheckpointId = snapshotEntry.getKey(); + + if (entryCheckpointId < confirmedCheckpointId) { + toRemove.add(snapshotEntry); + entryIterator.remove(); + } else { + // we can stop because the map is sorted. + break; + } + } + } + + asyncDiscardLocalStateForCollection(toRemove); + } + + /** + * Disposes the state of all local snapshots managed by this object. + */ + public CompletableFuture dispose() { + + Collection> statesCopy; + + synchronized (lock) { + disposed = true; + statesCopy = new ArrayList<>(storedTaskStateByCheckpointID.entrySet()); + storedTaskStateByCheckpointID.clear(); + } + + return CompletableFuture.runAsync( + () -> { + // discard all remaining state objects. + syncDiscardLocalStateForCollection(statesCopy); + + // delete the local state subdirectory that belong to this subtask. + LocalRecoveryDirectoryProvider directoryProvider = localRecoveryConfig.getLocalStateDirectoryProvider(); + for (int i = 0; i < directoryProvider.allocationBaseDirsCount(); ++i) { + File subtaskBaseDirectory = directoryProvider.selectSubtaskBaseDirectory(i); + try { + deleteDirectory(subtaskBaseDirectory); + } catch (IOException e) { + LOG.warn("Exception when deleting local recovery subtask base dir: " + subtaskBaseDirectory, e); + } + } + }, + discardExecutor); + } + + private void asyncDiscardLocalStateForCollection(Collection> toDiscard) { + if (!toDiscard.isEmpty()) { + discardExecutor.execute(() -> syncDiscardLocalStateForCollection(toDiscard)); + } + } + + private void syncDiscardLocalStateForCollection(Collection> toDiscard) { + for (Map.Entry entry : toDiscard) { + discardLocalStateForCheckpoint(entry.getKey(), entry.getValue()); + } + } + + /** + * Helper method that discards state objects with an executor and reports exceptions to the log. + */ + private void discardLocalStateForCheckpoint(long checkpointID, TaskStateSnapshot o) { + + try { + if (LOG.isTraceEnabled()) { + LOG.trace("Discarding local task state snapshot of checkpoint {} for {}/{}/{}.", + checkpointID, jobID, jobVertexID, subtaskIndex); + } else { + LOG.debug("Discarding local task state snapshot {} of checkpoint {} for {}/{}/{}.", + o, checkpointID, jobID, jobVertexID, subtaskIndex); + } + o.discardState(); + } catch (Exception discardEx) { + LOG.warn("Exception while discarding local task state snapshot of checkpoint " + checkpointID + ".", discardEx); + } + + LocalRecoveryDirectoryProvider directoryProvider = localRecoveryConfig.getLocalStateDirectoryProvider(); + File checkpointDir = directoryProvider.subtaskSpecificCheckpointDirectory(checkpointID); + LOG.debug("Deleting local state directory {} of checkpoint {} for {}/{}/{}/{}.", + checkpointDir, checkpointID, jobID, jobVertexID, subtaskIndex); + try { + deleteDirectory(checkpointDir); + } catch (IOException ex) { + LOG.warn("Exception while deleting local state directory of checkpoint " + checkpointID + ".", ex); + } + } + + /** + * Helper method to delete a directory. + */ + private void deleteDirectory(File directory) throws IOException { + Path path = new Path(directory.toURI()); + FileSystem fileSystem = path.getFileSystem(); + if (fileSystem.exists(path)) { + fileSystem.delete(path, true); + } + } + + @Override + public String toString() { + return "TaskLocalStateStore{" + + "jobID=" + jobID + + ", jobVertexID=" + jobVertexID + + ", allocationID=" + allocationID + + ", subtaskIndex=" + subtaskIndex + + ", localRecoveryConfig=" + localRecoveryConfig + + '}'; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateManager.java index 8b41e9ec1f2e0..82591babb8a84 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateManager.java @@ -20,7 +20,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.PrioritizedOperatorSubtaskState; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -44,19 +44,29 @@ public interface TaskStateManager extends CheckpointListener { * Report the state snapshots for the operator instances running in the owning task. * * @param checkpointMetaData meta data from the checkpoint request. - * @param checkpointMetrics task level metrics for the checkpoint. - * @param acknowledgedState the reported states from the owning task. + * @param checkpointMetrics task level metrics for the checkpoint. + * @param acknowledgedState the reported states to acknowledge to the job manager. + * @param localState the reported states for local recovery. */ - void reportStateHandles( + void reportTaskStateSnapshots( @Nonnull CheckpointMetaData checkpointMetaData, @Nonnull CheckpointMetrics checkpointMetrics, - @Nullable TaskStateSnapshot acknowledgedState); + @Nullable TaskStateSnapshot acknowledgedState, + @Nullable TaskStateSnapshot localState); /** * Returns means to restore previously reported state of an operator running in the owning task. * * @param operatorID the id of the operator for which we request state. - * @return previous state for the operator. Null if no previous state exists. + * @return Previous state for the operator. The previous state can be empty if the operator had no previous state. */ - OperatorSubtaskState operatorStates(OperatorID operatorID); + @Nonnull + PrioritizedOperatorSubtaskState prioritizedOperatorState(OperatorID operatorID); + + /** + * Returns the configuration for local recovery, i.e. the base directories for all file-based local state of the + * owning subtask and the general mode for local recovery. + */ + @Nonnull + LocalRecoveryConfig createLocalRecoveryConfig(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateManagerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateManagerImpl.java index 3cd66fb07c814..3acca7cade2a9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateManagerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateManagerImpl.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.PrioritizedOperatorSubtaskState; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -31,6 +32,8 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.util.Collections; + /** * This class is the default implementation of {@link TaskStateManager} and collaborates with the job manager * through {@link CheckpointResponder}) as well as a task-manager-local state store. Like this, client code does @@ -75,35 +78,67 @@ public TaskStateManagerImpl( } @Override - public void reportStateHandles( + public void reportTaskStateSnapshots( @Nonnull CheckpointMetaData checkpointMetaData, @Nonnull CheckpointMetrics checkpointMetrics, - @Nullable TaskStateSnapshot acknowledgedState) { + @Nullable TaskStateSnapshot acknowledgedState, + @Nullable TaskStateSnapshot localState) { + + long checkpointId = checkpointMetaData.getCheckpointId(); + + localStateStore.storeLocalState(checkpointId, localState); checkpointResponder.acknowledgeCheckpoint( jobId, executionAttemptID, - checkpointMetaData.getCheckpointId(), + checkpointId, checkpointMetrics, acknowledgedState); } + @Nonnull @Override - public OperatorSubtaskState operatorStates(OperatorID operatorID) { + public PrioritizedOperatorSubtaskState prioritizedOperatorState(OperatorID operatorID) { if (jobManagerTaskRestore == null) { - return null; + return PrioritizedOperatorSubtaskState.emptyNotRestored(); + } + + TaskStateSnapshot jobManagerStateSnapshot = + jobManagerTaskRestore.getTaskStateSnapshot(); + + OperatorSubtaskState jobManagerSubtaskState = + jobManagerStateSnapshot.getSubtaskStateByOperatorID(operatorID); + + if (jobManagerSubtaskState == null) { + return PrioritizedOperatorSubtaskState.emptyNotRestored(); } - TaskStateSnapshot taskStateSnapshot = jobManagerTaskRestore.getTaskStateSnapshot(); - return taskStateSnapshot.getSubtaskStateByOperatorID(operatorID); + TaskStateSnapshot localStateSnapshot = + localStateStore.retrieveLocalState(jobManagerTaskRestore.getRestoreCheckpointId()); - /* - TODO!!!!!!! - 1) lookup local states for a matching operatorID / checkpointID. - 2) if nothing available: look into job manager provided state. - 3) massage it into a snapshots and return stuff. - */ + if (localStateSnapshot != null) { + OperatorSubtaskState localSubtaskState = localStateSnapshot.getSubtaskStateByOperatorID(operatorID); + + if (localSubtaskState != null) { + PrioritizedOperatorSubtaskState.Builder builder = new PrioritizedOperatorSubtaskState.Builder( + jobManagerSubtaskState, + Collections.singletonList(localSubtaskState)); + return builder.build(); + } + } + + PrioritizedOperatorSubtaskState.Builder builder = new PrioritizedOperatorSubtaskState.Builder( + jobManagerSubtaskState, + Collections.emptyList(), + true); + return builder.build(); + } + + @Nonnull + @Override + public LocalRecoveryConfig createLocalRecoveryConfig() { + return localStateStore.getLocalRecoveryConfig(); } /** @@ -111,6 +146,6 @@ public OperatorSubtaskState operatorStates(OperatorID operatorID) { */ @Override public void notifyCheckpointComplete(long checkpointId) throws Exception { - //TODO activate and prune local state later + localStateStore.confirmCheckpoint(checkpointId); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileBasedStateOutputStream.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileBasedStateOutputStream.java new file mode 100644 index 0000000000000..054a98cc511d7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FileBasedStateOutputStream.java @@ -0,0 +1,157 @@ +/* + * 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.runtime.state.filesystem; + +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.FileSystem.WriteMode; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.state.CheckpointStreamFactory.CheckpointStateOutputStream; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link CheckpointStateOutputStream} that writes into a specified file and + * returns a {@link FileStateHandle} upon closing. + * + *

Unlike the {@link org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory.FsCheckpointStateOutputStream}, + * this stream does not have a threshold below which it returns a memory byte stream handle, + * and does not create random files, but writes to a specified file. + */ +public final class FileBasedStateOutputStream extends CheckpointStateOutputStream { + + private static final Logger LOG = LoggerFactory.getLogger(FileBasedStateOutputStream.class); + + // ------------------------------------------------------------------------ + + private final FSDataOutputStream out; + + private final Path path; + + private final FileSystem fileSystem; + + private volatile boolean closed; + + + public FileBasedStateOutputStream(FileSystem fileSystem, Path path) throws IOException { + this.fileSystem = checkNotNull(fileSystem); + this.path = checkNotNull(path); + + this.out = fileSystem.create(path, WriteMode.NO_OVERWRITE); + } + + // ------------------------------------------------------------------------ + // I/O + // ------------------------------------------------------------------------ + + @Override + public final void write(int b) throws IOException { + out.write(b); + } + + @Override + public final void write(@Nonnull byte[] b, int off, int len) throws IOException { + out.write(b, off, len); + } + + @Override + public long getPos() throws IOException { + return out.getPos(); + } + + @Override + public void flush() throws IOException { + out.flush(); + } + + @Override + public void sync() throws IOException { + out.sync(); + } + + // ------------------------------------------------------------------------ + // Closing + // ------------------------------------------------------------------------ + + public boolean isClosed() { + return closed; + } + + @Override + public void close() { + if (!closed) { + closed = true; + + try { + out.close(); + fileSystem.delete(path, false); + } + catch (Throwable t) { + LOG.warn("Could not close the state stream for {}.", path, t); + } + } + } + + @Nullable + @Override + public FileStateHandle closeAndGetHandle() throws IOException { + synchronized (this) { + if (!closed) { + try { + // make a best effort attempt to figure out the size + long size = 0; + try { + size = out.getPos(); + } catch (Exception ignored) {} + + // close and return + out.close(); + + return new FileStateHandle(path, size); + } + catch (Exception e) { + try { + fileSystem.delete(path, false); + } + catch (Exception deleteException) { + LOG.warn("Could not delete the checkpoint stream file {}.", path, deleteException); + } + + throw new IOException("Could not flush and close the file system " + + "output stream to " + path + " in order to obtain the " + + "stream state handle", e); + } + finally { + closed = true; + } + } + else { + throw new IOException("Stream has already been closed and discarded."); + } + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java index f9937865e0921..609ef698530d2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java @@ -29,6 +29,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.io.IOException; import java.util.Arrays; import java.util.UUID; @@ -115,8 +117,10 @@ public FsCheckpointStreamFactory( // ------------------------------------------------------------------------ @Override - public FsCheckpointStateOutputStream createCheckpointStateOutputStream(CheckpointedStateScope scope) throws Exception { - Path target = scope == CheckpointedStateScope.EXCLUSIVE ? checkpointDirectory : sharedStateDirectory; + public FsCheckpointStateOutputStream createCheckpointStateOutputStream(CheckpointedStateScope scope) throws IOException { + + + Path target = scope == CheckpointedStateScope.EXCLUSIVE ?checkpointDirectory: sharedStateDirectory; int bufferSize = Math.max(DEFAULT_WRITE_BUFFER_SIZE, fileStateThreshold); return new FsCheckpointStateOutputStream(target, filesystem, bufferSize, fileStateThreshold); @@ -275,6 +279,7 @@ public void close() { } } + @Nullable @Override public StreamStateHandle closeAndGetHandle() throws IOException { // check if there was nothing ever written diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java index 58791e27dce61..637effde06314 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java @@ -32,7 +32,9 @@ import org.apache.flink.runtime.state.ConfigurableStateBackend; import org.apache.flink.runtime.state.DefaultOperatorStateBackend; import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.LocalRecoveryConfig; import org.apache.flink.runtime.state.OperatorStateBackend; +import org.apache.flink.runtime.state.TaskStateManager; import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend; import org.apache.flink.util.TernaryBoolean; @@ -102,7 +104,7 @@ public class FsStateBackend extends AbstractFileStateBackend implements Configur * A value of 'undefined' means not yet configured, in which case the default will be used. */ private final TernaryBoolean asynchronousSnapshots; - // ------------------------------------------------------------------------ + // ----------------------------------------------------------------------- /** * Creates a new state backend that stores its checkpoint data in the file system and location @@ -451,7 +453,10 @@ public AbstractKeyedStateBackend createKeyedStateBackend( TypeSerializer keySerializer, int numberOfKeyGroups, KeyGroupRange keyGroupRange, - TaskKvStateRegistry kvStateRegistry) throws IOException { + TaskKvStateRegistry kvStateRegistry) { + + TaskStateManager taskStateManager = env.getTaskStateManager(); + LocalRecoveryConfig localRecoveryConfig = taskStateManager.createLocalRecoveryConfig(); return new HeapKeyedStateBackend<>( kvStateRegistry, @@ -460,13 +465,14 @@ public AbstractKeyedStateBackend createKeyedStateBackend( numberOfKeyGroups, keyGroupRange, isUsingAsynchronousSnapshots(), - env.getExecutionConfig()); + env.getExecutionConfig(), + localRecoveryConfig); } @Override public OperatorStateBackend createOperatorStateBackend( Environment env, - String operatorIdentifier) throws Exception { + String operatorIdentifier) { return new DefaultOperatorStateBackend( env.getUserClassLoader(), diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java index aa7ea6e4bced9..5d5f7162a4651 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java @@ -49,8 +49,10 @@ import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedBackendSerializationProxy; import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.LocalRecoveryConfig; import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo; import org.apache.flink.runtime.state.SnappyStreamCompressionDecorator; +import org.apache.flink.runtime.state.SnapshotResult; import org.apache.flink.runtime.state.StreamCompressionDecorator; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.UncompressedStreamCompressionDecorator; @@ -117,13 +119,14 @@ public class HeapKeyedStateBackend extends AbstractKeyedStateBackend { private final boolean asynchronousSnapshots; public HeapKeyedStateBackend( - TaskKvStateRegistry kvStateRegistry, - TypeSerializer keySerializer, - ClassLoader userCodeClassLoader, - int numberOfKeyGroups, - KeyGroupRange keyGroupRange, - boolean asynchronousSnapshots, - ExecutionConfig executionConfig) { + TaskKvStateRegistry kvStateRegistry, + TypeSerializer keySerializer, + ClassLoader userCodeClassLoader, + int numberOfKeyGroups, + KeyGroupRange keyGroupRange, + boolean asynchronousSnapshots, + ExecutionConfig executionConfig, + LocalRecoveryConfig localRecoveryConfig) { super(kvStateRegistry, keySerializer, userCodeClassLoader, numberOfKeyGroups, keyGroupRange, executionConfig); this.asynchronousSnapshots = asynchronousSnapshots; @@ -286,14 +289,14 @@ public InternalMapState createMapState(TypeSerializer @Override @SuppressWarnings("unchecked") - public RunnableFuture snapshot( + public RunnableFuture> snapshot( final long checkpointId, final long timestamp, final CheckpointStreamFactory streamFactory, CheckpointOptions checkpointOptions) throws Exception { if (!hasRegisteredState()) { - return DoneFuture.nullValue(); + return DoneFuture.of(SnapshotResult.empty()); } long syncStartTime = System.currentTimeMillis(); @@ -326,8 +329,8 @@ public RunnableFuture snapshot( //--------------------------------------------------- this becomes the end of sync part // implementation of the async IO operation, based on FutureTask - final AbstractAsyncCallableWithResources ioCallable = - new AbstractAsyncCallableWithResources() { + final AbstractAsyncCallableWithResources> ioCallable = + new AbstractAsyncCallableWithResources>() { CheckpointStreamFactory.CheckpointStateOutputStream stream = null; @@ -359,7 +362,7 @@ protected void stopOperation() throws Exception { } @Override - public KeyGroupsStateHandle performOperation() throws Exception { + public SnapshotResult performOperation() throws Exception { long asyncStartTime = System.currentTimeMillis(); CheckpointStreamFactory.CheckpointStateOutputStream localStream = this.stream; @@ -401,15 +404,15 @@ public KeyGroupsStateHandle performOperation() throws Exception { final KeyGroupsStateHandle keyGroupsStateHandle = new KeyGroupsStateHandle(offsets, streamStateHandle); - return keyGroupsStateHandle; + return SnapshotResult.of(keyGroupsStateHandle); } } - return null; + return SnapshotResult.empty(); } }; - AsyncStoppableTaskWithCallback task = AsyncStoppableTaskWithCallback.from(ioCallable); + AsyncStoppableTaskWithCallback> task = AsyncStoppableTaskWithCallback.from(ioCallable); if (!asynchronousSnapshots) { task.run(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemCheckpointStreamFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemCheckpointStreamFactory.java index 168e4ffb85eb9..080142937049b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemCheckpointStreamFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemCheckpointStreamFactory.java @@ -23,6 +23,8 @@ import org.apache.flink.runtime.state.CheckpointedStateScope; import org.apache.flink.runtime.state.StreamStateHandle; +import javax.annotation.Nullable; + import java.io.IOException; import java.util.UUID; import java.util.concurrent.atomic.AtomicBoolean; @@ -47,7 +49,8 @@ public MemCheckpointStreamFactory(int maxStateSize) { @Override public CheckpointStateOutputStream createCheckpointStateOutputStream( - CheckpointedStateScope scope) throws Exception { + CheckpointedStateScope scope) throws IOException + { return new MemoryCheckpointOutputStream(maxStateSize); } @@ -114,6 +117,7 @@ public void close() { } } + @Nullable @Override public StreamStateHandle closeAndGetHandle() throws IOException { if (isEmpty) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java index 88d7b01e236ea..3da60e4861f9f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.state.DefaultOperatorStateBackend; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.OperatorStateBackend; +import org.apache.flink.runtime.state.TaskStateManager; import org.apache.flink.runtime.state.filesystem.AbstractFileStateBackend; import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend; import org.apache.flink.util.TernaryBoolean; @@ -299,13 +300,16 @@ public OperatorStateBackend createOperatorStateBackend( @Override public AbstractKeyedStateBackend createKeyedStateBackend( - Environment env, JobID jobID, + Environment env, + JobID jobID, String operatorIdentifier, TypeSerializer keySerializer, int numberOfKeyGroups, KeyGroupRange keyGroupRange, TaskKvStateRegistry kvStateRegistry) { + TaskStateManager taskStateManager = env.getTaskStateManager(); + return new HeapKeyedStateBackend<>( kvStateRegistry, keySerializer, @@ -313,7 +317,8 @@ public AbstractKeyedStateBackend createKeyedStateBackend( numberOfKeyGroups, keyGroupRange, isUsingAsynchronousSnapshots(), - env.getExecutionConfig()); + env.getExecutionConfig(), + taskStateManager.createLocalRecoveryConfig()); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index f4c953ddb8d7e..927bd117f1601 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -205,7 +205,7 @@ public TaskExecutor( this.jobManagerTable = taskExecutorServices.getJobManagerTable(); this.jobLeaderService = taskExecutorServices.getJobLeaderService(); this.taskManagerLocation = taskExecutorServices.getTaskManagerLocation(); - this.localStateStoresManager = taskExecutorServices.getTaskStateManager(); + this.localStateStoresManager = taskExecutorServices.getTaskManagerStateStore(); this.networkEnvironment = taskExecutorServices.getNetworkEnvironment(); this.jobManagerConnections = new HashMap<>(4); @@ -452,8 +452,9 @@ public CompletableFuture submitTask( ResultPartitionConsumableNotifier resultPartitionConsumableNotifier = jobManagerConnection.getResultPartitionConsumableNotifier(); PartitionProducerStateChecker partitionStateChecker = jobManagerConnection.getPartitionStateChecker(); - final TaskLocalStateStore localStateStore = localStateStoresManager.localStateStoreForTask( + final TaskLocalStateStore localStateStore = localStateStoresManager.localStateStoreForSubtask( jobId, + tdd.getAllocationId(), taskInformation.getJobVertexId(), tdd.getSubtaskIndex()); @@ -744,6 +745,9 @@ public CompletableFuture requestSlot( onFatalError(slotNotFoundException); } + // release local state under the allocation id. + localStateStoresManager.releaseLocalStateForAllocationId(allocationId); + // sanity check if (!taskSlotTable.isSlotFree(slotId.getSlotNumber())) { onFatalError(new Exception("Could not free slot " + slotId)); @@ -1271,6 +1275,8 @@ private void freeSlotInternal(AllocationID allocationId, Throwable cause) { } catch (SlotNotFoundException e) { log.debug("Could not free slot for allocation id {}.", allocationId, e); } + + localStateStoresManager.releaseLocalStateForAllocationId(allocationId); } private void timeoutSlot(AllocationID allocationId, UUID ticket) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java index 2de1be81c98bd..08335b2594747 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java @@ -303,6 +303,7 @@ public static TaskExecutor startTaskManager( TaskManagerServices taskManagerServices = TaskManagerServices.fromConfiguration( taskManagerServicesConfiguration, resourceID, + rpcService.getExecutor(), // TODO replace this later with some dedicated executor for io. EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag(), EnvironmentInformation.getMaxJvmHeapMemory()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java index b710b6a7a54d2..32c7ff7171ee2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.taskexecutor; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.memory.MemoryType; @@ -42,6 +43,7 @@ import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.query.KvStateServer; import org.apache.flink.runtime.query.QueryableStateUtils; +import org.apache.flink.runtime.state.LocalRecoveryConfig; import org.apache.flink.runtime.state.TaskExecutorLocalStateStoresManager; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; import org.apache.flink.runtime.taskexecutor.slot.TimerService; @@ -58,6 +60,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.Executor; import java.util.concurrent.ScheduledThreadPoolExecutor; /** @@ -68,6 +71,9 @@ public class TaskManagerServices { private static final Logger LOG = LoggerFactory.getLogger(TaskManagerServices.class); + @VisibleForTesting + public static final String LOCAL_STATE_SUB_DIRECTORY_ROOT = "localState"; + /** TaskManager services. */ private final TaskManagerLocation taskManagerLocation; private final MemoryManager memoryManager; @@ -78,7 +84,7 @@ public class TaskManagerServices { private final TaskSlotTable taskSlotTable; private final JobManagerTable jobManagerTable; private final JobLeaderService jobLeaderService; - private final TaskExecutorLocalStateStoresManager taskStateManager; + private final TaskExecutorLocalStateStoresManager taskManagerStateStore; TaskManagerServices( TaskManagerLocation taskManagerLocation, @@ -90,7 +96,7 @@ public class TaskManagerServices { TaskSlotTable taskSlotTable, JobManagerTable jobManagerTable, JobLeaderService jobLeaderService, - TaskExecutorLocalStateStoresManager taskStateManager) { + TaskExecutorLocalStateStoresManager taskManagerStateStore) { this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation); this.memoryManager = Preconditions.checkNotNull(memoryManager); @@ -101,7 +107,7 @@ public class TaskManagerServices { this.taskSlotTable = Preconditions.checkNotNull(taskSlotTable); this.jobManagerTable = Preconditions.checkNotNull(jobManagerTable); this.jobLeaderService = Preconditions.checkNotNull(jobLeaderService); - this.taskStateManager = Preconditions.checkNotNull(taskStateManager); + this.taskManagerStateStore = Preconditions.checkNotNull(taskManagerStateStore); } // -------------------------------------------------------------------------------------------- @@ -144,8 +150,8 @@ public JobLeaderService getJobLeaderService() { return jobLeaderService; } - public TaskExecutorLocalStateStoresManager getTaskStateManager() { - return taskStateManager; + public TaskExecutorLocalStateStoresManager getTaskManagerStateStore() { + return taskManagerStateStore; } // -------------------------------------------------------------------------------------------- @@ -159,6 +165,12 @@ public void shutDown() throws FlinkException { Exception exception = null; + try { + taskManagerStateStore.shutdown(); + } catch (Exception e) { + exception = e; + } + try { memoryManager.shutdown(); } catch (Exception e) { @@ -209,6 +221,7 @@ public void shutDown() throws FlinkException { * * @param resourceID resource ID of the task manager * @param taskManagerServicesConfiguration task manager configuration + * @param taskIOExecutor executor for async IO operations. * @param freeHeapMemoryWithDefrag an estimate of the size of the free heap memory * @param maxJvmHeapMemory the maximum JVM heap size * @return task manager components @@ -217,6 +230,7 @@ public void shutDown() throws FlinkException { public static TaskManagerServices fromConfiguration( TaskManagerServicesConfiguration taskManagerServicesConfiguration, ResourceID resourceID, + Executor taskIOExecutor, long freeHeapMemoryWithDefrag, long maxJvmHeapMemory) throws Exception { @@ -256,7 +270,20 @@ public static TaskManagerServices fromConfiguration( final JobManagerTable jobManagerTable = new JobManagerTable(); final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation); - final TaskExecutorLocalStateStoresManager taskStateManager = new TaskExecutorLocalStateStoresManager(); + + LocalRecoveryConfig.LocalRecoveryMode localRecoveryMode = taskManagerServicesConfiguration.getLocalRecoveryMode(); + + final String[] stateRootDirectoryStrings = taskManagerServicesConfiguration.getLocalRecoveryStateRootDirectories(); + + final File[] stateRootDirectoryFiles = new File[stateRootDirectoryStrings.length]; + + for (int i = 0; i < stateRootDirectoryStrings.length; ++i) { + stateRootDirectoryFiles[i] = new File(stateRootDirectoryStrings[i], LOCAL_STATE_SUB_DIRECTORY_ROOT); + } + + final TaskExecutorLocalStateStoresManager taskStateManager = + new TaskExecutorLocalStateStoresManager(localRecoveryMode, stateRootDirectoryFiles, taskIOExecutor); + return new TaskManagerServices( taskManagerLocation, memoryManager, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java index 07cf660e25d4e..d029bc5838bc0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java @@ -29,6 +29,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.netty.NettyConfig; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.state.LocalRecoveryConfig; import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; import org.apache.flink.util.MathUtils; import org.apache.flink.util.NetUtils; @@ -54,6 +55,8 @@ public class TaskManagerServicesConfiguration { private final String[] tmpDirPaths; + private final String[] localRecoveryStateRootDirectories; + private final int numberOfSlots; private final NetworkEnvironmentConfiguration networkConfig; @@ -75,9 +78,13 @@ public class TaskManagerServicesConfiguration { private final long timerServiceShutdownTimeout; + private final LocalRecoveryConfig.LocalRecoveryMode localRecoveryMode; + public TaskManagerServicesConfiguration( InetAddress taskManagerAddress, String[] tmpDirPaths, + String[] localRecoveryStateRootDirectories, + LocalRecoveryConfig.LocalRecoveryMode localRecoveryMode, NetworkEnvironmentConfiguration networkConfig, QueryableStateConfiguration queryableStateConfig, int numberOfSlots, @@ -89,6 +96,8 @@ public TaskManagerServicesConfiguration( this.taskManagerAddress = checkNotNull(taskManagerAddress); this.tmpDirPaths = checkNotNull(tmpDirPaths); + this.localRecoveryStateRootDirectories = checkNotNull(localRecoveryStateRootDirectories); + this.localRecoveryMode = checkNotNull(localRecoveryMode); this.networkConfig = checkNotNull(networkConfig); this.queryableStateConfig = checkNotNull(queryableStateConfig); this.numberOfSlots = checkNotNull(numberOfSlots); @@ -115,6 +124,14 @@ public String[] getTmpDirPaths() { return tmpDirPaths; } + public String[] getLocalRecoveryStateRootDirectories() { + return localRecoveryStateRootDirectories; + } + + public LocalRecoveryConfig.LocalRecoveryMode getLocalRecoveryMode() { + return localRecoveryMode; + } + public NetworkEnvironmentConfiguration getNetworkConfig() { return networkConfig; } @@ -185,6 +202,15 @@ public static TaskManagerServicesConfiguration fromConfiguration( } final String[] tmpDirs = ConfigurationUtils.parseTempDirectories(configuration); + String[] localStateRootDir = ConfigurationUtils.parseLocalStateDirectories(configuration); + + if (localStateRootDir.length == 0) { + // default to temp dirs. + localStateRootDir = tmpDirs; + } + + LocalRecoveryConfig.LocalRecoveryMode localRecoveryMode = + LocalRecoveryConfig.LocalRecoveryMode.fromConfig(configuration); final NetworkEnvironmentConfiguration networkConfig = parseNetworkEnvironmentConfiguration( configuration, @@ -225,6 +251,8 @@ public static TaskManagerServicesConfiguration fromConfiguration( return new TaskManagerServicesConfiguration( remoteAddress, tmpDirs, + localStateRootDir, + localRecoveryMode, networkConfig, queryableStateConfig, slots, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index c22413ecfed66..1ecb47a6288d7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -1202,25 +1202,24 @@ public void notifyCheckpointComplete(final long checkpointID) { if (executionState == ExecutionState.RUNNING && invokable != null) { - Runnable runnable = new Runnable() { - @Override - public void run() { - try { - invokable.notifyCheckpointComplete(checkpointID); - taskStateManager.notifyCheckpointComplete(checkpointID);} - catch (Throwable t) { - if (getExecutionState() == ExecutionState.RUNNING) { - // fail task if checkpoint confirmation failed. - failExternally(new RuntimeException( - "Error while confirming checkpoint", - t)); - } + Runnable runnable = new Runnable() { + @Override + public void run() { + try { + invokable.notifyCheckpointComplete(checkpointID); + taskStateManager.notifyCheckpointComplete(checkpointID); + } catch (Throwable t) { + if (getExecutionState() == ExecutionState.RUNNING) { + // fail task if checkpoint confirmation failed. + failExternally(new RuntimeException( + "Error while confirming checkpoint", + t)); } } - }; - executeAsyncCallRunnable(runnable, "Checkpoint Confirmation for " + - taskNameWithSubtask); - + } + }; + executeAsyncCallRunnable(runnable, "Checkpoint Confirmation for " + + taskNameWithSubtask); } else { LOG.debug("Ignoring checkpoint commit notification for non-running task {}.", taskNameWithSubtask); diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala index 31739489c5bde..f62ef1bbf13cd 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala @@ -48,6 +48,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages import org.apache.flink.runtime.messages.JobManagerMessages.{RunningJobsStatus, StoppingFailure, StoppingResponse} import org.apache.flink.runtime.metrics.groups.{JobManagerMetricGroup, TaskManagerMetricGroup} import org.apache.flink.runtime.metrics.util.MetricUtils +import org.apache.flink.runtime.state.TaskExecutorLocalStateStoresManager import org.apache.flink.runtime.taskexecutor.{TaskExecutor, TaskManagerConfiguration, TaskManagerServices, TaskManagerServicesConfiguration} import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation} import org.apache.flink.runtime.util.EnvironmentInformation @@ -238,6 +239,7 @@ class LocalFlinkMiniCluster( val taskManagerServices = TaskManagerServices.fromConfiguration( taskManagerServicesConfiguration, resourceID, + ioExecutor, EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag, EnvironmentInformation.getMaxJvmHeapMemory) @@ -254,6 +256,7 @@ class LocalFlinkMiniCluster( taskManagerServices.getMemoryManager(), taskManagerServices.getIOManager(), taskManagerServices.getNetworkEnvironment, + taskManagerServices.getTaskManagerStateStore, taskManagerMetricGroup) system.actorOf(props, taskManagerActorName) @@ -318,6 +321,7 @@ class LocalFlinkMiniCluster( memoryManager: MemoryManager, ioManager: IOManager, networkEnvironment: NetworkEnvironment, + taskManagerLocalStateStoresManager: TaskExecutorLocalStateStoresManager, taskManagerMetricGroup: TaskManagerMetricGroup): Props = { TaskManager.getTaskManagerProps( @@ -328,6 +332,7 @@ class LocalFlinkMiniCluster( memoryManager, ioManager, networkEnvironment, + taskManagerLocalStateStoresManager, highAvailabilityServices, taskManagerMetricGroup) } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 485add57c6cfb..15581b2b647aa 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -35,11 +35,11 @@ import org.apache.flink.configuration._ import org.apache.flink.core.fs.FileSystem import org.apache.flink.runtime.accumulators.AccumulatorSnapshot import org.apache.flink.runtime.akka.{AkkaUtils, DefaultQuarantineHandler, QuarantineMonitor} -import org.apache.flink.runtime.blob.{BlobCacheService, BlobClient, BlobService} +import org.apache.flink.runtime.blob.BlobCacheService import org.apache.flink.runtime.broadcast.BroadcastVariableManager import org.apache.flink.runtime.clusterframework.BootstrapTools import org.apache.flink.runtime.clusterframework.messages.StopCluster -import org.apache.flink.runtime.clusterframework.types.ResourceID +import org.apache.flink.runtime.clusterframework.types.{AllocationID, ResourceID} import org.apache.flink.runtime.concurrent.{Executors, FutureUtils} import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor import org.apache.flink.runtime.execution.ExecutionState @@ -126,6 +126,7 @@ class TaskManager( protected val memoryManager: MemoryManager, protected val ioManager: IOManager, protected val network: NetworkEnvironment, + protected val taskManagerLocalStateStoresManager: TaskExecutorLocalStateStoresManager, protected val numberOfSlots: Int, protected val highAvailabilityServices: HighAvailabilityServices, protected val taskManagerMetricGroup: TaskManagerMetricGroup) @@ -252,6 +253,12 @@ class TaskManager( case t: Exception => log.error("Network environment did not shutdown properly.", t) } + try { + taskManagerLocalStateStoresManager.shutdown() + } catch { + case t: Exception => log.error("Task state manager did not shutdown properly.", t) + } + try { fileCache.shutdown() } catch { @@ -474,7 +481,7 @@ class TaskManager( log.debug(s"Cannot find task to stop for execution ${executionID})") sender ! decorateMessage(Acknowledge.get()) } - + // cancels a task case CancelTask(executionID) => val task = runningTasks.get(executionID) @@ -984,7 +991,7 @@ class TaskManager( log.error(message, e) throw new RuntimeException(message, e) } - + // watch job manager to detect when it dies context.watch(jobManager) @@ -1070,7 +1077,7 @@ class TaskManager( // clear the key-value location oracle proxy.updateKvStateLocationOracle(HighAvailabilityServices.DEFAULT_JOB_ID, null) } - + // failsafe shutdown of the metrics registry try { taskManagerMetricGroup.close() @@ -1195,18 +1202,21 @@ class TaskManager( config.getTimeout().getSize(), config.getTimeout().getUnit())) - // TODO: wire this so that the manager survives the end of the task - val taskExecutorLocalStateStoresManager = new TaskExecutorLocalStateStoresManager + val jobID = jobInformation.getJobId - val localStateStore = taskExecutorLocalStateStoresManager.localStateStoreForTask( - jobInformation.getJobId, + // Allocation ids do not work properly without flip-6, so we just fake one, based on the jid. + val fakeAllocationID = new AllocationID(jobID.getLowerPart, jobID.getUpperPart) + + val taskLocalStateStore = taskManagerLocalStateStoresManager.localStateStoreForSubtask( + jobID, + fakeAllocationID, taskInformation.getJobVertexId, tdd.getSubtaskIndex) - val slotStateManager = new TaskStateManagerImpl( - jobInformation.getJobId, + val taskStateManager = new TaskStateManagerImpl( + jobID, tdd.getExecutionAttemptId, - localStateStore, + taskLocalStateStore, tdd.getTaskRestore, checkpointResponder) @@ -1224,7 +1234,7 @@ class TaskManager( ioManager, network, bcVarManager, - slotStateManager, + taskStateManager, taskManagerConnection, inputSplitProvider, checkpointResponder, @@ -2013,6 +2023,7 @@ object TaskManager { val taskManagerServices = TaskManagerServices.fromConfiguration( taskManagerServicesConfiguration, resourceID, + actorSystem.dispatcher, EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag, EnvironmentInformation.getMaxJvmHeapMemory) @@ -2030,6 +2041,7 @@ object TaskManager { taskManagerServices.getMemoryManager(), taskManagerServices.getIOManager(), taskManagerServices.getNetworkEnvironment(), + taskManagerServices.getTaskManagerStateStore(), highAvailabilityServices, taskManagerMetricGroup) @@ -2047,6 +2059,7 @@ object TaskManager { memoryManager: MemoryManager, ioManager: IOManager, networkEnvironment: NetworkEnvironment, + taskStateManager: TaskExecutorLocalStateStoresManager, highAvailabilityServices: HighAvailabilityServices, taskManagerMetricGroup: TaskManagerMetricGroup ): Props = { @@ -2058,6 +2071,7 @@ object TaskManager { memoryManager, ioManager, networkEnvironment, + taskStateManager, taskManagerConfig.getNumberSlots(), highAvailabilityServices, taskManagerMetricGroup) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java index cbfe0ed585cc4..32b32cfb3d6e4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java @@ -25,8 +25,9 @@ import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; -import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.util.TestLogger; @@ -93,8 +94,8 @@ public void testFailingCompletedCheckpointStoreAdd() throws Exception { KeyedStateHandle managedKeyedHandle = mock(KeyedStateHandle.class); KeyedStateHandle rawKeyedHandle = mock(KeyedStateHandle.class); - OperatorStateHandle managedOpHandle = mock(OperatorStateHandle.class); - OperatorStateHandle rawOpHandle = mock(OperatorStateHandle.class); + OperatorStateHandle managedOpHandle = mock(OperatorStreamStateHandle.class); + OperatorStateHandle rawOpHandle = mock(OperatorStreamStateHandle.class); final OperatorSubtaskState operatorSubtaskState = spy(new OperatorSubtaskState( managedOpHandle, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index c791fd8d02e88..1b2062a748136 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -42,6 +42,7 @@ import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; import org.apache.flink.runtime.state.PlaceholderStreamStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.StateHandleID; @@ -52,7 +53,6 @@ import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.RecoverableCompletedCheckpointStore; -import org.apache.flink.runtime.util.TestByteStreamStateHandleDeepCompare; import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; @@ -62,7 +62,6 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; - import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.mockito.verification.VerificationMode; @@ -2739,7 +2738,7 @@ public void testReplicateModeStateHandle() { metaInfoMap.put("t-6", new OperatorStateHandle.StateMetaInfo(new long[]{121, 143, 147}, OperatorStateHandle.Mode.BROADCAST)); // this is what a single task will return - OperatorStateHandle osh = new OperatorStateHandle(metaInfoMap, new ByteStreamStateHandle("test", new byte[150])); + OperatorStateHandle osh = new OperatorStreamStateHandle(metaInfoMap, new ByteStreamStateHandle("test", new byte[150])); OperatorStateRepartitioner repartitioner = RoundRobinOperatorStateRepartitioner.INSTANCE; List> repartitionedStates = @@ -2817,7 +2816,7 @@ public static KeyGroupsStateHandle generateKeyGroupState( KeyGroupRangeOffsets keyGroupRangeOffsets = new KeyGroupRangeOffsets(keyGroupRange, serializedDataWithOffsets.f1.get(0)); - ByteStreamStateHandle allSerializedStatesHandle = new TestByteStreamStateHandleDeepCompare( + ByteStreamStateHandle allSerializedStatesHandle = new ByteStreamStateHandle( String.valueOf(UUID.randomUUID()), serializedDataWithOffsets.f0); @@ -2936,11 +2935,11 @@ private static OperatorStateHandle generatePartitionableStateHandle( ++idx; } - ByteStreamStateHandle streamStateHandle = new TestByteStreamStateHandleDeepCompare( + ByteStreamStateHandle streamStateHandle = new ByteStreamStateHandle( String.valueOf(UUID.randomUUID()), serializationWithOffsets.f0); - return new OperatorStateHandle(offsetsMap, streamStateHandle); + return new OperatorStreamStateHandle(offsetsMap, streamStateHandle); } static ExecutionJobVertex mockExecutionJobVertex( @@ -3265,7 +3264,7 @@ private void doTestPartitionableStateRepartitioning( } OperatorStateHandle.Mode mode = r.nextInt(10) == 0 ? - OperatorStateHandle.Mode.UNION : OperatorStateHandle.Mode.SPLIT_DISTRIBUTE; + OperatorStateHandle.Mode.UNION : OperatorStateHandle.Mode.SPLIT_DISTRIBUTE; namedStatesToOffsets.put( "State-" + s, new OperatorStateHandle.StateMetaInfo(offs, mode)); @@ -3282,7 +3281,7 @@ private void doTestPartitionableStateRepartitioning( } previousParallelOpInstanceStates.add( - new OperatorStateHandle(namedStatesToOffsets, new FileStateHandle(fakePath, -1))); + new OperatorStreamStateHandle(namedStatesToOffsets, new FileStateHandle(fakePath, -1))); } Map>> expected = new HashMap<>(); @@ -3769,10 +3768,10 @@ private void performIncrementalCheckpoint( OperatorSubtaskState operatorSubtaskState = spy(new OperatorSubtaskState( - Collections.emptyList(), - Collections.emptyList(), - Collections.singletonList(managedState), - Collections.emptyList())); + StateObjectCollection.empty(), + StateObjectCollection.empty(), + StateObjectCollection.singleton(managedState), + StateObjectCollection.empty())); Map opStates = new HashMap<>(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointMetadataLoadingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointMetadataLoadingTest.java index 70794c6d5fbc2..ff787ec60b05f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointMetadataLoadingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointMetadataLoadingTest.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; -import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; @@ -69,7 +69,7 @@ public void testLoadAndValidateSavepoint() throws Exception { OperatorID operatorID = OperatorID.fromJobVertexID(jobVertexID); OperatorSubtaskState subtaskState = new OperatorSubtaskState( - new OperatorStateHandle( + new OperatorStreamStateHandle( Collections.emptyMap(), new ByteStreamStateHandle("testHandler", new byte[0])), null, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java index ab353a9ffab56..af6ec71b98d05 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java @@ -30,7 +30,6 @@ import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyedStateHandle; -import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; @@ -123,10 +122,10 @@ public void testSetState() { subtaskStates.putSubtaskStateByOperatorID( OperatorID.fromJobVertexID(statefulId), new OperatorSubtaskState( - Collections.emptyList(), - Collections.emptyList(), - Collections.singletonList(serializedKeyGroupStates), - Collections.emptyList())); + StateObjectCollection.empty(), + StateObjectCollection.empty(), + StateObjectCollection.singleton(serializedKeyGroupStates), + StateObjectCollection.empty())); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec1.getAttemptId(), checkpointId, new CheckpointMetrics(), subtaskStates)); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec2.getAttemptId(), checkpointId, new CheckpointMetrics(), subtaskStates)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java index 96c95eac61a7e..815696496633c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java @@ -328,7 +328,7 @@ public int hashCode() { } } - static class TestOperatorSubtaskState extends OperatorSubtaskState { + public static class TestOperatorSubtaskState extends OperatorSubtaskState { private static final long serialVersionUID = 522580433699164230L; boolean registered; @@ -359,6 +359,14 @@ public void reset() { registered = false; discarded = false; } + + public boolean isRegistered() { + return registered; + } + + public boolean isDiscarded() { + return discarded; + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskStateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskStateTest.java new file mode 100644 index 0000000000000..09c9efb695994 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskStateTest.java @@ -0,0 +1,292 @@ +/* + * 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.runtime.checkpoint; + +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.util.TestLogger; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Random; +import java.util.function.Function; + +import static org.apache.flink.runtime.checkpoint.StateHandleDummyUtil.createNewKeyedStateHandle; +import static org.apache.flink.runtime.checkpoint.StateHandleDummyUtil.createNewOperatorStateHandle; +import static org.apache.flink.runtime.checkpoint.StateHandleDummyUtil.deepDummyCopy; + +public class PrioritizedOperatorSubtaskStateTest extends TestLogger { + + private final Random random = new Random(0x42); + + /** + * This tests attempts to test (almost) the full space of significantly different options for verifying and + * prioritizing {@link OperatorSubtaskState} options for local recovery over primary/remote state handles. + */ + @Test + public void testPrioritization() { + + for (int i = 0; i < 81; ++i) { // 3^4 possible configurations. + + OperatorSubtaskState primaryAndFallback = generateForConfiguration(i); + + for (int j = 0; j < 9; ++j) { // we test 3^2 configurations. + // mode 0: one valid state handle (deep copy of original). + // mode 1: empty StateHandleCollection. + // mode 2: one invalid state handle (e.g. wrong key group, different meta data) + int modeFirst = j % 3; + OperatorSubtaskState bestAlternative = createAlternativeSubtaskState(primaryAndFallback, modeFirst); + int modeSecond = (j / 3) % 3; + OperatorSubtaskState secondBestAlternative = createAlternativeSubtaskState(primaryAndFallback, modeSecond); + + List orderedAlternativesList = + Arrays.asList(bestAlternative, secondBestAlternative); + List validAlternativesList = new ArrayList<>(3); + if (modeFirst == 0) { + validAlternativesList.add(bestAlternative); + } + if (modeSecond == 0) { + validAlternativesList.add(secondBestAlternative); + } + validAlternativesList.add(primaryAndFallback); + + PrioritizedOperatorSubtaskState.Builder builder = + new PrioritizedOperatorSubtaskState.Builder(primaryAndFallback, orderedAlternativesList); + + PrioritizedOperatorSubtaskState prioritizedOperatorSubtaskState = builder.build(); + + OperatorSubtaskState[] validAlternatives = + validAlternativesList.toArray(new OperatorSubtaskState[validAlternativesList.size()]); + + OperatorSubtaskState[] onlyPrimary = + new OperatorSubtaskState[]{primaryAndFallback}; + + Assert.assertTrue(checkResultAsExpected( + OperatorSubtaskState::getManagedOperatorState, + PrioritizedOperatorSubtaskState::getPrioritizedManagedOperatorState, + prioritizedOperatorSubtaskState, + primaryAndFallback.getManagedOperatorState().size() == 1 ? validAlternatives : onlyPrimary)); + + Assert.assertTrue(checkResultAsExpected( + OperatorSubtaskState::getManagedKeyedState, + PrioritizedOperatorSubtaskState::getPrioritizedManagedKeyedState, + prioritizedOperatorSubtaskState, + primaryAndFallback.getManagedKeyedState().size() == 1 ? validAlternatives : onlyPrimary)); + + Assert.assertTrue(checkResultAsExpected( + OperatorSubtaskState::getRawOperatorState, + PrioritizedOperatorSubtaskState::getPrioritizedRawOperatorState, + prioritizedOperatorSubtaskState, + primaryAndFallback.getRawOperatorState().size() == 1 ? validAlternatives : onlyPrimary)); + + Assert.assertTrue(checkResultAsExpected( + OperatorSubtaskState::getRawKeyedState, + PrioritizedOperatorSubtaskState::getPrioritizedRawKeyedState, + prioritizedOperatorSubtaskState, + primaryAndFallback.getRawKeyedState().size() == 1 ? validAlternatives : onlyPrimary)); + } + } + } + + /** + * Generator for all 3^4 = 81 possible configurations of a OperatorSubtaskState: + * - 4 different sub-states: + * managed/raw + operator/keyed. + * - 3 different options per sub-state: + * empty (simulate no state), single handle (simulate recovery), 2 handles (simulate e.g. rescaling) + */ + private OperatorSubtaskState generateForConfiguration(int conf) { + + Preconditions.checkState(conf >= 0 && conf <= 80); // 3^4 + final int numModes = 3; + + KeyGroupRange keyGroupRange = new KeyGroupRange(0, 4); + KeyGroupRange keyGroupRange1 = new KeyGroupRange(0, 2); + KeyGroupRange keyGroupRange2 = new KeyGroupRange(3, 4); + + int div = 1; + int mode = (conf / div) % numModes; + StateObjectCollection s1 = + mode == 0 ? + StateObjectCollection.empty() : + mode == 1 ? + new StateObjectCollection<>( + Collections.singletonList(createNewOperatorStateHandle(2, random))) : + new StateObjectCollection<>( + Arrays.asList( + createNewOperatorStateHandle(2, random), + createNewOperatorStateHandle(2, random))); + div *= numModes; + mode = (conf / div) % numModes; + StateObjectCollection s2 = + mode == 0 ? + StateObjectCollection.empty() : + mode == 1 ? + new StateObjectCollection<>( + Collections.singletonList(createNewOperatorStateHandle(2, random))) : + new StateObjectCollection<>( + Arrays.asList( + createNewOperatorStateHandle(2, random), + createNewOperatorStateHandle(2, random))); + + div *= numModes; + mode = (conf / div) % numModes; + StateObjectCollection s3 = + mode == 0 ? + StateObjectCollection.empty() : + mode == 1 ? + new StateObjectCollection<>( + Collections.singletonList(createNewKeyedStateHandle(keyGroupRange))) : + new StateObjectCollection<>( + Arrays.asList( + createNewKeyedStateHandle(keyGroupRange1), + createNewKeyedStateHandle(keyGroupRange2))); + + div *= numModes; + mode = (conf / div) % numModes; + StateObjectCollection s4 = + mode == 0 ? + StateObjectCollection.empty() : + mode == 1 ? + new StateObjectCollection<>( + Collections.singletonList(createNewKeyedStateHandle(keyGroupRange))) : + new StateObjectCollection<>( + Arrays.asList( + createNewKeyedStateHandle(keyGroupRange1), + createNewKeyedStateHandle(keyGroupRange2))); + + return new OperatorSubtaskState(s1, s2, s3, s4); + } + + /** + * For all 4 sub-states: + * - mode 0: One valid state handle (deep copy of original). Only this creates an OperatorSubtaskState that + * qualifies as alternative. + * - mode 1: Empty StateHandleCollection. + * - mode 2: One invalid state handle (e.g. wrong key group, different meta data) + */ + private OperatorSubtaskState createAlternativeSubtaskState(OperatorSubtaskState primaryOriginal, int mode) { + switch (mode) { + case 0: + return new OperatorSubtaskState( + deepCopyFirstElement(primaryOriginal.getManagedOperatorState()), + deepCopyFirstElement(primaryOriginal.getRawOperatorState()), + deepCopyFirstElement(primaryOriginal.getManagedKeyedState()), + deepCopyFirstElement(primaryOriginal.getRawKeyedState())); + case 1: + return new OperatorSubtaskState(); + case 2: + KeyGroupRange otherRange = new KeyGroupRange(8, 16); + int numNamedStates = 2; + return new OperatorSubtaskState( + createNewOperatorStateHandle(numNamedStates, random), + createNewOperatorStateHandle(numNamedStates, random), + createNewKeyedStateHandle(otherRange), + createNewKeyedStateHandle(otherRange)); + default: + throw new IllegalArgumentException("Mode: " + mode); + } + } + + private boolean checkResultAsExpected( + Function> extractor, + Function>> extractor2, + PrioritizedOperatorSubtaskState prioritizedResult, + OperatorSubtaskState... expectedOrdered) { + + List> collector = new ArrayList<>(expectedOrdered.length); + for (OperatorSubtaskState operatorSubtaskState : expectedOrdered) { + collector.add(extractor.apply(operatorSubtaskState)); + } + + return checkRepresentSameOrder( + extractor2.apply(prioritizedResult), + collector.toArray(new StateObjectCollection[collector.size()])); + } + + private boolean checkRepresentSameOrder( + Iterator> ordered, + StateObjectCollection... expectedOrder) { + + for (StateObjectCollection objects : expectedOrder) { + if (!ordered.hasNext() || !checkContainedObjectsReferentialEquality(objects, ordered.next())) { + return false; + } + } + + return !ordered.hasNext(); + } + + /** + * Returns true iff, in iteration order, all objects in the first collection are equal by reference to their + * corresponding object (by order) in the second collection and the size of the collections is equal. + */ + public boolean checkContainedObjectsReferentialEquality(StateObjectCollection a, StateObjectCollection b) { + + if (a == b) { + return true; + } + + if(a == null || b == null) { + return false; + } + + if (a.size() != b.size()) { + return false; + } + + Iterator bIter = b.iterator(); + for (StateObject stateObject : a) { + if (!bIter.hasNext() || bIter.next() != stateObject) { + return false; + } + } + return true; + } + + /** + * Creates a deep copy of the first state object in the given collection, or null if the collection is empy. + */ + private T deepCopyFirstElement(StateObjectCollection original) { + if (original.isEmpty()) { + return null; + } + + T stateObject = original.iterator().next(); + StateObject result; + if (stateObject instanceof OperatorStreamStateHandle) { + result = deepDummyCopy((OperatorStateHandle) stateObject); + } else if (stateObject instanceof KeyedStateHandle) { + result = deepDummyCopy((KeyedStateHandle) stateObject); + } else { + throw new IllegalStateException(); + } + return (T) result; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StateHandleDummyUtil.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StateHandleDummyUtil.java new file mode 100644 index 0000000000000..548ca18ee08e0 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StateHandleDummyUtil.java @@ -0,0 +1,139 @@ +/* + * 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.runtime.checkpoint; + +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; +import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; + +import java.util.HashMap; +import java.util.Map; +import java.util.Random; +import java.util.UUID; + +public class StateHandleDummyUtil { + + /** + * Creates a new test {@link OperatorStreamStateHandle} with a given number of randomly created named states. + */ + public static OperatorStateHandle createNewOperatorStateHandle(int numNamedStates, Random random) { + Map operatorStateMetaData = new HashMap<>(numNamedStates); + byte[] streamData = new byte[numNamedStates * 4]; + random.nextBytes(streamData); + long off = 0; + for (int i = 0; i < numNamedStates; ++i) { + long[] offsets = new long[4]; + for (int o = 0; o < offsets.length; ++o) { + offsets[o] = off++; + } + OperatorStateHandle.StateMetaInfo metaInfo = + new OperatorStateHandle.StateMetaInfo(offsets, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE); + operatorStateMetaData.put(String.valueOf(UUID.randomUUID()), metaInfo); + } + ByteStreamStateHandle byteStreamStateHandle = + new ByteStreamStateHandle(String.valueOf(UUID.randomUUID()), streamData); + return new OperatorStreamStateHandle(operatorStateMetaData, byteStreamStateHandle); + } + + /** + * Creates a new test {@link KeyedStateHandle} for the given key-group. + */ + public static KeyedStateHandle createNewKeyedStateHandle(KeyGroupRange keyGroupRange) { + return new DummyKeyedStateHandle(keyGroupRange); + } + + /** + * Creates a deep copy of the given {@link OperatorStreamStateHandle}. + */ + public static OperatorStateHandle deepDummyCopy(OperatorStateHandle original) { + + if (original == null) { + return null; + } + + ByteStreamStateHandle stateHandle = (ByteStreamStateHandle) original.getDelegateStateHandle(); + ByteStreamStateHandle stateHandleCopy = new ByteStreamStateHandle( + String.valueOf(stateHandle.getHandleName()), + stateHandle.getData().clone()); + Map offsets = original.getStateNameToPartitionOffsets(); + Map offsetsCopy = new HashMap<>(offsets.size()); + + for (Map.Entry entry : offsets.entrySet()) { + OperatorStateHandle.StateMetaInfo metaInfo = entry.getValue(); + OperatorStateHandle.StateMetaInfo metaInfoCopy = + new OperatorStateHandle.StateMetaInfo(metaInfo.getOffsets(), metaInfo.getDistributionMode()); + offsetsCopy.put(String.valueOf(entry.getKey()), metaInfoCopy); + } + return new OperatorStreamStateHandle(offsetsCopy, stateHandleCopy); + } + + /** + * Creates deep copy of the given {@link KeyedStateHandle}. + */ + public static KeyedStateHandle deepDummyCopy(KeyedStateHandle original) { + + if (original == null) { + return null; + } + + KeyGroupRange keyGroupRange = original.getKeyGroupRange(); + return new DummyKeyedStateHandle( + new KeyGroupRange(keyGroupRange.getStartKeyGroup(), keyGroupRange.getEndKeyGroup())); + } + + /** + * KeyedStateHandle that only holds a key-group information. + */ + private static class DummyKeyedStateHandle implements KeyedStateHandle { + + private static final long serialVersionUID = 1L; + + private final KeyGroupRange keyGroupRange; + + private DummyKeyedStateHandle(KeyGroupRange keyGroupRange) { + this.keyGroupRange = keyGroupRange; + } + + @Override + public KeyGroupRange getKeyGroupRange() { + return keyGroupRange; + } + + @Override + public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) { + return new DummyKeyedStateHandle(this.keyGroupRange.getIntersection(keyGroupRange)); + } + + @Override + public void registerSharedStates(SharedStateRegistry stateRegistry) { + } + + @Override + public void discardState() throws Exception { + } + + @Override + public long getStateSize() { + return 0L; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StateObjectCollectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StateObjectCollectionTest.java new file mode 100644 index 0000000000000..b12ee27ea3f07 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StateObjectCollectionTest.java @@ -0,0 +1,70 @@ +/* + * 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.runtime.checkpoint; + +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.util.MethodForwardingTestUtil; +import org.apache.flink.util.TestLogger; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.function.Function; + +import static org.mockito.Mockito.mock; + +/** + * Tests for {@link StateObjectCollection}. + */ +public class StateObjectCollectionTest extends TestLogger { + + @Test + public void testEmptyCollection() { + StateObjectCollection empty = StateObjectCollection.empty(); + Assert.assertEquals(0, empty.getStateSize()); + } + + @Test + public void testForwardingCollectionMethods() throws Exception { + MethodForwardingTestUtil.testMethodForwarding( + Collection.class, + ((Function) StateObjectCollection::new)); + } + + @Test + public void testForwardingStateObjectMethods() throws Exception { + MethodForwardingTestUtil.testMethodForwarding( + StateObject.class, + object -> new StateObjectCollection<>(Collections.singletonList(object))); + } + + @Test + public void testHasState() { + StateObjectCollection stateObjects = new StateObjectCollection<>(new ArrayList<>()); + Assert.assertFalse(stateObjects.hasState()); + + stateObjects = new StateObjectCollection<>(Collections.singletonList(null)); + Assert.assertFalse(stateObjects.hasState()); + + stateObjects = new StateObjectCollection<>(Collections.singletonList(mock(StateObject.class))); + Assert.assertTrue(stateObjects.hasState()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TaskStateSnapshotTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TaskStateSnapshotTest.java new file mode 100644 index 0000000000000..76f39064fca9f --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TaskStateSnapshotTest.java @@ -0,0 +1,129 @@ +/* + * 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.runtime.checkpoint; + +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.util.TestLogger; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Random; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +public class TaskStateSnapshotTest extends TestLogger { + + @Test + public void putGetSubtaskStateByOperatorID() { + TaskStateSnapshot taskStateSnapshot = new TaskStateSnapshot(); + + OperatorID operatorID_1 = new OperatorID(); + OperatorID operatorID_2 = new OperatorID(); + OperatorSubtaskState operatorSubtaskState_1 = new OperatorSubtaskState(); + OperatorSubtaskState operatorSubtaskState_2 = new OperatorSubtaskState(); + OperatorSubtaskState operatorSubtaskState_1_replace = new OperatorSubtaskState(); + + Assert.assertNull(taskStateSnapshot.getSubtaskStateByOperatorID(operatorID_1)); + Assert.assertNull(taskStateSnapshot.getSubtaskStateByOperatorID(operatorID_2)); + taskStateSnapshot.putSubtaskStateByOperatorID(operatorID_1, operatorSubtaskState_1); + taskStateSnapshot.putSubtaskStateByOperatorID(operatorID_2, operatorSubtaskState_2); + Assert.assertEquals(operatorSubtaskState_1, taskStateSnapshot.getSubtaskStateByOperatorID(operatorID_1)); + Assert.assertEquals(operatorSubtaskState_2, taskStateSnapshot.getSubtaskStateByOperatorID(operatorID_2)); + Assert.assertEquals(operatorSubtaskState_1, taskStateSnapshot.putSubtaskStateByOperatorID(operatorID_1, operatorSubtaskState_1_replace)); + Assert.assertEquals(operatorSubtaskState_1_replace, taskStateSnapshot.getSubtaskStateByOperatorID(operatorID_1)); + } + + @Test + public void hasState() { + Random random = new Random(0x42); + TaskStateSnapshot taskStateSnapshot = new TaskStateSnapshot(); + Assert.assertFalse(taskStateSnapshot.hasState()); + + OperatorSubtaskState emptyOperatorSubtaskState = new OperatorSubtaskState(); + Assert.assertFalse(emptyOperatorSubtaskState.hasState()); + taskStateSnapshot.putSubtaskStateByOperatorID(new OperatorID(), emptyOperatorSubtaskState); + Assert.assertFalse(taskStateSnapshot.hasState()); + + OperatorStateHandle stateHandle = StateHandleDummyUtil.createNewOperatorStateHandle(2, random); + OperatorSubtaskState nonEmptyOperatorSubtaskState = new OperatorSubtaskState( + stateHandle, + null, + null, + null + ); + + Assert.assertTrue(nonEmptyOperatorSubtaskState.hasState()); + taskStateSnapshot.putSubtaskStateByOperatorID(new OperatorID(), nonEmptyOperatorSubtaskState); + Assert.assertTrue(taskStateSnapshot.hasState()); + } + + @Test + public void discardState() throws Exception { + TaskStateSnapshot taskStateSnapshot = new TaskStateSnapshot(); + OperatorID operatorID_1 = new OperatorID(); + OperatorID operatorID_2 = new OperatorID(); + + OperatorSubtaskState operatorSubtaskState_1 = mock(OperatorSubtaskState.class); + OperatorSubtaskState operatorSubtaskState_2 = mock(OperatorSubtaskState.class); + + taskStateSnapshot.putSubtaskStateByOperatorID(operatorID_1, operatorSubtaskState_1); + taskStateSnapshot.putSubtaskStateByOperatorID(operatorID_2, operatorSubtaskState_2); + + taskStateSnapshot.discardState(); + verify(operatorSubtaskState_1).discardState(); + verify(operatorSubtaskState_2).discardState(); + } + + @Test + public void getStateSize() { + Random random = new Random(0x42); + TaskStateSnapshot taskStateSnapshot = new TaskStateSnapshot(); + Assert.assertEquals(0, taskStateSnapshot.getStateSize()); + + OperatorSubtaskState emptyOperatorSubtaskState = new OperatorSubtaskState(); + Assert.assertFalse(emptyOperatorSubtaskState.hasState()); + taskStateSnapshot.putSubtaskStateByOperatorID(new OperatorID(), emptyOperatorSubtaskState); + Assert.assertEquals(0, taskStateSnapshot.getStateSize()); + + + OperatorStateHandle stateHandle_1 = StateHandleDummyUtil.createNewOperatorStateHandle(2, random); + OperatorSubtaskState nonEmptyOperatorSubtaskState_1 = new OperatorSubtaskState( + stateHandle_1, + null, + null, + null + ); + + OperatorStateHandle stateHandle_2 = StateHandleDummyUtil.createNewOperatorStateHandle(2, random); + OperatorSubtaskState nonEmptyOperatorSubtaskState_2 = new OperatorSubtaskState( + null, + stateHandle_2, + null, + null + ); + + taskStateSnapshot.putSubtaskStateByOperatorID(new OperatorID(), nonEmptyOperatorSubtaskState_1); + taskStateSnapshot.putSubtaskStateByOperatorID(new OperatorID(), nonEmptyOperatorSubtaskState_2); + + long totalSize = stateHandle_1.getStateSize() + stateHandle_2.getStateSize(); + Assert.assertEquals(totalSize, taskStateSnapshot.getStateSize()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/CheckpointTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/CheckpointTestUtils.java index d1d67ff940268..1963766fb1d94 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/CheckpointTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/savepoint/CheckpointTestUtils.java @@ -33,10 +33,10 @@ import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; -import org.apache.flink.runtime.state.OperatorStateHandle.StateMetaInfo; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; import org.apache.flink.runtime.state.StateHandleID; import org.apache.flink.runtime.state.StreamStateHandle; -import org.apache.flink.runtime.util.TestByteStreamStateHandleDeepCompare; +import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.apache.flink.util.StringUtils; import java.util.ArrayList; @@ -87,24 +87,24 @@ public static Collection createOperatorStates( for (int subtaskIdx = 0; subtaskIdx < numSubtasksPerTask; subtaskIdx++) { StreamStateHandle operatorStateBackend = - new TestByteStreamStateHandleDeepCompare("b", ("Beautiful").getBytes(ConfigConstants.DEFAULT_CHARSET)); + new ByteStreamStateHandle("b", ("Beautiful").getBytes(ConfigConstants.DEFAULT_CHARSET)); StreamStateHandle operatorStateStream = - new TestByteStreamStateHandleDeepCompare("b", ("Beautiful").getBytes(ConfigConstants.DEFAULT_CHARSET)); + new ByteStreamStateHandle("b", ("Beautiful").getBytes(ConfigConstants.DEFAULT_CHARSET)); OperatorStateHandle operatorStateHandleBackend = null; OperatorStateHandle operatorStateHandleStream = null; - Map offsetsMap = new HashMap<>(); + Map offsetsMap = new HashMap<>(); offsetsMap.put("A", new OperatorStateHandle.StateMetaInfo(new long[]{0, 10, 20}, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)); offsetsMap.put("B", new OperatorStateHandle.StateMetaInfo(new long[]{30, 40, 50}, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)); offsetsMap.put("C", new OperatorStateHandle.StateMetaInfo(new long[]{60, 70, 80}, OperatorStateHandle.Mode.UNION)); if (hasOperatorStateBackend) { - operatorStateHandleBackend = new OperatorStateHandle(offsetsMap, operatorStateBackend); + operatorStateHandleBackend = new OperatorStreamStateHandle(offsetsMap, operatorStateBackend); } if (hasOperatorStateStream) { - operatorStateHandleStream = new OperatorStateHandle(offsetsMap, operatorStateStream); + operatorStateHandleStream = new OperatorStreamStateHandle(offsetsMap, operatorStateStream); } KeyedStateHandle keyedStateBackend = null; @@ -173,23 +173,23 @@ public static Collection createTaskStates( for (int chainIdx = 0; chainIdx < chainLength; ++chainIdx) { StreamStateHandle operatorStateBackend = - new TestByteStreamStateHandleDeepCompare("b-" + chainIdx, ("Beautiful-" + chainIdx).getBytes(ConfigConstants.DEFAULT_CHARSET)); + new ByteStreamStateHandle("b-" + chainIdx, ("Beautiful-" + chainIdx).getBytes(ConfigConstants.DEFAULT_CHARSET)); StreamStateHandle operatorStateStream = - new TestByteStreamStateHandleDeepCompare("b-" + chainIdx, ("Beautiful-" + chainIdx).getBytes(ConfigConstants.DEFAULT_CHARSET)); - Map offsetsMap = new HashMap<>(); + new ByteStreamStateHandle("b-" + chainIdx, ("Beautiful-" + chainIdx).getBytes(ConfigConstants.DEFAULT_CHARSET)); + Map offsetsMap = new HashMap<>(); offsetsMap.put("A", new OperatorStateHandle.StateMetaInfo(new long[]{0, 10, 20}, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)); offsetsMap.put("B", new OperatorStateHandle.StateMetaInfo(new long[]{30, 40, 50}, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)); offsetsMap.put("C", new OperatorStateHandle.StateMetaInfo(new long[]{60, 70, 80}, OperatorStateHandle.Mode.UNION)); if (chainIdx != noOperatorStateBackendAtIndex) { OperatorStateHandle operatorStateHandleBackend = - new OperatorStateHandle(offsetsMap, operatorStateBackend); + new OperatorStreamStateHandle(offsetsMap, operatorStateBackend); operatorStatesBackend.add(operatorStateHandleBackend); } if (chainIdx != noOperatorStateStreamAtIndex) { OperatorStateHandle operatorStateHandleStream = - new OperatorStateHandle(offsetsMap, operatorStateStream); + new OperatorStreamStateHandle(offsetsMap, operatorStateStream); operatorStatesStream.add(operatorStateHandleStream); } } @@ -284,7 +284,7 @@ public static KeyGroupsStateHandle createDummyKeyGroupStateHandle(Random rnd) { } public static StreamStateHandle createDummyStreamStateHandle(Random rnd) { - return new TestByteStreamStateHandleDeepCompare( + return new ByteStreamStateHandle( String.valueOf(createRandomUUID(rnd)), String.valueOf(createRandomUUID(rnd)).getBytes(ConfigConstants.DEFAULT_CHARSET)); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java index 8f1e12c07b2a8..005dd98707883 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java @@ -37,7 +37,9 @@ import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.PrioritizedOperatorSubtaskState; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointIDCounter; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.checkpoint.TestingCheckpointRecoveryFactory; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -68,6 +70,7 @@ import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; import org.apache.flink.runtime.state.TaskStateManager; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.apache.flink.runtime.taskmanager.TaskManager; @@ -79,7 +82,6 @@ import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testutils.InMemorySubmittedJobGraphStore; import org.apache.flink.runtime.testutils.RecoverableCompletedCheckpointStore; -import org.apache.flink.runtime.util.TestByteStreamStateHandleDeepCompare; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; @@ -95,6 +97,7 @@ import akka.testkit.CallingThreadDispatcher; import akka.testkit.JavaTestKit; import org.junit.AfterClass; +import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; @@ -105,6 +108,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; @@ -479,17 +483,21 @@ public void invoke() throws Exception { OperatorID operatorID = OperatorID.fromJobVertexID(getEnvironment().getJobVertexId()); TaskStateManager taskStateManager = getEnvironment().getTaskStateManager(); - OperatorSubtaskState subtaskState = taskStateManager.operatorStates(operatorID); + PrioritizedOperatorSubtaskState subtaskState = taskStateManager.prioritizedOperatorState(operatorID); - if(subtaskState != null) { - int subtaskIndex = getIndexInSubtaskGroup(); - if (subtaskIndex < BlockingStatefulInvokable.recoveredStates.length) { - OperatorStateHandle operatorStateHandle = subtaskState.getManagedOperatorState().iterator().next(); + int subtaskIndex = getIndexInSubtaskGroup(); + if (subtaskIndex < BlockingStatefulInvokable.recoveredStates.length) { + Iterator iterator = + subtaskState.getJobManagerManagedOperatorState().iterator(); + + if (iterator.hasNext()) { + OperatorStateHandle operatorStateHandle = iterator.next(); try (FSDataInputStream in = operatorStateHandle.openInputStream()) { BlockingStatefulInvokable.recoveredStates[subtaskIndex] = InstantiationUtil.deserializeObject(in, getUserCodeClassLoader()); } } + Assert.assertFalse(iterator.hasNext()); } LATCH.await(); @@ -516,7 +524,7 @@ public BlockingStatefulInvokable(Environment environment) { @Override public boolean triggerCheckpoint(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions) throws Exception { - ByteStreamStateHandle byteStreamStateHandle = new TestByteStreamStateHandleDeepCompare( + ByteStreamStateHandle byteStreamStateHandle = new ByteStreamStateHandle( String.valueOf(UUID.randomUUID()), InstantiationUtil.serializeObject(checkpointMetaData.getCheckpointId())); @@ -525,16 +533,16 @@ public boolean triggerCheckpoint(CheckpointMetaData checkpointMetaData, Checkpoi "test-state", new OperatorStateHandle.StateMetaInfo(new long[]{0L}, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)); - OperatorStateHandle operatorStateHandle = new OperatorStateHandle(stateNameToPartitionOffsets, byteStreamStateHandle); + OperatorStateHandle operatorStateHandle = new OperatorStreamStateHandle(stateNameToPartitionOffsets, byteStreamStateHandle); TaskStateSnapshot checkpointStateHandles = new TaskStateSnapshot(); checkpointStateHandles.putSubtaskStateByOperatorID( OperatorID.fromJobVertexID(getEnvironment().getJobVertexId()), new OperatorSubtaskState( - Collections.singletonList(operatorStateHandle), - Collections.emptyList(), - Collections.emptyList(), - Collections.emptyList())); + StateObjectCollection.singleton(operatorStateHandle), + StateObjectCollection.empty(), + StateObjectCollection.empty(), + StateObjectCollection.empty())); getEnvironment().acknowledgeCheckpoint( checkpointMetaData.getCheckpointId(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java index 9454d90e05c0b..c1a7b536721a5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperSubmittedJobGraphsStoreITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.jobmanager; -import akka.actor.ActorRef; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.akka.ListeningBehaviour; import org.apache.flink.runtime.concurrent.Executors; @@ -28,11 +27,12 @@ import org.apache.flink.runtime.state.RetrievableStateHandle; import org.apache.flink.runtime.state.RetrievableStreamStateHandle; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; -import org.apache.flink.runtime.util.TestByteStreamStateHandleDeepCompare; import org.apache.flink.runtime.zookeeper.RetrievableStateStorageHelper; import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.TestLogger; + +import akka.actor.ActorRef; import org.junit.AfterClass; import org.junit.Before; import org.junit.Test; @@ -65,7 +65,7 @@ public class ZooKeeperSubmittedJobGraphsStoreITCase extends TestLogger { private final static RetrievableStateStorageHelper localStateStorage = new RetrievableStateStorageHelper() { @Override public RetrievableStateHandle store(SubmittedJobGraph state) throws IOException { - ByteStreamStateHandle byteStreamStateHandle = new TestByteStreamStateHandleDeepCompare( + ByteStreamStateHandle byteStreamStateHandle = new ByteStreamStateHandle( String.valueOf(UUID.randomUUID()), InstantiationUtil.serializeObject(state)); return new RetrievableStreamStateHandle<>(byteStreamStateHandle); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java index 418625599fb8b..b7aa97dfdfb4b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java @@ -91,7 +91,7 @@ public static Execution getDummyTask() { when(vertex.toString()).thenReturn("TEST-VERTEX"); when(vertex.getJobVertex()).thenReturn(executionJobVertex); when(vertex.getJobvertexId()).thenReturn(new JobVertexID()); - + Execution execution = mock(Execution.class); when(execution.getVertex()).thenReturn(vertex); @@ -126,6 +126,7 @@ public static Execution getTestVertex(Collection outputs; - private final JobID jobID = new JobID(); + private final JobID jobID; + + private final JobVertexID jobVertexID; private final BroadcastVariableManager bcVarManager = new BroadcastVariableManager(); @@ -170,11 +172,11 @@ public MockEnvironment( bufferSize, taskConfiguration, executionConfig, + taskStateManager, maxParallelism, parallelism, subtaskIndex, - Thread.currentThread().getContextClassLoader(), - taskStateManager); + Thread.currentThread().getContextClassLoader()); } @@ -185,11 +187,45 @@ public MockEnvironment( int bufferSize, Configuration taskConfiguration, ExecutionConfig executionConfig, + TaskStateManager taskStateManager, int maxParallelism, int parallelism, int subtaskIndex, - ClassLoader userCodeClassLoader, - TaskStateManager taskStateManager) { + ClassLoader userCodeClassLoader) { + this( + new JobID(), + new JobVertexID(), + taskName, + memorySize, + inputSplitProvider, + bufferSize, + taskConfiguration, + executionConfig, + taskStateManager, + maxParallelism, + parallelism, + subtaskIndex, + userCodeClassLoader); + } + + public MockEnvironment( + JobID jobID, + JobVertexID jobVertexID, + String taskName, + long memorySize, + MockInputSplitProvider inputSplitProvider, + int bufferSize, + Configuration taskConfiguration, + ExecutionConfig executionConfig, + TaskStateManager taskStateManager, + int maxParallelism, + int parallelism, + int subtaskIndex, + ClassLoader userCodeClassLoader) { + + this.jobID = jobID; + this.jobVertexID = jobVertexID; + this.taskInfo = new TaskInfo(taskName, maxParallelism, subtaskIndex, parallelism, 0); this.jobConfiguration = new Configuration(); this.taskConfiguration = taskConfiguration; @@ -325,7 +361,7 @@ public TaskEventDispatcher getTaskEventDispatcher() { @Override public JobVertexID getJobVertexId() { - return new JobVertexID(new byte[16]); + return jobVertexID; } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/CheckpointStreamWithResultProviderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/CheckpointStreamWithResultProviderTest.java new file mode 100644 index 0000000000000..2af25d9f9909e --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/CheckpointStreamWithResultProviderTest.java @@ -0,0 +1,210 @@ +/* + * 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.runtime.state; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory; +import org.apache.flink.util.MethodForwardingTestUtil; +import org.apache.flink.util.TestLogger; + +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; + +public class CheckpointStreamWithResultProviderTest extends TestLogger { + + private static TemporaryFolder temporaryFolder; + + @BeforeClass + public static void beforeClass() throws IOException { + temporaryFolder = new TemporaryFolder(); + temporaryFolder.create(); + } + + @AfterClass + public static void afterClass() { + temporaryFolder.delete(); + } + + @Test + public void testFactory() throws Exception { + + CheckpointStreamFactory primaryFactory = createCheckpointStreamFactory(); + try ( + CheckpointStreamWithResultProvider primaryOnly = + CheckpointStreamWithResultProvider.createSimpleStream( + CheckpointedStateScope.EXCLUSIVE, + primaryFactory)) { + + Assert.assertTrue(primaryOnly instanceof CheckpointStreamWithResultProvider.PrimaryStreamOnly); + } + + LocalRecoveryDirectoryProvider directoryProvider = createLocalRecoveryDirectoryProvider(); + try ( + CheckpointStreamWithResultProvider primaryAndSecondary = + CheckpointStreamWithResultProvider.createDuplicatingStream( + 42L, + CheckpointedStateScope.EXCLUSIVE, + primaryFactory, + directoryProvider)) { + + Assert.assertTrue(primaryAndSecondary instanceof CheckpointStreamWithResultProvider.PrimaryAndSecondaryStream); + } + } + + @Test + public void testCloseAndFinalizeCheckpointStreamResultPrimaryOnly() throws Exception { + CheckpointStreamFactory primaryFactory = createCheckpointStreamFactory(); + + CheckpointStreamWithResultProvider resultProvider = + CheckpointStreamWithResultProvider.createSimpleStream(CheckpointedStateScope.EXCLUSIVE, primaryFactory); + + SnapshotResult result = writeCheckpointTestData(resultProvider); + + Assert.assertNotNull(result.getJobManagerOwnedSnapshot()); + Assert.assertNull(result.getTaskLocalSnapshot()); + + try (FSDataInputStream inputStream = result.getJobManagerOwnedSnapshot().openInputStream()) { + Assert.assertEquals(0x42, inputStream.read()); + Assert.assertEquals(-1, inputStream.read()); + } + } + + @Test + public void testCloseAndFinalizeCheckpointStreamResultPrimaryAndSecondary() throws Exception { + CheckpointStreamFactory primaryFactory = createCheckpointStreamFactory(); + LocalRecoveryDirectoryProvider directoryProvider = createLocalRecoveryDirectoryProvider(); + + CheckpointStreamWithResultProvider resultProvider = + CheckpointStreamWithResultProvider.createDuplicatingStream( + 42L, + CheckpointedStateScope.EXCLUSIVE, + primaryFactory, + directoryProvider); + + SnapshotResult result = writeCheckpointTestData(resultProvider); + + Assert.assertNotNull(result.getJobManagerOwnedSnapshot()); + Assert.assertNotNull(result.getTaskLocalSnapshot()); + + try (FSDataInputStream inputStream = result.getJobManagerOwnedSnapshot().openInputStream()) { + Assert.assertEquals(0x42, inputStream.read()); + Assert.assertEquals(-1, inputStream.read()); + } + + try (FSDataInputStream inputStream = result.getTaskLocalSnapshot().openInputStream()) { + Assert.assertEquals(0x42, inputStream.read()); + Assert.assertEquals(-1, inputStream.read()); + } + } + + @Test + public void testCompletedAndCloseStateHandling() throws Exception { + CheckpointStreamFactory primaryFactory = createCheckpointStreamFactory(); + + testCloseBeforeComplete(new CheckpointStreamWithResultProvider.PrimaryStreamOnly( + primaryFactory.createCheckpointStateOutputStream(CheckpointedStateScope.EXCLUSIVE))); + testCompleteBeforeClose(new CheckpointStreamWithResultProvider.PrimaryStreamOnly( + primaryFactory.createCheckpointStateOutputStream(CheckpointedStateScope.EXCLUSIVE))); + + testCloseBeforeComplete(new CheckpointStreamWithResultProvider.PrimaryAndSecondaryStream( + primaryFactory.createCheckpointStateOutputStream(CheckpointedStateScope.EXCLUSIVE), + primaryFactory.createCheckpointStateOutputStream(CheckpointedStateScope.EXCLUSIVE))); + testCompleteBeforeClose(new CheckpointStreamWithResultProvider.PrimaryAndSecondaryStream( + primaryFactory.createCheckpointStateOutputStream(CheckpointedStateScope.EXCLUSIVE), + primaryFactory.createCheckpointStateOutputStream(CheckpointedStateScope.EXCLUSIVE))); + } + + @Test + public void testCloseMethodForwarding() throws Exception { + CheckpointStreamFactory streamFactory = createCheckpointStreamFactory(); + + MethodForwardingTestUtil.testMethodForwarding( + Closeable.class, + CheckpointStreamWithResultProvider.PrimaryStreamOnly::new, + () -> { + try { + return streamFactory.createCheckpointStateOutputStream(CheckpointedStateScope.EXCLUSIVE); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + MethodForwardingTestUtil.testMethodForwarding( + Closeable.class, + CheckpointStreamWithResultProvider.PrimaryAndSecondaryStream::new, + () -> { + try { + return new DuplicatingCheckpointOutputStream( + streamFactory.createCheckpointStateOutputStream(CheckpointedStateScope.EXCLUSIVE), + streamFactory.createCheckpointStateOutputStream(CheckpointedStateScope.EXCLUSIVE)); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + private SnapshotResult writeCheckpointTestData( + CheckpointStreamWithResultProvider resultProvider) throws IOException { + + CheckpointStreamFactory.CheckpointStateOutputStream checkpointOutputStream = + resultProvider.getCheckpointOutputStream(); + checkpointOutputStream.write(0x42); + return resultProvider.closeAndFinalizeCheckpointStreamResult(); + } + + private CheckpointStreamFactory createCheckpointStreamFactory() { + return new MemCheckpointStreamFactory(16 * 1024); + } + + /** + * Test that an exception is thrown if the stream was already closed before and we ask for a result later. + */ + private void testCloseBeforeComplete(CheckpointStreamWithResultProvider resultProvider) throws IOException { + resultProvider.getCheckpointOutputStream().write(0x42); + resultProvider.close(); + try { + resultProvider.closeAndFinalizeCheckpointStreamResult(); + Assert.fail(); + } catch (IOException ignore) { + } + } + + private void testCompleteBeforeClose(CheckpointStreamWithResultProvider resultProvider) throws IOException { + resultProvider.getCheckpointOutputStream().write(0x42); + Assert.assertNotNull(resultProvider.closeAndFinalizeCheckpointStreamResult()); + resultProvider.close(); + } + + private LocalRecoveryDirectoryProvider createLocalRecoveryDirectoryProvider() throws IOException { + File localStateDir = temporaryFolder.newFolder(); + JobID jobID = new JobID(); + JobVertexID jobVertexID = new JobVertexID(); + int subtaskIdx = 0; + return new LocalRecoveryDirectoryProviderImpl(localStateDir, jobID, jobVertexID, subtaskIdx); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/DuplicatingCheckpointOutputStreamTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/DuplicatingCheckpointOutputStreamTest.java new file mode 100644 index 0000000000000..886dbdd265785 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/DuplicatingCheckpointOutputStreamTest.java @@ -0,0 +1,310 @@ +/* + * 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.runtime.state; + +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.util.TestLogger; + +import org.apache.commons.io.IOUtils; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Random; + +public class DuplicatingCheckpointOutputStreamTest extends TestLogger { + + /** + * Test that all writes are duplicated to both streams and that the state reflects what was written. + */ + @Test + public void testDuplicatedWrite() throws Exception { + int streamCapacity = 1024 * 1024; + TestMemoryCheckpointOutputStream primaryStream = new TestMemoryCheckpointOutputStream(streamCapacity); + TestMemoryCheckpointOutputStream secondaryStream = new TestMemoryCheckpointOutputStream(streamCapacity); + TestMemoryCheckpointOutputStream referenceStream = new TestMemoryCheckpointOutputStream(streamCapacity); + DuplicatingCheckpointOutputStream duplicatingStream = + new DuplicatingCheckpointOutputStream(primaryStream, secondaryStream, 64); + Random random = new Random(42); + for (int i = 0; i < 500; ++i) { + int choice = random.nextInt(3); + if (choice == 0) { + int val = random.nextInt(); + referenceStream.write(val); + duplicatingStream.write(val); + } else { + byte[] bytes = new byte[random.nextInt(128)]; + random.nextBytes(bytes); + if (choice == 1) { + referenceStream.write(bytes); + duplicatingStream.write(bytes); + } else { + int off = bytes.length > 0 ? random.nextInt(bytes.length) : 0; + int len = bytes.length > 0 ? random.nextInt(bytes.length - off) : 0; + referenceStream.write(bytes, off, len); + duplicatingStream.write(bytes, off, len); + } + } + Assert.assertEquals(referenceStream.getPos(), duplicatingStream.getPos()); + } + + StreamStateHandle refStateHandle = referenceStream.closeAndGetHandle(); + StreamStateHandle primaryStateHandle = duplicatingStream.closeAndGetPrimaryHandle(); + StreamStateHandle secondaryStateHandle = duplicatingStream.closeAndGetSecondaryHandle(); + + Assert.assertTrue(CommonTestUtils.isSteamContentEqual( + refStateHandle.openInputStream(), + primaryStateHandle.openInputStream())); + + Assert.assertTrue(CommonTestUtils.isSteamContentEqual( + refStateHandle.openInputStream(), + secondaryStateHandle.openInputStream())); + + refStateHandle.discardState(); + primaryStateHandle.discardState(); + secondaryStateHandle.discardState(); + } + + /** + * This is the first of a set of tests that check that exceptions from the secondary stream do not impact that we + * can create a result for the first stream. + */ + @Test + public void testSecondaryWriteFail() throws Exception { + DuplicatingCheckpointOutputStream duplicatingStream = createDuplicatingStreamWithFailingSecondary(); + testFailingSecondaryStream(duplicatingStream, () -> { + for (int i = 0; i < 128; i++) { + duplicatingStream.write(42); + } + }); + } + + @Test + public void testFailingSecondaryWriteArrayFail() throws Exception { + DuplicatingCheckpointOutputStream duplicatingStream = createDuplicatingStreamWithFailingSecondary(); + testFailingSecondaryStream(duplicatingStream, () -> duplicatingStream.write(new byte[512])); + } + + @Test + public void testFailingSecondaryWriteArrayOffsFail() throws Exception { + DuplicatingCheckpointOutputStream duplicatingStream = createDuplicatingStreamWithFailingSecondary(); + testFailingSecondaryStream(duplicatingStream, () -> duplicatingStream.write(new byte[512], 20, 130)); + } + + @Test + public void testFailingSecondaryFlush() throws Exception { + DuplicatingCheckpointOutputStream duplicatingStream = createDuplicatingStreamWithFailingSecondary(); + testFailingSecondaryStream(duplicatingStream, duplicatingStream::flush); + } + + @Test + public void testFailingSecondarySync() throws Exception { + DuplicatingCheckpointOutputStream duplicatingStream = createDuplicatingStreamWithFailingSecondary(); + testFailingSecondaryStream(duplicatingStream, duplicatingStream::sync); + } + + /** + * This is the first of a set of tests that check that exceptions from the primary stream are immediately reported. + */ + @Test + public void testPrimaryWriteFail() throws Exception { + DuplicatingCheckpointOutputStream duplicatingStream = createDuplicatingStreamWithFailingPrimary(); + testFailingPrimaryStream(duplicatingStream, () -> { + for (int i = 0; i < 128; i++) { + duplicatingStream.write(42); + } + }); + } + + @Test + public void testFailingPrimaryWriteArrayFail() throws Exception { + DuplicatingCheckpointOutputStream duplicatingStream = createDuplicatingStreamWithFailingPrimary(); + testFailingPrimaryStream(duplicatingStream, () -> duplicatingStream.write(new byte[512])); + } + + @Test + public void testFailingPrimaryWriteArrayOffsFail() throws Exception { + DuplicatingCheckpointOutputStream duplicatingStream = createDuplicatingStreamWithFailingPrimary(); + testFailingPrimaryStream(duplicatingStream, () -> duplicatingStream.write(new byte[512], 20, 130)); + } + + @Test + public void testFailingPrimaryFlush() throws Exception { + DuplicatingCheckpointOutputStream duplicatingStream = createDuplicatingStreamWithFailingPrimary(); + testFailingPrimaryStream(duplicatingStream, duplicatingStream::flush); + } + + @Test + public void testFailingPrimarySync() throws Exception { + DuplicatingCheckpointOutputStream duplicatingStream = createDuplicatingStreamWithFailingPrimary(); + testFailingPrimaryStream(duplicatingStream, duplicatingStream::sync); + } + + /** + * Tests that an exception from interacting with the secondary stream does not effect duplicating to the primary + * stream, but is reflected later when we want the secondary state handle. + */ + private void testFailingSecondaryStream( + DuplicatingCheckpointOutputStream duplicatingStream, + StreamTestMethod testMethod) throws Exception { + + testMethod.call(); + + duplicatingStream.write(42); + + FailingCheckpointOutStream secondary = + (FailingCheckpointOutStream) duplicatingStream.getSecondaryOutputStream(); + + Assert.assertTrue(secondary.isClosed()); + + long pos = duplicatingStream.getPos(); + StreamStateHandle primaryHandle = duplicatingStream.closeAndGetPrimaryHandle(); + + if (primaryHandle != null) { + Assert.assertEquals(pos, primaryHandle.getStateSize()); + } + + try { + duplicatingStream.closeAndGetSecondaryHandle(); + Assert.fail(); + } catch (IOException ioEx) { + Assert.assertEquals(ioEx.getCause(), duplicatingStream.getSecondaryStreamException()); + } + } + + /** + * Test that a failing primary stream brings up an exception. + */ + private void testFailingPrimaryStream( + DuplicatingCheckpointOutputStream duplicatingStream, + StreamTestMethod testMethod) throws Exception { + try { + testMethod.call(); + Assert.fail(); + } catch (IOException ignore) { + } finally { + IOUtils.closeQuietly(duplicatingStream); + } + } + + /** + * Tests that in case of unaligned stream positions, the secondary stream is closed and the primary still works. + * This is important because some code may rely on seeking to stream offsets in the created state files and if the + * streams are not aligned this code could fail. + */ + @Test + public void testUnalignedStreamsException() throws IOException { + int streamCapacity = 1024 * 1024; + TestMemoryCheckpointOutputStream primaryStream = new TestMemoryCheckpointOutputStream(streamCapacity); + TestMemoryCheckpointOutputStream secondaryStream = new TestMemoryCheckpointOutputStream(streamCapacity); + + primaryStream.write(42); + + DuplicatingCheckpointOutputStream stream = + new DuplicatingCheckpointOutputStream(primaryStream, secondaryStream); + + Assert.assertNotNull(stream.getSecondaryStreamException()); + Assert.assertTrue(secondaryStream.isClosed()); + + stream.write(23); + + try { + stream.closeAndGetSecondaryHandle(); + Assert.fail(); + } catch (IOException ignore) { + Assert.assertEquals(ignore.getCause(), stream.getSecondaryStreamException()); + } + + StreamStateHandle primaryHandle = stream.closeAndGetPrimaryHandle(); + + try (FSDataInputStream inputStream = primaryHandle.openInputStream();) { + Assert.assertEquals(42, inputStream.read()); + Assert.assertEquals(23, inputStream.read()); + Assert.assertEquals(-1, inputStream.read()); + } + } + + /** + * Helper + */ + private DuplicatingCheckpointOutputStream createDuplicatingStreamWithFailingSecondary() throws IOException { + int streamCapacity = 1024 * 1024; + TestMemoryCheckpointOutputStream primaryStream = new TestMemoryCheckpointOutputStream(streamCapacity); + FailingCheckpointOutStream failSecondaryStream = new FailingCheckpointOutStream(); + return new DuplicatingCheckpointOutputStream(primaryStream, failSecondaryStream, 64); + } + + private DuplicatingCheckpointOutputStream createDuplicatingStreamWithFailingPrimary() throws IOException { + int streamCapacity = 1024 * 1024; + FailingCheckpointOutStream failPrimaryStream = new FailingCheckpointOutStream(); + TestMemoryCheckpointOutputStream secondary = new TestMemoryCheckpointOutputStream(streamCapacity); + return new DuplicatingCheckpointOutputStream(failPrimaryStream, secondary, 64); + } + + /** + * Stream that throws {@link IOException} on all relevant methods under test. + */ + private static class FailingCheckpointOutStream extends CheckpointStreamFactory.CheckpointStateOutputStream { + + private boolean closed = false; + + @Nullable + @Override + public StreamStateHandle closeAndGetHandle() throws IOException { + throw new IOException(); + } + + @Override + public long getPos() throws IOException { + return 0; + } + + @Override + public void write(int b) throws IOException { + throw new IOException(); + } + + @Override + public void flush() throws IOException { + throw new IOException(); + } + + @Override + public void sync() throws IOException { + throw new IOException(); + } + + @Override + public void close() throws IOException { + this.closed = true; + } + + public boolean isClosed() { + return closed; + } + } + + @FunctionalInterface + private interface StreamTestMethod { + void call() throws IOException; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/LocalRecoveryDirectoryProviderImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/LocalRecoveryDirectoryProviderImplTest.java new file mode 100644 index 0000000000000..cc97c0e861a41 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/LocalRecoveryDirectoryProviderImplTest.java @@ -0,0 +1,122 @@ +/* + * 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.runtime.state; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.util.TestLogger; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; + +/** + * Tests for {@link LocalRecoveryDirectoryProvider}. + */ +public class LocalRecoveryDirectoryProviderImplTest extends TestLogger { + + private static final JobID JOB_ID = new JobID(); + private static final JobVertexID JOB_VERTEX_ID = new JobVertexID(); + private static final int SUBTASK_INDEX = 0; + + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + private LocalRecoveryDirectoryProviderImpl directoryProvider; + private File[] allocBaseFolders; + + @Before + public void setup() throws IOException { + this.allocBaseFolders = new File[]{tmpFolder.newFolder(), tmpFolder.newFolder(), tmpFolder.newFolder()}; + this.directoryProvider = new LocalRecoveryDirectoryProviderImpl( + allocBaseFolders, + JOB_ID, + JOB_VERTEX_ID, + SUBTASK_INDEX); + } + + @Test + public void allocationBaseDir() { + for (int i = 0; i < 10; ++i) { + Assert.assertEquals(allocBaseFolders[i % allocBaseFolders.length], directoryProvider.allocationBaseDirectory(i)); + } + } + + @Test + public void selectAllocationBaseDir() { + for (int i = 0; i < allocBaseFolders.length; ++i) { + Assert.assertEquals(allocBaseFolders[i], directoryProvider.selectAllocationBaseDirectory(i)); + } + } + + @Test + public void allocationBaseDirectoriesCount() { + Assert.assertEquals(allocBaseFolders.length, directoryProvider.allocationBaseDirsCount()); + } + + @Test + public void subtaskSpecificDirectory() { + for (int i = 0; i < 10; ++i) { + Assert.assertEquals( + new File( + directoryProvider.allocationBaseDirectory(i), + directoryProvider.subtaskDirString()), + directoryProvider.subtaskBaseDirectory(i)); + } + } + + @Test + public void subtaskCheckpointSpecificDirectory() { + for (int i = 0; i < 10; ++i) { + Assert.assertEquals( + new File( + directoryProvider.subtaskBaseDirectory(i), + directoryProvider.checkpointDirString(i)), + directoryProvider.subtaskSpecificCheckpointDirectory(i)); + } + } + + @Test + public void testPathStringConstants() { + + Assert.assertEquals( + directoryProvider.subtaskDirString(), + "jid_" + JOB_ID + Path.SEPARATOR + "vtx_" + JOB_VERTEX_ID + "_sti_" + SUBTASK_INDEX); + + final long checkpointId = 42; + Assert.assertEquals( + directoryProvider.checkpointDirString(checkpointId), + "chk_" + checkpointId); + } + + @Test + public void testPreconditionsNotNullFiles() { + try { + new LocalRecoveryDirectoryProviderImpl(new File[]{null}, JOB_ID, JOB_VERTEX_ID, SUBTASK_INDEX); + Assert.fail(); + } catch (NullPointerException ignore) { + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java index 4ac64e0a4f8b5..23493b53ee311 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/MemoryStateBackendTest.java @@ -27,12 +27,12 @@ import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.runtime.checkpoint.CheckpointOptions; -import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend; import org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.util.FutureUtil; - import org.junit.Assert; import org.junit.Ignore; import org.junit.Test; @@ -43,7 +43,6 @@ import java.io.IOException; import java.io.Serializable; -import java.util.Collections; import java.util.concurrent.RunnableFuture; import static org.junit.Assert.assertEquals; @@ -51,7 +50,6 @@ import static org.mockito.Matchers.any; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; /** * Tests for the {@link org.apache.flink.runtime.state.memory.MemoryStateBackend}. @@ -94,13 +92,11 @@ public void testMapStateRestoreWithWrongSerializers() {} */ @Test public void testOperatorStateRestoreFailsIfSerializerDeserializationFails() throws Exception { + DummyEnvironment env = new DummyEnvironment(); AbstractStateBackend abstractStateBackend = new MemoryStateBackend(4096); - Environment env = mock(Environment.class); - when(env.getExecutionConfig()).thenReturn(new ExecutionConfig()); - when(env.getUserClassLoader()).thenReturn(OperatorStateBackendTest.class.getClassLoader()); - - OperatorStateBackend operatorStateBackend = abstractStateBackend.createOperatorStateBackend(env, "test-op-name"); + OperatorStateBackend operatorStateBackend = + abstractStateBackend.createOperatorStateBackend(env, "test-op-name"); // write some state ListStateDescriptor stateDescriptor1 = new ListStateDescriptor<>("test1", new JavaSerializer<>()); @@ -124,9 +120,11 @@ public void testOperatorStateRestoreFailsIfSerializerDeserializationFails() thro CheckpointStreamFactory streamFactory = new MemCheckpointStreamFactory(MemoryStateBackend.DEFAULT_MAX_STATE_SIZE); - RunnableFuture runnableFuture = + RunnableFuture> runnableFuture = operatorStateBackend.snapshot(1, 1, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); - OperatorStateHandle stateHandle = FutureUtil.runIfNotDoneAndGet(runnableFuture); + + SnapshotResult snapshotResult = FutureUtil.runIfNotDoneAndGet(runnableFuture); + OperatorStateHandle stateHandle = snapshotResult.getJobManagerOwnedSnapshot(); try { @@ -143,7 +141,7 @@ public void testOperatorStateRestoreFailsIfSerializerDeserializationFails() thro doThrow(new IOException()).when(mockProxy).read(any(DataInputViewStreamWrapper.class)); PowerMockito.whenNew(TypeSerializerSerializationUtil.TypeSerializerSerializationProxy.class).withAnyArguments().thenReturn(mockProxy); - operatorStateBackend.restore(Collections.singletonList(stateHandle)); + operatorStateBackend.restore(StateObjectCollection.singleton(stateHandle)); fail("The operator state restore should have failed if the previous state serializer could not be loaded."); } catch (IOException expected) { @@ -186,10 +184,6 @@ public void testKeyedStateRestoreFailsIfSerializerDeserializationFails() throws // ========== restore snapshot ========== - Environment env = mock(Environment.class); - when(env.getExecutionConfig()).thenReturn(new ExecutionConfig()); - when(env.getUserClassLoader()).thenReturn(OperatorStateBackendTest.class.getClassLoader()); - // mock failure when deserializing serializer TypeSerializerSerializationUtil.TypeSerializerSerializationProxy mockProxy = mock(TypeSerializerSerializationUtil.TypeSerializerSerializationProxy.class); @@ -197,7 +191,7 @@ public void testKeyedStateRestoreFailsIfSerializerDeserializationFails() throws PowerMockito.whenNew(TypeSerializerSerializationUtil.TypeSerializerSerializationProxy.class).withAnyArguments().thenReturn(mockProxy); try { - restoreKeyedBackend(IntSerializer.INSTANCE, snapshot, env); + restoreKeyedBackend(IntSerializer.INSTANCE, snapshot, new DummyEnvironment()); fail("The keyed state restore should have failed if the previous state serializer could not be loaded."); } catch (IOException expected) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/MultiStreamStateHandleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/MultiStreamStateHandleTest.java deleted file mode 100644 index dd34f030a019a..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/MultiStreamStateHandleTest.java +++ /dev/null @@ -1,135 +0,0 @@ -/* - * 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.runtime.state; - -import org.apache.flink.core.fs.FSDataInputStream; -import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; -import org.junit.Before; -import org.junit.Test; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Random; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -public class MultiStreamStateHandleTest { - - private static final int TEST_DATA_LENGTH = 123; - private Random random; - private byte[] testData; - private List streamStateHandles; - - @Before - public void setup() { - random = new Random(0x42); - testData = new byte[TEST_DATA_LENGTH]; - for (int i = 0; i < testData.length; ++i) { - testData[i] = (byte) i; - } - - int idx = 0; - streamStateHandles = new ArrayList<>(); - while (idx < testData.length) { - int len = random.nextInt(5); - byte[] sub = Arrays.copyOfRange(testData, idx, idx + len); - streamStateHandles.add(new ByteStreamStateHandle(String.valueOf(idx), sub)); - idx += len; - } - } - - @Test - public void testMetaData() throws IOException { - MultiStreamStateHandle multiStreamStateHandle = new MultiStreamStateHandle(streamStateHandles); - assertEquals(TEST_DATA_LENGTH, multiStreamStateHandle.getStateSize()); - } - - @Test - public void testLinearRead() throws IOException { - MultiStreamStateHandle multiStreamStateHandle = new MultiStreamStateHandle(streamStateHandles); - try (FSDataInputStream in = multiStreamStateHandle.openInputStream()) { - - for (int i = 0; i < TEST_DATA_LENGTH; ++i) { - assertEquals(i, in.getPos()); - assertEquals(testData[i], in.read()); - } - - assertEquals(-1, in.read()); - assertEquals(TEST_DATA_LENGTH, in.getPos()); - assertEquals(-1, in.read()); - assertEquals(TEST_DATA_LENGTH, in.getPos()); - } - } - - @Test - public void testRandomRead() throws IOException { - - MultiStreamStateHandle multiStreamStateHandle = new MultiStreamStateHandle(streamStateHandles); - - try (FSDataInputStream in = multiStreamStateHandle.openInputStream()) { - - for (int i = 0; i < 1000; ++i) { - int pos = random.nextInt(TEST_DATA_LENGTH); - int readLen = random.nextInt(TEST_DATA_LENGTH); - in.seek(pos); - while (--readLen > 0 && pos < TEST_DATA_LENGTH) { - assertEquals(pos, in.getPos()); - assertEquals(testData[pos++], in.read()); - } - } - - in.seek(TEST_DATA_LENGTH); - assertEquals(TEST_DATA_LENGTH, in.getPos()); - assertEquals(-1, in.read()); - - try { - in.seek(TEST_DATA_LENGTH + 1); - fail(); - } catch (Exception ignored) { - - } - } - } - - @Test - public void testEmptyList() throws IOException { - - MultiStreamStateHandle multiStreamStateHandle = - new MultiStreamStateHandle(Collections.emptyList()); - - try (FSDataInputStream in = multiStreamStateHandle.openInputStream()) { - - assertEquals(0, in.getPos()); - in.seek(0); - assertEquals(0, in.getPos()); - assertEquals(-1, in.read()); - - try { - in.seek(1); - fail(); - } catch (Exception ignored) { - - } - } - } -} \ No newline at end of file diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/OperatorStateBackendTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/OperatorStateBackendTest.java index 1881dad968074..d6d4af780b22f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/OperatorStateBackendTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/OperatorStateBackendTest.java @@ -34,6 +34,7 @@ import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.state.DefaultOperatorStateBackend.PartitionableListState; import org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory; @@ -41,6 +42,7 @@ import org.apache.flink.runtime.util.BlockerCheckpointStreamFactory; import org.apache.flink.util.FutureUtil; import org.apache.flink.util.Preconditions; + import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -51,7 +53,6 @@ import java.io.File; import java.io.IOException; import java.io.Serializable; -import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.Map; @@ -254,7 +255,7 @@ public void testCorrectClassLoaderUsedOnSnapshot() throws Exception { broadcastState.put(5, 6); CheckpointStreamFactory streamFactory = new MemCheckpointStreamFactory(4096); - RunnableFuture runnableFuture = + RunnableFuture> runnableFuture = operatorStateBackend.snapshot(1, 1, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); FutureUtil.runIfNotDoneAndGet(runnableFuture); @@ -375,10 +376,11 @@ public void testSnapshotEmpty() throws Exception { CheckpointStreamFactory streamFactory = new MemCheckpointStreamFactory(4096); - RunnableFuture snapshot = + RunnableFuture> snapshot = operatorStateBackend.snapshot(0L, 0L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); - OperatorStateHandle stateHandle = FutureUtil.runIfNotDoneAndGet(snapshot); + SnapshotResult snapshotResult = FutureUtil.runIfNotDoneAndGet(snapshot); + OperatorStateHandle stateHandle = snapshotResult.getJobManagerOwnedSnapshot(); assertNull(stateHandle); } @@ -404,15 +406,16 @@ public void testSnapshotBroadcastStateWithEmptyOperatorState() throws Exception OperatorStateHandle stateHandle = null; try { - RunnableFuture snapshot = + RunnableFuture> snapshot = operatorStateBackend.snapshot(0L, 0L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); - stateHandle = FutureUtil.runIfNotDoneAndGet(snapshot); + SnapshotResult snapshotResult = FutureUtil.runIfNotDoneAndGet(snapshot); + stateHandle = snapshotResult.getJobManagerOwnedSnapshot(); assertNotNull(stateHandle); final Map retrieved = new HashMap<>(); - operatorStateBackend.restore(Collections.singleton(stateHandle)); + operatorStateBackend.restore(StateObjectCollection.singleton(stateHandle)); BroadcastState retrievedState = operatorStateBackend.getBroadcastState(broadcastStateDesc); for (Map.Entry e: retrievedState.entries()) { retrieved.put(e.getKey(), e.getValue()); @@ -424,10 +427,13 @@ public void testSnapshotBroadcastStateWithEmptyOperatorState() throws Exception expected.remove(1); snapshot = operatorStateBackend.snapshot(1L, 1L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); - stateHandle = FutureUtil.runIfNotDoneAndGet(snapshot); + snapshotResult = FutureUtil.runIfNotDoneAndGet(snapshot); + + stateHandle.discardState(); + stateHandle = snapshotResult.getJobManagerOwnedSnapshot(); retrieved.clear(); - operatorStateBackend.restore(Collections.singleton(stateHandle)); + operatorStateBackend.restore(StateObjectCollection.singleton(stateHandle)); retrievedState = operatorStateBackend.getBroadcastState(broadcastStateDesc); for (Map.Entry e: retrievedState.immutableEntries()) { retrieved.put(e.getKey(), e.getValue()); @@ -439,16 +445,24 @@ public void testSnapshotBroadcastStateWithEmptyOperatorState() throws Exception expected.clear(); snapshot = operatorStateBackend.snapshot(2L, 2L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); - stateHandle = FutureUtil.runIfNotDoneAndGet(snapshot); + snapshotResult = FutureUtil.runIfNotDoneAndGet(snapshot); + if (stateHandle != null) { + stateHandle.discardState(); + } + stateHandle = snapshotResult.getJobManagerOwnedSnapshot(); retrieved.clear(); - operatorStateBackend.restore(Collections.singleton(stateHandle)); + operatorStateBackend.restore(StateObjectCollection.singleton(stateHandle)); retrievedState = operatorStateBackend.getBroadcastState(broadcastStateDesc); for (Map.Entry e: retrievedState.immutableEntries()) { retrieved.put(e.getKey(), e.getValue()); } assertTrue(expected.isEmpty()); assertEquals(expected, retrieved); + if (stateHandle != null) { + stateHandle.discardState(); + stateHandle = null; + } } finally { operatorStateBackend.close(); operatorStateBackend.dispose(); @@ -497,9 +511,11 @@ public void testSnapshotRestoreSync() throws Exception { broadcastState2.put(2, 5); CheckpointStreamFactory streamFactory = new MemCheckpointStreamFactory(2 * 4096); - RunnableFuture runnableFuture = - operatorStateBackend.snapshot(1, 1, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); - OperatorStateHandle stateHandle = FutureUtil.runIfNotDoneAndGet(runnableFuture); + RunnableFuture> snapshot = + operatorStateBackend.snapshot(1L, 1L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); + + SnapshotResult snapshotResult = FutureUtil.runIfNotDoneAndGet(snapshot); + OperatorStateHandle stateHandle = snapshotResult.getJobManagerOwnedSnapshot(); try { @@ -510,7 +526,7 @@ public void testSnapshotRestoreSync() throws Exception { createMockEnvironment(), "testOperator"); - operatorStateBackend.restore(Collections.singletonList(stateHandle)); + operatorStateBackend.restore(StateObjectCollection.singleton(stateHandle)); assertEquals(3, operatorStateBackend.getRegisteredStateNames().size()); assertEquals(3, operatorStateBackend.getRegisteredBroadcastStateNames().size()); @@ -624,7 +640,7 @@ public void testSnapshotRestoreAsync() throws Exception { streamFactory.setWaiterLatch(waiterLatch); streamFactory.setBlockerLatch(blockerLatch); - RunnableFuture runnableFuture = + RunnableFuture> runnableFuture = operatorStateBackend.snapshot(1, 1, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); ExecutorService executorService = Executors.newFixedThreadPool(1); @@ -657,7 +673,8 @@ public void testSnapshotRestoreAsync() throws Exception { new ListStateDescriptor<>("test4", new JavaSerializer())); // run the snapshot - OperatorStateHandle stateHandle = runnableFuture.get(); + SnapshotResult snapshotResult = runnableFuture.get(); + OperatorStateHandle stateHandle = snapshotResult.getJobManagerOwnedSnapshot(); try { @@ -670,7 +687,7 @@ public void testSnapshotRestoreAsync() throws Exception { createMockEnvironment(), "testOperator"); - operatorStateBackend.restore(Collections.singletonList(stateHandle)); + operatorStateBackend.restore(StateObjectCollection.singleton(stateHandle)); assertEquals(3, operatorStateBackend.getRegisteredStateNames().size()); assertEquals(3, operatorStateBackend.getRegisteredBroadcastStateNames().size()); @@ -762,7 +779,7 @@ public void testSnapshotAsyncClose() throws Exception { streamFactory.setWaiterLatch(waiterLatch); streamFactory.setBlockerLatch(blockerLatch); - RunnableFuture runnableFuture = + RunnableFuture> runnableFuture = operatorStateBackend.snapshot(1, 1, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); ExecutorService executorService = Executors.newFixedThreadPool(1); @@ -805,7 +822,7 @@ public void testSnapshotAsyncCancel() throws Exception { streamFactory.setWaiterLatch(waiterLatch); streamFactory.setBlockerLatch(blockerLatch); - RunnableFuture runnableFuture = + RunnableFuture> runnableFuture = operatorStateBackend.snapshot(1, 1, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); ExecutorService executorService = Executors.newFixedThreadPool(1); @@ -856,9 +873,11 @@ public void testRestoreFailsIfSerializerDeserializationFails() throws Exception listState3.add(20); CheckpointStreamFactory streamFactory = new MemCheckpointStreamFactory(4096); - RunnableFuture runnableFuture = + RunnableFuture> runnableFuture = operatorStateBackend.snapshot(1, 1, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); - OperatorStateHandle stateHandle = FutureUtil.runIfNotDoneAndGet(runnableFuture); + + SnapshotResult snapshotResult = FutureUtil.runIfNotDoneAndGet(runnableFuture); + OperatorStateHandle stateHandle = snapshotResult.getJobManagerOwnedSnapshot(); try { @@ -875,7 +894,7 @@ public void testRestoreFailsIfSerializerDeserializationFails() throws Exception doThrow(new IOException()).when(mockProxy).read(any(DataInputViewStreamWrapper.class)); PowerMockito.whenNew(TypeSerializerSerializationUtil.TypeSerializerSerializationProxy.class).withAnyArguments().thenReturn(mockProxy); - operatorStateBackend.restore(Collections.singletonList(stateHandle)); + operatorStateBackend.restore(StateObjectCollection.singleton(stateHandle)); fail("The operator state restore should have failed if the previous state serializer could not be loaded."); } catch (IOException expected) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/OperatorStateHandleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/OperatorStreamStateHandleTest.java similarity index 97% rename from flink-runtime/src/test/java/org/apache/flink/runtime/state/OperatorStateHandleTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/state/OperatorStreamStateHandleTest.java index 88f9cd7eb3ef7..57c6c643906ee 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/OperatorStateHandleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/OperatorStreamStateHandleTest.java @@ -21,7 +21,7 @@ import org.junit.Assert; import org.junit.Test; -public class OperatorStateHandleTest { +public class OperatorStreamStateHandleTest { @Test public void testFixedEnumOrder() { @@ -37,4 +37,4 @@ public void testFixedEnumOrder() { // Byte is used to encode enum value on serialization Assert.assertTrue(OperatorStateHandle.Mode.values().length <= Byte.MAX_VALUE); } -} \ No newline at end of file +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/SnapshotDirectoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/SnapshotDirectoryTest.java new file mode 100644 index 0000000000000..9b090d6a4eec6 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/SnapshotDirectoryTest.java @@ -0,0 +1,206 @@ +/* + * 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.runtime.state; + +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.util.TestLogger; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.UUID; + +public class SnapshotDirectoryTest extends TestLogger { + + private static TemporaryFolder temporaryFolder; + + @BeforeClass + public static void beforeClass() throws IOException { + temporaryFolder = new TemporaryFolder(); + temporaryFolder.create(); + } + + @AfterClass + public static void afterClass() { + temporaryFolder.delete(); + } + + /** + * Tests if mkdirs for snapshot directories works. + */ + @Test + public void mkdirs() throws Exception { + File folderRoot = temporaryFolder.getRoot(); + File newFolder = new File(folderRoot, String.valueOf(UUID.randomUUID())); + File innerNewFolder = new File(newFolder, String.valueOf(UUID.randomUUID())); + Path path = new Path(innerNewFolder.toURI()); + + Assert.assertFalse(newFolder.isDirectory()); + Assert.assertFalse(innerNewFolder.isDirectory()); + SnapshotDirectory snapshotDirectory = SnapshotDirectory.permanent(path); + Assert.assertFalse(snapshotDirectory.exists()); + Assert.assertFalse(newFolder.isDirectory()); + Assert.assertFalse(innerNewFolder.isDirectory()); + + Assert.assertTrue(snapshotDirectory.mkdirs()); + Assert.assertTrue(newFolder.isDirectory()); + Assert.assertTrue(innerNewFolder.isDirectory()); + Assert.assertTrue(snapshotDirectory.exists()); + } + + /** + * Tests if indication of directory existence works. + */ + @Test + public void exists() throws Exception { + File folderRoot = temporaryFolder.getRoot(); + File folderA = new File(folderRoot, String.valueOf(UUID.randomUUID())); + + Assert.assertFalse(folderA.isDirectory()); + Path path = new Path(folderA.toURI()); + SnapshotDirectory snapshotDirectory = SnapshotDirectory.permanent(path); + Assert.assertFalse(snapshotDirectory.exists()); + Assert.assertTrue(folderA.mkdirs()); + Assert.assertTrue(snapshotDirectory.exists()); + Assert.assertTrue(folderA.delete()); + Assert.assertFalse(snapshotDirectory.exists()); + } + + /** + * Tests listing of file statuses works like listing on the path directly. + */ + @Test + public void listStatus() throws Exception { + File folderRoot = temporaryFolder.getRoot(); + File folderA = new File(folderRoot, String.valueOf(UUID.randomUUID())); + File folderB = new File(folderA, String.valueOf(UUID.randomUUID())); + Assert.assertTrue(folderB.mkdirs()); + File file = new File(folderA, "test.txt"); + Assert.assertTrue(file.createNewFile()); + + Path path = new Path(folderA.toURI()); + FileSystem fileSystem = path.getFileSystem(); + SnapshotDirectory snapshotDirectory = SnapshotDirectory.permanent(path); + Assert.assertTrue(snapshotDirectory.exists()); + + Assert.assertEquals( + Arrays.toString(fileSystem.listStatus(path)), + Arrays.toString(snapshotDirectory.listStatus())); + } + + /** + * Tests that reporting the handle of a completed snapshot works as expected and that the directory for completed + * snapshot is not deleted by {@link #deleteIfNotCompeltedSnapshot()}. + */ + @Test + public void completeSnapshotAndGetHandle() throws Exception { + File folderRoot = temporaryFolder.getRoot(); + File folderA = new File(folderRoot, String.valueOf(UUID.randomUUID())); + Assert.assertTrue(folderA.mkdirs()); + Path folderAPath = new Path(folderA.toURI()); + + SnapshotDirectory snapshotDirectory = SnapshotDirectory.permanent(folderAPath); + + // check that completed checkpoint dirs are not deleted as incomplete. + DirectoryStateHandle handle = snapshotDirectory.completeSnapshotAndGetHandle(); + Assert.assertNotNull(handle); + Assert.assertTrue(snapshotDirectory.cleanup()); + Assert.assertTrue(folderA.isDirectory()); + Assert.assertEquals(folderAPath, handle.getDirectory()); + handle.discardState(); + + Assert.assertFalse(folderA.isDirectory()); + Assert.assertTrue(folderA.mkdirs()); + snapshotDirectory = SnapshotDirectory.permanent(folderAPath); + Assert.assertTrue(snapshotDirectory.cleanup()); + try { + snapshotDirectory.completeSnapshotAndGetHandle(); + Assert.fail(); + } catch (IOException ignore) { + } + } + + /** + * Tests that snapshot director behaves correct for delete calls. Completed snapshots should not be deleted, + * only ongoing snapshots can. + */ + @Test + public void deleteIfNotCompeltedSnapshot() throws Exception { + File folderRoot = temporaryFolder.getRoot(); + File folderA = new File(folderRoot, String.valueOf(UUID.randomUUID())); + File folderB = new File(folderA, String.valueOf(UUID.randomUUID())); + Assert.assertTrue(folderB.mkdirs()); + File file = new File(folderA, "test.txt"); + Assert.assertTrue(file.createNewFile()); + Path folderAPath = new Path(folderA.toURI()); + SnapshotDirectory snapshotDirectory = SnapshotDirectory.permanent(folderAPath); + Assert.assertTrue(snapshotDirectory.cleanup()); + Assert.assertFalse(folderA.isDirectory()); + Assert.assertTrue(folderA.mkdirs()); + Assert.assertTrue(file.createNewFile()); + snapshotDirectory = SnapshotDirectory.permanent(folderAPath); + snapshotDirectory.completeSnapshotAndGetHandle(); + Assert.assertTrue(snapshotDirectory.cleanup()); + Assert.assertTrue(folderA.isDirectory()); + Assert.assertTrue(file.exists()); + } + + /** + * This test checks that completing or deleting the snapshot influence the #isSnapshotOngoing() flag. + */ + @Test + public void isSnapshotOngoing() throws Exception { + File folderRoot = temporaryFolder.getRoot(); + File folderA = new File(folderRoot, String.valueOf(UUID.randomUUID())); + Assert.assertTrue(folderA.mkdirs()); + Path pathA = new Path(folderA.toURI()); + SnapshotDirectory snapshotDirectory = SnapshotDirectory.permanent(pathA); + Assert.assertFalse(snapshotDirectory.isSnapshotCompleted()); + Assert.assertNotNull(snapshotDirectory.completeSnapshotAndGetHandle()); + Assert.assertTrue(snapshotDirectory.isSnapshotCompleted()); + snapshotDirectory = SnapshotDirectory.permanent(pathA); + Assert.assertFalse(snapshotDirectory.isSnapshotCompleted()); + snapshotDirectory.cleanup(); + Assert.assertFalse(snapshotDirectory.isSnapshotCompleted()); + } + + /** + * Tests that temporary directories have the right behavior on completion and deletion. + */ + @Test + public void temporary() throws Exception { + File folderRoot = temporaryFolder.getRoot(); + File folder = new File(folderRoot, String.valueOf(UUID.randomUUID())); + Assert.assertTrue(folder.mkdirs()); + Path folderPath = new Path(folder.toURI()); + SnapshotDirectory tmpSnapshotDirectory = SnapshotDirectory.temporary(folderPath); + // temporary snapshot directories should not return a handle, because they will be deleted. + Assert.assertNull(tmpSnapshotDirectory.completeSnapshotAndGetHandle()); + // check that the directory is deleted even after we called #completeSnapshotAndGetHandle. + Assert.assertTrue(tmpSnapshotDirectory.cleanup()); + Assert.assertFalse(folder.exists()); + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/SnapshotResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/SnapshotResultTest.java new file mode 100644 index 0000000000000..63a93408d819e --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/SnapshotResultTest.java @@ -0,0 +1,68 @@ +/* + * 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.runtime.state; + +import org.apache.flink.util.TestLogger; +import org.junit.Assert; +import org.junit.Test; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +public class SnapshotResultTest extends TestLogger { + + @Test + public void discardState() throws Exception { + SnapshotResult result = SnapshotResult.withLocalState(mock(StateObject.class), mock(StateObject.class)); + result.discardState(); + verify(result.getJobManagerOwnedSnapshot()).discardState(); + verify(result.getTaskLocalSnapshot()).discardState(); + } + + @Test + public void getStateSize() { + long size = 42L; + + SnapshotResult result = SnapshotResult.withLocalState( + new DummyStateObject(size), + new DummyStateObject(size)); + Assert.assertEquals(size, result.getStateSize()); + } + + static class DummyStateObject implements StateObject { + + private static final long serialVersionUID = 1L; + + private final long size; + + DummyStateObject(long size) { + this.size = size; + } + + @Override + public void discardState() { + } + + @Override + public long getStateSize() { + return size; + } + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java index 783845027017b..11ae389da7380 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java @@ -18,6 +18,10 @@ package org.apache.flink.runtime.state; +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import org.apache.commons.io.output.ByteArrayOutputStream; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.functions.AggregateFunction; import org.apache.flink.api.common.functions.FoldFunction; @@ -53,6 +57,7 @@ import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.checkpoint.StateAssignmentOperation; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.operators.testutils.DummyEnvironment; @@ -69,15 +74,9 @@ import org.apache.flink.runtime.util.BlockerCheckpointStreamFactory; import org.apache.flink.shaded.guava18.com.google.common.base.Joiner; import org.apache.flink.types.IntValue; -import org.apache.flink.util.FutureUtil; import org.apache.flink.util.IOUtils; import org.apache.flink.util.StateMigrationException; import org.apache.flink.util.TestLogger; - -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.io.Output; -import org.apache.commons.io.output.ByteArrayOutputStream; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -141,7 +140,7 @@ protected CheckpointStreamFactory createStreamFactory() throws Exception { } protected AbstractKeyedStateBackend createKeyedBackend(TypeSerializer keySerializer) throws Exception { - return createKeyedBackend(keySerializer, new DummyEnvironment("test", 1, 0)); + return createKeyedBackend(keySerializer, new DummyEnvironment()); } protected AbstractKeyedStateBackend createKeyedBackend(TypeSerializer keySerializer, Environment env) throws Exception { @@ -173,7 +172,7 @@ protected AbstractKeyedStateBackend createKeyedBackend( } protected AbstractKeyedStateBackend restoreKeyedBackend(TypeSerializer keySerializer, KeyedStateHandle state) throws Exception { - return restoreKeyedBackend(keySerializer, state, new DummyEnvironment("test", 1, 0)); + return restoreKeyedBackend(keySerializer, state, new DummyEnvironment()); } protected AbstractKeyedStateBackend restoreKeyedBackend( @@ -204,7 +203,7 @@ protected AbstractKeyedStateBackend restoreKeyedBackend( keyGroupRange, env.getTaskKvStateRegistry()); - backend.restore(state); + backend.restore(new StateObjectCollection<>(state)); return backend; } @@ -274,7 +273,7 @@ public void testGetKeys() throws Exception { @SuppressWarnings("unchecked") public void testBackendUsesRegisteredKryoDefaultSerializer() throws Exception { CheckpointStreamFactory streamFactory = createStreamFactory(); - Environment env = new DummyEnvironment("test", 1, 0); + Environment env = new DummyEnvironment(); AbstractKeyedStateBackend backend = createKeyedBackend(IntSerializer.INSTANCE, env); // cast because our test serializer is not typed to TestPojo @@ -329,7 +328,7 @@ public void testBackendUsesRegisteredKryoDefaultSerializer() throws Exception { @SuppressWarnings("unchecked") public void testBackendUsesRegisteredKryoDefaultSerializerUsingGetOrCreate() throws Exception { CheckpointStreamFactory streamFactory = createStreamFactory(); - Environment env = new DummyEnvironment("test", 1, 0); + Environment env = new DummyEnvironment(); AbstractKeyedStateBackend backend = createKeyedBackend(IntSerializer.INSTANCE, env); // cast because our test serializer is not typed to TestPojo @@ -388,7 +387,7 @@ public void testBackendUsesRegisteredKryoDefaultSerializerUsingGetOrCreate() thr @Test public void testBackendUsesRegisteredKryoSerializer() throws Exception { CheckpointStreamFactory streamFactory = createStreamFactory(); - Environment env = new DummyEnvironment("test", 1, 0); + Environment env = new DummyEnvironment(); AbstractKeyedStateBackend backend = createKeyedBackend(IntSerializer.INSTANCE, env); env.getExecutionConfig() @@ -443,7 +442,7 @@ public void testBackendUsesRegisteredKryoSerializer() throws Exception { @SuppressWarnings("unchecked") public void testBackendUsesRegisteredKryoSerializerUsingGetOrCreate() throws Exception { CheckpointStreamFactory streamFactory = createStreamFactory(); - Environment env = new DummyEnvironment("test", 1, 0); + Environment env = new DummyEnvironment(); AbstractKeyedStateBackend backend = createKeyedBackend(IntSerializer.INSTANCE, env); env.getExecutionConfig().registerTypeWithKryoSerializer(TestPojo.class, ExceptionThrowingTestSerializer.class); @@ -507,7 +506,7 @@ public void testBackendUsesRegisteredKryoSerializerUsingGetOrCreate() throws Exc @Test public void testKryoRegisteringRestoreResilienceWithRegisteredType() throws Exception { CheckpointStreamFactory streamFactory = createStreamFactory(); - Environment env = new DummyEnvironment("test", 1, 0); + Environment env = new DummyEnvironment(); AbstractKeyedStateBackend backend = createKeyedBackend(IntSerializer.INSTANCE, env); TypeInformation pojoType = new GenericTypeInfo<>(TestPojo.class); @@ -569,7 +568,7 @@ public void testKryoRegisteringRestoreResilienceWithRegisteredType() throws Exce public void testKryoRegisteringRestoreResilienceWithDefaultSerializer() throws Exception { CheckpointStreamFactory streamFactory = createStreamFactory(); SharedStateRegistry sharedStateRegistry = new SharedStateRegistry(); - Environment env = new DummyEnvironment("test", 1, 0); + Environment env = new DummyEnvironment(); AbstractKeyedStateBackend backend = null; try { @@ -671,7 +670,7 @@ public void testKryoRegisteringRestoreResilienceWithDefaultSerializer() throws E public void testKryoRegisteringRestoreResilienceWithRegisteredSerializer() throws Exception { CheckpointStreamFactory streamFactory = createStreamFactory(); SharedStateRegistry sharedStateRegistry = new SharedStateRegistry(); - Environment env = new DummyEnvironment("test", 1, 0); + Environment env = new DummyEnvironment(); AbstractKeyedStateBackend backend = null; @@ -761,7 +760,7 @@ public void testKryoRegisteringRestoreResilienceWithRegisteredSerializer() throw @Test public void testKryoRestoreResilienceWithDifferentRegistrationOrder() throws Exception { CheckpointStreamFactory streamFactory = createStreamFactory(); - Environment env = new DummyEnvironment("test", 1, 0); + Environment env = new DummyEnvironment(); // register A first then B env.getExecutionConfig().registerKryoType(TestNestedPojoClassA.class); @@ -796,7 +795,7 @@ public void testKryoRestoreResilienceWithDifferentRegistrationOrder() throws Exc // ========== restore snapshot, with a different registration order in the configuration ========== - env = new DummyEnvironment("test", 1, 0); + env = new DummyEnvironment(); env.getExecutionConfig().registerKryoType(TestNestedPojoClassB.class); // this time register B first env.getExecutionConfig().registerKryoType(TestNestedPojoClassA.class); @@ -828,7 +827,7 @@ public void testKryoRestoreResilienceWithDifferentRegistrationOrder() throws Exc @Test public void testPojoRestoreResilienceWithDifferentRegistrationOrder() throws Exception { CheckpointStreamFactory streamFactory = createStreamFactory(); - Environment env = new DummyEnvironment("test", 1, 0); + Environment env = new DummyEnvironment(); // register A first then B env.getExecutionConfig().registerPojoType(TestNestedPojoClassA.class); @@ -863,7 +862,7 @@ public void testPojoRestoreResilienceWithDifferentRegistrationOrder() throws Exc // ========== restore snapshot, with a different registration order in the configuration ========== - env = new DummyEnvironment("test", 1, 0); + env = new DummyEnvironment(); env.getExecutionConfig().registerPojoType(TestNestedPojoClassB.class); // this time register B first env.getExecutionConfig().registerPojoType(TestNestedPojoClassA.class); @@ -924,7 +923,7 @@ public void testValueState() throws Exception { assertEquals("1", getSerializedValue(kvState, 1, keySerializer, VoidNamespace.INSTANCE, namespaceSerializer, valueSerializer)); // draw a snapshot - KeyedStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); + KeyedStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); // make some more modifications backend.setCurrentKey(1); @@ -935,7 +934,7 @@ public void testValueState() throws Exception { state.update("u3"); // draw another snapshot - KeyedStateHandle snapshot2 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); + KeyedStateHandle snapshot2 = runSnapshot(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); // validate the original state backend.setCurrentKey(1); @@ -1110,7 +1109,7 @@ public void testMultipleValueStates() throws Exception { IntSerializer.INSTANCE, 1, new KeyGroupRange(0, 0), - new DummyEnvironment("test_op", 1, 0)); + new DummyEnvironment()); ValueStateDescriptor desc1 = new ValueStateDescriptor<>("a-string", StringSerializer.INSTANCE); ValueStateDescriptor desc2 = new ValueStateDescriptor<>("an-integer", IntSerializer.INSTANCE); @@ -1134,7 +1133,8 @@ public void testMultipleValueStates() throws Exception { assertEquals(13, (int) state2.value()); // draw a snapshot - KeyedStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); + KeyedStateHandle snapshot1 = + runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); backend.dispose(); backend = restoreKeyedBackend( @@ -1142,7 +1142,7 @@ public void testMultipleValueStates() throws Exception { 1, new KeyGroupRange(0, 0), Collections.singletonList(snapshot1), - new DummyEnvironment("test_op", 1, 0)); + new DummyEnvironment()); snapshot1.discardState(); @@ -1206,7 +1206,7 @@ public void testValueStateNullUpdate() throws Exception { assertEquals(42L, (long) state.value()); // draw a snapshot - KeyedStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); + KeyedStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); backend.dispose(); backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot1); @@ -1934,7 +1934,7 @@ public void testAggregatingStateAddAndGetWithMutableAccumulator() throws Excepti final AggregatingStateDescriptor stateDescr = new AggregatingStateDescriptor<>("my-state", new MutableAggregatingAddingFunction(), MutableLong.class); - + AbstractKeyedStateBackend keyedBackend = createKeyedBackend(StringSerializer.INSTANCE); try { @@ -2700,7 +2700,7 @@ public void testKeyGroupSnapshotRestore() throws Exception { IntSerializer.INSTANCE, MAX_PARALLELISM, new KeyGroupRange(0, MAX_PARALLELISM - 1), - new DummyEnvironment("test", 1, 0)); + new DummyEnvironment()); ValueStateDescriptor kvId = new ValueStateDescriptor<>("id", String.class); @@ -2727,7 +2727,7 @@ public void testKeyGroupSnapshotRestore() throws Exception { state.update("ShouldBeInSecondHalf"); - KeyedStateHandle snapshot = FutureUtil.runIfNotDoneAndGet(backend.snapshot(0, 0, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); + KeyedStateHandle snapshot = runSnapshot(backend.snapshot(0, 0, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); List firstHalfKeyGroupStates = StateAssignmentOperation.getKeyedStateHandles( Collections.singletonList(snapshot), @@ -2745,7 +2745,7 @@ public void testKeyGroupSnapshotRestore() throws Exception { MAX_PARALLELISM, new KeyGroupRange(0, 4), firstHalfKeyGroupStates, - new DummyEnvironment("test", 1, 0)); + new DummyEnvironment()); // backend for the second half of the key group range final AbstractKeyedStateBackend secondHalfBackend = restoreKeyedBackend( @@ -2753,7 +2753,7 @@ public void testKeyGroupSnapshotRestore() throws Exception { MAX_PARALLELISM, new KeyGroupRange(5, 9), secondHalfKeyGroupStates, - new DummyEnvironment("test", 1, 0)); + new DummyEnvironment()); ValueState firstHalfState = firstHalfBackend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId); @@ -2794,7 +2794,7 @@ public void testRestoreWithWrongKeySerializer() throws Exception { state.update("2"); // draw a snapshot - KeyedStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); + KeyedStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); backend.dispose(); @@ -2825,7 +2825,7 @@ public void testValueStateRestoreWithWrongSerializers() throws Exception { state.update("2"); // draw a snapshot - KeyedStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); + KeyedStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); backend.dispose(); // restore the first snapshot and validate it @@ -2868,7 +2868,7 @@ public void testListStateRestoreWithWrongSerializers() throws Exception { state.add("2"); // draw a snapshot - KeyedStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); + KeyedStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); backend.dispose(); // restore the first snapshot and validate it @@ -2913,7 +2913,7 @@ public void testReducingStateRestoreWithWrongSerializers() throws Exception { state.add("2"); // draw a snapshot - KeyedStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); + KeyedStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); backend.dispose(); // restore the first snapshot and validate it @@ -2956,7 +2956,7 @@ public void testMapStateRestoreWithWrongSerializers() throws Exception { state.put("2", "Second"); // draw a snapshot - KeyedStateHandle snapshot1 = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); + KeyedStateHandle snapshot1 = runSnapshot(backend.snapshot(682375462378L, 2, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); backend.dispose(); // restore the first snapshot and validate it @@ -3050,7 +3050,7 @@ protected void testConcurrentMapIfQueryable() throws Exception { IntSerializer.INSTANCE, numberOfKeyGroups, new KeyGroupRange(0, 0), - new DummyEnvironment("test_op", 1, 0)); + new DummyEnvironment()); { // ValueState @@ -3193,7 +3193,7 @@ private void checkConcurrentStateTable(StateTable stateTable, int numbe */ @Test public void testQueryableStateRegistration() throws Exception { - DummyEnvironment env = new DummyEnvironment("test", 1, 0); + DummyEnvironment env = new DummyEnvironment(); KvStateRegistry registry = env.getKvStateRegistry(); CheckpointStreamFactory streamFactory = createStreamFactory(); @@ -3215,7 +3215,7 @@ public void testQueryableStateRegistration() throws Exception { eq(env.getJobID()), eq(env.getJobVertexId()), eq(expectedKeyGroupRange), eq("banana"), any(KvStateID.class)); - KeyedStateHandle snapshot = FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); + KeyedStateHandle snapshot = runSnapshot(backend.snapshot(682375462379L, 4, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); backend.dispose(); @@ -3247,7 +3247,7 @@ public void testEmptyStateCheckpointing() { // draw a snapshot KeyedStateHandle snapshot = - FutureUtil.runIfNotDoneAndGet(backend.snapshot(682375462379L, 1, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); + runSnapshot(backend.snapshot(682375462379L, 1, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation())); assertNull(snapshot); backend.dispose(); @@ -3335,7 +3335,7 @@ public void testParallelAsyncSnapshots() throws Exception { valueState.update(i); } - RunnableFuture snapshot1 = + RunnableFuture> snapshot1 = backend.snapshot(0L, 0L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); Thread runner1 = new Thread(snapshot1, "snapshot-1-runner"); @@ -3353,7 +3353,7 @@ public void testParallelAsyncSnapshots() throws Exception { streamFactory.setWaiterLatch(null); streamFactory.setBlockerLatch(null); - RunnableFuture snapshot2 = + RunnableFuture> snapshot2 = backend.snapshot(1L, 1L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); Thread runner2 = new Thread(snapshot2,"snapshot-2-runner"); @@ -3392,7 +3392,7 @@ public void testAsyncSnapshot() throws Exception { valueState.update(i); } - RunnableFuture snapshot = + RunnableFuture> snapshot = backend.snapshot(0L, 0L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); Thread runner = new Thread(snapshot); runner.start(); @@ -3405,7 +3405,8 @@ public void testAsyncSnapshot() throws Exception { } runner.join(); - stateHandle = snapshot.get(); + SnapshotResult snapshotResult = snapshot.get(); + stateHandle = snapshotResult.getJobManagerOwnedSnapshot(); // test isolation for (int i = 0; i < 20; ++i) { @@ -3476,7 +3477,7 @@ public void testAsyncSnapshotCancellation() throws Exception { valueState.update(i); } - RunnableFuture snapshot = + RunnableFuture> snapshot = backend.snapshot(0L, 0L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); Thread runner = new Thread(snapshot); @@ -3588,12 +3589,15 @@ private static Map getSerializedMap( } } - protected KeyedStateHandle runSnapshot(RunnableFuture snapshotRunnableFuture) throws Exception { - if(!snapshotRunnableFuture.isDone()) { - Thread runner = new Thread(snapshotRunnableFuture); - runner.start(); + protected KeyedStateHandle runSnapshot( + RunnableFuture> snapshotRunnableFuture) throws Exception { + + if (!snapshotRunnableFuture.isDone()) { + snapshotRunnableFuture.run(); } - return snapshotRunnableFuture.get(); + + SnapshotResult snapshotResult = snapshotRunnableFuture.get(); + return snapshotResult.getJobManagerOwnedSnapshot(); } public static class TestPojo implements Serializable { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateSnapshotCompressionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateSnapshotCompressionTest.java index a7a4b9a10a731..7d903cc204fdb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateSnapshotCompressionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateSnapshotCompressionTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend; import org.apache.flink.runtime.state.internal.InternalValueState; @@ -32,7 +33,6 @@ import org.junit.Assert; import org.junit.Test; -import java.util.Collections; import java.util.concurrent.RunnableFuture; import static org.mockito.Mockito.mock; @@ -40,7 +40,7 @@ public class StateSnapshotCompressionTest extends TestLogger { @Test - public void testCompressionConfiguration() throws Exception { + public void testCompressionConfiguration() { ExecutionConfig executionConfig = new ExecutionConfig(); executionConfig.setUseSnapshotCompression(true); @@ -52,7 +52,8 @@ public void testCompressionConfiguration() throws Exception { 16, new KeyGroupRange(0, 15), true, - executionConfig); + executionConfig, + TestLocalRecoveryConfig.disabled()); try { Assert.assertTrue( @@ -73,7 +74,8 @@ public void testCompressionConfiguration() throws Exception { 16, new KeyGroupRange(0, 15), true, - executionConfig); + executionConfig, + TestLocalRecoveryConfig.disabled()); try { Assert.assertTrue( @@ -112,7 +114,8 @@ private void snapshotRestoreRoundtrip(boolean useCompression) throws Exception { 16, new KeyGroupRange(0, 15), true, - executionConfig); + executionConfig, + TestLocalRecoveryConfig.disabled()); try { @@ -134,10 +137,11 @@ private void snapshotRestoreRoundtrip(boolean useCompression) throws Exception { state.setCurrentNamespace(VoidNamespace.INSTANCE); state.update("45"); CheckpointStreamFactory streamFactory = new MemCheckpointStreamFactory(4 * 1024 * 1024); - RunnableFuture snapshot = + RunnableFuture> snapshot = stateBackend.snapshot(0L, 0L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); snapshot.run(); - stateHandle = snapshot.get(); + SnapshotResult snapshotResult = snapshot.get(); + stateHandle = snapshotResult.getJobManagerOwnedSnapshot(); } finally { IOUtils.closeQuietly(stateBackend); @@ -153,10 +157,11 @@ private void snapshotRestoreRoundtrip(boolean useCompression) throws Exception { 16, new KeyGroupRange(0, 15), true, - executionConfig); + executionConfig, + TestLocalRecoveryConfig.disabled()); try { - stateBackend.restore(Collections.singletonList(stateHandle)); + stateBackend.restore(StateObjectCollection.singleton(stateHandle)); InternalValueState state = stateBackend.createValueState( new VoidNamespaceSerializer(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java new file mode 100644 index 0000000000000..2e9b107d557e9 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java @@ -0,0 +1,224 @@ +/* + * 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.runtime.state; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.taskexecutor.TaskManagerServices; +import org.apache.flink.runtime.taskexecutor.TaskManagerServicesConfiguration; +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.TestLogger; + +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.net.InetAddress; + +public class TaskExecutorLocalStateStoresManagerTest extends TestLogger { + + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private static final long MEM_SIZE_PARAM = 128L*1024*1024; + + /** + * This tests that the creation of {@link TaskManagerServices} correctly creates the local state root directory + * for the {@link TaskExecutorLocalStateStoresManager} with the configured root directory. + */ + @Test + public void testCreationFromConfig() throws Exception { + + final Configuration config = new Configuration(); + + File newFolder = temporaryFolder.newFolder(); + String tmpDir = newFolder.getAbsolutePath() + File.separator; + final String rootDirString = "__localStateRoot1,__localStateRoot2,__localStateRoot3".replaceAll("__", tmpDir); + + // test configuration of the local state directories + config.setString(CheckpointingOptions.LOCAL_RECOVERY_TASK_MANAGER_STATE_ROOT_DIRS, rootDirString); + + // test configuration of the local state mode + config.setString(CheckpointingOptions.LOCAL_RECOVERY, "ENABLE_FILE_BASED"); + + final ResourceID tmResourceID = ResourceID.generate(); + + TaskManagerServicesConfiguration taskManagerServicesConfiguration = + TaskManagerServicesConfiguration.fromConfiguration(config, InetAddress.getLocalHost(), true); + + TaskManagerServices taskManagerServices = TaskManagerServices.fromConfiguration( + taskManagerServicesConfiguration, + tmResourceID, + Executors.directExecutor(), + MEM_SIZE_PARAM, + MEM_SIZE_PARAM); + + TaskExecutorLocalStateStoresManager taskStateManager = taskManagerServices.getTaskManagerStateStore(); + + // verify configured directories for local state + String[] split = rootDirString.split(","); + File[] rootDirectories = taskStateManager.getLocalStateRootDirectories(); + for (int i = 0; i < split.length; ++i) { + Assert.assertEquals( + new File(split[i], TaskManagerServices.LOCAL_STATE_SUB_DIRECTORY_ROOT), + rootDirectories[i]); + } + + // verify local recovery mode + Assert.assertEquals( + LocalRecoveryConfig.LocalRecoveryMode.ENABLE_FILE_BASED, + taskStateManager.getLocalRecoveryMode()); + + Assert.assertEquals("localState", TaskManagerServices.LOCAL_STATE_SUB_DIRECTORY_ROOT); + for (File rootDirectory : rootDirectories) { + FileUtils.deleteFileOrDirectory(rootDirectory); + } + } + + /** + * This tests that the creation of {@link TaskManagerServices} correctly falls back to the first tmp directory of + * the IOManager as default for the local state root directory. + */ + @Test + public void testCreationFromConfigDefault() throws Exception { + + final Configuration config = new Configuration(); + + final ResourceID tmResourceID = ResourceID.generate(); + + TaskManagerServicesConfiguration taskManagerServicesConfiguration = + TaskManagerServicesConfiguration.fromConfiguration(config, InetAddress.getLocalHost(), true); + + TaskManagerServices taskManagerServices = TaskManagerServices.fromConfiguration( + taskManagerServicesConfiguration, + tmResourceID, + Executors.directExecutor(), + MEM_SIZE_PARAM, + MEM_SIZE_PARAM); + + TaskExecutorLocalStateStoresManager taskStateManager = taskManagerServices.getTaskManagerStateStore(); + + String[] tmpDirPaths = taskManagerServicesConfiguration.getTmpDirPaths(); + File[] localStateRootDirectories = taskStateManager.getLocalStateRootDirectories(); + + for (int i = 0; i < tmpDirPaths.length; ++i) { + Assert.assertEquals( + new File(tmpDirPaths[i], TaskManagerServices.LOCAL_STATE_SUB_DIRECTORY_ROOT), + localStateRootDirectories[i]); + } + + Assert.assertEquals( + LocalRecoveryConfig.LocalRecoveryMode.DISABLED, + taskStateManager.getLocalRecoveryMode()); + } + + /** + * This tests that the {@link TaskExecutorLocalStateStoresManager} creates {@link TaskLocalStateStoreImpl} that have + * a properly initialized local state base directory. It also checks that subdirectories are correctly deleted on + * shutdown. + */ + @Test + public void testSubtaskStateStoreDirectoryCreateAndDelete() throws Exception { + + JobID jobID = new JobID(); + JobVertexID jobVertexID = new JobVertexID(); + AllocationID allocationID = new AllocationID(); + int subtaskIdx = 23; + + File[] rootDirs = {temporaryFolder.newFolder(), temporaryFolder.newFolder(), temporaryFolder.newFolder()}; + TaskExecutorLocalStateStoresManager storesManager = new TaskExecutorLocalStateStoresManager( + LocalRecoveryConfig.LocalRecoveryMode.ENABLE_FILE_BASED, + rootDirs, + Executors.directExecutor()); + + TaskLocalStateStore taskLocalStateStore = + storesManager.localStateStoreForSubtask(jobID, allocationID, jobVertexID, subtaskIdx); + + LocalRecoveryDirectoryProvider directoryProvider = + taskLocalStateStore.getLocalRecoveryConfig().getLocalStateDirectoryProvider(); + + for (int i = 0; i < 10; ++i) { + Assert.assertEquals( + new File( + rootDirs[(i & Integer.MAX_VALUE) % rootDirs.length], + storesManager.allocationSubDirString(allocationID)), + directoryProvider.allocationBaseDirectory(i)); + } + + long chkId = 42L; + File allocBaseDirChk42 = directoryProvider.allocationBaseDirectory(chkId); + File subtaskSpecificCheckpointDirectory = directoryProvider.subtaskSpecificCheckpointDirectory(chkId); + Assert.assertEquals( + new File( + allocBaseDirChk42, + "jid_" + jobID + File.separator + + "vtx_" + jobVertexID + "_" + + "sti_" + subtaskIdx + File.separator + + "chk_" + chkId), + subtaskSpecificCheckpointDirectory); + + Assert.assertTrue(subtaskSpecificCheckpointDirectory.mkdirs()); + + File testFile = new File(subtaskSpecificCheckpointDirectory, "test"); + Assert.assertTrue(testFile.createNewFile()); + + // test that local recovery mode is forwarded to the created store + Assert.assertEquals( + storesManager.getLocalRecoveryMode(), + taskLocalStateStore.getLocalRecoveryConfig().getLocalRecoveryMode()); + + Assert.assertTrue(testFile.exists()); + + // check cleanup after releasing allocation id + storesManager.releaseLocalStateForAllocationId(allocationID); + checkRootDirsClean(rootDirs); + + AllocationID otherAllocationID = new AllocationID(); + + taskLocalStateStore = + storesManager.localStateStoreForSubtask(jobID, otherAllocationID, jobVertexID, subtaskIdx); + + directoryProvider = taskLocalStateStore.getLocalRecoveryConfig().getLocalStateDirectoryProvider(); + + File chkDir = directoryProvider.subtaskSpecificCheckpointDirectory(23L); + Assert.assertTrue(chkDir.mkdirs()); + testFile = new File(chkDir, "test"); + Assert.assertTrue(testFile.createNewFile()); + + // check cleanup after shutdown + storesManager.shutdown(); + checkRootDirsClean(rootDirs); + } + + private void checkRootDirsClean(File[] rootDirs) { + for (File rootDir : rootDirs) { + File[] files = rootDir.listFiles(); + if (files != null) { + Assert.assertArrayEquals(new File[0], files); + } + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskLocalStateStoreImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskLocalStateStoreImplTest.java new file mode 100644 index 0000000000000..16416762a525b --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskLocalStateStoreImplTest.java @@ -0,0 +1,174 @@ +/* + * 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.runtime.state; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.OperatorID; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.mockito.Mockito; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; + +import static org.powermock.api.mockito.PowerMockito.spy; + +public class TaskLocalStateStoreImplTest { + + private TemporaryFolder temporaryFolder; + private JobID jobID; + private AllocationID allocationID; + private JobVertexID jobVertexID; + private int subtaskIdx; + private File[] allocationBaseDirs; + private TaskLocalStateStoreImpl taskLocalStateStore; + + @Before + public void before() throws Exception { + this.temporaryFolder = new TemporaryFolder(); + this.temporaryFolder.create(); + this.jobID = new JobID(); + this.allocationID = new AllocationID(); + this.jobVertexID = new JobVertexID(); + this.subtaskIdx = 0; + this.allocationBaseDirs = new File[]{temporaryFolder.newFolder(), temporaryFolder.newFolder()}; + + LocalRecoveryDirectoryProviderImpl directoryProvider = + new LocalRecoveryDirectoryProviderImpl(allocationBaseDirs, jobID, jobVertexID, subtaskIdx); + + LocalRecoveryConfig localRecoveryConfig = + new LocalRecoveryConfig(LocalRecoveryConfig.LocalRecoveryMode.DISABLED, directoryProvider); + + this.taskLocalStateStore = new TaskLocalStateStoreImpl( + jobID, + allocationID, + jobVertexID, + subtaskIdx, + localRecoveryConfig, + Executors.directExecutor()); + } + + @After + public void after() { + this.temporaryFolder.delete(); + } + + /** + * Test that the instance delivers a correctly configured LocalRecoveryDirectoryProvider. + */ + @Test + public void getLocalRecoveryRootDirectoryProvider() { + + LocalRecoveryConfig directoryProvider = taskLocalStateStore.getLocalRecoveryConfig(); + Assert.assertEquals( + allocationBaseDirs.length, + directoryProvider.getLocalStateDirectoryProvider().allocationBaseDirsCount()); + + for (int i = 0; i < allocationBaseDirs.length; ++i) { + Assert.assertEquals( + allocationBaseDirs[i], + directoryProvider.getLocalStateDirectoryProvider().selectAllocationBaseDirectory(i)); + } + } + + /** + * Tests basic store/retrieve of local state. + */ + @Test + public void storeAndRetrieve() throws Exception { + + final int chkCount = 3; + + for (int i = 0; i < chkCount; ++i) { + Assert.assertNull(taskLocalStateStore.retrieveLocalState(i)); + } + + List taskStateSnapshots = storeStates(chkCount); + + checkStoredAsExpected(taskStateSnapshots, 0, chkCount); + + Assert.assertNull(taskLocalStateStore.retrieveLocalState(chkCount + 1)); + } + + /** + * Tests pruning of previous checkpoints if a new checkpoint is confirmed. + */ + @Test + public void confirmCheckpoint() throws Exception { + + final int chkCount = 3; + final int confirmed = chkCount - 1; + List taskStateSnapshots = storeStates(chkCount); + taskLocalStateStore.confirmCheckpoint(confirmed); + checkPrunedAndDiscarded(taskStateSnapshots, 0, confirmed); + checkStoredAsExpected(taskStateSnapshots, confirmed, chkCount); + } + + /** + * Tests that disposal of a {@link TaskLocalStateStoreImpl} works and discards all local states. + */ + @Test + public void dispose() throws Exception { + final int chkCount = 3; + final int confirmed = chkCount - 1; + List taskStateSnapshots = storeStates(chkCount); + taskLocalStateStore.confirmCheckpoint(confirmed); + taskLocalStateStore.dispose(); + + checkPrunedAndDiscarded(taskStateSnapshots, 0, chkCount); + } + + private void checkStoredAsExpected(List history, int off, int len) throws Exception { + for (int i = off; i < len; ++i) { + TaskStateSnapshot expected = history.get(i); + Assert.assertTrue(expected == taskLocalStateStore.retrieveLocalState(i)); + Mockito.verify(expected, Mockito.never()).discardState(); + } + } + + private void checkPrunedAndDiscarded(List history, int off, int len) throws Exception { + for (int i = off; i < len; ++i) { + Assert.assertNull(taskLocalStateStore.retrieveLocalState(i)); + Mockito.verify(history.get(i)).discardState(); + } + } + + private List storeStates(int count) { + List taskStateSnapshots = new ArrayList<>(count); + for (int i = 0; i < count; ++i) { + OperatorID operatorID = new OperatorID(); + TaskStateSnapshot taskStateSnapshot = spy(new TaskStateSnapshot()); + OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(); + taskStateSnapshot.putSubtaskStateByOperatorID(operatorID, operatorSubtaskState); + taskLocalStateStore.storeLocalState(i, taskStateSnapshot); + taskStateSnapshots.add(taskStateSnapshot); + } + return taskStateSnapshots; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskStateManagerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskStateManagerImplTest.java index 47bbebbcc88e3..926c1961c6652 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskStateManagerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskStateManagerImplTest.java @@ -23,93 +23,221 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.PrioritizedOperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateHandleDummyUtil; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.runtime.taskmanager.TestCheckpointResponder; - +import org.apache.flink.util.TestLogger; import org.junit.Assert; import org.junit.Test; +import org.junit.rules.TemporaryFolder; -import static org.mockito.Mockito.mock; +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import java.util.concurrent.Executor; -public class TaskStateManagerImplTest { +public class TaskStateManagerImplTest extends TestLogger { + /** + * Test reporting and retrieving prioritized local and remote state. + */ @Test public void testStateReportingAndRetrieving() { - JobID jobID = new JobID(42L, 43L); - ExecutionAttemptID executionAttemptID = new ExecutionAttemptID(23L, 24L); - TestCheckpointResponder checkpointResponderMock = new TestCheckpointResponder(); + JobID jobID = new JobID(); + ExecutionAttemptID executionAttemptID = new ExecutionAttemptID(); + + TestCheckpointResponder testCheckpointResponder = new TestCheckpointResponder(); + TestTaskLocalStateStore testTaskLocalStateStore = new TestTaskLocalStateStore(); TaskStateManager taskStateManager = taskStateManager( jobID, executionAttemptID, - checkpointResponderMock, - null); + testCheckpointResponder, + null, + testTaskLocalStateStore); //---------------------------------------- test reporting ----------------------------------------- CheckpointMetaData checkpointMetaData = new CheckpointMetaData(74L, 11L); CheckpointMetrics checkpointMetrics = new CheckpointMetrics(); - TaskStateSnapshot taskStateSnapshot = new TaskStateSnapshot(); + TaskStateSnapshot jmTaskStateSnapshot = new TaskStateSnapshot(); OperatorID operatorID_1 = new OperatorID(1L, 1L); OperatorID operatorID_2 = new OperatorID(2L, 2L); OperatorID operatorID_3 = new OperatorID(3L, 3L); - Assert.assertNull(taskStateManager.operatorStates(operatorID_1)); - Assert.assertNull(taskStateManager.operatorStates(operatorID_2)); - Assert.assertNull(taskStateManager.operatorStates(operatorID_3)); + Assert.assertFalse(taskStateManager.prioritizedOperatorState(operatorID_1).isRestored()); + Assert.assertFalse(taskStateManager.prioritizedOperatorState(operatorID_2).isRestored()); + Assert.assertFalse(taskStateManager.prioritizedOperatorState(operatorID_3).isRestored()); + + KeyGroupRange keyGroupRange = new KeyGroupRange(0,1); + // Remote state of operator 1 has only managed keyed state. + OperatorSubtaskState jmOperatorSubtaskState_1 = + new OperatorSubtaskState(null, null, StateHandleDummyUtil.createNewKeyedStateHandle(keyGroupRange), null); + // Remote state of operator 1 has only raw keyed state. + OperatorSubtaskState jmOperatorSubtaskState_2 = + new OperatorSubtaskState(null, null, null, StateHandleDummyUtil.createNewKeyedStateHandle(keyGroupRange)); + + jmTaskStateSnapshot.putSubtaskStateByOperatorID(operatorID_1, jmOperatorSubtaskState_1); + jmTaskStateSnapshot.putSubtaskStateByOperatorID(operatorID_2, jmOperatorSubtaskState_2); - OperatorSubtaskState operatorSubtaskState_1 = new OperatorSubtaskState(); - OperatorSubtaskState operatorSubtaskState_2 = new OperatorSubtaskState(); + TaskStateSnapshot tmTaskStateSnapshot = new TaskStateSnapshot(); - taskStateSnapshot.putSubtaskStateByOperatorID(operatorID_1, operatorSubtaskState_1); - taskStateSnapshot.putSubtaskStateByOperatorID(operatorID_2, operatorSubtaskState_2); + // Only operator 1 has a local alternative for the managed keyed state. + OperatorSubtaskState tmOperatorSubtaskState_1 = + new OperatorSubtaskState(null, null, StateHandleDummyUtil.createNewKeyedStateHandle(keyGroupRange), null); - taskStateManager.reportStateHandles(checkpointMetaData, checkpointMetrics, taskStateSnapshot); + tmTaskStateSnapshot.putSubtaskStateByOperatorID(operatorID_1, tmOperatorSubtaskState_1); + + taskStateManager.reportTaskStateSnapshots( + checkpointMetaData, + checkpointMetrics, + jmTaskStateSnapshot, + tmTaskStateSnapshot); TestCheckpointResponder.AcknowledgeReport acknowledgeReport = - checkpointResponderMock.getAcknowledgeReports().get(0); + testCheckpointResponder.getAcknowledgeReports().get(0); + // checks that the checkpoint responder and the local state store received state as expected. Assert.assertEquals(checkpointMetaData.getCheckpointId(), acknowledgeReport.getCheckpointId()); Assert.assertEquals(checkpointMetrics, acknowledgeReport.getCheckpointMetrics()); Assert.assertEquals(executionAttemptID, acknowledgeReport.getExecutionAttemptID()); Assert.assertEquals(jobID, acknowledgeReport.getJobID()); - Assert.assertEquals(taskStateSnapshot, acknowledgeReport.getSubtaskState()); + Assert.assertEquals(jmTaskStateSnapshot, acknowledgeReport.getSubtaskState()); + Assert.assertEquals(tmTaskStateSnapshot, testTaskLocalStateStore.retrieveLocalState(checkpointMetaData.getCheckpointId())); - //---------------------------------------- test retrieving ----------------------------------------- + //-------------------------------------- test prio retrieving --------------------------------------- JobManagerTaskRestore taskRestore = new JobManagerTaskRestore( - 0L, + checkpointMetaData.getCheckpointId(), acknowledgeReport.getSubtaskState()); taskStateManager = taskStateManager( jobID, executionAttemptID, - checkpointResponderMock, - taskRestore); + testCheckpointResponder, + taskRestore, + testTaskLocalStateStore); + + // this has remote AND local managed keyed state. + PrioritizedOperatorSubtaskState prioritized_1 = taskStateManager.prioritizedOperatorState(operatorID_1); + // this has only remote raw keyed state. + PrioritizedOperatorSubtaskState prioritized_2 = taskStateManager.prioritizedOperatorState(operatorID_2); + // not restored. + PrioritizedOperatorSubtaskState prioritized_3 = taskStateManager.prioritizedOperatorState(operatorID_3); + + Assert.assertTrue(prioritized_1.isRestored()); + Assert.assertTrue(prioritized_2.isRestored()); + Assert.assertFalse(prioritized_3.isRestored()); + Assert.assertFalse(taskStateManager.prioritizedOperatorState(new OperatorID()).isRestored()); + + // checks for operator 1. + Iterator> prioritizedManagedKeyedState_1 = + prioritized_1.getPrioritizedManagedKeyedState(); + + Assert.assertTrue(prioritizedManagedKeyedState_1.hasNext()); + StateObjectCollection current = prioritizedManagedKeyedState_1.next(); + KeyedStateHandle keyedStateHandleExp = tmOperatorSubtaskState_1.getManagedKeyedState().iterator().next(); + KeyedStateHandle keyedStateHandleAct = current.iterator().next(); + Assert.assertTrue(keyedStateHandleExp == keyedStateHandleAct); + Assert.assertTrue(prioritizedManagedKeyedState_1.hasNext()); + current = prioritizedManagedKeyedState_1.next(); + keyedStateHandleExp = jmOperatorSubtaskState_1.getManagedKeyedState().iterator().next(); + keyedStateHandleAct = current.iterator().next(); + Assert.assertTrue(keyedStateHandleExp == keyedStateHandleAct); + Assert.assertFalse(prioritizedManagedKeyedState_1.hasNext()); + + // checks for operator 2. + Iterator> prioritizedRawKeyedState_2 = + prioritized_2.getPrioritizedRawKeyedState(); + + Assert.assertTrue(prioritizedRawKeyedState_2.hasNext()); + current = prioritizedRawKeyedState_2.next(); + keyedStateHandleExp = jmOperatorSubtaskState_2.getRawKeyedState().iterator().next(); + keyedStateHandleAct = current.iterator().next(); + Assert.assertTrue(keyedStateHandleExp == keyedStateHandleAct); + Assert.assertFalse(prioritizedRawKeyedState_2.hasNext()); + } + + /** + * This tests if the {@link TaskStateManager} properly returns the the subtask local state dir from the + * corresponding {@link TaskLocalStateStoreImpl}. + */ + @Test + public void testForwardingSubtaskLocalStateBaseDirFromLocalStateStore() throws IOException { + JobID jobID = new JobID(42L, 43L); + AllocationID allocationID = new AllocationID(4711L, 23L); + JobVertexID jobVertexID = new JobVertexID(12L, 34L); + ExecutionAttemptID executionAttemptID = new ExecutionAttemptID(23L, 24L); + TestCheckpointResponder checkpointResponderMock = new TestCheckpointResponder(); - Assert.assertEquals(operatorSubtaskState_1, taskStateManager.operatorStates(operatorID_1)); - Assert.assertEquals(operatorSubtaskState_2, taskStateManager.operatorStates(operatorID_2)); - Assert.assertNull(taskStateManager.operatorStates(operatorID_3)); + Executor directExecutor = Executors.directExecutor(); + + TemporaryFolder tmpFolder = new TemporaryFolder(); + + try { + tmpFolder.create(); + + File[] allocBaseDirs = new File[]{tmpFolder.newFolder(), tmpFolder.newFolder(), tmpFolder.newFolder()}; + + LocalRecoveryDirectoryProviderImpl directoryProvider = + new LocalRecoveryDirectoryProviderImpl(allocBaseDirs, jobID, jobVertexID, 0); + + LocalRecoveryConfig localRecoveryConfig = + new LocalRecoveryConfig(LocalRecoveryConfig.LocalRecoveryMode.ENABLE_FILE_BASED, directoryProvider); + + TaskLocalStateStore taskLocalStateStore = + new TaskLocalStateStoreImpl(jobID, allocationID, jobVertexID, 13, localRecoveryConfig, directExecutor); + + TaskStateManager taskStateManager = taskStateManager( + jobID, + executionAttemptID, + checkpointResponderMock, + null, + taskLocalStateStore); + + LocalRecoveryConfig localRecoveryConfFromTaskLocalStateStore = + taskLocalStateStore.getLocalRecoveryConfig(); + + LocalRecoveryConfig localRecoveryConfFromTaskStateManager = + taskStateManager.createLocalRecoveryConfig(); + + + for (int i = 0; i < 10; ++i) { + Assert.assertEquals(allocBaseDirs[i % allocBaseDirs.length], + localRecoveryConfFromTaskLocalStateStore.getLocalStateDirectoryProvider().allocationBaseDirectory(i)); + Assert.assertEquals(allocBaseDirs[i % allocBaseDirs.length], + localRecoveryConfFromTaskStateManager.getLocalStateDirectoryProvider().allocationBaseDirectory(i)); + } + + Assert.assertEquals( + localRecoveryConfFromTaskLocalStateStore.getLocalRecoveryMode(), + localRecoveryConfFromTaskStateManager.getLocalRecoveryMode()); + } finally { + tmpFolder.delete(); + } } public static TaskStateManager taskStateManager( JobID jobID, ExecutionAttemptID executionAttemptID, CheckpointResponder checkpointResponderMock, - JobManagerTaskRestore jobManagerTaskRestore) { - - // for now just a mock because this is not yet implemented - TaskLocalStateStore taskLocalStateStore = mock(TaskLocalStateStore.class); + JobManagerTaskRestore jobManagerTaskRestore, + TaskLocalStateStore localStateStore) { return new TaskStateManagerImpl( jobID, executionAttemptID, - taskLocalStateStore, + localStateStore, jobManagerTaskRestore, checkpointResponderMock); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestLocalRecoveryConfig.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestLocalRecoveryConfig.java new file mode 100644 index 0000000000000..7801720140294 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestLocalRecoveryConfig.java @@ -0,0 +1,69 @@ +/* + * 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.runtime.state; + +import java.io.File; + +/** + * Helper methods to easily create a {@link LocalRecoveryConfig} for tests. + */ +public class TestLocalRecoveryConfig { + + private static final LocalRecoveryDirectoryProvider INSTANCE = new TestDummyLocalDirectoryProvider(); + + public static LocalRecoveryConfig disabled() { + return new LocalRecoveryConfig(LocalRecoveryConfig.LocalRecoveryMode.DISABLED, INSTANCE); + } + + public static class TestDummyLocalDirectoryProvider implements LocalRecoveryDirectoryProvider { + + private TestDummyLocalDirectoryProvider() { + } + + @Override + public File allocationBaseDirectory(long checkpointId) { + throw new UnsupportedOperationException("Test dummy"); + } + + @Override + public File subtaskBaseDirectory(long checkpointId) { + throw new UnsupportedOperationException("Test dummy"); + } + + @Override + public File subtaskSpecificCheckpointDirectory(long checkpointId) { + throw new UnsupportedOperationException("Test dummy"); + } + + @Override + public File selectAllocationBaseDirectory(int idx) { + throw new UnsupportedOperationException("Test dummy"); + } + + @Override + public File selectSubtaskBaseDirectory(int idx) { + throw new UnsupportedOperationException("Test dummy"); + } + + @Override + public int allocationBaseDirsCount() { + throw new UnsupportedOperationException("Test dummy"); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestMemoryCheckpointOutputStream.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestMemoryCheckpointOutputStream.java index 5accc1944d948..8ea76c6e06ed5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestMemoryCheckpointOutputStream.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestMemoryCheckpointOutputStream.java @@ -20,6 +20,8 @@ import org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory; +import javax.annotation.Nullable; + import java.io.IOException; final class TestMemoryCheckpointOutputStream extends MemCheckpointStreamFactory.MemoryCheckpointOutputStream { @@ -41,9 +43,10 @@ public boolean isClosed() { return this.closed; } + @Nullable @Override public StreamStateHandle closeAndGetHandle() throws IOException { this.closed = true; return super.closeAndGetHandle(); } -} \ No newline at end of file +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestTaskLocalStateStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestTaskLocalStateStore.java new file mode 100644 index 0000000000000..12c07ddd59542 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestTaskLocalStateStore.java @@ -0,0 +1,113 @@ +/* + * 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.runtime.state; + +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.util.Iterator; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; + +/** + * Test implementation of a {@link TaskLocalStateStore}. + */ +public class TestTaskLocalStateStore implements TaskLocalStateStore { + + private final SortedMap taskStateSnapshotsByCheckpointID; + + private final LocalRecoveryConfig localRecoveryConfig; + + private boolean disposed; + + public TestTaskLocalStateStore() { + this(TestLocalRecoveryConfig.disabled()); + } + + public TestTaskLocalStateStore(@Nonnull LocalRecoveryConfig localRecoveryConfig) { + this.localRecoveryConfig = localRecoveryConfig; + this.taskStateSnapshotsByCheckpointID = new TreeMap<>(); + this.disposed = false; + } + + @Override + public void storeLocalState(long checkpointId, @Nullable TaskStateSnapshot localState) { + Preconditions.checkState(!disposed); + taskStateSnapshotsByCheckpointID.put(checkpointId, localState); + } + + @Nullable + @Override + public TaskStateSnapshot retrieveLocalState(long checkpointID) { + Preconditions.checkState(!disposed); + return taskStateSnapshotsByCheckpointID.get(checkpointID); + } + + public void dispose() { + if (!disposed) { + disposed = true; + for (TaskStateSnapshot stateSnapshot : taskStateSnapshotsByCheckpointID.values()) { + try { + stateSnapshot.discardState(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + taskStateSnapshotsByCheckpointID.clear(); + } + } + + @Nonnull + @Override + public LocalRecoveryConfig getLocalRecoveryConfig() { + Preconditions.checkState(!disposed); + return Preconditions.checkNotNull(localRecoveryConfig); + } + + @Override + public void confirmCheckpoint(long confirmedCheckpointId) { + Preconditions.checkState(!disposed); + Iterator> iterator = taskStateSnapshotsByCheckpointID.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + if (entry.getKey() < confirmedCheckpointId) { + iterator.remove(); + try { + entry.getValue().discardState(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } else { + break; + } + } + } + + public boolean isDisposed() { + return disposed; + } + + public SortedMap getTaskStateSnapshotsByCheckpointID() { + return taskStateSnapshotsByCheckpointID; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestTaskStateManager.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestTaskStateManager.java index f8084ca6c1849..20f66c3275dbb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestTaskStateManager.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestTaskStateManager.java @@ -23,16 +23,20 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.PrioritizedOperatorSubtaskState; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.runtime.taskmanager.TestCheckpointResponder; +import org.apache.flink.util.Preconditions; import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; /** @@ -45,43 +49,59 @@ public class TestTaskStateManager implements TaskStateManager { private JobID jobId; private ExecutionAttemptID executionAttemptID; - private final Map taskStateSnapshotsByCheckpointId; + private final Map jobManagerTaskStateSnapshotsByCheckpointId; + private final Map taskManagerTaskStateSnapshotsByCheckpointId; private CheckpointResponder checkpointResponder; private OneShotLatch waitForReportLatch; + private LocalRecoveryConfig localRecoveryDirectoryProvider; public TestTaskStateManager() { - this(new JobID(), new ExecutionAttemptID(), new TestCheckpointResponder()); + this(TestLocalRecoveryConfig.disabled()); + } + + public TestTaskStateManager(LocalRecoveryConfig localRecoveryConfig) { + this( + new JobID(), + new ExecutionAttemptID(), + new TestCheckpointResponder(), + localRecoveryConfig); } public TestTaskStateManager( JobID jobId, ExecutionAttemptID executionAttemptID) { - - this(jobId, executionAttemptID, null); + this(jobId, executionAttemptID, null, TestLocalRecoveryConfig.disabled()); } public TestTaskStateManager( JobID jobId, ExecutionAttemptID executionAttemptID, - CheckpointResponder checkpointResponder) { + CheckpointResponder checkpointResponder, + LocalRecoveryConfig localRecoveryConfig) { this.jobId = jobId; this.executionAttemptID = executionAttemptID; this.checkpointResponder = checkpointResponder; - this.taskStateSnapshotsByCheckpointId = new HashMap<>(); + this.localRecoveryDirectoryProvider = localRecoveryConfig; + this.jobManagerTaskStateSnapshotsByCheckpointId = new HashMap<>(); + this.taskManagerTaskStateSnapshotsByCheckpointId = new HashMap<>(); this.reportedCheckpointId = -1L; } @Override - public void reportStateHandles( + public void reportTaskStateSnapshots( @Nonnull CheckpointMetaData checkpointMetaData, @Nonnull CheckpointMetrics checkpointMetrics, - @Nullable TaskStateSnapshot acknowledgedState) { + @Nullable TaskStateSnapshot acknowledgedState, + @Nullable TaskStateSnapshot localState) { - if (taskStateSnapshotsByCheckpointId != null) { - taskStateSnapshotsByCheckpointId.put( - checkpointMetaData.getCheckpointId(), - acknowledgedState); - } + + jobManagerTaskStateSnapshotsByCheckpointId.put( + checkpointMetaData.getCheckpointId(), + acknowledgedState); + + taskManagerTaskStateSnapshotsByCheckpointId.put( + checkpointMetaData.getCheckpointId(), + localState); if (checkpointResponder != null) { checkpointResponder.acknowledgeCheckpoint( @@ -99,10 +119,48 @@ public void reportStateHandles( } } + @Nonnull @Override - public OperatorSubtaskState operatorStates(OperatorID operatorID) { - TaskStateSnapshot taskStateSnapshot = getLastTaskStateSnapshot(); - return taskStateSnapshot != null ? taskStateSnapshot.getSubtaskStateByOperatorID(operatorID) : null; + public PrioritizedOperatorSubtaskState prioritizedOperatorState(OperatorID operatorID) { + TaskStateSnapshot jmTaskStateSnapshot = getLastJobManagerTaskStateSnapshot(); + TaskStateSnapshot tmTaskStateSnapshot = getLastTaskManagerTaskStateSnapshot(); + + if (jmTaskStateSnapshot == null) { + + return PrioritizedOperatorSubtaskState.emptyNotRestored(); + } else { + + OperatorSubtaskState jmOpState = jmTaskStateSnapshot.getSubtaskStateByOperatorID(operatorID); + + if (jmOpState == null) { + + return PrioritizedOperatorSubtaskState.emptyNotRestored(); + } else { + + List tmStateCollection = Collections.emptyList(); + + if (tmTaskStateSnapshot != null) { + OperatorSubtaskState tmOpState = tmTaskStateSnapshot.getSubtaskStateByOperatorID(operatorID); + if (tmOpState != null) { + tmStateCollection = Collections.singletonList(tmOpState); + } + } + PrioritizedOperatorSubtaskState.Builder builder = + new PrioritizedOperatorSubtaskState.Builder(jmOpState, tmStateCollection); + return builder.build(); + } + } + } + + @Nonnull + @Override + public LocalRecoveryConfig createLocalRecoveryConfig() { + return Preconditions.checkNotNull(localRecoveryDirectoryProvider, + "Local state directory was never set for this test object!"); + } + + public void setLocalRecoveryConfig(LocalRecoveryConfig recoveryDirectoryProvider) { + this.localRecoveryDirectoryProvider = recoveryDirectoryProvider; } @Override @@ -134,13 +192,24 @@ public void setCheckpointResponder(CheckpointResponder checkpointResponder) { this.checkpointResponder = checkpointResponder; } - public Map getTaskStateSnapshotsByCheckpointId() { - return taskStateSnapshotsByCheckpointId; + public Map getJobManagerTaskStateSnapshotsByCheckpointId() { + return jobManagerTaskStateSnapshotsByCheckpointId; } - public void setTaskStateSnapshotsByCheckpointId(Map taskStateSnapshotsByCheckpointId) { - this.taskStateSnapshotsByCheckpointId.clear(); - this.taskStateSnapshotsByCheckpointId.putAll(taskStateSnapshotsByCheckpointId); + public void setJobManagerTaskStateSnapshotsByCheckpointId( + Map jobManagerTaskStateSnapshotsByCheckpointId) { + this.jobManagerTaskStateSnapshotsByCheckpointId.clear(); + this.jobManagerTaskStateSnapshotsByCheckpointId.putAll(jobManagerTaskStateSnapshotsByCheckpointId); + } + + public Map getTaskManagerTaskStateSnapshotsByCheckpointId() { + return taskManagerTaskStateSnapshotsByCheckpointId; + } + + public void setTaskManagerTaskStateSnapshotsByCheckpointId( + Map taskManagerTaskStateSnapshotsByCheckpointId) { + this.taskManagerTaskStateSnapshotsByCheckpointId.clear(); + this.taskManagerTaskStateSnapshotsByCheckpointId.putAll(taskManagerTaskStateSnapshotsByCheckpointId); } public long getReportedCheckpointId() { @@ -151,9 +220,15 @@ public void setReportedCheckpointId(long reportedCheckpointId) { this.reportedCheckpointId = reportedCheckpointId; } - public TaskStateSnapshot getLastTaskStateSnapshot() { - return taskStateSnapshotsByCheckpointId != null ? - taskStateSnapshotsByCheckpointId.get(reportedCheckpointId) + public TaskStateSnapshot getLastJobManagerTaskStateSnapshot() { + return jobManagerTaskStateSnapshotsByCheckpointId != null ? + jobManagerTaskStateSnapshotsByCheckpointId.get(reportedCheckpointId) + : null; + } + + public TaskStateSnapshot getLastTaskManagerTaskStateSnapshot() { + return taskManagerTaskStateSnapshotsByCheckpointId != null ? + taskManagerTaskStateSnapshotsByCheckpointId.get(reportedCheckpointId) : null; } @@ -181,6 +256,6 @@ public void restoreLatestCheckpointState(Map taskStateS } setReportedCheckpointId(latestId); - setTaskStateSnapshotsByCheckpointId(taskStateSnapshotsByCheckpointId); + setJobManagerTaskStateSnapshotsByCheckpointId(taskStateSnapshotsByCheckpointId); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/AbstractCheckpointStateOutputStreamTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/AbstractCheckpointStateOutputStreamTestBase.java new file mode 100644 index 0000000000000..6b526f40b3915 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/AbstractCheckpointStateOutputStreamTestBase.java @@ -0,0 +1,324 @@ +/* + * 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.runtime.state.filesystem; + +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.fs.local.LocalDataOutputStream; +import org.apache.flink.core.fs.local.LocalFileSystem; +import org.apache.flink.core.testutils.CheckedThread; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.FunctionWithException; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.EOFException; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.util.Random; +import java.util.UUID; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; + +/** + * Abstract base class for tests against checkpointing streams. + */ +public abstract class AbstractCheckpointStateOutputStreamTestBase extends TestLogger { + + @Rule + public final TemporaryFolder tmp = new TemporaryFolder(); + + // ------------------------------------------------------------------------ + // Tests + // ------------------------------------------------------------------------ + + /** + * Validates that even empty streams create a file and a file state handle. + */ + @Test + public void testEmptyState() throws Exception { + final FileSystem fs = FileSystem.getLocalFileSystem(); + final Path folder = baseFolder(); + final String fileName = "myFileName"; + final Path filePath = new Path(folder, fileName); + + final FileStateHandle handle; + try (FSDataOutputStream stream = createTestStream(fs, folder, fileName)) { + handle = closeAndGetResult(stream); + } + + // must have created a handle + assertNotNull(handle); + assertEquals(filePath, handle.getFilePath()); + + // the pointer path should exist as a directory + assertTrue(fs.exists(handle.getFilePath())); + assertFalse(fs.getFileStatus(filePath).isDir()); + + // the contents should be empty + try (FSDataInputStream in = handle.openInputStream()) { + assertEquals(-1, in.read()); + } + } + + /** + * Simple write and read test + */ + @Test + public void testWriteAndRead() throws Exception { + final FileSystem fs = FileSystem.getLocalFileSystem(); + final Path folder = baseFolder(); + final String fileName = "fooBarName"; + + final Random rnd = new Random(); + final byte[] data = new byte[1694523]; + + // write the data (mixed single byte writes and array writes) + final FileStateHandle handle; + try (FSDataOutputStream stream = createTestStream(fs, folder, fileName)) { + for (int i = 0; i < data.length; ) { + if (rnd.nextBoolean()) { + stream.write(data[i++]); + } else { + int len = rnd.nextInt(Math.min(data.length - i, 32)); + stream.write(data, i, len); + i += len; + } + } + handle = closeAndGetResult(stream); + } + + // (1) stream from handle must hold the contents + try (FSDataInputStream in = handle.openInputStream()) { + byte[] buffer = new byte[data.length]; + readFully(in, buffer); + assertArrayEquals(data, buffer); + } + + // (2) the pointer must point to a file with that contents + try (FSDataInputStream in = fs.open(handle.getFilePath())) { + byte[] buffer = new byte[data.length]; + readFully(in, buffer); + assertArrayEquals(data, buffer); + } + } + + /** + * Tests that the underlying stream file is deleted upon calling close. + */ + @Test + public void testCleanupWhenClosingStream() throws IOException { + final FileSystem fs = FileSystem.getLocalFileSystem(); + final Path folder = new Path(tmp.newFolder().toURI()); + final String fileName = "nonCreativeTestFileName"; + final Path path = new Path(folder, fileName); + + // write some test data and close the stream + try (FSDataOutputStream stream = createTestStream(fs, folder, fileName)) { + Random rnd = new Random(); + for (int i = 0; i < rnd.nextInt(1000); i++) { + stream.write(rnd.nextInt(100)); + } + assertTrue(fs.exists(path)); + } + + assertFalse(fs.exists(path)); + } + + /** + * Tests that the underlying stream file is deleted if the closeAndGetHandle method fails. + */ + @Test + public void testCleanupWhenFailingCloseAndGetHandle() throws IOException { + final Path folder = new Path(tmp.newFolder().toURI()); + final String fileName = "test_name"; + final Path filePath = new Path(folder, fileName); + + final FileSystem fs = spy(new TestFs((path) -> new FailingCloseStream(new File(path.getPath())))); + + FSDataOutputStream stream = createTestStream(fs, folder, fileName); + stream.write(new byte[] {1, 2, 3, 4, 5}); + + try { + closeAndGetResult(stream); + fail("Expected IOException"); + } + catch (IOException ignored) { + // expected exception + } + + verify(fs).delete(filePath, false); + } + + /** + * This test validates that a close operation can happen even while a 'closeAndGetHandle()' + * call is in progress. + *

+ *

That behavior is essential for fast cancellation (concurrent cleanup). + */ + @Test + public void testCloseDoesNotLock() throws Exception { + final Path folder = new Path(tmp.newFolder().toURI()); + final String fileName = "this-is-ignored-anyways.file"; + + final FileSystem fileSystem = spy(new TestFs((path) -> new BlockerStream())); + + final FSDataOutputStream checkpointStream = + createTestStream(fileSystem, folder, fileName); + + final OneShotLatch sync = new OneShotLatch(); + + final CheckedThread thread = new CheckedThread() { + + @Override + public void go() throws Exception { + sync.trigger(); + // that call should now block, because it accesses the position + closeAndGetResult(checkpointStream); + } + }; + thread.start(); + + sync.await(); + checkpointStream.close(); + + // the thread may or may not fail, that depends on the thread race + // it is not important for this test, important is that the thread does not freeze/lock up + try { + thread.sync(); + } catch (IOException ignored) {} + } + + /** + * Creates a new test stream instance. + */ + protected abstract FSDataOutputStream createTestStream( + FileSystem fs, + Path dir, + String fileName) throws IOException; + + /** + * Closes the stream successfully and returns a FileStateHandle to the result. + */ + protected abstract FileStateHandle closeAndGetResult(FSDataOutputStream stream) throws IOException; + + // ------------------------------------------------------------------------ + // utilities + // ------------------------------------------------------------------------ + + private Path baseFolder() throws Exception { + return new Path(new File(tmp.newFolder(), UUID.randomUUID().toString()).toURI()); + } + + private static void readFully(InputStream in, byte[] buffer) throws IOException { + int pos = 0; + int remaining = buffer.length; + + while (remaining > 0) { + int read = in.read(buffer, pos, remaining); + if (read == -1) { + throw new EOFException(); + } + + pos += read; + remaining -= read; + } + } + + private static class BlockerStream extends FSDataOutputStream { + + private final OneShotLatch blocker = new OneShotLatch(); + + @Override + public long getPos() throws IOException { + block(); + return 0L; + } + + @Override + public void write(int b) throws IOException { + block(); + } + + @Override + public void flush() throws IOException { + block(); + } + + @Override + public void sync() throws IOException { + block(); + } + + @Override + public void close() throws IOException { + blocker.trigger(); + } + + private void block() throws IOException { + try { + blocker.await(); + } catch (InterruptedException e) { + throw new IOException("interrupted"); + } + throw new IOException("closed"); + } + } + + // ------------------------------------------------------------------------ + + private static class FailingCloseStream extends LocalDataOutputStream { + + FailingCloseStream(File file) throws IOException { + super(file); + } + + @Override + public void close() throws IOException { + throw new IOException(); + } + } + + private static class TestFs extends LocalFileSystem { + + private final FunctionWithException streamFactory; + + TestFs(FunctionWithException streamFactory) { + this.streamFactory = streamFactory; + } + + @Override + public FSDataOutputStream create(Path filePath, WriteMode overwrite) throws IOException { + return streamFactory.apply(filePath); + } + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FileBasedStateOutputStreamTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FileBasedStateOutputStreamTest.java new file mode 100644 index 0000000000000..9ee59d2826ef4 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FileBasedStateOutputStreamTest.java @@ -0,0 +1,41 @@ +/* + * 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.runtime.state.filesystem; + +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; + +import java.io.IOException; + +/** + * Tests for the {@link FileBasedStateOutputStream}. + */ +public class FileBasedStateOutputStreamTest extends AbstractCheckpointStateOutputStreamTestBase { + + @Override + protected FSDataOutputStream createTestStream(FileSystem fs, Path dir, String fileName) throws IOException { + return new FileBasedStateOutputStream(fs, new Path(dir, fileName)); + } + + @Override + protected FileStateHandle closeAndGetResult(FSDataOutputStream stream) throws IOException { + return ((FileBasedStateOutputStream) stream).closeAndGetHandle(); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointMetadataOutputStreamTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointMetadataOutputStreamTest.java index 5a15b4b57d0bc..4803e93af1703 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointMetadataOutputStreamTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointMetadataOutputStreamTest.java @@ -18,291 +18,25 @@ package org.apache.flink.runtime.state.filesystem; -import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.FSDataOutputStream; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; -import org.apache.flink.core.fs.local.LocalDataOutputStream; -import org.apache.flink.core.fs.local.LocalFileSystem; -import org.apache.flink.core.testutils.CheckedThread; -import org.apache.flink.core.testutils.OneShotLatch; -import org.apache.flink.util.function.FunctionWithException; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.EOFException; -import java.io.File; import java.io.IOException; -import java.io.InputStream; -import java.util.Random; - -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.verify; /** * Tests for the {@link FsCheckpointMetadataOutputStream}. */ -public class FsCheckpointMetadataOutputStreamTest { - - @Rule - public final TemporaryFolder tmp = new TemporaryFolder(); - - // ------------------------------------------------------------------------ - // Tests - // ------------------------------------------------------------------------ - - /** - * Validates that even empty streams create a file and a file state handle. - */ - @Test - public void testEmptyState() throws Exception { - final FileSystem fs = FileSystem.getLocalFileSystem(); - - final Path checkpointDir = Path.fromLocalFile(tmp.newFolder()); - final Path metadataPath = new Path(checkpointDir, "myFileName"); - - final FsCompletedCheckpointStorageLocation location; - try (FsCheckpointMetadataOutputStream stream = new FsCheckpointMetadataOutputStream(fs, metadataPath, checkpointDir)) { - location = stream.closeAndFinalizeCheckpoint(); - } - - // must have created a handle - assertNotNull(location); - assertNotNull(location.getMetadataHandle()); - assertEquals(metadataPath, location.getMetadataHandle().getFilePath()); - - // the pointer path should exist as a file - assertTrue(fs.exists(metadataPath)); - assertFalse(fs.getFileStatus(metadataPath).isDir()); - - // the contents should be empty - try (FSDataInputStream in = location.getMetadataHandle().openInputStream()) { - assertEquals(-1, in.read()); - } - } - - /** - * Simple write and read test. - */ - @Test - public void testWriteAndRead() throws Exception { - final FileSystem fs = FileSystem.getLocalFileSystem(); - - final Path checkpointDir = Path.fromLocalFile(tmp.newFolder()); - final Path metadataPath = new Path(checkpointDir, "fooBarName"); - - final Random rnd = new Random(); - final byte[] data = new byte[1694523]; - - // write the data (mixed single byte writes and array writes) - final FsCompletedCheckpointStorageLocation completed; - try (FsCheckpointMetadataOutputStream stream = new FsCheckpointMetadataOutputStream(fs, metadataPath, checkpointDir)) { - for (int i = 0; i < data.length;) { - if (rnd.nextBoolean()) { - stream.write(data[i++]); - } - else { - int len = rnd.nextInt(Math.min(data.length - i, 32)); - stream.write(data, i, len); - i += len; - } - } - completed = stream.closeAndFinalizeCheckpoint(); - } - - // (1) stream from handle must hold the contents - try (FSDataInputStream in = completed.getMetadataHandle().openInputStream()) { - byte[] buffer = new byte[data.length]; - readFully(in, buffer); - assertArrayEquals(data, buffer); - } - - // (2) the pointer must point to a file with that contents - try (FSDataInputStream in = fs.open(completed.getMetadataHandle().getFilePath())) { - byte[] buffer = new byte[data.length]; - readFully(in, buffer); - assertArrayEquals(data, buffer); - } - } - - /** - * Tests that the underlying stream file is deleted upon calling close. - */ - @Test - public void testCleanupWhenClosingStream() throws IOException { - final FileSystem fs = FileSystem.getLocalFileSystem(); - - final Path checkpointDir = Path.fromLocalFile(tmp.newFolder()); - final Path metadataPath = new Path(checkpointDir, "nonCreativeTestFileName"); - - // write some test data and close the stream - try (FsCheckpointMetadataOutputStream stream = new FsCheckpointMetadataOutputStream(fs, metadataPath, checkpointDir)) { - Random rnd = new Random(); - for (int i = 0; i < rnd.nextInt(1000); i++) { - stream.write(rnd.nextInt(100)); - } - assertTrue(fs.exists(metadataPath)); - } - - assertFalse(fs.exists(metadataPath)); - assertTrue(fs.exists(checkpointDir)); - } - - /** - * Tests that the underlying stream file is deleted if the closeAndGetHandle method fails. - */ - @Test - public void testCleanupWhenFailingCloseAndGetHandle() throws IOException { - final Path checkpointDir = Path.fromLocalFile(tmp.newFolder()); - final Path metadataPath = new Path(checkpointDir, "test_name"); - - final FileSystem fs = spy(new TestFs((path) -> new FailingCloseStream(new File(path.getPath())))); - - FsCheckpointMetadataOutputStream stream = new FsCheckpointMetadataOutputStream(fs, metadataPath, checkpointDir); - stream.write(new byte[] {1, 2, 3, 4, 5}); - - try { - stream.closeAndFinalizeCheckpoint(); - fail("Expected IOException"); - } - catch (IOException ignored) { - // expected exception - } - - verify(fs).delete(metadataPath, false); - } +public class FsCheckpointMetadataOutputStreamTest extends AbstractCheckpointStateOutputStreamTestBase { - /** - * This test validates that a close operation can happen even while a 'closeAndGetHandle()' - * call is in progress. - * - *

That behavior is essential for fast cancellation (concurrent cleanup). - */ - @Test - public void testCloseDoesNotLock() throws Exception { - final Path checkpointDir = Path.fromLocalFile(tmp.newFolder()); - final Path metadataPath = new Path(checkpointDir, "this-is-ignored-anyways.file"); - - final FileSystem fileSystem = spy(new TestFs((path) -> new BlockerStream())); - - final FsCheckpointMetadataOutputStream checkpointStream = - new FsCheckpointMetadataOutputStream(fileSystem, metadataPath, checkpointDir); - - final OneShotLatch sync = new OneShotLatch(); - - final CheckedThread thread = new CheckedThread() { - - @Override - public void go() throws Exception { - sync.trigger(); - // that call should now block, because it accesses the position - checkpointStream.closeAndFinalizeCheckpoint(); - } - }; - thread.start(); - - sync.await(); - checkpointStream.close(); - - // the thread may or may not fail, that depends on the thread race - // it is not important for this test, important is that the thread does not freeze/lock up - try { - thread.sync(); - } catch (IOException ignored) {} + @Override + protected FSDataOutputStream createTestStream(FileSystem fs, Path dir, String fileName) throws IOException { + Path fullPath = new Path(dir, fileName); + return new FsCheckpointMetadataOutputStream(fs, fullPath, dir); } - // ------------------------------------------------------------------------ - // utilities - // ------------------------------------------------------------------------ - - private static void readFully(InputStream in, byte[] buffer) throws IOException { - int pos = 0; - int remaining = buffer.length; - - while (remaining > 0) { - int read = in.read(buffer, pos, remaining); - if (read == -1) { - throw new EOFException(); - } - - pos += read; - remaining -= read; - } - } - - private static class BlockerStream extends FSDataOutputStream { - - private final OneShotLatch blocker = new OneShotLatch(); - - @Override - public long getPos() throws IOException { - block(); - return 0L; - } - - @Override - public void write(int b) throws IOException { - block(); - } - - @Override - public void flush() throws IOException { - block(); - } - - @Override - public void sync() throws IOException { - block(); - } - - @Override - public void close() throws IOException { - blocker.trigger(); - } - - private void block() throws IOException { - try { - blocker.await(); - } catch (InterruptedException e) { - throw new IOException("interrupted"); - } - throw new IOException("closed"); - } - } - - // ------------------------------------------------------------------------ - - private static class FailingCloseStream extends LocalDataOutputStream { - - FailingCloseStream(File file) throws IOException { - super(file); - } - - @Override - public void close() throws IOException { - throw new IOException(); - } - } - - private static class TestFs extends LocalFileSystem { - - private final FunctionWithException streamFactory; - - TestFs(FunctionWithException streamFactory) { - this.streamFactory = streamFactory; - } - - @Override - public FSDataOutputStream create(Path filePath, WriteMode overwrite) throws IOException { - return streamFactory.apply(filePath); - } + @Override + protected FileStateHandle closeAndGetResult(FSDataOutputStream stream) throws IOException { + return ((FsCheckpointMetadataOutputStream) stream).closeAndFinalizeCheckpoint().getMetadataHandle(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackendSnapshotMigrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackendSnapshotMigrationTest.java index 3ac3b6be83a20..d83aa9fafd182 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackendSnapshotMigrationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackendSnapshotMigrationTest.java @@ -21,11 +21,12 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; import org.apache.flink.runtime.state.KeyGroupsStateHandle; -import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.internal.InternalListState; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; + import org.junit.Test; import java.io.BufferedInputStream; @@ -64,7 +65,7 @@ public void testRestore1_2ToMaster() throws Exception { try (BufferedInputStream bis = new BufferedInputStream((new FileInputStream(resource.getFile())))) { stateHandle = InstantiationUtil.deserializeObject(bis, Thread.currentThread().getContextClassLoader()); } - keyedBackend.restore(Collections.singleton(stateHandle)); + keyedBackend.restore(StateObjectCollection.singleton(stateHandle)); final ListStateDescriptor stateDescr = new ListStateDescriptor<>("my-state", Long.class); stateDescr.initializeSerializerUnlessSet(new ExecutionConfig()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapStateBackendTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapStateBackendTestBase.java index b10c2c0132550..bf428dc0776af 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapStateBackendTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/HeapStateBackendTestBase.java @@ -23,6 +23,8 @@ import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.TestLocalRecoveryConfig; + import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -54,6 +56,7 @@ public HeapKeyedStateBackend createKeyedBackend(TypeSerializer keySeri 16, new KeyGroupRange(0, 15), async, - new ExecutionConfig()); + new ExecutionConfig(), + TestLocalRecoveryConfig.disabled()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/testutils/BackendForTestStream.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/testutils/BackendForTestStream.java index b40f179f0106b..d958fccbf87b3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/testutils/BackendForTestStream.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/testutils/BackendForTestStream.java @@ -68,7 +68,7 @@ public CheckpointStorage createCheckpointStorage(JobID jobId) throws IOException // ------------------------------------------------------------------------ public interface StreamFactory - extends SupplierWithException, java.io.Serializable {} + extends SupplierWithException, java.io.Serializable {} // ------------------------------------------------------------------------ @@ -119,7 +119,7 @@ private static final class TestFactory implements CheckpointStreamFactory, java. } @Override - public CheckpointStateOutputStream createCheckpointStateOutputStream(CheckpointedStateScope scope) throws Exception { + public CheckpointStateOutputStream createCheckpointStateOutputStream(CheckpointedStateScope scope) throws IOException { return streamFactory.get(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkBufferCalculationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkBufferCalculationTest.java index bc6f238f42512..8dcd6420b6b9a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkBufferCalculationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkBufferCalculationTest.java @@ -20,6 +20,7 @@ import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.memory.MemoryType; +import org.apache.flink.runtime.state.LocalRecoveryConfig; import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; import org.apache.flink.testutils.category.OldAndFlip6; import org.apache.flink.util.TestLogger; @@ -100,6 +101,8 @@ private static TaskManagerServicesConfiguration getTmConfig( return new TaskManagerServicesConfiguration( InetAddress.getLoopbackAddress(), new String[] {}, + new String[] {}, + LocalRecoveryConfig.LocalRecoveryMode.DISABLED, networkConfig, QueryableStateConfiguration.disabled(), 1, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java index 8f4ec5d7af073..d693f477affdf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java @@ -50,6 +50,8 @@ import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.state.LocalRecoveryConfig; +import org.apache.flink.runtime.state.TaskExecutorLocalStateStoresManager; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; import org.apache.flink.runtime.taskexecutor.slot.TimerService; @@ -64,6 +66,7 @@ import org.junit.experimental.categories.Category; import org.mockito.Mockito; +import java.io.File; import java.net.InetAddress; import java.util.Arrays; import java.util.Collection; @@ -131,6 +134,14 @@ public void testSlotAllocation() throws Exception { TestingUtils.infiniteTime(), TestingUtils.infiniteTime()); + final File[] taskExecutorLocalStateRootDirs = + new File[]{new File(System.getProperty("java.io.tmpdir"), "localRecovery")}; + + final TaskExecutorLocalStateStoresManager taskStateManager = new TaskExecutorLocalStateStoresManager( + LocalRecoveryConfig.LocalRecoveryMode.DISABLED, + taskExecutorLocalStateRootDirs, + rpcService.getExecutor()); + ResourceManager resourceManager = new StandaloneResourceManager( rpcService, FlinkResourceManager.RESOURCE_MANAGER_NAME, @@ -147,6 +158,7 @@ public void testSlotAllocation() throws Exception { final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskManagerLocation(taskManagerLocation) .setTaskSlotTable(taskSlotTable) + .setTaskStateManager(taskStateManager) .build(); TaskExecutor taskExecutor = new TaskExecutor( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index d7a1860697147..e972feb0c3bed 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.SlotID; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; @@ -74,6 +75,8 @@ import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.state.LocalRecoveryConfig; +import org.apache.flink.runtime.state.TaskExecutorLocalStateStoresManager; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; import org.apache.flink.runtime.taskexecutor.slot.TimerService; @@ -95,6 +98,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; import org.junit.rules.TestName; import org.mockito.ArgumentCaptor; import org.mockito.Matchers; @@ -103,6 +107,7 @@ import org.mockito.stubbing.Answer; import org.slf4j.Logger; +import java.io.File; import java.io.IOException; import java.util.Arrays; import java.util.Collection; @@ -137,6 +142,9 @@ @Category(Flip6.class) public class TaskExecutorTest extends TestLogger { + @Rule + public final TemporaryFolder tmp = new TemporaryFolder(); + private static final Time timeout = Time.milliseconds(10000L); private TestingRpcService rpc; @@ -227,10 +235,16 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { final SimpleJobMasterGateway jobMasterGateway = new SimpleJobMasterGateway( CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId))); + TaskExecutorLocalStateStoresManager localStateStoresManager = new TaskExecutorLocalStateStoresManager( + LocalRecoveryConfig.LocalRecoveryMode.DISABLED, + new File[]{tmp.newFolder()}, + Executors.directExecutor()); + final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskManagerLocation(taskManagerLocation) .setTaskSlotTable(taskSlotTable) .setJobLeaderService(jobLeaderService) + .setTaskStateManager(localStateStoresManager) .build(); final TaskExecutor taskManager = new TaskExecutor( @@ -310,9 +324,15 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { HeartbeatServices heartbeatServices = new HeartbeatServices(heartbeatInterval, heartbeatTimeout); + TaskExecutorLocalStateStoresManager localStateStoresManager = new TaskExecutorLocalStateStoresManager( + LocalRecoveryConfig.LocalRecoveryMode.DISABLED, + new File[]{tmp.newFolder()}, + Executors.directExecutor()); + final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskManagerLocation(taskManagerLocation) .setTaskSlotTable(taskSlotTable) + .setTaskStateManager(localStateStoresManager) .build(); final TaskExecutor taskManager = new TaskExecutor( @@ -439,9 +459,15 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation } ); + TaskExecutorLocalStateStoresManager localStateStoresManager = new TaskExecutorLocalStateStoresManager( + LocalRecoveryConfig.LocalRecoveryMode.DISABLED, + new File[]{tmp.newFolder()}, + Executors.directExecutor()); + final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskManagerLocation(taskManagerLocation) .setTaskSlotTable(taskSlotTable) + .setTaskStateManager(localStateStoresManager) .build(); final TaskExecutor taskManager = new TaskExecutor( @@ -518,9 +544,15 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { final SlotReport slotReport = new SlotReport(); when(taskSlotTable.createSlotReport(any(ResourceID.class))).thenReturn(slotReport); + TaskExecutorLocalStateStoresManager localStateStoresManager = new TaskExecutorLocalStateStoresManager( + LocalRecoveryConfig.LocalRecoveryMode.DISABLED, + new File[]{tmp.newFolder()}, + Executors.directExecutor()); + final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskManagerLocation(taskManagerLocation) .setTaskSlotTable(taskSlotTable) + .setTaskStateManager(localStateStoresManager) .build(); TaskExecutor taskManager = new TaskExecutor( @@ -575,9 +607,15 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { final SlotReport slotReport = new SlotReport(); when(taskSlotTable.createSlotReport(any(ResourceID.class))).thenReturn(slotReport); + TaskExecutorLocalStateStoresManager localStateStoresManager = new TaskExecutorLocalStateStoresManager( + LocalRecoveryConfig.LocalRecoveryMode.DISABLED, + new File[]{tmp.newFolder()}, + Executors.directExecutor()); + final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskManagerLocation(taskManagerLocation) .setTaskSlotTable(taskSlotTable) + .setTaskStateManager(localStateStoresManager) .build(); TaskExecutor taskManager = new TaskExecutor( @@ -690,10 +728,16 @@ public void testTaskSubmission() throws Exception { when(networkEnvironment.createKvStateTaskRegistry(eq(jobId), eq(jobVertexId))).thenReturn(mock(TaskKvStateRegistry.class)); when(networkEnvironment.getTaskEventDispatcher()).thenReturn(taskEventDispatcher); + TaskExecutorLocalStateStoresManager localStateStoresManager = new TaskExecutorLocalStateStoresManager( + LocalRecoveryConfig.LocalRecoveryMode.DISABLED, + new File[]{tmp.newFolder()}, + Executors.directExecutor()); + final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setNetworkEnvironment(networkEnvironment) .setTaskSlotTable(taskSlotTable) .setJobManagerTable(jobManagerTable) + .setTaskStateManager(localStateStoresManager) .build(); TaskExecutor taskManager = new TaskExecutor( @@ -787,11 +831,17 @@ public void testJobLeaderDetection() throws Exception { final SlotID slotId = new SlotID(taskManagerLocation.getResourceID(), 0); final SlotOffer slotOffer = new SlotOffer(allocationId, 0, ResourceProfile.UNKNOWN); + TaskExecutorLocalStateStoresManager localStateStoresManager = new TaskExecutorLocalStateStoresManager( + LocalRecoveryConfig.LocalRecoveryMode.DISABLED, + new File[]{tmp.newFolder()}, + Executors.directExecutor()); + final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskManagerLocation(taskManagerLocation) .setTaskSlotTable(taskSlotTable) .setJobManagerTable(jobManagerTable) .setJobLeaderService(jobLeaderService) + .setTaskStateManager(localStateStoresManager) .build(); TaskExecutor taskManager = new TaskExecutor( @@ -891,11 +941,17 @@ public void testSlotAcceptance() throws Exception { rpc.registerGateway(resourceManagerAddress, resourceManagerGateway); rpc.registerGateway(jobManagerAddress, jobMasterGateway); + TaskExecutorLocalStateStoresManager localStateStoresManager = new TaskExecutorLocalStateStoresManager( + LocalRecoveryConfig.LocalRecoveryMode.DISABLED, + new File[]{tmp.newFolder()}, + Executors.directExecutor()); + final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskManagerLocation(taskManagerLocation) .setTaskSlotTable(taskSlotTable) .setJobManagerTable(jobManagerTable) .setJobLeaderService(jobLeaderService) + .setTaskStateManager(localStateStoresManager) .build(); TaskExecutor taskManager = new TaskExecutor( @@ -1017,12 +1073,18 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { final NetworkEnvironment networkMock = mock(NetworkEnvironment.class, Mockito.RETURNS_MOCKS); + TaskExecutorLocalStateStoresManager localStateStoresManager = new TaskExecutorLocalStateStoresManager( + LocalRecoveryConfig.LocalRecoveryMode.DISABLED, + new File[]{tmp.newFolder()}, + Executors.directExecutor()); + final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskManagerLocation(taskManagerLocation) .setNetworkEnvironment(networkMock) .setTaskSlotTable(taskSlotTable) .setJobLeaderService(jobLeaderService) .setJobManagerTable(jobManagerTable) + .setTaskStateManager(localStateStoresManager) .build(); final TaskExecutor taskManager = new TaskExecutor( @@ -1130,10 +1192,16 @@ public void testFilterOutDuplicateJobMasterRegistrations() throws Exception { final JMTMRegistrationSuccess registrationMessage = new JMTMRegistrationSuccess(ResourceID.generate()); final JobManagerTable jobManagerTableMock = spy(new JobManagerTable()); + TaskExecutorLocalStateStoresManager localStateStoresManager = new TaskExecutorLocalStateStoresManager( + LocalRecoveryConfig.LocalRecoveryMode.DISABLED, + new File[]{tmp.newFolder()}, + Executors.directExecutor()); + final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskManagerLocation(taskManagerLocation) .setJobManagerTable(jobManagerTableMock) .setJobLeaderService(jobLeaderService) + .setTaskStateManager(localStateStoresManager) .build(); final TaskExecutor taskExecutor = new TaskExecutor( @@ -1198,9 +1266,15 @@ public void testRMHeartbeatStopWhenLeadershipRevoked() throws Exception { rpc.registerGateway(rmAddress, rmGateway); + TaskExecutorLocalStateStoresManager localStateStoresManager = new TaskExecutorLocalStateStoresManager( + LocalRecoveryConfig.LocalRecoveryMode.DISABLED, + new File[]{tmp.newFolder()}, + Executors.directExecutor()); + final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskManagerLocation(taskManagerLocation) .setTaskSlotTable(taskSlotTable) + .setTaskStateManager(localStateStoresManager) .build(); final TaskExecutor taskExecutor = new TaskExecutor( @@ -1248,9 +1322,15 @@ public void testRemoveJobFromJobLeaderService() throws Exception { Collections.singleton(ResourceProfile.UNKNOWN), timerService); + TaskExecutorLocalStateStoresManager localStateStoresManager = new TaskExecutorLocalStateStoresManager( + LocalRecoveryConfig.LocalRecoveryMode.DISABLED, + new File[]{tmp.newFolder()}, + Executors.directExecutor()); + final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskManagerLocation(taskManagerLocation) .setTaskSlotTable(taskSlotTable) + .setTaskStateManager(localStateStoresManager) .build(); final TaskExecutor taskExecutor = new TaskExecutor( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java index 2bd39f885c437..3af9a4647a98f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java @@ -63,8 +63,7 @@ public TaskManagerServicesBuilder() { taskSlotTable = mock(TaskSlotTable.class); jobManagerTable = new JobManagerTable(); jobLeaderService = new JobLeaderService(taskManagerLocation); - taskStateManager = new TaskExecutorLocalStateStoresManager(); - + taskStateManager = mock(TaskExecutorLocalStateStoresManager.class); } public TaskManagerServicesBuilder setTaskManagerLocation(TaskManagerLocation taskManagerLocation) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java index e8ecc5606685b..581d8edcd1018 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.clusterframework.FlinkResourceManager; import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServices; @@ -45,6 +46,8 @@ import org.apache.flink.runtime.metrics.NoOpMetricRegistry; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.query.KvStateRegistry; +import org.apache.flink.runtime.state.LocalRecoveryConfig; +import org.apache.flink.runtime.state.TaskExecutorLocalStateStoresManager; import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.util.TestLogger; @@ -158,6 +161,11 @@ public void testComponentsStartupShutdown() throws Exception { network.start(); + TaskExecutorLocalStateStoresManager storesManager = new TaskExecutorLocalStateStoresManager( + LocalRecoveryConfig.LocalRecoveryMode.DISABLED, + ioManager.getSpillingDirectories(), + Executors.directExecutor()); + MetricRegistryConfiguration metricRegistryConfiguration = MetricRegistryConfiguration.fromConfiguration(config); // create the task manager @@ -169,6 +177,7 @@ public void testComponentsStartupShutdown() throws Exception { memManager, ioManager, network, + storesManager, numberOfSlots, highAvailabilityServices, new TaskManagerMetricGroup(NoOpMetricRegistry.INSTANCE, connectionInfo.getHostname(), connectionInfo.getResourceID().getResourceIdString())); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java index 107826f4a22a1..fce962021b1c8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerStartupTest.java @@ -18,15 +18,13 @@ package org.apache.flink.runtime.taskmanager; -import static org.junit.Assert.*; - -import org.apache.commons.io.FileUtils; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServices; @@ -36,6 +34,8 @@ import org.apache.flink.util.NetUtils; import org.apache.flink.util.TestLogger; +import akka.actor.ActorSystem; +import org.apache.commons.io.FileUtils; import org.junit.After; import org.junit.Assume; import org.junit.Before; @@ -43,16 +43,19 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import scala.Option; - import java.io.File; import java.io.IOException; -import java.net.InetAddress; import java.net.BindException; +import java.net.InetAddress; import java.net.ServerSocket; import java.util.ArrayList; import java.util.List; +import scala.Option; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + /** * Tests that check how the TaskManager behaves when encountering startup * problems. @@ -246,11 +249,11 @@ public void testStartupWhenNetworkStackFailsToInitialize() throws Exception { cfg.setString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, "localhost"); cfg.setInteger(TaskManagerOptions.DATA_PORT, blocker.getLocalPort()); cfg.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 1L); - + ActorSystem actorSystem = AkkaUtils.createLocalActorSystem(cfg); TaskManager.startTaskManagerComponentsAndActor( cfg, ResourceID.generate(), - null, + actorSystem, highAvailabilityServices, NoOpMetricRegistry.INSTANCE, "localhost", diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockerCheckpointStreamFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockerCheckpointStreamFactory.java index bc9b04f31d272..ce41b10de24b3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockerCheckpointStreamFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockerCheckpointStreamFactory.java @@ -39,9 +39,9 @@ public class BlockerCheckpointStreamFactory implements CheckpointStreamFactory { protected volatile OneShotLatch blocker; protected volatile OneShotLatch waiter; - MemCheckpointStreamFactory.MemoryCheckpointOutputStream lastCreatedStream; + BlockingCheckpointOutputStream lastCreatedStream; - public MemCheckpointStreamFactory.MemoryCheckpointOutputStream getLastCreatedStream() { + public BlockingCheckpointOutputStream getLastCreatedStream() { return lastCreatedStream; } @@ -70,80 +70,13 @@ public OneShotLatch getWaiterLatch() { } @Override - public CheckpointStateOutputStream createCheckpointStateOutputStream(CheckpointedStateScope scope) throws Exception { - this.lastCreatedStream = new MemCheckpointStreamFactory.MemoryCheckpointOutputStream(maxSize) { - - private int afterNInvocations = afterNumberInvocations; - private final OneShotLatch streamBlocker = blocker; - private final OneShotLatch streamWaiter = waiter; - - @Override - public void write(int b) throws IOException { - - unblockWaiter(); - - if (afterNInvocations > 0) { - --afterNInvocations; - } else { - awaitBlocker(); - } - - try { - super.write(b); - } catch (IOException ex) { - unblockWaiter(); - throw ex; - } - - if (0 == afterNInvocations) { - unblockWaiter(); - } - - // We also check for close here, in case the underlying stream does not do this - if (isClosed()) { - throw new IOException("Stream closed."); - } - } - - //We override this to ensure that writes go through the blocking #write(int) method! - @Override - public void write(byte[] b, int off, int len) throws IOException { - for (int i = 0; i < len; i++) { - write(b[off + i]); - } - } - - @Override - public void close() { - super.close(); - // trigger all the latches, essentially all blocking ops on the stream should resume after close. - unblockAll(); - } - - private void unblockWaiter() { - if (null != streamWaiter) { - streamWaiter.trigger(); - } - } - - private void awaitBlocker() { - if (null != streamBlocker) { - try { - streamBlocker.await(); - } catch (InterruptedException ignored) { - } - } - } - - private void unblockAll() { - if (null != streamWaiter) { - streamWaiter.trigger(); - } - if (null != streamBlocker) { - streamBlocker.trigger(); - } - } - }; + public CheckpointStateOutputStream createCheckpointStateOutputStream(CheckpointedStateScope scope) throws IOException { + + this.lastCreatedStream = new BlockingCheckpointOutputStream( + new MemCheckpointStreamFactory.MemoryCheckpointOutputStream(maxSize), + waiter, + blocker, + afterNumberInvocations); return lastCreatedStream; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockingCheckpointOutputStream.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockingCheckpointOutputStream.java new file mode 100644 index 0000000000000..fd8e59d40ab94 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockingCheckpointOutputStream.java @@ -0,0 +1,202 @@ +/* + * 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.runtime.util; + +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.util.IOUtils; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + /** + * This test utility class provides a CheckpointStateOutputStream (which is also a FSDataOutputStream) that can block + * on a latch and takes a latch that the creator can block on until the stream is closed. This is typically used to + * test that a blocking read can be interrupted / closed. + */ +public class BlockingCheckpointOutputStream extends CheckpointStreamFactory.CheckpointStateOutputStream { + + /** Optional delegate stream to which all ops are forwarded. */ + private final FSDataOutputStream delegate; + + /** Optional latch on which the stream blocks, e.g. until the test triggers it after some call to #close(). */ + private final OneShotLatch triggerUnblock; + + /** Optional latch on which the test can block until the stream is blocked at the desired blocking position. */ + private final OneShotLatch waitForBlocking; + + /** Closed flag. */ + private final AtomicBoolean closed; + + /** The read position at which this will block. 0 by default. */ + private final long blockAtPosition; + + /** The current read position. */ + private long position; + + public BlockingCheckpointOutputStream( + @Nullable OneShotLatch waitForBlocking, + @Nullable OneShotLatch triggerUnblock) { + this(null, waitForBlocking, triggerUnblock, 0L); + } + + public BlockingCheckpointOutputStream( + @Nullable FSDataOutputStream delegate, + @Nullable OneShotLatch waitForBlock, + @Nullable OneShotLatch triggerUnblock) { + this(delegate, waitForBlock, triggerUnblock, 0L); + } + + public BlockingCheckpointOutputStream( + @Nullable FSDataOutputStream delegate, + @Nullable OneShotLatch waitForBlocking, + @Nullable OneShotLatch triggerUnblock, + long blockAtPosition) { + + this.delegate = delegate; + this.triggerUnblock = triggerUnblock; + this.waitForBlocking = waitForBlocking; + this.blockAtPosition = blockAtPosition; + if (delegate != null) { + try { + this.position = delegate.getPos(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } else { + this.position = 0; + } + this.closed = new AtomicBoolean(false); + } + + @Override + public void write(int b) throws IOException { + + if (position == blockAtPosition) { + unblockWaiter(); + awaitUnblocker(); + } + + if (delegate != null) { + try { + delegate.write(b); + } catch (IOException ex) { + unblockWaiter(); + throw ex; + } + } + + // We also check for close here, in case the underlying stream does not do this + if (closed.get()) { + throw new IOException("Stream closed."); + } + + ++position; + } + + //We override this to ensure that writes go through the blocking #write(int) method! + @Override + public void write(byte[] b, int off, int len) throws IOException { + for (int i = 0; i < len; i++) { + write(b[off + i]); + } + } + + @Override + public long getPos() throws IOException { + return position; + } + + @Override + public void flush() throws IOException { + if (delegate != null) { + delegate.flush(); + } + } + + @Override + public void sync() throws IOException { + if (delegate != null) { + delegate.sync(); + } + } + + @Override + public void close() { + if (closed.compareAndSet(false, true)) { + if (delegate != null) { + IOUtils.closeQuietly(delegate); + } + // trigger all the latches, essentially all blocking ops on the stream should resume after close. + unblockAll(); + } + } + + private void unblockWaiter() { + if (null != waitForBlocking) { + waitForBlocking.trigger(); + } + } + + private void awaitUnblocker() { + if (null != triggerUnblock) { + try { + triggerUnblock.await(); + } catch (InterruptedException ignored) { + } + } + } + + private void unblockAll() { + if (null != waitForBlocking) { + waitForBlocking.trigger(); + } + if (null != triggerUnblock) { + triggerUnblock.trigger(); + } + } + + @Nullable + @Override + public StreamStateHandle closeAndGetHandle() throws IOException { + + if (!closed.compareAndSet(false, true)) { + throw new IOException("Stream was already closed!"); + } + + if (delegate instanceof CheckpointStreamFactory.CheckpointStateOutputStream) { + StreamStateHandle streamStateHandle = + ((CheckpointStreamFactory.CheckpointStateOutputStream) delegate).closeAndGetHandle(); + unblockAll(); + return streamStateHandle; + } else { + unblockAll(); + throw new IOException("Delegate is not a CheckpointStateOutputStream!"); + } + } + + public boolean isClosed() { + return closed.get(); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockingFSDataInputStream.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockingFSDataInputStream.java new file mode 100644 index 0000000000000..4721980253939 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/BlockingFSDataInputStream.java @@ -0,0 +1,196 @@ +/* + * 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.runtime.util; + +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.util.IOUtils; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * This test utility class provides a {@link FSDataInputStream} that can block on a latch and takes a latch that + * the creator can block on until the stream is closed. This is typically used to test that a blocking read can be + * interrupted / closed. + */ +public class BlockingFSDataInputStream extends FSDataInputStream { + + /** Optional delegate stream to which all ops are forwarded. */ + private final FSDataInputStream delegate; + + /** Optional latch on which the stream blocks, e.g. until the test triggers it after some call to #close(). */ + private final OneShotLatch triggerUnblock; + + /** Optional latch on which the test can block until the stream is blocked at the desired blocking position. */ + private final OneShotLatch waitUntilStreamBlocked; + + /** Closed flag. */ + private final AtomicBoolean closed; + + /** The read position at which this will block. 0 by default. */ + private final long blockAtPosition; + + /** The current read position. */ + private long position; + + public BlockingFSDataInputStream( + @Nullable OneShotLatch waitForBlock, + @Nullable OneShotLatch triggerUnblock) { + this(null, waitForBlock, triggerUnblock, 0L); + } + + public BlockingFSDataInputStream( + @Nullable FSDataInputStream delegate, + @Nullable OneShotLatch waitForBlock, + @Nullable OneShotLatch triggerUnblock) { + this(delegate, waitForBlock, triggerUnblock, 0L); + } + + public BlockingFSDataInputStream( + @Nullable FSDataInputStream delegate, + @Nullable OneShotLatch waitForBlock, + @Nullable OneShotLatch triggerUnblock, + long blockAtPosition) { + + this.delegate = delegate; + this.triggerUnblock = triggerUnblock; + this.waitUntilStreamBlocked = waitForBlock; + this.blockAtPosition = blockAtPosition; + if (delegate != null) { + try { + this.position = delegate.getPos(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } else { + this.position = 0; + } + this.closed = new AtomicBoolean(false); + } + + @Override + public void seek(long desired) throws IOException { + if (delegate != null) { + delegate.seek(desired); + } + this.position = desired; + } + + @Override + public long getPos() throws IOException { + return position; + } + + @Override + public int read() throws IOException { + + if (position == blockAtPosition) { + unblockWaiter(); + awaitBlocker(); + } + + int val = 0; + if (delegate != null) { + try { + val = delegate.read(); + } catch (IOException ex) { + unblockWaiter(); + throw ex; + } + } + + // We also check for close here, in case the underlying stream does not do this + if (closed.get()) { + throw new IOException("Stream closed."); + } else { + ++position; + return val; + } + } + + + @Override + public int read(byte[] b, int off, int len) throws IOException { + // We override this to ensure that we use the blocking read method internally. + if (b == null) { + throw new NullPointerException(); + } else if (off < 0 || len < 0 || len > b.length - off) { + throw new IndexOutOfBoundsException(); + } else if (len == 0) { + return 0; + } + + int c = read(); + if (c == -1) { + return -1; + } + b[off] = (byte) c; + + int i = 1; + try { + for (; i < len; i++) { + c = read(); + if (c == -1) { + break; + } + b[off + i] = (byte) c; + } + } catch (IOException ee) { + } + return i; + } + + @Override + public void close() { + if (closed.compareAndSet(false, true)) { + if (delegate != null) { + IOUtils.closeQuietly(delegate); + } + // trigger all the latches, essentially all blocking ops on the stream should resume after close. + unblockAll(); + } + } + + private void unblockWaiter() { + if (null != waitUntilStreamBlocked) { + waitUntilStreamBlocked.trigger(); + } + } + + private void awaitBlocker() { + if (null != triggerUnblock) { + try { + triggerUnblock.await(); + } catch (InterruptedException ignored) { + } + } + } + + private void unblockAll() { + if (null != waitUntilStreamBlocked) { + waitUntilStreamBlocked.trigger(); + } + if (null != triggerUnblock) { + triggerUnblock.trigger(); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java index fe293d71cecbc..cc0d4fcac52cb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java @@ -56,8 +56,10 @@ import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.TaskLocalStateStore; +import org.apache.flink.runtime.state.TaskLocalStateStoreImpl; import org.apache.flink.runtime.state.TaskStateManager; import org.apache.flink.runtime.state.TaskStateManagerImpl; +import org.apache.flink.runtime.state.TestLocalRecoveryConfig; import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration; import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.runtime.taskmanager.Task; @@ -68,8 +70,9 @@ import org.apache.flink.runtime.testutils.TestJvmProcess; import org.apache.flink.util.OperatingSystem; import org.apache.flink.util.SerializedValue; - +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import java.util.Collections; import java.util.concurrent.CompletableFuture; @@ -86,6 +89,9 @@ */ public class JvmExitOnFatalErrorTest { + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Test public void testExitJvmOnOutOfMemory() throws Exception { // this test works only on linux @@ -138,12 +144,13 @@ public static void main(String[] args) throws Exception { System.err.println("creating task"); // we suppress process exits via errors here to not - // have a test that exits accidentally due to a programming error + // have a test that exits accidentally due to a programming error try { final Configuration taskManagerConfig = new Configuration(); taskManagerConfig.setBoolean(TaskManagerOptions.KILL_ON_OUT_OF_MEMORY, true); final JobID jid = new JobID(); + final AllocationID allocationID = new AllocationID(); final JobVertexID jobVertexId = new JobVertexID(); final ExecutionAttemptID executionAttemptID = new ExecutionAttemptID(); final AllocationID slotAllocationId = new AllocationID(); @@ -172,7 +179,15 @@ public static void main(String[] args) throws Exception { BlobCacheService blobService = new BlobCacheService(mock(PermanentBlobCache.class), mock(TransientBlobCache.class)); - final TaskLocalStateStore localStateStore = new TaskLocalStateStore(jid, jobVertexId, 0); + final TaskLocalStateStore localStateStore = + new TaskLocalStateStoreImpl( + jid, + allocationID, + jobVertexId, + 0, + TestLocalRecoveryConfig.disabled(), + executor); + final TaskStateManager slotStateManager = new TaskStateManagerImpl( jid, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestByteStreamStateHandleDeepCompare.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestByteStreamStateHandleDeepCompare.java deleted file mode 100644 index 7d8797b1ef571..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestByteStreamStateHandleDeepCompare.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.runtime.util; - -import org.apache.flink.runtime.state.StreamStateHandle; -import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; -import org.apache.flink.util.InstantiationUtil; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Arrays; - -public class TestByteStreamStateHandleDeepCompare extends ByteStreamStateHandle { - - private static final long serialVersionUID = -4946526195523509L; - - public TestByteStreamStateHandleDeepCompare(String handleName, byte[] data) { - super(handleName, data); - } - - @Override - public boolean equals(Object o) { - if (!super.equals(o)) { - return false; - } - ByteStreamStateHandle other = (ByteStreamStateHandle) o; - return Arrays.equals(getData(), other.getData()); - } - - @Override - public int hashCode() { - return 31 * super.hashCode() + Arrays.hashCode(getData()); - } - - public static StreamStateHandle fromSerializable(String handleName, Serializable value) throws IOException { - return new TestByteStreamStateHandleDeepCompare(handleName, InstantiationUtil.serializeObject(value)); - } -} \ No newline at end of file diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala index da753ae03cae7..5e60be9299e7d 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala @@ -24,6 +24,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup +import org.apache.flink.runtime.state.TaskExecutorLocalStateStoresManager import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation} @@ -38,6 +39,7 @@ class TestingTaskManager( memoryManager: MemoryManager, ioManager: IOManager, network: NetworkEnvironment, + taskManagerStateStore: TaskExecutorLocalStateStoresManager, numberOfSlots: Int, highAvailabilityServices: HighAvailabilityServices, taskManagerMetricGroup : TaskManagerMetricGroup) @@ -48,6 +50,7 @@ class TestingTaskManager( memoryManager, ioManager, network, + taskManagerStateStore, numberOfSlots, highAvailabilityServices, taskManagerMetricGroup) @@ -59,6 +62,7 @@ class TestingTaskManager( memoryManager: MemoryManager, ioManager: IOManager, network: NetworkEnvironment, + taskManagerLocalStateStoresManager: TaskExecutorLocalStateStoresManager, numberOfSlots: Int, highAvailabilityServices: HighAvailabilityServices, taskManagerMetricGroup : TaskManagerMetricGroup) { @@ -69,6 +73,7 @@ class TestingTaskManager( memoryManager, ioManager, network, + taskManagerLocalStateStoresManager, numberOfSlots, highAvailabilityServices, taskManagerMetricGroup) diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index 4fc02832c9102..344255fe95925 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -60,9 +60,11 @@ import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedBackendSerializationProxy; import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.LocalRecoveryConfig; import org.apache.flink.runtime.state.PlaceholderStreamStateHandle; import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo; import org.apache.flink.runtime.state.SnappyStreamCompressionDecorator; +import org.apache.flink.runtime.state.SnapshotResult; import org.apache.flink.runtime.state.StateHandleID; import org.apache.flink.runtime.state.StateObject; import org.apache.flink.runtime.state.StateUtil; @@ -212,6 +214,9 @@ public class RocksDBKeyedStateBackend extends AbstractKeyedStateBackend { /** Unique ID of this backend. */ private UUID backendUID; + /** The configuration of local recovery. */ + private final LocalRecoveryConfig localRecoveryConfig; + public RocksDBKeyedStateBackend( String operatorIdentifier, ClassLoader userCodeClassLoader, @@ -223,7 +228,8 @@ public RocksDBKeyedStateBackend( int numberOfKeyGroups, KeyGroupRange keyGroupRange, ExecutionConfig executionConfig, - boolean enableIncrementalCheckpointing + boolean enableIncrementalCheckpointing, + LocalRecoveryConfig localRecoveryConfig ) throws IOException { super(kvStateRegistry, keySerializer, userCodeClassLoader, numberOfKeyGroups, keyGroupRange, executionConfig); @@ -253,6 +259,7 @@ public RocksDBKeyedStateBackend( String.format("Could not create RocksDB data directory at %s.", instanceBasePath.getAbsolutePath())); } + this.localRecoveryConfig = Preconditions.checkNotNull(localRecoveryConfig); this.keyGroupPrefixBytes = getNumberOfKeyGroups() > (Byte.MAX_VALUE + 1) ? 2 : 1; this.kvStateInformation = new HashMap<>(); this.restoredKvStateMetaInfos = new HashMap<>(); @@ -365,10 +372,10 @@ public int getKeyGroupPrefixBytes() { * @param streamFactory The factory that we can use for writing our state to streams. * @param checkpointOptions Options for how to perform this checkpoint. * @return Future to the state handle of the snapshot data. - * @throws Exception + * @throws Exception indicating a problem in the synchronous part of the checkpoint. */ @Override - public RunnableFuture snapshot( + public RunnableFuture> snapshot( final long checkpointId, final long timestamp, final CheckpointStreamFactory streamFactory, @@ -382,7 +389,7 @@ public RunnableFuture snapshot( } } - private RunnableFuture snapshotIncrementally( + private RunnableFuture> snapshotIncrementally( final long checkpointId, final long checkpointTimestamp, final CheckpointStreamFactory checkpointStreamFactory) throws Exception { @@ -396,7 +403,7 @@ private RunnableFuture snapshotIncrementally( LOG.debug("Asynchronous RocksDB snapshot performed on empty keyed state at {}. Returning null.", checkpointTimestamp); } - return DoneFuture.nullValue(); + return DoneFuture.of(SnapshotResult.empty()); } final RocksDBIncrementalSnapshotOperation snapshotOperation = @@ -414,11 +421,12 @@ private RunnableFuture snapshotIncrementally( throw e; } - return new FutureTask( - new Callable() { + return new FutureTask>( + new Callable>() { @Override - public KeyedStateHandle call() throws Exception { - return snapshotOperation.materializeSnapshot(); + public SnapshotResult call() throws Exception { + KeyedStateHandle keyedStateHandle = snapshotOperation.materializeSnapshot(); + return SnapshotResult.of(keyedStateHandle); } } ) { @@ -435,7 +443,7 @@ protected void done() { }; } - private RunnableFuture snapshotFully( + private RunnableFuture> snapshotFully( final long checkpointId, final long timestamp, final CheckpointStreamFactory streamFactory) throws Exception { @@ -450,15 +458,15 @@ private RunnableFuture snapshotFully( LOG.debug("Asynchronous RocksDB snapshot performed on empty keyed state at {}. Returning null.", timestamp); } - return DoneFuture.nullValue(); + return DoneFuture.of(SnapshotResult.empty()); } snapshotOperation = new RocksDBFullSnapshotOperation<>(this, streamFactory, snapshotCloseableRegistry); snapshotOperation.takeDBSnapShot(checkpointId, timestamp); // implementation of the async IO operation, based on FutureTask - AbstractAsyncCallableWithResources ioCallable = - new AbstractAsyncCallableWithResources() { + AbstractAsyncCallableWithResources> ioCallable = + new AbstractAsyncCallableWithResources>() { @Override protected void acquireResources() throws Exception { @@ -493,7 +501,7 @@ private void closeLocalRegistry() { } @Override - public KeyGroupsStateHandle performOperation() throws Exception { + public SnapshotResult performOperation() throws Exception { long startTime = System.currentTimeMillis(); if (isStopped()) { @@ -505,7 +513,8 @@ public KeyGroupsStateHandle performOperation() throws Exception { LOG.info("Asynchronous RocksDB snapshot ({}, asynchronous part) in thread {} took {} ms.", streamFactory, Thread.currentThread(), (System.currentTimeMillis() - startTime)); - return snapshotOperation.getSnapshotResultStateHandle(); + KeyGroupsStateHandle snapshotResultStateHandle = snapshotOperation.getSnapshotResultStateHandle(); + return SnapshotResult.of(snapshotResultStateHandle); } }; diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java index b7e47948c7435..f60cb2cd82dec 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java @@ -33,10 +33,12 @@ import org.apache.flink.runtime.state.ConfigurableStateBackend; import org.apache.flink.runtime.state.DefaultOperatorStateBackend; import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.LocalRecoveryConfig; import org.apache.flink.runtime.state.OperatorStateBackend; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.util.AbstractID; +import org.apache.flink.util.TernaryBoolean; import org.rocksdb.ColumnFamilyOptions; import org.rocksdb.DBOptions; @@ -103,10 +105,8 @@ public class RocksDBStateBackend extends AbstractStateBackend implements Configu @Nullable private OptionsFactory optionsFactory; - /** True if incremental checkpointing is enabled. - * Null if not yet set, in which case the configuration values will be used. */ - @Nullable - private Boolean enableIncrementalCheckpointing; + /** This determines if incremental checkpointing is enabled. */ + private final TernaryBoolean enableIncrementalCheckpointing; // -- runtime values, set on TaskManager when initializing / using the backend @@ -201,7 +201,7 @@ public RocksDBStateBackend(URI checkpointDataUri, boolean enableIncrementalCheck * @param checkpointStreamBackend The backend write the checkpoint streams to. */ public RocksDBStateBackend(StateBackend checkpointStreamBackend) { - this.checkpointStreamBackend = checkNotNull(checkpointStreamBackend); + this(checkpointStreamBackend, TernaryBoolean.UNDEFINED); } /** @@ -215,7 +215,7 @@ public RocksDBStateBackend(StateBackend checkpointStreamBackend) { * @param checkpointStreamBackend The backend write the checkpoint streams to. * @param enableIncrementalCheckpointing True if incremental checkpointing is enabled. */ - public RocksDBStateBackend(StateBackend checkpointStreamBackend, boolean enableIncrementalCheckpointing) { + public RocksDBStateBackend(StateBackend checkpointStreamBackend, TernaryBoolean enableIncrementalCheckpointing) { this.checkpointStreamBackend = checkNotNull(checkpointStreamBackend); this.enableIncrementalCheckpointing = enableIncrementalCheckpointing; } @@ -225,16 +225,15 @@ public RocksDBStateBackend(StateBackend checkpointStreamBackend, boolean enableI */ @Deprecated public RocksDBStateBackend(AbstractStateBackend checkpointStreamBackend) { - this.checkpointStreamBackend = checkNotNull(checkpointStreamBackend); + this(checkpointStreamBackend, TernaryBoolean.UNDEFINED); } /** - * @deprecated Use {@link #RocksDBStateBackend(StateBackend, boolean)} instead. + * @deprecated Use {@link #RocksDBStateBackend(StateBackend, TernaryBoolean)} instead. */ @Deprecated public RocksDBStateBackend(AbstractStateBackend checkpointStreamBackend, boolean enableIncrementalCheckpointing) { - this.checkpointStreamBackend = checkNotNull(checkpointStreamBackend); - this.enableIncrementalCheckpointing = enableIncrementalCheckpointing; + this(checkpointStreamBackend, TernaryBoolean.fromBoolean(enableIncrementalCheckpointing)); } /** @@ -251,13 +250,8 @@ private RocksDBStateBackend(RocksDBStateBackend original, Configuration config) originalStreamBackend; // configure incremental checkpoints - if (original.enableIncrementalCheckpointing != null) { - this.enableIncrementalCheckpointing = original.enableIncrementalCheckpointing; - } - else { - this.enableIncrementalCheckpointing = - config.getBoolean(CheckpointingOptions.INCREMENTAL_CHECKPOINTS); - } + this.enableIncrementalCheckpointing = original.enableIncrementalCheckpointing.resolveUndefined( + config.getBoolean(CheckpointingOptions.INCREMENTAL_CHECKPOINTS)); // configure local directories if (original.localRocksDbDirectories != null) { @@ -407,6 +401,9 @@ public AbstractKeyedStateBackend createKeyedStateBackend( File instanceBasePath = new File(getNextStoragePath(), "job-" + jobId + "_op-" + operatorIdentifier + "_uuid-" + UUID.randomUUID()); + LocalRecoveryConfig localRecoveryConfig = + env.getTaskStateManager().createLocalRecoveryConfig(); + return new RocksDBKeyedStateBackend<>( operatorIdentifier, env.getUserClassLoader(), @@ -418,7 +415,8 @@ public AbstractKeyedStateBackend createKeyedStateBackend( numberOfKeyGroups, keyGroupRange, env.getExecutionConfig(), - isIncrementalCheckpointsEnabled()); + isIncrementalCheckpointsEnabled(), + localRecoveryConfig); } @Override @@ -511,12 +509,7 @@ public String[] getDbStoragePaths() { * Gets whether incremental checkpoints are enabled for this state backend. */ public boolean isIncrementalCheckpointsEnabled() { - if (enableIncrementalCheckpointing != null) { - return enableIncrementalCheckpointing; - } - else { - return CheckpointingOptions.INCREMENTAL_CHECKPOINTS.defaultValue(); - } + return enableIncrementalCheckpointing.getOrDefault(CheckpointingOptions.INCREMENTAL_CHECKPOINTS.defaultValue()); } // ------------------------------------------------------------------------ diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java index bae1f815cf25c..9958577647b87 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java @@ -43,8 +43,10 @@ import org.apache.flink.runtime.state.CheckpointedStateScope; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.SnapshotResult; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.TestLocalRecoveryConfig; import org.apache.flink.runtime.state.TestTaskStateManager; import org.apache.flink.runtime.state.VoidNamespace; import org.apache.flink.runtime.state.VoidNamespaceSerializer; @@ -55,6 +57,7 @@ import org.apache.flink.runtime.state.testutils.TestCheckpointStreamFactory; import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.runtime.util.BlockerCheckpointStreamFactory; +import org.apache.flink.runtime.util.BlockingCheckpointOutputStream; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; @@ -75,6 +78,8 @@ import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.modules.junit4.PowerMockRunner; +import javax.annotation.Nullable; + import java.io.File; import java.io.IOException; import java.lang.reflect.Field; @@ -186,7 +191,8 @@ public void declineCheckpoint( TestTaskStateManager taskStateManagerTestMock = new TestTaskStateManager( jobID, executionAttemptID, - checkpointResponderMock); + checkpointResponderMock, + TestLocalRecoveryConfig.disabled()); StreamMockEnvironment mockEnv = new StreamMockEnvironment( testHarness.jobConfig, @@ -256,12 +262,16 @@ public void testCancelFullyAsyncCheckpoints() throws Exception { int count = 1; @Override - public CheckpointStateOutputStream createCheckpointStateOutputStream(CheckpointedStateScope scope) throws Exception { + public CheckpointStateOutputStream createCheckpointStateOutputStream(CheckpointedStateScope scope) throws IOException { // we skip the first created stream, because it is used to checkpoint the timer service, which is // currently not asynchronous. if (count > 0) { --count; - return new MemCheckpointStreamFactory.MemoryCheckpointOutputStream(maxSize); + return new BlockingCheckpointOutputStream( + new MemCheckpointStreamFactory.MemoryCheckpointOutputStream(maxSize), + null, + null, + Integer.MAX_VALUE); } else { return super.createCheckpointStateOutputStream(scope); } @@ -373,7 +383,7 @@ public void testCleanupOfSnapshotsInFailureCase() throws Exception { StringSerializer.INSTANCE, new ValueStateDescriptor<>("foobar", String.class)); - RunnableFuture snapshotFuture = keyedStateBackend.snapshot( + RunnableFuture> snapshotFuture = keyedStateBackend.snapshot( checkpointId, timestamp, new TestCheckpointStreamFactory(() -> outputStream), CheckpointOptions.forCheckpointWithDefaultLocation()); @@ -459,7 +469,7 @@ private static class StaticForwardFactory implements StreamFactory { } @Override - public CheckpointStateOutputStream get() throws Exception { + public CheckpointStateOutputStream get() throws IOException { return factory.createCheckpointStateOutputStream(CheckpointedStateScope.EXCLUSIVE); } } @@ -472,6 +482,7 @@ private static class FailingStream extends CheckpointStateOutputStream { this.testException = testException; } + @Nullable @Override public StreamStateHandle closeAndGetHandle() throws IOException { throw new UnsupportedOperationException(); diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java index 3a39ba0f8b863..2dd67f5c2d9b9 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java @@ -31,6 +31,7 @@ import org.apache.flink.runtime.state.AbstractKeyedStateBackend; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.TestTaskStateManager; import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; @@ -411,6 +412,9 @@ static Environment getMockEnvironment(File[] tempDirs) { TaskManagerRuntimeInfo tmInfo = new TestingTaskManagerRuntimeInfo(new Configuration(), tempDirStrings); when(env.getTaskManagerInfo()).thenReturn(tmInfo); + TestTaskStateManager taskStateManager = new TestTaskStateManager(); + when(env.getTaskStateManager()).thenReturn(taskStateManager); + return env; } } diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java index 54af400082ff0..9466bc3362383 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java @@ -33,9 +33,11 @@ import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.SnapshotResult; import org.apache.flink.runtime.state.StateBackendTestBase; import org.apache.flink.runtime.state.StateHandleID; import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.TestLocalRecoveryConfig; import org.apache.flink.runtime.state.VoidNamespace; import org.apache.flink.runtime.state.VoidNamespaceSerializer; import org.apache.flink.runtime.state.filesystem.FsStateBackend; @@ -231,7 +233,8 @@ public void testCorrectMergeOperatorSet() throws IOException { 1, new KeyGroupRange(0, 0), new ExecutionConfig(), - enableIncrementalCheckpointing); + enableIncrementalCheckpointing, + TestLocalRecoveryConfig.disabled()); verify(columnFamilyOptions, Mockito.times(1)) .setMergeOperatorName(RocksDBKeyedStateBackend.MERGE_OPERATOR_NAME); @@ -249,7 +252,7 @@ public void testReleasingSnapshotAfterBackendClosed() throws Exception { setupRocksKeyedStateBackend(); try { - RunnableFuture snapshot = + RunnableFuture> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); RocksDB spyDB = keyedStateBackend.db; @@ -286,7 +289,7 @@ public void testReleasingSnapshotAfterBackendClosed() throws Exception { public void testDismissingSnapshot() throws Exception { setupRocksKeyedStateBackend(); try { - RunnableFuture snapshot = + RunnableFuture> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); snapshot.cancel(true); verifyRocksObjectsReleased(); @@ -300,7 +303,7 @@ public void testDismissingSnapshot() throws Exception { public void testDismissingSnapshotNotRunnable() throws Exception { setupRocksKeyedStateBackend(); try { - RunnableFuture snapshot = + RunnableFuture> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); snapshot.cancel(true); Thread asyncSnapshotThread = new Thread(snapshot); @@ -323,7 +326,7 @@ public void testDismissingSnapshotNotRunnable() throws Exception { public void testCompletingSnapshot() throws Exception { setupRocksKeyedStateBackend(); try { - RunnableFuture snapshot = + RunnableFuture> snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); Thread asyncSnapshotThread = new Thread(snapshot); asyncSnapshotThread.start(); @@ -332,7 +335,9 @@ public void testCompletingSnapshot() throws Exception { runStateUpdates(); blocker.trigger(); // allow checkpointing to start writing waiter.await(); // wait for snapshot stream writing to run - KeyedStateHandle keyedStateHandle = snapshot.get(); + + SnapshotResult snapshotResult = snapshot.get(); + KeyedStateHandle keyedStateHandle = snapshotResult.getJobManagerOwnedSnapshot(); assertNotNull(keyedStateHandle); assertTrue(keyedStateHandle.getStateSize() > 0); assertEquals(2, keyedStateHandle.getKeyGroupRange().getNumberOfKeyGroups()); @@ -349,7 +354,8 @@ public void testCompletingSnapshot() throws Exception { public void testCancelRunningSnapshot() throws Exception { setupRocksKeyedStateBackend(); try { - RunnableFuture snapshot = keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); + RunnableFuture> snapshot = + keyedStateBackend.snapshot(0L, 0L, testStreamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); Thread asyncSnapshotThread = new Thread(snapshot); asyncSnapshotThread.start(); waiter.await(); // wait for snapshot to run @@ -425,7 +431,7 @@ public void testSharedIncrementalStateDeRegistration() throws Exception { backend.setCurrentKey(checkpointId); state.update("Hello-" + checkpointId); - RunnableFuture snapshot = backend.snapshot( + RunnableFuture> snapshot = backend.snapshot( checkpointId, checkpointId, createStreamFactory(), @@ -433,7 +439,11 @@ public void testSharedIncrementalStateDeRegistration() throws Exception { snapshot.run(); - IncrementalKeyedStateHandle stateHandle = (IncrementalKeyedStateHandle) snapshot.get(); + SnapshotResult snapshotResult = snapshot.get(); + + IncrementalKeyedStateHandle stateHandle = + (IncrementalKeyedStateHandle) snapshotResult.getJobManagerOwnedSnapshot(); + Map sharedState = new HashMap<>(stateHandle.getSharedState()); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index 4f162591ab3fb..4d3f9f57fc77c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -347,10 +347,7 @@ public void dispose() throws Exception { } @Override - public final OperatorSnapshotFutures snapshotState( - long checkpointId, - long timestamp, - CheckpointOptions checkpointOptions, + public final OperatorSnapshotFutures snapshotState(long checkpointId, long timestamp, CheckpointOptions checkpointOptions, CheckpointStreamFactory factory) throws Exception { KeyGroupRange keyGroupRange = null != keyedStateBackend ? diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedure.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedure.java new file mode 100644 index 0000000000000..ba27a0a27dd4d --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedure.java @@ -0,0 +1,146 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.runtime.state.Snapshotable; +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.util.Disposable; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.function.SupplierWithException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; + +/** + * This class implements the logic that creates (and potentially restores) a state backend. The restore logic + * considers multiple, prioritized options of snapshots to restore from, where all of the options should recreate + * the same state for the backend. When we fail to restore from the snapshot with the highest priority (typically + * the "fastest" to restore), we fallback to the next snapshot with the next highest priority. We also take care + * of cleaning up from failed restore attempts. We only reattempt when the problem occurs during the restore call + * and will only stop after all snapshot alternatives are exhausted and all failed. + * + * @param type of the restored backend. + * @param type of the supplied snapshots from which the backend restores. + */ +public class BackendRestorerProcedure< + T extends Closeable & Disposable & Snapshotable>, + S extends StateObject> { + + /** Logger for this class. */ + private static final Logger LOG = LoggerFactory.getLogger(BackendRestorerProcedure.class); + + /** Factory for new, fresh backends without state. */ + private final SupplierWithException instanceSupplier; + + /** This registry is used so that recovery can participate in the task lifecycle, i.e. can be canceled. */ + private final CloseableRegistry backendCloseableRegistry; + + /** + * Creates a new backend restorer using the given backend supplier and the closeable registry. + * + * @param instanceSupplier factory function for new, empty backend instances. + * @param backendCloseableRegistry registry to allow participation in task lifecycle, e.g. react to cancel. + */ + public BackendRestorerProcedure( + @Nonnull SupplierWithException instanceSupplier, + @Nonnull CloseableRegistry backendCloseableRegistry) { + + this.instanceSupplier = Preconditions.checkNotNull(instanceSupplier); + this.backendCloseableRegistry = Preconditions.checkNotNull(backendCloseableRegistry); + } + + /** + * Creates a new state backend and restores it from the provided set of state snapshot alternatives. + * + * @param restoreOptions iterator over a prioritized set of state snapshot alternatives for recovery. + * @return the created (and restored) state backend. + * @throws Exception if the backend could not be created or restored. + */ + public @Nonnull + T createAndRestore(@Nonnull Iterator> restoreOptions) throws Exception { + + // This ensures that we always call the restore method even if there is no previous state + // (required by some backends). + Collection attemptState = restoreOptions.hasNext() ? + restoreOptions.next() : + Collections.emptyList(); + + while (true) { + try { + return attemptCreateAndRestore(attemptState); + } catch (Exception ex) { + // more attempts? + if (restoreOptions.hasNext()) { + + attemptState = restoreOptions.next(); + LOG.warn("Exception while restoring backend, will retry with another snapshot replica.", ex); + } else { + + throw new FlinkException("Could not restore from any of the provided restore options.", ex); + } + } + } + } + + private T attemptCreateAndRestore(Collection restoreState) throws Exception { + + // create a new, empty backend. + final T backendInstance = instanceSupplier.get(); + + try { + // register the backend with the registry to participate in task lifecycle w.r.t. cancellation. + backendCloseableRegistry.registerCloseable(backendInstance); + + // attempt to restore from snapshot (or null if no state was checkpointed). + backendInstance.restore(restoreState); + + return backendInstance; + } catch (Exception ex) { + + // under failure, we need do close... + if (backendCloseableRegistry.unregisterCloseable(backendInstance)) { + try { + backendInstance.close(); + } catch (IOException closeEx) { + ex = ExceptionUtils.firstOrSuppressed(closeEx, ex); + } + } + + // ... and dispose, e.g. to release native resources. + try { + backendInstance.dispose(); + } catch (Exception disposeEx) { + ex = ExceptionUtils.firstOrSuppressed(disposeEx, ex); + } + + throw ex; + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFinalizer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFinalizer.java new file mode 100644 index 0000000000000..7a93990744d85 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFinalizer.java @@ -0,0 +1,80 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.SnapshotResult; +import org.apache.flink.util.FutureUtil; + +import javax.annotation.Nonnull; + +import java.util.concurrent.ExecutionException; + +/** + * This class finalizes {@link OperatorSnapshotFutures}. Each object is created with a {@link OperatorSnapshotFutures} + * that is executed. The object can then deliver the results from the execution as {@link OperatorSubtaskState}. + */ +public class OperatorSnapshotFinalizer { + + /** Primary replica of the operator subtask state for report to JM. */ + private final OperatorSubtaskState jobManagerOwnedState; + + /** Secondary replica of the operator subtask state for faster, local recovery on TM. */ + private final OperatorSubtaskState taskLocalState; + + public OperatorSnapshotFinalizer( + @Nonnull OperatorSnapshotFutures snapshotFutures) throws ExecutionException, InterruptedException { + + SnapshotResult keyedManaged = + FutureUtil.runIfNotDoneAndGet(snapshotFutures.getKeyedStateManagedFuture()); + + SnapshotResult keyedRaw = + FutureUtil.runIfNotDoneAndGet(snapshotFutures.getKeyedStateRawFuture()); + + SnapshotResult operatorManaged = + FutureUtil.runIfNotDoneAndGet(snapshotFutures.getOperatorStateManagedFuture()); + + SnapshotResult operatorRaw = + FutureUtil.runIfNotDoneAndGet(snapshotFutures.getOperatorStateRawFuture()); + + jobManagerOwnedState = new OperatorSubtaskState( + operatorManaged.getJobManagerOwnedSnapshot(), + operatorRaw.getJobManagerOwnedSnapshot(), + keyedManaged.getJobManagerOwnedSnapshot(), + keyedRaw.getJobManagerOwnedSnapshot() + ); + + taskLocalState = new OperatorSubtaskState( + operatorManaged.getTaskLocalSnapshot(), + operatorRaw.getTaskLocalSnapshot(), + keyedManaged.getTaskLocalSnapshot(), + keyedRaw.getTaskLocalSnapshot() + ); + } + + public OperatorSubtaskState getTaskLocalState() { + return taskLocalState; + } + + public OperatorSubtaskState getJobManagerOwnedState() { + return jobManagerOwnedState; + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFutures.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFutures.java index bdaf64b37e05b..95eb213968fe8 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFutures.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFutures.java @@ -18,11 +18,15 @@ package org.apache.flink.streaming.api.operators; +import org.apache.flink.runtime.state.DoneFuture; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.SnapshotResult; import org.apache.flink.runtime.state.StateUtil; import org.apache.flink.util.ExceptionUtils; +import javax.annotation.Nonnull; + import java.util.concurrent.RunnableFuture; /** @@ -30,55 +34,74 @@ */ public class OperatorSnapshotFutures { - private RunnableFuture keyedStateManagedFuture; - private RunnableFuture keyedStateRawFuture; - private RunnableFuture operatorStateManagedFuture; - private RunnableFuture operatorStateRawFuture; + @Nonnull + private RunnableFuture> keyedStateManagedFuture; + + @Nonnull + private RunnableFuture> keyedStateRawFuture; + + @Nonnull + private RunnableFuture> operatorStateManagedFuture; + + @Nonnull + private RunnableFuture> operatorStateRawFuture; public OperatorSnapshotFutures() { - this(null, null, null, null); + this( + DoneFuture.of(SnapshotResult.empty()), + DoneFuture.of(SnapshotResult.empty()), + DoneFuture.of(SnapshotResult.empty()), + DoneFuture.of(SnapshotResult.empty())); } public OperatorSnapshotFutures( - RunnableFuture keyedStateManagedFuture, - RunnableFuture keyedStateRawFuture, - RunnableFuture operatorStateManagedFuture, - RunnableFuture operatorStateRawFuture) { + @Nonnull RunnableFuture> keyedStateManagedFuture, + @Nonnull RunnableFuture> keyedStateRawFuture, + @Nonnull RunnableFuture> operatorStateManagedFuture, + @Nonnull RunnableFuture> operatorStateRawFuture) { this.keyedStateManagedFuture = keyedStateManagedFuture; this.keyedStateRawFuture = keyedStateRawFuture; this.operatorStateManagedFuture = operatorStateManagedFuture; this.operatorStateRawFuture = operatorStateRawFuture; } - public RunnableFuture getKeyedStateManagedFuture() { + @Nonnull + public RunnableFuture> getKeyedStateManagedFuture() { return keyedStateManagedFuture; } - public void setKeyedStateManagedFuture(RunnableFuture keyedStateManagedFuture) { + public void setKeyedStateManagedFuture( + @Nonnull RunnableFuture> keyedStateManagedFuture) { this.keyedStateManagedFuture = keyedStateManagedFuture; } - public RunnableFuture getKeyedStateRawFuture() { + @Nonnull + public RunnableFuture> getKeyedStateRawFuture() { return keyedStateRawFuture; } - public void setKeyedStateRawFuture(RunnableFuture keyedStateRawFuture) { + public void setKeyedStateRawFuture( + @Nonnull RunnableFuture> keyedStateRawFuture) { this.keyedStateRawFuture = keyedStateRawFuture; } - public RunnableFuture getOperatorStateManagedFuture() { + @Nonnull + public RunnableFuture> getOperatorStateManagedFuture() { return operatorStateManagedFuture; } - public void setOperatorStateManagedFuture(RunnableFuture operatorStateManagedFuture) { + public void setOperatorStateManagedFuture( + @Nonnull RunnableFuture> operatorStateManagedFuture) { this.operatorStateManagedFuture = operatorStateManagedFuture; } - public RunnableFuture getOperatorStateRawFuture() { + @Nonnull + public RunnableFuture> getOperatorStateRawFuture() { return operatorStateRawFuture; } - public void setOperatorStateRawFuture(RunnableFuture operatorStateRawFuture) { + public void setOperatorStateRawFuture( + @Nonnull RunnableFuture> operatorStateRawFuture) { this.operatorStateRawFuture = operatorStateRawFuture; } @@ -119,16 +142,4 @@ public void cancel() throws Exception { throw exception; } } - - public boolean hasKeyedState() { - return keyedStateManagedFuture != null || keyedStateRawFuture != null; - } - - public boolean hasOperatorState() { - return operatorStateManagedFuture != null || operatorStateRawFuture != null; - } - - public boolean hasState() { - return hasKeyedState() || hasOperatorState(); - } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java index c3254f6ff005b..71f508b03c2da 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java @@ -26,6 +26,7 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.util.Disposable; import java.io.Serializable; @@ -45,7 +46,7 @@ * @param The output type of the operator */ @PublicEvolving -public interface StreamOperator extends CheckpointListener, KeyContext, Serializable { +public interface StreamOperator extends CheckpointListener, KeyContext, Disposable, Serializable { // ------------------------------------------------------------------------ // life cycle @@ -85,6 +86,7 @@ public interface StreamOperator extends CheckpointListener, KeyContext, Ser *

This method is expected to make a thorough effort to release all resources * that the operator has acquired. */ + @Override void dispose() throws Exception; // ------------------------------------------------------------------------ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateContext.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateContext.java index 75ead44630ea3..420b6bf9d33c9 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateContext.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateContext.java @@ -40,7 +40,6 @@ public interface StreamOperatorStateContext { */ OperatorStateBackend operatorStateBackend(); - /** * Returns the keyed state backend for the stream operator. This method returns null for non-keyed operators. */ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java index 9887e45e677f8..11e2dda82ea72 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java @@ -23,7 +23,8 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.core.fs.FSDataInputStream; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.PrioritizedOperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.AbstractKeyedStateBackend; @@ -116,10 +117,8 @@ public StreamOperatorStateContext streamOperatorStateContext( final String operatorIdentifierText = operatorSubtaskDescription.toString(); - final OperatorSubtaskState operatorSubtaskStateFromJobManager = - taskStateManager.operatorStates(operatorID); - - final boolean restoring = (operatorSubtaskStateFromJobManager != null); + final PrioritizedOperatorSubtaskState prioritizedOperatorSubtaskStates = + taskStateManager.prioritizedOperatorState(operatorID); AbstractKeyedStateBackend keyedStatedBackend = null; OperatorStateBackend operatorStateBackend = null; @@ -133,20 +132,22 @@ public StreamOperatorStateContext streamOperatorStateContext( keyedStatedBackend = keyedStatedBackend( keySerializer, operatorIdentifierText, - operatorSubtaskStateFromJobManager, + prioritizedOperatorSubtaskStates, streamTaskCloseableRegistry); // -------------- Operator State Backend -------------- operatorStateBackend = operatorStateBackend( operatorIdentifierText, - operatorSubtaskStateFromJobManager, + prioritizedOperatorSubtaskStates, streamTaskCloseableRegistry); // -------------- Raw State Streams -------------- - rawKeyedStateInputs = rawKeyedStateInputs(operatorSubtaskStateFromJobManager); + rawKeyedStateInputs = rawKeyedStateInputs( + prioritizedOperatorSubtaskStates.getPrioritizedRawKeyedState()); streamTaskCloseableRegistry.registerCloseable(rawKeyedStateInputs); - rawOperatorStateInputs = rawOperatorStateInputs(operatorSubtaskStateFromJobManager); + rawOperatorStateInputs = rawOperatorStateInputs( + prioritizedOperatorSubtaskStates.getPrioritizedRawOperatorState()); streamTaskCloseableRegistry.registerCloseable(rawOperatorStateInputs); // -------------- Internal Timer Service Manager -------------- @@ -155,7 +156,7 @@ public StreamOperatorStateContext streamOperatorStateContext( // -------------- Preparing return value -------------- return new StreamOperatorStateContextImpl( - restoring, + prioritizedOperatorSubtaskStates.isRestored(), operatorStateBackend, keyedStatedBackend, timeServiceManager, @@ -222,155 +223,114 @@ protected InternalTimeServiceManager internalTimeServiceManager( protected OperatorStateBackend operatorStateBackend( String operatorIdentifierText, - OperatorSubtaskState operatorSubtaskStateFromJobManager, + PrioritizedOperatorSubtaskState prioritizedOperatorSubtaskStates, CloseableRegistry backendCloseableRegistry) throws Exception { - //TODO search in local state for a local recovery opportunity. + BackendRestorerProcedure backendRestorer = + new BackendRestorerProcedure<>( + () -> stateBackend.createOperatorStateBackend(environment, operatorIdentifierText), + backendCloseableRegistry); - return createOperatorStateBackendFromJobManagerState( - operatorIdentifierText, - operatorSubtaskStateFromJobManager, - backendCloseableRegistry); + return backendRestorer.createAndRestore(prioritizedOperatorSubtaskStates.getPrioritizedManagedOperatorState()); } protected AbstractKeyedStateBackend keyedStatedBackend( TypeSerializer keySerializer, String operatorIdentifierText, - OperatorSubtaskState operatorSubtaskStateFromJobManager, + PrioritizedOperatorSubtaskState prioritizedOperatorSubtaskStates, CloseableRegistry backendCloseableRegistry) throws Exception { if (keySerializer == null) { return null; } - //TODO search in local state for a local recovery opportunity. + TaskInfo taskInfo = environment.getTaskInfo(); + + final KeyGroupRange keyGroupRange = KeyGroupRangeAssignment.computeKeyGroupRangeForOperatorIndex( + taskInfo.getMaxNumberOfParallelSubtasks(), + taskInfo.getNumberOfParallelSubtasks(), + taskInfo.getIndexOfThisSubtask()); - return createKeyedStatedBackendFromJobManagerState( - keySerializer, - operatorIdentifierText, - operatorSubtaskStateFromJobManager, - backendCloseableRegistry); + BackendRestorerProcedure, KeyedStateHandle> backendRestorer = + new BackendRestorerProcedure<>( + () -> stateBackend.createKeyedStateBackend( + environment, + environment.getJobID(), + operatorIdentifierText, + keySerializer, + taskInfo.getMaxNumberOfParallelSubtasks(), + keyGroupRange, + environment.getTaskKvStateRegistry()), + backendCloseableRegistry); + + return backendRestorer.createAndRestore(prioritizedOperatorSubtaskStates.getPrioritizedManagedKeyedState()); } protected CloseableIterable rawOperatorStateInputs( - OperatorSubtaskState operatorSubtaskStateFromJobManager) { + Iterator> restoreStateAlternatives) { - if (operatorSubtaskStateFromJobManager != null) { + if (restoreStateAlternatives.hasNext()) { final CloseableRegistry closeableRegistry = new CloseableRegistry(); - Collection rawOperatorState = - operatorSubtaskStateFromJobManager.getRawOperatorState(); - - return new CloseableIterable() { - @Override - public void close() throws IOException { - closeableRegistry.close(); - } - - @Nonnull - @Override - public Iterator iterator() { - return new OperatorStateStreamIterator( - DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME, - rawOperatorState.iterator(), closeableRegistry); - } - }; - } - - return CloseableIterable.empty(); - } - - protected CloseableIterable rawKeyedStateInputs( - OperatorSubtaskState operatorSubtaskStateFromJobManager) { + Collection rawOperatorState = restoreStateAlternatives.next(); + // TODO currently this does not support local state recovery, so we expect there is only one handle. + Preconditions.checkState( + !restoreStateAlternatives.hasNext(), + "Local recovery is currently not implemented for raw operator state, but found state alternative."); - if (operatorSubtaskStateFromJobManager != null) { + if (rawOperatorState != null) { - Collection rawKeyedState = operatorSubtaskStateFromJobManager.getRawKeyedState(); - Collection keyGroupsStateHandles = transform(rawKeyedState); - final CloseableRegistry closeableRegistry = new CloseableRegistry(); - - return new CloseableIterable() { - @Override - public void close() throws IOException { - closeableRegistry.close(); - } + return new CloseableIterable() { + @Override + public void close() throws IOException { + closeableRegistry.close(); + } - @Override - public Iterator iterator() { - return new KeyGroupStreamIterator(keyGroupsStateHandles.iterator(), closeableRegistry); - } - }; + @Nonnull + @Override + public Iterator iterator() { + return new OperatorStateStreamIterator( + DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME, + rawOperatorState.iterator(), closeableRegistry); + } + }; + } } return CloseableIterable.empty(); } - // ================================================================================================================= - - private OperatorStateBackend createOperatorStateBackendFromJobManagerState( - String operatorIdentifierText, - OperatorSubtaskState operatorSubtaskStateFromJobManager, - CloseableRegistry backendCloseableRegistry) throws Exception { - - final OperatorStateBackend operatorStateBackend = - stateBackend.createOperatorStateBackend(environment, operatorIdentifierText); - - backendCloseableRegistry.registerCloseable(operatorStateBackend); - - Collection managedOperatorState = null; - - if (operatorSubtaskStateFromJobManager != null) { - managedOperatorState = operatorSubtaskStateFromJobManager.getManagedOperatorState(); - } - - operatorStateBackend.restore(managedOperatorState); - - return operatorStateBackend; - } + protected CloseableIterable rawKeyedStateInputs( + Iterator> restoreStateAlternatives) { - private AbstractKeyedStateBackend createKeyedStatedBackendFromJobManagerState( - TypeSerializer keySerializer, - String operatorIdentifierText, - OperatorSubtaskState operatorSubtaskStateFromJobManager, - CloseableRegistry backendCloseableRegistry) throws Exception { + if (restoreStateAlternatives.hasNext()) { + Collection rawKeyedState = restoreStateAlternatives.next(); - final AbstractKeyedStateBackend keyedStateBackend = createKeyedStateBackend( - operatorIdentifierText, - keySerializer); + // TODO currently this does not support local state recovery, so we expect there is only one handle. + Preconditions.checkState( + !restoreStateAlternatives.hasNext(), + "Local recovery is currently not implemented for raw keyed state, but found state alternative."); - backendCloseableRegistry.registerCloseable(keyedStateBackend); + if (rawKeyedState != null) { + Collection keyGroupsStateHandles = transform(rawKeyedState); + final CloseableRegistry closeableRegistry = new CloseableRegistry(); - Collection managedKeyedState = null; + return new CloseableIterable() { + @Override + public void close() throws IOException { + closeableRegistry.close(); + } - if (operatorSubtaskStateFromJobManager != null) { - managedKeyedState = operatorSubtaskStateFromJobManager.getManagedKeyedState(); + @Override + public Iterator iterator() { + return new KeyGroupStreamIterator(keyGroupsStateHandles.iterator(), closeableRegistry); + } + }; + } } - keyedStateBackend.restore(managedKeyedState); - - return keyedStateBackend; - } - - private AbstractKeyedStateBackend createKeyedStateBackend( - String operatorIdentifier, - TypeSerializer keySerializer) throws Exception { - - TaskInfo taskInfo = environment.getTaskInfo(); - - final KeyGroupRange keyGroupRange = KeyGroupRangeAssignment.computeKeyGroupRangeForOperatorIndex( - taskInfo.getMaxNumberOfParallelSubtasks(), - taskInfo.getNumberOfParallelSubtasks(), - taskInfo.getIndexOfThisSubtask()); - - return stateBackend.createKeyedStateBackend( - environment, - environment.getJobID(), - operatorIdentifier, - keySerializer, - taskInfo.getMaxNumberOfParallelSubtasks(), //TODO check: this is numberOfKeyGroups !!!! - keyGroupRange, - environment.getTaskKvStateRegistry()); + return CloseableIterable.empty(); } // ================================================================================================================= diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 06cb18bedddf5..dba4c87ec80c1 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -27,7 +27,6 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointOptions; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.execution.CancelTaskException; import org.apache.flink.runtime.execution.Environment; @@ -46,6 +45,7 @@ import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.graph.StreamEdge; +import org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; @@ -58,7 +58,6 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.FutureUtil; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; @@ -800,7 +799,11 @@ public String toString() { // ------------------------------------------------------------------------ - private static final class AsyncCheckpointRunnable implements Runnable, Closeable { + /** + * This runnable executes the asynchronous parts of all involved backend snapshots for the subtask. + */ + @VisibleForTesting + protected static final class AsyncCheckpointRunnable implements Runnable, Closeable { private final StreamTask owner; @@ -831,11 +834,12 @@ private static final class AsyncCheckpointRunnable implements Runnable, Closeabl @Override public void run() { FileSystemSafetyNet.initializeSafetyNetForThread(); - final long checkpointId = checkpointMetaData.getCheckpointId(); try { - boolean hasState = false; - final TaskStateSnapshot taskOperatorSubtaskStates = + TaskStateSnapshot jobManagerTaskOperatorSubtaskStates = + new TaskStateSnapshot(operatorSnapshotsInProgress.size()); + + TaskStateSnapshot localTaskOperatorSubtaskStates = new TaskStateSnapshot(operatorSnapshotsInProgress.size()); for (Map.Entry entry : operatorSnapshotsInProgress.entrySet()) { @@ -843,15 +847,17 @@ public void run() { OperatorID operatorID = entry.getKey(); OperatorSnapshotFutures snapshotInProgress = entry.getValue(); - OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState( - FutureUtil.runIfNotDoneAndGet(snapshotInProgress.getOperatorStateManagedFuture()), - FutureUtil.runIfNotDoneAndGet(snapshotInProgress.getOperatorStateRawFuture()), - FutureUtil.runIfNotDoneAndGet(snapshotInProgress.getKeyedStateManagedFuture()), - FutureUtil.runIfNotDoneAndGet(snapshotInProgress.getKeyedStateRawFuture()) - ); + // finalize the async part of all by executing all snapshot runnables + OperatorSnapshotFinalizer finalizedSnapshots = + new OperatorSnapshotFinalizer(snapshotInProgress); - hasState |= operatorSubtaskState.hasState(); - taskOperatorSubtaskStates.putSubtaskStateByOperatorID(operatorID, operatorSubtaskState); + jobManagerTaskOperatorSubtaskStates.putSubtaskStateByOperatorID( + operatorID, + finalizedSnapshots.getJobManagerOwnedState()); + + localTaskOperatorSubtaskStates.putSubtaskStateByOperatorID( + operatorID, + finalizedSnapshots.getTaskLocalState()); } final long asyncEndNanos = System.nanoTime(); @@ -862,23 +868,10 @@ public void run() { if (asyncCheckpointState.compareAndSet(CheckpointingOperation.AsynCheckpointState.RUNNING, CheckpointingOperation.AsynCheckpointState.COMPLETED)) { - TaskStateSnapshot acknowledgedState = hasState ? taskOperatorSubtaskStates : null; - - TaskStateManager taskStateManager = owner.getEnvironment().getTaskStateManager(); - - // we signal stateless tasks by reporting null, so that there are no attempts to assign empty state - // to stateless tasks on restore. This enables simple job modifications that only concern - // stateless without the need to assign them uids to match their (always empty) states. - taskStateManager.reportStateHandles( - checkpointMetaData, - checkpointMetrics, - acknowledgedState); - - LOG.debug("{} - finished asynchronous part of checkpoint {}. Asynchronous duration: {} ms", - owner.getName(), checkpointMetaData.getCheckpointId(), asyncDurationMillis); - - LOG.trace("{} - reported the following states in snapshot for checkpoint {}: {}.", - owner.getName(), checkpointMetaData.getCheckpointId(), acknowledgedState); + reportCompletedSnapshotStates( + jobManagerTaskOperatorSubtaskStates, + localTaskOperatorSubtaskStates, + asyncDurationMillis); } else { LOG.debug("{} - asynchronous part of checkpoint {} could not be completed because it was closed before.", @@ -886,32 +879,66 @@ public void run() { checkpointMetaData.getCheckpointId()); } } catch (Exception e) { - // the state is completed if an exception occurred in the acknowledgeCheckpoint call - // in order to clean up, we have to set it to RUNNING again. - asyncCheckpointState.compareAndSet( - CheckpointingOperation.AsynCheckpointState.COMPLETED, - CheckpointingOperation.AsynCheckpointState.RUNNING); - - try { - cleanup(); - } catch (Exception cleanupException) { - e.addSuppressed(cleanupException); - } - - Exception checkpointException = new Exception( - "Could not materialize checkpoint " + checkpointId + " for operator " + - owner.getName() + '.', - e); - - owner.asynchronousCheckpointExceptionHandler.tryHandleCheckpointException( - checkpointMetaData, - checkpointException); + handleExecutionException(e); } finally { owner.cancelables.unregisterCloseable(this); FileSystemSafetyNet.closeSafetyNetAndGuardedResourcesForThread(); } } + private void reportCompletedSnapshotStates( + TaskStateSnapshot acknowledgedTaskStateSnapshot, + TaskStateSnapshot localTaskStateSnapshot, + long asyncDurationMillis) { + + TaskStateManager taskStateManager = owner.getEnvironment().getTaskStateManager(); + + boolean hasAckState = acknowledgedTaskStateSnapshot.hasState(); + boolean hasLocalState = localTaskStateSnapshot.hasState(); + + Preconditions.checkState(hasAckState || !hasLocalState, + "Found cached state but no corresponding primary state is reported to the job " + + "manager. This indicates a problem."); + + // we signal stateless tasks by reporting null, so that there are no attempts to assign empty state + // to stateless tasks on restore. This enables simple job modifications that only concern + // stateless without the need to assign them uids to match their (always empty) states. + taskStateManager.reportTaskStateSnapshots( + checkpointMetaData, + checkpointMetrics, + hasAckState ? acknowledgedTaskStateSnapshot : null, + hasLocalState ? localTaskStateSnapshot : null); + + LOG.debug("{} - finished asynchronous part of checkpoint {}. Asynchronous duration: {} ms", + owner.getName(), checkpointMetaData.getCheckpointId(), asyncDurationMillis); + + LOG.trace("{} - reported the following states in snapshot for checkpoint {}: {}.", + owner.getName(), checkpointMetaData.getCheckpointId(), acknowledgedTaskStateSnapshot); + } + + private void handleExecutionException(Exception e) { + // the state is completed if an exception occurred in the acknowledgeCheckpoint call + // in order to clean up, we have to set it to RUNNING again. + asyncCheckpointState.compareAndSet( + CheckpointingOperation.AsynCheckpointState.COMPLETED, + CheckpointingOperation.AsynCheckpointState.RUNNING); + + try { + cleanup(); + } catch (Exception cleanupException) { + e.addSuppressed(cleanupException); + } + + Exception checkpointException = new Exception( + "Could not materialize checkpoint " + checkpointMetaData.getCheckpointId() + " for operator " + + owner.getName() + '.', + e); + + owner.asynchronousCheckpointExceptionHandler.tryHandleCheckpointException( + checkpointMetaData, + checkpointException); + } + @Override public void close() { try { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java index 85069b5773e1a..904ff64025c14 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateBackend; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.SnapshotResult; import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl; import org.apache.flink.runtime.state.VoidNamespace; import org.apache.flink.runtime.state.VoidNamespaceSerializer; @@ -569,8 +570,8 @@ public void testFailingBackendSnapshotMethod() throws Exception { final CloseableRegistry closeableRegistry = new CloseableRegistry(); - RunnableFuture futureKeyedStateHandle = mock(RunnableFuture.class); - RunnableFuture futureOperatorStateHandle = mock(RunnableFuture.class); + RunnableFuture> futureKeyedStateHandle = mock(RunnableFuture.class); + RunnableFuture> futureOperatorStateHandle = mock(RunnableFuture.class); StateSnapshotContextSynchronousImpl context = mock(StateSnapshotContextSynchronousImpl.class); when(context.getKeyedStateStreamFuture()).thenReturn(futureKeyedStateHandle); @@ -593,7 +594,7 @@ public void testFailingBackendSnapshotMethod() throws Exception { doReturn(containingTask).when(operator).getContainingTask(); - RunnableFuture futureManagedOperatorStateHandle = mock(RunnableFuture.class); + RunnableFuture> futureManagedOperatorStateHandle = mock(RunnableFuture.class); OperatorStateBackend operatorStateBackend = mock(OperatorStateBackend.class); when(operatorStateBackend.snapshot( diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedureTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedureTest.java new file mode 100644 index 0000000000000..2126f707a621e --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedureTest.java @@ -0,0 +1,206 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; +import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.runtime.state.DefaultOperatorStateBackend; +import org.apache.flink.runtime.state.OperatorStateBackend; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.SnapshotResult; +import org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory; +import org.apache.flink.runtime.util.BlockingFSDataInputStream; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.SupplierWithException; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.RunnableFuture; +import java.util.concurrent.atomic.AtomicReference; + +import static org.mockito.Mockito.verify; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.spy; +import static org.powermock.api.mockito.PowerMockito.verifyZeroInteractions; +import static org.powermock.api.mockito.PowerMockito.when; + +/** + * Tests for {@link BackendRestorerProcedure}. + */ +public class BackendRestorerProcedureTest extends TestLogger { + + private final SupplierWithException backendSupplier = + () -> new DefaultOperatorStateBackend( + getClass().getClassLoader(), + new ExecutionConfig(), + true); + + /** + * Tests that the restore procedure follows the order of the iterator and will retries failed attempts if there are + * more options. + */ + @Test + public void testRestoreProcedureOrderAndFailure() throws Exception { + + CloseableRegistry closeableRegistry = new CloseableRegistry(); + CheckpointStreamFactory checkpointStreamFactory = new MemCheckpointStreamFactory(1024); + + ListStateDescriptor stateDescriptor = new ListStateDescriptor<>("test-state", Integer.class); + OperatorStateBackend originalBackend = backendSupplier.get(); + SnapshotResult snapshotResult; + + try { + ListState listState = originalBackend.getListState(stateDescriptor); + + listState.add(0); + listState.add(1); + listState.add(2); + listState.add(3); + + RunnableFuture> snapshot = + originalBackend.snapshot(0L, 0L, checkpointStreamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()); + + snapshot.run(); + snapshotResult = snapshot.get(); + + } finally { + originalBackend.close(); + originalBackend.dispose(); + } + + OperatorStateHandle firstFailHandle = mock(OperatorStateHandle.class); + OperatorStateHandle secondSuccessHandle = spy(snapshotResult.getJobManagerOwnedSnapshot()); + OperatorStateHandle thirdNotUsedHandle = mock(OperatorStateHandle.class); + + List> sortedRestoreOptions = Arrays.asList( + new StateObjectCollection<>(Collections.singletonList(firstFailHandle)), + new StateObjectCollection<>(Collections.singletonList(secondSuccessHandle)), + new StateObjectCollection<>(Collections.singletonList(thirdNotUsedHandle))); + Iterator> iterator = sortedRestoreOptions.iterator(); + + BackendRestorerProcedure restorerProcedure = + new BackendRestorerProcedure<>(backendSupplier, closeableRegistry); + + OperatorStateBackend restoredBackend = restorerProcedure.createAndRestore(iterator); + Assert.assertNotNull(restoredBackend); + + try { + Assert.assertTrue(iterator.hasNext()); + Assert.assertTrue(thirdNotUsedHandle == iterator.next().iterator().next()); + verify(firstFailHandle).openInputStream(); + verify(secondSuccessHandle).openInputStream(); + verifyZeroInteractions(thirdNotUsedHandle); + Assert.assertFalse(iterator.hasNext()); + + ListState listState = restoredBackend.getListState(stateDescriptor); + + Iterator stateIterator = listState.get().iterator(); + Assert.assertEquals(0, (int) stateIterator.next()); + Assert.assertEquals(1, (int) stateIterator.next()); + Assert.assertEquals(2, (int) stateIterator.next()); + Assert.assertEquals(3, (int) stateIterator.next()); + Assert.assertFalse(stateIterator.hasNext()); + + } finally { + restoredBackend.close(); + restoredBackend.dispose(); + } + } + + /** + * Tests if there is an exception if all restore attempts are exhausted and failed. + */ + @Test + public void testExceptionThrownIfAllRestoresFailed() throws Exception { + + CloseableRegistry closeableRegistry = new CloseableRegistry(); + + OperatorStateHandle firstFailHandle = mock(OperatorStateHandle.class); + OperatorStateHandle secondFailHandle = mock(OperatorStateHandle.class); + OperatorStateHandle thirdFailHandle = mock(OperatorStateHandle.class); + + List> sortedRestoreOptions = Arrays.asList( + new StateObjectCollection<>(Collections.singletonList(firstFailHandle)), + new StateObjectCollection<>(Collections.singletonList(secondFailHandle)), + new StateObjectCollection<>(Collections.singletonList(thirdFailHandle))); + Iterator> iterator = sortedRestoreOptions.iterator(); + + BackendRestorerProcedure restorerProcedure = + new BackendRestorerProcedure<>(backendSupplier, closeableRegistry); + + try { + restorerProcedure.createAndRestore(iterator); + Assert.fail(); + } catch (Exception ignore) { + } + + verify(firstFailHandle).openInputStream(); + verify(secondFailHandle).openInputStream(); + verify(thirdFailHandle).openInputStream(); + Assert.assertFalse(iterator.hasNext()); + } + + /** + * Test that the restore can be stopped via the provided closeable registry. + */ + @Test + public void testCanBeCanceledViaRegistry() throws Exception { + CloseableRegistry closeableRegistry = new CloseableRegistry(); + OneShotLatch waitForBlock = new OneShotLatch(); + OneShotLatch unblock = new OneShotLatch(); + OperatorStateHandle blockingRestoreHandle = mock(OperatorStateHandle.class); + when(blockingRestoreHandle.openInputStream()).thenReturn(new BlockingFSDataInputStream(waitForBlock, unblock)); + + List> sortedRestoreOptions = + Collections.singletonList(new StateObjectCollection<>(Collections.singletonList(blockingRestoreHandle))); + + BackendRestorerProcedure restorerProcedure = + new BackendRestorerProcedure<>(backendSupplier, closeableRegistry); + + AtomicReference exceptionReference = new AtomicReference<>(null); + Thread restoreThread = new Thread(() -> { + try { + restorerProcedure.createAndRestore(sortedRestoreOptions.iterator()); + } catch (Exception e) { + exceptionReference.set(e); + } + }); + + restoreThread.start(); + waitForBlock.await(); + closeableRegistry.close(); + unblock.trigger(); + restoreThread.join(); + + Exception exception = exceptionReference.get(); + Assert.assertTrue(exception instanceof FlinkException); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFinalizerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFinalizerTest.java new file mode 100644 index 0000000000000..173b49cb43efe --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFinalizerTest.java @@ -0,0 +1,130 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateHandleDummyUtil; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; +import org.apache.flink.runtime.state.DoneFuture; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.SnapshotResult; +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.util.TestLogger; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Random; + +/** + * Tests for {@link OperatorSnapshotFinalizer}. + */ +public class OperatorSnapshotFinalizerTest extends TestLogger { + + /** + * Test that the runnable futures are executed and the result is correctly extracted. + */ + @Test + public void testRunAndExtract() throws Exception{ + + Random random = new Random(0x42); + + KeyedStateHandle keyedTemplate = + StateHandleDummyUtil.createNewKeyedStateHandle(new KeyGroupRange(0, 0)); + OperatorStateHandle operatorTemplate = + StateHandleDummyUtil.createNewOperatorStateHandle(2, random); + + SnapshotResult snapKeyMan = SnapshotResult.withLocalState( + StateHandleDummyUtil.deepDummyCopy(keyedTemplate), + StateHandleDummyUtil.deepDummyCopy(keyedTemplate)); + + SnapshotResult snapKeyRaw = SnapshotResult.withLocalState( + StateHandleDummyUtil.deepDummyCopy(keyedTemplate), + StateHandleDummyUtil.deepDummyCopy(keyedTemplate)); + + SnapshotResult snapOpMan = SnapshotResult.withLocalState( + StateHandleDummyUtil.deepDummyCopy(operatorTemplate), + StateHandleDummyUtil.deepDummyCopy(operatorTemplate)); + + SnapshotResult snapOpRaw = SnapshotResult.withLocalState( + StateHandleDummyUtil.deepDummyCopy(operatorTemplate), + StateHandleDummyUtil.deepDummyCopy(operatorTemplate)); + + DoneFuture> managedKeyed = new PseudoNotDoneFuture<>(snapKeyMan); + DoneFuture> rawKeyed = new PseudoNotDoneFuture<>(snapKeyRaw); + DoneFuture> managedOp = new PseudoNotDoneFuture<>(snapOpMan); + DoneFuture> rawOp = new PseudoNotDoneFuture<>(snapOpRaw); + + Assert.assertFalse(managedKeyed.isDone()); + Assert.assertFalse(rawKeyed.isDone()); + Assert.assertFalse(managedOp.isDone()); + Assert.assertFalse(rawOp.isDone()); + + OperatorSnapshotFutures futures = new OperatorSnapshotFutures(managedKeyed, rawKeyed, managedOp, rawOp); + OperatorSnapshotFinalizer operatorSnapshotFinalizer = new OperatorSnapshotFinalizer(futures); + + Assert.assertTrue(managedKeyed.isDone()); + Assert.assertTrue(rawKeyed.isDone()); + Assert.assertTrue(managedOp.isDone()); + Assert.assertTrue(rawOp.isDone()); + + OperatorSubtaskState jobManagerOwnedState = operatorSnapshotFinalizer.getJobManagerOwnedState(); + Assert.assertTrue(checkResult(snapKeyMan.getJobManagerOwnedSnapshot(), jobManagerOwnedState.getManagedKeyedState())); + Assert.assertTrue(checkResult(snapKeyRaw.getJobManagerOwnedSnapshot(), jobManagerOwnedState.getRawKeyedState())); + Assert.assertTrue(checkResult(snapOpMan.getJobManagerOwnedSnapshot(), jobManagerOwnedState.getManagedOperatorState())); + Assert.assertTrue(checkResult(snapOpRaw.getJobManagerOwnedSnapshot(), jobManagerOwnedState.getRawOperatorState())); + + OperatorSubtaskState taskLocalState = operatorSnapshotFinalizer.getTaskLocalState(); + Assert.assertTrue(checkResult(snapKeyMan.getTaskLocalSnapshot(), taskLocalState.getManagedKeyedState())); + Assert.assertTrue(checkResult(snapKeyRaw.getTaskLocalSnapshot(), taskLocalState.getRawKeyedState())); + Assert.assertTrue(checkResult(snapOpMan.getTaskLocalSnapshot(), taskLocalState.getManagedOperatorState())); + Assert.assertTrue(checkResult(snapOpRaw.getTaskLocalSnapshot(), taskLocalState.getRawOperatorState())); + } + + private boolean checkResult(T expected, StateObjectCollection actual) { + if (expected == null) { + return actual.isEmpty(); + } + + return actual.size() == 1 && expected == actual.iterator().next(); + } + + static class PseudoNotDoneFuture extends DoneFuture { + + private boolean done; + + PseudoNotDoneFuture(T payload) { + super(payload); + this.done = false; + } + + @Override + public void run() { + super.run(); + this.done = true; + } + + @Override + public boolean isDone() { + return done; + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFuturesTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFuturesTest.java index 6da39afbbe239..4122a71d3c8a6 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFuturesTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/OperatorSnapshotFuturesTest.java @@ -18,8 +18,11 @@ package org.apache.flink.streaming.api.operators; +import org.apache.flink.runtime.state.DoneFuture; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; +import org.apache.flink.runtime.state.SnapshotResult; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -28,7 +31,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; -import static org.powermock.api.mockito.PowerMockito.when; +import static org.powermock.api.mockito.PowerMockito.spy; /** * Tests for {@link OperatorSnapshotFutures}. @@ -46,20 +49,28 @@ public void testCancelAndCleanup() throws Exception { operatorSnapshotResult.cancel(); KeyedStateHandle keyedManagedStateHandle = mock(KeyedStateHandle.class); - RunnableFuture keyedStateManagedFuture = mock(RunnableFuture.class); - when(keyedStateManagedFuture.get()).thenReturn(keyedManagedStateHandle); + SnapshotResult keyedStateManagedResult = + SnapshotResult.of(keyedManagedStateHandle); + RunnableFuture> keyedStateManagedFuture = + spy(DoneFuture.of(keyedStateManagedResult)); KeyedStateHandle keyedRawStateHandle = mock(KeyedStateHandle.class); - RunnableFuture keyedStateRawFuture = mock(RunnableFuture.class); - when(keyedStateRawFuture.get()).thenReturn(keyedRawStateHandle); - - OperatorStateHandle operatorManagedStateHandle = mock(OperatorStateHandle.class); - RunnableFuture operatorStateManagedFuture = mock(RunnableFuture.class); - when(operatorStateManagedFuture.get()).thenReturn(operatorManagedStateHandle); - - OperatorStateHandle operatorRawStateHandle = mock(OperatorStateHandle.class); - RunnableFuture operatorStateRawFuture = mock(RunnableFuture.class); - when(operatorStateRawFuture.get()).thenReturn(operatorRawStateHandle); + SnapshotResult keyedStateRawResult = + SnapshotResult.of(keyedRawStateHandle); + RunnableFuture> keyedStateRawFuture = + spy(DoneFuture.of(keyedStateRawResult)); + + OperatorStateHandle operatorManagedStateHandle = mock(OperatorStreamStateHandle.class); + SnapshotResult operatorStateManagedResult = + SnapshotResult.of(operatorManagedStateHandle); + RunnableFuture> operatorStateManagedFuture = + spy(DoneFuture.of(operatorStateManagedResult)); + + OperatorStateHandle operatorRawStateHandle = mock(OperatorStreamStateHandle.class); + SnapshotResult operatorStateRawResult = + SnapshotResult.of(operatorRawStateHandle); + RunnableFuture> operatorStateRawFuture = + spy(DoneFuture.of(operatorStateRawResult)); operatorSnapshotResult = new OperatorSnapshotFutures( keyedStateManagedFuture, diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java index c5e5df8537c8e..90649f2abeccc 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java @@ -30,6 +30,7 @@ import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -42,12 +43,13 @@ import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.StateInitializationContextImpl; import org.apache.flink.runtime.state.StatePartitionStreamProvider; -import org.apache.flink.runtime.state.TaskLocalStateStore; import org.apache.flink.runtime.state.TaskStateManager; import org.apache.flink.runtime.state.TaskStateManagerImpl; +import org.apache.flink.runtime.state.TestTaskLocalStateStore; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.runtime.taskmanager.CheckpointResponder; @@ -61,7 +63,6 @@ import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -135,15 +136,15 @@ public void setUp() throws Exception { DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME, new OperatorStateHandle.StateMetaInfo(offsets.toArray(), OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)); OperatorStateHandle operatorStateHandle = - new OperatorStateHandle(offsetsMap, new ByteStateHandleCloseChecking("os-" + i, out.toByteArray())); + new OperatorStreamStateHandle(offsetsMap, new ByteStateHandleCloseChecking("os-" + i, out.toByteArray())); operatorStateHandles.add(operatorStateHandle); } OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState( - Collections.emptyList(), - operatorStateHandles, - Collections.emptyList(), - keyedStateHandles); + StateObjectCollection.empty(), + new StateObjectCollection<>(operatorStateHandles), + StateObjectCollection.empty(), + new StateObjectCollection<>(keyedStateHandles)); OperatorID operatorID = new OperatorID(); TaskStateSnapshot taskStateSnapshot = new TaskStateSnapshot(); @@ -154,7 +155,7 @@ public void setUp() throws Exception { TaskStateManager manager = new TaskStateManagerImpl( new JobID(), new ExecutionAttemptID(), - mock(TaskLocalStateStore.class), + new TestTaskLocalStateStore(), jobManagerTaskRestore, mock(CheckpointResponder.class)); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImplTest.java index 0eb140a39349b..b33a69ea1e6f8 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImplTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImplTest.java @@ -38,10 +38,13 @@ import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider; import org.apache.flink.runtime.state.OperatorStateBackend; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.StatePartitionStreamProvider; +import org.apache.flink.runtime.state.TaskLocalStateStore; import org.apache.flink.runtime.state.TaskStateManager; import org.apache.flink.runtime.state.TaskStateManagerImplTest; +import org.apache.flink.runtime.state.TestTaskLocalStateStore; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.runtime.taskmanager.TestCheckpointResponder; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; @@ -56,8 +59,6 @@ import java.util.Collections; import java.util.Random; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -91,19 +92,6 @@ public void testNoRestore() throws Exception { typeSerializer, closeableRegistry); - verify(stateBackend).createKeyedStateBackend( - any(Environment.class), - any(JobID.class), - any(String.class), - eq(typeSerializer), - anyInt(), - any(KeyGroupRange.class), - any(TaskKvStateRegistry.class)); - - verify(stateBackend).createOperatorStateBackend( - any(Environment.class), - any(String.class)); - OperatorStateBackend operatorStateBackend = stateContext.operatorStateBackend(); AbstractKeyedStateBackend keyedStateBackend = stateContext.keyedStateBackend(); InternalTimeServiceManager timeServiceManager = stateContext.internalTimerServiceManager(); @@ -124,13 +112,8 @@ public void testNoRestore() throws Exception { keyedStateInputs, operatorStateInputs); - for (KeyGroupStatePartitionStreamProvider keyedStateInput : keyedStateInputs) { - Assert.fail(); - } - - for (StatePartitionStreamProvider operatorStateInput : operatorStateInputs) { - Assert.fail(); - } + Assert.assertFalse(keyedStateInputs.iterator().hasNext()); + Assert.assertFalse(operatorStateInputs.iterator().hasNext()); } @SuppressWarnings("unchecked") @@ -172,14 +155,14 @@ public OperatorStateBackend createOperatorStateBackend( Random random = new Random(0x42); OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState( - new OperatorStateHandle( + new OperatorStreamStateHandle( Collections.singletonMap( "a", new OperatorStateHandle.StateMetaInfo( new long[]{0, 10}, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)), CheckpointTestUtils.createDummyStreamStateHandle(random)), - new OperatorStateHandle( + new OperatorStreamStateHandle( Collections.singletonMap( "_default_", new OperatorStateHandle.StateMetaInfo( @@ -209,19 +192,6 @@ public OperatorStateBackend createOperatorStateBackend( typeSerializer, closeableRegistry); - verify(mockingBackend).createKeyedStateBackend( - any(Environment.class), - any(JobID.class), - any(String.class), - eq(typeSerializer), - anyInt(), - any(KeyGroupRange.class), - any(TaskKvStateRegistry.class)); - - verify(mockingBackend).createOperatorStateBackend( - any(Environment.class), - any(String.class)); - OperatorStateBackend operatorStateBackend = stateContext.operatorStateBackend(); AbstractKeyedStateBackend keyedStateBackend = stateContext.keyedStateBackend(); InternalTimeServiceManager timeServiceManager = stateContext.internalTimerServiceManager(); @@ -276,11 +246,14 @@ private StreamTaskStateInitializer streamTaskStateManager( ExecutionAttemptID executionAttemptID = new ExecutionAttemptID(23L, 24L); TestCheckpointResponder checkpointResponderMock = new TestCheckpointResponder(); + TaskLocalStateStore taskLocalStateStore = new TestTaskLocalStateStore(); + TaskStateManager taskStateManager = TaskStateManagerImplTest.taskStateManager( jobID, executionAttemptID, checkpointResponderMock, - jobManagerTaskRestore); + jobManagerTaskRestore, + taskLocalStateStore); DummyEnvironment dummyEnvironment = new DummyEnvironment("test-task", 1, 0); dummyEnvironment.setTaskStateManager(taskStateManager); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java index 8e80f443bd82b..35e2fbde0aa18 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java @@ -538,7 +538,7 @@ public void testStateSnapshotAndRestore() throws Exception { testHarness.waitForTaskCompletion(); // set the operator state from previous attempt into the restored one - TaskStateSnapshot subtaskStates = taskStateManagerMock.getLastTaskStateSnapshot(); + TaskStateSnapshot subtaskStates = taskStateManagerMock.getLastJobManagerTaskStateSnapshot(); final OneInputStreamTaskTestHarness restoredTaskHarness = new OneInputStreamTaskTestHarness<>( diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java index 69264808dbe43..6a13f11ef5035 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java @@ -405,7 +405,6 @@ public void testSessionWindows() throws Exception { OperatorSubtaskState snapshot = testHarness.snapshot(0L, 0L); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator()); - testHarness.close(); testHarness = createTestHarness(operator); @@ -483,7 +482,6 @@ public void testSessionWindowsWithProcessFunction() throws Exception { OperatorSubtaskState snapshot = testHarness.snapshot(0L, 0L); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator()); - testHarness.close(); testHarness = createTestHarness(operator); @@ -794,7 +792,6 @@ public void testSessionWindowsWithContinuousEventTimeTrigger() throws Exception OperatorSubtaskState snapshot = testHarness.snapshot(0L, 0L); TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new Tuple3ResultSortComparator()); - testHarness.close(); expectedOutput.clear(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java index 499dfb1647fd2..84bcf5a690811 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -60,6 +61,7 @@ import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.TestTaskStateManager; @@ -197,7 +199,7 @@ private static Task createTask( KeyGroupRangeOffsets keyGroupRangeOffsets = new KeyGroupRangeOffsets(new KeyGroupRange(0, 0)); Collection operatorStateHandles = - Collections.singletonList(new OperatorStateHandle(operatorStateMetadata, state)); + Collections.singletonList(new OperatorStreamStateHandle(operatorStateMetadata, state)); List keyedStateHandles = Collections.singletonList(new KeyGroupsStateHandle(keyGroupRangeOffsets, state)); @@ -220,10 +222,10 @@ private static Task createTask( } OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState( - operatorStateBackend, - operatorStateStream, - keyedStateFromBackend, - keyedStateFromStream); + new StateObjectCollection<>(operatorStateBackend), + new StateObjectCollection<>(operatorStateStream), + new StateObjectCollection<>(keyedStateFromBackend), + new StateObjectCollection<>(keyedStateFromStream)); JobVertexID jobVertexID = new JobVertexID(); OperatorID operatorID = OperatorID.fromJobVertexID(jobVertexID); @@ -254,7 +256,7 @@ private static Task createTask( TestTaskStateManager taskStateManager = new TestTaskStateManager(); taskStateManager.setReportedCheckpointId(taskRestore.getRestoreCheckpointId()); - taskStateManager.setTaskStateSnapshotsByCheckpointId( + taskStateManager.setJobManagerTaskStateSnapshotsByCheckpointId( Collections.singletonMap( taskRestore.getRestoreCheckpointId(), taskRestore.getTaskStateSnapshot())); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/LocalStateForwardingTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/LocalStateForwardingTest.java new file mode 100644 index 0000000000000..e35f97c25a186 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/LocalStateForwardingTest.java @@ -0,0 +1,238 @@ +/* + * 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.streaming.runtime.tasks; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.CheckpointMetaData; +import org.apache.flink.runtime.checkpoint.CheckpointMetrics; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.runtime.state.DoneFuture; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.LocalRecoveryConfig; +import org.apache.flink.runtime.state.LocalRecoveryDirectoryProviderImpl; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.SnapshotResult; +import org.apache.flink.runtime.state.StateObject; +import org.apache.flink.runtime.state.TaskLocalStateStore; +import org.apache.flink.runtime.state.TaskLocalStateStoreImpl; +import org.apache.flink.runtime.state.TaskStateManagerImpl; +import org.apache.flink.runtime.state.TestTaskStateManager; +import org.apache.flink.runtime.taskmanager.TestCheckpointResponder; +import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; +import org.apache.flink.util.TestLogger; + +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import javax.annotation.Nonnegative; +import javax.annotation.Nullable; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Executor; +import java.util.concurrent.Future; +import java.util.concurrent.RunnableFuture; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.mockito.Mockito.mock; + +/** + * Test for forwarding of state reporting to and from {@link org.apache.flink.runtime.state.TaskStateManager}. + */ +public class LocalStateForwardingTest extends TestLogger { + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + /** + * This tests the forwarding of jm and tm-local state from the futures reported by the backends, through the + * async checkpointing thread to the {@link org.apache.flink.runtime.state.TaskStateManager}. + */ + @Test + public void testReportingFromSnapshotToTaskStateManager() { + + TestTaskStateManager taskStateManager = new TestTaskStateManager(); + + StreamMockEnvironment streamMockEnvironment = new StreamMockEnvironment( + new Configuration(), + new Configuration(), + new ExecutionConfig(), + 1024 * 1024, + new MockInputSplitProvider(), + 0, + taskStateManager); + + StreamTask testStreamTask = new StreamTaskTest.NoOpStreamTask(streamMockEnvironment); + CheckpointMetaData checkpointMetaData = new CheckpointMetaData(0L, 0L); + CheckpointMetrics checkpointMetrics = new CheckpointMetrics(); + + Map snapshots = new HashMap<>(1); + OperatorSnapshotFutures osFuture = new OperatorSnapshotFutures(); + + osFuture.setKeyedStateManagedFuture(createSnapshotResult(KeyedStateHandle.class)); + osFuture.setKeyedStateRawFuture(createSnapshotResult(KeyedStateHandle.class)); + osFuture.setOperatorStateManagedFuture(createSnapshotResult(OperatorStateHandle.class)); + osFuture.setOperatorStateRawFuture(createSnapshotResult(OperatorStateHandle.class)); + + OperatorID operatorID = new OperatorID(); + snapshots.put(operatorID, osFuture); + + StreamTask.AsyncCheckpointRunnable checkpointRunnable = + new StreamTask.AsyncCheckpointRunnable( + testStreamTask, + snapshots, + checkpointMetaData, + checkpointMetrics, + 0L); + + checkpointRunnable.run(); + + TaskStateSnapshot lastJobManagerTaskStateSnapshot = taskStateManager.getLastJobManagerTaskStateSnapshot(); + TaskStateSnapshot lastTaskManagerTaskStateSnapshot = taskStateManager.getLastTaskManagerTaskStateSnapshot(); + + OperatorSubtaskState jmState = + lastJobManagerTaskStateSnapshot.getSubtaskStateByOperatorID(operatorID); + + OperatorSubtaskState tmState = + lastTaskManagerTaskStateSnapshot.getSubtaskStateByOperatorID(operatorID); + + performCheck(osFuture.getKeyedStateManagedFuture(), jmState.getManagedKeyedState(), tmState.getManagedKeyedState()); + performCheck(osFuture.getKeyedStateRawFuture(), jmState.getRawKeyedState(), tmState.getRawKeyedState()); + performCheck(osFuture.getOperatorStateManagedFuture(), jmState.getManagedOperatorState(), tmState.getManagedOperatorState()); + performCheck(osFuture.getOperatorStateRawFuture(), jmState.getRawOperatorState(), tmState.getRawOperatorState()); + } + + /** + * This tests that state that was reported to the {@link org.apache.flink.runtime.state.TaskStateManager} is also + * reported to {@link org.apache.flink.runtime.taskmanager.CheckpointResponder} and {@link TaskLocalStateStoreImpl}. + */ + @Test + public void testReportingFromTaskStateManagerToResponderAndTaskLocalStateStore() throws Exception { + + final JobID jobID = new JobID(); + final AllocationID allocationID = new AllocationID(); + final ExecutionAttemptID executionAttemptID = new ExecutionAttemptID(); + final CheckpointMetaData checkpointMetaData = new CheckpointMetaData(42L, 4711L); + final CheckpointMetrics checkpointMetrics = new CheckpointMetrics(); + final int subtaskIdx = 42; + JobVertexID jobVertexID = new JobVertexID(); + + TaskStateSnapshot jmSnapshot = new TaskStateSnapshot(); + TaskStateSnapshot tmSnapshot = new TaskStateSnapshot(); + + final AtomicBoolean jmReported = new AtomicBoolean(false); + final AtomicBoolean tmReported = new AtomicBoolean(false); + + TestCheckpointResponder checkpointResponder = new TestCheckpointResponder() { + + @Override + public void acknowledgeCheckpoint( + JobID lJobID, + ExecutionAttemptID lExecutionAttemptID, + long lCheckpointId, + CheckpointMetrics lCheckpointMetrics, + TaskStateSnapshot lSubtaskState) { + + Assert.assertEquals(jobID, lJobID); + Assert.assertEquals(executionAttemptID, lExecutionAttemptID); + Assert.assertEquals(checkpointMetaData.getCheckpointId(), lCheckpointId); + Assert.assertEquals(checkpointMetrics, lCheckpointMetrics); + jmReported.set(true); + } + }; + + Executor executor = Executors.directExecutor(); + + LocalRecoveryDirectoryProviderImpl directoryProvider = new LocalRecoveryDirectoryProviderImpl( + temporaryFolder.newFolder(), + jobID, + jobVertexID, + subtaskIdx); + + LocalRecoveryConfig localRecoveryConfig = new LocalRecoveryConfig( + LocalRecoveryConfig.LocalRecoveryMode.ENABLE_FILE_BASED, + directoryProvider); + + TaskLocalStateStore taskLocalStateStore = + new TaskLocalStateStoreImpl(jobID, allocationID, jobVertexID, subtaskIdx, localRecoveryConfig, executor) { + @Override + public void storeLocalState( + @Nonnegative long checkpointId, + @Nullable TaskStateSnapshot localState) { + + Assert.assertEquals(tmSnapshot, localState); + tmReported.set(true); + } + }; + + TaskStateManagerImpl taskStateManager = + new TaskStateManagerImpl( + jobID, + executionAttemptID, + taskLocalStateStore, + null, + checkpointResponder); + + taskStateManager.reportTaskStateSnapshots( + checkpointMetaData, + checkpointMetrics, + jmSnapshot, + tmSnapshot); + + Assert.assertTrue("Reporting for JM state was not called.", jmReported.get()); + Assert.assertTrue("Reporting for TM state was not called.", tmReported.get()); + } + + private static void performCheck( + Future> resultFuture, + StateObjectCollection jmState, + StateObjectCollection tmState) { + + SnapshotResult snapshotResult; + try { + snapshotResult = resultFuture.get(); + } catch (Exception e) { + throw new RuntimeException(e); + } + + Assert.assertEquals( + snapshotResult.getJobManagerOwnedSnapshot(), + jmState.iterator().next()); + + Assert.assertEquals( + snapshotResult.getTaskLocalSnapshot(), + tmState.iterator().next()); + } + + private static RunnableFuture> createSnapshotResult(Class clazz) { + return DoneFuture.of(SnapshotResult.withLocalState(mock(clazz), mock(clazz))); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java index 3e0459d205cf4..81fb447455ed4 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java @@ -538,20 +538,20 @@ public void testSnapshottingAndRestoring() throws Exception { restoredTaskHarness.configureForKeyedStream(keySelector, BasicTypeInfo.STRING_TYPE_INFO); - restoredTaskHarness.setTaskStateSnapshot(checkpointId, taskStateManager.getLastTaskStateSnapshot()); + restoredTaskHarness.setTaskStateSnapshot(checkpointId, taskStateManager.getLastJobManagerTaskStateSnapshot()); StreamConfig restoredTaskStreamConfig = restoredTaskHarness.getStreamConfig(); configureChainedTestingStreamOperator(restoredTaskStreamConfig, numberChainedTasks); - TaskStateSnapshot stateHandles = taskStateManager.getLastTaskStateSnapshot(); + TaskStateSnapshot stateHandles = taskStateManager.getLastJobManagerTaskStateSnapshot(); Assert.assertEquals(numberChainedTasks, stateHandles.getSubtaskStateMappings().size()); TestingStreamOperator.numberRestoreCalls = 0; // transfer state to new harness restoredTaskHarness.taskStateManager.restoreLatestCheckpointState( - taskStateManager.getTaskStateSnapshotsByCheckpointId()); + taskStateManager.getJobManagerTaskStateSnapshotsByCheckpointId()); restoredTaskHarness.invoke(); restoredTaskHarness.endInput(); restoredTaskHarness.waitForTaskCompletion(deadline.timeLeft().toMillis()); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/RestoreStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/RestoreStreamTaskTest.java index f9f44733ab70b..c66040a8089c5 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/RestoreStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/RestoreStreamTaskTest.java @@ -252,7 +252,7 @@ private JobManagerTaskRestore createRunAndCheckpointOperatorChain( JobManagerTaskRestore jobManagerTaskRestore = new JobManagerTaskRestore( taskStateManager.getReportedCheckpointId(), - taskStateManager.getLastTaskStateSnapshot()); + taskStateManager.getLastJobManagerTaskStateSnapshot()); testHarness.endInput(); testHarness.waitForTaskCompletion(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java index 0ba081e04961e..32de8d508bbe9 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java @@ -46,7 +46,6 @@ import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.query.TaskKvStateRegistry; -import org.apache.flink.runtime.state.TaskLocalStateStore; import org.apache.flink.runtime.state.TaskStateManager; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; @@ -159,8 +158,6 @@ public StreamMockEnvironment( KvStateRegistry registry = new KvStateRegistry(); this.kvStateRegistry = registry.createTaskRegistry(jobID, getJobVertexId()); - - final TaskLocalStateStore localStateStore = new TaskLocalStateStore(jobID, getJobVertexId(), subtaskIndex); } public StreamMockEnvironment( @@ -304,10 +301,11 @@ public void acknowledgeCheckpoint(long checkpointId, CheckpointMetrics checkpoin @Override public void acknowledgeCheckpoint(long checkpointId, CheckpointMetrics checkpointMetrics, TaskStateSnapshot subtaskState) { - taskStateManager.reportStateHandles( + taskStateManager.reportTaskStateSnapshots( new CheckpointMetaData(checkpointId, 0L), checkpointMetrics, - subtaskState); + subtaskState, + null); } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java index 24b201433dab1..62a903bafb0e3 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java @@ -58,6 +58,7 @@ import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.OperatorStateBackend; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.SnapshotResult; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.TestTaskStateManager; import org.apache.flink.runtime.state.memory.MemoryBackendCheckpointStorage; @@ -282,10 +283,10 @@ public OperatorStateBackend createOperatorStateBackend(Environment env, String o } } - static class BlockingCallable implements Callable { + static class BlockingCallable implements Callable> { @Override - public OperatorStateHandle call() throws Exception { + public SnapshotResult call() throws Exception { // notify that we have started the asynchronous checkpointed operation CHECKPOINTING_LATCH.trigger(); // wait until we have reached the StreamTask#cleanup --> This will already cancel this FutureTask diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 99d4e5b0a8065..caea662b132bb 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -35,6 +35,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; import org.apache.flink.runtime.checkpoint.SubtaskState; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.AllocationID; @@ -67,15 +68,16 @@ import org.apache.flink.runtime.state.CheckpointStorage; import org.apache.flink.runtime.state.CheckpointStreamFactory; import org.apache.flink.runtime.state.DoneFuture; -import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateBackend; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.OperatorStreamStateHandle; +import org.apache.flink.runtime.state.SnapshotResult; import org.apache.flink.runtime.state.StateBackendFactory; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StatePartitionStreamProvider; -import org.apache.flink.runtime.state.TaskLocalStateStore; +import org.apache.flink.runtime.state.TaskLocalStateStoreImpl; import org.apache.flink.runtime.state.TaskStateManager; import org.apache.flink.runtime.state.TaskStateManagerImpl; import org.apache.flink.runtime.state.TestTaskStateManager; @@ -94,6 +96,7 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.InternalTimeServiceManager; +import org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; @@ -147,7 +150,6 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyCollectionOf; import static org.mockito.Matchers.anyLong; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; @@ -221,7 +223,7 @@ public void testEarlyCanceling() throws Exception { @Test public void testStateBackendLoadingAndClosing() throws Exception { Configuration taskManagerConfig = new Configuration(); - taskManagerConfig.setString(CheckpointingOptions.STATE_BACKEND, MockStateBackend.class.getName()); + taskManagerConfig.setString(CheckpointingOptions.STATE_BACKEND, TestMemoryStateBackendFactory.class.getName()); StreamConfig cfg = new StreamConfig(new Configuration()); cfg.setStateKeySerializer(mock(TypeSerializer.class)); @@ -253,7 +255,7 @@ public void testStateBackendLoadingAndClosing() throws Exception { @Test public void testStateBackendClosingOnFailure() throws Exception { Configuration taskManagerConfig = new Configuration(); - taskManagerConfig.setString(CheckpointingOptions.STATE_BACKEND, MockStateBackend.class.getName()); + taskManagerConfig.setString(CheckpointingOptions.STATE_BACKEND, TestMemoryStateBackendFactory.class.getName()); StreamConfig cfg = new StreamConfig(new Configuration()); cfg.setStateKeySerializer(mock(TypeSerializer.class)); @@ -414,7 +416,7 @@ public void testFailingAsyncCheckpointRunnable() throws Exception { OperatorSnapshotFutures operatorSnapshotResult2 = mock(OperatorSnapshotFutures.class); OperatorSnapshotFutures operatorSnapshotResult3 = mock(OperatorSnapshotFutures.class); - RunnableFuture failingFuture = mock(RunnableFuture.class); + RunnableFuture> failingFuture = mock(RunnableFuture.class); when(failingFuture.get()).thenThrow(new ExecutionException(new Exception("Test exception"))); when(operatorSnapshotResult3.getOperatorStateRawFuture()).thenReturn(failingFuture); @@ -499,7 +501,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { TaskStateManager taskStateManager = new TaskStateManagerImpl( new JobID(1L, 2L), new ExecutionAttemptID(1L, 2L), - mock(TaskLocalStateStore.class), + mock(TaskLocalStateStoreImpl.class), null, checkpointResponder); @@ -514,17 +516,18 @@ public Object answer(InvocationOnMock invocation) throws Throwable { CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, timestamp); StreamOperator streamOperator = mock(StreamOperator.class); + when(streamOperator.getOperatorID()).thenReturn(new OperatorID(42, 42)); KeyedStateHandle managedKeyedStateHandle = mock(KeyedStateHandle.class); KeyedStateHandle rawKeyedStateHandle = mock(KeyedStateHandle.class); - OperatorStateHandle managedOperatorStateHandle = mock(OperatorStateHandle.class); - OperatorStateHandle rawOperatorStateHandle = mock(OperatorStateHandle.class); + OperatorStateHandle managedOperatorStateHandle = mock(OperatorStreamStateHandle.class); + OperatorStateHandle rawOperatorStateHandle = mock(OperatorStreamStateHandle.class); OperatorSnapshotFutures operatorSnapshotResult = new OperatorSnapshotFutures( - new DoneFuture<>(managedKeyedStateHandle), - new DoneFuture<>(rawKeyedStateHandle), - new DoneFuture<>(managedOperatorStateHandle), - new DoneFuture<>(rawOperatorStateHandle)); + DoneFuture.of(SnapshotResult.of(managedKeyedStateHandle)), + DoneFuture.of(SnapshotResult.of(rawKeyedStateHandle)), + DoneFuture.of(SnapshotResult.of(managedOperatorStateHandle)), + DoneFuture.of(SnapshotResult.of(rawOperatorStateHandle))); when(streamOperator.snapshotState(anyLong(), anyLong(), any(CheckpointOptions.class), any(CheckpointStreamFactory.class))).thenReturn(operatorSnapshotResult); @@ -561,10 +564,10 @@ public Object answer(InvocationOnMock invocation) throws Throwable { OperatorSubtaskState subtaskState = subtaskStates.getSubtaskStateMappings().iterator().next().getValue(); // check that the subtask state contains the expected state handles - assertEquals(Collections.singletonList(managedKeyedStateHandle), subtaskState.getManagedKeyedState()); - assertEquals(Collections.singletonList(rawKeyedStateHandle), subtaskState.getRawKeyedState()); - assertEquals(Collections.singletonList(managedOperatorStateHandle), subtaskState.getManagedOperatorState()); - assertEquals(Collections.singletonList(rawOperatorStateHandle), subtaskState.getRawOperatorState()); + assertEquals(StateObjectCollection.singleton(managedKeyedStateHandle), subtaskState.getManagedKeyedState()); + assertEquals(StateObjectCollection.singleton(rawKeyedStateHandle), subtaskState.getRawKeyedState()); + assertEquals(StateObjectCollection.singleton(managedOperatorStateHandle), subtaskState.getManagedOperatorState()); + assertEquals(StateObjectCollection.singleton(rawOperatorStateHandle), subtaskState.getRawOperatorState()); // check that the state handles have not been discarded verify(managedKeyedStateHandle, never()).discardState(); @@ -602,26 +605,15 @@ public void testAsyncCheckpointingConcurrentCloseBeforeAcknowledge() throws Exce Environment mockEnvironment = spy(new MockEnvironment()); - whenNew(OperatorSubtaskState.class). - withArguments( - anyCollectionOf(OperatorStateHandle.class), - anyCollectionOf(OperatorStateHandle.class), - anyCollectionOf(KeyedStateHandle.class), - anyCollectionOf(KeyedStateHandle.class)). - thenAnswer(new Answer() { - @Override - public OperatorSubtaskState answer(InvocationOnMock invocation) throws Throwable { - createSubtask.trigger(); - completeSubtask.await(); - Object[] arguments = invocation.getArguments(); - return new OperatorSubtaskState( - (OperatorStateHandle) arguments[0], - (OperatorStateHandle) arguments[1], - (KeyedStateHandle) arguments[2], - (KeyedStateHandle) arguments[3] - ); - } - }); + whenNew(OperatorSnapshotFinalizer.class). + withAnyArguments(). + thenAnswer((Answer) invocation -> { + createSubtask.trigger(); + completeSubtask.await(); + Object[] arguments = invocation.getArguments(); + return new OperatorSnapshotFinalizer((OperatorSnapshotFutures) arguments[0]); + } + ); StreamTask streamTask = new EmptyStreamTask(mockEnvironment); CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, timestamp); @@ -632,14 +624,14 @@ public OperatorSubtaskState answer(InvocationOnMock invocation) throws Throwable KeyedStateHandle managedKeyedStateHandle = mock(KeyedStateHandle.class); KeyedStateHandle rawKeyedStateHandle = mock(KeyedStateHandle.class); - OperatorStateHandle managedOperatorStateHandle = mock(OperatorStateHandle.class); - OperatorStateHandle rawOperatorStateHandle = mock(OperatorStateHandle.class); + OperatorStateHandle managedOperatorStateHandle = mock(OperatorStreamStateHandle.class); + OperatorStateHandle rawOperatorStateHandle = mock(OperatorStreamStateHandle.class); OperatorSnapshotFutures operatorSnapshotResult = new OperatorSnapshotFutures( - new DoneFuture<>(managedKeyedStateHandle), - new DoneFuture<>(rawKeyedStateHandle), - new DoneFuture<>(managedOperatorStateHandle), - new DoneFuture<>(rawOperatorStateHandle)); + DoneFuture.of(SnapshotResult.of(managedKeyedStateHandle)), + DoneFuture.of(SnapshotResult.of(rawKeyedStateHandle)), + DoneFuture.of(SnapshotResult.of(managedOperatorStateHandle)), + DoneFuture.of(SnapshotResult.of(rawOperatorStateHandle))); when(streamOperator.snapshotState(anyLong(), anyLong(), any(CheckpointOptions.class), any(CheckpointStreamFactory.class))).thenReturn(operatorSnapshotResult); @@ -722,7 +714,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { TaskStateManager taskStateManager = new TaskStateManagerImpl( new JobID(1L, 2L), new ExecutionAttemptID(1L, 2L), - mock(TaskLocalStateStore.class), + mock(TaskLocalStateStoreImpl.class), null, checkpointResponder); @@ -823,7 +815,13 @@ public void testOperatorClosingBeforeStopRunning() throws Throwable { // Test Utilities // ------------------------------------------------------------------------ - private static class NoOpStreamTask> extends StreamTask { + /** + * Operator that does nothing. + * + * @param + * @param + */ + public static class NoOpStreamTask> extends StreamTask { public NoOpStreamTask(Environment environment) { super(environment, null); @@ -897,10 +895,18 @@ public void notifyTaskExecutionStateChanged(TaskExecutionState taskExecutionStat } } + public static Task createTask( + Class invokable, + StreamConfig taskConfig, + Configuration taskManagerConfig) throws Exception { + return createTask(invokable, taskConfig, taskManagerConfig, new TestTaskStateManager()); + } + public static Task createTask( Class invokable, StreamConfig taskConfig, - Configuration taskManagerConfig) throws Exception { + Configuration taskManagerConfig, + TestTaskStateManager taskStateManager) throws Exception { BlobCacheService blobService = new BlobCacheService(mock(PermanentBlobCache.class), mock(TransientBlobCache.class)); @@ -951,7 +957,7 @@ public static Task createTask( mock(IOManager.class), network, mock(BroadcastVariableManager.class), - new TestTaskStateManager(), + taskStateManager, mock(TaskManagerActions.class), mock(InputSplitProvider.class), mock(CheckpointResponder.class), @@ -1021,22 +1027,16 @@ public void cancel() {} /** * Mocked state backend factory which returns mocks for the operator and keyed state backends. */ - public static final class MockStateBackend implements StateBackendFactory { + public static final class TestMemoryStateBackendFactory implements StateBackendFactory { private static final long serialVersionUID = 1L; @Override public AbstractStateBackend createFromConfig(Configuration config) { - return new MemoryStateBackend() { - @Override - public OperatorStateBackend createOperatorStateBackend(Environment env, String operatorIdentifier) throws Exception { - return spy(super.createOperatorStateBackend(env, operatorIdentifier)); - } + return new TestSpyWrapperStateBackend(createInnerBackend(config)); + } - @Override - public AbstractKeyedStateBackend createKeyedStateBackend(Environment env, JobID jobID, String operatorIdentifier, TypeSerializer keySerializer, int numberOfKeyGroups, KeyGroupRange keyGroupRange, TaskKvStateRegistry kvStateRegistry) { - return spy(super.createKeyedStateBackend(env, jobID, operatorIdentifier, keySerializer, numberOfKeyGroups, keyGroupRange, kvStateRegistry)); - } - }; + protected AbstractStateBackend createInnerBackend(Configuration config) { + return new MemoryStateBackend(); } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java index bcb833ead15c1..08032bdb61872 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java @@ -141,7 +141,7 @@ public TestTaskStateManager getTaskStateManager() { public void setTaskStateSnapshot(long checkpointId, TaskStateSnapshot taskStateSnapshot) { taskStateManager.setReportedCheckpointId(checkpointId); - taskStateManager.setTaskStateSnapshotsByCheckpointId( + taskStateManager.setJobManagerTaskStateSnapshotsByCheckpointId( Collections.singletonMap(checkpointId, taskStateSnapshot)); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java index 6df33b7edd5d6..3d3b28ed29930 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java @@ -61,6 +61,7 @@ import org.apache.flink.runtime.state.OperatorStateBackend; import org.apache.flink.runtime.state.OperatorStateCheckpointOutputStream; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.SnapshotResult; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.runtime.state.StreamStateHandle; @@ -81,9 +82,10 @@ import org.junit.Assert; import org.junit.Test; +import javax.annotation.Nullable; + import java.io.IOException; import java.util.Collections; -import java.util.concurrent.Callable; import java.util.concurrent.FutureTask; import java.util.concurrent.RunnableFuture; @@ -303,7 +305,7 @@ public OperatorStateBackend createOperatorStateBackend(Environment env, String o env.getExecutionConfig(), true) { @Override - public RunnableFuture snapshot( + public RunnableFuture> snapshot( long checkpointId, long timestamp, CheckpointStreamFactory streamFactory, @@ -332,17 +334,14 @@ public OperatorStateBackend createOperatorStateBackend(Environment env, String o env.getExecutionConfig(), true) { @Override - public RunnableFuture snapshot( + public RunnableFuture> snapshot( long checkpointId, long timestamp, CheckpointStreamFactory streamFactory, CheckpointOptions checkpointOptions) throws Exception { - return new FutureTask<>(new Callable() { - @Override - public OperatorStateHandle call() throws Exception { - throw new Exception("Async part snapshot exception."); - } + return new FutureTask<>(() -> { + throw new Exception("Async part snapshot exception."); }); } }; @@ -364,6 +363,7 @@ private static final class LockingOutputStream extends CheckpointStateOutputStre private final Object lock = new Object(); private volatile boolean closed; + @Nullable @Override public StreamStateHandle closeAndGetHandle() throws IOException { throw new UnsupportedOperationException(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TestSpyWrapperStateBackend.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TestSpyWrapperStateBackend.java new file mode 100644 index 0000000000000..914326bdbeb03 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TestSpyWrapperStateBackend.java @@ -0,0 +1,82 @@ +/* + * 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.streaming.runtime.tasks; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.query.TaskKvStateRegistry; +import org.apache.flink.runtime.state.AbstractKeyedStateBackend; +import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.CheckpointStorage; +import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.OperatorStateBackend; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; + +import static org.powermock.api.mockito.PowerMockito.spy; + +/** + * This class wraps an {@link AbstractStateBackend} and enriches all the created objects as spies. + */ +public class TestSpyWrapperStateBackend extends AbstractStateBackend { + + private final AbstractStateBackend delegate; + + public TestSpyWrapperStateBackend(AbstractStateBackend delegate) { + this.delegate = Preconditions.checkNotNull(delegate); + } + + @Override + public AbstractKeyedStateBackend createKeyedStateBackend( + Environment env, + JobID jobID, + String operatorIdentifier, + TypeSerializer keySerializer, + int numberOfKeyGroups, + KeyGroupRange keyGroupRange, + TaskKvStateRegistry kvStateRegistry) throws IOException { + return spy(delegate.createKeyedStateBackend( + env, + jobID, + operatorIdentifier, + keySerializer, + numberOfKeyGroups, + keyGroupRange, + kvStateRegistry)); + } + + @Override + public OperatorStateBackend createOperatorStateBackend( + Environment env, String operatorIdentifier) throws Exception { + return spy(delegate.createOperatorStateBackend(env, operatorIdentifier)); + } + + @Override + public CompletedCheckpointStorageLocation resolveCheckpoint(String externalPointer) throws IOException { + return spy(delegate.resolveCheckpoint(externalPointer)); + } + + @Override + public CheckpointStorage createCheckpointStorage(JobID jobId) throws IOException { + return spy(delegate.createCheckpointStorage(jobId)); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java index 28ad9302310ae..ed2da18d1b6a4 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.checkpoint.RoundRobinOperatorStateRepartitioner; import org.apache.flink.runtime.checkpoint.StateAssignmentOperation; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -47,6 +48,7 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorTest; +import org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamOperator; @@ -60,7 +62,6 @@ import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; -import org.apache.flink.util.FutureUtil; import org.apache.flink.util.OutputTag; import org.apache.flink.util.Preconditions; @@ -320,11 +321,27 @@ public void setup(TypeSerializer outputSerializer) { * subtask. */ public void initializeState(OperatorSubtaskState operatorStateHandles) throws Exception { + initializeState(operatorStateHandles, null); + } + + /** + * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#initializeState()}. + * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#setup(StreamTask, StreamConfig, Output)} + * if it was not called before. + * + * @param jmOperatorStateHandles the primary state (owned by JM) + * @param tmOperatorStateHandles the (optional) local state (owned by TM) or null. + * @throws Exception + */ + public void initializeState( + OperatorSubtaskState jmOperatorStateHandles, + OperatorSubtaskState tmOperatorStateHandles) throws Exception { + if (!setupCalled) { setup(); } - if (operatorStateHandles != null) { + if (jmOperatorStateHandles != null) { int numKeyGroups = getEnvironment().getTaskInfo().getMaxNumberOfParallelSubtasks(); int numSubtasks = getEnvironment().getTaskInfo().getNumberOfParallelSubtasks(); int subtaskIndex = getEnvironment().getTaskInfo().getIndexOfThisSubtask(); @@ -332,53 +349,54 @@ public void initializeState(OperatorSubtaskState operatorStateHandles) throws Ex // create a new OperatorStateHandles that only contains the state for our key-groups List keyGroupPartitions = StateAssignmentOperation.createKeyGroupPartitions( - numKeyGroups, - numSubtasks); + numKeyGroups, + numSubtasks); - KeyGroupRange localKeyGroupRange = - keyGroupPartitions.get(subtaskIndex); + KeyGroupRange localKeyGroupRange = keyGroupPartitions.get(subtaskIndex); - List localManagedKeyGroupState = null; - if (operatorStateHandles.getManagedKeyedState() != null) { - localManagedKeyGroupState = StateAssignmentOperation.getKeyedStateHandles( - operatorStateHandles.getManagedKeyedState(), - localKeyGroupRange); - } + List localManagedKeyGroupState = StateAssignmentOperation.getKeyedStateHandles( + jmOperatorStateHandles.getManagedKeyedState(), + localKeyGroupRange); - List localRawKeyGroupState = null; - if (operatorStateHandles.getRawKeyedState() != null) { - localRawKeyGroupState = StateAssignmentOperation.getKeyedStateHandles( - operatorStateHandles.getRawKeyedState(), - localKeyGroupRange); - } + List localRawKeyGroupState = StateAssignmentOperation.getKeyedStateHandles( + jmOperatorStateHandles.getRawKeyedState(), + localKeyGroupRange); List managedOperatorState = new ArrayList<>(); - if (operatorStateHandles.getManagedOperatorState() != null) { - managedOperatorState.addAll(operatorStateHandles.getManagedOperatorState()); - } + + managedOperatorState.addAll(jmOperatorStateHandles.getManagedOperatorState()); + Collection localManagedOperatorState = operatorStateRepartitioner.repartitionState( - managedOperatorState, - numSubtasks).get(subtaskIndex); + managedOperatorState, + numSubtasks).get(subtaskIndex); List rawOperatorState = new ArrayList<>(); - if (operatorStateHandles.getRawOperatorState() != null) { - rawOperatorState.addAll(operatorStateHandles.getRawOperatorState()); - } + + rawOperatorState.addAll(jmOperatorStateHandles.getRawOperatorState()); + Collection localRawOperatorState = operatorStateRepartitioner.repartitionState( - rawOperatorState, - numSubtasks).get(subtaskIndex); + rawOperatorState, + numSubtasks).get(subtaskIndex); - OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState( - nullToEmptyCollection(localManagedOperatorState), - nullToEmptyCollection(localRawOperatorState), - nullToEmptyCollection(localManagedKeyGroupState), - nullToEmptyCollection(localRawKeyGroupState)); + OperatorSubtaskState processedJmOpSubtaskState = new OperatorSubtaskState( + new StateObjectCollection<>(nullToEmptyCollection(localManagedOperatorState)), + new StateObjectCollection<>(nullToEmptyCollection(localRawOperatorState)), + new StateObjectCollection<>(nullToEmptyCollection(localManagedKeyGroupState)), + new StateObjectCollection<>(nullToEmptyCollection(localRawKeyGroupState))); - TaskStateSnapshot taskStateSnapshot = new TaskStateSnapshot(); - taskStateSnapshot.putSubtaskStateByOperatorID(operator.getOperatorID(), operatorSubtaskState); + TaskStateSnapshot jmTaskStateSnapshot = new TaskStateSnapshot(); + jmTaskStateSnapshot.putSubtaskStateByOperatorID(operator.getOperatorID(), processedJmOpSubtaskState); taskStateManager.setReportedCheckpointId(0); - taskStateManager.setTaskStateSnapshotsByCheckpointId(Collections.singletonMap(0L, taskStateSnapshot)); + taskStateManager.setJobManagerTaskStateSnapshotsByCheckpointId( + Collections.singletonMap(0L, jmTaskStateSnapshot)); + + if (tmOperatorStateHandles != null) { + TaskStateSnapshot tmTaskStateSnapshot = new TaskStateSnapshot(); + tmTaskStateSnapshot.putSubtaskStateByOperatorID(operator.getOperatorID(), tmOperatorStateHandles); + taskStateManager.setTaskManagerTaskStateSnapshotsByCheckpointId( + Collections.singletonMap(0L, tmTaskStateSnapshot)); + } } operator.initializeState(); @@ -422,35 +440,24 @@ public static OperatorSubtaskState repackageState(OperatorSubtaskState... handle List mergedManagedKeyedState = new ArrayList<>(handles.length); List mergedRawKeyedState = new ArrayList<>(handles.length); - for (OperatorSubtaskState handle: handles) { + for (OperatorSubtaskState handle : handles) { Collection managedOperatorState = handle.getManagedOperatorState(); Collection rawOperatorState = handle.getRawOperatorState(); Collection managedKeyedState = handle.getManagedKeyedState(); Collection rawKeyedState = handle.getRawKeyedState(); - if (managedOperatorState != null) { - mergedManagedOperatorState.addAll(managedOperatorState); - } - - if (rawOperatorState != null) { - mergedRawOperatorState.addAll(rawOperatorState); - } - - if (managedKeyedState != null) { - mergedManagedKeyedState.addAll(managedKeyedState); - } - - if (rawKeyedState != null) { - mergedRawKeyedState.addAll(rawKeyedState); - } + mergedManagedOperatorState.addAll(managedOperatorState); + mergedRawOperatorState.addAll(rawOperatorState); + mergedManagedKeyedState.addAll(managedKeyedState); + mergedRawKeyedState.addAll(rawKeyedState); } return new OperatorSubtaskState( - mergedManagedOperatorState, - mergedRawOperatorState, - mergedManagedKeyedState, - mergedRawKeyedState); + new StateObjectCollection<>(mergedManagedOperatorState), + new StateObjectCollection<>(mergedRawOperatorState), + new StateObjectCollection<>(mergedManagedKeyedState), + new StateObjectCollection<>(mergedRawKeyedState)); } /** @@ -469,6 +476,13 @@ public void open() throws Exception { * Calls {@link StreamOperator#snapshotState(long, long, CheckpointOptions, org.apache.flink.runtime.state.CheckpointStreamFactory)}. */ public OperatorSubtaskState snapshot(long checkpointId, long timestamp) throws Exception { + return snapshotWithLocalState(checkpointId, timestamp).getJobManagerOwnedState(); + } + + /** + * Calls {@link StreamOperator#snapshotState(long, long, CheckpointOptions)}. + */ + public OperatorSnapshotFinalizer snapshotWithLocalState(long checkpointId, long timestamp) throws Exception { OperatorSnapshotFutures operatorStateResult = operator.snapshotState( checkpointId, @@ -476,17 +490,7 @@ public OperatorSubtaskState snapshot(long checkpointId, long timestamp) throws E CheckpointOptions.forCheckpointWithDefaultLocation(), checkpointStorage.resolveCheckpointStorageLocation(checkpointId, CheckpointStorageLocationReference.getDefault())); - KeyedStateHandle keyedManaged = FutureUtil.runIfNotDoneAndGet(operatorStateResult.getKeyedStateManagedFuture()); - KeyedStateHandle keyedRaw = FutureUtil.runIfNotDoneAndGet(operatorStateResult.getKeyedStateRawFuture()); - - OperatorStateHandle opManaged = FutureUtil.runIfNotDoneAndGet(operatorStateResult.getOperatorStateManagedFuture()); - OperatorStateHandle opRaw = FutureUtil.runIfNotDoneAndGet(operatorStateResult.getOperatorStateRawFuture()); - - return new OperatorSubtaskState( - opManaged != null ? Collections.singletonList(opManaged) : Collections.emptyList(), - opRaw != null ? Collections.singletonList(opRaw) : Collections.emptyList(), - keyedManaged != null ? Collections.singletonList(keyedManaged) : Collections.emptyList(), - keyedRaw != null ? Collections.singletonList(keyedRaw) : Collections.emptyList()); + return new OperatorSnapshotFinalizer(operatorStateResult); } /** 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 8d37266446902..1b5113da2cbbf 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 @@ -19,6 +19,7 @@ package org.apache.flink.streaming.util; 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.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; @@ -50,7 +51,7 @@ public static void writeStateHandle(OperatorSubtaskState state, String path) thr try (DataOutputStream dos = new DataOutputStream(out)) { - // must be here for compatibility + // required for backwards compatibility. dos.writeInt(0); // still required for compatibility @@ -108,15 +109,16 @@ public static OperatorSubtaskState readStateHandle(String path) throws IOExcepti FileInputStream in = new FileInputStream(path); try (DataInputStream dis = new DataInputStream(in)) { - // ignored + // required for backwards compatibility. dis.readInt(); // still required for compatibility to consume the bytes. SavepointV1Serializer.deserializeStreamStateHandle(dis); - List rawOperatorState = new ArrayList<>(); + List rawOperatorState = null; int numRawOperatorStates = dis.readInt(); if (numRawOperatorStates >= 0) { + rawOperatorState = new ArrayList<>(); for (int i = 0; i < numRawOperatorStates; i++) { OperatorStateHandle operatorState = SavepointV1Serializer.deserializeOperatorStateHandle( dis); @@ -124,9 +126,10 @@ public static OperatorSubtaskState readStateHandle(String path) throws IOExcepti } } - List managedOperatorState = new ArrayList<>(); + List managedOperatorState = null; int numManagedOperatorStates = dis.readInt(); if (numManagedOperatorStates >= 0) { + managedOperatorState = new ArrayList<>(); for (int i = 0; i < numManagedOperatorStates; i++) { OperatorStateHandle operatorState = SavepointV1Serializer.deserializeOperatorStateHandle( dis); @@ -134,9 +137,10 @@ public static OperatorSubtaskState readStateHandle(String path) throws IOExcepti } } - List rawKeyedState = new ArrayList<>(); + List rawKeyedState = null; int numRawKeyedStates = dis.readInt(); if (numRawKeyedStates >= 0) { + rawKeyedState = new ArrayList<>(); for (int i = 0; i < numRawKeyedStates; i++) { KeyedStateHandle keyedState = SavepointV1Serializer.deserializeKeyedStateHandle( dis); @@ -144,9 +148,10 @@ public static OperatorSubtaskState readStateHandle(String path) throws IOExcepti } } - List managedKeyedState = new ArrayList<>(); + List managedKeyedState = null; int numManagedKeyedStates = dis.readInt(); if (numManagedKeyedStates >= 0) { + managedKeyedState = new ArrayList<>(); for (int i = 0; i < numManagedKeyedStates; i++) { KeyedStateHandle keyedState = SavepointV1Serializer.deserializeKeyedStateHandle( dis); @@ -155,10 +160,10 @@ public static OperatorSubtaskState readStateHandle(String path) throws IOExcepti } return new OperatorSubtaskState( - managedOperatorState, - rawOperatorState, - managedKeyedState, - rawKeyedState); + new StateObjectCollection<>(managedOperatorState), + new StateObjectCollection<>(rawOperatorState), + new StateObjectCollection<>(managedKeyedState), + new StateObjectCollection<>(rawKeyedState)); } } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java index 53d329b1e0242..557c097ffb782 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java @@ -104,7 +104,7 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog public TestName name = new TestName(); private StateBackendEnum stateBackendEnum; - private AbstractStateBackend stateBackend; + protected AbstractStateBackend stateBackend; AbstractEventTimeWindowCheckpointingITCase(StateBackendEnum stateBackendEnum) { this.stateBackendEnum = stateBackendEnum; @@ -128,23 +128,7 @@ public void startTestCluster() throws Exception { zkServer.start(); } - TemporaryFolder temporaryFolder = new TemporaryFolder(); - temporaryFolder.create(); - final File haDir = temporaryFolder.newFolder(); - - Configuration config = new Configuration(); - config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM / 2); - config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 48L); - // the default network buffers size (10% of heap max =~ 150MB) seems to much for this test case - config.setLong(TaskManagerOptions.NETWORK_BUFFERS_MEMORY_MAX, 80L << 20); // 80 MB - config.setString(AkkaOptions.FRAMESIZE, String.valueOf(MAX_MEM_STATE_SIZE) + "b"); - - if (zkServer != null) { - config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER"); - config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zkServer.getConnectString()); - config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, haDir.toURI().toString()); - } + Configuration config = createClusterConfig(); // purposefully delay in the executor to tease out races final ScheduledExecutorService executor = Executors.newScheduledThreadPool(10); @@ -208,6 +192,27 @@ public void execute(Runnable command) { } } + protected Configuration createClusterConfig() throws IOException { + TemporaryFolder temporaryFolder = new TemporaryFolder(); + temporaryFolder.create(); + final File haDir = temporaryFolder.newFolder(); + + Configuration config = new Configuration(); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM / 2); + config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 48L); + // the default network buffers size (10% of heap max =~ 150MB) seems to much for this test case + config.setLong(TaskManagerOptions.NETWORK_BUFFERS_MEMORY_MAX, 80L << 20); // 80 MB + config.setString(AkkaOptions.FRAMESIZE, String.valueOf(MAX_MEM_STATE_SIZE) + "b"); + + if (zkServer != null) { + config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER"); + config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zkServer.getConnectString()); + config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, haDir.toURI().toString()); + } + return config; + } + @After public void stopTestCluster() throws IOException { if (cluster != null) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/LocalRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/LocalRecoveryITCase.java new file mode 100644 index 0000000000000..51b3b8437eb9f --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/LocalRecoveryITCase.java @@ -0,0 +1,120 @@ +/* + * 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.test.checkpointing; + +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.TestLogger; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestName; + +import java.io.IOException; + +import static org.apache.flink.runtime.state.LocalRecoveryConfig.LocalRecoveryMode; +import static org.apache.flink.runtime.state.LocalRecoveryConfig.LocalRecoveryMode.ENABLE_FILE_BASED; +import static org.apache.flink.test.checkpointing.AbstractEventTimeWindowCheckpointingITCase.StateBackendEnum; +import static org.apache.flink.test.checkpointing.AbstractEventTimeWindowCheckpointingITCase.StateBackendEnum.FILE_ASYNC; +import static org.apache.flink.test.checkpointing.AbstractEventTimeWindowCheckpointingITCase.StateBackendEnum.ROCKSDB_FULLY_ASYNC; +import static org.apache.flink.test.checkpointing.AbstractEventTimeWindowCheckpointingITCase.StateBackendEnum.ROCKSDB_INCREMENTAL_ZK; + +/** + * This test delegates to instances of {@link AbstractEventTimeWindowCheckpointingITCase} that have been reconfigured + * to use local recovery. + */ +public class LocalRecoveryITCase extends TestLogger { + + @Rule + public TestName testName = new TestName(); + + @Test + public void testLocalRecoveryHeapBackendFileBased() throws Exception { + executeTest( + FILE_ASYNC, + ENABLE_FILE_BASED); + } + + @Test + public void testLocalRecoveryRocksIncrementalFileBased() throws Exception { + executeTest( + ROCKSDB_INCREMENTAL_ZK, + ENABLE_FILE_BASED); + } + + @Test + public void testLocalRecoveryRocksFullFileBased() throws Exception { + executeTest( + ROCKSDB_FULLY_ASYNC, + ENABLE_FILE_BASED); + } + + private void executeTest( + StateBackendEnum backendEnum, + LocalRecoveryMode recoveryMode) throws Exception { + + AbstractEventTimeWindowCheckpointingITCase windowChkITCase = + new AbstractEventTimeWindowCheckpointingITCaseWithLocalRecovery( + backendEnum, + recoveryMode); + + executeTest(windowChkITCase); + } + + private void executeTest(AbstractEventTimeWindowCheckpointingITCase delegate) throws Exception { + delegate.name = testName; + delegate.tempFolder.create(); + try { + delegate.startTestCluster(); + delegate.testTumblingTimeWindow(); + delegate.stopTestCluster(); + + delegate.startTestCluster(); + delegate.testSlidingTimeWindow(); + delegate.stopTestCluster(); + } finally { + delegate.tempFolder.delete(); + } + } + + private static class AbstractEventTimeWindowCheckpointingITCaseWithLocalRecovery + extends AbstractEventTimeWindowCheckpointingITCase { + + private final LocalRecoveryMode recoveryMode; + + AbstractEventTimeWindowCheckpointingITCaseWithLocalRecovery( + StateBackendEnum stateBackendEnum, + LocalRecoveryMode recoveryMode) { + + super(stateBackendEnum); + this.recoveryMode = recoveryMode; + } + + @Override + protected Configuration createClusterConfig() throws IOException { + Configuration config = super.createClusterConfig(); + + config.setString( + CheckpointingOptions.LOCAL_RECOVERY, + recoveryMode.toString()); + + return config; + } + } +} diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java index 95763bffc2cb4..a23c679e65f2b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java @@ -550,7 +550,6 @@ public void testSavepointRescalingPartitionedOperatorState(boolean scaleOut, Ope if (savepointResponse instanceof JobManagerMessages.TriggerSavepointSuccess) { break; } - System.out.println(savepointResponse); } assertTrue(savepointResponse instanceof JobManagerMessages.TriggerSavepointSuccess); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java index fba3b934c0612..537f8641e1dde 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.state.LocalRecoveryConfig; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.runtime.testingUtils.TestingCluster; @@ -44,6 +45,7 @@ import org.junit.rules.TemporaryFolder; import java.io.File; +import java.io.IOException; import java.util.concurrent.CountDownLatch; /** @@ -69,7 +71,8 @@ public void testExternalizedIncrementalRocksDBCheckpointsStandalone() throws Exc testExternalizedCheckpoints( checkpointDir, null, - new RocksDBStateBackend(checkpointDir.toURI().toString(), true)); + createRocksDBStateBackend(checkpointDir, true), + false); } @Test @@ -78,7 +81,28 @@ public void testExternalizedFullRocksDBCheckpointsStandalone() throws Exception testExternalizedCheckpoints( checkpointDir, null, - new RocksDBStateBackend(checkpointDir.toURI().toString(), false)); + createRocksDBStateBackend(checkpointDir, false), + false); + } + + @Test + public void testExternalizedIncrementalRocksDBCheckpointsWithLocalRecoveryStandalone() throws Exception { + final File checkpointDir = temporaryFolder.newFolder(); + testExternalizedCheckpoints( + checkpointDir, + null, + createRocksDBStateBackend(checkpointDir, true), + true); + } + + @Test + public void testExternalizedFullRocksDBCheckpointsWithLocalRecoveryStandalone() throws Exception { + final File checkpointDir = temporaryFolder.newFolder(); + testExternalizedCheckpoints( + checkpointDir, + null, + createRocksDBStateBackend(checkpointDir, false), + true); } @Test @@ -87,8 +111,18 @@ public void testExternalizedFSCheckpointsStandalone() throws Exception { testExternalizedCheckpoints( checkpointDir, null, - new FsStateBackend(checkpointDir.toURI().toString(), true)); + createFsStateBackend(checkpointDir), + false); + } + @Test + public void testExternalizedFSCheckpointsWithLocalRecoveryStandalone() throws Exception { + final File checkpointDir = temporaryFolder.newFolder(); + testExternalizedCheckpoints( + checkpointDir, + null, + createFsStateBackend(checkpointDir), + true); } @Test @@ -100,7 +134,8 @@ public void testExternalizedIncrementalRocksDBCheckpointsZookeeper() throws Exce testExternalizedCheckpoints( checkpointDir, zkServer.getConnectString(), - new RocksDBStateBackend(checkpointDir.toURI().toString(), true)); + createRocksDBStateBackend(checkpointDir, true), + false); } finally { zkServer.stop(); } @@ -115,7 +150,40 @@ public void testExternalizedFullRocksDBCheckpointsZookeeper() throws Exception { testExternalizedCheckpoints( checkpointDir, zkServer.getConnectString(), - new RocksDBStateBackend(checkpointDir.toURI().toString(), false)); + createRocksDBStateBackend(checkpointDir, false), + false); + } finally { + zkServer.stop(); + } + } + + @Test + public void testExternalizedIncrementalRocksDBCheckpointsWithLocalRecoveryZookeeper() throws Exception { + TestingServer zkServer = new TestingServer(); + zkServer.start(); + try { + final File checkpointDir = temporaryFolder.newFolder(); + testExternalizedCheckpoints( + checkpointDir, + zkServer.getConnectString(), + createRocksDBStateBackend(checkpointDir, true), + true); + } finally { + zkServer.stop(); + } + } + + @Test + public void testExternalizedFullRocksDBCheckpointsWithLocalRecoveryZookeeper() throws Exception { + TestingServer zkServer = new TestingServer(); + zkServer.start(); + try { + final File checkpointDir = temporaryFolder.newFolder(); + testExternalizedCheckpoints( + checkpointDir, + zkServer.getConnectString(), + createRocksDBStateBackend(checkpointDir, false), + true); } finally { zkServer.stop(); } @@ -130,16 +198,45 @@ public void testExternalizedFSCheckpointsZookeeper() throws Exception { testExternalizedCheckpoints( checkpointDir, zkServer.getConnectString(), - new FsStateBackend(checkpointDir.toURI().toString(), true)); + createFsStateBackend(checkpointDir), + false); } finally { zkServer.stop(); } } + @Test + public void testExternalizedFSCheckpointsWithLocalRecoveryZookeeper() throws Exception { + TestingServer zkServer = new TestingServer(); + zkServer.start(); + try { + final File checkpointDir = temporaryFolder.newFolder(); + testExternalizedCheckpoints( + checkpointDir, + zkServer.getConnectString(), + createFsStateBackend(checkpointDir), + true); + } finally { + zkServer.stop(); + } + } + + private FsStateBackend createFsStateBackend(File checkpointDir) throws IOException { + return new FsStateBackend(checkpointDir.toURI().toString(), true); + } + + private RocksDBStateBackend createRocksDBStateBackend( + File checkpointDir, + boolean incrementalCheckpointing) throws IOException { + + return new RocksDBStateBackend(checkpointDir.toURI().toString(), incrementalCheckpointing); + } + private void testExternalizedCheckpoints( File checkpointDir, String zooKeeperQuorum, - StateBackend backend) throws Exception { + StateBackend backend, + boolean localRecovery) throws Exception { final Configuration config = new Configuration(); @@ -152,6 +249,12 @@ private void testExternalizedCheckpoints( config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString()); config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString()); + if (localRecovery) { + config.setString( + CheckpointingOptions.LOCAL_RECOVERY, + LocalRecoveryConfig.LocalRecoveryMode.ENABLE_FILE_BASED.toString()); + } + // ZooKeeper recovery mode? if (zooKeeperQuorum != null) { final File haDir = temporaryFolder.newFolder(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/ManualWindowSpeedITCase.java b/flink-tests/src/test/java/org/apache/flink/test/state/ManualWindowSpeedITCase.java index d65c323b047e7..2968f13ae8432 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/ManualWindowSpeedITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/ManualWindowSpeedITCase.java @@ -64,7 +64,7 @@ public void testTumblingIngestionTimeWindowsWithFsBackend() throws Exception { String checkpoints = tempFolder.newFolder().toURI().toString(); env.setStateBackend(new FsStateBackend(checkpoints)); - env.addSource(new InfiniteTupleSource(10_000)) + env.addSource(new InfiniteTupleSource(1_000)) .keyBy(0) .timeWindow(Time.seconds(3)) .reduce(new ReduceFunction>() { diff --git a/flink-tests/src/test/java/org/apache/flink/test/typeserializerupgrade/PojoSerializerUpgradeTest.java b/flink-tests/src/test/java/org/apache/flink/test/typeserializerupgrade/PojoSerializerUpgradeTest.java index aed6663ce401e..aaa96fbaaad86 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/typeserializerupgrade/PojoSerializerUpgradeTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/typeserializerupgrade/PojoSerializerUpgradeTest.java @@ -348,7 +348,7 @@ private OperatorSubtaskState runOperator( boolean isKeyedState, StateBackend stateBackend, ClassLoader classLoader, - OperatorSubtaskState operatorStateHandles, + OperatorSubtaskState operatorSubtaskState, Iterable input) throws Exception { try (final MockEnvironment environment = new MockEnvironment( @@ -358,11 +358,11 @@ private OperatorSubtaskState runOperator( 256, taskConfiguration, executionConfig, + new TestTaskStateManager(), 16, 1, 0, - classLoader, - new TestTaskStateManager())) { + classLoader)) { OneInputStreamOperatorTestHarness harness = null; try { @@ -379,7 +379,7 @@ private OperatorSubtaskState runOperator( harness.setStateBackend(stateBackend); harness.setup(); - harness.initializeState(operatorStateHandles); + harness.initializeState(operatorSubtaskState); harness.open(); long timestamp = 0L; diff --git a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala index e33aa494ef397..9dd0b28211941 100644 --- a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala +++ b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnTaskManager.scala @@ -25,6 +25,7 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup import org.apache.flink.runtime.security.SecurityUtils +import org.apache.flink.runtime.state.TaskExecutorLocalStateStoresManager import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration import org.apache.flink.runtime.taskmanager.TaskManagerLocation import org.apache.flink.runtime.testingUtils.TestingTaskManagerLike @@ -40,6 +41,7 @@ import org.apache.flink.runtime.testingUtils.TestingTaskManagerLike * @param memoryManager MemoryManager which is responsible for Flink's managed memory allocation * @param ioManager IOManager responsible for I/O * @param network NetworkEnvironment for this actor + * @param taskManagerLocalStateStoresManager Task manager state store manager for this actor * @param numberOfSlots Number of slots for this TaskManager * @param highAvailabilityServices [[HighAvailabilityServices]] to create a leader retrieval * service for retrieving the leading JobManager @@ -51,6 +53,7 @@ class TestingYarnTaskManager( memoryManager: MemoryManager, ioManager: IOManager, network: NetworkEnvironment, + taskManagerLocalStateStoresManager: TaskExecutorLocalStateStoresManager, numberOfSlots: Int, highAvailabilityServices: HighAvailabilityServices, taskManagerMetricGroup : TaskManagerMetricGroup) @@ -61,6 +64,7 @@ class TestingYarnTaskManager( memoryManager, ioManager, network, + taskManagerLocalStateStoresManager, numberOfSlots, highAvailabilityServices, taskManagerMetricGroup) diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala index 08d24b440ba0e..d54ae9b8e292b 100644 --- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala @@ -25,6 +25,7 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment import org.apache.flink.runtime.memory.MemoryManager import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup import org.apache.flink.runtime.security.SecurityUtils +import org.apache.flink.runtime.state.TaskExecutorLocalStateStoresManager import org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerLocation} @@ -40,6 +41,7 @@ class YarnTaskManager( memoryManager: MemoryManager, ioManager: IOManager, network: NetworkEnvironment, + taskManagerLocalStateStoresManager: TaskExecutorLocalStateStoresManager, numberOfSlots: Int, highAvailabilityServices: HighAvailabilityServices, taskManagerMetricGroup: TaskManagerMetricGroup) @@ -50,6 +52,7 @@ class YarnTaskManager( memoryManager, ioManager, network, + taskManagerLocalStateStoresManager, numberOfSlots, highAvailabilityServices, taskManagerMetricGroup) {