Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id D2DB0200CD1 for ; Wed, 26 Jul 2017 17:22:35 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id D177C1690AB; Wed, 26 Jul 2017 15:22:35 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id F078A1690A9 for ; Wed, 26 Jul 2017 17:22:34 +0200 (CEST) Received: (qmail 74274 invoked by uid 500); 26 Jul 2017 15:22:34 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 74263 invoked by uid 99); 26 Jul 2017 15:22:34 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 26 Jul 2017 15:22:34 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id A5142C20DD for ; Wed, 26 Jul 2017 15:22:33 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 0.98 X-Spam-Level: X-Spam-Status: No, score=0.98 tagged_above=-999 required=6.31 tests=[KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-0.001, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id HXZ6dO-KedZd for ; Wed, 26 Jul 2017 15:22:23 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with SMTP id 271B75FCD8 for ; Wed, 26 Jul 2017 15:22:21 +0000 (UTC) Received: (qmail 67200 invoked by uid 99); 26 Jul 2017 15:22:21 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 26 Jul 2017 15:22:21 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 8CC50DF97C; Wed, 26 Jul 2017 15:22:20 +0000 (UTC) From: pnowojski To: issues@flink.incubator.apache.org Reply-To: issues@flink.incubator.apache.org References: In-Reply-To: Subject: [GitHub] flink pull request #4368: [FLINK-7210] Introduce TwoPhaseCommitSinkFunction Content-Type: text/plain Message-Id: <20170726152220.8CC50DF97C@git1-us-west.apache.org> Date: Wed, 26 Jul 2017 15:22:20 +0000 (UTC) archived-at: Wed, 26 Jul 2017 15:22:36 -0000 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 Input type for {@link SinkFunction} + * @param Transaction to store all of the information required to handle a transaction (must be Serializable) + */ +@PublicEvolving +public abstract class TwoPhaseCommitSinkFunction + extends RichSinkFunction + implements CheckpointedFunction, CheckpointListener { + + private static final Logger LOG = LoggerFactory.getLogger(TwoPhaseCommitSinkFunction.class); + + protected final ListStateDescriptor> pendingCommitTransactionsDescriptor; + protected final ListStateDescriptor pendingTransactionsDescriptor; + + protected final List> pendingCommitTransactions = new ArrayList<>(); + + @Nullable + protected TXN currentTransaction; + protected ListState pendingTransactionsState; + protected ListState> pendingCommitTransactionsState; + + public TwoPhaseCommitSinkFunction(Class txnClass) { + this( + TypeInformation.of(txnClass), + TypeInformation.of(new TypeHint>() {})); + } + + public TwoPhaseCommitSinkFunction( + TypeInformation txnTypeInformation, + TypeInformation> txnAndCheckpointTypeInformation) { + this( + new ListStateDescriptor<>("pendingTransactions", txnTypeInformation), + new ListStateDescriptor<>("pendingCommitTransactions", txnAndCheckpointTypeInformation)); + } + + public TwoPhaseCommitSinkFunction( + ListStateDescriptor pendingTransactionsDescriptor, + ListStateDescriptor> 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. + * + *

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> pendingTransactionsIterator = pendingCommitTransactions.iterator(); + checkState(pendingTransactionsIterator.hasNext(), "checkpoint completed, but no transaction pending"); + + List> remainingTransactions = new ArrayList<>(); + + for (TransactionAndCheckpoint 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. ---