Skip to content

Commit 07ae39d

Browse files
xuanyuankingzsxwing
authored andcommitted
[SPARK-22956][SS] Bug fix for 2 streams union failover scenario
## What changes were proposed in this pull request? This problem reported by yanlin-Lynn ivoson and LiangchangZ. Thanks! When we union 2 streams from kafka or other sources, while one of them have no continues data coming and in the same time task restart, this will cause an `IllegalStateException`. This mainly cause because the code in [MicroBatchExecution](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala#L190) , while one stream has no continues data, its comittedOffset same with availableOffset during `populateStartOffsets`, and `currentPartitionOffsets` not properly handled in KafkaSource. Also, maybe we should also consider this scenario in other Source. ## How was this patch tested? Add a UT in KafkaSourceSuite.scala Author: Yuanjian Li <[email protected]> Closes #20150 from xuanyuanking/SPARK-22956.
1 parent c7572b7 commit 07ae39d

File tree

4 files changed

+81
-9
lines changed

4 files changed

+81
-9
lines changed

external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala

Lines changed: 8 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -223,6 +223,14 @@ private[kafka010] class KafkaSource(
223223

224224
logInfo(s"GetBatch called with start = $start, end = $end")
225225
val untilPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(end)
226+
// On recovery, getBatch will get called before getOffset
227+
if (currentPartitionOffsets.isEmpty) {
228+
currentPartitionOffsets = Some(untilPartitionOffsets)
229+
}
230+
if (start.isDefined && start.get == end) {
231+
return sqlContext.internalCreateDataFrame(
232+
sqlContext.sparkContext.emptyRDD, schema, isStreaming = true)
233+
}
226234
val fromPartitionOffsets = start match {
227235
case Some(prevBatchEndOffset) =>
228236
KafkaSourceOffset.getPartitionOffsets(prevBatchEndOffset)
@@ -305,11 +313,6 @@ private[kafka010] class KafkaSource(
305313
logInfo("GetBatch generating RDD of offset range: " +
306314
offsetRanges.sortBy(_.topicPartition.toString).mkString(", "))
307315

308-
// On recovery, getBatch will get called before getOffset
309-
if (currentPartitionOffsets.isEmpty) {
310-
currentPartitionOffsets = Some(untilPartitionOffsets)
311-
}
312-
313316
sqlContext.internalCreateDataFrame(rdd, schema, isStreaming = true)
314317
}
315318

external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala

Lines changed: 65 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -318,6 +318,71 @@ class KafkaSourceSuite extends KafkaSourceTest {
318318
)
319319
}
320320

321+
test("SPARK-22956: currentPartitionOffsets should be set when no new data comes in") {
322+
def getSpecificDF(range: Range.Inclusive): org.apache.spark.sql.Dataset[Int] = {
323+
val topic = newTopic()
324+
testUtils.createTopic(topic, partitions = 1)
325+
testUtils.sendMessages(topic, range.map(_.toString).toArray, Some(0))
326+
327+
val reader = spark
328+
.readStream
329+
.format("kafka")
330+
.option("kafka.bootstrap.servers", testUtils.brokerAddress)
331+
.option("kafka.metadata.max.age.ms", "1")
332+
.option("maxOffsetsPerTrigger", 5)
333+
.option("subscribe", topic)
334+
.option("startingOffsets", "earliest")
335+
336+
reader.load()
337+
.selectExpr("CAST(value AS STRING)")
338+
.as[String]
339+
.map(k => k.toInt)
340+
}
341+
342+
val df1 = getSpecificDF(0 to 9)
343+
val df2 = getSpecificDF(100 to 199)
344+
345+
val kafka = df1.union(df2)
346+
347+
val clock = new StreamManualClock
348+
349+
val waitUntilBatchProcessed = AssertOnQuery { q =>
350+
eventually(Timeout(streamingTimeout)) {
351+
if (!q.exception.isDefined) {
352+
assert(clock.isStreamWaitingAt(clock.getTimeMillis()))
353+
}
354+
}
355+
if (q.exception.isDefined) {
356+
throw q.exception.get
357+
}
358+
true
359+
}
360+
361+
testStream(kafka)(
362+
StartStream(ProcessingTime(100), clock),
363+
waitUntilBatchProcessed,
364+
// 5 from smaller topic, 5 from bigger one
365+
CheckLastBatch((0 to 4) ++ (100 to 104): _*),
366+
AdvanceManualClock(100),
367+
waitUntilBatchProcessed,
368+
// 5 from smaller topic, 5 from bigger one
369+
CheckLastBatch((5 to 9) ++ (105 to 109): _*),
370+
AdvanceManualClock(100),
371+
waitUntilBatchProcessed,
372+
// smaller topic empty, 5 from bigger one
373+
CheckLastBatch(110 to 114: _*),
374+
StopStream,
375+
StartStream(ProcessingTime(100), clock),
376+
waitUntilBatchProcessed,
377+
// smallest now empty, 5 from bigger one
378+
CheckLastBatch(115 to 119: _*),
379+
AdvanceManualClock(100),
380+
waitUntilBatchProcessed,
381+
// smallest now empty, 5 from bigger one
382+
CheckLastBatch(120 to 124: _*)
383+
)
384+
}
385+
321386
test("cannot stop Kafka stream") {
322387
val topic = newTopic()
323388
testUtils.createTopic(topic, partitions = 5)

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -208,10 +208,8 @@ class MicroBatchExecution(
208208
* batch will be executed before getOffset is called again. */
209209
availableOffsets.foreach {
210210
case (source: Source, end: Offset) =>
211-
if (committedOffsets.get(source).map(_ != end).getOrElse(true)) {
212-
val start = committedOffsets.get(source)
213-
source.getBatch(start, end)
214-
}
211+
val start = committedOffsets.get(source)
212+
source.getBatch(start, end)
215213
case nonV1Tuple =>
216214
// The V2 API does not have the same edge case requiring getBatch to be called
217215
// here, so we do nothing here.

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -119,9 +119,15 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext)
119119
val newBlocks = synchronized {
120120
val sliceStart = startOrdinal - lastOffsetCommitted.offset.toInt - 1
121121
val sliceEnd = endOrdinal - lastOffsetCommitted.offset.toInt - 1
122+
assert(sliceStart <= sliceEnd, s"sliceStart: $sliceStart sliceEnd: $sliceEnd")
122123
batches.slice(sliceStart, sliceEnd)
123124
}
124125

126+
if (newBlocks.isEmpty) {
127+
return sqlContext.internalCreateDataFrame(
128+
sqlContext.sparkContext.emptyRDD, schema, isStreaming = true)
129+
}
130+
125131
logDebug(generateDebugString(newBlocks, startOrdinal, endOrdinal))
126132

127133
newBlocks

0 commit comments

Comments
 (0)