You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by pnowojski <gi...@git.apache.org> on 2017/07/19 14:38:46 UTC

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

GitHub user pnowojski opened a pull request:

    https://github.com/apache/flink/pull/4368

    [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

    This is intended to be a recommended base class for implementing exactly-once sinks in Flink

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/pnowojski/flink 2phase

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/4368.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4368
    
----
commit 6ea314a4abd9f609accbc9c5f450051560df43da
Author: Piotr Nowojski <pi...@gmail.com>
Date:   2017-07-04T15:45:53Z

    [FLINK-7210] Implement TwoPhaseCommitSinkFunction
    
    This is a recommended base class for implementing exactly-once sinks in Flink

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129531190
  
    --- 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.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<TransactionAndCheckpoint<TXN>> pendingTransactionsIterator = pendingCommitTransactions.iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		List<TransactionAndCheckpoint<TXN>> remainingTransactions = new ArrayList<>();
    --- End diff --
    
    Making copy of `ArrayList` is much faster then removing it elements one by one.
    
    Although I have reimplemented `pendingCommitTransactions` to use `LinkedList` instead.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by pnowojski <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r131091039
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -0,0 +1,311 @@
    +/*
    + * 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.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.util.Iterator;
    +import java.util.LinkedHashMap;
    +import java.util.Map;
    +
    +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
    --- End diff --
    
    nit: wrap TXN as `{@code TXN}`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129340541
  
    --- 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.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<TransactionAndCheckpoint<TXN>> pendingTransactionsIterator = pendingCommitTransactions.iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		List<TransactionAndCheckpoint<TXN>> remainingTransactions = new ArrayList<>();
    +
    +		for (TransactionAndCheckpoint<TXN> pendingTransaction : pendingCommitTransactions) {
    +			if (pendingTransaction.checkpointId > checkpointId) {
    +				remainingTransactions.add(pendingTransaction);
    +				continue;
    +			}
    +
    +			LOG.info("{} - checkpoint {} complete, committing completed checkpoint transaction {}",
    +				name(), checkpointId, pendingTransaction);
    +
    +			// If this fails, there is actually a data loss
    +			commit(pendingTransaction.transaction);
    +
    +			LOG.debug("{} - committed checkpoint transaction {}", name(), pendingTransaction);
    +		}
    +
    +		pendingCommitTransactions.clear();
    +		for (TransactionAndCheckpoint<TXN> remainingTransaction : remainingTransactions) {
    +			pendingCommitTransactions.add(remainingTransaction);
    +		}
    +	}
    +
    +	@Override
    +	public final void snapshotState(FunctionSnapshotContext context) throws Exception {
    +		// this is like the pre-commit of a 2-phase-commit transaction
    +		// we are ready to commit and remember the transaction
    +
    +		checkState(currentTransaction != null, "bug: no transaction object when performing state snapshot");
    +
    +		long checkpointId = context.getCheckpointId();
    +		LOG.debug("{} - checkpoint {} triggered, flushing transaction '{}'", name(), context.getCheckpointId(), currentTransaction);
    +
    +		preCommit(currentTransaction);
    +		pendingCommitTransactions.add(new TransactionAndCheckpoint<>(currentTransaction, checkpointId));
    +		LOG.debug("{} - stored pending transactions {}", name(), pendingCommitTransactions);
    +
    +		currentTransaction = beginTransaction();
    +		LOG.debug("{} - started new transaction '{}'", name(), currentTransaction);
    +
    +		pendingCommitTransactionsState.clear();
    +		for (TransactionAndCheckpoint<TXN> pendingCommitTransaction : pendingCommitTransactions) {
    +			pendingCommitTransactionsState.add(pendingCommitTransaction);
    +		}
    +
    +		pendingTransactionsState.clear();
    +		// in case of failure we might not be able to abort currentTransaction. Let's store it into the state
    +		// so it can be aborted after a restart/crash
    +		pendingTransactionsState.add(currentTransaction);
    +	}
    +
    +	@Override
    +	public final void initializeState(FunctionInitializationContext context) throws Exception {
    +		// when we are restoring state with pendingCommitTransactions, we don't really know whether the
    +		// transactions were already committed, or whether there was a failure between
    +		// completing the checkpoint on the master, and notifying the writer here.
    +
    +		// (the common case is actually that is was already committed, the window
    +		// between the commit on the master and the notification here is very small)
    +
    +		// it is possible to not have any transactions at all if there was a failure before
    +		// the first completed checkpoint, or in case of a scale-out event, where some of the
    +		// new task do not have and transactions assigned to check)
    +
    +		// we can have more than one transaction to check in case of a scale-in event, or
    +		// for the reasons discussed in the 'notifyCheckpointComplete()' method.
    +
    +		pendingTransactionsState = context.getOperatorStateStore().getListState(pendingTransactionsDescriptor);
    +		pendingCommitTransactionsState = context.getOperatorStateStore().getListState(pendingCommitTransactionsDescriptor);
    +
    +		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
    +		if (context.isRestored()) {
    +			LOG.info("{} - restoring state (subtaskIndex={}).", name(), subtaskIndex);
    +
    +			for (TransactionAndCheckpoint<TXN> recoveredTransaction : pendingCommitTransactionsState.get()) {
    +				recoverAndCommit(recoveredTransaction.transaction);
    +				LOG.debug("{} committed recovered transaction {} (subtaskIndex={})", name(), recoveredTransaction, subtaskIndex);
    +			}
    +
    +			// Explicitly abort transactions that could be not closed cleanly
    +			for (TXN pendingTransaction : pendingTransactionsState.get()) {
    +				recoverAndAbort(pendingTransaction);
    +				LOG.debug("{} aborted recovered transaction {} (subtaskIndex={})", name(), pendingTransaction, subtaskIndex);
    +			}
    +		} else {
    +			LOG.info("{} - no state to restore {} (subtaskIndex={}).", name(), subtaskIndex);
    +		}
    +		this.pendingCommitTransactions.clear();
    +	}
    +
    +	@Override
    +	public void open(Configuration parameters) throws Exception {
    +		super.open(parameters);
    +		currentTransaction = beginTransaction();
    +		LOG.debug("{} - started new transaction '{}'", name(), currentTransaction);
    +	}
    +
    +	@Override
    +	public void close() throws Exception {
    +		super.close();
    +
    +		if (currentTransaction != null) {
    +			abort(currentTransaction);
    +			currentTransaction = null;
    +		}
    +	}
    +
    +	private String name() {
    +		return this.getClass().getSimpleName();
    +	}
    +
    +	/**
    +	 * Mapping between transaction and a checkpoint.
    +	 */
    +	protected static class TransactionAndCheckpoint<TXN> implements Serializable {
    --- End diff --
    
    Same comment as above. Why is it `Serializable`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129235367
  
    --- 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.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<TransactionAndCheckpoint<TXN>> pendingTransactionsIterator = pendingCommitTransactions.iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		List<TransactionAndCheckpoint<TXN>> remainingTransactions = new ArrayList<>();
    +
    +		for (TransactionAndCheckpoint<TXN> pendingTransaction : pendingCommitTransactions) {
    +			if (pendingTransaction.checkpointId > checkpointId) {
    +				remainingTransactions.add(pendingTransaction);
    +				continue;
    +			}
    +
    +			LOG.info("{} - checkpoint {} complete, committing completed checkpoint transaction {}",
    +				name(), checkpointId, pendingTransaction);
    +
    +			// If this fails, there is actually a data loss
    +			commit(pendingTransaction.transaction);
    +
    +			LOG.debug("{} - committed checkpoint transaction {}", name(), pendingTransaction);
    +		}
    +
    +		pendingCommitTransactions.clear();
    +		for (TransactionAndCheckpoint<TXN> remainingTransaction : remainingTransactions) {
    +			pendingCommitTransactions.add(remainingTransaction);
    +		}
    +	}
    +
    +	@Override
    +	public final void snapshotState(FunctionSnapshotContext context) throws Exception {
    +		// this is like the pre-commit of a 2-phase-commit transaction
    +		// we are ready to commit and remember the transaction
    +
    +		checkState(currentTransaction != null, "bug: no transaction object when performing state snapshot");
    +
    +		long checkpointId = context.getCheckpointId();
    +		LOG.debug("{} - checkpoint {} triggered, flushing transaction '{}'", name(), context.getCheckpointId(), currentTransaction);
    +
    +		preCommit(currentTransaction);
    +		pendingCommitTransactions.add(new TransactionAndCheckpoint<>(currentTransaction, checkpointId));
    --- End diff --
    
    Would it actually be better if we use a `LinkedHashMap` to bookkeep this? That would neglect the need for these extra `TransactionAndCheckpoint` object instances.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r131093086
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -0,0 +1,311 @@
    +/*
    + * 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.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.util.Iterator;
    +import java.util.LinkedHashMap;
    +import java.util.Map;
    +
    +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 RichSinkFunction<IN>
    +		implements CheckpointedFunction, CheckpointListener {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(TwoPhaseCommitSinkFunction.class);
    +
    +	protected final ListStateDescriptor<Map<Long, TXN>> pendingCommitTransactionsDescriptor;
    +	protected final ListStateDescriptor<TXN> pendingTransactionsDescriptor;
    +
    +	protected final LinkedHashMap<Long, TXN> pendingCommitTransactions = new LinkedHashMap<>();
    +
    +	@Nullable
    +	protected TXN currentTransaction;
    +	protected ListState<TXN> pendingTransactionsState;
    +	protected ListState<Map<Long, TXN>> pendingCommitTransactionsState;
    +
    +	/**
    +	 * Use default {@link ListStateDescriptor} for internal state serialization. Helpful utilities for using this
    +	 * constructor are {@link TypeInformation#of(Class)}, {@link org.apache.flink.api.common.typeinfo.TypeHint} and
    +	 * {@link TypeInformation#of(TypeHint)}. Example:
    +	 * <pre>
    +	 * {@code
    +	 * TwoPhaseCommitSinkFunction(
    +	 *     TypeInformation.of(TXN.class),
    +	 *     TypeInformation.of(new TypeHint<TransactionAndCheckpoint<TXN>>() {}));
    +	 * }
    +	 * </pre>
    +	 * @param txnTypeInformation {@link TypeInformation} for transaction POJO.
    +	 * @param checkpointToTxnTypeInformation {@link TypeInformation} for mapping between checkpointId and transaction.
    +	 */
    +	public TwoPhaseCommitSinkFunction(
    +			TypeInformation<TXN> txnTypeInformation,
    +			TypeInformation<Map<Long, TXN>> checkpointToTxnTypeInformation) {
    +		this(new ListStateDescriptor<>("pendingTransactions", txnTypeInformation),
    +			new ListStateDescriptor<>("pendingCommitTransactions", checkpointToTxnTypeInformation));
    +	}
    +
    +	/**
    +	 * Instantiate {@link TwoPhaseCommitSinkFunction} with custom state descriptors.
    +	 *
    +	 * @param pendingTransactionsDescriptor descriptor for transaction POJO.
    +	 * @param pendingCommitTransactionsDescriptor descriptor for mapping between checkpointId and transaction POJO.
    +	 */
    +	public TwoPhaseCommitSinkFunction(
    +			ListStateDescriptor<TXN> pendingTransactionsDescriptor,
    +			ListStateDescriptor<Map<Long, 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(Object)} will be called again for the
    +	 * same transaction.
    +	 */
    +	protected abstract void commit(TXN transaction);
    +
    +	/**
    +	 * Invoked on recovered transactions after a failure. User implementation must ensure that this call will eventually
    +	 * succeed. If it fails, Flink application will be restarted and it will be invoked again. If it does not succeed
    +	 * a data loss will occur.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one. As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<Map.Entry<Long, TXN>> pendingTransactionsIterator = pendingCommitTransactions.entrySet().iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    --- End diff --
    
    I think Flink has its own `Preconditions.checkState(...)` for this, so you shouldn't need to implement your own `checkState`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129340400
  
    --- 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.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<TransactionAndCheckpoint<TXN>> pendingTransactionsIterator = pendingCommitTransactions.iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		List<TransactionAndCheckpoint<TXN>> remainingTransactions = new ArrayList<>();
    +
    +		for (TransactionAndCheckpoint<TXN> pendingTransaction : pendingCommitTransactions) {
    +			if (pendingTransaction.checkpointId > checkpointId) {
    +				remainingTransactions.add(pendingTransaction);
    +				continue;
    +			}
    +
    +			LOG.info("{} - checkpoint {} complete, committing completed checkpoint transaction {}",
    +				name(), checkpointId, pendingTransaction);
    +
    +			// If this fails, there is actually a data loss
    +			commit(pendingTransaction.transaction);
    +
    +			LOG.debug("{} - committed checkpoint transaction {}", name(), pendingTransaction);
    +		}
    +
    +		pendingCommitTransactions.clear();
    +		for (TransactionAndCheckpoint<TXN> remainingTransaction : remainingTransactions) {
    +			pendingCommitTransactions.add(remainingTransaction);
    +		}
    +	}
    +
    +	@Override
    +	public final void snapshotState(FunctionSnapshotContext context) throws Exception {
    +		// this is like the pre-commit of a 2-phase-commit transaction
    +		// we are ready to commit and remember the transaction
    +
    +		checkState(currentTransaction != null, "bug: no transaction object when performing state snapshot");
    +
    +		long checkpointId = context.getCheckpointId();
    +		LOG.debug("{} - checkpoint {} triggered, flushing transaction '{}'", name(), context.getCheckpointId(), currentTransaction);
    +
    +		preCommit(currentTransaction);
    +		pendingCommitTransactions.add(new TransactionAndCheckpoint<>(currentTransaction, checkpointId));
    +		LOG.debug("{} - stored pending transactions {}", name(), pendingCommitTransactions);
    +
    +		currentTransaction = beginTransaction();
    +		LOG.debug("{} - started new transaction '{}'", name(), currentTransaction);
    +
    +		pendingCommitTransactionsState.clear();
    +		for (TransactionAndCheckpoint<TXN> pendingCommitTransaction : pendingCommitTransactions) {
    +			pendingCommitTransactionsState.add(pendingCommitTransaction);
    +		}
    +
    +		pendingTransactionsState.clear();
    +		// in case of failure we might not be able to abort currentTransaction. Let's store it into the state
    +		// so it can be aborted after a restart/crash
    +		pendingTransactionsState.add(currentTransaction);
    +	}
    +
    +	@Override
    +	public final void initializeState(FunctionInitializationContext context) throws Exception {
    +		// when we are restoring state with pendingCommitTransactions, we don't really know whether the
    +		// transactions were already committed, or whether there was a failure between
    +		// completing the checkpoint on the master, and notifying the writer here.
    +
    +		// (the common case is actually that is was already committed, the window
    +		// between the commit on the master and the notification here is very small)
    +
    +		// it is possible to not have any transactions at all if there was a failure before
    +		// the first completed checkpoint, or in case of a scale-out event, where some of the
    +		// new task do not have and transactions assigned to check)
    +
    +		// we can have more than one transaction to check in case of a scale-in event, or
    +		// for the reasons discussed in the 'notifyCheckpointComplete()' method.
    +
    +		pendingTransactionsState = context.getOperatorStateStore().getListState(pendingTransactionsDescriptor);
    +		pendingCommitTransactionsState = context.getOperatorStateStore().getListState(pendingCommitTransactionsDescriptor);
    +
    +		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
    +		if (context.isRestored()) {
    +			LOG.info("{} - restoring state (subtaskIndex={}).", name(), subtaskIndex);
    +
    +			for (TransactionAndCheckpoint<TXN> recoveredTransaction : pendingCommitTransactionsState.get()) {
    +				recoverAndCommit(recoveredTransaction.transaction);
    +				LOG.debug("{} committed recovered transaction {} (subtaskIndex={})", name(), recoveredTransaction, subtaskIndex);
    +			}
    +
    +			// Explicitly abort transactions that could be not closed cleanly
    +			for (TXN pendingTransaction : pendingTransactionsState.get()) {
    +				recoverAndAbort(pendingTransaction);
    +				LOG.debug("{} aborted recovered transaction {} (subtaskIndex={})", name(), pendingTransaction, subtaskIndex);
    +			}
    +		} else {
    +			LOG.info("{} - no state to restore {} (subtaskIndex={}).", name(), subtaskIndex);
    +		}
    +		this.pendingCommitTransactions.clear();
    +	}
    +
    +	@Override
    +	public void open(Configuration parameters) throws Exception {
    --- End diff --
    
    In a function that has `initializeState()` you can also put this code in there because `initializeState()` will always be called. Having both `open()` and `initializeState()` is tricky because of the ordering in which those two are called (this has lead to problems in the past when people where changing code and not completely aware of the ordering). For the record, currently the ordering is: 1) `initializeState()` 2) `open()`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r131091549
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -0,0 +1,311 @@
    +/*
    + * 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.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.util.Iterator;
    +import java.util.LinkedHashMap;
    +import java.util.Map;
    +
    +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)
    --- End diff --
    
    nit: either wrap `Serializable` with `@link` or lower-case s.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/4368
  
    I would also like another look a bit later today. Can merge after that :)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by EronWright <gi...@git.apache.org>.
Github user EronWright commented on the issue:

    https://github.com/apache/flink/pull/4368
  
    @pnowojski looks great.   You mentioned the Pravaga connector as a motivation, did you look at its implementation  and do you anticipate any challenges in porting it o this new framework?
    
    https://github.com/pravega/flink-connectors/blob/master/src/main/java/io/pravega/connectors/flink/FlinkExactlyOncePravegaWriter.java



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r131094885
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -0,0 +1,311 @@
    +/*
    + * 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.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.util.Iterator;
    +import java.util.LinkedHashMap;
    +import java.util.Map;
    +
    +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 RichSinkFunction<IN>
    +		implements CheckpointedFunction, CheckpointListener {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(TwoPhaseCommitSinkFunction.class);
    +
    +	protected final ListStateDescriptor<Map<Long, TXN>> pendingCommitTransactionsDescriptor;
    +	protected final ListStateDescriptor<TXN> pendingTransactionsDescriptor;
    +
    +	protected final LinkedHashMap<Long, TXN> pendingCommitTransactions = new LinkedHashMap<>();
    +
    +	@Nullable
    +	protected TXN currentTransaction;
    +	protected ListState<TXN> pendingTransactionsState;
    +	protected ListState<Map<Long, TXN>> pendingCommitTransactionsState;
    +
    +	/**
    +	 * Use default {@link ListStateDescriptor} for internal state serialization. Helpful utilities for using this
    +	 * constructor are {@link TypeInformation#of(Class)}, {@link org.apache.flink.api.common.typeinfo.TypeHint} and
    +	 * {@link TypeInformation#of(TypeHint)}. Example:
    +	 * <pre>
    +	 * {@code
    +	 * TwoPhaseCommitSinkFunction(
    +	 *     TypeInformation.of(TXN.class),
    +	 *     TypeInformation.of(new TypeHint<TransactionAndCheckpoint<TXN>>() {}));
    +	 * }
    +	 * </pre>
    +	 * @param txnTypeInformation {@link TypeInformation} for transaction POJO.
    +	 * @param checkpointToTxnTypeInformation {@link TypeInformation} for mapping between checkpointId and transaction.
    +	 */
    +	public TwoPhaseCommitSinkFunction(
    +			TypeInformation<TXN> txnTypeInformation,
    +			TypeInformation<Map<Long, TXN>> checkpointToTxnTypeInformation) {
    +		this(new ListStateDescriptor<>("pendingTransactions", txnTypeInformation),
    +			new ListStateDescriptor<>("pendingCommitTransactions", checkpointToTxnTypeInformation));
    +	}
    +
    +	/**
    +	 * Instantiate {@link TwoPhaseCommitSinkFunction} with custom state descriptors.
    +	 *
    +	 * @param pendingTransactionsDescriptor descriptor for transaction POJO.
    +	 * @param pendingCommitTransactionsDescriptor descriptor for mapping between checkpointId and transaction POJO.
    +	 */
    +	public TwoPhaseCommitSinkFunction(
    +			ListStateDescriptor<TXN> pendingTransactionsDescriptor,
    +			ListStateDescriptor<Map<Long, 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(Object)} will be called again for the
    +	 * same transaction.
    +	 */
    +	protected abstract void commit(TXN transaction);
    +
    +	/**
    +	 * Invoked on recovered transactions after a failure. User implementation must ensure that this call will eventually
    +	 * succeed. If it fails, Flink application will be restarted and it will be invoked again. If it does not succeed
    +	 * a data loss will occur.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one. As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<Map.Entry<Long, TXN>> pendingTransactionsIterator = pendingCommitTransactions.entrySet().iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		while (pendingTransactionsIterator.hasNext()) {
    +			Map.Entry<Long, TXN> entry = pendingTransactionsIterator.next();
    +			Long pendingTransactionCheckpointId = entry.getKey();
    +			TXN pendingTransaction = entry.getValue();
    +			if (pendingTransactionCheckpointId > checkpointId) {
    +				continue;
    +			}
    +
    +			LOG.info("{} - checkpoint {} complete, committing completed checkpoint transaction {}",
    +				name(), checkpointId, pendingTransaction);
    --- End diff --
    
    One other thing:
    Right now we can't really guarantee that the user properly implements `toString` on `TXN`. Not sure if it really matters that much, though.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on the issue:

    https://github.com/apache/flink/pull/4368
  
    @EronWright, there shouldn't be any difficulties with that. I think that there is only one functional difference between `TwoPhaseCommitSourceFunction` and `PravegaWriter` - the first one automatically abort some (not all) of the "dangling" transactions after restoring the state.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129522314
  
    --- 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) {
    --- End diff --
    
    As you wish, I have no experience with serialization in Java, so I will leave this decision up to you.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r131094488
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -0,0 +1,311 @@
    +/*
    + * 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.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.util.Iterator;
    +import java.util.LinkedHashMap;
    +import java.util.Map;
    +
    +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 RichSinkFunction<IN>
    +		implements CheckpointedFunction, CheckpointListener {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(TwoPhaseCommitSinkFunction.class);
    +
    +	protected final ListStateDescriptor<Map<Long, TXN>> pendingCommitTransactionsDescriptor;
    +	protected final ListStateDescriptor<TXN> pendingTransactionsDescriptor;
    +
    +	protected final LinkedHashMap<Long, TXN> pendingCommitTransactions = new LinkedHashMap<>();
    +
    +	@Nullable
    +	protected TXN currentTransaction;
    +	protected ListState<TXN> pendingTransactionsState;
    +	protected ListState<Map<Long, TXN>> pendingCommitTransactionsState;
    +
    +	/**
    +	 * Use default {@link ListStateDescriptor} for internal state serialization. Helpful utilities for using this
    +	 * constructor are {@link TypeInformation#of(Class)}, {@link org.apache.flink.api.common.typeinfo.TypeHint} and
    +	 * {@link TypeInformation#of(TypeHint)}. Example:
    +	 * <pre>
    +	 * {@code
    +	 * TwoPhaseCommitSinkFunction(
    +	 *     TypeInformation.of(TXN.class),
    +	 *     TypeInformation.of(new TypeHint<TransactionAndCheckpoint<TXN>>() {}));
    +	 * }
    +	 * </pre>
    +	 * @param txnTypeInformation {@link TypeInformation} for transaction POJO.
    +	 * @param checkpointToTxnTypeInformation {@link TypeInformation} for mapping between checkpointId and transaction.
    +	 */
    +	public TwoPhaseCommitSinkFunction(
    +			TypeInformation<TXN> txnTypeInformation,
    +			TypeInformation<Map<Long, TXN>> checkpointToTxnTypeInformation) {
    +		this(new ListStateDescriptor<>("pendingTransactions", txnTypeInformation),
    +			new ListStateDescriptor<>("pendingCommitTransactions", checkpointToTxnTypeInformation));
    +	}
    +
    +	/**
    +	 * Instantiate {@link TwoPhaseCommitSinkFunction} with custom state descriptors.
    +	 *
    +	 * @param pendingTransactionsDescriptor descriptor for transaction POJO.
    +	 * @param pendingCommitTransactionsDescriptor descriptor for mapping between checkpointId and transaction POJO.
    +	 */
    +	public TwoPhaseCommitSinkFunction(
    +			ListStateDescriptor<TXN> pendingTransactionsDescriptor,
    +			ListStateDescriptor<Map<Long, 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(Object)} will be called again for the
    +	 * same transaction.
    +	 */
    +	protected abstract void commit(TXN transaction);
    +
    +	/**
    +	 * Invoked on recovered transactions after a failure. User implementation must ensure that this call will eventually
    +	 * succeed. If it fails, Flink application will be restarted and it will be invoked again. If it does not succeed
    +	 * a data loss will occur.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one. As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<Map.Entry<Long, TXN>> pendingTransactionsIterator = pendingCommitTransactions.entrySet().iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		while (pendingTransactionsIterator.hasNext()) {
    +			Map.Entry<Long, TXN> entry = pendingTransactionsIterator.next();
    +			Long pendingTransactionCheckpointId = entry.getKey();
    +			TXN pendingTransaction = entry.getValue();
    +			if (pendingTransactionCheckpointId > checkpointId) {
    +				continue;
    +			}
    +
    +			LOG.info("{} - checkpoint {} complete, committing completed checkpoint transaction {}",
    +				name(), checkpointId, pendingTransaction);
    +
    +			commit(pendingTransaction);
    +
    +			LOG.debug("{} - committed checkpoint transaction {}", name(), pendingTransaction);
    +
    +			pendingTransactionsIterator.remove();
    +		}
    +	}
    +
    +	@Override
    +	public final void snapshotState(FunctionSnapshotContext context) throws Exception {
    +		// this is like the pre-commit of a 2-phase-commit transaction
    +		// we are ready to commit and remember the transaction
    +
    +		checkState(currentTransaction != null, "bug: no transaction object when performing state snapshot");
    +
    +		long checkpointId = context.getCheckpointId();
    +		LOG.debug("{} - checkpoint {} triggered, flushing transaction '{}'", name(), context.getCheckpointId(), currentTransaction);
    +
    +		preCommit(currentTransaction);
    +		pendingCommitTransactions.put(checkpointId, currentTransaction);
    +		LOG.debug("{} - stored pending transactions {}", name(), pendingCommitTransactions);
    +
    +		currentTransaction = beginTransaction();
    +		LOG.debug("{} - started new transaction '{}'", name(), currentTransaction);
    +
    +		pendingCommitTransactionsState.clear();
    +		pendingCommitTransactionsState.add(pendingCommitTransactions);
    +
    +		pendingTransactionsState.clear();
    +		// in case of failure we might not be able to abort currentTransaction. Let's store it into the state
    +		// so it can be aborted after a restart/crash
    +		pendingTransactionsState.add(currentTransaction);
    +	}
    +
    +	@Override
    +	public final void initializeState(FunctionInitializationContext context) throws Exception {
    +		// when we are restoring state with pendingCommitTransactions, we don't really know whether the
    +		// transactions were already committed, or whether there was a failure between
    +		// completing the checkpoint on the master, and notifying the writer here.
    +
    +		// (the common case is actually that is was already committed, the window
    +		// between the commit on the master and the notification here is very small)
    +
    +		// it is possible to not have any transactions at all if there was a failure before
    +		// the first completed checkpoint, or in case of a scale-out event, where some of the
    +		// new task do not have and transactions assigned to check)
    +
    +		// we can have more than one transaction to check in case of a scale-in event, or
    +		// for the reasons discussed in the 'notifyCheckpointComplete()' method.
    +
    +		pendingTransactionsState = context.getOperatorStateStore().getListState(pendingTransactionsDescriptor);
    +		pendingCommitTransactionsState = context.getOperatorStateStore().getListState(pendingCommitTransactionsDescriptor);
    +
    +		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
    +		if (context.isRestored()) {
    +			LOG.info("{} - restoring state (subtaskIndex={}).", name(), subtaskIndex);
    --- End diff --
    
    Maybe we don't need the `(subtaskIndex={})` now, since `name()` contains that?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on the issue:

    https://github.com/apache/flink/pull/4368
  
    @pnowojski The changes look good! I'll have another look at the whole thing before merging? Or maybe @tzulitai wants to do that? 😃 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129234765
  
    --- 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.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<TransactionAndCheckpoint<TXN>> pendingTransactionsIterator = pendingCommitTransactions.iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		List<TransactionAndCheckpoint<TXN>> remainingTransactions = new ArrayList<>();
    +
    +		for (TransactionAndCheckpoint<TXN> pendingTransaction : pendingCommitTransactions) {
    +			if (pendingTransaction.checkpointId > checkpointId) {
    +				remainingTransactions.add(pendingTransaction);
    --- End diff --
    
    this seems redundant. It should be able to simply be removal while iterating, no?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129217538
  
    --- 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 --
    
    "If it does not succeed it means a data loss will occur."
    I stumbled across this statement here a bit.
    From what I understand, this would never happen since commits will always eventually succeed, correct? If so, I would suggest to remove this sentence.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129234426
  
    --- 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.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<TransactionAndCheckpoint<TXN>> pendingTransactionsIterator = pendingCommitTransactions.iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		List<TransactionAndCheckpoint<TXN>> remainingTransactions = new ArrayList<>();
    +
    +		for (TransactionAndCheckpoint<TXN> pendingTransaction : pendingCommitTransactions) {
    +			if (pendingTransaction.checkpointId > checkpointId) {
    +				remainingTransactions.add(pendingTransaction);
    +				continue;
    +			}
    +
    +			LOG.info("{} - checkpoint {} complete, committing completed checkpoint transaction {}",
    +				name(), checkpointId, pendingTransaction);
    +
    +			// If this fails, there is actually a data loss
    --- End diff --
    
    Is there actually a data loss? Wouldn't we recover from that when restoring with `recoverAndCommit`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129581412
  
    --- 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.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<TransactionAndCheckpoint<TXN>> pendingTransactionsIterator = pendingCommitTransactions.iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		List<TransactionAndCheckpoint<TXN>> remainingTransactions = new ArrayList<>();
    +
    +		for (TransactionAndCheckpoint<TXN> pendingTransaction : pendingCommitTransactions) {
    +			if (pendingTransaction.checkpointId > checkpointId) {
    +				remainingTransactions.add(pendingTransaction);
    +				continue;
    +			}
    +
    +			LOG.info("{} - checkpoint {} complete, committing completed checkpoint transaction {}",
    +				name(), checkpointId, pendingTransaction);
    +
    +			// If this fails, there is actually a data loss
    --- End diff --
    
    Yes. This comment should be near `recoverAndCommit` call, because that is where we would have a data loss.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129523821
  
    --- 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>
    --- End diff --
    
    I think that was a relict from a previous version. Dropped.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r131332988
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -0,0 +1,311 @@
    +/*
    + * 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.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.util.Iterator;
    +import java.util.LinkedHashMap;
    +import java.util.Map;
    +
    +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)
    --- End diff --
    
    it doesn't need to be serializable anymore! dropped this mention :)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/4368
  
    Alright, this looks good to merge now!
    Merging ..


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/4368


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r131093966
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -0,0 +1,311 @@
    +/*
    + * 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.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.util.Iterator;
    +import java.util.LinkedHashMap;
    +import java.util.Map;
    +
    +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 RichSinkFunction<IN>
    +		implements CheckpointedFunction, CheckpointListener {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(TwoPhaseCommitSinkFunction.class);
    +
    +	protected final ListStateDescriptor<Map<Long, TXN>> pendingCommitTransactionsDescriptor;
    +	protected final ListStateDescriptor<TXN> pendingTransactionsDescriptor;
    +
    +	protected final LinkedHashMap<Long, TXN> pendingCommitTransactions = new LinkedHashMap<>();
    +
    +	@Nullable
    +	protected TXN currentTransaction;
    +	protected ListState<TXN> pendingTransactionsState;
    +	protected ListState<Map<Long, TXN>> pendingCommitTransactionsState;
    +
    +	/**
    +	 * Use default {@link ListStateDescriptor} for internal state serialization. Helpful utilities for using this
    +	 * constructor are {@link TypeInformation#of(Class)}, {@link org.apache.flink.api.common.typeinfo.TypeHint} and
    +	 * {@link TypeInformation#of(TypeHint)}. Example:
    +	 * <pre>
    +	 * {@code
    +	 * TwoPhaseCommitSinkFunction(
    +	 *     TypeInformation.of(TXN.class),
    +	 *     TypeInformation.of(new TypeHint<TransactionAndCheckpoint<TXN>>() {}));
    +	 * }
    +	 * </pre>
    +	 * @param txnTypeInformation {@link TypeInformation} for transaction POJO.
    +	 * @param checkpointToTxnTypeInformation {@link TypeInformation} for mapping between checkpointId and transaction.
    +	 */
    +	public TwoPhaseCommitSinkFunction(
    +			TypeInformation<TXN> txnTypeInformation,
    +			TypeInformation<Map<Long, TXN>> checkpointToTxnTypeInformation) {
    +		this(new ListStateDescriptor<>("pendingTransactions", txnTypeInformation),
    +			new ListStateDescriptor<>("pendingCommitTransactions", checkpointToTxnTypeInformation));
    +	}
    +
    +	/**
    +	 * Instantiate {@link TwoPhaseCommitSinkFunction} with custom state descriptors.
    +	 *
    +	 * @param pendingTransactionsDescriptor descriptor for transaction POJO.
    +	 * @param pendingCommitTransactionsDescriptor descriptor for mapping between checkpointId and transaction POJO.
    +	 */
    +	public TwoPhaseCommitSinkFunction(
    +			ListStateDescriptor<TXN> pendingTransactionsDescriptor,
    +			ListStateDescriptor<Map<Long, 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(Object)} will be called again for the
    +	 * same transaction.
    +	 */
    +	protected abstract void commit(TXN transaction);
    +
    +	/**
    +	 * Invoked on recovered transactions after a failure. User implementation must ensure that this call will eventually
    +	 * succeed. If it fails, Flink application will be restarted and it will be invoked again. If it does not succeed
    +	 * a data loss will occur.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one. As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<Map.Entry<Long, TXN>> pendingTransactionsIterator = pendingCommitTransactions.entrySet().iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		while (pendingTransactionsIterator.hasNext()) {
    +			Map.Entry<Long, TXN> entry = pendingTransactionsIterator.next();
    +			Long pendingTransactionCheckpointId = entry.getKey();
    +			TXN pendingTransaction = entry.getValue();
    +			if (pendingTransactionCheckpointId > checkpointId) {
    +				continue;
    +			}
    +
    +			LOG.info("{} - checkpoint {} complete, committing completed checkpoint transaction {}",
    +				name(), checkpointId, pendingTransaction);
    --- End diff --
    
    Maybe we should include the `pendingTransactionCheckpointId` in this log?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r131094983
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -0,0 +1,311 @@
    +/*
    + * 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.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.util.Iterator;
    +import java.util.LinkedHashMap;
    +import java.util.Map;
    +
    +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 RichSinkFunction<IN>
    +		implements CheckpointedFunction, CheckpointListener {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(TwoPhaseCommitSinkFunction.class);
    +
    +	protected final ListStateDescriptor<Map<Long, TXN>> pendingCommitTransactionsDescriptor;
    +	protected final ListStateDescriptor<TXN> pendingTransactionsDescriptor;
    +
    +	protected final LinkedHashMap<Long, TXN> pendingCommitTransactions = new LinkedHashMap<>();
    +
    +	@Nullable
    +	protected TXN currentTransaction;
    +	protected ListState<TXN> pendingTransactionsState;
    +	protected ListState<Map<Long, TXN>> pendingCommitTransactionsState;
    +
    +	/**
    +	 * Use default {@link ListStateDescriptor} for internal state serialization. Helpful utilities for using this
    +	 * constructor are {@link TypeInformation#of(Class)}, {@link org.apache.flink.api.common.typeinfo.TypeHint} and
    +	 * {@link TypeInformation#of(TypeHint)}. Example:
    +	 * <pre>
    +	 * {@code
    +	 * TwoPhaseCommitSinkFunction(
    +	 *     TypeInformation.of(TXN.class),
    +	 *     TypeInformation.of(new TypeHint<TransactionAndCheckpoint<TXN>>() {}));
    +	 * }
    +	 * </pre>
    +	 * @param txnTypeInformation {@link TypeInformation} for transaction POJO.
    +	 * @param checkpointToTxnTypeInformation {@link TypeInformation} for mapping between checkpointId and transaction.
    +	 */
    +	public TwoPhaseCommitSinkFunction(
    +			TypeInformation<TXN> txnTypeInformation,
    +			TypeInformation<Map<Long, TXN>> checkpointToTxnTypeInformation) {
    +		this(new ListStateDescriptor<>("pendingTransactions", txnTypeInformation),
    +			new ListStateDescriptor<>("pendingCommitTransactions", checkpointToTxnTypeInformation));
    +	}
    +
    +	/**
    +	 * Instantiate {@link TwoPhaseCommitSinkFunction} with custom state descriptors.
    +	 *
    +	 * @param pendingTransactionsDescriptor descriptor for transaction POJO.
    +	 * @param pendingCommitTransactionsDescriptor descriptor for mapping between checkpointId and transaction POJO.
    +	 */
    +	public TwoPhaseCommitSinkFunction(
    +			ListStateDescriptor<TXN> pendingTransactionsDescriptor,
    +			ListStateDescriptor<Map<Long, 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(Object)} will be called again for the
    +	 * same transaction.
    +	 */
    +	protected abstract void commit(TXN transaction);
    +
    +	/**
    +	 * Invoked on recovered transactions after a failure. User implementation must ensure that this call will eventually
    +	 * succeed. If it fails, Flink application will be restarted and it will be invoked again. If it does not succeed
    +	 * a data loss will occur.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one. As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<Map.Entry<Long, TXN>> pendingTransactionsIterator = pendingCommitTransactions.entrySet().iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		while (pendingTransactionsIterator.hasNext()) {
    +			Map.Entry<Long, TXN> entry = pendingTransactionsIterator.next();
    +			Long pendingTransactionCheckpointId = entry.getKey();
    +			TXN pendingTransaction = entry.getValue();
    +			if (pendingTransactionCheckpointId > checkpointId) {
    +				continue;
    +			}
    +
    +			LOG.info("{} - checkpoint {} complete, committing completed checkpoint transaction {}",
    +				name(), checkpointId, pendingTransaction);
    +
    +			commit(pendingTransaction);
    +
    +			LOG.debug("{} - committed checkpoint transaction {}", name(), pendingTransaction);
    +
    +			pendingTransactionsIterator.remove();
    +		}
    +	}
    +
    +	@Override
    +	public final void snapshotState(FunctionSnapshotContext context) throws Exception {
    +		// this is like the pre-commit of a 2-phase-commit transaction
    +		// we are ready to commit and remember the transaction
    +
    +		checkState(currentTransaction != null, "bug: no transaction object when performing state snapshot");
    +
    +		long checkpointId = context.getCheckpointId();
    +		LOG.debug("{} - checkpoint {} triggered, flushing transaction '{}'", name(), context.getCheckpointId(), currentTransaction);
    +
    +		preCommit(currentTransaction);
    +		pendingCommitTransactions.put(checkpointId, currentTransaction);
    +		LOG.debug("{} - stored pending transactions {}", name(), pendingCommitTransactions);
    +
    +		currentTransaction = beginTransaction();
    +		LOG.debug("{} - started new transaction '{}'", name(), currentTransaction);
    +
    +		pendingCommitTransactionsState.clear();
    +		pendingCommitTransactionsState.add(pendingCommitTransactions);
    +
    +		pendingTransactionsState.clear();
    +		// in case of failure we might not be able to abort currentTransaction. Let's store it into the state
    +		// so it can be aborted after a restart/crash
    +		pendingTransactionsState.add(currentTransaction);
    +	}
    +
    +	@Override
    +	public final void initializeState(FunctionInitializationContext context) throws Exception {
    +		// when we are restoring state with pendingCommitTransactions, we don't really know whether the
    +		// transactions were already committed, or whether there was a failure between
    +		// completing the checkpoint on the master, and notifying the writer here.
    +
    +		// (the common case is actually that is was already committed, the window
    +		// between the commit on the master and the notification here is very small)
    +
    +		// it is possible to not have any transactions at all if there was a failure before
    +		// the first completed checkpoint, or in case of a scale-out event, where some of the
    +		// new task do not have and transactions assigned to check)
    +
    +		// we can have more than one transaction to check in case of a scale-in event, or
    +		// for the reasons discussed in the 'notifyCheckpointComplete()' method.
    +
    +		pendingTransactionsState = context.getOperatorStateStore().getListState(pendingTransactionsDescriptor);
    +		pendingCommitTransactionsState = context.getOperatorStateStore().getListState(pendingCommitTransactionsDescriptor);
    +
    +		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
    +		if (context.isRestored()) {
    +			LOG.info("{} - restoring state (subtaskIndex={}).", name(), subtaskIndex);
    +
    +			for (Map<Long, TXN> pendingCommitTransactionsMaps : pendingCommitTransactionsState.get()) {
    +				for (TXN recoveredTransaction : pendingCommitTransactionsMaps.values()) {
    +					// If this fails, there is actually a data loss
    +					recoverAndCommit(recoveredTransaction);
    +					LOG.info("{} committed recovered transaction {} (subtaskIndex={})", name(), recoveredTransaction, subtaskIndex);
    +				}
    +			}
    +
    +			// Explicitly abort transactions that could be not closed cleanly
    +			for (TXN pendingTransaction : pendingTransactionsState.get()) {
    +				recoverAndAbort(pendingTransaction);
    +				LOG.info("{} aborted recovered transaction {} (subtaskIndex={})", name(), pendingTransaction, subtaskIndex);
    +			}
    +		} else {
    +			LOG.info("{} - no state to restore {} (subtaskIndex={}).", name(), subtaskIndex);
    +		}
    +		this.pendingCommitTransactions.clear();
    +
    +		currentTransaction = beginTransaction();
    +		LOG.debug("{} - started new transaction '{}'", name(), currentTransaction);
    +	}
    +
    +	@Override
    +	public void close() throws Exception {
    +		super.close();
    +
    +		if (currentTransaction != null) {
    +			abort(currentTransaction);
    +			currentTransaction = null;
    +		}
    +	}
    +
    +	private String name() {
    +		return String.format(
    +			"%s %s/%s",
    +			this.getClass().getSimpleName(),
    +			getRuntimeContext().getIndexOfThisSubtask(),
    +			getRuntimeContext().getNumberOfParallelSubtasks());
    +	}
    +
    +	private static void checkState(boolean condition, String message, Object... args) {
    --- End diff --
    
    Can be replaced with `Preconditions.checkState`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129531220
  
    --- 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.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<TransactionAndCheckpoint<TXN>> pendingTransactionsIterator = pendingCommitTransactions.iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		List<TransactionAndCheckpoint<TXN>> remainingTransactions = new ArrayList<>();
    +
    +		for (TransactionAndCheckpoint<TXN> pendingTransaction : pendingCommitTransactions) {
    +			if (pendingTransaction.checkpointId > checkpointId) {
    +				remainingTransactions.add(pendingTransaction);
    --- End diff --
    
    see above


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129235825
  
    --- 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.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<TransactionAndCheckpoint<TXN>> pendingTransactionsIterator = pendingCommitTransactions.iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		List<TransactionAndCheckpoint<TXN>> remainingTransactions = new ArrayList<>();
    +
    +		for (TransactionAndCheckpoint<TXN> pendingTransaction : pendingCommitTransactions) {
    +			if (pendingTransaction.checkpointId > checkpointId) {
    +				remainingTransactions.add(pendingTransaction);
    +				continue;
    +			}
    +
    +			LOG.info("{} - checkpoint {} complete, committing completed checkpoint transaction {}",
    +				name(), checkpointId, pendingTransaction);
    +
    +			// If this fails, there is actually a data loss
    +			commit(pendingTransaction.transaction);
    +
    +			LOG.debug("{} - committed checkpoint transaction {}", name(), pendingTransaction);
    +		}
    +
    +		pendingCommitTransactions.clear();
    +		for (TransactionAndCheckpoint<TXN> remainingTransaction : remainingTransactions) {
    +			pendingCommitTransactions.add(remainingTransaction);
    +		}
    +	}
    +
    +	@Override
    +	public final void snapshotState(FunctionSnapshotContext context) throws Exception {
    +		// this is like the pre-commit of a 2-phase-commit transaction
    +		// we are ready to commit and remember the transaction
    +
    +		checkState(currentTransaction != null, "bug: no transaction object when performing state snapshot");
    +
    +		long checkpointId = context.getCheckpointId();
    +		LOG.debug("{} - checkpoint {} triggered, flushing transaction '{}'", name(), context.getCheckpointId(), currentTransaction);
    +
    +		preCommit(currentTransaction);
    +		pendingCommitTransactions.add(new TransactionAndCheckpoint<>(currentTransaction, checkpointId));
    +		LOG.debug("{} - stored pending transactions {}", name(), pendingCommitTransactions);
    +
    +		currentTransaction = beginTransaction();
    +		LOG.debug("{} - started new transaction '{}'", name(), currentTransaction);
    +
    +		pendingCommitTransactionsState.clear();
    +		for (TransactionAndCheckpoint<TXN> pendingCommitTransaction : pendingCommitTransactions) {
    +			pendingCommitTransactionsState.add(pendingCommitTransaction);
    +		}
    +
    +		pendingTransactionsState.clear();
    +		// in case of failure we might not be able to abort currentTransaction. Let's store it into the state
    +		// so it can be aborted after a restart/crash
    +		pendingTransactionsState.add(currentTransaction);
    +	}
    +
    +	@Override
    +	public final void initializeState(FunctionInitializationContext context) throws Exception {
    +		// when we are restoring state with pendingCommitTransactions, we don't really know whether the
    +		// transactions were already committed, or whether there was a failure between
    +		// completing the checkpoint on the master, and notifying the writer here.
    +
    +		// (the common case is actually that is was already committed, the window
    +		// between the commit on the master and the notification here is very small)
    +
    +		// it is possible to not have any transactions at all if there was a failure before
    +		// the first completed checkpoint, or in case of a scale-out event, where some of the
    +		// new task do not have and transactions assigned to check)
    +
    +		// we can have more than one transaction to check in case of a scale-in event, or
    +		// for the reasons discussed in the 'notifyCheckpointComplete()' method.
    +
    +		pendingTransactionsState = context.getOperatorStateStore().getListState(pendingTransactionsDescriptor);
    +		pendingCommitTransactionsState = context.getOperatorStateStore().getListState(pendingCommitTransactionsDescriptor);
    +
    +		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
    +		if (context.isRestored()) {
    +			LOG.info("{} - restoring state (subtaskIndex={}).", name(), subtaskIndex);
    +
    +			for (TransactionAndCheckpoint<TXN> recoveredTransaction : pendingCommitTransactionsState.get()) {
    +				recoverAndCommit(recoveredTransaction.transaction);
    +				LOG.debug("{} committed recovered transaction {} (subtaskIndex={})", name(), recoveredTransaction, subtaskIndex);
    +			}
    +
    +			// Explicitly abort transactions that could be not closed cleanly
    +			for (TXN pendingTransaction : pendingTransactionsState.get()) {
    +				recoverAndAbort(pendingTransaction);
    +				LOG.debug("{} aborted recovered transaction {} (subtaskIndex={})", name(), pendingTransaction, subtaskIndex);
    --- End diff --
    
    I think these are worth INFO level, what do you think?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r131070533
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -51,39 +48,52 @@
      * @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>
    +public abstract class TwoPhaseCommitSinkFunction<IN, TXN>
     		extends RichSinkFunction<IN>
     		implements CheckpointedFunction, CheckpointListener {
     
     	private static final Logger LOG = LoggerFactory.getLogger(TwoPhaseCommitSinkFunction.class);
     
    -	protected final ListStateDescriptor<TransactionAndCheckpoint<TXN>> pendingCommitTransactionsDescriptor;
    +	protected final ListStateDescriptor<Map<Long, TXN>> pendingCommitTransactionsDescriptor;
     	protected final ListStateDescriptor<TXN> pendingTransactionsDescriptor;
     
    -	protected final List<TransactionAndCheckpoint<TXN>> pendingCommitTransactions = new ArrayList<>();
    +	protected final LinkedHashMap<Long, TXN> pendingCommitTransactions = new LinkedHashMap<>();
     
     	@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>>() {}));
    -	}
    +	protected ListState<Map<Long, TXN>> pendingCommitTransactionsState;
    --- End diff --
    
    I think this has to be `ListState<Tuple2<Long, TXN>>` or the original `ListState<TransactionAndCheckpoint<TXN>>`.
    
    Using a map instead of a list for `pendingCommitTransactions ` is ok for bookkeeping, but when snapshotting this map we need to make sure the snapshotted transactions are state elements that can be redistributable independent of each other. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129339683
  
    --- 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>
    --- End diff --
    
    Why does `TXN` need to be `Serializable`? It should always be serialised using Flink mechanics (`TypeSerializer` and so on).  


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129235807
  
    --- 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.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<TransactionAndCheckpoint<TXN>> pendingTransactionsIterator = pendingCommitTransactions.iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		List<TransactionAndCheckpoint<TXN>> remainingTransactions = new ArrayList<>();
    +
    +		for (TransactionAndCheckpoint<TXN> pendingTransaction : pendingCommitTransactions) {
    +			if (pendingTransaction.checkpointId > checkpointId) {
    +				remainingTransactions.add(pendingTransaction);
    +				continue;
    +			}
    +
    +			LOG.info("{} - checkpoint {} complete, committing completed checkpoint transaction {}",
    +				name(), checkpointId, pendingTransaction);
    +
    +			// If this fails, there is actually a data loss
    +			commit(pendingTransaction.transaction);
    +
    +			LOG.debug("{} - committed checkpoint transaction {}", name(), pendingTransaction);
    +		}
    +
    +		pendingCommitTransactions.clear();
    +		for (TransactionAndCheckpoint<TXN> remainingTransaction : remainingTransactions) {
    +			pendingCommitTransactions.add(remainingTransaction);
    +		}
    +	}
    +
    +	@Override
    +	public final void snapshotState(FunctionSnapshotContext context) throws Exception {
    +		// this is like the pre-commit of a 2-phase-commit transaction
    +		// we are ready to commit and remember the transaction
    +
    +		checkState(currentTransaction != null, "bug: no transaction object when performing state snapshot");
    +
    +		long checkpointId = context.getCheckpointId();
    +		LOG.debug("{} - checkpoint {} triggered, flushing transaction '{}'", name(), context.getCheckpointId(), currentTransaction);
    +
    +		preCommit(currentTransaction);
    +		pendingCommitTransactions.add(new TransactionAndCheckpoint<>(currentTransaction, checkpointId));
    +		LOG.debug("{} - stored pending transactions {}", name(), pendingCommitTransactions);
    +
    +		currentTransaction = beginTransaction();
    +		LOG.debug("{} - started new transaction '{}'", name(), currentTransaction);
    +
    +		pendingCommitTransactionsState.clear();
    +		for (TransactionAndCheckpoint<TXN> pendingCommitTransaction : pendingCommitTransactions) {
    +			pendingCommitTransactionsState.add(pendingCommitTransaction);
    +		}
    +
    +		pendingTransactionsState.clear();
    +		// in case of failure we might not be able to abort currentTransaction. Let's store it into the state
    +		// so it can be aborted after a restart/crash
    +		pendingTransactionsState.add(currentTransaction);
    +	}
    +
    +	@Override
    +	public final void initializeState(FunctionInitializationContext context) throws Exception {
    +		// when we are restoring state with pendingCommitTransactions, we don't really know whether the
    +		// transactions were already committed, or whether there was a failure between
    +		// completing the checkpoint on the master, and notifying the writer here.
    +
    +		// (the common case is actually that is was already committed, the window
    +		// between the commit on the master and the notification here is very small)
    +
    +		// it is possible to not have any transactions at all if there was a failure before
    +		// the first completed checkpoint, or in case of a scale-out event, where some of the
    +		// new task do not have and transactions assigned to check)
    +
    +		// we can have more than one transaction to check in case of a scale-in event, or
    +		// for the reasons discussed in the 'notifyCheckpointComplete()' method.
    +
    +		pendingTransactionsState = context.getOperatorStateStore().getListState(pendingTransactionsDescriptor);
    +		pendingCommitTransactionsState = context.getOperatorStateStore().getListState(pendingCommitTransactionsDescriptor);
    +
    +		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
    +		if (context.isRestored()) {
    +			LOG.info("{} - restoring state (subtaskIndex={}).", name(), subtaskIndex);
    +
    +			for (TransactionAndCheckpoint<TXN> recoveredTransaction : pendingCommitTransactionsState.get()) {
    +				recoverAndCommit(recoveredTransaction.transaction);
    +				LOG.debug("{} committed recovered transaction {} (subtaskIndex={})", name(), recoveredTransaction, subtaskIndex);
    --- End diff --
    
    I think these are worth INFO level, what do you think?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129234336
  
    --- 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.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one As a consequence, we need to commit
    --- End diff --
    
    nit: missing period between "one" and "A".


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129234789
  
    --- 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.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<TransactionAndCheckpoint<TXN>> pendingTransactionsIterator = pendingCommitTransactions.iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		List<TransactionAndCheckpoint<TXN>> remainingTransactions = new ArrayList<>();
    +
    +		for (TransactionAndCheckpoint<TXN> pendingTransaction : pendingCommitTransactions) {
    +			if (pendingTransaction.checkpointId > checkpointId) {
    +				remainingTransactions.add(pendingTransaction);
    +				continue;
    +			}
    +
    +			LOG.info("{} - checkpoint {} complete, committing completed checkpoint transaction {}",
    +				name(), checkpointId, pendingTransaction);
    +
    +			// If this fails, there is actually a data loss
    +			commit(pendingTransaction.transaction);
    +
    +			LOG.debug("{} - committed checkpoint transaction {}", name(), pendingTransaction);
    +		}
    +
    +		pendingCommitTransactions.clear();
    +		for (TransactionAndCheckpoint<TXN> remainingTransaction : remainingTransactions) {
    +			pendingCommitTransactions.add(remainingTransaction);
    --- End diff --
    
    this seems redundant. It should be able to simply be removal while iterating, no?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r131334260
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -0,0 +1,311 @@
    +/*
    + * 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.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.util.Iterator;
    +import java.util.LinkedHashMap;
    +import java.util.Map;
    +
    +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 RichSinkFunction<IN>
    +		implements CheckpointedFunction, CheckpointListener {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(TwoPhaseCommitSinkFunction.class);
    +
    +	protected final ListStateDescriptor<Map<Long, TXN>> pendingCommitTransactionsDescriptor;
    +	protected final ListStateDescriptor<TXN> pendingTransactionsDescriptor;
    +
    +	protected final LinkedHashMap<Long, TXN> pendingCommitTransactions = new LinkedHashMap<>();
    +
    +	@Nullable
    +	protected TXN currentTransaction;
    +	protected ListState<TXN> pendingTransactionsState;
    +	protected ListState<Map<Long, TXN>> pendingCommitTransactionsState;
    +
    +	/**
    +	 * Use default {@link ListStateDescriptor} for internal state serialization. Helpful utilities for using this
    +	 * constructor are {@link TypeInformation#of(Class)}, {@link org.apache.flink.api.common.typeinfo.TypeHint} and
    +	 * {@link TypeInformation#of(TypeHint)}. Example:
    +	 * <pre>
    +	 * {@code
    +	 * TwoPhaseCommitSinkFunction(
    +	 *     TypeInformation.of(TXN.class),
    +	 *     TypeInformation.of(new TypeHint<TransactionAndCheckpoint<TXN>>() {}));
    +	 * }
    +	 * </pre>
    +	 * @param txnTypeInformation {@link TypeInformation} for transaction POJO.
    +	 * @param checkpointToTxnTypeInformation {@link TypeInformation} for mapping between checkpointId and transaction.
    +	 */
    +	public TwoPhaseCommitSinkFunction(
    +			TypeInformation<TXN> txnTypeInformation,
    +			TypeInformation<Map<Long, TXN>> checkpointToTxnTypeInformation) {
    +		this(new ListStateDescriptor<>("pendingTransactions", txnTypeInformation),
    +			new ListStateDescriptor<>("pendingCommitTransactions", checkpointToTxnTypeInformation));
    +	}
    +
    +	/**
    +	 * Instantiate {@link TwoPhaseCommitSinkFunction} with custom state descriptors.
    +	 *
    +	 * @param pendingTransactionsDescriptor descriptor for transaction POJO.
    +	 * @param pendingCommitTransactionsDescriptor descriptor for mapping between checkpointId and transaction POJO.
    +	 */
    +	public TwoPhaseCommitSinkFunction(
    +			ListStateDescriptor<TXN> pendingTransactionsDescriptor,
    +			ListStateDescriptor<Map<Long, 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(Object)} will be called again for the
    +	 * same transaction.
    +	 */
    +	protected abstract void commit(TXN transaction);
    +
    +	/**
    +	 * Invoked on recovered transactions after a failure. User implementation must ensure that this call will eventually
    +	 * succeed. If it fails, Flink application will be restarted and it will be invoked again. If it does not succeed
    +	 * a data loss will occur.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one. As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<Map.Entry<Long, TXN>> pendingTransactionsIterator = pendingCommitTransactions.entrySet().iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		while (pendingTransactionsIterator.hasNext()) {
    +			Map.Entry<Long, TXN> entry = pendingTransactionsIterator.next();
    +			Long pendingTransactionCheckpointId = entry.getKey();
    +			TXN pendingTransaction = entry.getValue();
    +			if (pendingTransactionCheckpointId > checkpointId) {
    +				continue;
    +			}
    +
    +			LOG.info("{} - checkpoint {} complete, committing completed checkpoint transaction {}",
    +				name(), checkpointId, pendingTransaction);
    --- End diff --
    
    `toString()` - I don't think that it is an issue.
    
    `pendingTransactionCheckpointId` - done


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129234639
  
    --- 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.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<TransactionAndCheckpoint<TXN>> pendingTransactionsIterator = pendingCommitTransactions.iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		List<TransactionAndCheckpoint<TXN>> remainingTransactions = new ArrayList<>();
    --- End diff --
    
    Why do we need an extra list here? Can't we just remove while iterating?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129233968
  
    --- 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(
    --- End diff --
    
    Could we have Javadocs for these constructors?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129234261
  
    --- 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.
    +	 */
    +	protected void recoverAndCommit(TXN transaction) {
    +		commit(transaction);
    +	}
    +
    +	/**
    +	 * Abort a transaction.
    +	 */
    +	protected abstract void abort(TXN transaction);
    +
    +	/**
    +	 * Abort a transaction that was rejected by a coordinator after a failure.
    +	 */
    +	protected void recoverAndAbort(TXN transaction) {
    +		abort(transaction);
    +	}
    +
    +	// ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------
    +
    +	@Override
    +	public final void invoke(IN value) throws Exception {
    +		invoke(currentTransaction, value);
    +	}
    +
    +	@Override
    +	public final void notifyCheckpointComplete(long checkpointId) throws Exception {
    +		// the following scenarios are possible here
    +		//
    +		//  (1) there is exactly one transaction from the latest checkpoint that
    +		//      was triggered and completed. That should be the common case.
    +		//      Simply commit that transaction in that case.
    +		//
    +		//  (2) there are multiple pending transactions because one previous
    +		//      checkpoint was skipped. That is a rare case, but can happen
    +		//      for example when:
    +		//
    +		//        - the master cannot persist the metadata of the last
    +		//          checkpoint (temporary outage in the storage system) but
    +		//          could persist a successive checkpoint (the one notified here)
    +		//
    +		//        - other tasks could not persist their status during
    +		//          the previous checkpoint, but did not trigger a failure because they
    +		//          could hold onto their state and could successfully persist it in
    +		//          a successive checkpoint (the one notified here)
    +		//
    +		//      In both cases, the prior checkpoint never reach a committed state, but
    +		//      this checkpoint is always expected to subsume the prior one and cover all
    +		//      changes since the last successful one As a consequence, we need to commit
    +		//      all pending transactions.
    +		//
    +		//  (3) Multiple transactions are pending, but the checkpoint complete notification
    +		//      relates not to the latest. That is possible, because notification messages
    +		//      can be delayed (in an extreme case till arrive after a succeeding checkpoint
    +		//      was triggered) and because there can be concurrent overlapping checkpoints
    +		//      (a new one is started before the previous fully finished).
    +		//
    +		// ==> There should never be a case where we have no pending transaction here
    +		//
    +
    +		Iterator<TransactionAndCheckpoint<TXN>> pendingTransactionsIterator = pendingCommitTransactions.iterator();
    +		checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending");
    +
    +		List<TransactionAndCheckpoint<TXN>> remainingTransactions = new ArrayList<>();
    +
    +		for (TransactionAndCheckpoint<TXN> pendingTransaction : pendingCommitTransactions) {
    +			if (pendingTransaction.checkpointId > checkpointId) {
    +				remainingTransactions.add(pendingTransaction);
    +				continue;
    +			}
    +
    +			LOG.info("{} - checkpoint {} complete, committing completed checkpoint transaction {}",
    +				name(), checkpointId, pendingTransaction);
    +
    +			// If this fails, there is actually a data loss
    +			commit(pendingTransaction.transaction);
    +
    +			LOG.debug("{} - committed checkpoint transaction {}", name(), pendingTransaction);
    +		}
    +
    +		pendingCommitTransactions.clear();
    +		for (TransactionAndCheckpoint<TXN> remainingTransaction : remainingTransactions) {
    +			pendingCommitTransactions.add(remainingTransaction);
    +		}
    +	}
    +
    +	@Override
    +	public final void snapshotState(FunctionSnapshotContext context) throws Exception {
    +		// this is like the pre-commit of a 2-phase-commit transaction
    +		// we are ready to commit and remember the transaction
    +
    +		checkState(currentTransaction != null, "bug: no transaction object when performing state snapshot");
    +
    +		long checkpointId = context.getCheckpointId();
    +		LOG.debug("{} - checkpoint {} triggered, flushing transaction '{}'", name(), context.getCheckpointId(), currentTransaction);
    +
    +		preCommit(currentTransaction);
    +		pendingCommitTransactions.add(new TransactionAndCheckpoint<>(currentTransaction, checkpointId));
    +		LOG.debug("{} - stored pending transactions {}", name(), pendingCommitTransactions);
    +
    +		currentTransaction = beginTransaction();
    +		LOG.debug("{} - started new transaction '{}'", name(), currentTransaction);
    +
    +		pendingCommitTransactionsState.clear();
    +		for (TransactionAndCheckpoint<TXN> pendingCommitTransaction : pendingCommitTransactions) {
    +			pendingCommitTransactionsState.add(pendingCommitTransaction);
    +		}
    +
    +		pendingTransactionsState.clear();
    +		// in case of failure we might not be able to abort currentTransaction. Let's store it into the state
    +		// so it can be aborted after a restart/crash
    +		pendingTransactionsState.add(currentTransaction);
    +	}
    +
    +	@Override
    +	public final void initializeState(FunctionInitializationContext context) throws Exception {
    +		// when we are restoring state with pendingCommitTransactions, we don't really know whether the
    +		// transactions were already committed, or whether there was a failure between
    +		// completing the checkpoint on the master, and notifying the writer here.
    +
    +		// (the common case is actually that is was already committed, the window
    +		// between the commit on the master and the notification here is very small)
    +
    +		// it is possible to not have any transactions at all if there was a failure before
    +		// the first completed checkpoint, or in case of a scale-out event, where some of the
    +		// new task do not have and transactions assigned to check)
    +
    +		// we can have more than one transaction to check in case of a scale-in event, or
    +		// for the reasons discussed in the 'notifyCheckpointComplete()' method.
    +
    +		pendingTransactionsState = context.getOperatorStateStore().getListState(pendingTransactionsDescriptor);
    +		pendingCommitTransactionsState = context.getOperatorStateStore().getListState(pendingCommitTransactionsDescriptor);
    +
    +		int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask();
    +		if (context.isRestored()) {
    +			LOG.info("{} - restoring state (subtaskIndex={}).", name(), subtaskIndex);
    +
    +			for (TransactionAndCheckpoint<TXN> recoveredTransaction : pendingCommitTransactionsState.get()) {
    +				recoverAndCommit(recoveredTransaction.transaction);
    +				LOG.debug("{} committed recovered transaction {} (subtaskIndex={})", name(), recoveredTransaction, subtaskIndex);
    +			}
    +
    +			// Explicitly abort transactions that could be not closed cleanly
    +			for (TXN pendingTransaction : pendingTransactionsState.get()) {
    +				recoverAndAbort(pendingTransaction);
    +				LOG.debug("{} aborted recovered transaction {} (subtaskIndex={})", name(), pendingTransaction, subtaskIndex);
    +			}
    +		} else {
    +			LOG.info("{} - no state to restore {} (subtaskIndex={}).", name(), subtaskIndex);
    +		}
    +		this.pendingCommitTransactions.clear();
    +	}
    +
    +	@Override
    +	public void open(Configuration parameters) throws Exception {
    +		super.open(parameters);
    +		currentTransaction = beginTransaction();
    +		LOG.debug("{} - started new transaction '{}'", name(), currentTransaction);
    +	}
    +
    +	@Override
    +	public void close() throws Exception {
    +		super.close();
    +
    +		if (currentTransaction != null) {
    +			abort(currentTransaction);
    +			currentTransaction = null;
    +		}
    +	}
    +
    +	private String name() {
    +		return this.getClass().getSimpleName();
    --- End diff --
    
    I think it would be more informative if we include the index of the sink subtask inside the name.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r131330545
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -51,39 +48,52 @@
      * @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>
    +public abstract class TwoPhaseCommitSinkFunction<IN, TXN>
     		extends RichSinkFunction<IN>
     		implements CheckpointedFunction, CheckpointListener {
     
     	private static final Logger LOG = LoggerFactory.getLogger(TwoPhaseCommitSinkFunction.class);
     
    -	protected final ListStateDescriptor<TransactionAndCheckpoint<TXN>> pendingCommitTransactionsDescriptor;
    +	protected final ListStateDescriptor<Map<Long, TXN>> pendingCommitTransactionsDescriptor;
     	protected final ListStateDescriptor<TXN> pendingTransactionsDescriptor;
     
    -	protected final List<TransactionAndCheckpoint<TXN>> pendingCommitTransactions = new ArrayList<>();
    +	protected final LinkedHashMap<Long, TXN> pendingCommitTransactions = new LinkedHashMap<>();
     
     	@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>>() {}));
    -	}
    +	protected ListState<Map<Long, TXN>> pendingCommitTransactionsState;
    --- End diff --
    
    Whether transactions can be redistributed depends on the system that we communicate with. For Kafka 0.11 I'm not sure. Now when I think about it a little bit more, I will change it to `ListState<List<Tuple2<Long, TXN>>>`, so that we can guarantee that transactions will be recovered in the same order they were created.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4368#discussion_r129233893
  
    --- 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) {
    --- End diff --
    
    I'm not too sure we actually want to expose this specific constructor.
    I'm not really fond of the fact that the user leaves the decision of how to serialize the state completely to this base class. One disadvantage I can think immediately of that is that the `TransactionAndCheckpoint` is not a POJO, and therefore Flink will try to serialize it using Kryo into state. In general, it is a bad idea to use Kryo for these kinds of persisted data.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on the issue:

    https://github.com/apache/flink/pull/4368
  
    Thanks! :)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---