1818package org .apache .spark .sql .kafka010
1919
2020import java .{util => ju }
21- import java .util .concurrent .Executors
2221
2322import scala .collection .JavaConverters ._
2423import scala .collection .mutable .ArrayBuffer
25- import scala .concurrent .{ExecutionContext , Future }
26- import scala .concurrent .duration .Duration
2724import scala .util .control .NonFatal
2825
2926import org .apache .kafka .clients .consumer .{Consumer , ConsumerConfig , KafkaConsumer , OffsetAndTimestamp }
@@ -33,7 +30,7 @@ import org.apache.spark.SparkEnv
3330import org .apache .spark .internal .Logging
3431import org .apache .spark .scheduler .ExecutorCacheTaskLocation
3532import org .apache .spark .sql .catalyst .util .CaseInsensitiveMap
36- import org .apache .spark .util .{ThreadUtils , UninterruptibleThread }
33+ import org .apache .spark .util .{UninterruptibleThread , UninterruptibleThreadRunner }
3734
3835/**
3936 * This class uses Kafka's own [[KafkaConsumer ]] API to read data offsets from Kafka.
@@ -51,19 +48,13 @@ private[kafka010] class KafkaOffsetReader(
5148 val driverKafkaParams : ju.Map [String , Object ],
5249 readerOptions : CaseInsensitiveMap [String ],
5350 driverGroupIdPrefix : String ) extends Logging {
51+
5452 /**
55- * Used to ensure execute fetch operations execute in an UninterruptibleThread
53+ * [[UninterruptibleThreadRunner ]] ensures that all [[KafkaConsumer ]] communication called in an
54+ * [[UninterruptibleThread ]]. In the case of streaming queries, we are already running in an
55+ * [[UninterruptibleThread ]], however for batch mode this is not the case.
5656 */
57- val kafkaReaderThread = Executors .newSingleThreadExecutor((r : Runnable ) => {
58- val t = new UninterruptibleThread (" Kafka Offset Reader" ) {
59- override def run (): Unit = {
60- r.run()
61- }
62- }
63- t.setDaemon(true )
64- t
65- })
66- val execContext = ExecutionContext .fromExecutorService(kafkaReaderThread)
57+ val uninterruptibleThreadRunner = new UninterruptibleThreadRunner (" Kafka Offset Reader" )
6758
6859 /**
6960 * Place [[groupId ]] and [[nextId ]] here so that they are initialized before any consumer is
@@ -126,14 +117,14 @@ private[kafka010] class KafkaOffsetReader(
126117 * Closes the connection to Kafka, and cleans up state.
127118 */
128119 def close (): Unit = {
129- if (_consumer != null ) runUninterruptibly { stopConsumer() }
130- kafkaReaderThread .shutdown()
120+ if (_consumer != null ) uninterruptibleThreadRunner. runUninterruptibly { stopConsumer() }
121+ uninterruptibleThreadRunner .shutdown()
131122 }
132123
133124 /**
134125 * @return The Set of TopicPartitions for a given topic
135126 */
136- def fetchTopicPartitions (): Set [TopicPartition ] = runUninterruptibly {
127+ def fetchTopicPartitions (): Set [TopicPartition ] = uninterruptibleThreadRunner. runUninterruptibly {
137128 assert(Thread .currentThread().isInstanceOf [UninterruptibleThread ])
138129 // Poll to get the latest assigned partitions
139130 consumer.poll(0 )
@@ -531,7 +522,7 @@ private[kafka010] class KafkaOffsetReader(
531522 private def partitionsAssignedToConsumer (
532523 body : ju.Set [TopicPartition ] => Map [TopicPartition , Long ],
533524 fetchingEarliestOffset : Boolean = false )
534- : Map [TopicPartition , Long ] = runUninterruptibly {
525+ : Map [TopicPartition , Long ] = uninterruptibleThreadRunner. runUninterruptibly {
535526
536527 withRetriesWithoutInterrupt {
537528 // Poll to get the latest assigned partitions
@@ -551,23 +542,6 @@ private[kafka010] class KafkaOffsetReader(
551542 }
552543 }
553544
554- /**
555- * This method ensures that the closure is called in an [[UninterruptibleThread ]].
556- * This is required when communicating with the [[KafkaConsumer ]]. In the case
557- * of streaming queries, we are already running in an [[UninterruptibleThread ]],
558- * however for batch mode this is not the case.
559- */
560- private def runUninterruptibly [T ](body : => T ): T = {
561- if (! Thread .currentThread.isInstanceOf [UninterruptibleThread ]) {
562- val future = Future {
563- body
564- }(execContext)
565- ThreadUtils .awaitResult(future, Duration .Inf )
566- } else {
567- body
568- }
569- }
570-
571545 /**
572546 * Helper function that does multiple retries on a body of code that returns offsets.
573547 * Retries are needed to handle transient failures. For e.g. race conditions between getting
0 commit comments