Skip to content

Commit

Permalink
[FLINK-17073] [checkpointing] Introduce CheckpointsCleaner responsibl…
Browse files Browse the repository at this point in the history
…e for asynchronous checkpoints cleaning via the ioExecutor. This class counts the number of checkpoints to clean and reports it to CheckpointRequestDecider. Add a test for too many checkpoints to clean.
  • Loading branch information
echauchot authored and pnowojski committed Oct 14, 2020
1 parent a91e3c7 commit 81c7fa6
Show file tree
Hide file tree
Showing 28 changed files with 468 additions and 120 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,8 @@ public class CheckpointCoordinator {
/** The executor used for asynchronous calls, like potentially blocking I/O. */
private final Executor executor;

private final CheckpointsCleaner checkpointsCleaner;

/** Tasks who need to be sent a message when a checkpoint is started. */
private final ExecutionVertex[] tasksToTrigger;

Expand Down Expand Up @@ -217,6 +219,7 @@ public CheckpointCoordinator(
CompletedCheckpointStore completedCheckpointStore,
StateBackend checkpointStateBackend,
Executor executor,
CheckpointsCleaner checkpointsCleaner,
ScheduledExecutor timer,
SharedStateRegistryFactory sharedStateRegistryFactory,
CheckpointFailureManager failureManager) {
Expand All @@ -232,6 +235,7 @@ public CheckpointCoordinator(
completedCheckpointStore,
checkpointStateBackend,
executor,
checkpointsCleaner,
timer,
sharedStateRegistryFactory,
failureManager,
Expand All @@ -250,6 +254,7 @@ public CheckpointCoordinator(
CompletedCheckpointStore completedCheckpointStore,
StateBackend checkpointStateBackend,
Executor executor,
CheckpointsCleaner checkpointsCleaner,
ScheduledExecutor timer,
SharedStateRegistryFactory sharedStateRegistryFactory,
CheckpointFailureManager failureManager,
Expand Down Expand Up @@ -283,6 +288,7 @@ public CheckpointCoordinator(
this.checkpointIdCounter = checkNotNull(checkpointIDCounter);
this.completedCheckpointStore = checkNotNull(completedCheckpointStore);
this.executor = checkNotNull(executor);
this.checkpointsCleaner = checkNotNull(checkpointsCleaner);
this.sharedStateRegistryFactory = checkNotNull(sharedStateRegistryFactory);
this.sharedStateRegistry = sharedStateRegistryFactory.create(executor);
this.isPreferCheckpointForRecovery = chkConfig.isPreferCheckpointForRecovery();
Expand Down Expand Up @@ -317,7 +323,8 @@ public CheckpointCoordinator(
this::rescheduleTrigger,
this.clock,
this.minPauseBetweenCheckpoints,
this.pendingCheckpoints::size);
this.pendingCheckpoints::size,
this.checkpointsCleaner::getNumberOfCheckpointsToClean);
}

// --------------------------------------------------------------------------------------------
Expand Down Expand Up @@ -669,7 +676,9 @@ private PendingCheckpoint createPendingCheckpoint(
props,
checkpointStorageLocation,
executor,
onCompletionPromise);
onCompletionPromise,
checkpointsCleaner,
this::scheduleTriggerRequest);

if (statsTracker != null) {
PendingCheckpointStats callback = statsTracker.reportPendingCheckpoint(
Expand Down Expand Up @@ -1062,7 +1071,7 @@ private void completePendingCheckpoint(PendingCheckpoint pendingCheckpoint) thro

try {
try {
completedCheckpoint = pendingCheckpoint.finalizeCheckpoint();
completedCheckpoint = pendingCheckpoint.finalizeCheckpoint(checkpointsCleaner::cleanCheckpoint);
failureManager.handleCheckpointSuccess(pendingCheckpoint.getCheckpointId());
}
catch (Exception e1) {
Expand Down Expand Up @@ -1102,7 +1111,7 @@ public void run() {
}
} finally {
pendingCheckpoints.remove(checkpointId);
timer.execute(this::executeQueuedRequest);
scheduleTriggerRequest();
}

rememberRecentCheckpointId(checkpointId);
Expand Down Expand Up @@ -1134,6 +1143,10 @@ public void run() {
sendAcknowledgeMessages(checkpointId, completedCheckpoint.getTimestamp());
}

void scheduleTriggerRequest() {
timer.execute(this::executeQueuedRequest);
}

private void sendAcknowledgeMessages(long checkpointId, long timestamp) {
// commit tasks
for (ExecutionVertex ev : tasksToCommitTo) {
Expand Down Expand Up @@ -1396,7 +1409,7 @@ public boolean restoreSavepoint(

// Load the savepoint as a checkpoint into the system
CompletedCheckpoint savepoint = Checkpoints.loadAndValidateCheckpoint(
job, tasks, checkpointLocation, userClassLoader, allowNonRestored);
job, tasks, checkpointLocation, userClassLoader, allowNonRestored, checkpointsCleaner::cleanCheckpoint, this::scheduleTriggerRequest);

completedCheckpointStore.addCheckpoint(savepoint);

Expand Down Expand Up @@ -1678,7 +1691,7 @@ private void abortPendingCheckpoint(
sendAbortedMessages(pendingCheckpoint.getCheckpointId(), pendingCheckpoint.getCheckpointTimestamp());
pendingCheckpoints.remove(pendingCheckpoint.getCheckpointId());
rememberRecentCheckpointId(pendingCheckpoint.getCheckpointId());
timer.execute(this::executeQueuedRequest);
scheduleTriggerRequest();
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@ class CheckpointRequestDecider {
private final Clock clock;
private final long minPauseBetweenCheckpoints;
private final Supplier<Integer> pendingCheckpointsSizeSupplier;
private final Supplier<Integer> numberOfCleaningCheckpointsSupplier;
private final NavigableSet<CheckpointTriggerRequest> queuedRequests = new TreeSet<>(checkpointTriggerRequestsComparator());
private final int maxQueuedRequests;

Expand All @@ -70,13 +71,15 @@ class CheckpointRequestDecider {
Consumer<Long> rescheduleTrigger,
Clock clock,
long minPauseBetweenCheckpoints,
Supplier<Integer> pendingCheckpointsSizeSupplier) {
Supplier<Integer> pendingCheckpointsSizeSupplier,
Supplier<Integer> numberOfCleaningCheckpointsSupplier) {
this(
maxConcurrentCheckpointAttempts,
rescheduleTrigger,
clock,
minPauseBetweenCheckpoints,
pendingCheckpointsSizeSupplier,
numberOfCleaningCheckpointsSupplier,
DEFAULT_MAX_QUEUED_REQUESTS
);
}
Expand All @@ -87,6 +90,7 @@ class CheckpointRequestDecider {
Clock clock,
long minPauseBetweenCheckpoints,
Supplier<Integer> pendingCheckpointsSizeSupplier,
Supplier<Integer> numberOfCleaningCheckpointsSupplier,
int maxQueuedRequests) {
Preconditions.checkArgument(maxConcurrentCheckpointAttempts > 0);
Preconditions.checkArgument(maxQueuedRequests > 0);
Expand All @@ -95,6 +99,7 @@ class CheckpointRequestDecider {
this.clock = clock;
this.minPauseBetweenCheckpoints = minPauseBetweenCheckpoints;
this.pendingCheckpointsSizeSupplier = pendingCheckpointsSizeSupplier;
this.numberOfCleaningCheckpointsSupplier = numberOfCleaningCheckpointsSupplier;
this.maxQueuedRequests = maxQueuedRequests;
}

Expand Down Expand Up @@ -134,10 +139,10 @@ Optional<CheckpointTriggerRequest> chooseQueuedRequestToExecute(boolean isTrigge
* @return request that should be executed
*/
private Optional<CheckpointTriggerRequest> chooseRequestToExecute(boolean isTriggering, long lastCompletionMs) {
if (isTriggering || queuedRequests.isEmpty()) {
if (isTriggering || queuedRequests.isEmpty()
|| numberOfCleaningCheckpointsSupplier.get() > maxConcurrentCheckpointAttempts) {
return Optional.empty();
}

if (pendingCheckpointsSizeSupplier.get() >= maxConcurrentCheckpointAttempts) {
return Optional.of(queuedRequests.first())
.filter(CheckpointTriggerRequest::isForce)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,9 @@ public static CompletedCheckpoint loadAndValidateCheckpoint(
Map<JobVertexID, ExecutionJobVertex> tasks,
CompletedCheckpointStorageLocation location,
ClassLoader classLoader,
boolean allowNonRestoredState) throws IOException {
boolean allowNonRestoredState,
CheckpointsCleaningRunner cleanCallback,
SerializableRunnable checkpointCleaningFinishedCallback) throws IOException {

checkNotNull(jobId, "jobId");
checkNotNull(tasks, "tasks");
Expand Down Expand Up @@ -196,7 +198,9 @@ public static CompletedCheckpoint loadAndValidateCheckpoint(
operatorStates,
checkpointMetadata.getMasterStates(),
props,
location);
location,
cleanCallback,
checkpointCleaningFinishedCallback);
}

private static void throwNonRestoredStateException(String checkpointPointer, OperatorID operatorId) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* 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.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.Serializable;
import java.util.concurrent.Executor;
import java.util.concurrent.atomic.AtomicInteger;

/**
* Delegate class responsible for checkpoints cleaning and counting the number of checkpoints yet
* to clean.
*/
public class CheckpointsCleaner implements Serializable {
private static final Logger LOG = LoggerFactory.getLogger(CheckpointsCleaner.class);

private final AtomicInteger numberOfCheckpointsToClean;

public CheckpointsCleaner() {
this.numberOfCheckpointsToClean = new AtomicInteger(0);
}

int getNumberOfCheckpointsToClean() {
return numberOfCheckpointsToClean.get();
}

public void cleanCheckpoint(Runnable cleanAction, Runnable postCleanAction, Executor executor) {
numberOfCheckpointsToClean.incrementAndGet();
executor.execute(() -> {
try {
cleanAction.run();
} finally {
numberOfCheckpointsToClean.decrementAndGet();
postCleanAction.run();
}
});
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
/*
* 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 java.io.Serializable;
import java.util.concurrent.Executor;

/**
* Interface that allows to run a checkpoint cleaning task and a post checkpoint cleaning task on an {@link Executor}.
*/
@FunctionalInterface
public interface CheckpointsCleaningRunner extends Serializable {
void accept(Runnable cleanAction, Runnable postCleanAction, Executor executor);
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.flink.runtime.state.StateUtil;
import org.apache.flink.runtime.state.StreamStateHandle;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.function.ThrowingConsumer;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -40,6 +41,7 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Executor;

import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
Expand Down Expand Up @@ -108,6 +110,11 @@ public class CompletedCheckpoint implements Serializable {
@Nullable
private transient volatile CompletedCheckpointStats.DiscardCallback discardCallback;

private final CheckpointsCleaningRunner cleanupCallback;

private final SerializableRunnable cleanupFinishedCallback;


// ------------------------------------------------------------------------

public CompletedCheckpoint(
Expand All @@ -118,7 +125,9 @@ public CompletedCheckpoint(
Map<OperatorID, OperatorState> operatorStates,
@Nullable Collection<MasterState> masterHookStates,
CheckpointProperties props,
CompletedCheckpointStorageLocation storageLocation) {
CompletedCheckpointStorageLocation storageLocation,
CheckpointsCleaningRunner cleanupCallback,
SerializableRunnable cleanupFinishedCallback) {

checkArgument(checkpointID >= 0);
checkArgument(timestamp >= 0);
Expand All @@ -140,6 +149,8 @@ public CompletedCheckpoint(
this.storageLocation = checkNotNull(storageLocation);
this.metadataHandle = storageLocation.getMetadataHandle();
this.externalPointer = storageLocation.getExternalPointer();
this.cleanupCallback = cleanupCallback;
this.cleanupFinishedCallback = cleanupFinishedCallback;
}

// ------------------------------------------------------------------------
Expand Down Expand Up @@ -210,6 +221,21 @@ public void registerSharedStatesAfterRestored(SharedStateRegistry sharedStateReg
// Discard and Dispose
// ------------------------------------------------------------------------

/**
* Asynchronously call a discard on the ioExecutor
* (FixedThreadPool of configurable size of default 4*CPU cores)
* and count the number of checkpoints that are waiting to clean.
*/
void asyncDiscardCheckpointAndCountCheckpoint(ThrowingConsumer<CompletedCheckpoint, Exception> discardCallback, Executor executor){
cleanupCallback.accept(() -> {
try {
discardCallback.accept(this);
} catch (Exception e) {
LOG.warn("Could not discard completed checkpoint {}.", checkpointID, e);
}
}, cleanupFinishedCallback, executor);
}

public void discardOnFailedStoring() throws Exception {
doDiscard();
}
Expand Down
Loading

0 comments on commit 81c7fa6

Please sign in to comment.