-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-23541][SS] Allow Kafka source to read data with greater parallelism than the number of topic-partitions #20698
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
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -26,12 +26,12 @@ import org.apache.spark.sql.sources.v2.DataSourceOptions | |
| * Class to calculate offset ranges to process based on the the from and until offsets, and | ||
| * the configured `minPartitions`. | ||
| */ | ||
| private[kafka010] class KafkaOffsetRangeCalculator(val minPartitions: Int) { | ||
| require(minPartitions >= 0) | ||
| private[kafka010] class KafkaOffsetRangeCalculator(val minPartitions: Option[Int]) { | ||
| require(minPartitions.isEmpty || minPartitions.get > 0) | ||
|
|
||
| import KafkaOffsetRangeCalculator._ | ||
| /** | ||
| * Calculate the offset ranges that we are going to process this batch. If `numPartitions` | ||
| * Calculate the offset ranges that we are going to process this batch. If `minPartitions` | ||
| * is not set or is set less than or equal the number of `topicPartitions` that we're going to | ||
| * consume, then we fall back to a 1-1 mapping of Spark tasks to Kafka partitions. If | ||
| * `numPartitions` is set higher than the number of our `topicPartitions`, then we will split up | ||
|
|
@@ -50,32 +50,30 @@ private[kafka010] class KafkaOffsetRangeCalculator(val minPartitions: Int) { | |
| } | ||
|
|
||
| // If minPartitions not set or there are enough partitions to satisfy minPartitions | ||
| if (minPartitions == DEFAULT_MIN_PARTITIONS || offsetRanges.size > minPartitions) { | ||
| if (minPartitions.isEmpty || offsetRanges.size > minPartitions.get) { | ||
| // Assign preferred executor locations to each range such that the same topic-partition is | ||
| // always read from the same executor and the KafkaConsumer can be reused | ||
| // preferentially read from the same executor and the KafkaConsumer can be reused. | ||
| offsetRanges.map { range => | ||
| range.copy(preferredLoc = getLocation(range.topicPartition, executorLocations)) | ||
| } | ||
| } else { | ||
|
|
||
| // Splits offset ranges with relatively large amount of data to smaller ones. | ||
| val totalSize = offsetRanges.map(o => o.untilOffset - o.fromOffset).sum | ||
| offsetRanges.flatMap { offsetRange => | ||
| val tp = offsetRange.topicPartition | ||
| val size = offsetRange.untilOffset - offsetRange.fromOffset | ||
| // number of partitions to divvy up this topic partition to | ||
| val parts = math.max(math.round(size * 1.0 / totalSize * minPartitions), 1).toInt | ||
| var remaining = size | ||
| var startOffset = offsetRange.fromOffset | ||
| (0 until parts).map { part => | ||
| // Fine to do integer division. Last partition will consume all the round off errors | ||
| val thisPartition = remaining / (parts - part) | ||
| remaining -= thisPartition | ||
| val endOffset = startOffset + thisPartition | ||
| val offsetRange = KafkaOffsetRange(tp, startOffset, endOffset, preferredLoc = None) | ||
| startOffset = endOffset | ||
| offsetRange | ||
| val idealRangeSize = totalSize.toDouble / minPartitions.get | ||
|
|
||
| offsetRanges.flatMap { range => | ||
| // Split the current range into subranges as close to the ideal range size | ||
| val rangeSize = range.untilOffset - range.fromOffset | ||
| val numSplitsInRange = math.round(rangeSize.toDouble / idealRangeSize).toInt | ||
|
||
|
|
||
| (0 until numSplitsInRange).map { i => | ||
| val splitStart = range.fromOffset + rangeSize * (i.toDouble / numSplitsInRange) | ||
| val splitEnd = range.fromOffset + rangeSize * ((i.toDouble + 1) / numSplitsInRange) | ||
| KafkaOffsetRange( | ||
| range.topicPartition, splitStart.toLong, splitEnd.toLong, preferredLoc = None) | ||
| } | ||
|
|
||
|
||
| } | ||
| } | ||
| } | ||
|
|
@@ -94,10 +92,9 @@ private[kafka010] class KafkaOffsetRangeCalculator(val minPartitions: Int) { | |
|
|
||
| private[kafka010] object KafkaOffsetRangeCalculator { | ||
|
|
||
| private val DEFAULT_MIN_PARTITIONS = 0 | ||
|
|
||
| def apply(options: DataSourceOptions): KafkaOffsetRangeCalculator = { | ||
| new KafkaOffsetRangeCalculator(options.getInt("minPartitions", DEFAULT_MIN_PARTITIONS)) | ||
| val optionalValue = Option(options.get("minPartitions").orElse(null)).map(_.toInt) | ||
|
Contributor
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. nit:
Contributor
Author
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. Because it returns java Optional and not scala Option. |
||
| new KafkaOffsetRangeCalculator(optionalValue) | ||
| } | ||
| } | ||
|
|
||
|
|
||
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.
nit:
map(_.size).sum