New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-28367][SS] Use new KafkaConsumer.poll API in Kafka connector #25135
Changes from all commits
ad1863a
f92c758
2968c2a
2ed24d3
69c7ca5
82f1520
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,6 +17,7 @@ | |
|
||
package org.apache.spark.sql.kafka010 | ||
|
||
import java.{time => jt} | ||
import java.{util => ju} | ||
import java.util.concurrent.Executors | ||
|
||
|
@@ -29,7 +30,9 @@ import scala.util.control.NonFatal | |
import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer} | ||
import org.apache.kafka.common.TopicPartition | ||
|
||
import org.apache.spark.SparkEnv | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT | ||
import org.apache.spark.sql.types._ | ||
import org.apache.spark.util.{ThreadUtils, UninterruptibleThread} | ||
|
||
|
@@ -49,6 +52,11 @@ private[kafka010] class KafkaOffsetReader( | |
val driverKafkaParams: ju.Map[String, Object], | ||
readerOptions: Map[String, String], | ||
driverGroupIdPrefix: String) extends Logging { | ||
private val pollTimeoutMs = readerOptions.getOrElse( | ||
KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, | ||
(SparkEnv.get.conf.get(NETWORK_TIMEOUT) * 1000L).toString | ||
).toLong | ||
|
||
/** | ||
* Used to ensure execute fetch operations execute in an UninterruptibleThread | ||
*/ | ||
|
@@ -115,9 +123,7 @@ private[kafka010] class KafkaOffsetReader( | |
*/ | ||
def fetchTopicPartitions(): Set[TopicPartition] = runUninterruptibly { | ||
assert(Thread.currentThread().isInstanceOf[UninterruptibleThread]) | ||
// Poll to get the latest assigned partitions | ||
consumer.poll(0) | ||
val partitions = consumer.assignment() | ||
val partitions = getPartitions() | ||
consumer.pause(partitions) | ||
partitions.asScala.toSet | ||
} | ||
|
@@ -163,9 +169,7 @@ private[kafka010] class KafkaOffsetReader( | |
reportDataLoss: String => Unit): KafkaSourceOffset = { | ||
val fetched = runUninterruptibly { | ||
withRetriesWithoutInterrupt { | ||
// Poll to get the latest assigned partitions | ||
consumer.poll(0) | ||
val partitions = consumer.assignment() | ||
val partitions = getPartitions() | ||
|
||
// Call `position` to wait until the potential offset request triggered by `poll(0)` is | ||
// done. This is a workaround for KAFKA-7703, which an async `seekToBeginning` triggered by | ||
|
@@ -177,7 +181,7 @@ private[kafka010] class KafkaOffsetReader( | |
"If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + | ||
"Use -1 for latest, -2 for earliest, if you don't care.\n" + | ||
s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions.asScala}") | ||
logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") | ||
logDebug(s"Seeking to $partitionOffsets") | ||
|
||
partitionOffsets.foreach { | ||
case (tp, KafkaOffsetRangeLimit.LATEST) => | ||
|
@@ -211,11 +215,9 @@ private[kafka010] class KafkaOffsetReader( | |
*/ | ||
def fetchEarliestOffsets(): Map[TopicPartition, Long] = runUninterruptibly { | ||
withRetriesWithoutInterrupt { | ||
// Poll to get the latest assigned partitions | ||
consumer.poll(0) | ||
val partitions = consumer.assignment() | ||
val partitions = getPartitions() | ||
consumer.pause(partitions) | ||
logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the beginning") | ||
logDebug(s"Seeking to the beginning") | ||
|
||
consumer.seekToBeginning(partitions) | ||
val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap | ||
|
@@ -241,17 +243,15 @@ private[kafka010] class KafkaOffsetReader( | |
def fetchLatestOffsets( | ||
knownOffsets: Option[PartitionOffsetMap]): PartitionOffsetMap = runUninterruptibly { | ||
withRetriesWithoutInterrupt { | ||
// Poll to get the latest assigned partitions | ||
consumer.poll(0) | ||
val partitions = consumer.assignment() | ||
val partitions = getPartitions() | ||
|
||
// Call `position` to wait until the potential offset request triggered by `poll(0)` is | ||
// done. This is a workaround for KAFKA-7703, which an async `seekToBeginning` triggered by | ||
// `poll(0)` may reset offsets that should have been set by another request. | ||
partitions.asScala.map(p => p -> consumer.position(p)).foreach(_ => {}) | ||
|
||
consumer.pause(partitions) | ||
logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the end.") | ||
logDebug(s"Seeking to the end.") | ||
|
||
if (knownOffsets.isEmpty) { | ||
consumer.seekToEnd(partitions) | ||
|
@@ -317,11 +317,8 @@ private[kafka010] class KafkaOffsetReader( | |
} else { | ||
runUninterruptibly { | ||
withRetriesWithoutInterrupt { | ||
// Poll to get the latest assigned partitions | ||
consumer.poll(0) | ||
val partitions = consumer.assignment() | ||
val partitions = getPartitions() | ||
consumer.pause(partitions) | ||
logDebug(s"\tPartitions assigned to consumer: $partitions") | ||
|
||
// Get the earliest offset of each partition | ||
consumer.seekToBeginning(partitions) | ||
|
@@ -419,6 +416,21 @@ private[kafka010] class KafkaOffsetReader( | |
stopConsumer() | ||
_consumer = null // will automatically get reinitialized again | ||
} | ||
|
||
private def getPartitions(): ju.Set[TopicPartition] = { | ||
consumer.poll(jt.Duration.ZERO) | ||
var partitions = consumer.assignment() | ||
val startTimeMs = System.currentTimeMillis() | ||
while (partitions.isEmpty && System.currentTimeMillis() - startTimeMs < pollTimeoutMs) { | ||
gaborgsomogyi marked this conversation as resolved.
Show resolved
Hide resolved
|
||
// Poll to get the latest assigned partitions | ||
consumer.poll(jt.Duration.ofMillis(100)) | ||
gaborgsomogyi marked this conversation as resolved.
Show resolved
Hide resolved
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. So using this new API will pull data to driver. Right? The previous There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good point. While Kafka doc says the behavior of such hack has been indeterministic and Kafka never support it officially, we expect such behavior in any way. I've initiated thread to ask about viable alternatives of There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm aware of that but since the doc says:
I've considered @HeartSaVioR thanks for initiating the discussion and let's see where it goes. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah you're right that's also true as well. One thing slightly different between two is, we are providing small amount of timeout (not 0) and we don't know how much amount of remaining timeout would be used as polling records (instead of polling metadata). It would be unlikely to be exactly 0 as it would be timed out if it goes below 0. |
||
partitions = consumer.assignment() | ||
} | ||
require(!partitions.isEmpty, "Partitions assigned to the Kafka consumer can't be empty. " + | ||
"Setting kafkaConsumer.pollTimeoutMs to a too low value can potentially cause this.") | ||
logDebug(s"Partitions assigned to consumer: $partitions") | ||
partitions | ||
} | ||
} | ||
|
||
private[kafka010] object KafkaOffsetReader { | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
For this kind of logic it's better to use
System.nanoTime()
which is monotonic. Also you can do a little less computation this way:There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good point, since @zsxwing suggested new API usage I would wait here and check the Kafka side.