-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-24882][SQL] improve data source v2 API #22009
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
Closed
Closed
Changes from all commits
Commits
Show all changes
28 commits
Select commit
Hold shift + click to select a range
3ce6b7c
improve data source v2 API
cloud-fan 220976b
improve comments and naming
cloud-fan 0b3f7a0
fix kafka
cloud-fan cab6d28
fix java style
cloud-fan 2fc3b05
fix java doc
cloud-fan 29b4f33
more document
cloud-fan c224999
fix test
cloud-fan b062220
Merge branch 'master' into redesign
cloud-fan f620297
address comments
cloud-fan 063fe27
address more comments
cloud-fan 2b1c22a
fix java doc
cloud-fan c4b5469
fix document
cloud-fan 2fa12d7
fix java style
cloud-fan 6728d33
Merge branch 'master' into redesign
cloud-fan 7c21af9
address comments
cloud-fan c175be4
Merge remote-tracking branch 'origin/master' into redesign
cloud-fan f4f85a8
fix conflicts
cloud-fan ff2ed26
Merge branch 'master' into redesign
cloud-fan 4545ed2
reduce diff
cloud-fan e6e599a
add debug string
cloud-fan 2018981
Revert "add debug string"
cloud-fan 76f8e6b
Merge branch 'master' into redesign
cloud-fan 844bd6f
Merge branch 'master' into redesign
cloud-fan 9acda35
improve document
cloud-fan ca80080
more fix
cloud-fan f938614
address comments
cloud-fan 8833b67
fix test
cloud-fan 51cda76
fix java style
cloud-fan File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -25,16 +25,15 @@ import org.apache.kafka.common.TopicPartition | |
|
|
||
| import org.apache.spark.TaskContext | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.sql.SparkSession | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.expressions.UnsafeRow | ||
| import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE} | ||
| import org.apache.spark.sql.sources.v2.reader._ | ||
| import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousInputPartitionReader, ContinuousReader, Offset, PartitionOffset} | ||
| import org.apache.spark.sql.sources.v2.reader.streaming._ | ||
| import org.apache.spark.sql.types.StructType | ||
|
|
||
| /** | ||
| * A [[ContinuousReader]] for data from kafka. | ||
| * A [[ContinuousReadSupport]] for data from kafka. | ||
| * | ||
| * @param offsetReader a reader used to get kafka offsets. Note that the actual data will be | ||
| * read by per-task consumers generated later. | ||
|
|
@@ -47,70 +46,49 @@ import org.apache.spark.sql.types.StructType | |
| * scenarios, where some offsets after the specified initial ones can't be | ||
| * properly read. | ||
| */ | ||
| class KafkaContinuousReader( | ||
| class KafkaContinuousReadSupport( | ||
| offsetReader: KafkaOffsetReader, | ||
| kafkaParams: ju.Map[String, Object], | ||
| sourceOptions: Map[String, String], | ||
| metadataPath: String, | ||
| initialOffsets: KafkaOffsetRangeLimit, | ||
| failOnDataLoss: Boolean) | ||
| extends ContinuousReader with Logging { | ||
|
|
||
| private lazy val session = SparkSession.getActiveSession.get | ||
| private lazy val sc = session.sparkContext | ||
| extends ContinuousReadSupport with Logging { | ||
|
|
||
| private val pollTimeoutMs = sourceOptions.getOrElse("kafkaConsumer.pollTimeoutMs", "512").toLong | ||
|
|
||
| // Initialized when creating reader factories. If this diverges from the partitions at the latest | ||
| // offsets, we need to reconfigure. | ||
| // Exposed outside this object only for unit tests. | ||
| @volatile private[sql] var knownPartitions: Set[TopicPartition] = _ | ||
|
|
||
| override def readSchema: StructType = KafkaOffsetReader.kafkaSchema | ||
|
|
||
| private var offset: Offset = _ | ||
| override def setStartOffset(start: ju.Optional[Offset]): Unit = { | ||
| offset = start.orElse { | ||
| val offsets = initialOffsets match { | ||
| case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets()) | ||
| case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets()) | ||
| case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, reportDataLoss) | ||
| } | ||
| logInfo(s"Initial offsets: $offsets") | ||
| offsets | ||
| override def initialOffset(): Offset = { | ||
| val offsets = initialOffsets match { | ||
| case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets()) | ||
| case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets()) | ||
| case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, reportDataLoss) | ||
| } | ||
| logInfo(s"Initial offsets: $offsets") | ||
| offsets | ||
| } | ||
|
|
||
| override def getStartOffset(): Offset = offset | ||
| override def fullSchema(): StructType = KafkaOffsetReader.kafkaSchema | ||
|
|
||
| override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = { | ||
| new KafkaContinuousScanConfigBuilder(fullSchema(), start, offsetReader, reportDataLoss) | ||
| } | ||
|
|
||
| override def deserializeOffset(json: String): Offset = { | ||
| KafkaSourceOffset(JsonUtils.partitionOffsets(json)) | ||
| } | ||
|
|
||
| override def planInputPartitions(): ju.List[InputPartition[InternalRow]] = { | ||
| import scala.collection.JavaConverters._ | ||
|
|
||
| val oldStartPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(offset) | ||
|
|
||
| val currentPartitionSet = offsetReader.fetchEarliestOffsets().keySet | ||
| val newPartitions = currentPartitionSet.diff(oldStartPartitionOffsets.keySet) | ||
| val newPartitionOffsets = offsetReader.fetchEarliestOffsets(newPartitions.toSeq) | ||
|
|
||
| val deletedPartitions = oldStartPartitionOffsets.keySet.diff(currentPartitionSet) | ||
| if (deletedPartitions.nonEmpty) { | ||
| reportDataLoss(s"Some partitions were deleted: $deletedPartitions") | ||
| } | ||
|
|
||
| val startOffsets = newPartitionOffsets ++ | ||
| oldStartPartitionOffsets.filterKeys(!deletedPartitions.contains(_)) | ||
| knownPartitions = startOffsets.keySet | ||
|
|
||
| override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { | ||
| val startOffsets = config.asInstanceOf[KafkaContinuousScanConfig].startOffsets | ||
| startOffsets.toSeq.map { | ||
| case (topicPartition, start) => | ||
| KafkaContinuousInputPartition( | ||
| topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss | ||
| ): InputPartition[InternalRow] | ||
| }.asJava | ||
| topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss) | ||
| }.toArray | ||
| } | ||
|
|
||
| override def createContinuousReaderFactory( | ||
| config: ScanConfig): ContinuousPartitionReaderFactory = { | ||
| KafkaContinuousReaderFactory | ||
| } | ||
|
|
||
| /** Stop this source and free any resources it has allocated. */ | ||
|
|
@@ -127,8 +105,9 @@ class KafkaContinuousReader( | |
| KafkaSourceOffset(mergedMap) | ||
| } | ||
|
|
||
| override def needsReconfiguration(): Boolean = { | ||
| knownPartitions != null && offsetReader.fetchLatestOffsets().keySet != knownPartitions | ||
| override def needsReconfiguration(config: ScanConfig): Boolean = { | ||
| val knownPartitions = config.asInstanceOf[KafkaContinuousScanConfig].knownPartitions | ||
| offsetReader.fetchLatestOffsets().keySet != knownPartitions | ||
| } | ||
|
|
||
| override def toString(): String = s"KafkaSource[$offsetReader]" | ||
|
|
@@ -162,23 +141,51 @@ case class KafkaContinuousInputPartition( | |
| startOffset: Long, | ||
| kafkaParams: ju.Map[String, Object], | ||
| pollTimeoutMs: Long, | ||
| failOnDataLoss: Boolean) extends ContinuousInputPartition[InternalRow] { | ||
|
|
||
| override def createContinuousReader( | ||
| offset: PartitionOffset): InputPartitionReader[InternalRow] = { | ||
| val kafkaOffset = offset.asInstanceOf[KafkaSourcePartitionOffset] | ||
| require(kafkaOffset.topicPartition == topicPartition, | ||
| s"Expected topicPartition: $topicPartition, but got: ${kafkaOffset.topicPartition}") | ||
| new KafkaContinuousInputPartitionReader( | ||
| topicPartition, kafkaOffset.partitionOffset, kafkaParams, pollTimeoutMs, failOnDataLoss) | ||
| failOnDataLoss: Boolean) extends InputPartition | ||
|
|
||
| object KafkaContinuousReaderFactory extends ContinuousPartitionReaderFactory { | ||
| override def createReader(partition: InputPartition): ContinuousPartitionReader[InternalRow] = { | ||
| val p = partition.asInstanceOf[KafkaContinuousInputPartition] | ||
| new KafkaContinuousPartitionReader( | ||
| p.topicPartition, p.startOffset, p.kafkaParams, p.pollTimeoutMs, p.failOnDataLoss) | ||
| } | ||
| } | ||
|
|
||
| class KafkaContinuousScanConfigBuilder( | ||
| schema: StructType, | ||
| startOffset: Offset, | ||
| offsetReader: KafkaOffsetReader, | ||
| reportDataLoss: String => Unit) | ||
| extends ScanConfigBuilder { | ||
|
|
||
| override def build(): ScanConfig = { | ||
| val oldStartPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(startOffset) | ||
|
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. |
||
|
|
||
| val currentPartitionSet = offsetReader.fetchEarliestOffsets().keySet | ||
| val newPartitions = currentPartitionSet.diff(oldStartPartitionOffsets.keySet) | ||
| val newPartitionOffsets = offsetReader.fetchEarliestOffsets(newPartitions.toSeq) | ||
|
|
||
| override def createPartitionReader(): KafkaContinuousInputPartitionReader = { | ||
| new KafkaContinuousInputPartitionReader( | ||
| topicPartition, startOffset, kafkaParams, pollTimeoutMs, failOnDataLoss) | ||
| val deletedPartitions = oldStartPartitionOffsets.keySet.diff(currentPartitionSet) | ||
| if (deletedPartitions.nonEmpty) { | ||
| reportDataLoss(s"Some partitions were deleted: $deletedPartitions") | ||
| } | ||
|
|
||
| val startOffsets = newPartitionOffsets ++ | ||
| oldStartPartitionOffsets.filterKeys(!deletedPartitions.contains(_)) | ||
| KafkaContinuousScanConfig(schema, startOffsets) | ||
| } | ||
| } | ||
|
|
||
| case class KafkaContinuousScanConfig( | ||
| readSchema: StructType, | ||
| startOffsets: Map[TopicPartition, Long]) | ||
| extends ScanConfig { | ||
|
|
||
| // Created when building the scan config builder. If this diverges from the partitions at the | ||
| // latest offsets, we need to reconfigure the kafka read support. | ||
| def knownPartitions: Set[TopicPartition] = startOffsets.keySet | ||
| } | ||
|
|
||
| /** | ||
| * A per-task data reader for continuous Kafka processing. | ||
| * | ||
|
|
@@ -189,12 +196,12 @@ case class KafkaContinuousInputPartition( | |
| * @param failOnDataLoss Flag indicating whether data reader should fail if some offsets | ||
| * are skipped. | ||
| */ | ||
| class KafkaContinuousInputPartitionReader( | ||
| class KafkaContinuousPartitionReader( | ||
| topicPartition: TopicPartition, | ||
| startOffset: Long, | ||
| kafkaParams: ju.Map[String, Object], | ||
| pollTimeoutMs: Long, | ||
| failOnDataLoss: Boolean) extends ContinuousInputPartitionReader[InternalRow] { | ||
| failOnDataLoss: Boolean) extends ContinuousPartitionReader[InternalRow] { | ||
| private val consumer = KafkaDataConsumer.acquire(topicPartition, kafkaParams, useCache = false) | ||
| private val converter = new KafkaRecordToUnsafeRowConverter | ||
|
|
||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Moved to
KafkaContinuousScanConfig