[
https://issues.apache.org/jira/browse/FLINK-5048?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15657656#comment-15657656
]
ASF GitHub Bot commented on FLINK-5048:
---------------------------------------
Github user tzulitai commented on a diff in the pull request:
https://github.com/apache/flink/pull/2789#discussion_r87621040
--- Diff:
flink-streaming-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java
---
@@ -0,0 +1,325 @@
+/*
+ * 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.metrics.MetricGroup;
+import
org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartitionState;
+import
org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricWrapper;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.WakeupException;
+
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The thread the runs the {@link KafkaConsumer}, connecting to the
brokers and polling records.
+ * The thread pushes the data into a {@link Handover} to be picked up by
the fetcher that will
+ * deserialize and emit the records.
+ *
+ * <p><b>IMPORTANT:</b> This thread must not be interrupted when
attempting to shut it down.
+ * The Kafka consumer code was found to not always handle interrupts well,
and to even
+ * deadlock in certain situations.
+ *
+ * <p>Implementation Note: This code is written to be reusable in later
versions of the KafkaConsumer.
+ * Because Kafka is not maintaining binary compatibility, we use a "call
bridge" as an indirection
+ * to the KafkaConsumer calls that change signature.
+ */
+public class KafkaConsumerThread extends Thread {
+
+ /** Logger for this consumer */
+ final Logger log;
--- End diff --
Private?
> Kafka Consumer (0.9/0.10) threading model leads problematic cancellation
> behavior
> ---------------------------------------------------------------------------------
>
> Key: FLINK-5048
> URL: https://issues.apache.org/jira/browse/FLINK-5048
> Project: Flink
> Issue Type: Bug
> Components: Kafka Connector
> Affects Versions: 1.1.3
> Reporter: Stephan Ewen
> Assignee: Stephan Ewen
> Fix For: 1.2.0
>
>
> The {{FLinkKafkaConsumer}} (0.9 / 0.10) spawns a separate thread that
> operates the KafkaConsumer. That thread is shielded from interrupts, because
> the Kafka Consumer has not been handling thread interrupts well.
> Since that thread is also the thread that emits records, it may block in the
> network stack (backpressure) or in chained operators. The later case leads to
> situations where cancellations get very slow unless that thread would be
> interrupted (which it cannot be).
> I propose to change the thread model as follows:
> - A spawned consumer thread pull from the KafkaConsumer and pushes its
> pulled batch of records into a blocking queue (size one)
> - The main thread of the task will pull the record batches from the
> blocking queue and emit the records.
> This allows actually for some additional I/O overlay while limiting the
> additional memory consumption - only two batches are ever held, one being
> fetched and one being emitted.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)