protected long getAndIncrementCheckpointId() { try { // this must happen outside the locked scope, because it communicates // with external services (in HA mode) and may block for a while. return checkpointIdCounter.getAndIncrement(); } catch (Throwable t) { int numUnsuccessful = ++numUnsuccessfulCheckpointsTriggers; LOG.warn( "Failed to trigger checkpoint (" + numUnsuccessful + " consecutive failed attempts so far)", t); return -1; } }
/** * Shuts down the checkpoint coordinator. * * <p>After this method has been called, the coordinator does not accept and further messages and * cannot trigger any further checkpoints. */ public void shutdown() throws Exception { synchronized (lock) { try { if (!shutdown) { shutdown = true; LOG.info("Stopping checkpoint coordinator for job " + job); periodicScheduling = false; triggerRequestQueued = false; // shut down the thread that handles the timeouts and pending triggers timer.cancel(); // make sure that the actor does not linger if (jobStatusListener != null) { jobStatusListener.tell(PoisonPill.getInstance()); jobStatusListener = null; } checkpointIdCounter.stop(); // clear and discard all pending checkpoints for (PendingCheckpoint pending : pendingCheckpoints.values()) { pending.discard(userClassLoader); } pendingCheckpoints.clear(); // clean and discard all successful checkpoints completedCheckpointStore.discardAllCheckpoints(); onShutdown(); } } finally { // Remove shutdown hook to prevent resource leaks, unless this is invoked by the // shutdown hook itself. if (shutdownHook != null && shutdownHook != Thread.currentThread()) { try { Runtime.getRuntime().removeShutdownHook(shutdownHook); } catch (IllegalStateException ignored) { // race, JVM is in shutdown already, we can safely ignore this } catch (Throwable t) { LOG.warn("Error unregistering checkpoint coordinator shutdown hook.", t); } } } } }
public void startCheckpointScheduler() { synchronized (lock) { if (shutdown) { throw new IllegalArgumentException("Checkpoint coordinator is shut down"); } // make sure all prior timers are cancelled stopCheckpointScheduler(); try { // Multiple start calls are OK checkpointIdCounter.start(); } catch (Exception e) { String msg = "Failed to start checkpoint ID counter: " + e.getMessage(); throw new RuntimeException(msg, e); } periodicScheduling = true; currentPeriodicTrigger = new ScheduledTrigger(); timer.scheduleAtFixedRate(currentPeriodicTrigger, baseInterval, baseInterval); } }
/** * Triggers a new checkpoint and uses the given timestamp as the checkpoint timestamp. * * @param timestamp The timestamp for the checkpoint. * @param nextCheckpointId The checkpoint ID to use for this checkpoint or <code>-1</code> if the * checkpoint ID counter should be queried. */ public boolean triggerCheckpoint(long timestamp, long nextCheckpointId) throws Exception { // make some eager pre-checks synchronized (lock) { // abort if the coordinator has been shutdown in the meantime if (shutdown) { return false; } // sanity check: there should never be more than one trigger request queued if (triggerRequestQueued) { LOG.warn("Trying to trigger another checkpoint while one was queued already"); return false; } // if too many checkpoints are currently in progress, we need to mark that a request is queued if (pendingCheckpoints.size() >= maxConcurrentCheckpointAttempts) { triggerRequestQueued = true; if (currentPeriodicTrigger != null) { currentPeriodicTrigger.cancel(); currentPeriodicTrigger = null; } return false; } // make sure the minimum interval between checkpoints has passed if (lastTriggeredCheckpoint + minPauseBetweenCheckpoints > timestamp) { if (currentPeriodicTrigger != null) { currentPeriodicTrigger.cancel(); currentPeriodicTrigger = null; } ScheduledTrigger trigger = new ScheduledTrigger(); timer.scheduleAtFixedRate(trigger, minPauseBetweenCheckpoints, baseInterval); return false; } } // first check if all tasks that we need to trigger are running. // if not, abort the checkpoint ExecutionAttemptID[] triggerIDs = new ExecutionAttemptID[tasksToTrigger.length]; for (int i = 0; i < tasksToTrigger.length; i++) { Execution ee = tasksToTrigger[i].getCurrentExecutionAttempt(); if (ee != null && ee.getState() == ExecutionState.RUNNING) { triggerIDs[i] = ee.getAttemptId(); } else { LOG.info( "Checkpoint triggering task {} is not being executed at the moment. Aborting checkpoint.", tasksToTrigger[i].getSimpleName()); return false; } } // next, check if all tasks that need to acknowledge the checkpoint are running. // if not, abort the checkpoint Map<ExecutionAttemptID, ExecutionVertex> ackTasks = new HashMap<>(tasksToWaitFor.length); for (ExecutionVertex ev : tasksToWaitFor) { Execution ee = ev.getCurrentExecutionAttempt(); if (ee != null) { ackTasks.put(ee.getAttemptId(), ev); } else { LOG.info( "Checkpoint acknowledging task {} is not being executed at the moment. Aborting checkpoint.", ev.getSimpleName()); return false; } } // we will actually trigger this checkpoint! lastTriggeredCheckpoint = timestamp; final long checkpointID; if (nextCheckpointId < 0) { try { // this must happen outside the locked scope, because it communicates // with external services (in HA mode) and may block for a while. checkpointID = checkpointIdCounter.getAndIncrement(); } catch (Throwable t) { int numUnsuccessful = ++numUnsuccessfulCheckpointsTriggers; LOG.warn( "Failed to trigger checkpoint (" + numUnsuccessful + " consecutive failed attempts so far)", t); return false; } } else { checkpointID = nextCheckpointId; } LOG.info("Triggering checkpoint " + checkpointID + " @ " + timestamp); final PendingCheckpoint checkpoint = new PendingCheckpoint(job, checkpointID, timestamp, ackTasks); // schedule the timer that will clean up the expired checkpoints TimerTask canceller = new TimerTask() { @Override public void run() { try { synchronized (lock) { // only do the work if the checkpoint is not discarded anyways // note that checkpoint completion discards the pending checkpoint object if (!checkpoint.isDiscarded()) { LOG.info("Checkpoint " + checkpointID + " expired before completing."); checkpoint.discard(userClassLoader); pendingCheckpoints.remove(checkpointID); rememberRecentCheckpointId(checkpointID); onCancelCheckpoint(checkpointID); triggerQueuedRequests(); } } } catch (Throwable t) { LOG.error("Exception while handling checkpoint timeout", t); } } }; try { // re-acquire the lock synchronized (lock) { // since we released the lock in the meantime, we need to re-check // that the conditions still hold. this is clumsy, but it allows us to // release the lock in the meantime while calls to external services are // blocking progress, and still gives us early checks that skip work // if no checkpoint can happen anyways if (shutdown) { return false; } else if (triggerRequestQueued) { LOG.warn("Trying to trigger another checkpoint while one was queued already"); return false; } else if (pendingCheckpoints.size() >= maxConcurrentCheckpointAttempts) { triggerRequestQueued = true; if (currentPeriodicTrigger != null) { currentPeriodicTrigger.cancel(); currentPeriodicTrigger = null; } return false; } pendingCheckpoints.put(checkpointID, checkpoint); timer.schedule(canceller, checkpointTimeout); } // end of lock scope // send the messages to the tasks that trigger their checkpoint for (int i = 0; i < tasksToTrigger.length; i++) { ExecutionAttemptID id = triggerIDs[i]; TriggerCheckpoint message = new TriggerCheckpoint(job, id, checkpointID, timestamp); tasksToTrigger[i].sendMessageToCurrentExecution(message, id); } numUnsuccessfulCheckpointsTriggers = 0; return true; } catch (Throwable t) { // guard the map against concurrent modifications synchronized (lock) { pendingCheckpoints.remove(checkpointID); } int numUnsuccessful = ++numUnsuccessfulCheckpointsTriggers; LOG.warn( "Failed to trigger checkpoint (" + numUnsuccessful + " consecutive failed attempts so far)", t); if (!checkpoint.isDiscarded()) { checkpoint.discard(userClassLoader); } return false; } }
public CheckpointCoordinator( JobID job, long baseInterval, long checkpointTimeout, long minPauseBetweenCheckpoints, int maxConcurrentCheckpointAttempts, ExecutionVertex[] tasksToTrigger, ExecutionVertex[] tasksToWaitFor, ExecutionVertex[] tasksToCommitTo, ClassLoader userClassLoader, CheckpointIDCounter checkpointIDCounter, CompletedCheckpointStore completedCheckpointStore, RecoveryMode recoveryMode, CheckpointStatsTracker statsTracker) throws Exception { // Sanity check checkArgument(baseInterval > 0, "Checkpoint timeout must be larger than zero"); checkArgument(checkpointTimeout >= 1, "Checkpoint timeout must be larger than zero"); checkArgument(minPauseBetweenCheckpoints >= 0, "minPauseBetweenCheckpoints must be >= 0"); checkArgument( maxConcurrentCheckpointAttempts >= 1, "maxConcurrentCheckpointAttempts must be >= 1"); this.job = checkNotNull(job); this.baseInterval = baseInterval; this.checkpointTimeout = checkpointTimeout; this.minPauseBetweenCheckpoints = minPauseBetweenCheckpoints; this.maxConcurrentCheckpointAttempts = maxConcurrentCheckpointAttempts; this.tasksToTrigger = checkNotNull(tasksToTrigger); this.tasksToWaitFor = checkNotNull(tasksToWaitFor); this.tasksToCommitTo = checkNotNull(tasksToCommitTo); this.pendingCheckpoints = new LinkedHashMap<Long, PendingCheckpoint>(); this.completedCheckpointStore = checkNotNull(completedCheckpointStore); this.recentPendingCheckpoints = new ArrayDeque<Long>(NUM_GHOST_CHECKPOINT_IDS); this.userClassLoader = userClassLoader; this.checkpointIdCounter = checkNotNull(checkpointIDCounter); checkpointIDCounter.start(); this.timer = new Timer("Checkpoint Timer", true); this.statsTracker = checkNotNull(statsTracker); if (recoveryMode == RecoveryMode.STANDALONE) { // Add shutdown hook to clean up state handles when no checkpoint recovery is // possible. In case of another configured recovery mode, the checkpoints need to be // available for the standby job managers. this.shutdownHook = new Thread( new Runnable() { @Override public void run() { try { CheckpointCoordinator.this.shutdown(); } catch (Throwable t) { LOG.error( "Error during shutdown of checkpoint coordinator via " + "JVM shutdown hook: " + t.getMessage(), t); } } }); try { // Add JVM shutdown hook to call shutdown of service Runtime.getRuntime().addShutdownHook(shutdownHook); } catch (IllegalStateException ignored) { // JVM is already shutting down. No need to do anything. } catch (Throwable t) { LOG.error("Cannot register checkpoint coordinator shutdown hook.", t); } } else { this.shutdownHook = null; } }