[ https://issues.apache.org/jira/browse/FLINK-7210?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16100204#comment-16100204 ]
ASF GitHub Bot commented on FLINK-7210: --------------------------------------- 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). > 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)