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/06/30 15:55:41 UTC

[GitHub] flink pull request #4239: [FLINK-6988] Initial flink-connector-kafka-0.11 wi...

GitHub user pnowojski opened a pull request:

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

    [FLINK-6988] Initial flink-connector-kafka-0.11 with at-least-once semantic

    Couple of first commits are from other PRs #4206 #4209 #4213 

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

    $ git pull https://github.com/pnowojski/flink kafka011

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

    https://github.com/apache/flink/pull/4239.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 #4239
    
----
commit 5191d5b4b78620cfc5ecfc9088afba0d611eaacb
Author: Piotr Nowojski <pi...@gmail.com>
Date:   2017-06-26T09:28:51Z

    [FLINK-6996] Refactor and automaticall inherit KafkaProducer integration tests

commit 1c7d349ce425ec0213059e062f10c90773cc780d
Author: Piotr Nowojski <pi...@gmail.com>
Date:   2017-06-26T10:20:36Z

    [FLINK-6996] Fix formatting in KafkaConsumerTestBase and KafkaProducerTestBase

commit 5b849f98191439e69ca2357a4767f47957ee0250
Author: Piotr Nowojski <pi...@gmail.com>
Date:   2017-06-23T11:41:55Z

    [FLINK-7030] Build with scala-2.11 by default

commit 3f62aecb57cea9d43611ecfa24e2233a63197341
Author: Piotr Nowojski <pi...@gmail.com>
Date:   2017-06-26T10:36:40Z

    [FLINK-6996] Fix at-least-once semantic for FlinkKafkaProducer010
    
    Add tests coverage for Kafka 0.10 and 0.9

commit 4b78626df474a8d49a406714a7142ad44d8a8faf
Author: Piotr Nowojski <pi...@gmail.com>
Date:   2017-06-28T18:30:08Z

    [FLINK-7032] Overwrite inherited values of compiler version from parent pom
    
    Default values were 1.6 and were causing Intellij to constantly switch language
    level to 1.6, which in turn was causing compilation errors. It worked fine
    for compiling from console using  maven, because those values are separetly set
    in maven-compiler-plugin configuration.

commit 2c2556e72dd73c5e470e5afd6dab4a11cb41772d
Author: Piotr Nowojski <pi...@gmail.com>
Date:   2017-06-23T07:14:28Z

    [FLINK-6988] Initial flink-connector-kafka-0.11 with at-least-once semantic
    
    Code of 0.11 connector is based on 0.10 version

