Skip to content

Commit b4cf770

Browse files
committed
Revert "[SPARK-24882][SQL] improve data source v2 API"
This reverts commit e754887.
1 parent ffd036a commit b4cf770

File tree

105 files changed

+2148
-2560
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

105 files changed

+2148
-2560
lines changed
Lines changed: 63 additions & 70 deletions
Original file line numberDiff line numberDiff line change
@@ -25,15 +25,16 @@ import org.apache.kafka.common.TopicPartition
2525

2626
import org.apache.spark.TaskContext
2727
import org.apache.spark.internal.Logging
28+
import org.apache.spark.sql.SparkSession
2829
import org.apache.spark.sql.catalyst.InternalRow
2930
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
3031
import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE}
3132
import org.apache.spark.sql.sources.v2.reader._
32-
import org.apache.spark.sql.sources.v2.reader.streaming._
33+
import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousInputPartitionReader, ContinuousReader, Offset, PartitionOffset}
3334
import org.apache.spark.sql.types.StructType
3435

3536
/**
36-
* A [[ContinuousReadSupport]] for data from kafka.
37+
* A [[ContinuousReader]] for data from kafka.
3738
*
3839
* @param offsetReader a reader used to get kafka offsets. Note that the actual data will be
3940
* read by per-task consumers generated later.
@@ -46,49 +47,70 @@ import org.apache.spark.sql.types.StructType
4647
* scenarios, where some offsets after the specified initial ones can't be
4748
* properly read.
4849
*/
49-
class KafkaContinuousReadSupport(
50+
class KafkaContinuousReader(
5051
offsetReader: KafkaOffsetReader,
5152
kafkaParams: ju.Map[String, Object],
5253
sourceOptions: Map[String, String],
5354
metadataPath: String,
5455
initialOffsets: KafkaOffsetRangeLimit,
5556
failOnDataLoss: Boolean)
56-
extends ContinuousReadSupport with Logging {
57+
extends ContinuousReader with Logging {
58+
59+
private lazy val session = SparkSession.getActiveSession.get
60+
private lazy val sc = session.sparkContext
5761

5862
private val pollTimeoutMs = sourceOptions.getOrElse("kafkaConsumer.pollTimeoutMs", "512").toLong
5963

60-
override def initialOffset(): Offset = {
61-
val offsets = initialOffsets match {
62-
case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets())
63-
case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets())
64-
case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, reportDataLoss)
65-
}
66-
logInfo(s"Initial offsets: $offsets")
67-
offsets
68-
}
64+
// Initialized when creating reader factories. If this diverges from the partitions at the latest
65+
// offsets, we need to reconfigure.
66+
// Exposed outside this object only for unit tests.
67+
@volatile private[sql] var knownPartitions: Set[TopicPartition] = _
6968

70-
override def fullSchema(): StructType = KafkaOffsetReader.kafkaSchema
69+
override def readSchema: StructType = KafkaOffsetReader.kafkaSchema
7170

72-
override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = {
73-
new KafkaContinuousScanConfigBuilder(fullSchema(), start, offsetReader, reportDataLoss)
71+
private var offset: Offset = _
72+
override def setStartOffset(start: ju.Optional[Offset]): Unit = {
73+
offset = start.orElse {
74+
val offsets = initialOffsets match {
75+
case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets())
76+
case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets())
77+
case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, reportDataLoss)
78+
}
79+
logInfo(s"Initial offsets: $offsets")
80+
offsets
81+
}
7482
}
7583

84+
override def getStartOffset(): Offset = offset
85+
7686
override def deserializeOffset(json: String): Offset = {
7787
KafkaSourceOffset(JsonUtils.partitionOffsets(json))
7888
}
7989

