Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io 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 815F116647C for ; Tue, 25 Jul 2017 10:02:12 +0200 (CEST) Received: (qmail 54038 invoked by uid 500); 25 Jul 2017 08:02:11 -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 53910 invoked by uid 99); 25 Jul 2017 08:02:11 -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; Tue, 25 Jul 2017 08:02:11 +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 142F8C15E3 for ; Tue, 25 Jul 2017 08:02:11 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -99.2 X-Spam-Level: X-Spam-Status: No, score=-99.2 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RP_MATCHES_RCVD=-0.001, URIBL_BLOCKED=0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id x9p3Btygj3wN for ; Tue, 25 Jul 2017 08:02:05 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id E70D56112E for ; Tue, 25 Jul 2017 08:02:03 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 30799E0E6B for ; Tue, 25 Jul 2017 08:02:03 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 8D18323FE8 for ; Tue, 25 Jul 2017 08:02:01 +0000 (UTC) Date: Tue, 25 Jul 2017 08:02:01 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-7210) Add TwoPhaseCommitSinkFunction (implementing exactly-once semantic in generic way) MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/FLINK-7210?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16099669#comment-16099669 ] ASF GitHub Bot commented on FLINK-7210: --------------------------------------- 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 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( --- End diff -- Could we have Javadocs for these constructors? > Add TwoPhaseCommitSinkFunction (implementing exactly-once semantic in generic way) > ---------------------------------------------------------------------------------- > > Key: FLINK-7210 > URL: https://issues.apache.org/jira/browse/FLINK-7210 > Project: Flink > Issue Type: Improvement > Components: Streaming Connectors > Reporter: Piotr Nowojski > Assignee: Piotr Nowojski > > To implement exactly-once sink there is a re-occurring pattern for doing it - two phase commit algorithm. It is used both in `BucketingSink` and in `Pravega` sink and it will be used in `Kafka 0.11` connector. It would be good to extract this common logic into one class, both to improve existing implementation (for exampe `Pravega`'s sink doesn't abort interrupted transactions) and to make it easier for the users to implement their own custom exactly-once sinks. -- This message was sent by Atlassian JIRA (v6.4.14#64029)