----


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    @aljoscha I have addressed you "high level" comments and fixed some bugs. Please latest 5 commits (one of them is a new dependency on another PR: https://github.com/apache/flink/pull/4631 )


---

[GitHub] flink pull request #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134399119
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaProducer.java ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.connectors.kafka.internal;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +
    +import org.apache.kafka.clients.consumer.OffsetAndMetadata;
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.KafkaProducer;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.clients.producer.internals.TransactionalRequestResult;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.Node;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.errors.ProducerFencedException;
    +import org.apache.kafka.common.requests.FindCoordinatorRequest;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.lang.reflect.Field;
    +import java.lang.reflect.InvocationTargetException;
    +import java.lang.reflect.Method;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Wrapper around KafkaProducer that allows to resume transactions in case of node failure, which allows to implement
    + * two phase commit algorithm for exactly-once semantic FlinkKafkaProducer.
    + *
    + * <p>For happy path usage is exactly the same as {@link org.apache.kafka.clients.producer.KafkaProducer}. User is
    + * expected to call:
    + *
    + * <ul>
    + *     <li>{@link FlinkKafkaProducer#initTransactions()}</li>
    + *     <li>{@link FlinkKafkaProducer#beginTransaction()}</li>
    + *     <li>{@link FlinkKafkaProducer#send(org.apache.kafka.clients.producer.ProducerRecord)}</li>
    + *     <li>{@link FlinkKafkaProducer#flush()}</li>
    + *     <li>{@link FlinkKafkaProducer#commitTransaction()}</li>
    + * </ul>
    + *
    + * <p>To actually implement two phase commit, it must be possible to always commit a transaction after pre-committing
    + * it (here, pre-commit is just a {@link FlinkKafkaProducer#flush()}). In case of some failure between
    + * {@link FlinkKafkaProducer#flush()} and {@link FlinkKafkaProducer#commitTransaction()} this class allows to resume
    + * interrupted transaction and commit if after a restart:
    + *
    + * <ul>
    + *     <li>{@link FlinkKafkaProducer#initTransactions()}</li>
    + *     <li>{@link FlinkKafkaProducer#beginTransaction()}</li>
    + *     <li>{@link FlinkKafkaProducer#send(org.apache.kafka.clients.producer.ProducerRecord)}</li>
    + *     <li>{@link FlinkKafkaProducer#flush()}</li>
    + *     <li>{@link FlinkKafkaProducer#getProducerId()}</li>
    + *     <li>{@link FlinkKafkaProducer#getEpoch()}</li>
    + *     <li>node failure... restore producerId and epoch from state</li>
    + *     <li>{@link FlinkKafkaProducer#resumeTransaction(long, short)}</li>
    + *     <li>{@link FlinkKafkaProducer#commitTransaction()}</li>
    + * </ul>
    + *
    + * <p>{@link FlinkKafkaProducer#resumeTransaction(long, short)} replaces {@link FlinkKafkaProducer#initTransactions()}
    + * as a way to obtain the producerId and epoch counters. It has to be done, because otherwise
    + * {@link FlinkKafkaProducer#initTransactions()} would automatically abort all on going transactions.
    + *
    + * <p>Second way this implementation differs from the reference {@link org.apache.kafka.clients.producer.KafkaProducer}
    + * is that this one actually flushes new partitions on {@link FlinkKafkaProducer#flush()} instead of on
    + * {@link FlinkKafkaProducer#commitTransaction()}.
    + *
    + * <p>The last one minor difference is that it allows to obtain the producerId and epoch counters via
    + * {@link FlinkKafkaProducer#getProducerId()} and {@link FlinkKafkaProducer#getEpoch()} methods (which are unfortunately
    + * private fields).
    + *
    + * <p>Those changes are compatible with Kafka's 0.11.0 REST API although it clearly was not the intention of the Kafka's
    + * API authors to make them possible.
    + *
    + * <p>Internally this implementation uses {@link org.apache.kafka.clients.producer.KafkaProducer} and implements
    + * required changes via Java Reflection API. It might not be the prettiest solution. An alternative would be to
    + * re-implement whole Kafka's 0.11 REST API client on our own.
    + */
    +public class FlinkKafkaProducer<K, V> implements Producer<K, V> {
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducer.class);
    +
    +	private final KafkaProducer<K, V> kafkaProducer;
    +	@Nullable
    +	private final String transactionalId;
    +
    +	public FlinkKafkaProducer(Properties properties) {
    +		transactionalId = properties.getProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG);
    +		kafkaProducer = new KafkaProducer<>(properties);
    +	}
    +
    +	// -------------------------------- Simple proxy method calls --------------------------------
    +
    +	@Override
    +	public void initTransactions() {
    +		kafkaProducer.initTransactions();
    +	}
    +
    +	@Override
    +	public void beginTransaction() throws ProducerFencedException {
    +		kafkaProducer.beginTransaction();
    +	}
    +
    +	@Override
    +	public void commitTransaction() throws ProducerFencedException {
    +		kafkaProducer.commitTransaction();
    +	}
    +
    +	@Override
    +	public void abortTransaction() throws ProducerFencedException {
    +		kafkaProducer.abortTransaction();
    +	}
    +
    +	@Override
    +	public void sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata> offsets, String consumerGroupId) throws ProducerFencedException {
    +		kafkaProducer.sendOffsetsToTransaction(offsets, consumerGroupId);
    +	}
    +
    +	@Override
    +	public Future<RecordMetadata> send(ProducerRecord<K, V> record) {
    +		return kafkaProducer.send(record);
    +	}
    +
    +	@Override
    +	public Future<RecordMetadata> send(ProducerRecord<K, V> record, Callback callback) {
    +		return kafkaProducer.send(record, callback);
    +	}
    +
    +	@Override
    +	public List<PartitionInfo> partitionsFor(String topic) {
    +		return kafkaProducer.partitionsFor(topic);
    +	}
    +
    +	@Override
    +	public Map<MetricName, ? extends Metric> metrics() {
    +		return kafkaProducer.metrics();
    +	}
    +
    +	@Override
    +	public void close() {
    +		kafkaProducer.close();
    +	}
    +
    +	@Override
    +	public void close(long timeout, TimeUnit unit) {
    +		kafkaProducer.close(timeout, unit);
    +	}
    +
    +	// -------------------------------- New methods or methods with changed behaviour --------------------------------
    +
    +	@Override
    +	public void flush() {
    +		kafkaProducer.flush();
    +		if (transactionalId != null) {
    +			flushNewPartitions();
    +		}
    +	}
    +
    +	public void resumeTransaction(long producerId, short epoch) {
    +		if (!(producerId >= 0 && epoch >= 0)) {
    +			throw new IllegalStateException(String.format("Incorrect values for producerId [%s] and epoch [%s]", producerId, epoch));
    +		}
    +		LOG.info("Attempting to resume transaction with producerId [%s] and epoch [%s]", producerId, epoch);
    --- End diff --
    
    `{}` instead of `[%s]`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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    Writing records in state would be very costly. It is only a "last resort" solution.
    
    > That would imply exactly-once consumers can not read past that transaction as long as it is open
    
    Hmmm, are you sure about this thing? That would mean that Kafka doesn't support transactional parallel writes from two different process, which would be very strange. Could you point to a source of this information? 
    
    Resuming transactions is not a part of `KafkaProducer`'s API, however Kafka's REST API allows to do that. However I'm aware that it wasn't an intention of the authors to do so. Kafka Streams do not need to do that, because they achieve exactly-once semantic by using persistent communication channels (Kafka topics), so they can easily restart each operator on it's own by replay/rewinding every input channel (Kafka topic). This comes with a cost, because it makes communication between operators extremely, since every message must goes to HDDs at some point. 


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    May be an extra shuffle to make small batches could help. Another option is to buffer all the records in state and write them all inside commit(). But not sure how costly it is to save all the records in checkpointed state. 
    
    Another issue I see with using random txn id : if a worker looks unresponsive and work is moved to another worker, it is possible that the old worker still lingers around with open transaction. That would imply it the exactly-once consumers can not read past that transaction as long as it is open.
    
    I didn't know it was possible to resume a transaction since it was not part of producer API. This PR uses an undocumented way to do it.. do you know if Kafka Streams also does something like that? May be the producer will support `resumeTransaction()` properly in future.



---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    I did a first high-level review of the code. I think it's good so far!
    
    Before we can merge this, however, we need a few more things around it:
     - A section in the Kafka doc about the new exactly-once mode, how it can be configured etc.
     - A big disclaimer (possibly in an "alert" box) about the interplay with the transaction timeout and what the caveats there are
     - A section in the Javadocs about the aforementioned caveats
     - A check that ensures that the transaction timeout is set to a reasonably high setting (say 1 hour) when exactly-once semantics are enabled. (With an override setting that allows the user to set a lower transaction time out if they want to.)
    
    Also, this has interplay with #4616 but we can resolve that by merging them in any order and fixing up the later changes when 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 issue #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    Now that the prerequisite PRs are merged, we can rebase this now :)


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134396704
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    +		ClosureCleaner.clean(customPartitioner, true);
    +		ClosureCleaner.ensureSerializable(serializationSchema);
    +
    +		this.defaultTopicId = defaultTopicId;
    +		this.schema = serializationSchema;
    +		this.producerConfig = producerConfig;
    +		this.flinkKafkaPartitioner = customPartitioner;
    +		this.semantic = semantic;
    +		this.kafkaProducersPoolSize = kafkaProducersPoolSize;
    +
    +		// set the producer configuration properties for kafka record key value serializers.
    +		if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		// eagerly ensure that bootstrap servers are set.
    +		if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
    +			throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must be supplied in the producer config properties.");
    +		}
    +
    +		this.topicPartitionsMap = new HashMap<>();
    +	}
    +
    +	// ---------------------------------- Properties --------------------------
    +
    +	/**
    +	 * Defines whether the producer should fail on errors, or only log them.
    +	 * If this is set to true, then exceptions will be only logged, if set to false,
    +	 * exceptions will be eventually thrown and cause the streaming program to
    +	 * fail (and enter recovery).
    +	 *
    +	 * <p>Method is only accessible for approach (a) (see above)
    +	 *
    +	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
    +	 */
    +	public void setLogFailuresOnly(boolean logFailuresOnly) {
    +		this.logFailuresOnly = logFailuresOnly;
    +	}
    +
    +	// ----------------------------------- Utilities --------------------------
    +
    +	/**
    +	 * Initializes the connection to Kafka.
    +	 *
    +	 * <p>This method is used for approach (a) (see above).
    +	 */
    +	@Override
    +	public void open(Configuration configuration) throws Exception {
    +		if (semantic != Semantic.NONE && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) {
    +			LOG.warn(String.format("Using [%s] semantic, but checkpointing is not enabled. Switching to [%s] semantic.", semantic, Semantic.NONE));
    +			semantic = Semantic.NONE;
    +		}
    +
    +		if (logFailuresOnly) {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception e) {
    +					if (e != null) {
    +						LOG.error("Error while sending record to Kafka: " + e.getMessage(), e);
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +		else {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception exception) {
    +					if (exception != null && asyncException == null) {
    +						asyncException = exception;
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +
    +		super.open(configuration);
    +	}
    +
    +	@Override
    +	public void invoke(KafkaTransactionState transaction, IN next) throws Exception {
    +		invokeInternal(transaction, next, Long.MAX_VALUE);
    +	}
    +
    +	private void invokeInternal(KafkaTransactionState transaction, IN next, long elementTimestamp) throws Exception {
    +		checkErroneous();
    +
    +		byte[] serializedKey = schema.serializeKey(next);
    +		byte[] serializedValue = schema.serializeValue(next);
    +		String targetTopic = schema.getTargetTopic(next);
    +		if (targetTopic == null) {
    +			targetTopic = defaultTopicId;
    +		}
    +
    +		Long timestamp = null;
    +		if (this.writeTimestampToKafka) {
    +			timestamp = elementTimestamp;
    +		}
    +
    +		ProducerRecord<byte[], byte[]> record;
    +		int[] partitions = topicPartitionsMap.get(targetTopic);
    +		if (null == partitions) {
    +			partitions = getPartitionsByTopic(targetTopic, transaction.producer);
    +			topicPartitionsMap.put(targetTopic, partitions);
    +		}
    +		if (flinkKafkaPartitioner == null) {
    +			record = new ProducerRecord<>(targetTopic, null, timestamp, serializedKey, serializedValue);
    +		} else {
    +			record = new ProducerRecord<>(targetTopic, flinkKafkaPartitioner.partition(next, serializedKey, serializedValue, targetTopic, partitions), timestamp, serializedKey, serializedValue);
    +		}
    +		pendingRecords.incrementAndGet();
    +		transaction.producer.send(record, callback);
    +	}
    +
    +	@Override
    +	public void close() throws Exception {
    +		if (currentTransaction != null) {
    +			// to avoid exceptions on aborting transactions with some pending records
    +			flush(currentTransaction);
    +		}
    +		try {
    +			super.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		try {
    +			producersPool.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		// make sure we propagate pending errors
    +		checkErroneous();
    +	}
    +
    +	// ------------------- Logic for handling checkpoint flushing -------------------------- //
    +
    +	@Override
    +	protected KafkaTransactionState beginTransaction() throws Exception {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				FlinkKafkaProducer<byte[], byte[]> producer = producersPool.poll();
    +				if (producer == null) {
    +					String transactionalId = availableTransactionalIds.poll();
    +					if (transactionalId == null) {
    +						throw new Exception(
    +							"Too many ongoing snapshots. Increase kafka producers pool size or decrease number of concurrent checktpoins.");
    +					}
    +					usedTransactionalIds.add(transactionalId);
    +					producer = initTransactionalProducer(transactionalId, true);
    +					producer.initTransactions();
    +				}
    +				producer.beginTransaction();
    +				return new KafkaTransactionState(producer.getTransactionalId(), producer);
    +			case AT_LEAST_ONCE:
    +			case NONE:
    --- End diff --
    
    I'm a bit curious why `NONE` and `AT_LEAST_ONCE` is treated in the same manner here.
    Shouldn't we don't even need any state / transactions in `NONE` mode?


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r128164162
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,818 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.UUID;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic.EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular sink function (a)
    + * and a custom operator (b).
    + *
    + * <p>For (a), the class implements the SinkFunction and RichFunction interfaces.
    + * For (b), it extends the StreamTask class.
    + *
    + * <p>Details about approach (a):
    + *  Pre Kafka 0.11 producers only follow approach (a), allowing users to use the producer using the
    + *  DataStream.addSink() method.
    + *  Since the APIs exposed in that variant do not allow accessing the the timestamp attached to the record
    + *  the Kafka 0.11 producer has a second invocation option, approach (b).
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When adding the
    + *  FlinkKafkaProducer011 using the FlinkKafkaProducer011.writeToKafkaWithTimestamps() method, the Kafka producer
    + *  can access the internal record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	private Semantic semantic;
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    --- End diff --
    
    nit: also include Javadoc for consistency.



---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134395283
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    --- End diff --
    
    We can probably make this `transient` also for documentation purposes.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    @aljoscha rebased on latest master and integrated your changes


---

[GitHub] flink pull request #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134725699
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    +		ClosureCleaner.clean(customPartitioner, true);
    +		ClosureCleaner.ensureSerializable(serializationSchema);
    +
    +		this.defaultTopicId = defaultTopicId;
    +		this.schema = serializationSchema;
    +		this.producerConfig = producerConfig;
    +		this.flinkKafkaPartitioner = customPartitioner;
    +		this.semantic = semantic;
    +		this.kafkaProducersPoolSize = kafkaProducersPoolSize;
    +
    +		// set the producer configuration properties for kafka record key value serializers.
    +		if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		// eagerly ensure that bootstrap servers are set.
    +		if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
    +			throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must be supplied in the producer config properties.");
    +		}
    +
    +		this.topicPartitionsMap = new HashMap<>();
    +	}
    +
    +	// ---------------------------------- Properties --------------------------
    +
    +	/**
    +	 * Defines whether the producer should fail on errors, or only log them.
    +	 * If this is set to true, then exceptions will be only logged, if set to false,
    +	 * exceptions will be eventually thrown and cause the streaming program to
    +	 * fail (and enter recovery).
    +	 *
    +	 * <p>Method is only accessible for approach (a) (see above)
    +	 *
    +	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
    +	 */
    +	public void setLogFailuresOnly(boolean logFailuresOnly) {
    +		this.logFailuresOnly = logFailuresOnly;
    +	}
    +
    +	// ----------------------------------- Utilities --------------------------
    +
    +	/**
    +	 * Initializes the connection to Kafka.
    +	 *
    +	 * <p>This method is used for approach (a) (see above).
    +	 */
    +	@Override
    +	public void open(Configuration configuration) throws Exception {
    +		if (semantic != Semantic.NONE && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) {
    +			LOG.warn(String.format("Using [%s] semantic, but checkpointing is not enabled. Switching to [%s] semantic.", semantic, Semantic.NONE));
    +			semantic = Semantic.NONE;
    +		}
    +
    +		if (logFailuresOnly) {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception e) {
    +					if (e != null) {
    +						LOG.error("Error while sending record to Kafka: " + e.getMessage(), e);
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +		else {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception exception) {
    +					if (exception != null && asyncException == null) {
    +						asyncException = exception;
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +
    +		super.open(configuration);
    +	}
    +
    +	@Override
    +	public void invoke(KafkaTransactionState transaction, IN next) throws Exception {
    +		invokeInternal(transaction, next, Long.MAX_VALUE);
    +	}
    +
    +	private void invokeInternal(KafkaTransactionState transaction, IN next, long elementTimestamp) throws Exception {
    +		checkErroneous();
    +
    +		byte[] serializedKey = schema.serializeKey(next);
    +		byte[] serializedValue = schema.serializeValue(next);
    +		String targetTopic = schema.getTargetTopic(next);
    +		if (targetTopic == null) {
    +			targetTopic = defaultTopicId;
    +		}
    +
    +		Long timestamp = null;
    +		if (this.writeTimestampToKafka) {
    +			timestamp = elementTimestamp;
    +		}
    +
    +		ProducerRecord<byte[], byte[]> record;
    +		int[] partitions = topicPartitionsMap.get(targetTopic);
    +		if (null == partitions) {
    +			partitions = getPartitionsByTopic(targetTopic, transaction.producer);
    +			topicPartitionsMap.put(targetTopic, partitions);
    +		}
    +		if (flinkKafkaPartitioner == null) {
    +			record = new ProducerRecord<>(targetTopic, null, timestamp, serializedKey, serializedValue);
    +		} else {
    +			record = new ProducerRecord<>(targetTopic, flinkKafkaPartitioner.partition(next, serializedKey, serializedValue, targetTopic, partitions), timestamp, serializedKey, serializedValue);
    +		}
    +		pendingRecords.incrementAndGet();
    +		transaction.producer.send(record, callback);
    +	}
    +
    +	@Override
    +	public void close() throws Exception {
    +		if (currentTransaction != null) {
    +			// to avoid exceptions on aborting transactions with some pending records
    +			flush(currentTransaction);
    +		}
    +		try {
    +			super.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		try {
    +			producersPool.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		// make sure we propagate pending errors
    +		checkErroneous();
    +	}
    +
    +	// ------------------- Logic for handling checkpoint flushing -------------------------- //
    +
    +	@Override
    +	protected KafkaTransactionState beginTransaction() throws Exception {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				FlinkKafkaProducer<byte[], byte[]> producer = producersPool.poll();
    +				if (producer == null) {
    +					String transactionalId = availableTransactionalIds.poll();
    +					if (transactionalId == null) {
    +						throw new Exception(
    +							"Too many ongoing snapshots. Increase kafka producers pool size or decrease number of concurrent checktpoins.");
    +					}
    +					usedTransactionalIds.add(transactionalId);
    +					producer = initTransactionalProducer(transactionalId, true);
    +					producer.initTransactions();
    +				}
    +				producer.beginTransaction();
    +				return new KafkaTransactionState(producer.getTransactionalId(), producer);
    +			case AT_LEAST_ONCE:
    +			case NONE:
    --- End diff --
    
    Because `NONE` and `AT_LEAST_ONCE` do not need to start new transaction, the `beginTransaction()` implementation for both them is blank. Those two differ only in `preCommit()` where the later one flushes the 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 pull request #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134398783
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    +		ClosureCleaner.clean(customPartitioner, true);
    +		ClosureCleaner.ensureSerializable(serializationSchema);
    +
    +		this.defaultTopicId = defaultTopicId;
    +		this.schema = serializationSchema;
    +		this.producerConfig = producerConfig;
    +		this.flinkKafkaPartitioner = customPartitioner;
    +		this.semantic = semantic;
    +		this.kafkaProducersPoolSize = kafkaProducersPoolSize;
    +
    +		// set the producer configuration properties for kafka record key value serializers.
    +		if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		// eagerly ensure that bootstrap servers are set.
    +		if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
    +			throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must be supplied in the producer config properties.");
    +		}
    +
    +		this.topicPartitionsMap = new HashMap<>();
    +	}
    +
    +	// ---------------------------------- Properties --------------------------
    +
    +	/**
    +	 * Defines whether the producer should fail on errors, or only log them.
    +	 * If this is set to true, then exceptions will be only logged, if set to false,
    +	 * exceptions will be eventually thrown and cause the streaming program to
    +	 * fail (and enter recovery).
    +	 *
    +	 * <p>Method is only accessible for approach (a) (see above)
    +	 *
    +	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
    +	 */
    +	public void setLogFailuresOnly(boolean logFailuresOnly) {
    +		this.logFailuresOnly = logFailuresOnly;
    +	}
    +
    +	// ----------------------------------- Utilities --------------------------
    +
    +	/**
    +	 * Initializes the connection to Kafka.
    +	 *
    +	 * <p>This method is used for approach (a) (see above).
    +	 */
    +	@Override
    +	public void open(Configuration configuration) throws Exception {
    +		if (semantic != Semantic.NONE && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) {
    +			LOG.warn(String.format("Using [%s] semantic, but checkpointing is not enabled. Switching to [%s] semantic.", semantic, Semantic.NONE));
    +			semantic = Semantic.NONE;
    +		}
    +
    +		if (logFailuresOnly) {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception e) {
    +					if (e != null) {
    +						LOG.error("Error while sending record to Kafka: " + e.getMessage(), e);
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +		else {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception exception) {
    +					if (exception != null && asyncException == null) {
    +						asyncException = exception;
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +
    +		super.open(configuration);
    +	}
    +
    +	@Override
    +	public void invoke(KafkaTransactionState transaction, IN next) throws Exception {
    +		invokeInternal(transaction, next, Long.MAX_VALUE);
    +	}
    +
    +	private void invokeInternal(KafkaTransactionState transaction, IN next, long elementTimestamp) throws Exception {
    +		checkErroneous();
    +
    +		byte[] serializedKey = schema.serializeKey(next);
    +		byte[] serializedValue = schema.serializeValue(next);
    +		String targetTopic = schema.getTargetTopic(next);
    +		if (targetTopic == null) {
    +			targetTopic = defaultTopicId;
    +		}
    +
    +		Long timestamp = null;
    +		if (this.writeTimestampToKafka) {
    +			timestamp = elementTimestamp;
    +		}
    +
    +		ProducerRecord<byte[], byte[]> record;
    +		int[] partitions = topicPartitionsMap.get(targetTopic);
    +		if (null == partitions) {
    +			partitions = getPartitionsByTopic(targetTopic, transaction.producer);
    +			topicPartitionsMap.put(targetTopic, partitions);
    +		}
    +		if (flinkKafkaPartitioner == null) {
    +			record = new ProducerRecord<>(targetTopic, null, timestamp, serializedKey, serializedValue);
    +		} else {
    +			record = new ProducerRecord<>(targetTopic, flinkKafkaPartitioner.partition(next, serializedKey, serializedValue, targetTopic, partitions), timestamp, serializedKey, serializedValue);
    +		}
    +		pendingRecords.incrementAndGet();
    +		transaction.producer.send(record, callback);
    +	}
    +
    +	@Override
    +	public void close() throws Exception {
    +		if (currentTransaction != null) {
    +			// to avoid exceptions on aborting transactions with some pending records
    +			flush(currentTransaction);
    +		}
    +		try {
    +			super.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		try {
    +			producersPool.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		// make sure we propagate pending errors
    +		checkErroneous();
    +	}
    +
    +	// ------------------- Logic for handling checkpoint flushing -------------------------- //
    +
    +	@Override
    +	protected KafkaTransactionState beginTransaction() throws Exception {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				FlinkKafkaProducer<byte[], byte[]> producer = producersPool.poll();
    +				if (producer == null) {
    +					String transactionalId = availableTransactionalIds.poll();
    +					if (transactionalId == null) {
    +						throw new Exception(
    +							"Too many ongoing snapshots. Increase kafka producers pool size or decrease number of concurrent checktpoins.");
    +					}
    +					usedTransactionalIds.add(transactionalId);
    +					producer = initTransactionalProducer(transactionalId, true);
    +					producer.initTransactions();
    +				}
    +				producer.beginTransaction();
    +				return new KafkaTransactionState(producer.getTransactionalId(), producer);
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				// Do not create new producer on each beginTransaction() if it is not necessary
    +				if (currentTransaction != null && currentTransaction.producer != null) {
    +					return new KafkaTransactionState(currentTransaction.producer);
    +				}
    +				return new KafkaTransactionState(initProducer(true));
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void preCommit(KafkaTransactionState transaction) throws Exception {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +			case AT_LEAST_ONCE:
    +				flush(transaction);
    +				break;
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +		checkErroneous();
    +	}
    +
    +	@Override
    +	protected void commit(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				transaction.producer.commitTransaction();
    +				producersPool.add(transaction.producer);
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void recoverAndCommit(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				KafkaTransactionState kafkaTransaction = transaction;
    +				FlinkKafkaProducer<byte[], byte[]> producer =
    +					initTransactionalProducer(kafkaTransaction.transactionalId, false);
    +				producer.resumeTransaction(kafkaTransaction.producerId, kafkaTransaction.epoch);
    +				try {
    +					producer.commitTransaction();
    +					producer.close();
    +				}
    +				catch (InvalidTxnStateException ex) {
    +					// That means we have committed this transaction before.
    +					LOG.warn("Encountered error [%s] while recovering transaction [%s]. " +
    +						"Presumably this transaction has been already committed before",
    +						ex,
    +						transaction);
    +				}
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void abort(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				transaction.producer.abortTransaction();
    +				producersPool.add(transaction.producer);
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				producersPool.add(transaction.producer);
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void recoverAndAbort(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				FlinkKafkaProducer<byte[], byte[]> producer =
    +					initTransactionalProducer(transaction.transactionalId, false);
    +				producer.resumeTransaction(transaction.producerId, transaction.epoch);
    +				producer.abortTransaction();
    +				producer.close();
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	private void acknowledgeMessage() {
    +		pendingRecords.decrementAndGet();
    +	}
    +
    +	/**
    +	 * Flush pending records.
    +	 * @param transaction
    +	 */
    +	private void flush(KafkaTransactionState transaction) throws Exception {
    +		if (transaction.producer != null) {
    +			transaction.producer.flush();
    +		}
    +		long pendingRecordsCount = pendingRecords.get();
    +		if (pendingRecordsCount != 0) {
    +			throw new IllegalStateException("Pending record count must be zero at this point: " + pendingRecordsCount);
    +		}
    +
    +		// if the flushed requests has errors, we should propagate it also and fail the checkpoint
    +		checkErroneous();
    +	}
    +
    +	@Override
    +	public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +		super.snapshotState(context);
    +
    +		transactionalIdsState.clear();
    +		for (String transactionalId : availableTransactionalIds) {
    +			transactionalIdsState.add(transactionalId);
    +		}
    +		for (String transactionalId : usedTransactionalIds) {
    +			transactionalIdsState.add(transactionalId);
    +		}
    +	}
    +
    +	@Override
    +	public void initializeState(FunctionInitializationContext context) throws Exception {
    +		availableTransactionalIds.clear();
    +		for (int i = 0; i < kafkaProducersPoolSize; i++) {
    +			availableTransactionalIds.add(UUID.randomUUID().toString());
    +		}
    +
    +		super.initializeState(context);
    --- End diff --
    
    Could we initialize the base `TwoPhaseCommitSink` first?


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    This solution (basically a pool with a fixed size of 2) would work, only if there would be at most one pending commit transaction. Which is not always true in Flink - there can be multiple triggered checkpoints pending completion.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    We are really looking forward to this 👍 


---

[GitHub] flink issue #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    Bugs in tests (those that you can see in fixup commits)


---

[GitHub] flink pull request #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134726007
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    +		ClosureCleaner.clean(customPartitioner, true);
    +		ClosureCleaner.ensureSerializable(serializationSchema);
    +
    +		this.defaultTopicId = defaultTopicId;
    +		this.schema = serializationSchema;
    +		this.producerConfig = producerConfig;
    +		this.flinkKafkaPartitioner = customPartitioner;
    +		this.semantic = semantic;
    +		this.kafkaProducersPoolSize = kafkaProducersPoolSize;
    +
    +		// set the producer configuration properties for kafka record key value serializers.
    +		if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		// eagerly ensure that bootstrap servers are set.
    +		if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
    +			throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must be supplied in the producer config properties.");
    +		}
    +
    +		this.topicPartitionsMap = new HashMap<>();
    +	}
    +
    +	// ---------------------------------- Properties --------------------------
    +
    +	/**
    +	 * Defines whether the producer should fail on errors, or only log them.
    +	 * If this is set to true, then exceptions will be only logged, if set to false,
    +	 * exceptions will be eventually thrown and cause the streaming program to
    +	 * fail (and enter recovery).
    +	 *
    +	 * <p>Method is only accessible for approach (a) (see above)
    +	 *
    +	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
    +	 */
    +	public void setLogFailuresOnly(boolean logFailuresOnly) {
    +		this.logFailuresOnly = logFailuresOnly;
    +	}
    +
    +	// ----------------------------------- Utilities --------------------------
    +
    +	/**
    +	 * Initializes the connection to Kafka.
    +	 *
    +	 * <p>This method is used for approach (a) (see above).
    +	 */
    +	@Override
    +	public void open(Configuration configuration) throws Exception {
    +		if (semantic != Semantic.NONE && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) {
    +			LOG.warn(String.format("Using [%s] semantic, but checkpointing is not enabled. Switching to [%s] semantic.", semantic, Semantic.NONE));
    +			semantic = Semantic.NONE;
    +		}
    +
    +		if (logFailuresOnly) {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception e) {
    +					if (e != null) {
    +						LOG.error("Error while sending record to Kafka: " + e.getMessage(), e);
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +		else {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception exception) {
    +					if (exception != null && asyncException == null) {
    +						asyncException = exception;
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +
    +		super.open(configuration);
    +	}
    +
    +	@Override
    +	public void invoke(KafkaTransactionState transaction, IN next) throws Exception {
    +		invokeInternal(transaction, next, Long.MAX_VALUE);
    +	}
    +
    +	private void invokeInternal(KafkaTransactionState transaction, IN next, long elementTimestamp) throws Exception {
    +		checkErroneous();
    +
    +		byte[] serializedKey = schema.serializeKey(next);
    +		byte[] serializedValue = schema.serializeValue(next);
    +		String targetTopic = schema.getTargetTopic(next);
    +		if (targetTopic == null) {
    +			targetTopic = defaultTopicId;
    +		}
    +
    +		Long timestamp = null;
    +		if (this.writeTimestampToKafka) {
    +			timestamp = elementTimestamp;
    +		}
    +
    +		ProducerRecord<byte[], byte[]> record;
    +		int[] partitions = topicPartitionsMap.get(targetTopic);
    +		if (null == partitions) {
    +			partitions = getPartitionsByTopic(targetTopic, transaction.producer);
    +			topicPartitionsMap.put(targetTopic, partitions);
    +		}
    +		if (flinkKafkaPartitioner == null) {
    +			record = new ProducerRecord<>(targetTopic, null, timestamp, serializedKey, serializedValue);
    +		} else {
    +			record = new ProducerRecord<>(targetTopic, flinkKafkaPartitioner.partition(next, serializedKey, serializedValue, targetTopic, partitions), timestamp, serializedKey, serializedValue);
    +		}
    +		pendingRecords.incrementAndGet();
    +		transaction.producer.send(record, callback);
    +	}
    +
    +	@Override
    +	public void close() throws Exception {
    +		if (currentTransaction != null) {
    +			// to avoid exceptions on aborting transactions with some pending records
    +			flush(currentTransaction);
    +		}
    +		try {
    +			super.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		try {
    +			producersPool.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		// make sure we propagate pending errors
    +		checkErroneous();
    +	}
    +
    +	// ------------------- Logic for handling checkpoint flushing -------------------------- //
    +
    +	@Override
    +	protected KafkaTransactionState beginTransaction() throws Exception {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				FlinkKafkaProducer<byte[], byte[]> producer = producersPool.poll();
    +				if (producer == null) {
    +					String transactionalId = availableTransactionalIds.poll();
    +					if (transactionalId == null) {
    +						throw new Exception(
    +							"Too many ongoing snapshots. Increase kafka producers pool size or decrease number of concurrent checktpoins.");
    +					}
    +					usedTransactionalIds.add(transactionalId);
    +					producer = initTransactionalProducer(transactionalId, true);
    +					producer.initTransactions();
    +				}
    +				producer.beginTransaction();
    +				return new KafkaTransactionState(producer.getTransactionalId(), producer);
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				// Do not create new producer on each beginTransaction() if it is not necessary
    +				if (currentTransaction != null && currentTransaction.producer != null) {
    +					return new KafkaTransactionState(currentTransaction.producer);
    +				}
    +				return new KafkaTransactionState(initProducer(true));
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void preCommit(KafkaTransactionState transaction) throws Exception {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +			case AT_LEAST_ONCE:
    +				flush(transaction);
    +				break;
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +		checkErroneous();
    +	}
    +
    +	@Override
    +	protected void commit(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				transaction.producer.commitTransaction();
    +				producersPool.add(transaction.producer);
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void recoverAndCommit(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				KafkaTransactionState kafkaTransaction = transaction;
    +				FlinkKafkaProducer<byte[], byte[]> producer =
    +					initTransactionalProducer(kafkaTransaction.transactionalId, false);
    +				producer.resumeTransaction(kafkaTransaction.producerId, kafkaTransaction.epoch);
    +				try {
    +					producer.commitTransaction();
    +					producer.close();
    +				}
    +				catch (InvalidTxnStateException ex) {
    +					// That means we have committed this transaction before.
    +					LOG.warn("Encountered error [%s] while recovering transaction [%s]. " +
    +						"Presumably this transaction has been already committed before",
    +						ex,
    +						transaction);
    +				}
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void abort(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				transaction.producer.abortTransaction();
    +				producersPool.add(transaction.producer);
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				producersPool.add(transaction.producer);
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void recoverAndAbort(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				FlinkKafkaProducer<byte[], byte[]> producer =
    +					initTransactionalProducer(transaction.transactionalId, false);
    +				producer.resumeTransaction(transaction.producerId, transaction.epoch);
    +				producer.abortTransaction();
    +				producer.close();
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	private void acknowledgeMessage() {
    +		pendingRecords.decrementAndGet();
    +	}
    +
    +	/**
    +	 * Flush pending records.
    +	 * @param transaction
    +	 */
    +	private void flush(KafkaTransactionState transaction) throws Exception {
    +		if (transaction.producer != null) {
    +			transaction.producer.flush();
    +		}
    +		long pendingRecordsCount = pendingRecords.get();
    +		if (pendingRecordsCount != 0) {
    +			throw new IllegalStateException("Pending record count must be zero at this point: " + pendingRecordsCount);
    +		}
    +
    +		// if the flushed requests has errors, we should propagate it also and fail the checkpoint
    +		checkErroneous();
    +	}
    +
    +	@Override
    +	public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +		super.snapshotState(context);
    +
    +		transactionalIdsState.clear();
    +		for (String transactionalId : availableTransactionalIds) {
    +			transactionalIdsState.add(transactionalId);
    +		}
    +		for (String transactionalId : usedTransactionalIds) {
    +			transactionalIdsState.add(transactionalId);
    +		}
    +	}
    +
    +	@Override
    +	public void initializeState(FunctionInitializationContext context) throws Exception {
    +		availableTransactionalIds.clear();
    +		for (int i = 0; i < kafkaProducersPoolSize; i++) {
    +			availableTransactionalIds.add(UUID.randomUUID().toString());
    +		}
    +
    +		super.initializeState(context);
    --- End diff --
    
    nope :(


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134394869
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    --- End diff --
    
    Could you briefly describe the reason of the number 5?
    Why not use `numConcurrentCheckpoints + 1` (as we discussed offline)?


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    Ok :) I can agree that we keep 321a142 a separate commit.
    For df6d5e0 to 5ff8106, I actually found it easier to ignore all that (because a lot of it is irrelevant in the end) and went straight to 41ad973.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134399048
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaProducer.java ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.connectors.kafka.internal;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +
    +import org.apache.kafka.clients.consumer.OffsetAndMetadata;
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.KafkaProducer;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.clients.producer.internals.TransactionalRequestResult;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.Node;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.errors.ProducerFencedException;
    +import org.apache.kafka.common.requests.FindCoordinatorRequest;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.lang.reflect.Field;
    +import java.lang.reflect.InvocationTargetException;
    +import java.lang.reflect.Method;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Wrapper around KafkaProducer that allows to resume transactions in case of node failure, which allows to implement
    + * two phase commit algorithm for exactly-once semantic FlinkKafkaProducer.
    + *
    + * <p>For happy path usage is exactly the same as {@link org.apache.kafka.clients.producer.KafkaProducer}. User is
    + * expected to call:
    + *
    + * <ul>
    + *     <li>{@link FlinkKafkaProducer#initTransactions()}</li>
    + *     <li>{@link FlinkKafkaProducer#beginTransaction()}</li>
    + *     <li>{@link FlinkKafkaProducer#send(org.apache.kafka.clients.producer.ProducerRecord)}</li>
    + *     <li>{@link FlinkKafkaProducer#flush()}</li>
    + *     <li>{@link FlinkKafkaProducer#commitTransaction()}</li>
    + * </ul>
    + *
    + * <p>To actually implement two phase commit, it must be possible to always commit a transaction after pre-committing
    + * it (here, pre-commit is just a {@link FlinkKafkaProducer#flush()}). In case of some failure between
    + * {@link FlinkKafkaProducer#flush()} and {@link FlinkKafkaProducer#commitTransaction()} this class allows to resume
    + * interrupted transaction and commit if after a restart:
    + *
    + * <ul>
    + *     <li>{@link FlinkKafkaProducer#initTransactions()}</li>
    + *     <li>{@link FlinkKafkaProducer#beginTransaction()}</li>
    + *     <li>{@link FlinkKafkaProducer#send(org.apache.kafka.clients.producer.ProducerRecord)}</li>
    + *     <li>{@link FlinkKafkaProducer#flush()}</li>
    + *     <li>{@link FlinkKafkaProducer#getProducerId()}</li>
    + *     <li>{@link FlinkKafkaProducer#getEpoch()}</li>
    + *     <li>node failure... restore producerId and epoch from state</li>
    + *     <li>{@link FlinkKafkaProducer#resumeTransaction(long, short)}</li>
    + *     <li>{@link FlinkKafkaProducer#commitTransaction()}</li>
    + * </ul>
    + *
    + * <p>{@link FlinkKafkaProducer#resumeTransaction(long, short)} replaces {@link FlinkKafkaProducer#initTransactions()}
    + * as a way to obtain the producerId and epoch counters. It has to be done, because otherwise
    + * {@link FlinkKafkaProducer#initTransactions()} would automatically abort all on going transactions.
    + *
    + * <p>Second way this implementation differs from the reference {@link org.apache.kafka.clients.producer.KafkaProducer}
    + * is that this one actually flushes new partitions on {@link FlinkKafkaProducer#flush()} instead of on
    + * {@link FlinkKafkaProducer#commitTransaction()}.
    + *
    + * <p>The last one minor difference is that it allows to obtain the producerId and epoch counters via
    + * {@link FlinkKafkaProducer#getProducerId()} and {@link FlinkKafkaProducer#getEpoch()} methods (which are unfortunately
    + * private fields).
    + *
    + * <p>Those changes are compatible with Kafka's 0.11.0 REST API although it clearly was not the intention of the Kafka's
    + * API authors to make them possible.
    + *
    + * <p>Internally this implementation uses {@link org.apache.kafka.clients.producer.KafkaProducer} and implements
    + * required changes via Java Reflection API. It might not be the prettiest solution. An alternative would be to
    + * re-implement whole Kafka's 0.11 REST API client on our own.
    + */
    +public class FlinkKafkaProducer<K, V> implements Producer<K, V> {
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducer.class);
    +
    +	private final KafkaProducer<K, V> kafkaProducer;
    +	@Nullable
    --- End diff --
    
    nit: empty line before this field annotation.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r128166034
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -0,0 +1,317 @@
    +/*
    + * 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.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);
    +
    +	@Nullable
    +	protected TXN currentTransaction;
    +
    +	protected final List<TransactionAndCheckpoint<TXN>> pendingCommitTransactions = new ArrayList<>();
    +
    +	protected ListState<TXN> pendingTransactionsState;
    +
    +	protected ListState<TransactionAndCheckpoint<TXN>> pendingCommitTransactionsState;
    +
    +	// ------ 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 (non Pravega sink) 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
    --- End diff --
    
    Lets move this comment block as a Javadoc on the method.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    One other comment regarding the commits:
    I would argue that df6d5e0 to 5ff8106 should not appear in the commit log history, since in the end we actually have a completely new producer for 011 anyways.
    Also, 321a142 to 2cf5f3b should be squashed to a single commit for the addition of an "exactly-once producer for 011" (the new `FlinkKafkaProducer` implementation and exactly-once tests shouldn't stand alone as independent commits, IMO. `FlinkKafkaProducer` isn't used by other producer version, and the exactly-once producer addition wouldn't be valid without the tests).


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r128166822
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -0,0 +1,317 @@
    +/*
    + * 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.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 --
    
    Overall, though, I would like to see unit tests specifically for this `TwoPhaseCommitSinkFunction` class.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134711336
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    +		ClosureCleaner.clean(customPartitioner, true);
    +		ClosureCleaner.ensureSerializable(serializationSchema);
    +
    +		this.defaultTopicId = defaultTopicId;
    +		this.schema = serializationSchema;
    +		this.producerConfig = producerConfig;
    +		this.flinkKafkaPartitioner = customPartitioner;
    +		this.semantic = semantic;
    +		this.kafkaProducersPoolSize = kafkaProducersPoolSize;
    +
    +		// set the producer configuration properties for kafka record key value serializers.
    +		if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		// eagerly ensure that bootstrap servers are set.
    +		if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
    +			throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must be supplied in the producer config properties.");
    +		}
    +
    +		this.topicPartitionsMap = new HashMap<>();
    +	}
    +
    +	// ---------------------------------- Properties --------------------------
    +
    +	/**
    +	 * Defines whether the producer should fail on errors, or only log them.
    +	 * If this is set to true, then exceptions will be only logged, if set to false,
    +	 * exceptions will be eventually thrown and cause the streaming program to
    +	 * fail (and enter recovery).
    +	 *
    +	 * <p>Method is only accessible for approach (a) (see above)
    +	 *
    +	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
    +	 */
    +	public void setLogFailuresOnly(boolean logFailuresOnly) {
    +		this.logFailuresOnly = logFailuresOnly;
    +	}
    +
    +	// ----------------------------------- Utilities --------------------------
    +
    +	/**
    +	 * Initializes the connection to Kafka.
    +	 *
    +	 * <p>This method is used for approach (a) (see above).
    +	 */
    +	@Override
    +	public void open(Configuration configuration) throws Exception {
    +		if (semantic != Semantic.NONE && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) {
    --- End diff --
    
    I'm confused. In some previous offline discussion you (or was it maybe Stephan?) told me that we need separate enum for setting up `FlinkKafaProducer011`. What's the point of having this separate enum, if we want to tie it up to `CheckpointMode`?


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    Merged! 😃 
    
    Could you please close this PR?


---

[GitHub] flink issue #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    Regarding how I would proceed with this big contribution:
    Lets first try to clean up the commits that are bundled all together here.
    
    1. I would first try to merge #4321 (the first 4 commits) and #4310 (af7ed19) and get those out of the way.
    2. For a06cb94 (`TwoPhaseCommitSinkFunction`), could you open a separate PR with unit tests covered?
    3. After the above is all sorted out, we rebase this again.
    
    



---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r128168357
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -0,0 +1,317 @@
    +/*
    + * 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.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);
    +
    +	@Nullable
    +	protected TXN currentTransaction;
    +
    +	protected final List<TransactionAndCheckpoint<TXN>> pendingCommitTransactions = new ArrayList<>();
    +
    +	protected ListState<TXN> pendingTransactionsState;
    +
    +	protected ListState<TransactionAndCheckpoint<TXN>> pendingCommitTransactionsState;
    +
    +	// ------ 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 (non Pravega sink) 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.
    +
    +		pendingCommitTransactionsState = context.getOperatorStateStore().getSerializableListState("pendingCommitTransactions");
    +		pendingTransactionsState = context.getOperatorStateStore().getSerializableListState("pendingTransactions");
    --- End diff --
    
    `getSerializableListState` is deprecated and discouraged usage.
    I would recommend that implementations may also pass in either the `TypeInformation` or their own `TypeSerializer` for the transaction state holder.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134421636
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    +		ClosureCleaner.clean(customPartitioner, true);
    +		ClosureCleaner.ensureSerializable(serializationSchema);
    +
    +		this.defaultTopicId = defaultTopicId;
    +		this.schema = serializationSchema;
    +		this.producerConfig = producerConfig;
    +		this.flinkKafkaPartitioner = customPartitioner;
    +		this.semantic = semantic;
    +		this.kafkaProducersPoolSize = kafkaProducersPoolSize;
    +
    +		// set the producer configuration properties for kafka record key value serializers.
    +		if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		// eagerly ensure that bootstrap servers are set.
    +		if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
    +			throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must be supplied in the producer config properties.");
    +		}
    +
    +		this.topicPartitionsMap = new HashMap<>();
    +	}
    +
    +	// ---------------------------------- Properties --------------------------
    +
    +	/**
    +	 * Defines whether the producer should fail on errors, or only log them.
    +	 * If this is set to true, then exceptions will be only logged, if set to false,
    +	 * exceptions will be eventually thrown and cause the streaming program to
    +	 * fail (and enter recovery).
    +	 *
    +	 * <p>Method is only accessible for approach (a) (see above)
    +	 *
    +	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
    +	 */
    +	public void setLogFailuresOnly(boolean logFailuresOnly) {
    +		this.logFailuresOnly = logFailuresOnly;
    +	}
    +
    +	// ----------------------------------- Utilities --------------------------
    +
    +	/**
    +	 * Initializes the connection to Kafka.
    +	 *
    +	 * <p>This method is used for approach (a) (see above).
    +	 */
    +	@Override
    +	public void open(Configuration configuration) throws Exception {
    +		if (semantic != Semantic.NONE && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) {
    +			LOG.warn(String.format("Using [%s] semantic, but checkpointing is not enabled. Switching to [%s] semantic.", semantic, Semantic.NONE));
    +			semantic = Semantic.NONE;
    +		}
    +
    +		if (logFailuresOnly) {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception e) {
    +					if (e != null) {
    +						LOG.error("Error while sending record to Kafka: " + e.getMessage(), e);
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +		else {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception exception) {
    +					if (exception != null && asyncException == null) {
    +						asyncException = exception;
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +
    +		super.open(configuration);
    +	}
    +
    +	@Override
    +	public void invoke(KafkaTransactionState transaction, IN next) throws Exception {
    +		invokeInternal(transaction, next, Long.MAX_VALUE);
    +	}
    +
    +	private void invokeInternal(KafkaTransactionState transaction, IN next, long elementTimestamp) throws Exception {
    +		checkErroneous();
    +
    +		byte[] serializedKey = schema.serializeKey(next);
    +		byte[] serializedValue = schema.serializeValue(next);
    +		String targetTopic = schema.getTargetTopic(next);
    +		if (targetTopic == null) {
    +			targetTopic = defaultTopicId;
    +		}
    +
    +		Long timestamp = null;
    +		if (this.writeTimestampToKafka) {
    +			timestamp = elementTimestamp;
    +		}
    +
    +		ProducerRecord<byte[], byte[]> record;
    +		int[] partitions = topicPartitionsMap.get(targetTopic);
    +		if (null == partitions) {
    +			partitions = getPartitionsByTopic(targetTopic, transaction.producer);
    +			topicPartitionsMap.put(targetTopic, partitions);
    +		}
    +		if (flinkKafkaPartitioner == null) {
    +			record = new ProducerRecord<>(targetTopic, null, timestamp, serializedKey, serializedValue);
    +		} else {
    +			record = new ProducerRecord<>(targetTopic, flinkKafkaPartitioner.partition(next, serializedKey, serializedValue, targetTopic, partitions), timestamp, serializedKey, serializedValue);
    +		}
    +		pendingRecords.incrementAndGet();
    +		transaction.producer.send(record, callback);
    +	}
    +
    +	@Override
    +	public void close() throws Exception {
    +		if (currentTransaction != null) {
    +			// to avoid exceptions on aborting transactions with some pending records
    +			flush(currentTransaction);
    +		}
    +		try {
    +			super.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		try {
    +			producersPool.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		// make sure we propagate pending errors
    +		checkErroneous();
    +	}
    +
    +	// ------------------- Logic for handling checkpoint flushing -------------------------- //
    +
    +	@Override
    +	protected KafkaTransactionState beginTransaction() throws Exception {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				FlinkKafkaProducer<byte[], byte[]> producer = producersPool.poll();
    +				if (producer == null) {
    +					String transactionalId = availableTransactionalIds.poll();
    +					if (transactionalId == null) {
    +						throw new Exception(
    +							"Too many ongoing snapshots. Increase kafka producers pool size or decrease number of concurrent checktpoins.");
    +					}
    +					usedTransactionalIds.add(transactionalId);
    +					producer = initTransactionalProducer(transactionalId, true);
    +					producer.initTransactions();
    +				}
    +				producer.beginTransaction();
    +				return new KafkaTransactionState(producer.getTransactionalId(), producer);
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				// Do not create new producer on each beginTransaction() if it is not necessary
    +				if (currentTransaction != null && currentTransaction.producer != null) {
    +					return new KafkaTransactionState(currentTransaction.producer);
    +				}
    +				return new KafkaTransactionState(initProducer(true));
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void preCommit(KafkaTransactionState transaction) throws Exception {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +			case AT_LEAST_ONCE:
    +				flush(transaction);
    +				break;
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +		checkErroneous();
    +	}
    +
    +	@Override
    +	protected void commit(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				transaction.producer.commitTransaction();
    +				producersPool.add(transaction.producer);
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void recoverAndCommit(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				KafkaTransactionState kafkaTransaction = transaction;
    +				FlinkKafkaProducer<byte[], byte[]> producer =
    +					initTransactionalProducer(kafkaTransaction.transactionalId, false);
    +				producer.resumeTransaction(kafkaTransaction.producerId, kafkaTransaction.epoch);
    +				try {
    +					producer.commitTransaction();
    +					producer.close();
    +				}
    +				catch (InvalidTxnStateException ex) {
    +					// That means we have committed this transaction before.
    +					LOG.warn("Encountered error [%s] while recovering transaction [%s]. " +
    +						"Presumably this transaction has been already committed before",
    +						ex,
    +						transaction);
    +				}
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void abort(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				transaction.producer.abortTransaction();
    +				producersPool.add(transaction.producer);
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				producersPool.add(transaction.producer);
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void recoverAndAbort(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				FlinkKafkaProducer<byte[], byte[]> producer =
    +					initTransactionalProducer(transaction.transactionalId, false);
    +				producer.resumeTransaction(transaction.producerId, transaction.epoch);
    +				producer.abortTransaction();
    +				producer.close();
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	private void acknowledgeMessage() {
    +		pendingRecords.decrementAndGet();
    +	}
    +
    +	/**
    +	 * Flush pending records.
    +	 * @param transaction
    +	 */
    +	private void flush(KafkaTransactionState transaction) throws Exception {
    +		if (transaction.producer != null) {
    +			transaction.producer.flush();
    +		}
    +		long pendingRecordsCount = pendingRecords.get();
    +		if (pendingRecordsCount != 0) {
    +			throw new IllegalStateException("Pending record count must be zero at this point: " + pendingRecordsCount);
    +		}
    +
    +		// if the flushed requests has errors, we should propagate it also and fail the checkpoint
    +		checkErroneous();
    +	}
    +
    +	@Override
    +	public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +		super.snapshotState(context);
    +
    +		transactionalIdsState.clear();
    +		for (String transactionalId : availableTransactionalIds) {
    +			transactionalIdsState.add(transactionalId);
    +		}
    +		for (String transactionalId : usedTransactionalIds) {
    +			transactionalIdsState.add(transactionalId);
    +		}
    +	}
    +
    +	@Override
    +	public void initializeState(FunctionInitializationContext context) throws Exception {
    +		availableTransactionalIds.clear();
    +		for (int i = 0; i < kafkaProducersPoolSize; i++) {
    +			availableTransactionalIds.add(UUID.randomUUID().toString());
    --- End diff --
    
    That's a valid issue, however on it's one this solution would not be enough. It would not work for a case when we first (1) scale down, then we (2) scale up. On event (2), we would need to create new transactional ids, but we wouldn't know from which id we can start.
    
    However I think we can deduce the starting point for new IDs using `getUnionListState` to track down globally what is the next available transactional id.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134395975
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    +		ClosureCleaner.clean(customPartitioner, true);
    --- End diff --
    
    We need to make sure that the `customPartitioner` is also serializable.
    
    Also, I recommend following this example to do that:
    ```
    checkArgument(
        InitializationUtil.isSerializable(customPartitioner), "Provided custom partitioner is not serializable".
    ```
    
    Same goes for `serializationSchema`


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r128271792
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,818 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.UUID;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic.EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular sink function (a)
    --- End diff --
    
    Yes, according to all of the tests it those.
    
    (b) version works by passing instance of `FlinkKafkaProducer011` as a`SinkFunction` in the `KafkaStreamSink<IN> extends StreamSink<IN>` class. Under the hood this `StreamSink` makes some checking if `SinkFunction` actually implements various versions of checkpointing interfaces and in that way it calls the appropriate methods on `FlinkKafkaProducer011`.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r128163374
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer011.java ---
    @@ -0,0 +1,117 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.streaming.util.serialization.DeserializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
    +
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Properties;
    +
    +/**
    + * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from
    + * Apache Kafka 0.11.x. The consumer can run in multiple parallel instances, each of which will pull
    + * data from one or more Kafka partitions.
    + *
    + * <p>The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost
    + * during a failure, and that the computation processes elements "exactly once".
    + * (Note: These guarantees naturally assume that Kafka itself does not loose any data.)</p>
    + *
    + * <p>Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets
    + * committed to Kafka / ZooKeeper are only to bring the outside view of progress in sync with Flink's view
    + * of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer
    + * has consumed a topic.</p>
    + *
    + * <p>Please refer to Kafka's documentation for the available configuration properties:
    + * http://kafka.apache.org/documentation.html#newconsumerconfigs</p>
    + *
    + * <p><b>NOTE:</b> The implementation currently accesses partition metadata when the consumer
    + * is constructed. That means that the client that submits the program needs to be able to
    + * reach the Kafka brokers or ZooKeeper.</p>
    --- End diff --
    
    This NOTE is no longer valid and can be removed.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    I think there is no way we can to handle it in any different way then to increase the timeout to some very large value. Or is it?


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    Yep, that makes sense.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    How does exactly-once sink handle large gap between `preCommit()` and `recoverAndCommit()` in case of a recovery? The server seems to abort a transaction after a timeout `max.transaction.timeout.ms`. 


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

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


---

[GitHub] flink pull request #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134394721
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    --- End diff --
    
    nit: empty line before comment block.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r128180978
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer011.java ---
    @@ -0,0 +1,117 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.streaming.util.serialization.DeserializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
    +
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Properties;
    +
    +/**
    + * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from
    + * Apache Kafka 0.11.x. The consumer can run in multiple parallel instances, each of which will pull
    + * data from one or more Kafka partitions.
    + *
    + * <p>The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost
    + * during a failure, and that the computation processes elements "exactly once".
    + * (Note: These guarantees naturally assume that Kafka itself does not loose any data.)</p>
    + *
    + * <p>Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets
    + * committed to Kafka / ZooKeeper are only to bring the outside view of progress in sync with Flink's view
    + * of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer
    + * has consumed a topic.</p>
    + *
    + * <p>Please refer to Kafka's documentation for the available configuration properties:
    + * http://kafka.apache.org/documentation.html#newconsumerconfigs</p>
    + *
    + * <p><b>NOTE:</b> The implementation currently accesses partition metadata when the consumer
    + * is constructed. That means that the client that submits the program needs to be able to
    + * reach the Kafka brokers or ZooKeeper.</p>
    --- End diff --
    
    Is it also true for 0.10?


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    Implemented fixed size pool of producers, please check last commit.
    
    If we run out of producers in the pool, exception is being thrown aborting ongoing snapshot.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    Thanks :)


---

[GitHub] flink pull request #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134838729
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    +		ClosureCleaner.clean(customPartitioner, true);
    +		ClosureCleaner.ensureSerializable(serializationSchema);
    +
    +		this.defaultTopicId = defaultTopicId;
    +		this.schema = serializationSchema;
    +		this.producerConfig = producerConfig;
    +		this.flinkKafkaPartitioner = customPartitioner;
    +		this.semantic = semantic;
    +		this.kafkaProducersPoolSize = kafkaProducersPoolSize;
    +
    +		// set the producer configuration properties for kafka record key value serializers.
    +		if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		// eagerly ensure that bootstrap servers are set.
    +		if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
    +			throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must be supplied in the producer config properties.");
    +		}
    +
    +		this.topicPartitionsMap = new HashMap<>();
    +	}
    +
    +	// ---------------------------------- Properties --------------------------
    +
    +	/**
    +	 * Defines whether the producer should fail on errors, or only log them.
    +	 * If this is set to true, then exceptions will be only logged, if set to false,
    +	 * exceptions will be eventually thrown and cause the streaming program to
    +	 * fail (and enter recovery).
    +	 *
    +	 * <p>Method is only accessible for approach (a) (see above)
    +	 *
    +	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
    +	 */
    +	public void setLogFailuresOnly(boolean logFailuresOnly) {
    +		this.logFailuresOnly = logFailuresOnly;
    +	}
    +
    +	// ----------------------------------- Utilities --------------------------
    +
    +	/**
    +	 * Initializes the connection to Kafka.
    +	 *
    +	 * <p>This method is used for approach (a) (see above).
    +	 */
    +	@Override
    +	public void open(Configuration configuration) throws Exception {
    +		if (semantic != Semantic.NONE && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) {
    +			LOG.warn(String.format("Using [%s] semantic, but checkpointing is not enabled. Switching to [%s] semantic.", semantic, Semantic.NONE));
    +			semantic = Semantic.NONE;
    +		}
    +
    +		if (logFailuresOnly) {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception e) {
    +					if (e != null) {
    +						LOG.error("Error while sending record to Kafka: " + e.getMessage(), e);
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +		else {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception exception) {
    +					if (exception != null && asyncException == null) {
    +						asyncException = exception;
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +
    +		super.open(configuration);
    +	}
    +
    +	@Override
    +	public void invoke(KafkaTransactionState transaction, IN next) throws Exception {
    +		invokeInternal(transaction, next, Long.MAX_VALUE);
    +	}
    +
    +	private void invokeInternal(KafkaTransactionState transaction, IN next, long elementTimestamp) throws Exception {
    +		checkErroneous();
    +
    +		byte[] serializedKey = schema.serializeKey(next);
    +		byte[] serializedValue = schema.serializeValue(next);
    +		String targetTopic = schema.getTargetTopic(next);
    +		if (targetTopic == null) {
    +			targetTopic = defaultTopicId;
    +		}
    +
    +		Long timestamp = null;
    +		if (this.writeTimestampToKafka) {
    +			timestamp = elementTimestamp;
    +		}
    +
    +		ProducerRecord<byte[], byte[]> record;
    +		int[] partitions = topicPartitionsMap.get(targetTopic);
    +		if (null == partitions) {
    +			partitions = getPartitionsByTopic(targetTopic, transaction.producer);
    +			topicPartitionsMap.put(targetTopic, partitions);
    +		}
    +		if (flinkKafkaPartitioner == null) {
    +			record = new ProducerRecord<>(targetTopic, null, timestamp, serializedKey, serializedValue);
    +		} else {
    +			record = new ProducerRecord<>(targetTopic, flinkKafkaPartitioner.partition(next, serializedKey, serializedValue, targetTopic, partitions), timestamp, serializedKey, serializedValue);
    +		}
    +		pendingRecords.incrementAndGet();
    +		transaction.producer.send(record, callback);
    +	}
    +
    +	@Override
    +	public void close() throws Exception {
    +		if (currentTransaction != null) {
    +			// to avoid exceptions on aborting transactions with some pending records
    +			flush(currentTransaction);
    +		}
    +		try {
    +			super.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		try {
    +			producersPool.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		// make sure we propagate pending errors
    +		checkErroneous();
    +	}
    +
    +	// ------------------- Logic for handling checkpoint flushing -------------------------- //
    +
    +	@Override
    +	protected KafkaTransactionState beginTransaction() throws Exception {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				FlinkKafkaProducer<byte[], byte[]> producer = producersPool.poll();
    +				if (producer == null) {
    +					String transactionalId = availableTransactionalIds.poll();
    +					if (transactionalId == null) {
    +						throw new Exception(
    +							"Too many ongoing snapshots. Increase kafka producers pool size or decrease number of concurrent checktpoins.");
    +					}
    +					usedTransactionalIds.add(transactionalId);
    +					producer = initTransactionalProducer(transactionalId, true);
    +					producer.initTransactions();
    +				}
    +				producer.beginTransaction();
    +				return new KafkaTransactionState(producer.getTransactionalId(), producer);
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				// Do not create new producer on each beginTransaction() if it is not necessary
    +				if (currentTransaction != null && currentTransaction.producer != null) {
    +					return new KafkaTransactionState(currentTransaction.producer);
    +				}
    +				return new KafkaTransactionState(initProducer(true));
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void preCommit(KafkaTransactionState transaction) throws Exception {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +			case AT_LEAST_ONCE:
    +				flush(transaction);
    +				break;
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +		checkErroneous();
    +	}
    +
    +	@Override
    +	protected void commit(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				transaction.producer.commitTransaction();
    +				producersPool.add(transaction.producer);
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void recoverAndCommit(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				KafkaTransactionState kafkaTransaction = transaction;
    +				FlinkKafkaProducer<byte[], byte[]> producer =
    +					initTransactionalProducer(kafkaTransaction.transactionalId, false);
    +				producer.resumeTransaction(kafkaTransaction.producerId, kafkaTransaction.epoch);
    +				try {
    +					producer.commitTransaction();
    +					producer.close();
    +				}
    +				catch (InvalidTxnStateException ex) {
    +					// That means we have committed this transaction before.
    +					LOG.warn("Encountered error [%s] while recovering transaction [%s]. " +
    +						"Presumably this transaction has been already committed before",
    +						ex,
    +						transaction);
    +				}
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void abort(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				transaction.producer.abortTransaction();
    +				producersPool.add(transaction.producer);
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				producersPool.add(transaction.producer);
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void recoverAndAbort(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				FlinkKafkaProducer<byte[], byte[]> producer =
    +					initTransactionalProducer(transaction.transactionalId, false);
    +				producer.resumeTransaction(transaction.producerId, transaction.epoch);
    +				producer.abortTransaction();
    +				producer.close();
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	private void acknowledgeMessage() {
    +		pendingRecords.decrementAndGet();
    +	}
    +
    +	/**
    +	 * Flush pending records.
    +	 * @param transaction
    +	 */
    +	private void flush(KafkaTransactionState transaction) throws Exception {
    +		if (transaction.producer != null) {
    +			transaction.producer.flush();
    +		}
    +		long pendingRecordsCount = pendingRecords.get();
    +		if (pendingRecordsCount != 0) {
    +			throw new IllegalStateException("Pending record count must be zero at this point: " + pendingRecordsCount);
    +		}
    +
    +		// if the flushed requests has errors, we should propagate it also and fail the checkpoint
    +		checkErroneous();
    +	}
    +
    +	@Override
    +	public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +		super.snapshotState(context);
    +
    +		transactionalIdsState.clear();
    +		for (String transactionalId : availableTransactionalIds) {
    +			transactionalIdsState.add(transactionalId);
    +		}
    +		for (String transactionalId : usedTransactionalIds) {
    +			transactionalIdsState.add(transactionalId);
    +		}
    +	}
    +
    +	@Override
    +	public void initializeState(FunctionInitializationContext context) throws Exception {
    +		availableTransactionalIds.clear();
    +		for (int i = 0; i < kafkaProducersPoolSize; i++) {
    +			availableTransactionalIds.add(UUID.randomUUID().toString());
    --- End diff --
    
    > we wouldn't know from which id we can start.
    Not sure if you need 'start id'. You can just abort all of them whether they are any open transactions or not (in fact if you open a new producer with the id, Kafka aborts any that are open). This is mainly a for clarification, will leave it to you guys to decide on specifics. 


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134396368
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    +		ClosureCleaner.clean(customPartitioner, true);
    +		ClosureCleaner.ensureSerializable(serializationSchema);
    +
    +		this.defaultTopicId = defaultTopicId;
    +		this.schema = serializationSchema;
    +		this.producerConfig = producerConfig;
    +		this.flinkKafkaPartitioner = customPartitioner;
    +		this.semantic = semantic;
    +		this.kafkaProducersPoolSize = kafkaProducersPoolSize;
    +
    +		// set the producer configuration properties for kafka record key value serializers.
    +		if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		// eagerly ensure that bootstrap servers are set.
    +		if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
    +			throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must be supplied in the producer config properties.");
    +		}
    +
    +		this.topicPartitionsMap = new HashMap<>();
    +	}
    +
    +	// ---------------------------------- Properties --------------------------
    +
    +	/**
    +	 * Defines whether the producer should fail on errors, or only log them.
    +	 * If this is set to true, then exceptions will be only logged, if set to false,
    +	 * exceptions will be eventually thrown and cause the streaming program to
    +	 * fail (and enter recovery).
    +	 *
    +	 * <p>Method is only accessible for approach (a) (see above)
    +	 *
    +	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
    +	 */
    +	public void setLogFailuresOnly(boolean logFailuresOnly) {
    +		this.logFailuresOnly = logFailuresOnly;
    +	}
    +
    +	// ----------------------------------- Utilities --------------------------
    +
    +	/**
    +	 * Initializes the connection to Kafka.
    +	 *
    +	 * <p>This method is used for approach (a) (see above).
    +	 */
    +	@Override
    +	public void open(Configuration configuration) throws Exception {
    +		if (semantic != Semantic.NONE && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) {
    --- End diff --
    
    I think we also need to check whether the checkpointing guarantee is `AT_LEAST_ONCE` or `EXACTLY_ONCE`


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    > Hmmm, are you sure about this thing? That would mean that Kafka doesn't support transactional parallel writes from two different process, which would be very strange. Could you point to a source of this information?
    
    It does not prohibit parallel transactions. Just restricts what an EOS consumer, which reads only the committed messages can see.
    
    See 'Reading Transactional Messages' section in JavaDoc for KafkaConsumer : https://github.com/apache/kafka/blob/0.11.0/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L421 : 
    
    > In read_committed mode, the consumer will read only those transactional messages which have been successfully committed. It will continue to read non-transactional messages as before. There is no client-side buffering in read_committed mode. Instead, the end offset of a partition for a read_committed consumer would be the offset of the first message in the partition belonging to an open transaction. This offset is known as the 'Last Stable Offset'(LSO).
    
    If there is an open transaction, the EOS consumers don't read past it.



---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    please add an entry to the `MODULES_CONNECTORS'  variable in the `tools/travis_mvn_watchdog` sh script.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r128428361
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer011.java ---
    @@ -0,0 +1,117 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.streaming.util.serialization.DeserializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
    +
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Properties;
    +
    +/**
    + * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from
    + * Apache Kafka 0.11.x. The consumer can run in multiple parallel instances, each of which will pull
    + * data from one or more Kafka partitions.
    + *
    + * <p>The Flink Kafka Consumer participates in checkpointing and guarantees that no data is lost
    + * during a failure, and that the computation processes elements "exactly once".
    + * (Note: These guarantees naturally assume that Kafka itself does not loose any data.)</p>
    + *
    + * <p>Please note that Flink snapshots the offsets internally as part of its distributed checkpoints. The offsets
    + * committed to Kafka / ZooKeeper are only to bring the outside view of progress in sync with Flink's view
    + * of the progress. That way, monitoring and other jobs can get a view of how far the Flink Kafka consumer
    + * has consumed a topic.</p>
    + *
    + * <p>Please refer to Kafka's documentation for the available configuration properties:
    + * http://kafka.apache.org/documentation.html#newconsumerconfigs</p>
    + *
    + * <p><b>NOTE:</b> The implementation currently accesses partition metadata when the consumer
    + * is constructed. That means that the client that submits the program needs to be able to
    + * reach the Kafka brokers or ZooKeeper.</p>
    --- End diff --
    
    Yes. I have a separate PR which cleans that up for all Kafka versions.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134044031
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    +		ClosureCleaner.clean(customPartitioner, true);
    +		ClosureCleaner.ensureSerializable(serializationSchema);
    +
    +		this.defaultTopicId = defaultTopicId;
    +		this.schema = serializationSchema;
    +		this.producerConfig = producerConfig;
    +		this.flinkKafkaPartitioner = customPartitioner;
    +		this.semantic = semantic;
    +		this.kafkaProducersPoolSize = kafkaProducersPoolSize;
    +
    +		// set the producer configuration properties for kafka record key value serializers.
    +		if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		// eagerly ensure that bootstrap servers are set.
    +		if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
    +			throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must be supplied in the producer config properties.");
    +		}
    +
    +		this.topicPartitionsMap = new HashMap<>();
    +	}
    +
    +	// ---------------------------------- Properties --------------------------
    +
    +	/**
    +	 * Defines whether the producer should fail on errors, or only log them.
    +	 * If this is set to true, then exceptions will be only logged, if set to false,
    +	 * exceptions will be eventually thrown and cause the streaming program to
    +	 * fail (and enter recovery).
    +	 *
    +	 * <p>Method is only accessible for approach (a) (see above)
    +	 *
    +	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
    +	 */
    +	public void setLogFailuresOnly(boolean logFailuresOnly) {
    +		this.logFailuresOnly = logFailuresOnly;
    +	}
    +
    +	// ----------------------------------- Utilities --------------------------
    +
    +	/**
    +	 * Initializes the connection to Kafka.
    +	 *
    +	 * <p>This method is used for approach (a) (see above).
    +	 */
    +	@Override
    +	public void open(Configuration configuration) throws Exception {
    +		if (semantic != Semantic.NONE && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) {
    +			LOG.warn(String.format("Using [%s] semantic, but checkpointing is not enabled. Switching to [%s] semantic.", semantic, Semantic.NONE));
    +			semantic = Semantic.NONE;
    +		}
    +
    +		if (logFailuresOnly) {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception e) {
    +					if (e != null) {
    +						LOG.error("Error while sending record to Kafka: " + e.getMessage(), e);
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +		else {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception exception) {
    +					if (exception != null && asyncException == null) {
    +						asyncException = exception;
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +
    +		super.open(configuration);
    +	}
    +
    +	@Override
    +	public void invoke(KafkaTransactionState transaction, IN next) throws Exception {
    +		invokeInternal(transaction, next, Long.MAX_VALUE);
    +	}
    +
    +	private void invokeInternal(KafkaTransactionState transaction, IN next, long elementTimestamp) throws Exception {
    +		checkErroneous();
    +
    +		byte[] serializedKey = schema.serializeKey(next);
    +		byte[] serializedValue = schema.serializeValue(next);
    +		String targetTopic = schema.getTargetTopic(next);
    +		if (targetTopic == null) {
    +			targetTopic = defaultTopicId;
    +		}
    +
    +		Long timestamp = null;
    +		if (this.writeTimestampToKafka) {
    +			timestamp = elementTimestamp;
    +		}
    +
    +		ProducerRecord<byte[], byte[]> record;
    +		int[] partitions = topicPartitionsMap.get(targetTopic);
    +		if (null == partitions) {
    +			partitions = getPartitionsByTopic(targetTopic, transaction.producer);
    +			topicPartitionsMap.put(targetTopic, partitions);
    +		}
    +		if (flinkKafkaPartitioner == null) {
    +			record = new ProducerRecord<>(targetTopic, null, timestamp, serializedKey, serializedValue);
    +		} else {
    +			record = new ProducerRecord<>(targetTopic, flinkKafkaPartitioner.partition(next, serializedKey, serializedValue, targetTopic, partitions), timestamp, serializedKey, serializedValue);
    +		}
    +		pendingRecords.incrementAndGet();
    +		transaction.producer.send(record, callback);
    +	}
    +
    +	@Override
    +	public void close() throws Exception {
    +		if (currentTransaction != null) {
    +			// to avoid exceptions on aborting transactions with some pending records
    +			flush(currentTransaction);
    +		}
    +		try {
    +			super.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		try {
    +			producersPool.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		// make sure we propagate pending errors
    +		checkErroneous();
    +	}
    +
    +	// ------------------- Logic for handling checkpoint flushing -------------------------- //
    +
    +	@Override
    +	protected KafkaTransactionState beginTransaction() throws Exception {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				FlinkKafkaProducer<byte[], byte[]> producer = producersPool.poll();
    +				if (producer == null) {
    +					String transactionalId = availableTransactionalIds.poll();
    +					if (transactionalId == null) {
    +						throw new Exception(
    +							"Too many ongoing snapshots. Increase kafka producers pool size or decrease number of concurrent checktpoins.");
    +					}
    +					usedTransactionalIds.add(transactionalId);
    +					producer = initTransactionalProducer(transactionalId, true);
    +					producer.initTransactions();
    +				}
    +				producer.beginTransaction();
    +				return new KafkaTransactionState(producer.getTransactionalId(), producer);
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				// Do not create new producer on each beginTransaction() if it is not necessary
    +				if (currentTransaction != null && currentTransaction.producer != null) {
    +					return new KafkaTransactionState(currentTransaction.producer);
    +				}
    +				return new KafkaTransactionState(initProducer(true));
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void preCommit(KafkaTransactionState transaction) throws Exception {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +			case AT_LEAST_ONCE:
    +				flush(transaction);
    +				break;
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +		checkErroneous();
    +	}
    +
    +	@Override
    +	protected void commit(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				transaction.producer.commitTransaction();
    +				producersPool.add(transaction.producer);
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void recoverAndCommit(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				KafkaTransactionState kafkaTransaction = transaction;
    +				FlinkKafkaProducer<byte[], byte[]> producer =
    +					initTransactionalProducer(kafkaTransaction.transactionalId, false);
    +				producer.resumeTransaction(kafkaTransaction.producerId, kafkaTransaction.epoch);
    +				try {
    +					producer.commitTransaction();
    +					producer.close();
    +				}
    +				catch (InvalidTxnStateException ex) {
    +					// That means we have committed this transaction before.
    +					LOG.warn("Encountered error [%s] while recovering transaction [%s]. " +
    +						"Presumably this transaction has been already committed before",
    +						ex,
    +						transaction);
    +				}
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void abort(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				transaction.producer.abortTransaction();
    +				producersPool.add(transaction.producer);
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				producersPool.add(transaction.producer);
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void recoverAndAbort(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				FlinkKafkaProducer<byte[], byte[]> producer =
    +					initTransactionalProducer(transaction.transactionalId, false);
    +				producer.resumeTransaction(transaction.producerId, transaction.epoch);
    +				producer.abortTransaction();
    +				producer.close();
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	private void acknowledgeMessage() {
    +		pendingRecords.decrementAndGet();
    +	}
    +
    +	/**
    +	 * Flush pending records.
    +	 * @param transaction
    +	 */
    +	private void flush(KafkaTransactionState transaction) throws Exception {
    +		if (transaction.producer != null) {
    +			transaction.producer.flush();
    +		}
    +		long pendingRecordsCount = pendingRecords.get();
    +		if (pendingRecordsCount != 0) {
    +			throw new IllegalStateException("Pending record count must be zero at this point: " + pendingRecordsCount);
    +		}
    +
    +		// if the flushed requests has errors, we should propagate it also and fail the checkpoint
    +		checkErroneous();
    +	}
    +
    +	@Override
    +	public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +		super.snapshotState(context);
    +
    +		transactionalIdsState.clear();
    +		for (String transactionalId : availableTransactionalIds) {
    +			transactionalIdsState.add(transactionalId);
    +		}
    +		for (String transactionalId : usedTransactionalIds) {
    +			transactionalIdsState.add(transactionalId);
    +		}
    +	}
    +
    +	@Override
    +	public void initializeState(FunctionInitializationContext context) throws Exception {
    +		availableTransactionalIds.clear();
    +		for (int i = 0; i < kafkaProducersPoolSize; i++) {
    +			availableTransactionalIds.add(UUID.randomUUID().toString());
    --- End diff --
    
    Probably better to reuse stored ids rather than creating new ones each time. I am thinking of a case where  a task goes into crash loop dying even before first commit. 


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    Thanks a lot for opening a pull request for this very important feature @pnowojski.
    I did a rough first pass and had some comments I would like to clear out first (this is a big chunk of code, we would probably need to go through this quite a few times before it can be mergeable.)
    
    Most notably, some comments so far:
    1. I think we need UTs for the `TwoPhaseCommitSinkFunction`. It alone is a very important addition (I would even prefer a separate PR for it and try to merge that first.)
    2. Serialization of the transaction state in `TwoPhaseCommitSinkFunction` needs to be changed
    2. Is the `FlinkKafkaProducer011` actually supporting hybrid (normal sink function and `writeToKafkaWithTimestamps` as a custom sink operator)? From the looks of it, it doesn't seem like it.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    I guess you could store the transactional.id for _next_ transaction in committed state. That way the new task starts the new transaction with the name stored in state which automatically aborts the open transaction.  


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r128163984
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,818 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.UUID;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic.EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular sink function (a)
    + * and a custom operator (b).
    + *
    + * <p>For (a), the class implements the SinkFunction and RichFunction interfaces.
    + * For (b), it extends the StreamTask class.
    + *
    + * <p>Details about approach (a):
    + *  Pre Kafka 0.11 producers only follow approach (a), allowing users to use the producer using the
    + *  DataStream.addSink() method.
    --- End diff --
    
    "Pre Kafka 0.11 producers only follow approach (a)" is incorrect.
    Kafka 0.10 also supports hybrid.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r128428437
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -0,0 +1,317 @@
    +/*
    + * 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.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);
    +
    +	@Nullable
    +	protected TXN currentTransaction;
    +
    +	protected final List<TransactionAndCheckpoint<TXN>> pendingCommitTransactions = new ArrayList<>();
    +
    +	protected ListState<TXN> pendingTransactionsState;
    +
    +	protected ListState<TransactionAndCheckpoint<TXN>> pendingCommitTransactionsState;
    +
    +	// ------ 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 (non Pravega sink) 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
    --- End diff --
    
    Ok, makes sense. No strong objection here, can keep as is.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r128167617
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,818 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.UUID;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic.EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular sink function (a)
    --- End diff --
    
    Does this implementation really support the hybrid modes?
    As far as I can understand it, `FlinkKafkaProducer011` only extends `TwoPhaseCommitSinkFunction`, which doesn't support the hybrid modes.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134394707
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    --- End diff --
    
    nit: empty line before comment block.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r128201074
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -0,0 +1,317 @@
    +/*
    + * 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.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);
    +
    +	@Nullable
    +	protected TXN currentTransaction;
    +
    +	protected final List<TransactionAndCheckpoint<TXN>> pendingCommitTransactions = new ArrayList<>();
    +
    +	protected ListState<TXN> pendingTransactionsState;
    +
    +	protected ListState<TransactionAndCheckpoint<TXN>> pendingCommitTransactionsState;
    +
    +	// ------ 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 (non Pravega sink) 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
    --- End diff --
    
    Hmm, why do you think so? This is a purely implementation detail, nothing that should bother the user of this class.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134676224
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    +		ClosureCleaner.clean(customPartitioner, true);
    --- End diff --
    
    This has been carried on from Kafka 0.10


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134396137
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    +		ClosureCleaner.clean(customPartitioner, true);
    +		ClosureCleaner.ensureSerializable(serializationSchema);
    +
    +		this.defaultTopicId = defaultTopicId;
    +		this.schema = serializationSchema;
    +		this.producerConfig = producerConfig;
    +		this.flinkKafkaPartitioner = customPartitioner;
    +		this.semantic = semantic;
    +		this.kafkaProducersPoolSize = kafkaProducersPoolSize;
    --- End diff --
    
    Check for negative / 0 poll sizes.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134398934
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    +		ClosureCleaner.clean(customPartitioner, true);
    +		ClosureCleaner.ensureSerializable(serializationSchema);
    +
    +		this.defaultTopicId = defaultTopicId;
    +		this.schema = serializationSchema;
    +		this.producerConfig = producerConfig;
    +		this.flinkKafkaPartitioner = customPartitioner;
    +		this.semantic = semantic;
    +		this.kafkaProducersPoolSize = kafkaProducersPoolSize;
    +
    +		// set the producer configuration properties for kafka record key value serializers.
    +		if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		// eagerly ensure that bootstrap servers are set.
    +		if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
    +			throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must be supplied in the producer config properties.");
    +		}
    +
    +		this.topicPartitionsMap = new HashMap<>();
    +	}
    +
    +	// ---------------------------------- Properties --------------------------
    +
    +	/**
    +	 * Defines whether the producer should fail on errors, or only log them.
    +	 * If this is set to true, then exceptions will be only logged, if set to false,
    +	 * exceptions will be eventually thrown and cause the streaming program to
    +	 * fail (and enter recovery).
    +	 *
    +	 * <p>Method is only accessible for approach (a) (see above)
    +	 *
    +	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
    +	 */
    +	public void setLogFailuresOnly(boolean logFailuresOnly) {
    +		this.logFailuresOnly = logFailuresOnly;
    +	}
    +
    +	// ----------------------------------- Utilities --------------------------
    +
    +	/**
    +	 * Initializes the connection to Kafka.
    +	 *
    +	 * <p>This method is used for approach (a) (see above).
    +	 */
    +	@Override
    +	public void open(Configuration configuration) throws Exception {
    +		if (semantic != Semantic.NONE && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) {
    +			LOG.warn(String.format("Using [%s] semantic, but checkpointing is not enabled. Switching to [%s] semantic.", semantic, Semantic.NONE));
    +			semantic = Semantic.NONE;
    +		}
    +
    +		if (logFailuresOnly) {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception e) {
    +					if (e != null) {
    +						LOG.error("Error while sending record to Kafka: " + e.getMessage(), e);
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +		else {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception exception) {
    +					if (exception != null && asyncException == null) {
    +						asyncException = exception;
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +
    +		super.open(configuration);
    +	}
    +
    +	@Override
    +	public void invoke(KafkaTransactionState transaction, IN next) throws Exception {
    +		invokeInternal(transaction, next, Long.MAX_VALUE);
    +	}
    +
    +	private void invokeInternal(KafkaTransactionState transaction, IN next, long elementTimestamp) throws Exception {
    +		checkErroneous();
    +
    +		byte[] serializedKey = schema.serializeKey(next);
    +		byte[] serializedValue = schema.serializeValue(next);
    +		String targetTopic = schema.getTargetTopic(next);
    +		if (targetTopic == null) {
    +			targetTopic = defaultTopicId;
    +		}
    +
    +		Long timestamp = null;
    +		if (this.writeTimestampToKafka) {
    +			timestamp = elementTimestamp;
    +		}
    +
    +		ProducerRecord<byte[], byte[]> record;
    +		int[] partitions = topicPartitionsMap.get(targetTopic);
    +		if (null == partitions) {
    +			partitions = getPartitionsByTopic(targetTopic, transaction.producer);
    +			topicPartitionsMap.put(targetTopic, partitions);
    +		}
    +		if (flinkKafkaPartitioner == null) {
    +			record = new ProducerRecord<>(targetTopic, null, timestamp, serializedKey, serializedValue);
    +		} else {
    +			record = new ProducerRecord<>(targetTopic, flinkKafkaPartitioner.partition(next, serializedKey, serializedValue, targetTopic, partitions), timestamp, serializedKey, serializedValue);
    +		}
    +		pendingRecords.incrementAndGet();
    +		transaction.producer.send(record, callback);
    +	}
    +
    +	@Override
    +	public void close() throws Exception {
    +		if (currentTransaction != null) {
    +			// to avoid exceptions on aborting transactions with some pending records
    +			flush(currentTransaction);
    +		}
    +		try {
    +			super.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		try {
    +			producersPool.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		// make sure we propagate pending errors
    +		checkErroneous();
    +	}
    +
    +	// ------------------- Logic for handling checkpoint flushing -------------------------- //
    +
    +	@Override
    +	protected KafkaTransactionState beginTransaction() throws Exception {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				FlinkKafkaProducer<byte[], byte[]> producer = producersPool.poll();
    +				if (producer == null) {
    +					String transactionalId = availableTransactionalIds.poll();
    +					if (transactionalId == null) {
    +						throw new Exception(
    +							"Too many ongoing snapshots. Increase kafka producers pool size or decrease number of concurrent checktpoins.");
    +					}
    +					usedTransactionalIds.add(transactionalId);
    +					producer = initTransactionalProducer(transactionalId, true);
    +					producer.initTransactions();
    +				}
    +				producer.beginTransaction();
    +				return new KafkaTransactionState(producer.getTransactionalId(), producer);
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				// Do not create new producer on each beginTransaction() if it is not necessary
    +				if (currentTransaction != null && currentTransaction.producer != null) {
    +					return new KafkaTransactionState(currentTransaction.producer);
    +				}
    +				return new KafkaTransactionState(initProducer(true));
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void preCommit(KafkaTransactionState transaction) throws Exception {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +			case AT_LEAST_ONCE:
    +				flush(transaction);
    +				break;
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +		checkErroneous();
    +	}
    +
    +	@Override
    +	protected void commit(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				transaction.producer.commitTransaction();
    +				producersPool.add(transaction.producer);
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void recoverAndCommit(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				KafkaTransactionState kafkaTransaction = transaction;
    +				FlinkKafkaProducer<byte[], byte[]> producer =
    +					initTransactionalProducer(kafkaTransaction.transactionalId, false);
    +				producer.resumeTransaction(kafkaTransaction.producerId, kafkaTransaction.epoch);
    +				try {
    +					producer.commitTransaction();
    +					producer.close();
    +				}
    +				catch (InvalidTxnStateException ex) {
    +					// That means we have committed this transaction before.
    +					LOG.warn("Encountered error [%s] while recovering transaction [%s]. " +
    +						"Presumably this transaction has been already committed before",
    +						ex,
    +						transaction);
    +				}
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void abort(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				transaction.producer.abortTransaction();
    +				producersPool.add(transaction.producer);
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				producersPool.add(transaction.producer);
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void recoverAndAbort(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				FlinkKafkaProducer<byte[], byte[]> producer =
    +					initTransactionalProducer(transaction.transactionalId, false);
    +				producer.resumeTransaction(transaction.producerId, transaction.epoch);
    +				producer.abortTransaction();
    +				producer.close();
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	private void acknowledgeMessage() {
    +		pendingRecords.decrementAndGet();
    +	}
    +
    +	/**
    +	 * Flush pending records.
    +	 * @param transaction
    +	 */
    +	private void flush(KafkaTransactionState transaction) throws Exception {
    +		if (transaction.producer != null) {
    +			transaction.producer.flush();
    +		}
    +		long pendingRecordsCount = pendingRecords.get();
    +		if (pendingRecordsCount != 0) {
    +			throw new IllegalStateException("Pending record count must be zero at this point: " + pendingRecordsCount);
    +		}
    +
    +		// if the flushed requests has errors, we should propagate it also and fail the checkpoint
    +		checkErroneous();
    +	}
    +
    +	@Override
    +	public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +		super.snapshotState(context);
    +
    +		transactionalIdsState.clear();
    +		for (String transactionalId : availableTransactionalIds) {
    +			transactionalIdsState.add(transactionalId);
    +		}
    +		for (String transactionalId : usedTransactionalIds) {
    +			transactionalIdsState.add(transactionalId);
    +		}
    +	}
    +
    +	@Override
    +	public void initializeState(FunctionInitializationContext context) throws Exception {
    +		availableTransactionalIds.clear();
    +		for (int i = 0; i < kafkaProducersPoolSize; i++) {
    +			availableTransactionalIds.add(UUID.randomUUID().toString());
    +		}
    +
    +		super.initializeState(context);
    +
    +		transactionalIdsState = context.getOperatorStateStore().getListState(TRANSACTIONAL_IDS_DESCRIPTOR);
    +		abortPreviousTransactions(transactionalIdsState.get());
    +	}
    +
    +	private void abortPreviousTransactions(Iterable<String> transactionalIds) {
    +		for (String transactionalid : transactionalIds) {
    +			try (FlinkKafkaProducer<byte[], byte[]> kafkaProducer =
    +					initTransactionalProducer(transactionalid, false)) {
    +				kafkaProducer.initTransactions();
    +			}
    +		}
    +	}
    +
    +	// ----------------------------------- Utilities --------------------------
    +
    +	int getTransactionCoordinatorId() {
    +		if (currentTransaction == null || currentTransaction.producer == null) {
    +			throw new IllegalArgumentException();
    +		}
    +		return currentTransaction.producer.getTransactionCoordinatorId();
    +	}
    +
    +	private FlinkKafkaProducer<byte[], byte[]> initTransactionalProducer(String transactionalId, boolean registerMetrics) {
    +		producerConfig.put("transactional.id", transactionalId);
    +		return initProducer(registerMetrics);
    +	}
    +
    +	private FlinkKafkaProducer<byte[], byte[]> initProducer(boolean registerMetrics) {
    +		FlinkKafkaProducer<byte[], byte[]> producer = new FlinkKafkaProducer<>(this.producerConfig);
    +
    +		RuntimeContext ctx = getRuntimeContext();
    +
    +		if (null != flinkKafkaPartitioner) {
    +			if (flinkKafkaPartitioner instanceof FlinkKafkaDelegatePartitioner) {
    +				((FlinkKafkaDelegatePartitioner) flinkKafkaPartitioner).setPartitions(
    +					getPartitionsByTopic(this.defaultTopicId, producer));
    +			}
    +			flinkKafkaPartitioner.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks());
    +		}
    +
    +		LOG.info("Starting FlinkKafkaProducer ({}/{}) to produce into default topic {}",
    +			ctx.getIndexOfThisSubtask() + 1, ctx.getNumberOfParallelSubtasks(), defaultTopicId);
    +
    +		// register Kafka metrics to Flink accumulators
    +		if (registerMetrics && !Boolean.parseBoolean(producerConfig.getProperty(KEY_DISABLE_METRICS, "false"))) {
    +			Map<MetricName, ? extends Metric> metrics = producer.metrics();
    +
    +			if (metrics == null) {
    +				// MapR's Kafka implementation returns null here.
    +				LOG.info("Producer implementation does not support metrics");
    +			} else {
    +				final MetricGroup kafkaMetricGroup = getRuntimeContext().getMetricGroup().addGroup("KafkaProducer");
    +				for (Map.Entry<MetricName, ? extends Metric> entry: metrics.entrySet()) {
    +					String name = entry.getKey().name();
    +					Metric metric = entry.getValue();
    +
    +					KafkaMetricMuttableWrapper wrapper = previouslyCreatedMetrics.get(name);
    +					if (wrapper != null) {
    +						wrapper.setKafkaMetric(metric);
    +					} else {
    +						// TODO: somehow merge metrics from all active producers?
    +						wrapper = new KafkaMetricMuttableWrapper(metric);
    +						previouslyCreatedMetrics.put(name, wrapper);
    +						kafkaMetricGroup.gauge(name, wrapper);
    +					}
    +				}
    +			}
    +		}
    +		return producer;
    +	}
    +
    +	private void checkErroneous() throws Exception {
    +		Exception e = asyncException;
    +		if (e != null) {
    +			// prevent double throwing
    +			asyncException = null;
    +			throw new Exception("Failed to send data to Kafka: " + e.getMessage(), e);
    +		}
    +	}
    +
    +	private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
    +		in.defaultReadObject();
    +		producersPool = new ProducersPool();
    +	}
    +
    +	private static Properties getPropertiesFromBrokerList(String brokerList) {
    +		String[] elements = brokerList.split(",");
    +
    +		// validate the broker addresses
    +		for (String broker: elements) {
    +			NetUtils.getCorrectHostnamePort(broker);
    +		}
    +
    +		Properties props = new Properties();
    +		props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList);
    +		return props;
    +	}
    +
    +	private static int[] getPartitionsByTopic(String topic, Producer<byte[], byte[]> producer) {
    +		// the fetched list is immutable, so we're creating a mutable copy in order to sort it
    +		List<PartitionInfo> partitionsList = new ArrayList<>(producer.partitionsFor(topic));
    +
    +		// sort the partitions by partition id to make sure the fetched partition list is the same across subtasks
    +		Collections.sort(partitionsList, new Comparator<PartitionInfo>() {
    +			@Override
    +			public int compare(PartitionInfo o1, PartitionInfo o2) {
    +				return Integer.compare(o1.partition(), o2.partition());
    +			}
    +		});
    +
    +		int[] partitions = new int[partitionsList.size()];
    +		for (int i = 0; i < partitions.length; i++) {
    +			partitions[i] = partitionsList.get(i).partition();
    +		}
    +
    +		return partitions;
    +	}
    +
    +	/**
    +	 * Configuration object returned by the writeToKafkaWithTimestamps() call.
    +	 */
    +	public static class FlinkKafkaProducer011Configuration<IN> extends DataStreamSink<IN> {
    +
    +		private final FlinkKafkaProducer011 producer;
    +
    +		private FlinkKafkaProducer011Configuration(DataStream stream, KafkaStreamSink streamSink) {
    +			//noinspection unchecked
    +			super(stream, streamSink);
    +			this.producer = streamSink.kafkaProducer;
    +		}
    +
    +		/**
    +		 * Defines whether the producer should fail on errors, or only log them.
    +		 * If this is set to true, then exceptions will be only logged, if set to false,
    +		 * exceptions will be eventually thrown and cause the streaming program to
    +		 * fail (and enter recovery).
    +		 *
    +		 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
    +		 */
    +		public void setLogFailuresOnly(boolean logFailuresOnly) {
    +			this.producer.setLogFailuresOnly(logFailuresOnly);
    +		}
    +
    +		/**
    +		 * If set to true, Flink will write the (event time) timestamp attached to each record into Kafka.
    +		 * Timestamps must be positive for Kafka to accept them.
    +		 *
    +		 * @param writeTimestampToKafka Flag indicating if Flink's internal timestamps are written to Kafka.
    +		 */
    +		public void setWriteTimestampToKafka(boolean writeTimestampToKafka) {
    +			this.producer.writeTimestampToKafka = writeTimestampToKafka;
    +		}
    +	}
    +
    +	/**
    +	 * State for handling transactions.
    +	 */
    +	public static class KafkaTransactionState implements Serializable {
    +
    +		private final transient FlinkKafkaProducer<byte[], byte[]> producer;
    +
    +		@Nullable
    +		public final String transactionalId;
    +
    +		public final long producerId;
    +
    +		public final short epoch;
    +
    +		public KafkaTransactionState(String transactionalId, FlinkKafkaProducer<byte[], byte[]> producer) {
    +			this.producer = producer;
    +			this.transactionalId = transactionalId;
    +			this.producerId = producer.getProducerId();
    +			this.epoch = producer.getEpoch();
    +		}
    +
    +		public KafkaTransactionState(FlinkKafkaProducer<byte[], byte[]> producer) {
    +			this.producer = producer;
    +			this.transactionalId = null;
    +			this.producerId = -1;
    +			this.epoch = -1;
    +		}
    +	}
    +
    +	private static class KafkaStreamSink<IN> extends StreamSink<IN> {
    +		private final FlinkKafkaProducer011<IN> kafkaProducer;
    +
    +		public KafkaStreamSink(FlinkKafkaProducer011<IN> kafkaProducer) {
    +			super(kafkaProducer);
    +			this.kafkaProducer = kafkaProducer;
    +		}
    +
    +		// TODO: is this used anywhere?
    --- End diff --
    
    I think this variant is used when using `writeToKafkaWithTimestamps`


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r128166188
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -0,0 +1,317 @@
    +/*
    + * 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.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);
    +
    +	@Nullable
    +	protected TXN currentTransaction;
    +
    +	protected final List<TransactionAndCheckpoint<TXN>> pendingCommitTransactions = new ArrayList<>();
    +
    +	protected ListState<TXN> pendingTransactionsState;
    +
    +	protected ListState<TransactionAndCheckpoint<TXN>> pendingCommitTransactionsState;
    +
    +	// ------ 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 (non Pravega sink) tasks could not persist their status during
    --- End diff --
    
    Is it required to mention Pravega here?


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134399156
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/FlinkKafkaProducer.java ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.connectors.kafka.internal;
    +
    +import org.apache.flink.annotation.VisibleForTesting;
    +
    +import org.apache.kafka.clients.consumer.OffsetAndMetadata;
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.KafkaProducer;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.clients.producer.internals.TransactionalRequestResult;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.Node;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.errors.ProducerFencedException;
    +import org.apache.kafka.common.requests.FindCoordinatorRequest;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.lang.reflect.Field;
    +import java.lang.reflect.InvocationTargetException;
    +import java.lang.reflect.Method;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Wrapper around KafkaProducer that allows to resume transactions in case of node failure, which allows to implement
    + * two phase commit algorithm for exactly-once semantic FlinkKafkaProducer.
    + *
    + * <p>For happy path usage is exactly the same as {@link org.apache.kafka.clients.producer.KafkaProducer}. User is
    + * expected to call:
    + *
    + * <ul>
    + *     <li>{@link FlinkKafkaProducer#initTransactions()}</li>
    + *     <li>{@link FlinkKafkaProducer#beginTransaction()}</li>
    + *     <li>{@link FlinkKafkaProducer#send(org.apache.kafka.clients.producer.ProducerRecord)}</li>
    + *     <li>{@link FlinkKafkaProducer#flush()}</li>
    + *     <li>{@link FlinkKafkaProducer#commitTransaction()}</li>
    + * </ul>
    + *
    + * <p>To actually implement two phase commit, it must be possible to always commit a transaction after pre-committing
    + * it (here, pre-commit is just a {@link FlinkKafkaProducer#flush()}). In case of some failure between
    + * {@link FlinkKafkaProducer#flush()} and {@link FlinkKafkaProducer#commitTransaction()} this class allows to resume
    + * interrupted transaction and commit if after a restart:
    + *
    + * <ul>
    + *     <li>{@link FlinkKafkaProducer#initTransactions()}</li>
    + *     <li>{@link FlinkKafkaProducer#beginTransaction()}</li>
    + *     <li>{@link FlinkKafkaProducer#send(org.apache.kafka.clients.producer.ProducerRecord)}</li>
    + *     <li>{@link FlinkKafkaProducer#flush()}</li>
    + *     <li>{@link FlinkKafkaProducer#getProducerId()}</li>
    + *     <li>{@link FlinkKafkaProducer#getEpoch()}</li>
    + *     <li>node failure... restore producerId and epoch from state</li>
    + *     <li>{@link FlinkKafkaProducer#resumeTransaction(long, short)}</li>
    + *     <li>{@link FlinkKafkaProducer#commitTransaction()}</li>
    + * </ul>
    + *
    + * <p>{@link FlinkKafkaProducer#resumeTransaction(long, short)} replaces {@link FlinkKafkaProducer#initTransactions()}
    + * as a way to obtain the producerId and epoch counters. It has to be done, because otherwise
    + * {@link FlinkKafkaProducer#initTransactions()} would automatically abort all on going transactions.
    + *
    + * <p>Second way this implementation differs from the reference {@link org.apache.kafka.clients.producer.KafkaProducer}
    + * is that this one actually flushes new partitions on {@link FlinkKafkaProducer#flush()} instead of on
    + * {@link FlinkKafkaProducer#commitTransaction()}.
    + *
    + * <p>The last one minor difference is that it allows to obtain the producerId and epoch counters via
    + * {@link FlinkKafkaProducer#getProducerId()} and {@link FlinkKafkaProducer#getEpoch()} methods (which are unfortunately
    + * private fields).
    + *
    + * <p>Those changes are compatible with Kafka's 0.11.0 REST API although it clearly was not the intention of the Kafka's
    + * API authors to make them possible.
    + *
    + * <p>Internally this implementation uses {@link org.apache.kafka.clients.producer.KafkaProducer} and implements
    + * required changes via Java Reflection API. It might not be the prettiest solution. An alternative would be to
    + * re-implement whole Kafka's 0.11 REST API client on our own.
    + */
    +public class FlinkKafkaProducer<K, V> implements Producer<K, V> {
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducer.class);
    +
    +	private final KafkaProducer<K, V> kafkaProducer;
    +	@Nullable
    +	private final String transactionalId;
    +
    +	public FlinkKafkaProducer(Properties properties) {
    +		transactionalId = properties.getProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG);
    +		kafkaProducer = new KafkaProducer<>(properties);
    +	}
    +
    +	// -------------------------------- Simple proxy method calls --------------------------------
    +
    +	@Override
    +	public void initTransactions() {
    +		kafkaProducer.initTransactions();
    +	}
    +
    +	@Override
    +	public void beginTransaction() throws ProducerFencedException {
    +		kafkaProducer.beginTransaction();
    +	}
    +
    +	@Override
    +	public void commitTransaction() throws ProducerFencedException {
    +		kafkaProducer.commitTransaction();
    +	}
    +
    +	@Override
    +	public void abortTransaction() throws ProducerFencedException {
    +		kafkaProducer.abortTransaction();
    +	}
    +
    +	@Override
    +	public void sendOffsetsToTransaction(Map<TopicPartition, OffsetAndMetadata> offsets, String consumerGroupId) throws ProducerFencedException {
    +		kafkaProducer.sendOffsetsToTransaction(offsets, consumerGroupId);
    +	}
    +
    +	@Override
    +	public Future<RecordMetadata> send(ProducerRecord<K, V> record) {
    +		return kafkaProducer.send(record);
    +	}
    +
    +	@Override
    +	public Future<RecordMetadata> send(ProducerRecord<K, V> record, Callback callback) {
    +		return kafkaProducer.send(record, callback);
    +	}
    +
    +	@Override
    +	public List<PartitionInfo> partitionsFor(String topic) {
    +		return kafkaProducer.partitionsFor(topic);
    +	}
    +
    +	@Override
    +	public Map<MetricName, ? extends Metric> metrics() {
    +		return kafkaProducer.metrics();
    +	}
    +
    +	@Override
    +	public void close() {
    +		kafkaProducer.close();
    +	}
    +
    +	@Override
    +	public void close(long timeout, TimeUnit unit) {
    +		kafkaProducer.close(timeout, unit);
    +	}
    +
    +	// -------------------------------- New methods or methods with changed behaviour --------------------------------
    +
    +	@Override
    +	public void flush() {
    +		kafkaProducer.flush();
    +		if (transactionalId != null) {
    +			flushNewPartitions();
    +		}
    +	}
    +
    +	public void resumeTransaction(long producerId, short epoch) {
    +		if (!(producerId >= 0 && epoch >= 0)) {
    +			throw new IllegalStateException(String.format("Incorrect values for producerId [%s] and epoch [%s]", producerId, epoch));
    --- End diff --
    
    Can use `Preconditions.checkState(...)` here.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    `df6d5e0` to `5ff8106` should definitely be squashed, I left them only to make it easier for reviewers to follow the changes made in 0.11 vs 0.10 connectors (those changes would be invisible in one blob commit).
    
    For `321a142` to `2cf5f3b` I'm not sure about the first one, `FlinkKafkaProducer` is that hacky that it could deserve separate commit. It would make it stand out more if anyone in the future would look at the commit history/changes (it could hide in larger change).


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134396258
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    +		ClosureCleaner.clean(customPartitioner, true);
    +		ClosureCleaner.ensureSerializable(serializationSchema);
    +
    +		this.defaultTopicId = defaultTopicId;
    +		this.schema = serializationSchema;
    +		this.producerConfig = producerConfig;
    +		this.flinkKafkaPartitioner = customPartitioner;
    +		this.semantic = semantic;
    +		this.kafkaProducersPoolSize = kafkaProducersPoolSize;
    +
    +		// set the producer configuration properties for kafka record key value serializers.
    +		if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		// eagerly ensure that bootstrap servers are set.
    +		if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
    +			throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must be supplied in the producer config properties.");
    +		}
    +
    +		this.topicPartitionsMap = new HashMap<>();
    +	}
    +
    +	// ---------------------------------- Properties --------------------------
    +
    +	/**
    +	 * Defines whether the producer should fail on errors, or only log them.
    +	 * If this is set to true, then exceptions will be only logged, if set to false,
    +	 * exceptions will be eventually thrown and cause the streaming program to
    +	 * fail (and enter recovery).
    +	 *
    +	 * <p>Method is only accessible for approach (a) (see above)
    +	 *
    +	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
    +	 */
    +	public void setLogFailuresOnly(boolean logFailuresOnly) {
    +		this.logFailuresOnly = logFailuresOnly;
    +	}
    +
    +	// ----------------------------------- Utilities --------------------------
    +
    +	/**
    +	 * Initializes the connection to Kafka.
    +	 *
    +	 * <p>This method is used for approach (a) (see above).
    +	 */
    +	@Override
    +	public void open(Configuration configuration) throws Exception {
    +		if (semantic != Semantic.NONE && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) {
    +			LOG.warn(String.format("Using [%s] semantic, but checkpointing is not enabled. Switching to [%s] semantic.", semantic, Semantic.NONE));
    --- End diff --
    
    Can we use `{}` instead of `[%s]`? Just for consistency with other loggings.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134398289
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    +		ClosureCleaner.clean(customPartitioner, true);
    +		ClosureCleaner.ensureSerializable(serializationSchema);
    +
    +		this.defaultTopicId = defaultTopicId;
    +		this.schema = serializationSchema;
    +		this.producerConfig = producerConfig;
    +		this.flinkKafkaPartitioner = customPartitioner;
    +		this.semantic = semantic;
    +		this.kafkaProducersPoolSize = kafkaProducersPoolSize;
    +
    +		// set the producer configuration properties for kafka record key value serializers.
    +		if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) {
    +			this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName());
    +		} else {
    +			LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
    +		}
    +
    +		// eagerly ensure that bootstrap servers are set.
    +		if (!this.producerConfig.containsKey(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)) {
    +			throw new IllegalArgumentException(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + " must be supplied in the producer config properties.");
    +		}
    +
    +		this.topicPartitionsMap = new HashMap<>();
    +	}
    +
    +	// ---------------------------------- Properties --------------------------
    +
    +	/**
    +	 * Defines whether the producer should fail on errors, or only log them.
    +	 * If this is set to true, then exceptions will be only logged, if set to false,
    +	 * exceptions will be eventually thrown and cause the streaming program to
    +	 * fail (and enter recovery).
    +	 *
    +	 * <p>Method is only accessible for approach (a) (see above)
    +	 *
    +	 * @param logFailuresOnly The flag to indicate logging-only on exceptions.
    +	 */
    +	public void setLogFailuresOnly(boolean logFailuresOnly) {
    +		this.logFailuresOnly = logFailuresOnly;
    +	}
    +
    +	// ----------------------------------- Utilities --------------------------
    +
    +	/**
    +	 * Initializes the connection to Kafka.
    +	 *
    +	 * <p>This method is used for approach (a) (see above).
    +	 */
    +	@Override
    +	public void open(Configuration configuration) throws Exception {
    +		if (semantic != Semantic.NONE && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) {
    +			LOG.warn(String.format("Using [%s] semantic, but checkpointing is not enabled. Switching to [%s] semantic.", semantic, Semantic.NONE));
    +			semantic = Semantic.NONE;
    +		}
    +
    +		if (logFailuresOnly) {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception e) {
    +					if (e != null) {
    +						LOG.error("Error while sending record to Kafka: " + e.getMessage(), e);
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +		else {
    +			callback = new Callback() {
    +				@Override
    +				public void onCompletion(RecordMetadata metadata, Exception exception) {
    +					if (exception != null && asyncException == null) {
    +						asyncException = exception;
    +					}
    +					acknowledgeMessage();
    +				}
    +			};
    +		}
    +
    +		super.open(configuration);
    +	}
    +
    +	@Override
    +	public void invoke(KafkaTransactionState transaction, IN next) throws Exception {
    +		invokeInternal(transaction, next, Long.MAX_VALUE);
    +	}
    +
    +	private void invokeInternal(KafkaTransactionState transaction, IN next, long elementTimestamp) throws Exception {
    +		checkErroneous();
    +
    +		byte[] serializedKey = schema.serializeKey(next);
    +		byte[] serializedValue = schema.serializeValue(next);
    +		String targetTopic = schema.getTargetTopic(next);
    +		if (targetTopic == null) {
    +			targetTopic = defaultTopicId;
    +		}
    +
    +		Long timestamp = null;
    +		if (this.writeTimestampToKafka) {
    +			timestamp = elementTimestamp;
    +		}
    +
    +		ProducerRecord<byte[], byte[]> record;
    +		int[] partitions = topicPartitionsMap.get(targetTopic);
    +		if (null == partitions) {
    +			partitions = getPartitionsByTopic(targetTopic, transaction.producer);
    +			topicPartitionsMap.put(targetTopic, partitions);
    +		}
    +		if (flinkKafkaPartitioner == null) {
    +			record = new ProducerRecord<>(targetTopic, null, timestamp, serializedKey, serializedValue);
    +		} else {
    +			record = new ProducerRecord<>(targetTopic, flinkKafkaPartitioner.partition(next, serializedKey, serializedValue, targetTopic, partitions), timestamp, serializedKey, serializedValue);
    +		}
    +		pendingRecords.incrementAndGet();
    +		transaction.producer.send(record, callback);
    +	}
    +
    +	@Override
    +	public void close() throws Exception {
    +		if (currentTransaction != null) {
    +			// to avoid exceptions on aborting transactions with some pending records
    +			flush(currentTransaction);
    +		}
    +		try {
    +			super.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		try {
    +			producersPool.close();
    +		}
    +		catch (Exception e) {
    +			asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException);
    +		}
    +		// make sure we propagate pending errors
    +		checkErroneous();
    +	}
    +
    +	// ------------------- Logic for handling checkpoint flushing -------------------------- //
    +
    +	@Override
    +	protected KafkaTransactionState beginTransaction() throws Exception {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				FlinkKafkaProducer<byte[], byte[]> producer = producersPool.poll();
    +				if (producer == null) {
    +					String transactionalId = availableTransactionalIds.poll();
    +					if (transactionalId == null) {
    +						throw new Exception(
    +							"Too many ongoing snapshots. Increase kafka producers pool size or decrease number of concurrent checktpoins.");
    +					}
    +					usedTransactionalIds.add(transactionalId);
    +					producer = initTransactionalProducer(transactionalId, true);
    +					producer.initTransactions();
    +				}
    +				producer.beginTransaction();
    +				return new KafkaTransactionState(producer.getTransactionalId(), producer);
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				// Do not create new producer on each beginTransaction() if it is not necessary
    +				if (currentTransaction != null && currentTransaction.producer != null) {
    +					return new KafkaTransactionState(currentTransaction.producer);
    +				}
    +				return new KafkaTransactionState(initProducer(true));
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void preCommit(KafkaTransactionState transaction) throws Exception {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +			case AT_LEAST_ONCE:
    +				flush(transaction);
    +				break;
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +		checkErroneous();
    +	}
    +
    +	@Override
    +	protected void commit(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				transaction.producer.commitTransaction();
    +				producersPool.add(transaction.producer);
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void recoverAndCommit(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				KafkaTransactionState kafkaTransaction = transaction;
    +				FlinkKafkaProducer<byte[], byte[]> producer =
    +					initTransactionalProducer(kafkaTransaction.transactionalId, false);
    +				producer.resumeTransaction(kafkaTransaction.producerId, kafkaTransaction.epoch);
    +				try {
    +					producer.commitTransaction();
    +					producer.close();
    +				}
    +				catch (InvalidTxnStateException ex) {
    +					// That means we have committed this transaction before.
    +					LOG.warn("Encountered error [%s] while recovering transaction [%s]. " +
    +						"Presumably this transaction has been already committed before",
    +						ex,
    +						transaction);
    +				}
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void abort(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				transaction.producer.abortTransaction();
    +				producersPool.add(transaction.producer);
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				producersPool.add(transaction.producer);
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	@Override
    +	protected void recoverAndAbort(KafkaTransactionState transaction) {
    +		switch (semantic) {
    +			case EXACTLY_ONCE:
    +				FlinkKafkaProducer<byte[], byte[]> producer =
    +					initTransactionalProducer(transaction.transactionalId, false);
    +				producer.resumeTransaction(transaction.producerId, transaction.epoch);
    +				producer.abortTransaction();
    +				producer.close();
    +				break;
    +			case AT_LEAST_ONCE:
    +			case NONE:
    +				break;
    +			default:
    +				throw new UnsupportedOperationException("Not implemented semantic");
    +		}
    +	}
    +
    +	private void acknowledgeMessage() {
    +		pendingRecords.decrementAndGet();
    +	}
    +
    +	/**
    +	 * Flush pending records.
    +	 * @param transaction
    +	 */
    +	private void flush(KafkaTransactionState transaction) throws Exception {
    +		if (transaction.producer != null) {
    +			transaction.producer.flush();
    +		}
    +		long pendingRecordsCount = pendingRecords.get();
    +		if (pendingRecordsCount != 0) {
    +			throw new IllegalStateException("Pending record count must be zero at this point: " + pendingRecordsCount);
    +		}
    +
    +		// if the flushed requests has errors, we should propagate it also and fail the checkpoint
    +		checkErroneous();
    +	}
    +
    +	@Override
    +	public void snapshotState(FunctionSnapshotContext context) throws Exception {
    +		super.snapshotState(context);
    +
    +		transactionalIdsState.clear();
    +		for (String transactionalId : availableTransactionalIds) {
    +			transactionalIdsState.add(transactionalId);
    +		}
    +		for (String transactionalId : usedTransactionalIds) {
    +			transactionalIdsState.add(transactionalId);
    +		}
    +	}
    +
    +	@Override
    +	public void initializeState(FunctionInitializationContext context) throws Exception {
    +		availableTransactionalIds.clear();
    +		for (int i = 0; i < kafkaProducersPoolSize; i++) {
    +			availableTransactionalIds.add(UUID.randomUUID().toString());
    --- End diff --
    
    I think that makes sense, but I guess its mostly due to that the current code isn't differentiating between used and unused transaction ids in the state. If we differentiate that, it would be possible to reuse stored ids.
    
    Piotr, 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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134396090
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    +		ClosureCleaner.clean(customPartitioner, true);
    +		ClosureCleaner.ensureSerializable(serializationSchema);
    +
    +		this.defaultTopicId = defaultTopicId;
    +		this.schema = serializationSchema;
    +		this.producerConfig = producerConfig;
    +		this.flinkKafkaPartitioner = customPartitioner;
    +		this.semantic = semantic;
    --- End diff --
    
    There is no argument value check on semantic (it could have been null)


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    Indeed it seems like you are right. `read_committed` doesn't play along with long `max.transaction.timeout.ms`. I'm not sure about Beam, but in Flink we can not use one single `transactional.id`, because our checkpoints are asynchronous - `notifyCheckpointComplete` (which triggers `KafkaProducer#commit`) can come long after `preCommit`. In that time we can not use the same `transactional.id` for new transactions. 
    
    We can walk around this issue by implementing a pool of `transactional.id`s, which we can save on the state. This will allows on restoring state to not only `recoverAndCommit` all pending transactions, but to abort all other unknown "lingering" transactions


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134395636
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * Descriptor of the transacionalIds list.
    +	 */
    +	private static final ListStateDescriptor<String> TRANSACTIONAL_IDS_DESCRIPTOR =
    +		new ListStateDescriptor<>("transactional-ids", TypeInformation.of(String.class));
    +
    +	/**
    +	 * Pool of transacional ids backed up in state.
    +	 */
    +	private ListState<String> transactionalIdsState;
    +
    +	/**
    +	 * Already used transactional ids.
    +	 */
    +	private final Set<String> usedTransactionalIds = new HashSet<>();
    +
    +	/**
    +	 * Available to use transactional ids.
    +	 */
    +	private final BlockingDeque<String> availableTransactionalIds = new LinkedBlockingDeque<>();
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Max number of producers in the pool. If all producers are in use, snapshoting state will throw an exception.
    +	 */
    +	private final int kafkaProducersPoolSize;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	/**
    +	 * Semantic chosen for this instance.
    +	 */
    +	private Semantic semantic;
    +
    +	/**
    +	 * Pool of KafkaProducers objects.
    +	 */
    +	private transient ProducersPool producersPool = new ProducersPool();
    +
    +	// -------------------------------- Runtime fields ------------------------------------------
    +
    +	/** The callback than handles error propagation or logging callbacks. */
    +	@Nullable
    +	private transient Callback callback;
    +
    +	/** Errors encountered in the async producer are stored here. */
    +	@Nullable
    +	private transient volatile Exception asyncException;
    +
    +	/** Lock for accessing the pending records. */
    +	private final SerializableObject pendingRecordsLock = new SerializableObject();
    +
    +	/** Number of unacknowledged records. */
    +	private final AtomicLong pendingRecords = new AtomicLong();
    +
    +	/** Cache of metrics to replace already registered metrics instead of overwriting existing ones. */
    +	private final Map<String, KafkaMetricMuttableWrapper> previouslyCreatedMetrics = new HashMap<>();
    +
    +	// ---------------------- "Constructors" for timestamp writing ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined serialization schema supporting key/value messages
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 * @param inStream The stream to write to Kafka
    +	 * @param topicId ID of the Kafka topic.
    +	 * @param serializationSchema User defined (keyless) serialization schema.
    +	 * @param producerConfig Properties with the producer configuration.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					SerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig) {
    +		return writeToKafkaWithTimestamps(inStream, topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner) {
    +		return writeToKafkaWithTimestamps(
    +			inStream,
    +			topicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * <p>This constructor allows writing timestamps to Kafka, it follow approach (b) (see above)
    +	 *
    +	 *  @param inStream The stream to write to Kafka
    +	 *  @param topicId The name of the target topic
    +	 *  @param serializationSchema A serializable serialization schema for turning user objects into a
    +	 *                             kafka-consumable byte[] supporting key/value messages
    +	 *  @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only
    +	 *                        required argument.
    +	 *  @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions.
    +	 *  @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 *  @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public static <IN> FlinkKafkaProducer011Configuration<IN> writeToKafkaWithTimestamps(DataStream<IN> inStream,
    +																					String topicId,
    +																					KeyedSerializationSchema<IN> serializationSchema,
    +																					Properties producerConfig,
    +																					FlinkKafkaPartitioner<IN> customPartitioner,
    +																					Semantic semantic,
    +																					int kafkaProducersPoolSize) {
    +
    +		GenericTypeInfo<Object> objectTypeInfo = new GenericTypeInfo<>(Object.class);
    +		FlinkKafkaProducer011<IN> kafkaProducer =
    +			new FlinkKafkaProducer011<>(
    +				topicId,
    +				serializationSchema,
    +				producerConfig,
    +				customPartitioner,
    +				semantic,
    +				kafkaProducersPoolSize);
    +		KafkaStreamSink streamSink = new KafkaStreamSink(kafkaProducer);
    +		SingleOutputStreamOperator<Object> transformation = inStream.transform("FlinKafkaProducer 0.11.x", objectTypeInfo, streamSink);
    +		return new FlinkKafkaProducer011Configuration<>(transformation, streamSink);
    +	}
    +
    +	// ---------------------- Regular constructors w/o timestamp support  ------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, SerializationSchema<IN> serializationSchema) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined (keyless) serialization schema.
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. the sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId The topic to write data to
    +	 * @param serializationSchema A (keyless) serializable serialization schema for turning user objects into a kafka-consumable byte[]
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions (when passing null, we'll use Kafka's partitioner)
    +	 */
    +	public FlinkKafkaProducer011(String topicId, SerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(topicId, new KeyedSerializationSchemaWrapper<>(serializationSchema), producerConfig, customPartitioner);
    +	}
    +
    +	// ------------------- Key/Value serialization schema constructors ----------------------
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param brokerList
    +	 *			Comma separated addresses of the brokers
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 */
    +	public FlinkKafkaProducer011(String brokerList, String topicId, KeyedSerializationSchema<IN> serializationSchema) {
    +		this(topicId, serializationSchema, getPropertiesFromBrokerList(brokerList), new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * Creates a FlinkKafkaProducer for a given topic. The sink produces a DataStream to
    +	 * the topic.
    +	 *
    +	 * @param topicId
    +	 * 			ID of the Kafka topic.
    +	 * @param serializationSchema
    +	 * 			User defined serialization schema supporting key/value messages
    +	 * @param producerConfig
    +	 * 			Properties with the producer configuration.
    +	 */
    +	public FlinkKafkaProducer011(String topicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig) {
    +		this(topicId, serializationSchema, producerConfig, new FlinkFixedPartitioner<IN>());
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 */
    +	public FlinkKafkaProducer011(String defaultTopicId, KeyedSerializationSchema<IN> serializationSchema, Properties producerConfig, FlinkKafkaPartitioner<IN> customPartitioner) {
    +		this(
    +			defaultTopicId,
    +			serializationSchema,
    +			producerConfig,
    +			customPartitioner,
    +			Semantic.EXACTLY_ONCE,
    +			DEFAULT_KAFKA_PRODUCERS_POOL_SIZE);
    +	}
    +
    +	/**
    +	 * The main constructor for creating a FlinkKafkaProducer.
    +	 *
    +	 * <p>This constructor does not allow writing timestamps to Kafka, it follow approach (a) (see above)
    +	 *
    +	 * @param defaultTopicId The default topic to write data to
    +	 * @param serializationSchema A serializable serialization schema for turning user objects into a kafka-consumable byte[] supporting key/value messages
    +	 * @param producerConfig Configuration properties for the KafkaProducer. 'bootstrap.servers.' is the only required argument.
    +	 * @param customPartitioner A serializable partitioner for assigning messages to Kafka partitions. Passing null will use Kafka's partitioner.
    +	 * @param semantic Defines semantic that will be used by this producer (see {@link Semantic}).
    +	 * @param kafkaProducersPoolSize Overwrite default KafkaProducers pool size (see {@link Semantic#EXACTLY_ONCE}).
    +	 */
    +	public FlinkKafkaProducer011(
    +			String defaultTopicId,
    +			KeyedSerializationSchema<IN> serializationSchema,
    +			Properties producerConfig,
    +			FlinkKafkaPartitioner<IN> customPartitioner,
    +			Semantic semantic,
    +			int kafkaProducersPoolSize) {
    +		super(
    +			TypeInformation.of(KafkaTransactionState.class),
    +			TypeInformation.of(new TypeHint<List<KafkaTransactionState>>() {}));
    +
    +		requireNonNull(defaultTopicId, "TopicID not set");
    +		requireNonNull(serializationSchema, "serializationSchema not set");
    +		requireNonNull(producerConfig, "producerConfig not set");
    --- End diff --
    
    In Flink we usually use `Preconditions.checkNotNull` for this.


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134394729
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    --- End diff --
    
    nit: empty line before comment block.



---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exactly-once...

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

    https://github.com/apache/flink/pull/4239
  
    What were the bugs that you fixed?


---

[GitHub] flink pull request #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r128164632
  
    --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---
    @@ -0,0 +1,317 @@
    +/*
    + * 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.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 really like the idea of introducing this abstraction :)


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r134673133
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,1000 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +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.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.errors.InvalidTxnStateException;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.UUID;
    +import java.util.concurrent.BlockingDeque;
    +import java.util.concurrent.LinkedBlockingDeque;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic#EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular
    + * {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} (a) and a custom operator (b).
    + *
    + * <p>Details about approach (a):
    + *  Because of regular {@link org.apache.flink.streaming.api.functions.sink.SinkFunction} APIs limitations, this
    + *  variant do not allow accessing the timestamp attached to the record.
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When using the
    + *  {@link FlinkKafkaProducer011#writeToKafkaWithTimestamps} method, the Kafka producer can access the internal
    + *  record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 *
    +		 * <p>In this mode {@link FlinkKafkaProducer011} sets up a pool of {@link FlinkKafkaProducer}. Between each
    +		 * checkpoint there is created new Kafka transaction, which is being committed on
    +		 * {@link FlinkKafkaProducer011#notifyCheckpointComplete(long)}. If checkpoint complete notifications are
    +		 * running late, {@link FlinkKafkaProducer011} can run out of {@link FlinkKafkaProducer}s in the pool. In that
    +		 * case any subsequent {@link FlinkKafkaProducer011#snapshotState(FunctionSnapshotContext)} requests will fail
    +		 * and {@link FlinkKafkaProducer011} will keep using the {@link FlinkKafkaProducer} from previous checkpoint.
    +		 * To decrease chances of failing checkpoints there are three options:
    +		 * <li>decrease number of max concurrent checkpoints</li>
    +		 * <li>make checkpoints more reliable (so that they complete faster)</li>
    +		 * <li>increase delay between checkpoints</li>
    +		 * <li>increase size of {@link FlinkKafkaProducer}s pool</li>
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed. Messages can be lost and/or duplicated in case
    +		 * of failure.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Default number of KafkaProducers in the pool. See {@link Semantic#EXACTLY_ONCE}.
    +	 */
    +	public static final int DEFAULT_KAFKA_PRODUCERS_POOL_SIZE = 5;
    --- End diff --
    
    As I remember the reason was that it is not easy/not possible at the moment to get this information in the operator. It should be a follow up work. Regardless of this, code of this operator would look the same (because we don't have guarantees for the `notifyCheckpointComplete` to always reach us on time).


---
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 #4239: [FLINK-6988] flink-connector-kafka-0.11 with exact...

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

    https://github.com/apache/flink/pull/4239#discussion_r128164127
  
    --- Diff: flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java ---
    @@ -0,0 +1,818 @@
    +/*
    + * 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.connectors.kafka;
    +
    +import org.apache.flink.api.common.functions.RuntimeContext;
    +import org.apache.flink.api.java.ClosureCleaner;
    +import org.apache.flink.api.java.typeutils.GenericTypeInfo;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.MetricGroup;
    +import org.apache.flink.runtime.util.SerializableObject;
    +import org.apache.flink.streaming.api.datastream.DataStream;
    +import org.apache.flink.streaming.api.datastream.DataStreamSink;
    +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
    +import org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction;
    +import org.apache.flink.streaming.api.operators.StreamSink;
    +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
    +import org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer;
    +import org.apache.flink.streaming.connectors.kafka.internal.metrics.KafkaMetricMuttableWrapper;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
    +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SerializationSchema;
    +import org.apache.flink.util.ExceptionUtils;
    +import org.apache.flink.util.NetUtils;
    +
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.annotation.Nullable;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.UUID;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static java.util.Objects.requireNonNull;
    +
    +/**
    + * Flink Sink to produce data into a Kafka topic. This producer is compatible with Kafka 0.11.x. By default producer
    + * will use {@link Semantic.EXACTLY_ONCE} semantic.
    + *
    + * <p>Implementation note: This producer is a hybrid between a regular regular sink function (a)
    + * and a custom operator (b).
    + *
    + * <p>For (a), the class implements the SinkFunction and RichFunction interfaces.
    + * For (b), it extends the StreamTask class.
    + *
    + * <p>Details about approach (a):
    + *  Pre Kafka 0.11 producers only follow approach (a), allowing users to use the producer using the
    + *  DataStream.addSink() method.
    + *  Since the APIs exposed in that variant do not allow accessing the the timestamp attached to the record
    + *  the Kafka 0.11 producer has a second invocation option, approach (b).
    + *
    + * <p>Details about approach (b):
    + *  Kafka 0.11 supports writing the timestamp attached to a record to Kafka. When adding the
    + *  FlinkKafkaProducer011 using the FlinkKafkaProducer011.writeToKafkaWithTimestamps() method, the Kafka producer
    + *  can access the internal record timestamp of the record and write it to Kafka.
    + *
    + * <p>All methods and constructors in this class are marked with the approach they are needed for.
    + */
    +public class FlinkKafkaProducer011<IN>
    +		extends TwoPhaseCommitSinkFunction<IN, FlinkKafkaProducer011.KafkaTransactionState> {
    +	/**
    +	 *  Semantics that can be chosen.
    +	 *  <li>{@link #EXACTLY_ONCE}</li>
    +	 *  <li>{@link #AT_LEAST_ONCE}</li>
    +	 *  <li>{@link #NONE}</li>
    +	 */
    +	public enum Semantic {
    +		/**
    +		 * Semantic.EXACTLY_ONCE the Flink producer will write all messages in a Kafka transaction that will be
    +		 * committed to the Kafka on a checkpoint.
    +		 */
    +		EXACTLY_ONCE,
    +		/**
    +		 * Semantic.AT_LEAST_ONCE the Flink producer will wait for all outstanding messages in the Kafka buffers
    +		 * to be acknowledged by the Kafka producer on a checkpoint.
    +		 */
    +		AT_LEAST_ONCE,
    +		/**
    +		 * Semantic.NONE means that nothing will be guaranteed.
    +		 */
    +		NONE
    +	}
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(FlinkKafkaProducerBase.class);
    +
    +	private static final long serialVersionUID = 1L;
    +
    +	/**
    +	 * Configuration key for disabling the metrics reporting.
    +	 */
    +	public static final String KEY_DISABLE_METRICS = "flink.disable-metrics";
    +
    +	/**
    +	 * User defined properties for the Producer.
    +	 */
    +	private final Properties producerConfig;
    +
    +	/**
    +	 * The name of the default topic this producer is writing data to.
    +	 */
    +	private final String defaultTopicId;
    +
    +	/**
    +	 * (Serializable) SerializationSchema for turning objects used with Flink into.
    +	 * byte[] for Kafka.
    +	 */
    +	private final KeyedSerializationSchema<IN> schema;
    +
    +	/**
    +	 * User-provided partitioner for assigning an object to a Kafka partition for each topic.
    +	 */
    +	private final FlinkKafkaPartitioner<IN> flinkKafkaPartitioner;
    +
    +	/**
    +	 * Partitions of each topic.
    +	 */
    +	private final Map<String, int[]> topicPartitionsMap;
    +
    +	/**
    +	 * Flag controlling whether we are writing the Flink record's timestamp into Kafka.
    +	 */
    +	private boolean writeTimestampToKafka = false;
    +
    +	/**
    +	 * Flag indicating whether to accept failures (and log them), or to fail on failures.
    +	 */
    +	private boolean logFailuresOnly;
    +
    +	private Semantic semantic;
    --- End diff --
    
    nit: also include Javadoc for consistency.


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