80-
override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
81-
val startOffsets = config.asInstanceOf[KafkaContinuousScanConfig].startOffsets
90+
override def planInputPartitions(): ju.List[InputPartition[InternalRow]] = {
91+
import scala.collection.JavaConverters._
92+
93+
val oldStartPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(offset)
94+
95+
val currentPartitionSet = offsetReader.fetchEarliestOffsets().keySet
96+
val newPartitions = currentPartitionSet.diff(oldStartPartitionOffsets.keySet)
97+
val newPartitionOffsets = offsetReader.fetchEarliestOffsets(newPartitions.toSeq)
98+
99+
val deletedPartitions = oldStartPartitionOffsets.keySet.diff(currentPartitionSet)
100+
if (deletedPartitions.nonEmpty) {
101+
reportDataLoss(s"Some partitions were deleted: $deletedPartitions")
102+
}
103+
104+
val startOffsets = newPartitionOffsets ++
105+
oldStartPartitionOffsets.filterKeys(!deletedPartitions.contains(_))
106+
knownPartitions = startOffsets.keySet
107+
82108
startOffsets.toSeq.map {
83109
case (topicPartition, start) =>
84110
KafkaContinuousInputPartition(
85-
topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss)
86-
}.toArray
87-
}
88-
89-
override def createContinuousReaderFactory(
90-
config: ScanConfig): ContinuousPartitionReaderFactory = {
91-
KafkaContinuousReaderFactory
111+
topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss
112+
): InputPartition[InternalRow]
113+
}.asJava
92114
}
93115

94116
/** Stop this source and free any resources it has allocated. */
@@ -105,9 +127,8 @@ class KafkaContinuousReadSupport(
105127
KafkaSourceOffset(mergedMap)
106128
}
107129

