-
Notifications
You must be signed in to change notification settings - Fork 29.2k
[SPARK-36649][SQL] Support Trigger.AvailableNow on Kafka data source
#35238
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 all commits
efebe3c
a13e3ea
e542b04
6881d74
fc18b1b
1d05a0b
53e797d
1369e31
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 |
|---|---|---|
|
|
@@ -57,7 +57,7 @@ private[kafka010] class KafkaMicroBatchStream( | |
| metadataPath: String, | ||
| startingOffsets: KafkaOffsetRangeLimit, | ||
| failOnDataLoss: Boolean) | ||
| extends SupportsAdmissionControl with ReportsSourceMetrics with MicroBatchStream with Logging { | ||
| extends SupportsTriggerAvailableNow with ReportsSourceMetrics with MicroBatchStream with Logging { | ||
|
|
||
| private[kafka010] val pollTimeoutMs = options.getLong( | ||
| KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, | ||
|
|
@@ -81,6 +81,8 @@ private[kafka010] class KafkaMicroBatchStream( | |
|
|
||
| private var latestPartitionOffsets: PartitionOffsetMap = _ | ||
|
|
||
| private var allDataForTriggerAvailableNow: PartitionOffsetMap = _ | ||
|
|
||
| /** | ||
| * Lazily initialize `initialPartitionOffsets` to make sure that `KafkaConsumer.poll` is only | ||
| * called in StreamExecutionThread. Otherwise, interrupting a thread while running | ||
|
|
@@ -98,7 +100,8 @@ private[kafka010] class KafkaMicroBatchStream( | |
| } else if (minOffsetPerTrigger.isDefined) { | ||
| ReadLimit.minRows(minOffsetPerTrigger.get, maxTriggerDelayMs) | ||
| } else { | ||
| maxOffsetsPerTrigger.map(ReadLimit.maxRows).getOrElse(super.getDefaultReadLimit) | ||
| // TODO (SPARK-37973) Directly call super.getDefaultReadLimit when scala issue 12523 is fixed | ||
| maxOffsetsPerTrigger.map(ReadLimit.maxRows).getOrElse(ReadLimit.allAvailable()) | ||
|
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. why this change? Isn't it better to delegate to the parent if you have no opinion - e.g. the parent could use some config default or whatever.
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. Previous comment seems to be out of date. It's to avoid Scala compiler bug. While we are here, probably good to leave code comment for the reason, and tag
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.
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. I will add a comment
Member
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. how about fill a Spark Jira with the detailed description. Then we just need a single line comment here like:
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. sure |
||
| } | ||
| } | ||
|
|
||
|
|
@@ -113,7 +116,13 @@ private[kafka010] class KafkaMicroBatchStream( | |
|
|
||
| override def latestOffset(start: Offset, readLimit: ReadLimit): Offset = { | ||
| val startPartitionOffsets = start.asInstanceOf[KafkaSourceOffset].partitionToOffsets | ||
| latestPartitionOffsets = kafkaOffsetReader.fetchLatestOffsets(Some(startPartitionOffsets)) | ||
|
|
||
| // Use the pre-fetched list of partition offsets when Trigger.AvailableNow is enabled. | ||
| latestPartitionOffsets = if (allDataForTriggerAvailableNow != null) { | ||
| allDataForTriggerAvailableNow | ||
| } else { | ||
| kafkaOffsetReader.fetchLatestOffsets(Some(startPartitionOffsets)) | ||
| } | ||
|
|
||
| val limits: Seq[ReadLimit] = readLimit match { | ||
| case rows: CompositeReadLimit => rows.getReadLimits | ||
|
|
@@ -298,6 +307,11 @@ private[kafka010] class KafkaMicroBatchStream( | |
| logWarning(message + s". $INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE") | ||
| } | ||
| } | ||
|
|
||
| override def prepareForTriggerAvailableNow(): Unit = { | ||
| allDataForTriggerAvailableNow = kafkaOffsetReader.fetchLatestOffsets( | ||
| Some(getOrCreateInitialPartitionOffsets())) | ||
| } | ||
| } | ||
|
|
||
| object KafkaMicroBatchStream extends Logging { | ||
|
|
||
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.
what about SupportsAdmissionControl - I see other APIs specifying both - https://livegrep.dev.databricks.com/view/databricks/runtime/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala#L57
even if one extends the other, is it better to list them explicitly?
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.
I don't think it is necessary. It is redundant
Uh oh!
There was an error while loading. Please reload this page.
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.
SupportsTriggerAvailableNowalready extendsSupportsAdmissionControl:https://github.com/apache/spark/blob/master/sql/catalyst/src/main/java/org/apache/spark/sql/connector/read/streaming/SupportsTriggerAvailableNow.java