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

    https://github.com/apache/flink/pull/4368#discussion_r129527454
  
    --- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
 ---
    @@ -0,0 +1,342 @@
    +/*
    + * 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
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * 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.api.functions.sink;
    +
    +import org.apache.flink.annotation.PublicEvolving;
    +import org.apache.flink.api.common.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.typeinfo.TypeHint;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Objects;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * This is a recommended base class for all of the {@link SinkFunction} 
that intend to implement exactly-once semantic.
    + * It does that by implementing two phase commit algorithm on top of the 
{@link CheckpointedFunction} and
    + * {@link CheckpointListener}. User should provide custom TXN (transaction 
handle) and implement abstract methods
    + * handling this transaction handle.
    + *
    + * @param <IN> Input type for {@link SinkFunction}
    + * @param <TXN> Transaction to store all of the information required to 
handle a transaction (must be Serializable)
    + */
    +@PublicEvolving
    +public abstract class TwoPhaseCommitSinkFunction<IN, TXN extends 
Serializable>
    +           extends RichSinkFunction<IN>
    +           implements CheckpointedFunction, CheckpointListener {
    +
    +   private static final Logger LOG = 
LoggerFactory.getLogger(TwoPhaseCommitSinkFunction.class);
    +
    +   protected final ListStateDescriptor<TransactionAndCheckpoint<TXN>> 
pendingCommitTransactionsDescriptor;
    +   protected final ListStateDescriptor<TXN> pendingTransactionsDescriptor;
    +
    +   protected final List<TransactionAndCheckpoint<TXN>> 
pendingCommitTransactions = new ArrayList<>();
    +
    +   @Nullable
    +   protected TXN currentTransaction;
    +   protected ListState<TXN> pendingTransactionsState;
    +   protected ListState<TransactionAndCheckpoint<TXN>> 
pendingCommitTransactionsState;
    +
    +   public TwoPhaseCommitSinkFunction(Class<TXN> txnClass) {
    +           this(
    +                   TypeInformation.of(txnClass),
    +                   TypeInformation.of(new 
TypeHint<TransactionAndCheckpoint<TXN>>() {}));
    +   }
    +
    +   public TwoPhaseCommitSinkFunction(
    +                   TypeInformation<TXN> txnTypeInformation,
    +                   TypeInformation<TransactionAndCheckpoint<TXN>> 
txnAndCheckpointTypeInformation) {
    +           this(
    +                   new ListStateDescriptor<>("pendingTransactions", 
txnTypeInformation),
    +                   new ListStateDescriptor<>("pendingCommitTransactions", 
txnAndCheckpointTypeInformation));
    +   }
    +
    +   public TwoPhaseCommitSinkFunction(
    +                   ListStateDescriptor<TXN> pendingTransactionsDescriptor,
    +                   ListStateDescriptor<TransactionAndCheckpoint<TXN>> 
pendingCommitTransactionsDescriptor) {
    +           this.pendingTransactionsDescriptor = 
requireNonNull(pendingTransactionsDescriptor, "pendingTransactionsDescriptor is 
null");
    +           this.pendingCommitTransactionsDescriptor = 
requireNonNull(pendingCommitTransactionsDescriptor, 
"pendingCommitTransactionsDescriptor is null");
    +   }
    +
    +   // ------ methods that should be implemented in child class to support 
two phase commit algorithm ------
    +
    +   /**
    +    * Write value within a transaction.
    +    */
    +   protected abstract void invoke(TXN transaction, IN value) throws 
Exception;
    +
    +   /**
    +    * Method that starts a new transaction.
    +    *
    +    * @return newly created transaction.
    +    */
    +   protected abstract TXN beginTransaction() throws Exception;
    +
    +   /**
    +    * Pre commit previously created transaction. Pre commit must make all 
of the necessary steps to prepare the
    +    * transaction for a commit that might happen in the future. After this 
point the transaction might still be
    +    * aborted, but underlying implementation must ensure that commit calls 
on already pre committed transactions
    +    * will always succeed.
    +    *
    +    * <p>Usually implementation involves flushing the data.
    +    */
    +   protected abstract void preCommit(TXN transaction) throws Exception;
    +
    +   /**
    +    * Commit a pre-committed transaction. If this method fail, Flink 
application will be
    +    * restarted and {@link 
TwoPhaseCommitSinkFunction#recoverAndCommit(Serializable)} will be called again 
for the
    +    * same transaction.
    +    */
    +   protected abstract void commit(TXN transaction);
    +
    +   /**
    +    * Invoked on recovered transactions after a failure. Must eventually 
succeed. If it fails, Flink application will
    +    * be restarted and it will be invoked again. If it does not succeed it 
means a data loss will occur.
    --- End diff --
    
    It is a just warning, that if it doesn't succeed eventually, there will be 
a data loss. But since it is using a user code for recovering and committing a 
transaction, it is the user that must ensure that "commits will always 
eventually succeed,".
    
    I rephrased a little bit this java doc.


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