108-
override def needsReconfiguration(config: ScanConfig): Boolean = {
109-
val knownPartitions = config.asInstanceOf[KafkaContinuousScanConfig].knownPartitions
110-
offsetReader.fetchLatestOffsets().keySet != knownPartitions
130+
override def needsReconfiguration(): Boolean = {
131+
knownPartitions != null && offsetReader.fetchLatestOffsets().keySet != knownPartitions
111132
}
112133

113134
override def toString(): String = s"KafkaSource[$offsetReader]"
@@ -141,51 +162,23 @@ case class KafkaContinuousInputPartition(
141162
startOffset: Long,
142163
kafkaParams: ju.Map[String, Object],
143164
pollTimeoutMs: Long,
144-
failOnDataLoss: Boolean) extends InputPartition
145-
146-
object KafkaContinuousReaderFactory extends ContinuousPartitionReaderFactory {
147-
override def createReader(partition: InputPartition): ContinuousPartitionReader[InternalRow] = {
148-
val p = partition.asInstanceOf[KafkaContinuousInputPartition]
149-
new KafkaContinuousPartitionReader(
150-
p.topicPartition, p.startOffset, p.kafkaParams, p.pollTimeoutMs, p.failOnDataLoss)
165+
failOnDataLoss: Boolean) extends ContinuousInputPartition[InternalRow] {
166+
167+
override def createContinuousReader(
168+
offset: PartitionOffset): InputPartitionReader[InternalRow] = {
169+
val kafkaOffset = offset.asInstanceOf[KafkaSourcePartitionOffset]
170+
require(kafkaOffset.topicPartition == topicPartition,
171+
s"Expected topicPartition: $topicPartition, but got: ${kafkaOffset.topicPartition}")
172+
new KafkaContinuousInputPartitionReader(
173+
topicPartition, kafkaOffset.partitionOffset, kafkaParams, pollTimeoutMs, failOnDataLoss)
151174
}
152-
}
153-
154-
class KafkaContinuousScanConfigBuilder(
155-
schema: StructType,
156-
startOffset: Offset,
157-
offsetReader: KafkaOffsetReader,
158-
reportDataLoss: String => Unit)
159-
extends ScanConfigBuilder {
160-
161-
override def build(): ScanConfig = {
162-
val oldStartPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(startOffset)
163-
164-
val currentPartitionSet = offsetReader.fetchEarliestOffsets().keySet
165-
val newPartitions = currentPartitionSet.diff(oldStartPartitionOffsets.keySet)
166-
val newPartitionOffsets = offsetReader.fetchEarliestOffsets(newPartitions.toSeq)
167175

168-
val deletedPartitions = oldStartPartitionOffsets.keySet.diff(currentPartitionSet)
169-
if (deletedPartitions.nonEmpty) {
170-
reportDataLoss(s"Some partitions were deleted: $deletedPartitions")
171-
}
172-
173-
val startOffsets = newPartitionOffsets ++
174-
oldStartPartitionOffsets.filterKeys(!deletedPartitions.contains(_))
175-
KafkaContinuousScanConfig(schema, startOffsets)
176+
override def createPartitionReader(): KafkaContinuousInputPartitionReader = {
177+
new KafkaContinuousInputPartitionReader(
178+
topicPartition, startOffset, kafkaParams, pollTimeoutMs, failOnDataLoss)
176179
}
177180
}
178181

179-
case class KafkaContinuousScanConfig(
180-
readSchema: StructType,
181-
startOffsets: Map[TopicPartition, Long])
182-
extends ScanConfig {
183-
184-
// Created when building the scan config builder. If this diverges from the partitions at the
185-
// latest offsets, we need to reconfigure the kafka read support.
186-
def knownPartitions: Set[TopicPartition] = startOffsets.keySet
187-
}
188-
189182
/**
190183
* A per-task data reader for continuous Kafka processing.
191184
*
@@ -196,12 +189,12 @@ case class KafkaContinuousScanConfig(
196189
* @param failOnDataLoss Flag indicating whether data reader should fail if some offsets
197190
* are skipped.
198191
*/
199-
class KafkaContinuousPartitionReader(
192+
class KafkaContinuousInputPartitionReader(
200193
topicPartition: TopicPartition,
201194
startOffset: Long,
202195
kafkaParams: ju.Map[String, Object],
203196
pollTimeoutMs: Long,
204-
failOnDataLoss: Boolean) extends ContinuousPartitionReader[InternalRow] {
197+
failOnDataLoss: Boolean) extends ContinuousInputPartitionReader[InternalRow] {
205198
private val consumer = KafkaDataConsumer.acquire(topicPartition, kafkaParams, useCache = false)
206199
private val converter = new KafkaRecordToUnsafeRowConverter
207200

Lines changed: 54 additions & 49 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,8 @@ import java.{util => ju}
2121
import java.io._
2222
import java.nio.charset.StandardCharsets
2323

24+
import scala.collection.JavaConverters._
25+
2426
import org.apache.commons.io.IOUtils
2527

2628
import org.apache.spark.SparkEnv
@@ -29,17 +31,16 @@ import org.apache.spark.scheduler.ExecutorCacheTaskLocation
2931
import org.apache.spark.sql.SparkSession
3032
import org.apache.spark.sql.catalyst.InternalRow
3133
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
32-
import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, SerializedOffset, SimpleStreamingScanConfig, SimpleStreamingScanConfigBuilder}
33-
import org.apache.spark.sql.execution.streaming.sources.RateControlMicroBatchReadSupport
34+
import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, SerializedOffset}
3435
import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE}
3536
import org.apache.spark.sql.sources.v2.DataSourceOptions
36-
import org.apache.spark.sql.sources.v2.reader._
37-
import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset}
37+
import org.apache.spark.sql.sources.v2.reader.{InputPartition, InputPartitionReader}
38+
import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReader, Offset}
3839
import org.apache.spark.sql.types.StructType
3940
import org.apache.spark.util.UninterruptibleThread
4041

