[ https://issues.apache.org/jira/browse/FLINK-4939?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15636586#comment-15636586 ]
ASF GitHub Bot commented on FLINK-4939: --------------------------------------- Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2707#discussion_r86563313 --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericWriteAheadSink.java --- @@ -77,111 +84,144 @@ public GenericWriteAheadSink(CheckpointCommitter committer, TypeSerializer<IN> s public void open() throws Exception { super.open(); committer.setOperatorId(id); - committer.setOperatorSubtaskId(getRuntimeContext().getIndexOfThisSubtask()); committer.open(); - cleanState(); - checkpointStreamFactory = - getContainingTask().createCheckpointStreamFactory(this); + + checkpointStreamFactory = getContainingTask() + .createCheckpointStreamFactory(this); + + cleanRestoredHandles(); } public void close() throws Exception { committer.close(); } /** - * Saves a handle in the state. + * Called when a checkpoint barrier arrives. It closes any open streams to the backend + * and marks them as pending for committing to the external, third-party storage system. * - * @param checkpointId - * @throws IOException + * @param checkpointId the id of the latest received checkpoint. + * @throws IOException in case something went wrong when handling the stream to the backend. */ private void saveHandleInState(final long checkpointId, final long timestamp) throws Exception { + Preconditions.checkNotNull(this.pendingCheckpoints, "The operator has not been properly initialized."); + //only add handle if a new OperatorState was created since the last snapshot if (out != null) { + int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask(); StreamStateHandle handle = out.closeAndGetHandle(); - if (state.pendingHandles.containsKey(checkpointId)) { + + PendingCheckpoint pendingCheckpoint = new PendingCheckpoint( + checkpointId, subtaskIdx, timestamp, handle); + + if (pendingCheckpoints.contains(pendingCheckpoint)) { //we already have a checkpoint stored for that ID that may have been partially written, //so we discard this "alternate version" and use the stored checkpoint handle.discardState(); } else { - state.pendingHandles.put(checkpointId, new Tuple2<>(timestamp, handle)); + pendingCheckpoints.add(pendingCheckpoint); } out = null; } } @Override - public void snapshotState(FSDataOutputStream out, - long checkpointId, - long timestamp) throws Exception { + public void snapshotState(FSDataOutputStream out, long checkpointId, long timestamp) throws Exception { saveHandleInState(checkpointId, timestamp); - InstantiationUtil.serializeObject(out, state); + DataOutputViewStreamWrapper outStream = new DataOutputViewStreamWrapper(out); + outStream.writeInt(pendingCheckpoints.size()); + for (PendingCheckpoint pendingCheckpoint : pendingCheckpoints) { + pendingCheckpoint.serialize(outStream); + } } @Override public void restoreState(FSDataInputStream in) throws Exception { - this.state = InstantiationUtil.deserializeObject(in, getUserCodeClassloader()); + final DataInputViewStreamWrapper inStream = new DataInputViewStreamWrapper(in); + int numPendingHandles = inStream.readInt(); + for (int i = 0; i < numPendingHandles; i++) { + pendingCheckpoints.add(PendingCheckpoint.restore(inStream, getUserCodeClassloader())); + } } - private void cleanState() throws Exception { - synchronized (this.state.pendingHandles) { //remove all handles that were already committed - Set<Long> pastCheckpointIds = this.state.pendingHandles.keySet(); - Set<Long> checkpointsToRemove = new HashSet<>(); - for (Long pastCheckpointId : pastCheckpointIds) { - if (committer.isCheckpointCommitted(pastCheckpointId)) { - checkpointsToRemove.add(pastCheckpointId); + /** + * Called at {@link #open()} to clean-up the pending handle list. + * It iterates over all restored pending handles, checks which ones are already + * committed to the outside storage system and removes them from the list. + */ + private void cleanRestoredHandles() throws Exception { + synchronized (pendingCheckpoints) { + + // for each of the pending handles... + Iterator<PendingCheckpoint> pendingCheckpointIt = pendingCheckpoints.iterator(); + while (pendingCheckpointIt.hasNext()) { + + PendingCheckpoint pendingCheckpoint = pendingCheckpointIt.next(); + long checkpointId = pendingCheckpoint.checkpointId; + int subtaskId = pendingCheckpoint.subtaskId; + + //...check if the temporary buffer is already committed and if yes, + // remove it from the list of pending checkpoints. + if (committer.isCheckpointCommitted(subtaskId, checkpointId)) { + pendingCheckpoint.stateHandle.discardState(); + pendingCheckpointIt.remove(); --- End diff -- ah, my bad :) > GenericWriteAheadSink: Decouple the creating from the committing subtask for > a pending checkpoint > ------------------------------------------------------------------------------------------------- > > Key: FLINK-4939 > URL: https://issues.apache.org/jira/browse/FLINK-4939 > Project: Flink > Issue Type: Improvement > Components: Cassandra Connector > Reporter: Kostas Kloudas > Assignee: Kostas Kloudas > Fix For: 1.2.0 > > > So far the GenericWriteAheadSink expected that > the subtask that wrote a pending checkpoint to the > state backend, will be also the one to commit it to > the third-party storage system. > This issue targets at removing this assumption. To do this > the CheckpointCommitter has to be able to dynamically > take the subtaskIdx as a parameter when asking > if a checkpoint was committed and also change the > state kept by the GenericWriteAheadSink to also > include that subtask index of the subtask that wrote > the pending checkpoint. > This change is also necessary for making the operator rescalable. -- This message was sent by Atlassian JIRA (v6.3.4#6332)