Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2707#discussion_r86108509
  
    --- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericWriteAheadSink.java
 ---
    @@ -77,63 +86,96 @@ 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.
    +    * Closes any open streams to the backend and marks them as pending for
    +    * committing to the final output system, e.g. Cassandra.
         *
    -    * @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.pendingHandles, "The operator 
has not been properly initialized.");
    +
                //only add handle if a new OperatorState was created since the 
last snapshot
                if (out != null) {
                        StreamStateHandle handle = out.closeAndGetHandle();
    -                   if (state.pendingHandles.containsKey(checkpointId)) {
    +
    +                   PendingCheckpointId pendingCheckpoint = new 
PendingCheckpointId(
    +                           checkpointId, 
getRuntimeContext().getIndexOfThisSubtask());
    +
    +                   if (pendingHandles.containsKey(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));
    +                           this.pendingHandles.put(pendingCheckpoint, new 
PendingHandle(timestamp, handle));
                        }
                        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(pendingHandles.size());
    +           for (Map.Entry<PendingCheckpointId, PendingHandle> 
pendingCheckpoint : pendingHandles.entrySet()) {
    +                   pendingCheckpoint.getKey().serialize(outStream);
    +                   pendingCheckpoint.getValue().serialize(outStream);
    +           }
        }
     
        @Override
        public void restoreState(FSDataInputStream in) throws Exception {
    -           this.state = InstantiationUtil.deserializeObject(in, 
getUserCodeClassloader());
    +           final DataInputViewStreamWrapper inStream = new 
DataInputViewStreamWrapper(in);
    +           int noOfPendingHandles = inStream.readInt();
    --- End diff --
    
    noOfPendingHandlers -> numPendingHandles


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to