4142
/**
42-
* A [[MicroBatchReadSupport]] that reads data from Kafka.
43+
* A [[MicroBatchReader]] that reads data from Kafka.
4344
*
4445
* The [[KafkaSourceOffset]] is the custom [[Offset]] defined for this source that contains
4546
* a map of TopicPartition -> offset. Note that this offset is 1 + (available offset). For
@@ -54,13 +55,17 @@ import org.apache.spark.util.UninterruptibleThread
5455
* To avoid this issue, you should make sure stopping the query before stopping the Kafka brokers
5556
* and not use wrong broker addresses.
5657
*/
57-
private[kafka010] class KafkaMicroBatchReadSupport(
58+
private[kafka010] class KafkaMicroBatchReader(
5859
kafkaOffsetReader: KafkaOffsetReader,
5960
executorKafkaParams: ju.Map[String, Object],
6061
options: DataSourceOptions,
6162
metadataPath: String,
6263
startingOffsets: KafkaOffsetRangeLimit,
63-
failOnDataLoss: Boolean) extends RateControlMicroBatchReadSupport with Logging {
64+
failOnDataLoss: Boolean)
65+
extends MicroBatchReader with Logging {
66+
67+
private var startPartitionOffsets: PartitionOffsetMap = _
68+
private var endPartitionOffsets: PartitionOffsetMap = _
6469

6570
private val pollTimeoutMs = options.getLong(
6671
"kafkaConsumer.pollTimeoutMs",
@@ -70,40 +75,34 @@ private[kafka010] class KafkaMicroBatchReadSupport(
7075
Option(options.get("maxOffsetsPerTrigger").orElse(null)).map(_.toLong)
7176

7277
private val rangeCalculator = KafkaOffsetRangeCalculator(options)
73-
74-
private var endPartitionOffsets: KafkaSourceOffset = _
75-
7678
/**
7779
* Lazily initialize `initialPartitionOffsets` to make sure that `KafkaConsumer.poll` is only
7880
* called in StreamExecutionThread. Otherwise, interrupting a thread while running
7981
* `KafkaConsumer.poll` may hang forever (KAFKA-1894).
8082
*/
81-
override def initialOffset(): Offset = {
82-
KafkaSourceOffset(getOrCreateInitialPartitionOffsets())
83-
}
84-
85-
override def latestOffset(start: Offset): Offset = {
86-
val startPartitionOffsets = start.asInstanceOf[KafkaSourceOffset].partitionToOffsets
87-
val latestPartitionOffsets = kafkaOffsetReader.fetchLatestOffsets()
88-
endPartitionOffsets = KafkaSourceOffset(maxOffsetsPerTrigger.map { maxOffsets =>
89-
rateLimit(maxOffsets, startPartitionOffsets, latestPartitionOffsets)
90-
}.getOrElse {
91-
latestPartitionOffsets
92-
})
93-
endPartitionOffsets
94-
}
95-
96-
override def fullSchema(): StructType = KafkaOffsetReader.kafkaSchema
97-
98-
override def newScanConfigBuilder(start: Offset, end: Offset): ScanConfigBuilder = {
99-
new SimpleStreamingScanConfigBuilder(fullSchema(), start, Some(end))
83+
private lazy val initialPartitionOffsets = getOrCreateInitialPartitionOffsets()
84+
85+
override def setOffsetRange(start: ju.Optional[Offset], end: ju.Optional[Offset]): Unit = {
86+
// Make sure initialPartitionOffsets is initialized
87+
initialPartitionOffsets
88+
89+
startPartitionOffsets = Option(start.orElse(null))
90+
.map(_.asInstanceOf[KafkaSourceOffset].partitionToOffsets)
91+
.getOrElse(initialPartitionOffsets)
92+
93+
endPartitionOffsets = Option(end.orElse(null))
94+
.map(_.asInstanceOf[KafkaSourceOffset].partitionToOffsets)
95+
.getOrElse {
96+
val latestPartitionOffsets = kafkaOffsetReader.fetchLatestOffsets()
97+
maxOffsetsPerTrigger.map { maxOffsets =>
98+
rateLimit(maxOffsets, startPartitionOffsets, latestPartitionOffsets)
99+
}.getOrElse {
100+
latestPartitionOffsets
101+
}
102+
}
100103
}
101104

102-
override def planInputPartitions(config: ScanConfig): Array[InputPartition] = {
103-
val sc = config.asInstanceOf[SimpleStreamingScanConfig]
104-
val startPartitionOffsets = sc.start.asInstanceOf[KafkaSourceOffset].partitionToOffsets
105-
val endPartitionOffsets = sc.end.get.asInstanceOf[KafkaSourceOffset].partitionToOffsets
106-
105+
override def planInputPartitions(): ju.List[InputPartition[InternalRow]] = {
107106
// Find the new partitions, and get their earliest offsets
108107
val newPartitions = endPartitionOffsets.keySet.diff(startPartitionOffsets.keySet)
109108
val newPartitionInitialOffsets = kafkaOffsetReader.fetchEarliestOffsets(newPartitions.toSeq)
@@ -145,19 +144,26 @@ private[kafka010] class KafkaMicroBatchReadSupport(
145144

146145
// Generate factories based on the offset ranges
147146
offsetRanges.map { range =>
148-
KafkaMicroBatchInputPartition(
149-
range, executorKafkaParams, pollTimeoutMs, failOnDataLoss, reuseKafkaConsumer)
150-
}.toArray
147+
new KafkaMicroBatchInputPartition(
148+
range, executorKafkaParams, pollTimeoutMs, failOnDataLoss, reuseKafkaConsumer
149+
): InputPartition[InternalRow]
150+
}.asJava
151+
}
152+
153+
override def getStartOffset: Offset = {
154+
KafkaSourceOffset(startPartitionOffsets)
151155
}
152156

153-
override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = {
154-
KafkaMicroBatchReaderFactory
157+
override def getEndOffset: Offset = {
158+
KafkaSourceOffset(endPartitionOffsets)
155159
}
156160

157161
override def deserializeOffset(json: String): Offset = {
158162
KafkaSourceOffset(JsonUtils.partitionOffsets(json))
159163
}
160164

165+
override def readSchema(): StructType = KafkaOffsetReader.kafkaSchema
166+
161167
override def commit(end: Offset): Unit = {}
162168

163169
override def stop(): Unit = {
@@ -300,23 +306,22 @@ private[kafka010] case class KafkaMicroBatchInputPartition(
300306
executorKafkaParams: ju.Map[String, Object],
301307
pollTimeoutMs: Long,
302308
failOnDataLoss: Boolean,
303-
reuseKafkaConsumer: Boolean) extends InputPartition
309+
reuseKafkaConsumer: Boolean) extends InputPartition[InternalRow] {
304310

305-
private[kafka010] object KafkaMicroBatchReaderFactory extends PartitionReaderFactory {
306-
override def createReader(partition: InputPartition): PartitionReader[InternalRow] = {
307-
val p = partition.asInstanceOf[KafkaMicroBatchInputPartition]
308-
KafkaMicroBatchPartitionReader(p.offsetRange, p.executorKafkaParams, p.pollTimeoutMs,
309-
p.failOnDataLoss, p.reuseKafkaConsumer)
310-
}
311+
override def preferredLocations(): Array[String] = offsetRange.preferredLoc.toArray
312+
313+
override def createPartitionReader(): InputPartitionReader[InternalRow] =
314+
new KafkaMicroBatchInputPartitionReader(offsetRange, executorKafkaParams, pollTimeoutMs,
315+
failOnDataLoss, reuseKafkaConsumer)
311316
}
312317

313-
/** A [[PartitionReader]] for reading Kafka data in a micro-batch streaming query. */
314-
private[kafka010] case class KafkaMicroBatchPartitionReader(
318+
/** A [[InputPartitionReader]] for reading Kafka data in a micro-batch streaming query. */
319+
private[kafka010] case class KafkaMicroBatchInputPartitionReader(
315320
offsetRange: KafkaOffsetRange,
316321
executorKafkaParams: ju.Map[String, Object],
317322
pollTimeoutMs: Long,
318323
failOnDataLoss: Boolean,
319-
reuseKafkaConsumer: Boolean) extends PartitionReader[InternalRow] with Logging {
324+
reuseKafkaConsumer: Boolean) extends InputPartitionReader[InternalRow] with Logging {
320325

321326
private val consumer = KafkaDataConsumer.acquire(
322327
offsetRange.topicPartition, executorKafkaParams, reuseKafkaConsumer)

0 commit comments

Comments
 (0)