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

    https://github.com/apache/flink/pull/1771#discussion_r63678427
  
    --- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/GenericAtLeastOnceSink.java
 ---
    @@ -0,0 +1,192 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.streaming.runtime.operators;
    +
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.runtime.io.disk.InputViewIterator;
    +import org.apache.flink.runtime.state.AbstractStateBackend;
    +import org.apache.flink.runtime.state.StateHandle;
    +import 
org.apache.flink.runtime.util.ReusingMutableToRegularIteratorWrapper;
    +import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
    +import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.UUID;
    +
    +/**
    + * Generic Sink that emits its input elements into an arbitrary backend. 
This sink is integrated with the checkpointing
    + * mechanism and can provide exactly-once guarantees; depending on the 
storage backend and sink/committer implementation.
    + * <p/>
    + * Incoming records are stored within a {@link 
org.apache.flink.runtime.state.AbstractStateBackend}, and only committed if a
    + * checkpoint is completed.
    + *
    + * @param <IN> Type of the elements emitted by this sink
    + */
    +public abstract class GenericAtLeastOnceSink<IN> extends 
AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
    +   protected static final Logger LOG = 
LoggerFactory.getLogger(GenericAtLeastOnceSink.class);
    +   private final CheckpointCommitter committer;
    +   private transient AbstractStateBackend.CheckpointStateOutputView out;
    +   protected final TypeSerializer<IN> serializer;
    +   private final String id;
    +
    +   private ExactlyOnceState state = new ExactlyOnceState();
    +
    +   public GenericAtLeastOnceSink(CheckpointCommitter committer, 
TypeSerializer<IN> serializer, String jobID) throws Exception {
    +           this.committer = committer;
    +           this.serializer = serializer;
    +           this.id = UUID.randomUUID().toString();
    +           this.committer.setJobId(jobID);
    +           this.committer.createResource();
    +   }
    +
    +   @Override
    +   public void open() throws Exception {
    +           committer.setOperatorId(id);
    +           
committer.setOperatorSubtaskId(getRuntimeContext().getIndexOfThisSubtask());
    +           committer.open();
    +   }
    +
    +   public void close() throws Exception {
    +           committer.close();
    +   }
    +
    +   /**
    +    * Saves a handle in the state.
    +    *
    +    * @param checkpointId
    +    * @throws IOException
    +    */
    +   private void saveHandleInState(final long checkpointId, final long 
timestamp) throws Exception {
    +           //only add handle if a new OperatorState was created since the 
last snapshot
    +           if (out != null) {
    +                   StateHandle<DataInputView> handle = 
out.closeAndGetHandle();
    +                   if (state.pendingHandles.containsKey(checkpointId)) {
    +                           //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));
    +                   }
    +                   out = null;
    +           }
    +   }
    +
    +   @Override
    +   public StreamTaskState snapshotOperatorState(final long checkpointId, 
final long timestamp) throws Exception {
    +           StreamTaskState taskState = 
super.snapshotOperatorState(checkpointId, timestamp);
    +           saveHandleInState(checkpointId, timestamp);
    +           taskState.setFunctionState(state);
    +           return taskState;
    +   }
    +
    +   @Override
    +   public void restoreState(StreamTaskState state, long recoveryTimestamp) 
throws Exception {
    +           super.restoreState(state, recoveryTimestamp);
    +           this.state = (ExactlyOnceState) state.getFunctionState();
    +
    +           out = null;
    +   }
    +
    +   @Override
    +   public void notifyOfCompletedCheckpoint(long checkpointId) throws 
Exception {
    +           super.notifyOfCompletedCheckpoint(checkpointId);
    +
    +           synchronized (state.pendingHandles) {
    +                   Set<Long> pastCheckpointIds = 
state.pendingHandles.keySet();
    +                   Set<Long> checkpointsToRemove = new HashSet<>();
    +                   for (Long pastCheckpointId : pastCheckpointIds) {
    +                           if (pastCheckpointId <= checkpointId) {
    +                                   if 
(!committer.isCheckpointCommitted(pastCheckpointId)) {
    --- End diff --
    
    whether it triggers a lookup is up to the Committer implementation. You 
could always keep a local version, and only lookup the value when the local 
version is null.


---
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