From c384527a3a5668853334be08f5f3be98aaa43a8e Mon Sep 17 00:00:00 2001 From: summerleafs Date: Wed, 22 Nov 2017 13:03:21 +0800 Subject: [PATCH 1/7] introduce CheckpointCacheManager for local recovery. Add note and unit test for CheckpointCacheManager. add more tests for local recovery. add more tests for local recovery. add comment for SharedCacheRegistry. --- .../state/RocksDBKeyedStateBackend.java | 78 ++- .../streaming/state/RocksDBStateBackend.java | 2 + .../state/RocksDBStateBackendTest.java | 2 + .../clusterframework/MesosTaskManager.scala | 3 + .../KVStateRequestSerializerRocksDBTest.java | 6 +- .../checkpoint/CachedStreamStateHandle.java | 156 +++++ .../runtime/checkpoint/CheckpointCache.java | 596 ++++++++++++++++++ .../checkpoint/CheckpointCacheManager.java | 147 +++++ .../checkpoint/SharedCacheRegistry.java | 128 ++++ .../savepoint/SavepointV2Serializer.java | 13 + .../deployment/TaskDeploymentDescriptor.java | 12 +- .../executiongraph/ExecutionVertex.java | 6 +- .../state/CachedCheckpointStreamFactory.java | 176 ++++++ .../runtime/state/CachedStateHandle.java | 25 + .../runtime/state/KeyGroupsStateHandle.java | 9 + .../state/PlaceholderStreamStateHandle.java | 14 +- .../runtime/taskexecutor/TaskExecutor.java | 29 + .../taskexecutor/TaskManagerRunner.java | 1 + .../taskexecutor/TaskManagerServices.java | 19 +- .../taskmanager/RuntimeEnvironment.java | 9 + .../flink/runtime/taskmanager/Task.java | 13 +- .../minicluster/LocalFlinkMiniCluster.scala | 5 +- .../runtime/taskmanager/TaskManager.scala | 34 + .../CachedCheckpointStreamFactoryTest.java | 81 +++ .../CheckpointCacheManagerTest.java | 139 ++++ .../checkpoint/CheckpointCacheTest.java | 297 +++++++++ .../TaskDeploymentDescriptorTest.java | 3 +- .../metrics/TaskManagerMetricsTest.java | 1 + .../state/SharedCacheRegistryTest.java | 116 ++++ .../taskexecutor/TaskExecutorITCase.java | 2 + .../taskexecutor/TaskExecutorTest.java | 18 +- .../taskmanager/TaskAsyncCallTest.java | 2 + .../runtime/taskmanager/TaskManagerTest.java | 3 +- .../flink/runtime/taskmanager/TaskTest.java | 2 + .../runtime/util/JvmExitOnFatalErrorTest.java | 2 + .../testingUtils/TestingTaskManager.scala | 5 + .../tasks/InterruptSensitiveRestoreTest.java | 2 + .../tasks/StreamTaskTerminationTest.java | 2 + .../runtime/tasks/StreamTaskTest.java | 2 + .../tasks/TaskCheckpointingBehaviourTest.java | 2 + .../flink/yarn/TestingYarnTaskManager.scala | 3 + .../apache/flink/yarn/YarnTaskManager.scala | 3 + 42 files changed, 2134 insertions(+), 34 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CachedStreamStateHandle.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManager.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SharedCacheRegistry.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedCheckpointStreamFactory.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedStateHandle.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CachedCheckpointStreamFactoryTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManagerTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/state/SharedCacheRegistryTest.java diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index 9185ad02b8a33..dee1accd3d9e3 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -44,11 +44,14 @@ import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.runtime.checkpoint.CachedStreamStateHandle; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.io.async.AbstractAsyncCallableWithResources; import org.apache.flink.runtime.io.async.AsyncStoppableTaskWithCallback; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.AbstractKeyedStateBackend; +import org.apache.flink.runtime.state.CachedCheckpointStreamFactory; import org.apache.flink.runtime.state.CheckpointStreamFactory; import org.apache.flink.runtime.state.DoneFuture; import org.apache.flink.runtime.state.IncrementalKeyedStateHandle; @@ -205,6 +208,8 @@ public class RocksDBKeyedStateBackend extends AbstractKeyedStateBackend { /** Unique ID of this backend. */ private UUID backendUID; + private CheckpointCache cache; + public RocksDBKeyedStateBackend( String operatorIdentifier, ClassLoader userCodeClassLoader, @@ -216,6 +221,7 @@ public RocksDBKeyedStateBackend( int numberOfKeyGroups, KeyGroupRange keyGroupRange, ExecutionConfig executionConfig, + CheckpointCache cache, boolean enableIncrementalCheckpointing ) throws IOException { @@ -223,6 +229,8 @@ public RocksDBKeyedStateBackend( this.operatorIdentifier = Preconditions.checkNotNull(operatorIdentifier); + this.cache = cache; + this.enableIncrementalCheckpointing = enableIncrementalCheckpointing; this.rocksDBResourceGuard = new ResourceGuard(); @@ -370,7 +378,7 @@ private RunnableFuture snapshotIncrementally( final RocksDBIncrementalSnapshotOperation snapshotOperation = new RocksDBIncrementalSnapshotOperation<>( this, - checkpointStreamFactory, + new CachedCheckpointStreamFactory(cache, checkpointStreamFactory), checkpointId, checkpointTimestamp); @@ -416,7 +424,10 @@ private RunnableFuture snapshotFully( return DoneFuture.nullValue(); } - snapshotOperation = new RocksDBFullSnapshotOperation<>(this, streamFactory, snapshotCloseableRegistry); + snapshotOperation = new RocksDBFullSnapshotOperation<>( + this, + new CachedCheckpointStreamFactory(cache, streamFactory), + snapshotCloseableRegistry); snapshotOperation.takeDBSnapShot(checkpointId, timestamp); // implementation of the async IO operation, based on FutureTask @@ -488,7 +499,7 @@ static final class RocksDBFullSnapshotOperation { private final RocksDBKeyedStateBackend stateBackend; private final KeyGroupRangeOffsets keyGroupRangeOffsets; - private final CheckpointStreamFactory checkpointStreamFactory; + private final CachedCheckpointStreamFactory checkpointStreamFactory; private final CloseableRegistry snapshotCloseableRegistry; private final ResourceGuard.Lease dbLease; @@ -504,7 +515,7 @@ static final class RocksDBFullSnapshotOperation { RocksDBFullSnapshotOperation( RocksDBKeyedStateBackend stateBackend, - CheckpointStreamFactory checkpointStreamFactory, + CachedCheckpointStreamFactory checkpointStreamFactory, CloseableRegistry registry) throws IOException { this.stateBackend = stateBackend; @@ -535,7 +546,8 @@ public void takeDBSnapShot(long checkpointId, long checkpointTimeStamp) { */ public void openCheckpointStream() throws Exception { Preconditions.checkArgument(outStream == null, "Output stream for snapshot is already set."); - outStream = checkpointStreamFactory.createCheckpointStateOutputStream(checkpointId, checkpointTimeStamp); + outStream = checkpointStreamFactory. + createCheckpointStateOutputStream(checkpointId, checkpointTimeStamp, new StateHandleID(stateBackend.backendUID + "$" + checkpointId)); snapshotCloseableRegistry.registerCloseable(outStream); outputView = new DataOutputViewStreamWrapper(outStream); } @@ -754,7 +766,7 @@ private static final class RocksDBIncrementalSnapshotOperation { private final RocksDBKeyedStateBackend stateBackend; /** Stream factory that creates the outpus streams to DFS. */ - private final CheckpointStreamFactory checkpointStreamFactory; + private final CachedCheckpointStreamFactory checkpointStreamFactory; /** Id for the current checkpoint. */ private final long checkpointId; @@ -787,7 +799,7 @@ private static final class RocksDBIncrementalSnapshotOperation { private RocksDBIncrementalSnapshotOperation( RocksDBKeyedStateBackend stateBackend, - CheckpointStreamFactory checkpointStreamFactory, + CachedCheckpointStreamFactory checkpointStreamFactory, long checkpointId, long checkpointTimestamp) throws IOException { @@ -798,19 +810,27 @@ private RocksDBIncrementalSnapshotOperation( this.dbLease = this.stateBackend.rocksDBResourceGuard.acquireResource(); } - private StreamStateHandle materializeStateData(Path filePath) throws Exception { + private StreamStateHandle materializeStateData(StateHandleID handleID, Path filePath) throws Exception { FSDataInputStream inputStream = null; CheckpointStreamFactory.CheckpointStateOutputStream outputStream = null; + //fast path for placeholder + if (filePath == null) { + outputStream = checkpointStreamFactory + .createCheckpointStateOutputStream(checkpointId, checkpointTimestamp, handleID, true); + outputStream.closeAndGetHandle(); + outputStream = null; + return new PlaceholderStreamStateHandle(handleID); + } + try { final byte[] buffer = new byte[8 * 1024]; FileSystem backupFileSystem = backupPath.getFileSystem(); inputStream = backupFileSystem.open(filePath); closeableRegistry.registerCloseable(inputStream); - outputStream = checkpointStreamFactory - .createCheckpointStateOutputStream(checkpointId, checkpointTimestamp); + .createCheckpointStateOutputStream(checkpointId, checkpointTimestamp, handleID); closeableRegistry.registerCloseable(outputStream); while (true) { @@ -846,7 +866,7 @@ private StreamStateHandle materializeMetaData() throws Exception { try { outputStream = checkpointStreamFactory - .createCheckpointStateOutputStream(checkpointId, checkpointTimestamp); + .createCheckpointStateOutputStream(checkpointId, checkpointTimestamp, new StateHandleID(stateBackend.backendUID + "$" + checkpointId + "meta")); closeableRegistry.registerCloseable(outputStream); //no need for compression scheme support because sst-files are already compressed @@ -924,23 +944,23 @@ KeyedStateHandle materializeSnapshot() throws Exception { for (FileStatus fileStatus : fileStatuses) { final Path filePath = fileStatus.getPath(); final String fileName = filePath.getName(); - final StateHandleID stateHandleID = new StateHandleID(fileName); if (fileName.endsWith(SST_FILE_SUFFIX)) { + final StateHandleID stateHandleID = new StateHandleID(stateBackend.backendUID + "$" + fileName); final boolean existsAlready = baseSstFiles != null && baseSstFiles.contains(stateHandleID); if (existsAlready) { // we introduce a placeholder state handle, that is replaced with the // original from the shared state registry (created from a previous checkpoint) - sstFiles.put( - stateHandleID, - new PlaceholderStreamStateHandle()); + // materializeStateData should return a placeholder in a fast path. + sstFiles.put(stateHandleID, materializeStateData(stateHandleID, null)); } else { - sstFiles.put(stateHandleID, materializeStateData(filePath)); + sstFiles.put(stateHandleID, materializeStateData(stateHandleID, filePath)); } } else { - StreamStateHandle fileHandle = materializeStateData(filePath); + final StateHandleID stateHandleID = new StateHandleID(stateBackend.backendUID + "$" + checkpointId + "$" + fileName); + StreamStateHandle fileHandle = materializeStateData(stateHandleID, filePath); miscFiles.put(stateHandleID, fileHandle); } } @@ -1136,6 +1156,7 @@ public void doRestore(Collection keyedStateHandles) ", but found: " + keyedStateHandle.getClass()); } this.currentKeyGroupsStateHandle = (KeyGroupsStateHandle) keyedStateHandle; + this.currentKeyGroupsStateHandle.setCache(rocksDBKeyedStateBackend.cache); restoreKeyGroupsInStateHandle(); } } @@ -1289,11 +1310,16 @@ private RocksDBIncrementalRestoreOperation(RocksDBKeyedStateBackend stateBack } private List> readMetaData( - StreamStateHandle metaStateHandle) throws Exception { + StreamStateHandle metaStateHandle, + boolean hasExtraKeys) throws Exception { FSDataInputStream inputStream = null; try { + if (metaStateHandle instanceof CachedStreamStateHandle) { + ((CachedStreamStateHandle) metaStateHandle).setCheckpointCache(stateBackend.cache); + ((CachedStreamStateHandle) metaStateHandle).reCache(!hasExtraKeys); + } inputStream = metaStateHandle.openInputStream(); stateBackend.cancelStreamRegistry.registerCloseable(inputStream); @@ -1375,12 +1401,12 @@ private void restoreInstance( final Map miscFiles = restoreStateHandle.getPrivateState(); - readAllStateData(sstFiles, restoreInstancePath); - readAllStateData(miscFiles, restoreInstancePath); + readAllStateData(sstFiles, restoreInstancePath, hasExtraKeys); + readAllStateData(miscFiles, restoreInstancePath, hasExtraKeys); // read meta data List> stateMetaInfoSnapshots = - readMetaData(restoreStateHandle.getMetaStateHandle()); + readMetaData(restoreStateHandle.getMetaStateHandle(), hasExtraKeys); List columnFamilyDescriptors = new ArrayList<>(1 + stateMetaInfoSnapshots.size()); @@ -1516,6 +1542,9 @@ private void restoreInstance( } stateBackend.lastCompletedCheckpointId = restoreStateHandle.getCheckpointId(); + + // commit re-cache + stateBackend.cache.commitCache(CheckpointCache.CHECKPOINT_ID_FOR_RESTORE, false); } } finally { FileSystem restoreFileSystem = restoreInstancePath.getFileSystem(); @@ -1527,11 +1556,16 @@ private void restoreInstance( private void readAllStateData( Map stateHandleMap, - Path restoreInstancePath) throws IOException { + Path restoreInstancePath, + boolean hasExtraKeys) throws IOException { for (Map.Entry entry : stateHandleMap.entrySet()) { StateHandleID stateHandleID = entry.getKey(); StreamStateHandle remoteFileHandle = entry.getValue(); + if (remoteFileHandle instanceof CachedStreamStateHandle) { + ((CachedStreamStateHandle) remoteFileHandle).setCheckpointCache(stateBackend.cache); + ((CachedStreamStateHandle) remoteFileHandle).reCache(!hasExtraKeys); + } readStateData(new Path(restoreInstancePath, stateHandleID.toString()), remoteFileHandle); } } diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java index 6ec7ec836c04c..8301b4455142f 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.OperatorStateBackend; import org.apache.flink.runtime.state.filesystem.FsStateBackend; +import org.apache.flink.runtime.taskmanager.RuntimeEnvironment; import org.apache.flink.util.AbstractID; import org.rocksdb.ColumnFamilyOptions; @@ -313,6 +314,7 @@ public AbstractKeyedStateBackend createKeyedStateBackend( numberOfKeyGroups, keyGroupRange, env.getExecutionConfig(), + ((RuntimeEnvironment) env).getCheckpointCache(), enableIncrementalCheckpointing); } diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java index e1be744a9701a..e212b4a3fa938 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.operators.testutils.DummyEnvironment; @@ -231,6 +232,7 @@ public void testCorrectMergeOperatorSet() throws IOException { 1, new KeyGroupRange(0, 0), new ExecutionConfig(), + mock(CheckpointCache.class), enableIncrementalCheckpointing); verify(columnFamilyOptions, Mockito.times(1)) 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..f0e56ce14603b 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 @@ -18,6 +18,7 @@ package org.apache.flink.mesos.runtime.clusterframework +import org.apache.flink.runtime.checkpoint.CheckpointCacheManager import org.apache.flink.runtime.clusterframework.types.ResourceID import org.apache.flink.runtime.highavailability.HighAvailabilityServices import org.apache.flink.runtime.io.disk.iomanager.IOManager @@ -37,6 +38,7 @@ class MesosTaskManager( memoryManager: MemoryManager, ioManager: IOManager, network: NetworkEnvironment, + checkpointCacheManager: CheckpointCacheManager, numberOfSlots: Int, highAvailabilityServices: HighAvailabilityServices, taskManagerMetricGroup : TaskManagerMetricGroup) @@ -47,6 +49,7 @@ class MesosTaskManager( memoryManager, ioManager, network, + checkpointCacheManager, 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..109a1ce4d1291 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 @@ -28,6 +28,7 @@ import org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend; import org.apache.flink.queryablestate.client.VoidNamespace; import org.apache.flink.queryablestate.client.VoidNamespaceSerializer; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.internal.InternalListState; @@ -76,7 +77,9 @@ static final class RocksDBKeyedStateBackend2 extends RocksDBKeyedStateBackend super(operatorIdentifier, userCodeClassLoader, instanceBasePath, dbOptions, columnFamilyOptions, kvStateRegistry, keySerializer, - numberOfKeyGroups, keyGroupRange, executionConfig, false); + numberOfKeyGroups, keyGroupRange, executionConfig, + mock(CheckpointCache.class), + false); } @Override @@ -152,6 +155,7 @@ public void testMapSerialization() throws Exception { LongSerializer.INSTANCE, 1, new KeyGroupRange(0, 0), new ExecutionConfig(), + mock(CheckpointCache.class), false); longHeapKeyedStateBackend.restore(null); longHeapKeyedStateBackend.setCurrentKey(key); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CachedStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CachedStreamStateHandle.java new file mode 100644 index 0000000000000..72534421b08b4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CachedStreamStateHandle.java @@ -0,0 +1,156 @@ +/* + * 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.core.fs.FSDataInputStream; +import org.apache.flink.runtime.checkpoint.CheckpointCache.CachedOutputStream; +import org.apache.flink.runtime.state.CachedStateHandle; +import org.apache.flink.runtime.state.StateHandleID; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.util.FlinkRuntimeException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/** + * It mainly has two member fields: + * 1, cacheId {@link StateHandleID} + * 2, remoteHandle {@link StreamStateHandle} + * When building the input stream via {@link CachedStreamStateHandle}, it first try to build the local input stream from {@link CheckpointCache} + * based on the cache id, and use remoteHandle to build the input stream from the remote end only when building the local input stream failed. + */ +public class CachedStreamStateHandle implements StreamStateHandle, CachedStateHandle { + + private static Logger LOG = LoggerFactory.getLogger(CachedStreamStateHandle.class); + + private transient CheckpointCache cache; + private transient boolean reCache; + + private final StateHandleID cacheId; + private final StreamStateHandle remoteHandle; + + public CachedStreamStateHandle(StateHandleID cacheId, StreamStateHandle remoteHandle) { + this.cacheId = cacheId; + this.remoteHandle = remoteHandle; + } + + @Override + public FSDataInputStream openInputStream() throws IOException { + FSDataInputStream in = cache.openInputStream(cacheId); + if (in != null) { + LOG.info("Open the local input stream."); + return in; + } + + CachedOutputStream output = null; + if (reCache) { + output = cache.createOutputStream(CheckpointCache.CHECKPOINT_ID_FOR_RESTORE, cacheId); + } + LOG.info("Open the remote input stream, re-cache: {}.", reCache); + return new CachedInputStream(remoteHandle.openInputStream(), output); + } + + @Override + public void discardState() throws Exception { + remoteHandle.discardState(); + } + + @Override + public long getStateSize() { + return remoteHandle.getStateSize(); + } + + public StreamStateHandle getRemoteHandle() { + return this.remoteHandle; + } + + public void setCheckpointCache(CheckpointCache cache) { + this.cache = cache; + } + + public void reCache(boolean reCache) { + this.reCache = reCache; + } + + @Override + public StateHandleID getStateHandleId() { + return cacheId; + } + + public static class CachedInputStream extends FSDataInputStream { + + private final FSDataInputStream remoteInputStream ; + private final CachedOutputStream cacheOut; + + public CachedInputStream(FSDataInputStream fsDataInputStream, CachedOutputStream output) { + this.cacheOut = output; + this.remoteInputStream = fsDataInputStream; + } + + @Override + public int read() throws IOException { + int o = this.remoteInputStream.read(); + if (o != -1) { + //re-cache and ignore exception + if (cacheOut != null && !cacheOut.isDiscarded()) { + try { + cacheOut.write(o); + } catch (Exception ignore) { + cacheOut.discard(); + } + } + } + return o; + } + + @Override + public int read(byte[] b) throws IOException { + int n = this.remoteInputStream.read(b); + if (n != -1) { + //re-cache and ignore exception + if (cacheOut != null && !cacheOut.isDiscarded()) { + try { + this.cacheOut.write(b, 0, n); + } catch (Exception ignore) { + cacheOut.discard(); + } + } + } + return n; + } + + @Override + public void seek(long desired) throws IOException { + throw new FlinkRuntimeException("Unsupported method in CachedSteamStateHandle."); + } + + @Override + public long getPos() throws IOException { + throw new FlinkRuntimeException("Unsupported method in CachedSteamStateHandle."); + } + + @Override + public void close() throws IOException { + if (this.cacheOut != null) { + this.cacheOut.end(); + } + this.remoteInputStream.close(); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java new file mode 100644 index 0000000000000..f858814f12073 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java @@ -0,0 +1,596 @@ +/* + * 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.VisibleForTesting; +import org.apache.flink.api.common.JobID; +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.state.StateHandleID; +import org.apache.flink.runtime.state.filesystem.FileStateHandle; +import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory; +import org.apache.flink.util.FlinkRuntimeException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.util.ArrayDeque; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Created from {@link CheckpointCacheManager} and used by {@link org.apache.flink.runtime.taskmanager.Task} to cache checkpoint data locally, + * it maintains the last success completed checkpoint cache and uses {@link SharedCacheRegistry} + * to manage the cache entry, backend uses the output stream to transmission data to both network and local, + * local data maintained by {@link PendingCheckpointCache}, we turn {@link PendingCheckpointCache} into {@link CompletedCheckpointCache} + * in notifyCheckpointComplete(). + */ +public class CheckpointCache { + + private static Logger LOG = LoggerFactory.getLogger(CheckpointCache.class); + + // a special checkpoint id used when do re-cache for incremental checkpoint + public static final long CHECKPOINT_ID_FOR_RESTORE = -1L; + + // max retain completed cache num + public static final int MAX_RETAIN_NUM = 1; + + // cache manager + private final CheckpointCacheManager cacheManager; + private final JobID jobID; + private final Path basePath; + private final AtomicInteger reference; + private final Object lock = new Object(); + + // pending checkpoint cache timeout + private final long pendingCheckpointCacheTimeout; + + //checkpoint cache lease ttl + private final long leaseTimeout; + + // executor for dispose resource + private final Executor executor; + + private final ScheduledThreadPoolExecutor timer; + + // pending cache map + private final Map pendingCacheMap; + + // completed checkpoint cache map + private final ArrayDeque completedCheckpointCaches; + + // shared cache registry + private final SharedCacheRegistry sharedCacheRegistry; + + public CheckpointCache(JobID jobID, String basePath, long pendingCheckpointCacheTimeout, long leaseTimeout, CheckpointCacheManager manager, Executor executor) { + this.basePath = new Path(basePath + File.separator + "checkpoint_cache" + File.separator + jobID); + try { + this.basePath.getFileSystem().mkdirs(this.basePath); + } catch (IOException e) { + throw new FlinkRuntimeException("init checkpoint cache manager failed:{}", e); + } + this.cacheManager = manager; + this.jobID = jobID; + this.reference = new AtomicInteger(0); + this.pendingCheckpointCacheTimeout = pendingCheckpointCacheTimeout; + this.leaseTimeout = leaseTimeout; + this.executor = executor; + this.timer = new ScheduledThreadPoolExecutor(1, + new DispatcherThreadFactory(Thread.currentThread().getThreadGroup(), "Checkpoint Cache Timer")); + this.pendingCacheMap = new ConcurrentHashMap<>(); + this.completedCheckpointCaches = new ArrayDeque<>(MAX_RETAIN_NUM + 1); + this.sharedCacheRegistry = new SharedCacheRegistry(executor); + + LOG.info("new checkpoint cache, pendingCacheTimeout: {}, leaseTimeout: {}", pendingCheckpointCacheTimeout, leaseTimeout); + } + + protected void registerCacheEntry(long checkpointID, StateHandleID handleID, String filePath) { + synchronized (lock) { + LOG.debug("register cache entry: { cpkID:[{}] handleID:[{}] }", checkpointID, handleID); + PendingCheckpointCache pendingCheckpointCache = pendingCacheMap.get(checkpointID); + if (pendingCheckpointCache == null) { + PendingCheckpointCache newPendingCheckpointCache = new PendingCheckpointCache(executor, checkpointID); + LOG.debug("add pending cache map: { cpkID:[{}] }", checkpointID); + pendingCacheMap.put(checkpointID, newPendingCheckpointCache); + pendingCheckpointCache = newPendingCheckpointCache; + + // schedule the timer that will clean up the expired checkpoints + ScheduledFuture cancellerHandle = timer.schedule( + () -> { + synchronized (lock) { + if (!newPendingCheckpointCache.isDiscarded()) { + LOG.info("Checkpoint cache " + checkpointID + " expired before completing."); + newPendingCheckpointCache.abortExpired(); + pendingCacheMap.remove(checkpointID); + } + } + }, + pendingCheckpointCacheTimeout, TimeUnit.MILLISECONDS); + + if (!newPendingCheckpointCache.setCancellerHandle(cancellerHandle)) { + cancellerHandle.cancel(false); + } + } + + pendingCheckpointCache.addEntry( + new CacheKey(handleID), + new CacheEntry(new FileStateHandle(new Path(filePath), getFileSize(filePath)))); + } + } + + private long getFileSize(String filePath) { + File file = new File(filePath); + if (file.exists() && file.isFile()) { + return file.length(); + } else { + return 0L; + } + } + + protected void abortPendingCache(long checkpointID) { + LOG.info("abort pending cache: {}", checkpointID); + synchronized (lock) { + PendingCheckpointCache pendingCheckpointCache = pendingCacheMap.get(checkpointID); + if (pendingCheckpointCache != null) { + pendingCheckpointCache.abortSubsumed(); + } + } + } + + public void commitCache(long checkpointID) { + commitCache(checkpointID, true); + } + + public void commitCache(long checkpointID, boolean dropUnRetainCheckpointCache) { + synchronized (lock) { + final PendingCheckpointCache pendingCheckpointCache; + pendingCheckpointCache = pendingCacheMap.remove(checkpointID); + if (pendingCheckpointCache != null) { + LOG.info("commit pending checkpoint cache: {}", checkpointID); + // here will build reference on cache entry + CompletedCheckpointCache completedCheckpointCache = new CompletedCheckpointCache(sharedCacheRegistry, checkpointID); + for (Map.Entry entry : pendingCheckpointCache.getEntryIterable()) { + completedCheckpointCache.addCacheEntry(entry.getKey(), entry.getValue()); + } + + this.completedCheckpointCaches.add(completedCheckpointCache); + + if (dropUnRetainCheckpointCache) { + // only maintain the last complete checkpoint + dropUnRetainCheckpointCache(MAX_RETAIN_NUM); + } + + // subsume pending checkpoint cache + dropSubsumedPendingCheckpointCache(checkpointID); + } else { + LOG.debug("{} pending checkpoint cache is not exists. This means it has been committed or expired", checkpointID); + } + } + } + + public int getPendingCheckpointCacheSize() { + synchronized (lock) { + return pendingCacheMap.size(); + } + } + + public int getCompletedCheckpointCacheSize() { + synchronized (lock) { + return completedCheckpointCaches.size(); + } + } + + private void dropUnRetainCheckpointCache(int maxRetainNum) { + while (this.completedCheckpointCaches.size() > maxRetainNum) { + CompletedCheckpointCache completedCheckpointCache = completedCheckpointCaches.removeFirst(); + LOG.debug("remove checkpoint cache:{}", completedCheckpointCache.getCheckpointID()); + completedCheckpointCache.discard(); + } + } + + private void dropSubsumedPendingCheckpointCache(long checkpointID) { + Iterator> entries = this.pendingCacheMap.entrySet().iterator(); + while (entries.hasNext()) { + PendingCheckpointCache p = entries.next().getValue(); + // remove all pending checkpoints that are lesser than the current completed checkpoint + if (p.getCheckpointID() < checkpointID) { + LOG.debug("remove subsumed pending checkpoint: {} < {}", p.getCheckpointID(), checkpointID); + p.abortSubsumed(); + entries.remove(); + } + } + } + + public String getBasePath() { + return basePath.getPath(); + } + + public void discard() { + cacheManager.unregisterCheckpointCache(jobID); + } + + public void release() { + synchronized (lock) { + dropSubsumedPendingCheckpointCache(Long.MAX_VALUE); + dropUnRetainCheckpointCache(0); + } + } + + @VisibleForTesting + protected SharedCacheRegistry getSharedCacheRegister() { + return this.sharedCacheRegistry; + } + + public CachedOutputStream createOutputStream(long checkpointID, StateHandleID handleID) { + return createOutputStream(checkpointID, handleID, false); + } + + public CachedOutputStream createOutputStream(long checkpointID, StateHandleID handleID, boolean placeholder) { + LOG.debug("create cache output: {} {}", checkpointID, placeholder); + try { + File basePathDir = new File(this.basePath.getPath()); + //sanity check + if (!basePathDir.exists()) { + if (!basePathDir.mkdirs()) { + LOG.warn("init checkpoint cache base path {} failed.", this.basePath.getPath()); + return null; + } + } + final String cacheFilePath = basePath + File.separator + handleID + "_" + UUID.randomUUID(); + return new CachedOutputStream(checkpointID, handleID, cacheFilePath, this, placeholder); + } catch (Exception ignore) { + // warning + LOG.warn("create output stream failed: {}", ignore); + } + return null; + } + + public FSDataInputStream openInputStream(StateHandleID cacheId) { + LOG.debug("try to open input stream from cache with cacheID:" + cacheId); + CacheEntry entry = sharedCacheRegistry.getCacheEntry(new CacheKey(cacheId)); + if (entry != null) { + try { + LOG.debug("entry path: {}", entry.getHandle().getFilePath()); + return entry.getHandle().openInputStream(); + } catch (Exception ignore) { + entry.rot(true); + return null; + } + } + return null; + } + + public int reference() { + return this.reference.incrementAndGet(); + } + + public int dereference() { + return this.reference.decrementAndGet(); + } + + public int getReference() { + return this.reference.get(); + } + + public static class CachedOutputStream extends OutputStream { + + private final OutputStream outputStream; + private final StateHandleID cacheID; + private final long checkpointID; + + private final String cacheFilePath; + private final CheckpointCache cache; + private boolean discarded; + + public CachedOutputStream( + long checkpointID, + StateHandleID cacheID, + String cacheFilePath, + CheckpointCache cache, + boolean placeholder + ) throws FileNotFoundException { + this.checkpointID = checkpointID; + this.cacheID = cacheID; + this.cacheFilePath = cacheFilePath; + if (!placeholder) { + this.outputStream = new FileOutputStream(cacheFilePath); + } else { + this.outputStream = null; + } + this.cache = cache; + this.discarded = false; + } + + public long getCheckpointID() { + return this.checkpointID; + } + + public StateHandleID getCacheID() { + return this.cacheID; + } + + public boolean isDiscarded() { + return this.discarded; + } + + public void discard() { + LOG.info("cache output stream discard: {}", checkpointID); + discarded = true; + } + + @Override + public void write(int b) throws IOException { + if (!discarded && outputStream != null) { + outputStream.write(b); + } + } + + @Override + public void write(byte[] b) throws IOException { + if (!discarded && outputStream != null) { + outputStream.write(b); + } + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + if (!discarded && outputStream != null) { + outputStream.write(b, off, len); + } + } + + @Override + public void flush() throws IOException { + if (!discarded && outputStream != null) { + outputStream.flush(); + } + } + + @Override + public void close() throws IOException { + if (outputStream != null) { + outputStream.close(); + } + } + + public void end() { + if (!discarded) { + this.cache.registerCacheEntry(checkpointID, cacheID, cacheFilePath); + } else { + this.cache.abortPendingCache(checkpointID); + } + } + + public String getCacheFilePath() { + return cacheFilePath; + } + } + + public long getLeaseTimeout() { + return leaseTimeout; + } + + public static class CacheKey { + private StateHandleID handleID; + public CacheKey(StateHandleID handleID) { + this.handleID = handleID; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + CacheKey cacheKey = (CacheKey) o; + + return handleID != null ? handleID.equals(cacheKey.handleID) : cacheKey.handleID == null; + } + + @Override + public int hashCode() { + int result = handleID != null ? handleID.hashCode() : 0; + return result; + } + + @Override + public String toString() { + return handleID.toString(); + } + } + + public static class CacheEntry { + private final FileStateHandle handle; + private final AtomicInteger reference; + + public boolean isRot() { + return rot; + } + + public void rot(boolean isRot) { + this.rot = isRot; + } + + private boolean rot; + + public CacheEntry(FileStateHandle handle) { + this.handle = handle; + this.reference = new AtomicInteger(0); + this.rot = false; + } + + public FileStateHandle getHandle() { + return handle; + } + + public void discard() { + try { + //TODO: this should discard with future. + this.handle.discardState(); + } catch (Exception e) { + LOG.warn("discard handle failed: {}", e); + } + } + + public int increaseReferenceCount() { + return reference.incrementAndGet(); + } + + public int decreaseReferenceCount() { + return reference.decrementAndGet(); + } + + public int getReferenceCount() { + return reference.get(); + } + + public void setReference(int refer) { + reference.set(refer); + } + } + + public static class CompletedCheckpointCache { + private final long checkpointID; + private final Set cacheKeys; + private final SharedCacheRegistry registry; + + public CompletedCheckpointCache(SharedCacheRegistry registry, long checkpointID) { + this.checkpointID = checkpointID; + this.cacheKeys = new HashSet<>(); + this.registry = registry; + } + + public void addCacheEntry(CacheKey key, CacheEntry value) { + registry.registerReference(key, value); + cacheKeys.add(key); + } + + public long getCheckpointID() { + return checkpointID; + } + + public void discard() { + for (CacheKey key : cacheKeys) { + registry.unregisterReference(key); + } + } + } + + public static class PendingCheckpointCache { + private final long checkpointID; + private boolean discarded; + private final Map pendingEntry; + private ScheduledFuture cancellerHandle; + private final Executor executor; + + public PendingCheckpointCache(Executor executor, long checkpointID) { + this.checkpointID = checkpointID; + this.pendingEntry = new HashMap<>(); + this.discarded = false; + this.executor = executor; + } + + public void addEntry(CacheKey key, CacheEntry entry) { + if (!discarded) { + CacheEntry preEntry = pendingEntry.get(key); + if (preEntry != null) { + throw new FlinkRuntimeException("register twice in pending cache map with the same key: { " + key.handleID + "}"); + } + pendingEntry.put(key, entry); + } + } + + public Iterable> getEntryIterable() { + return discarded ? null : pendingEntry.entrySet(); + } + + public long getCheckpointID() { + return checkpointID; + } + + public void abortSubsumed() { + dispose(); + } + + public void abortExpired() { + dispose(); + } + + public boolean isDiscarded() { + return discarded; + } + + public boolean setCancellerHandle(ScheduledFuture cancellerHandle) { + if (this.cancellerHandle == null) { + if (!discarded) { + this.cancellerHandle = cancellerHandle; + return true; + } else { + return false; + } + } + else { + throw new IllegalStateException("A canceller handle was already set"); + } + } + + private void cancelCanceller() { + try { + final ScheduledFuture canceller = this.cancellerHandle; + if (canceller != null) { + canceller.cancel(false); + } + } + catch (Exception e) { + // this code should not throw exceptions + LOG.warn("Error while cancelling checkpoint cache timeout task", e); + } + } + + private void dispose() { + try { + executor.execute(() -> { + for (CacheEntry entry : pendingEntry.values()) { + entry.discard(); + } + pendingEntry.clear(); + }); + } finally { + discarded = true; + cancelCanceller(); + } + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManager.java new file mode 100644 index 0000000000000..bcb7a4ba77259 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManager.java @@ -0,0 +1,147 @@ +/* + * 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.VisibleForTesting; +import org.apache.flink.api.common.JobID; +import org.apache.flink.core.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +/** + * It is responsible for managing all {@link CheckpointCache} instances on a TM, + * whenever the TM receives the task submit message, it registers a CheckpointCache + * from CheckpointCacheManager for the coming Task. It is initialized in the form of + * service during TM initialization and runs on TM until TM exits. + */ +public class CheckpointCacheManager { + + private static Logger LOG = LoggerFactory.getLogger(CheckpointCacheManager.class); + + private final Object lock = new Object(); + private final Path basePath; + private final ScheduledExecutorService scheduledExecutorService; + + private final Map checkpointCaches; + private final Map> cacheClearFutures; + + private final Executor executor; + + public CheckpointCacheManager(ScheduledExecutorService scheduledExecutorService, Executor executor, String basePath) { + this.scheduledExecutorService = scheduledExecutorService; + this.executor = executor; + this.basePath = new Path(basePath); + this.checkpointCaches = new ConcurrentHashMap<>(); + this.cacheClearFutures = new ConcurrentHashMap<>(); + } + + /** Reregister a {@linke CheckpointCache} from {@link CheckpointCacheManager}, create a new one or refer a exists one. */ + public CheckpointCache registerCheckpointCache(JobID jobID, long pendingCheckpointCacheTimeout, long leaseTimeout) { + if (pendingCheckpointCacheTimeout == -1) { + return null; + } + synchronized (lock) { + CheckpointCache checkpointCache = checkpointCaches.get(jobID); + if (checkpointCache == null) { + LOG.info("jobID: {} create checkpoint cache", jobID); + checkpointCache = new CheckpointCache(jobID, + basePath.getPath(), + pendingCheckpointCacheTimeout, + leaseTimeout, + this, + executor); + checkpointCaches.put(jobID, checkpointCache); + } else { + ScheduledFuture cacheClearRunner = cacheClearFutures.get(jobID); + if (cacheClearRunner != null && !cacheClearRunner.isDone()) { + LOG.info("checkpoint cache {}, reassign a lease", jobID); + cacheClearRunner.cancel(false); + cacheClearFutures.remove(jobID); + } + } + int reference = checkpointCache.reference(); + LOG.info("jobID: {} registered, current reference: {}", jobID, reference); + return checkpointCache; + } + } + + /** Unregister {@link CheckpointCache} with jobId, try to release it when it's reference equal zero. */ + public void unregisterCheckpointCache(JobID jobID) { + synchronized (lock) { + final CheckpointCache checkpointCache = checkpointCaches.get(jobID); + int reference = checkpointCache.dereference(); + if (reference <= 0) { + ScheduledFuture cacheClearRunner = cacheClearFutures.get(jobID); + if (cacheClearRunner == null || cacheClearRunner.isDone()) { + cacheClearRunner = scheduledExecutorService.schedule(() -> { + synchronized (lock) { + LOG.info("try to remove checkpoint cache for jobID:{}", jobID); + if (checkpointCache.getReference() <= 0) { + checkpointCaches.remove(jobID); + checkpointCache.release(); + cacheClearFutures.remove(jobID); + LOG.info("remove checkpoint cache for jobID:{}", jobID); + } else { + LOG.info("failed to remove checkpoint cache for jobID:{}", jobID); + } + } + }, checkpointCache.getLeaseTimeout(), TimeUnit.SECONDS + ); + cacheClearFutures.put(jobID, cacheClearRunner); + } + } + LOG.info("jobID: {} unregistered, current reference: {}", jobID, reference); + } + } + + /** Shutdown service, called when TM shutdown. */ + public void shutdown() { + synchronized (lock) { + for (CheckpointCache checkpointCache : checkpointCaches.values()) { + checkpointCache.release(); + } + checkpointCaches.clear(); + } + } + + @VisibleForTesting + protected CheckpointCache getCheckpointCache(JobID jobID) { + synchronized (lock) { + return checkpointCaches.get(jobID); + } + } + + @VisibleForTesting + protected int getCheckpointCacheSize() { + synchronized (lock) { + return checkpointCaches.size(); + } + } + + @VisibleForTesting + protected ScheduledFuture getCacheClearFuture(JobID jobID) { + return cacheClearFutures.get(jobID); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SharedCacheRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SharedCacheRegistry.java new file mode 100644 index 0000000000000..cc54fc634d88b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/SharedCacheRegistry.java @@ -0,0 +1,128 @@ +/* + * 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.VisibleForTesting; +import org.apache.flink.runtime.checkpoint.CheckpointCache.CacheEntry; +import org.apache.flink.runtime.checkpoint.CheckpointCache.CacheKey; +import org.apache.flink.util.FlinkRuntimeException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Iterator; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executor; + +/** + * Used by {@link CheckpointCache} to support incremental Cache. SharedCacheRegistry maintains + * all CacheEntries, {@link CacheEntry} is deleted only if the reference count become zero. + */ +public class SharedCacheRegistry { + + private static Logger LOG = LoggerFactory.getLogger(SharedCacheRegistry.class); + private final Map registeredCacheEntry; + + /** Executor for async state deletion */ + private final Executor asyncDisposalExecutor; + + public SharedCacheRegistry(Executor asyncDisposalExecutor) { + this.registeredCacheEntry = new ConcurrentHashMap<>(); + this.asyncDisposalExecutor = asyncDisposalExecutor; + } + + public void registerReference(CacheKey key, CacheEntry value) { + synchronized (registeredCacheEntry) { + CacheEntry entry = registeredCacheEntry.get(key); + CacheEntry entryToDelete = null; + if (entry == null || entry.isRot()) { + if (entry != null) { + value.setReference(entry.getReferenceCount()); + entryToDelete = entry; + } + entry = value; + registeredCacheEntry.put(key, value); + } else { + if (!Objects.equals(entry, value)) { + entryToDelete = value; + } + } + entry.increaseReferenceCount(); + + scheduleAsyncDelete(entryToDelete); + } + } + + public void unregisterReference(CacheKey key) { + synchronized (registeredCacheEntry) { + CacheEntry entry = registeredCacheEntry.get(key); + if (entry.decreaseReferenceCount() <= 0) { + registeredCacheEntry.remove(key); + scheduleAsyncDelete(entry); + } + } + } + + public CacheEntry getCacheEntry(CacheKey key) { + synchronized (registeredCacheEntry) { + return registeredCacheEntry.get(key); + } + } + + private void scheduleAsyncDelete(CacheEntry entry) { + // We do the small optimization to not issue discards for placeholders, which are NOPs. + if (entry != null) { + LOG.trace("Scheduled delete of cache entry {}.", entry); + asyncDisposalExecutor.execute(() -> { + try { + entry.discard(); + } catch (Exception e) { + LOG.warn("A problem occurred during asynchronous disposal of a cache entry: {}", entry, e); + } + }); + } + } + + @VisibleForTesting + protected Iterable> getIterable() { + final Iterator> iterator = registeredCacheEntry.entrySet().iterator(); + return new Iterable>() { + @Override + public Iterator> iterator() { + return new Iterator>() { + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public Map.Entry next() { + return iterator.next(); + } + + @Override + public void remove() { + throw new FlinkRuntimeException("unsupported method."); + } + }; + } + }; + } +} 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..d41c12897e379 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 @@ -19,6 +19,7 @@ package org.apache.flink.runtime.checkpoint.savepoint; import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.checkpoint.CachedStreamStateHandle; import org.apache.flink.runtime.checkpoint.MasterState; import org.apache.flink.runtime.checkpoint.OperatorState; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -77,6 +78,7 @@ class SavepointV2Serializer implements SavepointSerializer { private static final byte KEY_GROUPS_HANDLE = 3; private static final byte PARTITIONABLE_OPERATOR_STATE_HANDLE = 4; private static final byte INCREMENTAL_KEY_GROUPS_HANDLE = 5; + private static final byte CACHED_STREAM_STATE_HANDLE = 6; /** The singleton instance of the serializer */ public static final SavepointV2Serializer INSTANCE = new SavepointV2Serializer(); @@ -510,6 +512,13 @@ private static void serializeStreamStateHandle( byte[] internalData = byteStreamStateHandle.getData(); dos.writeInt(internalData.length); dos.write(byteStreamStateHandle.getData()); + } else if (stateHandle instanceof CachedStreamStateHandle) { + dos.writeByte(CACHED_STREAM_STATE_HANDLE); + CachedStreamStateHandle cachedStreamStateHandle = (CachedStreamStateHandle) stateHandle; + StateHandleID handleID = cachedStreamStateHandle.getStateHandleId(); + StreamStateHandle streamHandle = cachedStreamStateHandle.getRemoteHandle(); + dos.writeUTF(handleID.getKeyString()); + serializeStreamStateHandle(streamHandle, dos); } else { throw new IOException("Unknown implementation of StreamStateHandle: " + stateHandle.getClass()); } @@ -531,6 +540,10 @@ private static StreamStateHandle deserializeStreamStateHandle(DataInputStream di byte[] data = new byte[numBytes]; dis.readFully(data); return new ByteStreamStateHandle(handleName, data); + } else if (CACHED_STREAM_STATE_HANDLE == type) { + String handleID = dis.readUTF(); + StreamStateHandle remoteHandle = deserializeStreamStateHandle(dis); + return new CachedStreamStateHandle(new StateHandleID(handleID), remoteHandle); } else { throw new IOException("Unknown implementation of StreamStateHandle, code: " + type); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java index 0c7e308bfd7bd..22f847fc41b67 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java @@ -145,6 +145,9 @@ public Offloaded(PermanentBlobKey serializedValueKey) { /** State handles for the sub task. */ private final TaskStateSnapshot taskStateHandles; + /** checkpoint timeout. */ + private final long checkpointTimeout; + public TaskDeploymentDescriptor( JobID jobId, MaybeOffloaded serializedJobInformation, @@ -156,7 +159,8 @@ public TaskDeploymentDescriptor( int targetSlotNumber, TaskStateSnapshot taskStateHandles, Collection resultPartitionDeploymentDescriptors, - Collection inputGateDeploymentDescriptors) { + Collection inputGateDeploymentDescriptors, + long checkpointTimeout) { this.jobId = Preconditions.checkNotNull(jobId); @@ -179,6 +183,8 @@ public TaskDeploymentDescriptor( this.producedPartitions = Preconditions.checkNotNull(resultPartitionDeploymentDescriptors); this.inputGates = Preconditions.checkNotNull(inputGateDeploymentDescriptors); + + this.checkpointTimeout = checkpointTimeout; } /** @@ -322,6 +328,10 @@ public void loadBigData(@Nullable PermanentBlobService blobService) Preconditions.checkNotNull(serializedTaskInformation); } + public long getCheckpointTimeout() { + return this.checkpointTimeout; + } + @Override public String toString() { return String.format("TaskDeploymentDescriptor [execution id: %s, attempt: %d, " + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index 6d45d06d8f952..74e43df08b38f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -41,6 +41,7 @@ import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.jobgraph.JobEdge; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmanager.scheduler.LocationPreferenceConstraint; @@ -824,6 +825,8 @@ TaskDeploymentDescriptor createDeploymentDescriptor( serializedTaskInformation = new TaskDeploymentDescriptor.Offloaded<>(taskInformationOrBlobKey.right()); } + // get jobCheckpointingSettings via job graph + CheckpointCoordinatorConfiguration checkpointCoordinatorConfiguration = this.getExecutionGraph().getCheckpointCoordinatorConfiguration(); return new TaskDeploymentDescriptor( getJobId(), serializedJobInformation, @@ -835,7 +838,8 @@ TaskDeploymentDescriptor createDeploymentDescriptor( targetSlot.getRoot().getSlotNumber(), taskStateHandles, producedPartitions, - consumedPartitions); + consumedPartitions, + checkpointCoordinatorConfiguration == null ? -1L : checkpointCoordinatorConfiguration.getCheckpointTimeout()); } // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedCheckpointStreamFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedCheckpointStreamFactory.java new file mode 100644 index 0000000000000..9d1f4ddbe59a1 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedCheckpointStreamFactory.java @@ -0,0 +1,176 @@ +/* + * 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.runtime.checkpoint.CachedStreamStateHandle; +import org.apache.flink.runtime.checkpoint.CheckpointCache; +import org.apache.flink.runtime.checkpoint.CheckpointCache.CachedOutputStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/** + * {@link CachedCheckpointStreamFactory} is used to build an output stream that writes data to both remote end (e.g:DFS) and local end. + * Local data is managed by {@link CheckpointCache}. It simply wraps {@link CheckpointCache} and {@link CheckpointStreamFactory} and + * create a hybrid output stream by {@link CheckpointCache} and {@link CheckpointStreamFactory}, this hybrid output stream will write + * to both remote end and local end. + */ +public class CachedCheckpointStreamFactory implements CheckpointStreamFactory { + + private static Logger LOG = LoggerFactory.getLogger(CachedCheckpointStreamFactory.class); + + private final CheckpointCache cache; + private final CheckpointStreamFactory remoteFactory; + + public CachedCheckpointStreamFactory(CheckpointCache cache, CheckpointStreamFactory factory) { + this.cache = cache; + this.remoteFactory = factory; + } + + public CheckpointStateOutputStream createCheckpointStateOutputStream(long checkpointID, long timestamp, StateHandleID handleID) throws Exception { + return createCheckpointStateOutputStream(checkpointID, timestamp, handleID, false); + } + + public CheckpointStateOutputStream createCheckpointStateOutputStream(long checkpointID, long timestamp, StateHandleID handleID, boolean placeholder) throws Exception { + if (LOG.isDebugEnabled()) { + LOG.debug("create cache output stream: cpkID:{} placeHolder:{}", checkpointID, placeholder); + } + CachedOutputStream cachedOut = cache.createOutputStream(checkpointID, handleID, placeholder); + CheckpointStateOutputStream remoteOut = null; + if (!placeholder) { + remoteOut = remoteFactory.createCheckpointStateOutputStream(checkpointID, timestamp); + } + CachedCheckpointStateOutputStream output = new CachedCheckpointStateOutputStream(cachedOut, remoteOut); + return output; + } + + @Override + public CheckpointStateOutputStream createCheckpointStateOutputStream(long checkpointID, long timestamp) throws Exception { + LOG.warn("create output stream which is not cacheable."); + return remoteFactory.createCheckpointStateOutputStream(checkpointID, timestamp); + } + + @Override + public void close() throws Exception { + remoteFactory.close(); + } + + /** + * A hybrid checkpoint output stream which write data to both remote end and local end, + * writing data locally failed won't stop writing to remote. This hybrid output stream + * will return a {@link CachedStreamStateHandle} in closeAndGetHandle(), it can be used for read data locally. + */ + public static class CachedCheckpointStateOutputStream extends CheckpointStateOutputStream { + + private CachedOutputStream cacheOut = null; + private CheckpointStateOutputStream remoteOut = null; + + public CachedCheckpointStateOutputStream(CachedOutputStream cacheOut, CheckpointStateOutputStream remoteOut) { + this.cacheOut = cacheOut; + this.remoteOut = remoteOut; + } + + @Override + public StreamStateHandle closeAndGetHandle() throws IOException { + + // finalize cache data + StateHandleID cacheId = cacheOut.getCacheID(); + cacheOut.end(); + + StreamStateHandle remoteHandle; + if (remoteOut != null) { + remoteHandle = remoteOut.closeAndGetHandle(); + } else { + remoteHandle = new PlaceholderStreamStateHandle(cacheId); + } + return new CachedStreamStateHandle(cacheId, remoteHandle); + } + + @Override + public long getPos() throws IOException { + return remoteOut != null ? remoteOut.getPos() :-1L; + } + + @Override + public void write(int b) throws IOException { + // write to local + try { + cacheOut.write(b); + } catch (Exception e) { + //discard + cacheOut.discard(); + } + + // write to remote + if (remoteOut != null) { + remoteOut.write(b); + } + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + // write to local + try { + cacheOut.write(b, off, len); + } catch (Exception e) { + //discard + cacheOut.discard(); + } + + // write to remote + if (remoteOut != null) { + remoteOut.write(b, off, len); + } + } + + @Override + public void flush() throws IOException { + cacheOut.flush(); + if (remoteOut != null) { + remoteOut.flush(); + } + } + + @Override + public void sync() throws IOException { + if (remoteOut != null) { + remoteOut.sync(); + } + } + + @Override + public void close() throws IOException { + cacheOut.close(); + if (remoteOut != null) { + remoteOut.close(); + } + } + + @VisibleForTesting + public void setCacheOut(CachedOutputStream cacheOut) { + this.cacheOut = cacheOut; + } + + @VisibleForTesting + public void setRemoteOut(CheckpointStateOutputStream remoteOut) { + this.remoteOut = remoteOut; + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedStateHandle.java new file mode 100644 index 0000000000000..94c326fe8b060 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedStateHandle.java @@ -0,0 +1,25 @@ +/* + * 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; + +/** + * Interface for StateHandle that can be cached. + */ +public interface CachedStateHandle { + StateHandleID getStateHandleId(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java index 8092f6c72de8c..d8296aec63560 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java @@ -20,6 +20,8 @@ import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.runtime.checkpoint.CachedStreamStateHandle; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.util.Preconditions; import java.io.IOException; @@ -87,6 +89,13 @@ public KeyGroupsStateHandle getIntersection(KeyGroupRange keyGroupRange) { return new KeyGroupsStateHandle(groupRangeOffsets.getIntersection(keyGroupRange), stateHandle); } + public void setCache(CheckpointCache cache) { + if (stateHandle instanceof CachedStreamStateHandle) { + ((CachedStreamStateHandle) stateHandle).setCheckpointCache(cache); + ((CachedStreamStateHandle) stateHandle).reCache(false); + } + } + @Override public KeyGroupRange getKeyGroupRange() { return groupRangeOffsets.getKeyGroupRange(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/PlaceholderStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/PlaceholderStreamStateHandle.java index 7c948a169cb3b..18cc4e3c9ca37 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/PlaceholderStreamStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/PlaceholderStreamStateHandle.java @@ -27,11 +27,18 @@ * case of {@link ByteStreamStateHandle}. This class is used in the referenced states of * {@link IncrementalKeyedStateHandle}. */ -public class PlaceholderStreamStateHandle implements StreamStateHandle { +public class PlaceholderStreamStateHandle implements StreamStateHandle, CachedStateHandle { private static final long serialVersionUID = 1L; + private final StateHandleID handleID; + + public PlaceholderStreamStateHandle(StateHandleID handleID) { + this.handleID = handleID; + } + public PlaceholderStreamStateHandle() { + this(null); } @Override @@ -49,4 +56,9 @@ public void discardState() throws Exception { public long getStateSize() { return 0L; } + + @Override + public StateHandleID getStateHandleId() { + return this.handleID; + } } 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 a348948c7077e..02e6cde678e8b 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 @@ -21,9 +21,13 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; import org.apache.flink.runtime.blob.BlobCacheService; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCache; +import org.apache.flink.runtime.checkpoint.CheckpointCacheManager; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -89,6 +93,7 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; +import scala.concurrent.duration.Duration; import java.io.IOException; import java.net.InetSocketAddress; @@ -165,6 +170,8 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { private final JobLeaderService jobLeaderService; + private final CheckpointCacheManager checkpointCacheManager; + // ------------------------------------------------------------------------ private final HardwareDescription hardwareDescription; @@ -176,6 +183,7 @@ public TaskExecutor( MemoryManager memoryManager, IOManager ioManager, NetworkEnvironment networkEnvironment, + CheckpointCacheManager checkpointCacheManager, HighAvailabilityServices haServices, HeartbeatServices heartbeatServices, TaskManagerMetricGroup taskManagerMetricGroup, @@ -203,6 +211,7 @@ public TaskExecutor( this.fileCache = checkNotNull(fileCache); this.jobManagerTable = checkNotNull(jobManagerTable); this.jobLeaderService = checkNotNull(jobLeaderService); + this.checkpointCacheManager = checkpointCacheManager; this.jobManagerConnections = new HashMap<>(4); @@ -278,6 +287,8 @@ public void postStop() throws Exception { fileCache.shutdown(); + checkpointCacheManager.shutdown(); + try { super.postStop(); } catch (Throwable e) { @@ -380,6 +391,23 @@ public CompletableFuture submitTask( ResultPartitionConsumableNotifier resultPartitionConsumableNotifier = jobManagerConnection.getResultPartitionConsumableNotifier(); PartitionProducerStateChecker partitionStateChecker = jobManagerConnection.getPartitionStateChecker(); + long checkpointCacheLeaseTimeout; + try { + Configuration configuration = jobInformation.getJobConfiguration(); + String timeoutString = configuration.getString( + ConfigConstants.AKKA_WATCH_HEARTBEAT_INTERVAL, + ConfigConstants.DEFAULT_AKKA_ASK_TIMEOUT); + String delayString = configuration.getString( + ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, + timeoutString + ); + checkpointCacheLeaseTimeout = Duration.apply(delayString).toMillis() * 2; + } catch (Exception nfe) { + checkpointCacheLeaseTimeout = tdd.getCheckpointTimeout(); + } + + CheckpointCache checkpointCache = checkpointCacheManager.registerCheckpointCache(jobInformation.getJobId(), tdd.getCheckpointTimeout(), checkpointCacheLeaseTimeout); + Task task = new Task( jobInformation, taskInformation, @@ -401,6 +429,7 @@ public CompletableFuture submitTask( blobService, libraryCache, fileCache, + checkpointCache, taskManagerConfiguration, taskMetricGroup, resultPartitionConsumableNotifier, 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 a24daf0f608fa..208739e0c736a 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 @@ -286,6 +286,7 @@ public static TaskExecutor startTaskManager( taskManagerServices.getMemoryManager(), taskManagerServices.getIOManager(), taskManagerServices.getNetworkEnvironment(), + taskManagerServices.getCheckpointCacheManager(), highAvailabilityServices, heartbeatServices, taskManagerMetricGroup, 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 0756529ed2e31..e976dc6a139cb 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 @@ -23,9 +23,11 @@ import org.apache.flink.core.memory.MemoryType; import org.apache.flink.queryablestate.network.stats.DisabledKvStateRequestStats; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCacheManager; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.filecache.FileCache; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; @@ -74,6 +76,7 @@ public class TaskManagerServices { private final TaskSlotTable taskSlotTable; private final JobManagerTable jobManagerTable; private final JobLeaderService jobLeaderService; + private final CheckpointCacheManager checkpointCacheManager; private TaskManagerServices( TaskManagerLocation taskManagerLocation, @@ -84,7 +87,8 @@ private TaskManagerServices( FileCache fileCache, TaskSlotTable taskSlotTable, JobManagerTable jobManagerTable, - JobLeaderService jobLeaderService) { + JobLeaderService jobLeaderService, + CheckpointCacheManager checkpointCacheManager) { this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation); this.memoryManager = Preconditions.checkNotNull(memoryManager); @@ -95,6 +99,7 @@ private TaskManagerServices( this.taskSlotTable = Preconditions.checkNotNull(taskSlotTable); this.jobManagerTable = Preconditions.checkNotNull(jobManagerTable); this.jobLeaderService = Preconditions.checkNotNull(jobLeaderService); + this.checkpointCacheManager = Preconditions.checkNotNull(checkpointCacheManager); } // -------------------------------------------------------------------------------------------- @@ -137,6 +142,10 @@ public JobLeaderService getJobLeaderService() { return jobLeaderService; } + public CheckpointCacheManager getCheckpointCacheManager() { + return checkpointCacheManager; + } + // -------------------------------------------------------------------------------------------- // Static factory methods for task manager services // -------------------------------------------------------------------------------------------- @@ -190,6 +199,11 @@ public static TaskManagerServices fromConfiguration( final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation); + final CheckpointCacheManager checkpointCacheManager = new CheckpointCacheManager( + new ScheduledThreadPoolExecutor(1), + Executors.directExecutor(), + taskManagerServicesConfiguration.getTmpDirPaths()[0]); + return new TaskManagerServices( taskManagerLocation, memoryManager, @@ -199,7 +213,8 @@ public static TaskManagerServices fromConfiguration( fileCache, taskSlotTable, jobManagerTable, - jobLeaderService); + jobLeaderService, + checkpointCacheManager); } /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java index 92b58868d666f..626b492f2e66a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java @@ -25,6 +25,7 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.execution.Environment; @@ -81,6 +82,8 @@ public class RuntimeEnvironment implements Environment { private final Task containingTask; + private final CheckpointCache checkpointCache; + // ------------------------------------------------------------------------ public RuntimeEnvironment( @@ -104,6 +107,7 @@ public RuntimeEnvironment( CheckpointResponder checkpointResponder, TaskManagerRuntimeInfo taskManagerInfo, TaskMetricGroup metrics, + CheckpointCache cache, Task containingTask) { this.jobId = checkNotNull(jobId); @@ -127,6 +131,7 @@ public RuntimeEnvironment( this.taskManagerInfo = checkNotNull(taskManagerInfo); this.containingTask = containingTask; this.metrics = metrics; + this.checkpointCache = cache; } // ------------------------------------------------------------------------ @@ -261,4 +266,8 @@ public void declineCheckpoint(long checkpointId, Throwable cause) { public void failExternally(Throwable cause) { this.containingTask.failExternally(cause); } + + public CheckpointCache getCheckpointCache() { + return checkpointCache; + } } 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 2cb356c81f116..17f6d3ccfb13d 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 @@ -33,6 +33,7 @@ import org.apache.flink.runtime.blob.BlobCacheService; import org.apache.flink.runtime.blob.PermanentBlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; @@ -271,6 +272,8 @@ public class Task implements Runnable, TaskActions { */ private ClassLoader userCodeClassLoader; + private CheckpointCache checkpointCache; + /** *

IMPORTANT: This constructor may not start any work that would need to * be undone in the case of a failing task deployment.

@@ -296,6 +299,7 @@ public Task( BlobCacheService blobService, LibraryCacheManager libraryCache, FileCache fileCache, + CheckpointCache cache, TaskManagerRuntimeInfo taskManagerConfig, @Nonnull TaskMetricGroup metricGroup, ResultPartitionConsumableNotifier resultPartitionConsumableNotifier, @@ -411,6 +415,8 @@ public Task( // finally, create the executing thread, but do not start it executingThread = new Thread(TASK_THREADS_GROUP, this, taskNameWithSubtask); + + this.checkpointCache = cache; } // ------------------------------------------------------------------------ @@ -672,7 +678,7 @@ else if (current == ExecutionState.CANCELING) { memoryManager, ioManager, broadcastVariableManager, accumulatorRegistry, kvStateRegistry, inputSplitProvider, distributedCacheEntries, writers, inputGates, - checkpointResponder, taskManagerConfig, metrics, this); + checkpointResponder, taskManagerConfig, metrics, checkpointCache, this); // let the task code create its readers and writers invokable.setEnvironment(env); @@ -843,6 +849,11 @@ else if (transitionState(current, ExecutionState.FAILED, t)) { libraryCache.unregisterTask(jobId, executionId); blobService.getPermanentBlobService().releaseJob(jobId); + // unregister checkpoint cache + if (checkpointCache != null) { + checkpointCache.discard(); + } + // remove all files in the distributed cache removeCachedFiles(distributedCacheEntries, fileCache); 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 8ef2e364293f6..6bdd5df096b77 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 @@ -27,7 +27,7 @@ import org.apache.flink.api.common.io.FileOutputFormat import org.apache.flink.configuration._ import org.apache.flink.core.fs.Path import org.apache.flink.runtime.blob.BlobServer -import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory +import org.apache.flink.runtime.checkpoint.{CheckpointCacheManager, CheckpointRecoveryFactory} import org.apache.flink.runtime.clusterframework.FlinkResourceManager import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager import org.apache.flink.runtime.clusterframework.types.{ResourceID, ResourceIDRetrievable} @@ -254,6 +254,7 @@ class LocalFlinkMiniCluster( taskManagerServices.getMemoryManager(), taskManagerServices.getIOManager(), taskManagerServices.getNetworkEnvironment, + taskManagerServices.getCheckpointCacheManager, taskManagerMetricGroup) system.actorOf(props, taskManagerActorName) @@ -318,6 +319,7 @@ class LocalFlinkMiniCluster( memoryManager: MemoryManager, ioManager: IOManager, networkEnvironment: NetworkEnvironment, + checkpointCacheManager: CheckpointCacheManager, taskManagerMetricGroup: TaskManagerMetricGroup): Props = { TaskManager.getTaskManagerProps( @@ -328,6 +330,7 @@ class LocalFlinkMiniCluster( memoryManager, ioManager, networkEnvironment, + checkpointCacheManager, 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 f948df4c08ea6..a96f1e580ddcd 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 @@ -37,6 +37,7 @@ 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.broadcast.BroadcastVariableManager +import org.apache.flink.runtime.checkpoint.CheckpointCacheManager import org.apache.flink.runtime.clusterframework.BootstrapTools import org.apache.flink.runtime.clusterframework.messages.StopCluster import org.apache.flink.runtime.clusterframework.types.ResourceID @@ -125,6 +126,7 @@ class TaskManager( protected val memoryManager: MemoryManager, protected val ioManager: IOManager, protected val network: NetworkEnvironment, + protected val checkpointCacheManager: CheckpointCacheManager, protected val numberOfSlots: Int, protected val highAvailabilityServices: HighAvailabilityServices, protected val taskManagerMetricGroup: TaskManagerMetricGroup) @@ -259,6 +261,12 @@ class TaskManager( taskManagerMetricGroup.close() + try { + checkpointCacheManager.shutdown(); + } catch { + case t: Exception => log.error("CheckpointCacheManager did not shutdown properly", t) + } + log.info(s"Task manager ${self.path} is completely shut down.") } @@ -1176,6 +1184,28 @@ class TaskManager( config.getTimeout().getSize(), config.getTimeout().getUnit())) + var checkpointCacheLeaseTimeout: Long = { + try { + var configuration = jobInformation.getJobConfiguration() + var timeoutString = configuration.getString( + ConfigConstants.AKKA_WATCH_HEARTBEAT_INTERVAL, + ConfigConstants.DEFAULT_AKKA_ASK_TIMEOUT) + var delayString = configuration.getString( + ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, + timeoutString + ) + Duration.apply(delayString).toMillis * 2; + } catch { + case e: Exception => + tdd.getCheckpointTimeout + } + } + + var checkpointCache = checkpointCacheManager.registerCheckpointCache( + jobInformation.getJobId, + tdd.getCheckpointTimeout, + checkpointCacheLeaseTimeout) + val task = new Task( jobInformation, taskInformation, @@ -1197,6 +1227,7 @@ class TaskManager( blobCache, libCache, fileCache, + checkpointCache, config, taskMetricGroup, resultPartitionConsumableNotifier, @@ -2018,6 +2049,7 @@ object TaskManager { taskManagerServices.getMemoryManager(), taskManagerServices.getIOManager(), taskManagerServices.getNetworkEnvironment(), + taskManagerServices.getCheckpointCacheManager(), highAvailabilityServices, taskManagerMetricGroup) @@ -2035,6 +2067,7 @@ object TaskManager { memoryManager: MemoryManager, ioManager: IOManager, networkEnvironment: NetworkEnvironment, + checkpointCacheManager: CheckpointCacheManager, highAvailabilityServices: HighAvailabilityServices, taskManagerMetricGroup: TaskManagerMetricGroup ): Props = { @@ -2046,6 +2079,7 @@ object TaskManager { memoryManager, ioManager, networkEnvironment, + checkpointCacheManager, taskManagerConfig.getNumberSlots(), highAvailabilityServices, taskManagerMetricGroup) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CachedCheckpointStreamFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CachedCheckpointStreamFactoryTest.java new file mode 100644 index 0000000000000..c0b6ea8da441e --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CachedCheckpointStreamFactoryTest.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.checkpoint; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.state.CachedCheckpointStreamFactory; +import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.runtime.state.StateHandleID; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import static org.mockito.Mockito.mock; + +/** + * Tests for the cached checkpoint stream factory. + */ +public class CachedCheckpointStreamFactoryTest { + + @Rule + public final TemporaryFolder tmp = new TemporaryFolder(); + + @Test + public void testCachedCheckpointStateOutputStream() throws Exception { + JobID jobID = new JobID(); + + CheckpointCache checkpointCache = new CheckpointCache( + jobID, + tmp.newFolder().getAbsolutePath(), + 1000L, + 1000L, + mock(CheckpointCacheManager.class), + Executors.directExecutor() + ); + + CheckpointStreamFactory streamFactory = new FsCheckpointStreamFactory( + new Path(tmp.newFolder().getAbsolutePath()), + jobID, + 1024); + + CachedCheckpointStreamFactory factory = new CachedCheckpointStreamFactory( + checkpointCache, + streamFactory); + StateHandleID handleID = new StateHandleID("handleId"); + CheckpointStreamFactory.CheckpointStateOutputStream outputStream = factory.createCheckpointStateOutputStream(1, 1, handleID); + Assert.assertTrue(outputStream instanceof CachedCheckpointStreamFactory.CachedCheckpointStateOutputStream); + + String testStr = "hello"; + outputStream.write(testStr.getBytes()); + StreamStateHandle resultStateHandle = outputStream.closeAndGetHandle(); + Assert.assertTrue(resultStateHandle instanceof CachedStreamStateHandle); + ((CachedStreamStateHandle)resultStateHandle).setCheckpointCache(checkpointCache); + + checkpointCache.commitCache(1); + + byte[] buffer = new byte[1024]; + int n = resultStateHandle.openInputStream().read(buffer); + Assert.assertEquals(testStr.length(), n); + Assert.assertEquals(new String(buffer, 0, testStr.length()), testStr); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManagerTest.java new file mode 100644 index 0000000000000..99f1821341019 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManagerTest.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.api.common.JobID; +import org.apache.flink.runtime.concurrent.Executors; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Tests for CheckpointCacheManager + */ +public class CheckpointCacheManagerTest { + + @Rule + public final TemporaryFolder tmp = new TemporaryFolder(); + + @Test + public void testCheckpointCacheManager() throws Exception { + CheckpointCacheManager cacheManager = new CheckpointCacheManager(new ScheduledThreadPoolExecutor(1), Executors.directExecutor(), tmp.newFolder().getAbsolutePath()); + JobID jobID1 = new JobID(1L, 1L); + cacheManager.registerCheckpointCache(jobID1, 10000, 5); + Assert.assertEquals(1, cacheManager.getCheckpointCacheSize()); + cacheManager.unregisterCheckpointCache(jobID1); + TimeUnit.SECONDS.sleep(6); + Assert.assertEquals(0, cacheManager.getCheckpointCacheSize()); + } + + @Test + public void testCheckpointCacheRetouchFromRelease() throws Exception { + + CheckpointCacheManager cacheManager = new CheckpointCacheManager(new ScheduledThreadPoolExecutor(1), Executors.directExecutor(), tmp.newFolder().getAbsolutePath()); + JobID jobID = new JobID(1L, 1L); + CheckpointCache cache1 = cacheManager.registerCheckpointCache(jobID, 10000, 5); + + // this should release the CheckpointCache in future. + cacheManager.unregisterCheckpointCache(jobID); + Assert.assertEquals(0, cache1.getReference()); + ScheduledFuture clearFuture = cacheManager.getCacheClearFuture(jobID); + TimeUnit.SECONDS.sleep(3); + Assert.assertTrue(!clearFuture.isDone()); + + // reassign a lease for CheckpointCache + CheckpointCache cache2 = cacheManager.registerCheckpointCache(jobID, 10000, 5); + TimeUnit.SECONDS.sleep(3); + + Assert.assertTrue(clearFuture.isCancelled()); + Assert.assertNull(cacheManager.getCacheClearFuture(jobID)); + Assert.assertEquals(cache1, cache2); + } + + @Test + public void testConcurrencyRequest() throws Exception { + CheckpointCacheManager cacheManager = new CheckpointCacheManager(new ScheduledThreadPoolExecutor(1), Executors.directExecutor(), tmp.newFolder().getAbsolutePath()); + // init jobs + int[] referenceCount = new int[10]; + JobID[] jobIDS = new JobID[10]; + for (int i = 0; i < 10; ++i) { + jobIDS[i] = new JobID(); + for (int j = 0; j < 50; ++j) { + cacheManager.registerCheckpointCache(jobIDS[i], 1, 1); + } + referenceCount[i] = 50; + } + + // init runnables but dont submit immediately + Runnable[] runnables = new Runnable[50]; + for (int i = 0; i < runnables.length; ++i) { + final int index = new Random().nextInt(jobIDS.length); + final int opType = new Random().nextInt(2); + if (opType == 0) { + referenceCount[index]--; + } else { + referenceCount[index]++; + } + runnables[i] = () -> { + if (opType == 0) { + cacheManager.unregisterCheckpointCache(jobIDS[index]); + } else { + cacheManager.registerCheckpointCache(jobIDS[index], 1, 1); + } + }; + } + + //submit all runnables + ExecutorService executor = java.util.concurrent.Executors.newFixedThreadPool(10); + Future[] futures = new Future[runnables.length]; + for (int i = 0; i < futures.length; ++i) { + futures[i] = executor.submit(runnables[i]); + } + + //wait all requests finished + for (int i = 0; i < futures.length; ++i) { + futures[i].get(); + } + + //valid result + for (int i = 0; i < jobIDS.length; ++i) { + Assert.assertEquals(referenceCount[i], cacheManager.getCheckpointCache(jobIDS[i]).getReference()); + } + } + + @Test + public void testShowdown() throws Exception { + CheckpointCacheManager cacheManager = new CheckpointCacheManager(new ScheduledThreadPoolExecutor(1), Executors.directExecutor(), tmp.newFolder().getAbsolutePath()); + JobID jobID1 = new JobID(1L, 1L); + cacheManager.registerCheckpointCache(jobID1, 10000, 5); + cacheManager.shutdown(); + Assert.assertEquals(0, cacheManager.getCheckpointCacheSize()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheTest.java new file mode 100644 index 0000000000000..0234f7ac113f7 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheTest.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.api.common.JobID; +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.state.StateHandleID; +import org.apache.flink.runtime.state.filesystem.FileStateHandle; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.OutputStream; +import java.util.Map; + +import static org.mockito.Mockito.mock; + +/** + * Tests for CheckpointCache + */ +public class CheckpointCacheTest { + + @Rule + public final TemporaryFolder tmp = new TemporaryFolder(); + + @Test + public void testCommitCache() throws Exception { + File tmpFolder = tmp.newFolder(); + final CheckpointCache cache = new CheckpointCache(new JobID(), tmpFolder.getAbsolutePath(), 10000, 10000, mock(CheckpointCacheManager.class), Executors.directExecutor()); + + StateHandleID handleID1 = new StateHandleID("handle1"); + StateHandleID handleID2 = new StateHandleID("handle2"); + + Assert.assertEquals(cache.getPendingCheckpointCacheSize(), 0); + + cache.registerCacheEntry(1, handleID1, tmp.newFile().getAbsolutePath()); + cache.registerCacheEntry(1, handleID2, tmp.newFile().getAbsolutePath()); + + Assert.assertEquals(cache.getPendingCheckpointCacheSize(), 1); + + cache.commitCache(1); + + Assert.assertEquals(cache.getPendingCheckpointCacheSize(), 0); + Assert.assertEquals(cache.getCompletedCheckpointCacheSize(), 1); + + cache.release(); + } + + @Test + public void testOpenInputStream() throws Exception { + File tmpFolder = tmp.newFolder(); + final CheckpointCache cache = new CheckpointCache(new JobID(), tmpFolder.getAbsolutePath(), 10000, 10000, mock(CheckpointCacheManager.class), Executors.directExecutor()); + + StateHandleID handleID1 = new StateHandleID("handle1"); + cache.registerCacheEntry(1, handleID1, "handle1"); + + File handle2File = tmp.newFile("handle2"); + + final String testStr = "hello"; + + FileOutputStream outputStream = new FileOutputStream(handle2File); + outputStream.write(testStr.getBytes(), 0, testStr.getBytes().length); + outputStream.close(); + StateHandleID handleID2 = new StateHandleID(handle2File.getAbsolutePath()); + cache.registerCacheEntry(1, handleID2, handle2File.getAbsolutePath()); + + cache.commitCache(1); + + Assert.assertNull(cache.openInputStream(handleID1)); + + //valid content + FSDataInputStream inputStream = cache.openInputStream(handleID2); + byte[] readBytes = new byte[testStr.length()]; + inputStream.read(readBytes); + Assert.assertEquals(testStr, new String(readBytes)); + + cache.release(); + } + + @Test + public void testCreateOutputStream() throws Exception { + File tmpFolder = tmp.newFolder(); + final CheckpointCache cache = new CheckpointCache(new JobID(), tmpFolder.getAbsolutePath(), 10000, 10000, mock(CheckpointCacheManager.class), Executors.directExecutor()); + + StateHandleID handleID1 = new StateHandleID("handle1"); + CheckpointCache.CachedOutputStream outputStream = cache.createOutputStream(1, handleID1); + + final String testStr = "hello"; + outputStream.write(testStr.getBytes(), 0, testStr.length()); + outputStream.end(); + + cache.commitCache(1); + + FSDataInputStream inputStream = cache.openInputStream(handleID1); + byte[] readBytes = new byte[testStr.length()]; + inputStream.read(readBytes); + Assert.assertEquals(testStr, new String(readBytes)); + + cache.release(); + } + + @Test + public void testDiscardOutputStream() throws Exception { + File tmpFolder = tmp.newFolder(); + final CheckpointCache cache = new CheckpointCache(new JobID(), tmpFolder.getAbsolutePath(), 10000, 10000, mock(CheckpointCacheManager.class), Executors.directExecutor()); + + StateHandleID handleID1 = new StateHandleID("handle1"); + CheckpointCache.CachedOutputStream outputStream = cache.createOutputStream(1, handleID1); + + final String testStr = "test str"; + outputStream.write(testStr.getBytes(), 0, testStr.length()); + outputStream.discard(); + outputStream.end(); + Assert.assertEquals(cache.getPendingCheckpointCacheSize(), 0); + + cache.release(); + } + + @Test + public void testOnlyMaintainTheLastCheckpointCache() throws Exception { + File tmpFolder = tmp.newFolder(); + final CheckpointCache cache = new CheckpointCache(new JobID(), tmpFolder.getAbsolutePath(), 10000, 10000, mock(CheckpointCacheManager.class), Executors.directExecutor()); + + StateHandleID handleID1 = new StateHandleID("handle1"); + CheckpointCache.CachedOutputStream outputStream = cache.createOutputStream(1, handleID1); + final String testStr = "test str"; + outputStream.write(testStr.getBytes(), 0, testStr.length()); + outputStream.end(); + outputStream.close(); + cache.commitCache(1); + + Assert.assertEquals(cache.getCompletedCheckpointCacheSize(), 1); + + StateHandleID handleID2 = new StateHandleID("handle2"); + CheckpointCache.CachedOutputStream outputStream2 = cache.createOutputStream(2, handleID2); + final String testStr2 = "test str2"; + outputStream2.write(testStr2.getBytes(), 0, testStr2.length()); + outputStream2.end(); + outputStream2.close(); + cache.commitCache(2); + + Assert.assertEquals(cache.getCompletedCheckpointCacheSize(), 1); + + cache.release(); + } + + @Test + public void testClose() throws Exception { + File tmpFolder = tmp.newFolder(); + final CheckpointCache cache = new CheckpointCache(new JobID(), tmpFolder.getAbsolutePath(), 10000, 10000, mock(CheckpointCacheManager.class), Executors.directExecutor()); + + for (int i = 0; i < 5; ++i) { + for (int j = 0; j < 10; ++j) { + final StateHandleID handleID = new StateHandleID("handle_" + i + "_" + j); + final CheckpointCache.CachedOutputStream output = cache.createOutputStream(i, handleID); + String testStr = "123"; + output.write(testStr.getBytes(), 0, testStr.getBytes().length); + output.end(); + output.close(); + } + } + + Assert.assertEquals(cache.getCompletedCheckpointCacheSize(), 0);; + Assert.assertEquals(cache.getPendingCheckpointCacheSize(), 5); + + for (int i = 0; i < 3; ++i) { + cache.commitCache(i); + } + + Assert.assertEquals(1, cache.getCompletedCheckpointCacheSize() ); + Assert.assertEquals(2, cache.getPendingCheckpointCacheSize()); + + cache.release(); + + Assert.assertEquals(0, cache.getCompletedCheckpointCacheSize()); + Assert.assertEquals( 0, cache.getPendingCheckpointCacheSize()); + } + + @Test + public void testReCache() throws Exception { + File tmpFolder = tmp.newFolder(); + final CheckpointCache cache = new CheckpointCache(new JobID(), tmpFolder.getAbsolutePath(), 10000, 10000, mock(CheckpointCacheManager.class), Executors.directExecutor()); + + CachedStreamStateHandle[] cachedHandles = new CachedStreamStateHandle[5]; + final String testStr = "test re-cache logic."; + String[] cacheFilePaths = new String[5]; + + // checkpoint + for (int i = 0; i < 5; ++i) { + // cache stream + final StateHandleID handleID = new StateHandleID("cache_" + i); + final CheckpointCache.CachedOutputStream output = cache.createOutputStream(1, handleID); + output.write(testStr.getBytes(), 0, testStr.getBytes().length); + output.end(); + cacheFilePaths[i] = output.getCacheFilePath(); + output.close(); + + // remote output stream + final String remoteFilePath = tmp.newFile("remote_" + i).getAbsolutePath(); + OutputStream outputStream = new FileOutputStream(remoteFilePath); + outputStream.write(testStr.getBytes(), 0, testStr.getBytes().length); + cachedHandles[i] = new CachedStreamStateHandle(handleID, new FileStateHandle(new Path(remoteFilePath), testStr.getBytes().length)); + } + cache.commitCache(1); + + // delete cache file + for (int i = 0; i < 5; ++i) { + new File(cacheFilePaths[i]).delete(); + } + + // read from cached handle, this should read from remote + for (int i = 0; i < 5; ++i) { + CachedStreamStateHandle cachedStreamStateHandle = cachedHandles[i]; + cachedStreamStateHandle.setCheckpointCache(cache); + cachedStreamStateHandle.reCache(true); + FSDataInputStream inputStream = cachedStreamStateHandle.openInputStream(); + byte[] bytes = new byte[1024]; + int n = inputStream.read(bytes); + inputStream.close(); + Assert.assertEquals(n, testStr.length()); + } + cache.commitCache(CheckpointCache.CHECKPOINT_ID_FOR_RESTORE); + + // read from cached handle again, this should read from cache + for (int i = 0; i < 5; ++i) { + CachedStreamStateHandle cachedStreamStateHandle = cachedHandles[i]; + cachedStreamStateHandle.setCheckpointCache(cache); + cachedStreamStateHandle.reCache(true); + FSDataInputStream inputStream = cachedStreamStateHandle.openInputStream(); + byte[] bytes = new byte[1024]; + int n = inputStream.read(bytes); + Assert.assertEquals(n, testStr.length()); + } + + // check refer count + SharedCacheRegistry register = cache.getSharedCacheRegister(); + for (Map.Entry entry : register.getIterable()) { + Assert.assertEquals(1, entry.getValue().getReferenceCount()); + } + } + + @Test + public void testCacheRegister() throws Exception { + + File tmpFolder = tmp.newFolder(); + final CheckpointCache cache = new CheckpointCache(new JobID(), tmpFolder.getAbsolutePath(), 10000, 10000, mock(CheckpointCacheManager.class), Executors.directExecutor()); + final String testStr = "test re-cache logic."; + String[] cacheFilePaths = new String[5]; + + // checkpoint + for (int i = 0; i < 5; ++i) { + // cache stream + final StateHandleID handleID = new StateHandleID("cache_" + i); + final CheckpointCache.CachedOutputStream output = cache.createOutputStream(1, handleID); + output.write(testStr.getBytes(), 0, testStr.getBytes().length); + output.end(); + cacheFilePaths[i] = output.getCacheFilePath(); + output.close(); + } + cache.commitCache(1); + + // checkpoint + for (int i = 4; i < 6; ++i) { + // cache stream + final StateHandleID handleID = new StateHandleID("cache_" + i); + final CheckpointCache.CachedOutputStream output = cache.createOutputStream(2, handleID); + output.write(testStr.getBytes(), 0, testStr.getBytes().length); + output.end(); + output.close(); + } + cache.commitCache(2); + + for (int i = 0; i < 4; ++i) { + Assert.assertTrue(!new File(cacheFilePaths[i]).exists()); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java index 324702485d8f9..6a2d95c9fd762 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java @@ -86,7 +86,8 @@ public void testSerialization() { targetSlotNumber, taskStateHandles, producedResults, - inputGates); + inputGates, + -1); final TaskDeploymentDescriptor copy = CommonTestUtils.createCopySerializable(orig); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java index d934ea9936d4b..ccbdfa7e76c21 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/TaskManagerMetricsTest.java @@ -112,6 +112,7 @@ public void testMetricRegistryLifeCycle() throws Exception { taskManagerServices.getMemoryManager(), taskManagerServices.getIOManager(), taskManagerServices.getNetworkEnvironment(), + taskManagerServices.getCheckpointCacheManager(), highAvailabilityServices, taskManagerMetricGroup); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/SharedCacheRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/SharedCacheRegistryTest.java new file mode 100644 index 0000000000000..03e293a726749 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/SharedCacheRegistryTest.java @@ -0,0 +1,116 @@ +/* + * 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.CheckpointCache; +import org.apache.flink.runtime.checkpoint.SharedCacheRegistry; +import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.state.filesystem.FileStateHandle; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Random; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +/** + * Tests for SharedCacheRegistry + */ +public class SharedCacheRegistryTest { + + @Test + public void testRegister() throws Exception { + SharedCacheRegistry registry = new SharedCacheRegistry(Executors.directExecutor()); + CheckpointCache.CacheKey key = new CheckpointCache.CacheKey(new StateHandleID("hanleId1")); + CheckpointCache.CacheEntry entry1 = mock(CheckpointCache.CacheEntry.class); + CheckpointCache.CacheEntry entry2 = mock(CheckpointCache.CacheEntry.class); + + registry.registerReference(key, entry1); + registry.registerReference(key, entry2); + + TimeUnit.MILLISECONDS.sleep(500); + verify(entry2, times(1)).discard(); + verify(entry1, times(2)).increaseReferenceCount(); + } + + @Test + public void testConcurrencyRequest() throws Exception { + + ExecutorService executorService = java.util.concurrent.Executors.newFixedThreadPool(10); + + SharedCacheRegistry registry = new SharedCacheRegistry(Executors.directExecutor()); + + CheckpointCache.CacheKey[] keys = new CheckpointCache.CacheKey[10]; + int[] referCount = new int[keys.length]; + for (int i = 0; i < keys.length; ++i) { + keys[i] = new CheckpointCache.CacheKey(new StateHandleID("handleId" + i)); + // for prevent unregister except + for (int j = 0; j < 100; ++j) { + CheckpointCache.CacheEntry entry = new CheckpointCache.CacheEntry(mock(FileStateHandle.class)); + registry.registerReference(keys[i], entry); + } + Assert.assertEquals(100, registry.getCacheEntry(keys[i]).getReferenceCount()); + referCount[i] = 100; + } + + // init 100 runners + Runnable[] runnables = new Runnable[100]; + for (int i = 0; i < runnables.length; ++i) { + final int index = new Random().nextInt(keys.length); + final int opType = new Random().nextInt(2); + if (opType == 0) { + referCount[index]--; + } else { + referCount[index]++; + } + runnables[i] = () -> { + if (opType == 0) { + registry.unregisterReference(keys[index]); + } else { + CheckpointCache.CacheEntry entry = new CheckpointCache.CacheEntry(mock(FileStateHandle.class)); + registry.registerReference(keys[index], entry); + } + }; + } + + // submit runners + Future[] futures = new Future[runnables.length]; + for (int i = 0; i < runnables.length; ++i) { + futures[i] = executorService.submit(runnables[i]); + } + + // wait for finish + for (int i = 0; i < futures.length; ++i) { + futures[i].get(); + } + + // valid result + for (int i = 0; i < keys.length; ++i) { + if (referCount[i] == 0) { + Assert.assertNull(registry.getCacheEntry(keys[i])); + } else { + Assert.assertEquals(referCount[i], registry.getCacheEntry(keys[i]).getReferenceCount()); + } + } + } +} 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 1f1d09d1d88a6..6f00b127f307e 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 @@ -22,6 +22,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCacheManager; import org.apache.flink.runtime.clusterframework.FlinkResourceManager; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -151,6 +152,7 @@ public void testSlotAllocation() throws Exception { memoryManager, ioManager, networkEnvironment, + mock(CheckpointCacheManager.class), testingHAServices, heartbeatServices, taskManagerMetricGroup, 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 776bdf9a5a17e..f6d1740e4f7f7 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 @@ -26,6 +26,7 @@ import org.apache.flink.runtime.blob.PermanentBlobCache; import org.apache.flink.runtime.blob.TransientBlobCache; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCacheManager; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; @@ -212,6 +213,7 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation) thro mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), + mock(CheckpointCacheManager.class), haServices, heartbeatServices, mock(TaskManagerMetricGroup.class), @@ -317,6 +319,7 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), + mock(CheckpointCacheManager.class), haServices, heartbeatServices, mock(TaskManagerMetricGroup.class), @@ -434,6 +437,7 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), + mock(CheckpointCacheManager.class), haServices, heartbeatServices, mock(TaskManagerMetricGroup.class), @@ -526,6 +530,7 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), + mock(CheckpointCacheManager.class), haServices, mock(HeartbeatServices.class, RETURNS_MOCKS), mock(TaskManagerMetricGroup.class), @@ -608,6 +613,7 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), + mock(CheckpointCacheManager.class), haServices, mock(HeartbeatServices.class, RETURNS_MOCKS), mock(TaskManagerMetricGroup.class), @@ -694,7 +700,8 @@ public void testTaskSubmission() throws Exception { 0, null, Collections.emptyList(), - Collections.emptyList()); + Collections.emptyList(), + -1); final LibraryCacheManager libraryCacheManager = mock(LibraryCacheManager.class); when(libraryCacheManager.getClassLoader(any(JobID.class))).thenReturn(ClassLoader.getSystemClassLoader()); @@ -749,6 +756,7 @@ public void testTaskSubmission() throws Exception { mock(MemoryManager.class), mock(IOManager.class), networkEnvironment, + mock(CheckpointCacheManager.class), haServices, mock(HeartbeatServices.class, RETURNS_MOCKS), taskManagerMetricGroup, @@ -866,6 +874,7 @@ public void testJobLeaderDetection() throws Exception { mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), + mock(CheckpointCacheManager.class), haServices, mock(HeartbeatServices.class, RETURNS_MOCKS), mock(TaskManagerMetricGroup.class), @@ -985,6 +994,7 @@ public void testSlotAcceptance() throws Exception { mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), + mock(CheckpointCacheManager.class), haServices, mock(HeartbeatServices.class, RETURNS_MOCKS), mock(TaskManagerMetricGroup.class), @@ -1079,6 +1089,7 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), + mock(CheckpointCacheManager.class), haServices, mock(HeartbeatServices.class, RETURNS_MOCKS), mock(TaskManagerMetricGroup.class), @@ -1254,6 +1265,7 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { mock(MemoryManager.class), mock(IOManager.class), networkMock, + mock(CheckpointCacheManager.class), haServices, mock(HeartbeatServices.class, RETURNS_MOCKS), taskManagerMetricGroup, @@ -1304,7 +1316,8 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { 0, null, Collections.emptyList(), - Collections.emptyList()); + Collections.emptyList(), + -1); CompletableFuture> offerResultFuture = new CompletableFuture<>(); @@ -1376,6 +1389,7 @@ public void testFilterOutDuplicateJobMasterRegistrations() throws Exception { mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), + mock(CheckpointCacheManager.class), haServicesMock, heartbeatServicesMock, mock(TaskManagerMetricGroup.class), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java index 50456067cf955..0eb3a46c2110c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.blob.PermanentBlobCache; import org.apache.flink.runtime.blob.TransientBlobCache; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointOptions; @@ -263,6 +264,7 @@ private Task createTask(Class invokableClass) throw blobService, libCache, mock(FileCache.class), + mock(CheckpointCache.class), new TestingTaskManagerRuntimeInfo(), taskMetricGroup, consumableNotifier, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java index 31e9e223b6900..9afbbb8d4e850 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java @@ -2159,7 +2159,8 @@ private static TaskDeploymentDescriptor createTaskDeploymentDescriptor( targetSlotNumber, null, producedPartitions, - inputGates); + inputGates, + -1); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index b08999794e5c3..5dc1e049c594a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.blob.PermanentBlobCache; import org.apache.flink.runtime.blob.TransientBlobCache; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; @@ -1020,6 +1021,7 @@ private Task createTask( blobService, libCache, mock(FileCache.class), + mock(CheckpointCache.class), new TestingTaskManagerRuntimeInfo(taskManagerConfig), taskMetricGroup, consumableNotifier, 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 807229580855a..0becc4d5033c5 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 @@ -28,6 +28,7 @@ import org.apache.flink.runtime.blob.PermanentBlobCache; import org.apache.flink.runtime.blob.TransientBlobCache; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.AllocationID; @@ -189,6 +190,7 @@ public static void main(String[] args) throws Exception { FlinkUserCodeClassLoaders.ResolveOrder.CHILD_FIRST, new String[0]), new FileCache(tmInfo.getTmpDirectories()), + mock(CheckpointCache.class), tmInfo, new UnregisteredTaskMetricsGroup(), new NoOpResultPartitionConsumableNotifier(), 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..38ea4b519bccf 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 @@ -18,6 +18,7 @@ package org.apache.flink.runtime.testingUtils +import org.apache.flink.runtime.checkpoint.CheckpointCacheManager import org.apache.flink.runtime.clusterframework.types.ResourceID import org.apache.flink.runtime.highavailability.HighAvailabilityServices import org.apache.flink.runtime.io.disk.iomanager.IOManager @@ -38,6 +39,7 @@ class TestingTaskManager( memoryManager: MemoryManager, ioManager: IOManager, network: NetworkEnvironment, + checkpointCacheManager: CheckpointCacheManager, numberOfSlots: Int, highAvailabilityServices: HighAvailabilityServices, taskManagerMetricGroup : TaskManagerMetricGroup) @@ -48,6 +50,7 @@ class TestingTaskManager( memoryManager, ioManager, network, + checkpointCacheManager, numberOfSlots, highAvailabilityServices, taskManagerMetricGroup) @@ -59,6 +62,7 @@ class TestingTaskManager( memoryManager: MemoryManager, ioManager: IOManager, network: NetworkEnvironment, + checkpointCacheManager: CheckpointCacheManager, numberOfSlots: Int, highAvailabilityServices: HighAvailabilityServices, taskManagerMetricGroup : TaskManagerMetricGroup) { @@ -69,6 +73,7 @@ class TestingTaskManager( memoryManager, ioManager, network, + checkpointCacheManager, numberOfSlots, highAvailabilityServices, taskManagerMetricGroup) 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 c641aa80739a0..2ddcfce7c9218 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 @@ -28,6 +28,7 @@ import org.apache.flink.runtime.blob.PermanentBlobCache; import org.apache.flink.runtime.blob.TransientBlobCache; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.AllocationID; @@ -268,6 +269,7 @@ private static Task createTask( FlinkUserCodeClassLoaders.ResolveOrder.CHILD_FIRST, new String[0]), new FileCache(new String[] { EnvironmentInformation.getTemporaryFileDirectory() }), + mock(CheckpointCache.class), new TestingTaskManagerRuntimeInfo(), new UnregisteredTaskMetricsGroup(), mock(ResultPartitionConsumableNotifier.class), 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 4c73e7254835e..ab9a24320b2ec 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 @@ -27,6 +27,7 @@ import org.apache.flink.runtime.blob.PermanentBlobCache; import org.apache.flink.runtime.blob.TransientBlobCache; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.concurrent.Executors; @@ -164,6 +165,7 @@ public void testConcurrentAsyncCheckpointCannotFailFinishedStreamTask() throws E FlinkUserCodeClassLoaders.ResolveOrder.CHILD_FIRST, new String[0]), mock(FileCache.class), + mock(CheckpointCache.class), taskManagerRuntimeInfo, new UnregisteredTaskMetricsGroup(), mock(ResultPartitionConsumableNotifier.class), 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 b31fb41993686..93354a7056f34 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 @@ -31,6 +31,7 @@ import org.apache.flink.runtime.blob.PermanentBlobCache; import org.apache.flink.runtime.blob.TransientBlobCache; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointOptions; @@ -931,6 +932,7 @@ public static Task createTask( blobService, libCache, mock(FileCache.class), + mock(CheckpointCache.class), new TestingTaskManagerRuntimeInfo(taskManagerConfig, new String[] {System.getProperty("java.io.tmpdir")}), new UnregisteredTaskMetricsGroup(), consumableNotifier, 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 d755c566efe62..3e3fe720e0679 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 @@ -27,6 +27,7 @@ import org.apache.flink.runtime.blob.PermanentBlobCache; import org.apache.flink.runtime.blob.TransientBlobCache; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointOptions; @@ -240,6 +241,7 @@ private static Task createTask( FlinkUserCodeClassLoaders.ResolveOrder.CHILD_FIRST, new String[0]), new FileCache(new String[] { EnvironmentInformation.getTemporaryFileDirectory() }), + mock(CheckpointCache.class), new TestingTaskManagerRuntimeInfo(), new UnregisteredTaskMetricsGroup(), mock(ResultPartitionConsumableNotifier.class), 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 228eaaa0a74b0..f5c5e138ea724 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 @@ -18,6 +18,7 @@ package org.apache.flink.yarn +import org.apache.flink.runtime.checkpoint.CheckpointCacheManager import org.apache.flink.runtime.clusterframework.types.ResourceID import org.apache.flink.runtime.highavailability.HighAvailabilityServices import org.apache.flink.runtime.io.disk.iomanager.IOManager @@ -50,6 +51,7 @@ class TestingYarnTaskManager( memoryManager: MemoryManager, ioManager: IOManager, network: NetworkEnvironment, + checkpointCacheManager: CheckpointCacheManager, numberOfSlots: Int, highAvailabilityServices: HighAvailabilityServices, taskManagerMetricGroup : TaskManagerMetricGroup) @@ -60,6 +62,7 @@ class TestingYarnTaskManager( memoryManager, ioManager, network, + checkpointCacheManager, 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 615466db0e8be..b70f895a4a55f 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 @@ -18,6 +18,7 @@ package org.apache.flink.yarn +import org.apache.flink.runtime.checkpoint.CheckpointCacheManager import org.apache.flink.runtime.clusterframework.types.ResourceID import org.apache.flink.runtime.highavailability.HighAvailabilityServices import org.apache.flink.runtime.io.disk.iomanager.IOManager @@ -37,6 +38,7 @@ class YarnTaskManager( memoryManager: MemoryManager, ioManager: IOManager, network: NetworkEnvironment, + checkpointCacheManager: CheckpointCacheManager, numberOfSlots: Int, highAvailabilityServices: HighAvailabilityServices, taskManagerMetricGroup: TaskManagerMetricGroup) @@ -47,6 +49,7 @@ class YarnTaskManager( memoryManager, ioManager, network, + checkpointCacheManager, numberOfSlots, highAvailabilityServices, taskManagerMetricGroup) { From b40fd80a57f59725eb888f2d6b56be97da0a5818 Mon Sep 17 00:00:00 2001 From: summerleafs Date: Fri, 24 Nov 2017 14:00:38 +0800 Subject: [PATCH 2/7] add checkpoint cache for HeapKeyedBackend. fix tests. fix migrate bug. --- .../state/RocksDBKeyedStateBackend.java | 9 ++- .../streaming/state/RocksDBStateBackend.java | 17 +++-- .../network/KvStateRequestSerializerTest.java | 7 ++- .../runtime/checkpoint/CheckpointCache.java | 1 + .../checkpoint/CheckpointCacheManager.java | 7 +++ .../flink/runtime/execution/Environment.java | 7 +++ .../state/CachedCheckpointStreamFactory.java | 63 ++++++++++++------- .../state/filesystem/FsStateBackend.java | 17 ++--- .../state/heap/HeapKeyedStateBackend.java | 20 +++++- .../state/memory/MemoryStateBackend.java | 15 ++--- .../taskmanager/RuntimeEnvironment.java | 1 + .../operators/testutils/DummyEnvironment.java | 6 ++ .../operators/testutils/MockEnvironment.java | 6 ++ .../state/StateSnapshotCompressionTest.java | 13 ++-- .../state/heap/HeapStateBackendTestBase.java | 4 +- ...kManagerComponentsStartupShutdownTest.java | 6 ++ .../streaming/runtime/tasks/StreamTask.java | 5 ++ .../runtime/tasks/StreamMockEnvironment.java | 6 ++ 18 files changed, 150 insertions(+), 60 deletions(-) diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index dee1accd3d9e3..35e558ea377d0 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -1566,7 +1566,7 @@ private void readAllStateData( ((CachedStreamStateHandle) remoteFileHandle).setCheckpointCache(stateBackend.cache); ((CachedStreamStateHandle) remoteFileHandle).reCache(!hasExtraKeys); } - readStateData(new Path(restoreInstancePath, stateHandleID.toString()), remoteFileHandle); + readStateData(new Path(restoreInstancePath, getFileNameFromStateHanldeId(stateHandleID)), remoteFileHandle); } } @@ -1575,13 +1575,18 @@ private void createFileHardLinksInRestorePath( Path restoreInstancePath) throws IOException { for (StateHandleID stateHandleID : stateHandleMap.keySet()) { - String newSstFileName = stateHandleID.toString(); + String newSstFileName = getFileNameFromStateHanldeId(stateHandleID); File restoreFile = new File(restoreInstancePath.getPath(), newSstFileName); File targetFile = new File(stateBackend.instanceRocksDBPath, newSstFileName); Files.createLink(targetFile.toPath(), restoreFile.toPath()); } } + private String getFileNameFromStateHanldeId(StateHandleID handleID) { + final String[] arr = handleID.toString().split("\\$"); + return arr[arr.length - 1]; + } + void restore(Collection restoreStateHandles) throws Exception { boolean hasExtraKeys = (restoreStateHandles.size() > 1 || diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java index 8301b4455142f..1a9da8ef0ccdd 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java @@ -30,7 +30,6 @@ import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.OperatorStateBackend; import org.apache.flink.runtime.state.filesystem.FsStateBackend; -import org.apache.flink.runtime.taskmanager.RuntimeEnvironment; import org.apache.flink.util.AbstractID; import org.rocksdb.ColumnFamilyOptions; @@ -285,13 +284,13 @@ public CheckpointStreamFactory createSavepointStreamFactory( @Override public AbstractKeyedStateBackend createKeyedStateBackend( - Environment env, - JobID jobID, - String operatorIdentifier, - TypeSerializer keySerializer, - int numberOfKeyGroups, - KeyGroupRange keyGroupRange, - TaskKvStateRegistry kvStateRegistry) throws IOException { + Environment env, + JobID jobID, + String operatorIdentifier, + TypeSerializer keySerializer, + int numberOfKeyGroups, + KeyGroupRange keyGroupRange, + TaskKvStateRegistry kvStateRegistry) throws IOException { // first, make sure that the RocksDB JNI library is loaded // we do this explicitly here to have better error handling @@ -314,7 +313,7 @@ public AbstractKeyedStateBackend createKeyedStateBackend( numberOfKeyGroups, keyGroupRange, env.getExecutionConfig(), - ((RuntimeEnvironment) env).getCheckpointCache(), + env.getCheckpointCache(), enableIncrementalCheckpointing); } 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..cf7650942101e 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 @@ -28,6 +28,7 @@ import org.apache.flink.queryablestate.client.VoidNamespace; import org.apache.flink.queryablestate.client.VoidNamespaceSerializer; import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend; @@ -194,7 +195,8 @@ public void testListSerialization() throws Exception { 1, new KeyGroupRange(0, 0), async, - new ExecutionConfig() + new ExecutionConfig(), + mock(CheckpointCache.class) ); longHeapKeyedStateBackend.setCurrentKey(key); @@ -296,7 +298,8 @@ public void testMapSerialization() throws Exception { 1, new KeyGroupRange(0, 0), async, - new ExecutionConfig() + new ExecutionConfig(), + mock(CheckpointCache.class) ); longHeapKeyedStateBackend.setCurrentKey(key); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java index f858814f12073..61d7ff5ee11d7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java @@ -183,6 +183,7 @@ public void commitCache(long checkpointID, boolean dropUnRetainCheckpointCache) } this.completedCheckpointCaches.add(completedCheckpointCache); + pendingCheckpointCache.cancelCanceller(); if (dropUnRetainCheckpointCache) { // only maintain the last complete checkpoint diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManager.java index bcb7a4ba77259..2fb99e1033b81 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManager.java @@ -48,6 +48,7 @@ public class CheckpointCacheManager { private final Map> cacheClearFutures; private final Executor executor; + private boolean isShutdown; public CheckpointCacheManager(ScheduledExecutorService scheduledExecutorService, Executor executor, String basePath) { this.scheduledExecutorService = scheduledExecutorService; @@ -55,6 +56,7 @@ public CheckpointCacheManager(ScheduledExecutorService scheduledExecutorService, this.basePath = new Path(basePath); this.checkpointCaches = new ConcurrentHashMap<>(); this.cacheClearFutures = new ConcurrentHashMap<>(); + this.isShutdown = false; } /** Reregister a {@linke CheckpointCache} from {@link CheckpointCacheManager}, create a new one or refer a exists one. */ @@ -123,6 +125,7 @@ public void shutdown() { checkpointCache.release(); } checkpointCaches.clear(); + isShutdown = true; } } @@ -144,4 +147,8 @@ protected int getCheckpointCacheSize() { protected ScheduledFuture getCacheClearFuture(JobID jobID) { return cacheClearFutures.get(jobID); } + + public boolean isShutdown() { + return this.isShutdown; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java index 203ee8547cf42..32ecdfdc105f5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java @@ -25,6 +25,7 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -198,6 +199,12 @@ public interface Environment { */ void failExternally(Throwable cause); + /** + * Get the CheckpointCache for manage local checkpoint data. + * @return The CheckpointCache instance + */ + CheckpointCache getCheckpointCache(); + // -------------------------------------------------------------------------------------------- // Fields relevant to the I/O system. Should go into Task // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedCheckpointStreamFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedCheckpointStreamFactory.java index 9d1f4ddbe59a1..b704819d69b17 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedCheckpointStreamFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedCheckpointStreamFactory.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.checkpoint.CachedStreamStateHandle; import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointCache.CachedOutputStream; +import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,7 +42,7 @@ public class CachedCheckpointStreamFactory implements CheckpointStreamFactory { public CachedCheckpointStreamFactory(CheckpointCache cache, CheckpointStreamFactory factory) { this.cache = cache; - this.remoteFactory = factory; + this.remoteFactory = Preconditions.checkNotNull(factory, "Remote stream factory is null."); } public CheckpointStateOutputStream createCheckpointStateOutputStream(long checkpointID, long timestamp, StateHandleID handleID) throws Exception { @@ -52,7 +53,10 @@ public CheckpointStateOutputStream createCheckpointStateOutputStream(long checkp if (LOG.isDebugEnabled()) { LOG.debug("create cache output stream: cpkID:{} placeHolder:{}", checkpointID, placeholder); } - CachedOutputStream cachedOut = cache.createOutputStream(checkpointID, handleID, placeholder); + CachedOutputStream cachedOut = null; + if (cache != null) { + cachedOut = cache.createOutputStream(checkpointID, handleID, placeholder); + } CheckpointStateOutputStream remoteOut = null; if (!placeholder) { remoteOut = remoteFactory.createCheckpointStateOutputStream(checkpointID, timestamp); @@ -89,18 +93,21 @@ public CachedCheckpointStateOutputStream(CachedOutputStream cacheOut, Checkpoint @Override public StreamStateHandle closeAndGetHandle() throws IOException { - - // finalize cache data - StateHandleID cacheId = cacheOut.getCacheID(); - cacheOut.end(); - - StreamStateHandle remoteHandle; - if (remoteOut != null) { - remoteHandle = remoteOut.closeAndGetHandle(); + if (cacheOut != null) { + // finalize cache data + StateHandleID cacheId = cacheOut.getCacheID(); + cacheOut.end(); + + StreamStateHandle remoteHandle; + if (remoteOut != null) { + remoteHandle = remoteOut.closeAndGetHandle(); + } else { + remoteHandle = new PlaceholderStreamStateHandle(cacheId); + } + return new CachedStreamStateHandle(cacheId, remoteHandle); } else { - remoteHandle = new PlaceholderStreamStateHandle(cacheId); + return remoteOut.closeAndGetHandle(); } - return new CachedStreamStateHandle(cacheId, remoteHandle); } @Override @@ -111,11 +118,13 @@ public long getPos() throws IOException { @Override public void write(int b) throws IOException { // write to local - try { - cacheOut.write(b); - } catch (Exception e) { - //discard - cacheOut.discard(); + if (cacheOut != null) { + try { + cacheOut.write(b); + } catch (Exception e) { + //discard + cacheOut.discard(); + } } // write to remote @@ -127,11 +136,13 @@ public void write(int b) throws IOException { @Override public void write(byte[] b, int off, int len) throws IOException { // write to local - try { - cacheOut.write(b, off, len); - } catch (Exception e) { - //discard - cacheOut.discard(); + if (cacheOut != null) { + try { + cacheOut.write(b, off, len); + } catch (Exception e) { + //discard + cacheOut.discard(); + } } // write to remote @@ -142,7 +153,9 @@ public void write(byte[] b, int off, int len) throws IOException { @Override public void flush() throws IOException { - cacheOut.flush(); + if (cacheOut != null) { + cacheOut.flush(); + } if (remoteOut != null) { remoteOut.flush(); } @@ -157,7 +170,9 @@ public void sync() throws IOException { @Override public void close() throws IOException { - cacheOut.close(); + if (cacheOut != null) { + cacheOut.close(); + } if (remoteOut != null) { remoteOut.close(); } 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 952988f72d5d0..9083d1d042cb1 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 @@ -288,13 +288,13 @@ public CheckpointStreamFactory createSavepointStreamFactory( @Override public AbstractKeyedStateBackend createKeyedStateBackend( - Environment env, - JobID jobID, - String operatorIdentifier, - TypeSerializer keySerializer, - int numberOfKeyGroups, - KeyGroupRange keyGroupRange, - TaskKvStateRegistry kvStateRegistry) throws IOException { + Environment env, + JobID jobID, + String operatorIdentifier, + TypeSerializer keySerializer, + int numberOfKeyGroups, + KeyGroupRange keyGroupRange, + TaskKvStateRegistry kvStateRegistry) throws IOException { return new HeapKeyedStateBackend<>( kvStateRegistry, @@ -303,7 +303,8 @@ public AbstractKeyedStateBackend createKeyedStateBackend( numberOfKeyGroups, keyGroupRange, asynchronousSnapshots, - env.getExecutionConfig()); + env.getExecutionConfig(), + env.getCheckpointCache()); } @Override 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 5255b7bae9cd1..25603411ce6bb 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 @@ -35,12 +35,14 @@ import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.io.async.AbstractAsyncCallableWithResources; import org.apache.flink.runtime.io.async.AsyncStoppableTaskWithCallback; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.AbstractKeyedStateBackend; import org.apache.flink.runtime.state.ArrayListSerializer; +import org.apache.flink.runtime.state.CachedCheckpointStreamFactory; import org.apache.flink.runtime.state.CheckpointStreamFactory; import org.apache.flink.runtime.state.DoneFuture; import org.apache.flink.runtime.state.HashMapSerializer; @@ -51,6 +53,7 @@ import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo; import org.apache.flink.runtime.state.SnappyStreamCompressionDecorator; +import org.apache.flink.runtime.state.StateHandleID; import org.apache.flink.runtime.state.StreamCompressionDecorator; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.UncompressedStreamCompressionDecorator; @@ -77,6 +80,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.UUID; import java.util.concurrent.RunnableFuture; import java.util.stream.Stream; @@ -116,6 +120,10 @@ public class HeapKeyedStateBackend extends AbstractKeyedStateBackend { */ private final boolean asynchronousSnapshots; + private final CheckpointCache checkpointCache; + + private final UUID backendUID; + public HeapKeyedStateBackend( TaskKvStateRegistry kvStateRegistry, TypeSerializer keySerializer, @@ -123,13 +131,16 @@ public HeapKeyedStateBackend( int numberOfKeyGroups, KeyGroupRange keyGroupRange, boolean asynchronousSnapshots, - ExecutionConfig executionConfig) { + ExecutionConfig executionConfig, + CheckpointCache checkpointCache) { super(kvStateRegistry, keySerializer, userCodeClassLoader, numberOfKeyGroups, keyGroupRange, executionConfig); this.asynchronousSnapshots = asynchronousSnapshots; LOG.info("Initializing heap keyed state backend with stream factory."); this.restoredKvStateMetaInfos = new HashMap<>(); + this.checkpointCache = checkpointCache; + this.backendUID = UUID.randomUUID(); } // ------------------------------------------------------------------------ @@ -333,6 +344,7 @@ public RunnableFuture snapshot( !Objects.equals(UncompressedStreamCompressionDecorator.INSTANCE, keyGroupCompressionDecorator)); //--------------------------------------------------- this becomes the end of sync part + final CachedCheckpointStreamFactory cachedCheckpointStreamFactory = new CachedCheckpointStreamFactory(checkpointCache, streamFactory); // implementation of the async IO operation, based on FutureTask final AbstractAsyncCallableWithResources ioCallable = @@ -342,7 +354,8 @@ public RunnableFuture snapshot( @Override protected void acquireResources() throws Exception { - stream = streamFactory.createCheckpointStateOutputStream(checkpointId, timestamp); + stream = cachedCheckpointStreamFactory.createCheckpointStateOutputStream( + checkpointId, timestamp, new StateHandleID(backendUID + "-" + checkpointId)); cancelStreamRegistry.registerCloseable(stream); } @@ -399,7 +412,7 @@ public KeyGroupsStateHandle performOperation() throws Exception { if (asynchronousSnapshots) { LOG.info("Heap backend snapshot ({}, asynchronous part) in thread {} took {} ms.", - streamFactory, Thread.currentThread(), (System.currentTimeMillis() - asyncStartTime)); + cachedCheckpointStreamFactory, Thread.currentThread(), (System.currentTimeMillis() - asyncStartTime)); } if (streamStateHandle != null) { @@ -468,6 +481,7 @@ private void restorePartitionedState(Collection state) throws } KeyGroupsStateHandle keyGroupsStateHandle = (KeyGroupsStateHandle) keyedStateHandle; + keyGroupsStateHandle.setCache(checkpointCache); FSDataInputStream fsDataInputStream = keyGroupsStateHandle.openInputStream(); cancelStreamRegistry.registerCloseable(fsDataInputStream); 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 b8ebedfe9285b..8488c739c0739 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 @@ -123,12 +123,12 @@ public CheckpointStreamFactory createSavepointStreamFactory( @Override public AbstractKeyedStateBackend createKeyedStateBackend( - Environment env, JobID jobID, - String operatorIdentifier, - TypeSerializer keySerializer, - int numberOfKeyGroups, - KeyGroupRange keyGroupRange, - TaskKvStateRegistry kvStateRegistry) { + Environment env, JobID jobID, + String operatorIdentifier, + TypeSerializer keySerializer, + int numberOfKeyGroups, + KeyGroupRange keyGroupRange, + TaskKvStateRegistry kvStateRegistry) { return new HeapKeyedStateBackend<>( kvStateRegistry, @@ -137,6 +137,7 @@ public AbstractKeyedStateBackend createKeyedStateBackend( numberOfKeyGroups, keyGroupRange, asynchronousSnapshots, - env.getExecutionConfig()); + env.getExecutionConfig(), + env.getCheckpointCache()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java index 626b492f2e66a..ad5be330e9e94 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java @@ -267,6 +267,7 @@ public void failExternally(Throwable cause) { this.containingTask.failExternally(cause); } + @Override public CheckpointCache getCheckpointCache() { return checkpointCache; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java index 0125a5e9fc3b7..87baa86f971e9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java @@ -25,6 +25,7 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.execution.Environment; @@ -170,6 +171,11 @@ public void failExternally(Throwable cause) { throw new UnsupportedOperationException("DummyEnvironment does not support external task failure."); } + @Override + public CheckpointCache getCheckpointCache() { + return null; + } + @Override public ResultPartitionWriter getWriter(int index) { return null; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java index 7514cc4200d74..e5e9c6816639b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java @@ -26,6 +26,7 @@ import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.execution.Environment; @@ -367,4 +368,9 @@ public void declineCheckpoint(long checkpointId, Throwable cause) { public void failExternally(Throwable cause) { throw new UnsupportedOperationException("MockEnvironment does not support external task failure."); } + + @Override + public CheckpointCache getCheckpointCache() { + return mock(CheckpointCache.class); + } } 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 168ed9730915a..3aedf24161a4b 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend; @@ -52,7 +53,8 @@ public void testCompressionConfiguration() throws Exception { 16, new KeyGroupRange(0, 15), true, - executionConfig); + executionConfig, + mock(CheckpointCache.class)); try { Assert.assertTrue( @@ -73,7 +75,8 @@ public void testCompressionConfiguration() throws Exception { 16, new KeyGroupRange(0, 15), true, - executionConfig); + executionConfig, + mock(CheckpointCache.class)); try { Assert.assertTrue( @@ -112,7 +115,8 @@ private void snapshotRestoreRoundtrip(boolean useCompression) throws Exception { 16, new KeyGroupRange(0, 15), true, - executionConfig); + executionConfig, + mock(CheckpointCache.class)); try { @@ -153,7 +157,8 @@ private void snapshotRestoreRoundtrip(boolean useCompression) throws Exception { 16, new KeyGroupRange(0, 15), true, - executionConfig); + executionConfig, + mock(CheckpointCache.class)); try { stateBackend.restore(Collections.singletonList(stateHandle)); 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..9e806814479ab 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.KeyGroupRange; import org.junit.runner.RunWith; @@ -54,6 +55,7 @@ public HeapKeyedStateBackend createKeyedBackend(TypeSerializer keySeri 16, new KeyGroupRange(0, 15), async, - new ExecutionConfig()); + new ExecutionConfig(), + mock(CheckpointCache.class)); } } 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 98b5b8b111693..6e4927c38177d 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 @@ -24,9 +24,11 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.checkpoint.CheckpointCacheManager; 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; @@ -155,6 +157,8 @@ public void testComponentsStartupShutdown() throws Exception { network.start(); + final CheckpointCacheManager checkpointCacheManager = new CheckpointCacheManager(java.util.concurrent.Executors.newSingleThreadScheduledExecutor(), Executors.directExecutor(), TMP_DIR[0]); + MetricRegistryConfiguration metricRegistryConfiguration = MetricRegistryConfiguration.fromConfiguration(config); // create the task manager @@ -166,6 +170,7 @@ public void testComponentsStartupShutdown() throws Exception { memManager, ioManager, network, + checkpointCacheManager, numberOfSlots, highAvailabilityServices, new TaskManagerMetricGroup(new NoOpMetricRegistry(), connectionInfo.getHostname(), connectionInfo.getResourceID().getResourceIdString())); @@ -202,6 +207,7 @@ protected void run() { assertTrue(network.isShutdown()); assertTrue(ioManager.isProperlyShutDown()); assertTrue(memManager.isShutdown()); + assertTrue(checkpointCacheManager.isShutdown()); } finally { if (actorSystem != null) { actorSystem.shutdown(); 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 eff8a291fb791..5de7039f908b0 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 @@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.core.fs.FileSystemSafetyNet; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointOptions; @@ -664,6 +665,10 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { operator.notifyOfCompletedCheckpoint(checkpointId); } } + final CheckpointCache checkpointCache = getEnvironment().getCheckpointCache(); + if (checkpointCache != null) { + checkpointCache.commitCache(checkpointId); + } } else { LOG.debug("Ignoring notification of complete checkpoint for not-running task {}", getName()); 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 231f59e97fb2a..d61b669a19ddb 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 @@ -27,6 +27,7 @@ import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.CheckpointCache; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.event.AbstractEvent; @@ -344,6 +345,11 @@ public void failExternally(Throwable cause) { this.wasFailedExternally = true; } + @Override + public CheckpointCache getCheckpointCache() { + return mock(CheckpointCache.class); + } + public boolean wasFailedExternally() { return wasFailedExternally; } From c912f04d70035b1995446a03f531a19da163897e Mon Sep 17 00:00:00 2001 From: summerleafs Date: Mon, 27 Nov 2017 14:46:59 +0800 Subject: [PATCH 3/7] fix build. --- .../streaming/state/RocksDBKeyedStateBackend.java | 4 +++- .../runtime/checkpoint/CachedStreamStateHandle.java | 9 +++++++-- .../runtime/state/CachedCheckpointStreamFactory.java | 6 +++++- 3 files changed, 15 insertions(+), 4 deletions(-) diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index 35e558ea377d0..47c3d28767e60 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -1544,7 +1544,9 @@ private void restoreInstance( stateBackend.lastCompletedCheckpointId = restoreStateHandle.getCheckpointId(); // commit re-cache - stateBackend.cache.commitCache(CheckpointCache.CHECKPOINT_ID_FOR_RESTORE, false); + if (stateBackend.cache != null) { + stateBackend.cache.commitCache(CheckpointCache.CHECKPOINT_ID_FOR_RESTORE, false); + } } } finally { FileSystem restoreFileSystem = restoreInstancePath.getFileSystem(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CachedStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CachedStreamStateHandle.java index 72534421b08b4..a99315499ca73 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CachedStreamStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CachedStreamStateHandle.java @@ -37,6 +37,8 @@ */ public class CachedStreamStateHandle implements StreamStateHandle, CachedStateHandle { + private static final long serialVersionUID = 350284443258002366L; + private static Logger LOG = LoggerFactory.getLogger(CachedStreamStateHandle.class); private transient CheckpointCache cache; @@ -137,12 +139,15 @@ public int read(byte[] b) throws IOException { @Override public void seek(long desired) throws IOException { - throw new FlinkRuntimeException("Unsupported method in CachedSteamStateHandle."); + this.remoteInputStream.seek(desired); + if (cacheOut != null) { + cacheOut.discard(); + } } @Override public long getPos() throws IOException { - throw new FlinkRuntimeException("Unsupported method in CachedSteamStateHandle."); + return this.remoteInputStream.getPos(); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedCheckpointStreamFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedCheckpointStreamFactory.java index b704819d69b17..63c0f00953690 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedCheckpointStreamFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedCheckpointStreamFactory.java @@ -106,7 +106,11 @@ public StreamStateHandle closeAndGetHandle() throws IOException { } return new CachedStreamStateHandle(cacheId, remoteHandle); } else { - return remoteOut.closeAndGetHandle(); + if (remoteOut != null) { + return remoteOut.closeAndGetHandle(); + } else { + return null; + } } } From 1bd8a9b044646aa89f5771bacbe550f989fa3d5c Mon Sep 17 00:00:00 2001 From: summerleafs Date: Mon, 27 Nov 2017 18:18:46 +0800 Subject: [PATCH 4/7] fix build. --- .../apache/flink/runtime/checkpoint/CachedStreamStateHandle.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CachedStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CachedStreamStateHandle.java index a99315499ca73..d5fee0c5ed1fa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CachedStreamStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CachedStreamStateHandle.java @@ -22,7 +22,6 @@ import org.apache.flink.runtime.state.CachedStateHandle; import org.apache.flink.runtime.state.StateHandleID; import org.apache.flink.runtime.state.StreamStateHandle; -import org.apache.flink.util.FlinkRuntimeException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From aa5a57a188f83018d6cef0b620e88731087808b6 Mon Sep 17 00:00:00 2001 From: summerleafs Date: Wed, 29 Nov 2017 20:12:42 +0800 Subject: [PATCH 5/7] Decouple the CachedOutputStream from the interface and provide a file-based implementation FsCachedOutputStream. --- .../checkpoint/CachedStreamStateHandle.java | 2 +- .../runtime/checkpoint/CheckpointCache.java | 136 +++++++++++------- .../checkpoint/CheckpointCacheManager.java | 14 +- .../deployment/TaskDeploymentDescriptor.java | 11 +- .../executiongraph/ExecutionVertex.java | 12 +- .../restart/FixedDelayRestartStrategy.java | 4 + .../state/CachedCheckpointStreamFactory.java | 40 +++++- .../runtime/taskexecutor/TaskExecutor.java | 20 +-- .../taskexecutor/TaskManagerServices.java | 2 +- .../runtime/taskmanager/TaskManager.scala | 19 +-- .../CheckpointCacheManagerTest.java | 8 +- .../checkpoint/CheckpointCacheTest.java | 45 +++--- .../TaskDeploymentDescriptorTest.java | 1 + .../taskexecutor/TaskExecutorTest.java | 2 + ...kManagerComponentsStartupShutdownTest.java | 2 +- .../runtime/taskmanager/TaskManagerTest.java | 1 + 16 files changed, 188 insertions(+), 131 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CachedStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CachedStreamStateHandle.java index d5fee0c5ed1fa..28e5f1c432eaf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CachedStreamStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CachedStreamStateHandle.java @@ -152,7 +152,7 @@ public long getPos() throws IOException { @Override public void close() throws IOException { if (this.cacheOut != null) { - this.cacheOut.end(); + this.cacheOut.closeAndGetHandle(); } this.remoteInputStream.close(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java index 61d7ff5ee11d7..0a7403504e038 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java @@ -21,8 +21,10 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.state.CheckpointStreamFactory; import org.apache.flink.runtime.state.StateHandleID; -import org.apache.flink.runtime.state.filesystem.FileStateHandle; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory; import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory; import org.apache.flink.util.FlinkRuntimeException; import org.slf4j.Logger; @@ -30,16 +32,13 @@ import java.io.File; import java.io.FileNotFoundException; -import java.io.FileOutputStream; import java.io.IOException; -import java.io.OutputStream; import java.util.ArrayDeque; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.Map; import java.util.Set; -import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledFuture; @@ -113,7 +112,7 @@ public CheckpointCache(JobID jobID, String basePath, long pendingCheckpointCache LOG.info("new checkpoint cache, pendingCacheTimeout: {}, leaseTimeout: {}", pendingCheckpointCacheTimeout, leaseTimeout); } - protected void registerCacheEntry(long checkpointID, StateHandleID handleID, String filePath) { + protected void registerCacheEntry(long checkpointID, StateHandleID handleID, StreamStateHandle stateHandle) { synchronized (lock) { LOG.debug("register cache entry: { cpkID:[{}] handleID:[{}] }", checkpointID, handleID); PendingCheckpointCache pendingCheckpointCache = pendingCacheMap.get(checkpointID); @@ -143,16 +142,7 @@ protected void registerCacheEntry(long checkpointID, StateHandleID handleID, Str pendingCheckpointCache.addEntry( new CacheKey(handleID), - new CacheEntry(new FileStateHandle(new Path(filePath), getFileSize(filePath)))); - } - } - - private long getFileSize(String filePath) { - File file = new File(filePath); - if (file.exists() && file.isFile()) { - return file.length(); - } else { - return 0L; + new CacheEntry(stateHandle)); } } @@ -266,8 +256,7 @@ public CachedOutputStream createOutputStream(long checkpointID, StateHandleID ha return null; } } - final String cacheFilePath = basePath + File.separator + handleID + "_" + UUID.randomUUID(); - return new CachedOutputStream(checkpointID, handleID, cacheFilePath, this, placeholder); + return new FsCachedOutputStream(checkpointID, handleID, basePath, this, placeholder); } catch (Exception ignore) { // warning LOG.warn("create output stream failed: {}", ignore); @@ -280,7 +269,7 @@ public FSDataInputStream openInputStream(StateHandleID cacheId) { CacheEntry entry = sharedCacheRegistry.getCacheEntry(new CacheKey(cacheId)); if (entry != null) { try { - LOG.debug("entry path: {}", entry.getHandle().getFilePath()); + LOG.debug("entry path: {}", entry.getHandle()); return entry.getHandle().openInputStream(); } catch (Exception ignore) { entry.rot(true); @@ -302,80 +291,123 @@ public int getReference() { return this.reference.get(); } - public static class CachedOutputStream extends OutputStream { + public long getLeaseTimeout() { + return leaseTimeout; + } + + /** + * CachedOutputStream interface + */ + public abstract static class CachedOutputStream extends CheckpointStreamFactory.CheckpointStateOutputStream { + private boolean discarded; + + public CachedOutputStream() { + discarded = false; + } + + public boolean isDiscarded() { + return discarded; + } + + public void discard() { + this.discarded = true; + } + + public abstract StateHandleID getCacheID(); + } + + /** + * {@link CachedOutputStream} which is implemented base on File System + */ + public static class FsCachedOutputStream extends CachedOutputStream { - private final OutputStream outputStream; + private FsCheckpointStreamFactory.FsCheckpointStateOutputStream outputStream; private final StateHandleID cacheID; private final long checkpointID; - private final String cacheFilePath; + private final Path cacheBasePath; private final CheckpointCache cache; - private boolean discarded; - public CachedOutputStream( + public FsCachedOutputStream( long checkpointID, StateHandleID cacheID, - String cacheFilePath, + Path basePath, CheckpointCache cache, boolean placeholder ) throws FileNotFoundException { + super(); + this.checkpointID = checkpointID; this.cacheID = cacheID; - this.cacheFilePath = cacheFilePath; + this.cacheBasePath = basePath; if (!placeholder) { - this.outputStream = new FileOutputStream(cacheFilePath); + try { + this.outputStream = new FsCheckpointStreamFactory.FsCheckpointStateOutputStream( + cacheBasePath, + cacheBasePath.getFileSystem(), + 4096, + 1024 * 1024); + } catch (Exception ignored) { + this.outputStream = null; + } } else { this.outputStream = null; } this.cache = cache; - this.discarded = false; } public long getCheckpointID() { return this.checkpointID; } + @Override public StateHandleID getCacheID() { return this.cacheID; } - public boolean isDiscarded() { - return this.discarded; - } - - public void discard() { - LOG.info("cache output stream discard: {}", checkpointID); - discarded = true; - } - @Override public void write(int b) throws IOException { - if (!discarded && outputStream != null) { + if (!isDiscarded() && outputStream != null) { outputStream.write(b); } } @Override public void write(byte[] b) throws IOException { - if (!discarded && outputStream != null) { + if (!isDiscarded() && outputStream != null) { outputStream.write(b); } } @Override public void write(byte[] b, int off, int len) throws IOException { - if (!discarded && outputStream != null) { + if (!isDiscarded() && outputStream != null) { outputStream.write(b, off, len); } } @Override public void flush() throws IOException { - if (!discarded && outputStream != null) { + if (!isDiscarded() && outputStream != null) { outputStream.flush(); } } + @Override + public void sync() throws IOException { + if (!isDiscarded() && outputStream != null) { + outputStream.sync(); + } + } + + @Override + public long getPos() throws IOException { + if (!isDiscarded() && outputStream != null) { + return outputStream.getPos(); + } + return -1; + } + @Override public void close() throws IOException { if (outputStream != null) { @@ -383,21 +415,17 @@ public void close() throws IOException { } } - public void end() { - if (!discarded) { - this.cache.registerCacheEntry(checkpointID, cacheID, cacheFilePath); + @Override + public StreamStateHandle closeAndGetHandle() throws IOException { + if (!isDiscarded() && outputStream != null) { + StreamStateHandle stateHandle = outputStream.closeAndGetHandle(); + this.cache.registerCacheEntry(checkpointID, cacheID, stateHandle); + return stateHandle; } else { this.cache.abortPendingCache(checkpointID); } + return null; } - - public String getCacheFilePath() { - return cacheFilePath; - } - } - - public long getLeaseTimeout() { - return leaseTimeout; } public static class CacheKey { @@ -433,7 +461,7 @@ public String toString() { } public static class CacheEntry { - private final FileStateHandle handle; + private final StreamStateHandle handle; private final AtomicInteger reference; public boolean isRot() { @@ -446,13 +474,13 @@ public void rot(boolean isRot) { private boolean rot; - public CacheEntry(FileStateHandle handle) { + public CacheEntry(StreamStateHandle handle) { this.handle = handle; this.reference = new AtomicInteger(0); this.rot = false; } - public FileStateHandle getHandle() { + public StreamStateHandle getHandle() { return handle; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManager.java index 2fb99e1033b81..e74e255b4343d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManager.java @@ -19,7 +19,6 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; -import org.apache.flink.core.fs.Path; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,8 +39,10 @@ public class CheckpointCacheManager { private static Logger LOG = LoggerFactory.getLogger(CheckpointCacheManager.class); + private final long DEFAULT_LEASE_TIMEOUT = 60000; private final Object lock = new Object(); - private final Path basePath; + private int nextBasePathIndex; + private final String[] basePaths; private final ScheduledExecutorService scheduledExecutorService; private final Map checkpointCaches; @@ -50,10 +51,10 @@ public class CheckpointCacheManager { private final Executor executor; private boolean isShutdown; - public CheckpointCacheManager(ScheduledExecutorService scheduledExecutorService, Executor executor, String basePath) { + public CheckpointCacheManager(ScheduledExecutorService scheduledExecutorService, Executor executor, String[] basePaths) { this.scheduledExecutorService = scheduledExecutorService; this.executor = executor; - this.basePath = new Path(basePath); + this.basePaths = basePaths; this.checkpointCaches = new ConcurrentHashMap<>(); this.cacheClearFutures = new ConcurrentHashMap<>(); this.isShutdown = false; @@ -68,8 +69,11 @@ public CheckpointCache registerCheckpointCache(JobID jobID, long pendingCheckpoi CheckpointCache checkpointCache = checkpointCaches.get(jobID); if (checkpointCache == null) { LOG.info("jobID: {} create checkpoint cache", jobID); + if (leaseTimeout < 0) { + leaseTimeout = DEFAULT_LEASE_TIMEOUT; + } checkpointCache = new CheckpointCache(jobID, - basePath.getPath(), + basePaths[nextBasePathIndex++ % basePaths.length], pendingCheckpointCacheTimeout, leaseTimeout, this, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java index 22f847fc41b67..83c3bd661cb36 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java @@ -148,6 +148,9 @@ public Offloaded(PermanentBlobKey serializedValueKey) { /** checkpoint timeout. */ private final long checkpointTimeout; + /** checkpoint cache lease timeout. */ + private final long leaseTimeout; + public TaskDeploymentDescriptor( JobID jobId, MaybeOffloaded serializedJobInformation, @@ -160,7 +163,8 @@ public TaskDeploymentDescriptor( TaskStateSnapshot taskStateHandles, Collection resultPartitionDeploymentDescriptors, Collection inputGateDeploymentDescriptors, - long checkpointTimeout) { + long checkpointTimeout, + long leaseTimeout) { this.jobId = Preconditions.checkNotNull(jobId); @@ -185,6 +189,7 @@ public TaskDeploymentDescriptor( this.inputGates = Preconditions.checkNotNull(inputGateDeploymentDescriptors); this.checkpointTimeout = checkpointTimeout; + this.leaseTimeout = leaseTimeout; } /** @@ -332,6 +337,10 @@ public long getCheckpointTimeout() { return this.checkpointTimeout; } + public long getLeaseTimeout() { + return leaseTimeout; + } + @Override public String toString() { return String.format("TaskDeploymentDescriptor [execution id: %s, attempt: %d, " + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index 74e43df08b38f..37e9a5cf4b1bb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -32,6 +32,8 @@ import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.restart.FixedDelayRestartStrategy; +import org.apache.flink.runtime.executiongraph.restart.RestartStrategy; import org.apache.flink.runtime.instance.SimpleSlot; import org.apache.flink.runtime.instance.SlotProvider; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; @@ -827,6 +829,13 @@ TaskDeploymentDescriptor createDeploymentDescriptor( // get jobCheckpointingSettings via job graph CheckpointCoordinatorConfiguration checkpointCoordinatorConfiguration = this.getExecutionGraph().getCheckpointCoordinatorConfiguration(); + + long leaseTimeout = 60000; //1 min + RestartStrategy restartStrategy = this.getExecutionGraph().getRestartStrategy(); + if (restartStrategy != null && restartStrategy instanceof FixedDelayRestartStrategy) { + leaseTimeout += ((FixedDelayRestartStrategy) restartStrategy).getDelayBetweenRestartAttempts(); + } + return new TaskDeploymentDescriptor( getJobId(), serializedJobInformation, @@ -839,7 +848,8 @@ TaskDeploymentDescriptor createDeploymentDescriptor( taskStateHandles, producedPartitions, consumedPartitions, - checkpointCoordinatorConfiguration == null ? -1L : checkpointCoordinatorConfiguration.getCheckpointTimeout()); + checkpointCoordinatorConfiguration == null ? -1L : checkpointCoordinatorConfiguration.getCheckpointTimeout(), + leaseTimeout); } // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java index 1916bea83586c..ed26514f3e66c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/restart/FixedDelayRestartStrategy.java @@ -49,6 +49,10 @@ public FixedDelayRestartStrategy( currentRestartAttempt = 0; } + public long getDelayBetweenRestartAttempts() { + return this.delayBetweenRestartAttempts; + } + public int getCurrentRestartAttempt() { return currentRestartAttempt; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedCheckpointStreamFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedCheckpointStreamFactory.java index 63c0f00953690..0ad6b9586aa27 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedCheckpointStreamFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CachedCheckpointStreamFactory.java @@ -96,7 +96,7 @@ public StreamStateHandle closeAndGetHandle() throws IOException { if (cacheOut != null) { // finalize cache data StateHandleID cacheId = cacheOut.getCacheID(); - cacheOut.end(); + cacheOut.closeAndGetHandle(); StreamStateHandle remoteHandle; if (remoteOut != null) { @@ -116,7 +116,17 @@ public StreamStateHandle closeAndGetHandle() throws IOException { @Override public long getPos() throws IOException { - return remoteOut != null ? remoteOut.getPos() :-1L; + if (cacheOut != null && remoteOut != null) { + // See if the position is aligned + try { + if (cacheOut.getPos() != remoteOut.getPos()) { + cacheOut.discard(); + } + } catch (Exception ignored) { + cacheOut.discard(); + } + } + return remoteOut != null ? remoteOut.getPos() : -1L; } @Override @@ -125,7 +135,7 @@ public void write(int b) throws IOException { if (cacheOut != null) { try { cacheOut.write(b); - } catch (Exception e) { + } catch (Exception ignored) { //discard cacheOut.discard(); } @@ -143,7 +153,7 @@ public void write(byte[] b, int off, int len) throws IOException { if (cacheOut != null) { try { cacheOut.write(b, off, len); - } catch (Exception e) { + } catch (Exception ignored) { //discard cacheOut.discard(); } @@ -158,7 +168,12 @@ public void write(byte[] b, int off, int len) throws IOException { @Override public void flush() throws IOException { if (cacheOut != null) { - cacheOut.flush(); + try { + cacheOut.flush(); + } catch (Exception ignored) { + //discard + cacheOut.discard(); + } } if (remoteOut != null) { remoteOut.flush(); @@ -167,6 +182,14 @@ public void flush() throws IOException { @Override public void sync() throws IOException { + if (cacheOut != null) { + try { + cacheOut.sync(); + } catch (Exception ignored) { + //discard + cacheOut.discard(); + } + } if (remoteOut != null) { remoteOut.sync(); } @@ -175,7 +198,12 @@ public void sync() throws IOException { @Override public void close() throws IOException { if (cacheOut != null) { - cacheOut.close(); + try { + cacheOut.close(); + } catch (Exception ignored) { + //discard + cacheOut.discard(); + } } if (remoteOut != null) { remoteOut.close(); 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 02e6cde678e8b..5ff8367bd68e6 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 @@ -21,8 +21,6 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; import org.apache.flink.runtime.blob.BlobCacheService; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; @@ -93,7 +91,6 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; -import scala.concurrent.duration.Duration; import java.io.IOException; import java.net.InetSocketAddress; @@ -391,22 +388,7 @@ public CompletableFuture submitTask( ResultPartitionConsumableNotifier resultPartitionConsumableNotifier = jobManagerConnection.getResultPartitionConsumableNotifier(); PartitionProducerStateChecker partitionStateChecker = jobManagerConnection.getPartitionStateChecker(); - long checkpointCacheLeaseTimeout; - try { - Configuration configuration = jobInformation.getJobConfiguration(); - String timeoutString = configuration.getString( - ConfigConstants.AKKA_WATCH_HEARTBEAT_INTERVAL, - ConfigConstants.DEFAULT_AKKA_ASK_TIMEOUT); - String delayString = configuration.getString( - ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, - timeoutString - ); - checkpointCacheLeaseTimeout = Duration.apply(delayString).toMillis() * 2; - } catch (Exception nfe) { - checkpointCacheLeaseTimeout = tdd.getCheckpointTimeout(); - } - - CheckpointCache checkpointCache = checkpointCacheManager.registerCheckpointCache(jobInformation.getJobId(), tdd.getCheckpointTimeout(), checkpointCacheLeaseTimeout); + CheckpointCache checkpointCache = checkpointCacheManager.registerCheckpointCache(jobInformation.getJobId(), tdd.getCheckpointTimeout(), tdd.getLeaseTimeout()); Task task = new Task( jobInformation, 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 e976dc6a139cb..d92f826e810b3 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 @@ -202,7 +202,7 @@ public static TaskManagerServices fromConfiguration( final CheckpointCacheManager checkpointCacheManager = new CheckpointCacheManager( new ScheduledThreadPoolExecutor(1), Executors.directExecutor(), - taskManagerServicesConfiguration.getTmpDirPaths()[0]); + taskManagerServicesConfiguration.getTmpDirPaths()); return new TaskManagerServices( taskManagerLocation, 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 a96f1e580ddcd..f993c2feda5f3 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 @@ -1184,27 +1184,10 @@ class TaskManager( config.getTimeout().getSize(), config.getTimeout().getUnit())) - var checkpointCacheLeaseTimeout: Long = { - try { - var configuration = jobInformation.getJobConfiguration() - var timeoutString = configuration.getString( - ConfigConstants.AKKA_WATCH_HEARTBEAT_INTERVAL, - ConfigConstants.DEFAULT_AKKA_ASK_TIMEOUT) - var delayString = configuration.getString( - ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, - timeoutString - ) - Duration.apply(delayString).toMillis * 2; - } catch { - case e: Exception => - tdd.getCheckpointTimeout - } - } - var checkpointCache = checkpointCacheManager.registerCheckpointCache( jobInformation.getJobId, tdd.getCheckpointTimeout, - checkpointCacheLeaseTimeout) + tdd.getLeaseTimeout) val task = new Task( jobInformation, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManagerTest.java index 99f1821341019..88bcb489ab068 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheManagerTest.java @@ -45,7 +45,7 @@ public class CheckpointCacheManagerTest { @Test public void testCheckpointCacheManager() throws Exception { - CheckpointCacheManager cacheManager = new CheckpointCacheManager(new ScheduledThreadPoolExecutor(1), Executors.directExecutor(), tmp.newFolder().getAbsolutePath()); + CheckpointCacheManager cacheManager = new CheckpointCacheManager(new ScheduledThreadPoolExecutor(1), Executors.directExecutor(), new String[] {tmp.newFolder().getAbsolutePath()}); JobID jobID1 = new JobID(1L, 1L); cacheManager.registerCheckpointCache(jobID1, 10000, 5); Assert.assertEquals(1, cacheManager.getCheckpointCacheSize()); @@ -57,7 +57,7 @@ public void testCheckpointCacheManager() throws Exception { @Test public void testCheckpointCacheRetouchFromRelease() throws Exception { - CheckpointCacheManager cacheManager = new CheckpointCacheManager(new ScheduledThreadPoolExecutor(1), Executors.directExecutor(), tmp.newFolder().getAbsolutePath()); + CheckpointCacheManager cacheManager = new CheckpointCacheManager(new ScheduledThreadPoolExecutor(1), Executors.directExecutor(), new String[] {tmp.newFolder().getAbsolutePath()}); JobID jobID = new JobID(1L, 1L); CheckpointCache cache1 = cacheManager.registerCheckpointCache(jobID, 10000, 5); @@ -79,7 +79,7 @@ public void testCheckpointCacheRetouchFromRelease() throws Exception { @Test public void testConcurrencyRequest() throws Exception { - CheckpointCacheManager cacheManager = new CheckpointCacheManager(new ScheduledThreadPoolExecutor(1), Executors.directExecutor(), tmp.newFolder().getAbsolutePath()); + CheckpointCacheManager cacheManager = new CheckpointCacheManager(new ScheduledThreadPoolExecutor(1), Executors.directExecutor(), new String[] {tmp.newFolder().getAbsolutePath()}); // init jobs int[] referenceCount = new int[10]; JobID[] jobIDS = new JobID[10]; @@ -130,7 +130,7 @@ public void testConcurrencyRequest() throws Exception { @Test public void testShowdown() throws Exception { - CheckpointCacheManager cacheManager = new CheckpointCacheManager(new ScheduledThreadPoolExecutor(1), Executors.directExecutor(), tmp.newFolder().getAbsolutePath()); + CheckpointCacheManager cacheManager = new CheckpointCacheManager(new ScheduledThreadPoolExecutor(1), Executors.directExecutor(), new String[] {tmp.newFolder().getAbsolutePath()}); JobID jobID1 = new JobID(1L, 1L); cacheManager.registerCheckpointCache(jobID1, 10000, 5); cacheManager.shutdown(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheTest.java index 0234f7ac113f7..51f6a4e2ce4e0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheTest.java @@ -22,6 +22,7 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.state.StateHandleID; +import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.filesystem.FileStateHandle; import org.junit.Assert; import org.junit.Rule; @@ -30,7 +31,9 @@ import java.io.File; import java.io.FileOutputStream; +import java.io.IOException; import java.io.OutputStream; +import java.nio.file.Files; import java.util.Map; import static org.mockito.Mockito.mock; @@ -53,8 +56,8 @@ public void testCommitCache() throws Exception { Assert.assertEquals(cache.getPendingCheckpointCacheSize(), 0); - cache.registerCacheEntry(1, handleID1, tmp.newFile().getAbsolutePath()); - cache.registerCacheEntry(1, handleID2, tmp.newFile().getAbsolutePath()); + cache.registerCacheEntry(1, handleID1, new FileStateHandle(new Path(tmp.newFile().getAbsolutePath()), 0L)); + cache.registerCacheEntry(1, handleID2, new FileStateHandle(new Path(tmp.newFile().getAbsolutePath()), 0L)); Assert.assertEquals(cache.getPendingCheckpointCacheSize(), 1); @@ -72,17 +75,18 @@ public void testOpenInputStream() throws Exception { final CheckpointCache cache = new CheckpointCache(new JobID(), tmpFolder.getAbsolutePath(), 10000, 10000, mock(CheckpointCacheManager.class), Executors.directExecutor()); StateHandleID handleID1 = new StateHandleID("handle1"); - cache.registerCacheEntry(1, handleID1, "handle1"); + cache.registerCacheEntry(1, handleID1, new FileStateHandle(new Path("handle1"), 0L)); File handle2File = tmp.newFile("handle2"); - final String testStr = "hello"; + final String testStr = "test str"; FileOutputStream outputStream = new FileOutputStream(handle2File); outputStream.write(testStr.getBytes(), 0, testStr.getBytes().length); outputStream.close(); + StateHandleID handleID2 = new StateHandleID(handle2File.getAbsolutePath()); - cache.registerCacheEntry(1, handleID2, handle2File.getAbsolutePath()); + cache.registerCacheEntry(1, handleID2, new FileStateHandle(new Path(handle2File.getAbsolutePath()), 0L)); cache.commitCache(1); @@ -105,9 +109,9 @@ public void testCreateOutputStream() throws Exception { StateHandleID handleID1 = new StateHandleID("handle1"); CheckpointCache.CachedOutputStream outputStream = cache.createOutputStream(1, handleID1); - final String testStr = "hello"; + final String testStr = "test str"; outputStream.write(testStr.getBytes(), 0, testStr.length()); - outputStream.end(); + outputStream.closeAndGetHandle(); cache.commitCache(1); @@ -130,7 +134,7 @@ public void testDiscardOutputStream() throws Exception { final String testStr = "test str"; outputStream.write(testStr.getBytes(), 0, testStr.length()); outputStream.discard(); - outputStream.end(); + outputStream.closeAndGetHandle(); Assert.assertEquals(cache.getPendingCheckpointCacheSize(), 0); cache.release(); @@ -145,7 +149,7 @@ public void testOnlyMaintainTheLastCheckpointCache() throws Exception { CheckpointCache.CachedOutputStream outputStream = cache.createOutputStream(1, handleID1); final String testStr = "test str"; outputStream.write(testStr.getBytes(), 0, testStr.length()); - outputStream.end(); + outputStream.closeAndGetHandle(); outputStream.close(); cache.commitCache(1); @@ -155,7 +159,7 @@ public void testOnlyMaintainTheLastCheckpointCache() throws Exception { CheckpointCache.CachedOutputStream outputStream2 = cache.createOutputStream(2, handleID2); final String testStr2 = "test str2"; outputStream2.write(testStr2.getBytes(), 0, testStr2.length()); - outputStream2.end(); + outputStream2.closeAndGetHandle(); outputStream2.close(); cache.commitCache(2); @@ -175,7 +179,7 @@ public void testClose() throws Exception { final CheckpointCache.CachedOutputStream output = cache.createOutputStream(i, handleID); String testStr = "123"; output.write(testStr.getBytes(), 0, testStr.getBytes().length); - output.end(); + output.closeAndGetHandle(); output.close(); } } @@ -203,7 +207,7 @@ public void testReCache() throws Exception { CachedStreamStateHandle[] cachedHandles = new CachedStreamStateHandle[5]; final String testStr = "test re-cache logic."; - String[] cacheFilePaths = new String[5]; + StreamStateHandle[] cacheFilePaths = new StreamStateHandle[5]; // checkpoint for (int i = 0; i < 5; ++i) { @@ -211,8 +215,7 @@ public void testReCache() throws Exception { final StateHandleID handleID = new StateHandleID("cache_" + i); final CheckpointCache.CachedOutputStream output = cache.createOutputStream(1, handleID); output.write(testStr.getBytes(), 0, testStr.getBytes().length); - output.end(); - cacheFilePaths[i] = output.getCacheFilePath(); + cacheFilePaths[i] = output.closeAndGetHandle(); output.close(); // remote output stream @@ -225,7 +228,7 @@ public void testReCache() throws Exception { // delete cache file for (int i = 0; i < 5; ++i) { - new File(cacheFilePaths[i]).delete(); + cacheFilePaths[i].discardState(); } // read from cached handle, this should read from remote @@ -265,7 +268,7 @@ public void testCacheRegister() throws Exception { File tmpFolder = tmp.newFolder(); final CheckpointCache cache = new CheckpointCache(new JobID(), tmpFolder.getAbsolutePath(), 10000, 10000, mock(CheckpointCacheManager.class), Executors.directExecutor()); final String testStr = "test re-cache logic."; - String[] cacheFilePaths = new String[5]; + StreamStateHandle[] cacheHandles = new StreamStateHandle[5]; // checkpoint for (int i = 0; i < 5; ++i) { @@ -273,8 +276,7 @@ public void testCacheRegister() throws Exception { final StateHandleID handleID = new StateHandleID("cache_" + i); final CheckpointCache.CachedOutputStream output = cache.createOutputStream(1, handleID); output.write(testStr.getBytes(), 0, testStr.getBytes().length); - output.end(); - cacheFilePaths[i] = output.getCacheFilePath(); + cacheHandles[i] = output.closeAndGetHandle(); output.close(); } cache.commitCache(1); @@ -285,13 +287,16 @@ public void testCacheRegister() throws Exception { final StateHandleID handleID = new StateHandleID("cache_" + i); final CheckpointCache.CachedOutputStream output = cache.createOutputStream(2, handleID); output.write(testStr.getBytes(), 0, testStr.getBytes().length); - output.end(); + output.closeAndGetHandle(); output.close(); } cache.commitCache(2); for (int i = 0; i < 4; ++i) { - Assert.assertTrue(!new File(cacheFilePaths[i]).exists()); + try { + cacheHandles[i].openInputStream(); + Assert.fail(); + } catch (IOException expected) {} } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java index 6a2d95c9fd762..4dbcf2b2eb13a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java @@ -87,6 +87,7 @@ public void testSerialization() { taskStateHandles, producedResults, inputGates, + -1, -1); final TaskDeploymentDescriptor copy = CommonTestUtils.createCopySerializable(orig); 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 f6d1740e4f7f7..3b830854a3796 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 @@ -701,6 +701,7 @@ public void testTaskSubmission() throws Exception { null, Collections.emptyList(), Collections.emptyList(), + -1, -1); final LibraryCacheManager libraryCacheManager = mock(LibraryCacheManager.class); @@ -1317,6 +1318,7 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { null, Collections.emptyList(), Collections.emptyList(), + -1, -1); CompletableFuture> offerResultFuture = new CompletableFuture<>(); 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 6e4927c38177d..a1dee69351554 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 @@ -157,7 +157,7 @@ public void testComponentsStartupShutdown() throws Exception { network.start(); - final CheckpointCacheManager checkpointCacheManager = new CheckpointCacheManager(java.util.concurrent.Executors.newSingleThreadScheduledExecutor(), Executors.directExecutor(), TMP_DIR[0]); + final CheckpointCacheManager checkpointCacheManager = new CheckpointCacheManager(java.util.concurrent.Executors.newSingleThreadScheduledExecutor(), Executors.directExecutor(), TMP_DIR); MetricRegistryConfiguration metricRegistryConfiguration = MetricRegistryConfiguration.fromConfiguration(config); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java index 9afbbb8d4e850..5a7634aa87535 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java @@ -2160,6 +2160,7 @@ private static TaskDeploymentDescriptor createTaskDeploymentDescriptor( null, producedPartitions, inputGates, + -1, -1); } From ca81152b9d08c6169ee39a321ffc467e4cd5ce3c Mon Sep 17 00:00:00 2001 From: summerleafs Date: Thu, 30 Nov 2017 01:31:15 +0800 Subject: [PATCH 6/7] fix build. --- .../apache/flink/runtime/checkpoint/CheckpointCache.java | 2 +- .../flink/runtime/checkpoint/CheckpointCacheTest.java | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java index 0a7403504e038..8d5ad607e9025 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java @@ -346,7 +346,7 @@ public FsCachedOutputStream( cacheBasePath, cacheBasePath.getFileSystem(), 4096, - 1024 * 1024); + 2048); } catch (Exception ignored) { this.outputStream = null; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheTest.java index 51f6a4e2ce4e0..990ea01dd6e29 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCacheTest.java @@ -33,7 +33,6 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; -import java.nio.file.Files; import java.util.Map; import static org.mockito.Mockito.mock; @@ -267,7 +266,11 @@ public void testCacheRegister() throws Exception { File tmpFolder = tmp.newFolder(); final CheckpointCache cache = new CheckpointCache(new JobID(), tmpFolder.getAbsolutePath(), 10000, 10000, mock(CheckpointCacheManager.class), Executors.directExecutor()); - final String testStr = "test re-cache logic."; + StringBuilder builder = new StringBuilder(4097); + for (int i = 0; i < 500; ++i) { + builder.append("1234567890"); + } + final String testStr = builder.toString(); StreamStateHandle[] cacheHandles = new StreamStateHandle[5]; // checkpoint From 73aeee9ebf5e53673074822fd52f57edbb599d8e Mon Sep 17 00:00:00 2001 From: summerleafs Date: Mon, 4 Dec 2017 23:13:46 +0800 Subject: [PATCH 7/7] fix bug for incremental checkpoint. --- .../flink/runtime/checkpoint/CheckpointCache.java | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java index 8d5ad607e9025..c12e1ee2c7940 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCache.java @@ -22,6 +22,7 @@ import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.runtime.state.PlaceholderStreamStateHandle; import org.apache.flink.runtime.state.StateHandleID; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory; @@ -152,6 +153,7 @@ protected void abortPendingCache(long checkpointID) { PendingCheckpointCache pendingCheckpointCache = pendingCacheMap.get(checkpointID); if (pendingCheckpointCache != null) { pendingCheckpointCache.abortSubsumed(); + pendingCacheMap.remove(checkpointID); } } } @@ -164,7 +166,7 @@ public void commitCache(long checkpointID, boolean dropUnRetainCheckpointCache) synchronized (lock) { final PendingCheckpointCache pendingCheckpointCache; pendingCheckpointCache = pendingCacheMap.remove(checkpointID); - if (pendingCheckpointCache != null) { + if (pendingCheckpointCache != null && !pendingCheckpointCache.isDiscarded()) { LOG.info("commit pending checkpoint cache: {}", checkpointID); // here will build reference on cache entry CompletedCheckpointCache completedCheckpointCache = new CompletedCheckpointCache(sharedCacheRegistry, checkpointID); @@ -417,8 +419,13 @@ public void close() throws IOException { @Override public StreamStateHandle closeAndGetHandle() throws IOException { - if (!isDiscarded() && outputStream != null) { - StreamStateHandle stateHandle = outputStream.closeAndGetHandle(); + if (!isDiscarded()) { + StreamStateHandle stateHandle; + if (outputStream != null) { + stateHandle = outputStream.closeAndGetHandle(); + } else { + stateHandle = new PlaceholderStreamStateHandle(cacheID) ; + } this.cache.registerCacheEntry(checkpointID, cacheID, stateHandle); return stateHandle; } else {