yanghua commented on a change in pull request #6703: [FLINK-9697] Provide 
connector for Kafka 2.0.0
URL: https://github.com/apache/flink/pull/6703#discussion_r224976123
 
 

 ##########
 File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaFetcher.java
 ##########
 @@ -0,0 +1,238 @@
+/*
+ * 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.Internal;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
+import 
org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
+import 
org.apache.flink.streaming.connectors.kafka.internals.KafkaCommitCallback;
+import 
org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
+import 
org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionState;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import 
org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
+import org.apache.flink.util.SerializedValue;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A fetcher that fetches data from Kafka brokers via the Kafka 2.0 consumer 
API.
+ *
+ * @param <T> The type of elements produced by the fetcher.
+ */
+@Internal
+public class KafkaFetcher<T> extends AbstractFetcher<T, TopicPartition> {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(KafkaFetcher.class);
+
+       // 
------------------------------------------------------------------------
+
+       /** The schema to convert between Kafka's byte messages, and Flink's 
objects. */
+       private final KeyedDeserializationSchema<T> deserializer;
+
+       /** The handover of data and exceptions between the consumer thread and 
the task thread. */
+       private final Handover handover;
+
+       /** The thread that runs the actual KafkaConsumer and hand the record 
batches to this fetcher. */
+       private final KafkaConsumerThread consumerThread;
+
+       /** Flag to mark the main work loop as alive. */
+       private volatile boolean running = true;
+
+       // 
------------------------------------------------------------------------
+
+       public KafkaFetcher(
+               SourceFunction.SourceContext<T> sourceContext,
+               Map<KafkaTopicPartition, Long> 
assignedPartitionsWithInitialOffsets,
+               SerializedValue<AssignerWithPeriodicWatermarks<T>> 
watermarksPeriodic,
+               SerializedValue<AssignerWithPunctuatedWatermarks<T>> 
watermarksPunctuated,
+               ProcessingTimeService processingTimeProvider,
+               long autoWatermarkInterval,
+               ClassLoader userCodeClassLoader,
+               String taskNameWithSubtasks,
+               KeyedDeserializationSchema<T> deserializer,
+               Properties kafkaProperties,
+               long pollTimeout,
+               MetricGroup subtaskMetricGroup,
+               MetricGroup consumerMetricGroup,
+               boolean useMetrics) throws Exception {
+               super(
+                       sourceContext,
+                       assignedPartitionsWithInitialOffsets,
+                       watermarksPeriodic,
+                       watermarksPunctuated,
+                       processingTimeProvider,
+                       autoWatermarkInterval,
+                       userCodeClassLoader,
+                       consumerMetricGroup,
+                       useMetrics);
+
+               this.deserializer = deserializer;
+               this.handover = new Handover();
+
+               this.consumerThread = new KafkaConsumerThread(
+                       LOG,
+                       handover,
+                       kafkaProperties,
+                       unassignedPartitionsQueue,
+                       createCallBridge(),
+                       getFetcherName() + " for " + taskNameWithSubtasks,
+                       pollTimeout,
+                       useMetrics,
+                       consumerMetricGroup,
+                       subtaskMetricGroup);
+       }
+
+       // 
------------------------------------------------------------------------
+       //  Fetcher work methods
+       // 
------------------------------------------------------------------------
+
+       @Override
+       public void runFetchLoop() throws Exception {
+               try {
+                       final Handover handover = this.handover;
+
+                       // kick off the actual Kafka consumer
+                       consumerThread.start();
+
+                       while (running) {
+                               // this blocks until we get the next records
+                               // it automatically re-throws exceptions 
encountered in the consumer thread
+                               final ConsumerRecords<byte[], byte[]> records = 
handover.pollNext();
+
+                               // get the records for each topic partition
+                               for (KafkaTopicPartitionState<TopicPartition> 
partition : subscribedPartitionStates()) {
+
+                                       List<ConsumerRecord<byte[], byte[]>> 
partitionRecords =
+                                               
records.records(partition.getKafkaPartitionHandle());
+
+                                       for (ConsumerRecord<byte[], byte[]> 
record : partitionRecords) {
+                                               final T value = 
deserializer.deserialize(
+                                                       record.key(), 
record.value(),
+                                                       record.topic(), 
record.partition(), record.offset());
+
+                                               if 
(deserializer.isEndOfStream(value)) {
+                                                       // end of stream 
signaled
+                                                       running = false;
+                                                       break;
+                                               }
+
+                                               // emit the actual record. this 
also updates offset state atomically
+                                               // and deals with timestamps 
and watermark generation
+                                               emitRecord(value, partition, 
record.offset(), record);
+                                       }
+                               }
+                       }
+               }
+               finally {
+                       // this signals the consumer thread that no more work 
is to be done
+                       consumerThread.shutdown();
+               }
+
+               // on a clean exit, wait for the runner thread
+               try {
+                       consumerThread.join();
+               }
+               catch (InterruptedException e) {
+                       // may be the result of a wake-up interruption after an 
exception.
+                       // we ignore this here and only restore the 
interruption state
+                       Thread.currentThread().interrupt();
+               }
+       }
+
+       @Override
+       public void cancel() {
+               // flag the main thread to exit. A thread interrupt will come 
anyways.
+               running = false;
+               handover.close();
+               consumerThread.shutdown();
+       }
+
+       protected void emitRecord(
+               T record,
+               KafkaTopicPartitionState<TopicPartition> partition,
+               long offset,
+               ConsumerRecord<?, ?> consumerRecord) throws Exception {
+
+               emitRecordWithTimestamp(record, partition, offset, 
consumerRecord.timestamp());
+       }
+
+       /**
+        * Gets the name of this fetcher, for thread naming and logging 
purposes.
+        */
+       protected String getFetcherName() {
+               return "Kafka 2.0 Fetcher";
 
 Review comment:
   Can we give it a unified name? What about `Kafka Fetcher`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to