[
https://issues.apache.org/jira/browse/FLINK-6988?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16092691#comment-16092691
]
ASF GitHub Bot commented on FLINK-6988:
---------------------------------------
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.
> Add Apache Kafka 0.11 connector
> -------------------------------
>
> Key: FLINK-6988
> URL: https://issues.apache.org/jira/browse/FLINK-6988
> Project: Flink
> Issue Type: Improvement
> Components: Kafka Connector
> Affects Versions: 1.3.1
> Reporter: Piotr Nowojski
> Assignee: Piotr Nowojski
>
> Kafka 0.11 (it will be released very soon) add supports for transactions.
> Thanks to that, Flink might be able to implement Kafka sink supporting
> "exactly-once" semantic. API changes and whole transactions support is
> described in
> [KIP-98|https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging].
> The goal is to mimic implementation of existing BucketingSink. New
> FlinkKafkaProducer011 would
> * upon creation begin transaction, store transaction identifiers into the
> state and would write all incoming data to an output Kafka topic using that
> transaction
> * on `snapshotState` call, it would flush the data and write in state
> information that current transaction is pending to be committed
> * on `notifyCheckpointComplete` we would commit this pending transaction
> * in case of crash between `snapshotState` and `notifyCheckpointComplete` we
> either abort this pending transaction (if not every participant successfully
> saved the snapshot) or restore and commit it.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)