private void dropSubsumedCheckpoints(long timestamp) { Iterator<Map.Entry<Long, PendingCheckpoint>> entries = pendingCheckpoints.entrySet().iterator(); while (entries.hasNext()) { PendingCheckpoint p = entries.next().getValue(); if (p.getCheckpointTimestamp() < timestamp) { rememberRecentCheckpointId(p.getCheckpointId()); p.discard(userClassLoader); onCancelCheckpoint(p.getCheckpointId()); entries.remove(); } } }
/** * 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 stopCheckpointScheduler() { synchronized (lock) { triggerRequestQueued = false; periodicScheduling = false; if (currentPeriodicTrigger != null) { currentPeriodicTrigger.cancel(); currentPeriodicTrigger = null; } for (PendingCheckpoint p : pendingCheckpoints.values()) { p.discard(userClassLoader); } pendingCheckpoints.clear(); numUnsuccessfulCheckpointsTriggers = 0; } }
/** * Receives an AcknowledgeCheckpoint message and returns whether the message was associated with a * pending checkpoint. * * @param message Checkpoint ack from the task manager * @return Flag indicating whether the ack'd checkpoint was associated with a pending checkpoint. * @throws Exception If the checkpoint cannot be added to the completed checkpoint store. */ public boolean receiveAcknowledgeMessage(AcknowledgeCheckpoint message) throws Exception { if (shutdown || message == null) { return false; } if (!job.equals(message.getJob())) { LOG.error("Received AcknowledgeCheckpoint message for wrong job: {}", message); return false; } final long checkpointId = message.getCheckpointId(); CompletedCheckpoint completed = null; PendingCheckpoint checkpoint; // Flag indicating whether the ack message was for a known pending // checkpoint. boolean isPendingCheckpoint; synchronized (lock) { // we need to check inside the lock for being shutdown as well, otherwise we // get races and invalid error log messages if (shutdown) { return false; } checkpoint = pendingCheckpoints.get(checkpointId); if (checkpoint != null && !checkpoint.isDiscarded()) { isPendingCheckpoint = true; if (checkpoint.acknowledgeTask( message.getTaskExecutionId(), message.getState(), message.getStateSize(), null)) { // TODO: Give KV-state to the acknowledgeTask method if (checkpoint.isFullyAcknowledged()) { completed = checkpoint.toCompletedCheckpoint(); completedCheckpointStore.addCheckpoint(completed); LOG.info( "Completed checkpoint " + checkpointId + " (in " + completed.getDuration() + " ms)"); if (LOG.isDebugEnabled()) { StringBuilder builder = new StringBuilder(); for (Map.Entry<JobVertexID, TaskState> entry : completed.getTaskStates().entrySet()) { builder .append("JobVertexID: ") .append(entry.getKey()) .append(" {") .append(entry.getValue()) .append("}"); } LOG.debug(builder.toString()); } pendingCheckpoints.remove(checkpointId); rememberRecentCheckpointId(checkpointId); dropSubsumedCheckpoints(completed.getTimestamp()); onFullyAcknowledgedCheckpoint(completed); triggerQueuedRequests(); } } else { // checkpoint did not accept message LOG.error( "Received duplicate or invalid acknowledge message for checkpoint " + checkpointId + " , task " + message.getTaskExecutionId()); } } else if (checkpoint != null) { // this should not happen throw new IllegalStateException( "Received message for discarded but non-removed checkpoint " + checkpointId); } else { // message is for an unknown checkpoint, or comes too late (checkpoint disposed) if (recentPendingCheckpoints.contains(checkpointId)) { isPendingCheckpoint = true; LOG.warn("Received late message for now expired checkpoint attempt " + checkpointId); } else { isPendingCheckpoint = false; } } } // send the confirmation messages to the necessary targets. we do this here // to be outside the lock scope if (completed != null) { final long timestamp = completed.getTimestamp(); for (ExecutionVertex ev : tasksToCommitTo) { Execution ee = ev.getCurrentExecutionAttempt(); if (ee != null) { ExecutionAttemptID attemptId = ee.getAttemptId(); NotifyCheckpointComplete notifyMessage = new NotifyCheckpointComplete(job, attemptId, checkpointId, timestamp); ev.sendMessageToCurrentExecution(notifyMessage, ee.getAttemptId()); } } statsTracker.onCompletedCheckpoint(completed); } return isPendingCheckpoint; }
/** * Receives a {@link DeclineCheckpoint} message and returns whether the message was associated * with a pending checkpoint. * * @param message Checkpoint decline from the task manager * @return Flag indicating whether the declined checkpoint was associated with a pending * checkpoint. */ public boolean receiveDeclineMessage(DeclineCheckpoint message) throws Exception { if (shutdown || message == null) { return false; } if (!job.equals(message.getJob())) { LOG.error("Received DeclineCheckpoint message for wrong job: {}", message); return false; } final long checkpointId = message.getCheckpointId(); PendingCheckpoint checkpoint; // Flag indicating whether the ack message was for a known pending // checkpoint. boolean isPendingCheckpoint; synchronized (lock) { // we need to check inside the lock for being shutdown as well, otherwise we // get races and invalid error log messages if (shutdown) { return false; } checkpoint = pendingCheckpoints.get(checkpointId); if (checkpoint != null && !checkpoint.isDiscarded()) { isPendingCheckpoint = true; LOG.info( "Discarding checkpoint " + checkpointId + " because of checkpoint decline from task " + message.getTaskExecutionId()); pendingCheckpoints.remove(checkpointId); checkpoint.discard(userClassLoader); rememberRecentCheckpointId(checkpointId); boolean haveMoreRecentPending = false; Iterator<Map.Entry<Long, PendingCheckpoint>> entries = pendingCheckpoints.entrySet().iterator(); while (entries.hasNext()) { PendingCheckpoint p = entries.next().getValue(); if (!p.isDiscarded() && p.getCheckpointTimestamp() >= checkpoint.getCheckpointTimestamp()) { haveMoreRecentPending = true; break; } } if (!haveMoreRecentPending && !triggerRequestQueued) { LOG.info("Triggering new checkpoint because of discarded checkpoint " + checkpointId); triggerCheckpoint(System.currentTimeMillis()); } else if (!haveMoreRecentPending) { LOG.info( "Promoting queued checkpoint request because of discarded checkpoint " + checkpointId); triggerQueuedRequests(); } } else if (checkpoint != null) { // this should not happen throw new IllegalStateException( "Received message for discarded but non-removed checkpoint " + checkpointId); } else { // message is for an unknown checkpoint, or comes too late (checkpoint disposed) if (recentPendingCheckpoints.contains(checkpointId)) { isPendingCheckpoint = true; LOG.info( "Received another decline checkpoint message for now expired checkpoint attempt " + checkpointId); } else { isPendingCheckpoint = false; } } } return isPendingCheckpoint; }
/** * 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; } }