-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-19873][SS] Record num shuffle partitions in offset log and enforce in next batch. #17216
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
12f5fd3
9ff4d29
60ec7da
c688e84
f6bd071
3af1cb4
1cacd32
030e635
5c851a5
dfae7be
4733b4e
3ae4414
a2b32ce
3abe0a0
a0c71af
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 |
|---|---|---|
| @@ -0,0 +1 @@ | ||
| {"id":"dddc5e7f-1e71-454c-8362-de184444fb5a"} |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,3 @@ | ||
| v1 | ||
| {"batchWatermarkMs":0,"batchTimestampMs":1489180207737} | ||
| 0 |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,3 @@ | ||
| v1 | ||
| {"batchWatermarkMs":0,"batchTimestampMs":1489180209261} | ||
| 2 |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,6 +21,7 @@ import java.io.File | |
|
|
||
| import org.apache.spark.SparkFunSuite | ||
| import org.apache.spark.sql.catalyst.util.stringToFile | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.test.SharedSQLContext | ||
|
|
||
| class OffsetSeqLogSuite extends SparkFunSuite with SharedSQLContext { | ||
|
|
@@ -29,12 +30,37 @@ class OffsetSeqLogSuite extends SparkFunSuite with SharedSQLContext { | |
| case class StringOffset(override val json: String) extends Offset | ||
|
|
||
| test("OffsetSeqMetadata - deserialization") { | ||
| assert(OffsetSeqMetadata(0, 0) === OffsetSeqMetadata("""{}""")) | ||
| assert(OffsetSeqMetadata(1, 0) === OffsetSeqMetadata("""{"batchWatermarkMs":1}""")) | ||
| assert(OffsetSeqMetadata(0, 2) === OffsetSeqMetadata("""{"batchTimestampMs":2}""")) | ||
| assert( | ||
| OffsetSeqMetadata(1, 2) === | ||
| OffsetSeqMetadata("""{"batchWatermarkMs":1,"batchTimestampMs":2}""")) | ||
| val key = SQLConf.SHUFFLE_PARTITIONS.key | ||
|
|
||
| def getConfWith(shufflePartitions: Int): Map[String, String] = { | ||
| Map(key -> shufflePartitions.toString) | ||
| } | ||
|
|
||
| // None set | ||
| assert(OffsetSeqMetadata(0, 0, Map.empty) === OffsetSeqMetadata("""{}""")) | ||
|
|
||
| // One set | ||
| assert(OffsetSeqMetadata(1, 0, Map.empty) === OffsetSeqMetadata("""{"batchWatermarkMs":1}""")) | ||
| assert(OffsetSeqMetadata(0, 2, Map.empty) === OffsetSeqMetadata("""{"batchTimestampMs":2}""")) | ||
| assert(OffsetSeqMetadata(0, 0, getConfWith(shufflePartitions = 2)) === | ||
| OffsetSeqMetadata(s"""{"conf": {"$key":2}}""")) | ||
|
|
||
| // Two set | ||
| assert(OffsetSeqMetadata(1, 2, Map.empty) === | ||
| OffsetSeqMetadata("""{"batchWatermarkMs":1,"batchTimestampMs":2}""")) | ||
| assert(OffsetSeqMetadata(1, 0, getConfWith(shufflePartitions = 3)) === | ||
| OffsetSeqMetadata(s"""{"batchWatermarkMs":1,"conf": {"$key":3}}""")) | ||
| assert(OffsetSeqMetadata(0, 2, getConfWith(shufflePartitions = 3)) === | ||
| OffsetSeqMetadata(s"""{"batchTimestampMs":2,"conf": {"$key":3}}""")) | ||
|
|
||
| // All set | ||
| assert(OffsetSeqMetadata(1, 2, getConfWith(shufflePartitions = 3)) === | ||
| OffsetSeqMetadata(s"""{"batchWatermarkMs":1,"batchTimestampMs":2,"conf": {"$key":3}}""")) | ||
|
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. nit: could you add a test to verify that unknown fields don't break the serialization? Such as assert(OffsetSeqMetadata(1, 2, getConfWith(shufflePartitions = 3)) ===
OffsetSeqMetadata(
s"""{"batchWatermarkMs":1,"batchTimestampMs":2,"conf": {"$key":3}},"unknown":1"""))
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. Added. |
||
|
|
||
| // Drop unknown fields | ||
| assert(OffsetSeqMetadata(1, 2, getConfWith(shufflePartitions = 3)) === | ||
| OffsetSeqMetadata( | ||
| s"""{"batchWatermarkMs":1,"batchTimestampMs":2,"conf": {"$key":3}},"unknown":1""")) | ||
| } | ||
|
|
||
| test("OffsetSeqLog - serialization - deserialization") { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,17 +17,20 @@ | |
|
|
||
| package org.apache.spark.sql.streaming | ||
|
|
||
| import java.io.{InterruptedIOException, IOException} | ||
| import java.io.{File, InterruptedIOException, IOException} | ||
| import java.util.concurrent.{CountDownLatch, TimeoutException, TimeUnit} | ||
|
|
||
| import scala.reflect.ClassTag | ||
| import scala.util.control.ControlThrowable | ||
|
|
||
| import org.apache.commons.io.FileUtils | ||
|
|
||
| import org.apache.spark.sql._ | ||
| import org.apache.spark.sql.catalyst.streaming.InternalOutputModes | ||
| import org.apache.spark.sql.execution.command.ExplainCommand | ||
| import org.apache.spark.sql.execution.streaming._ | ||
| import org.apache.spark.sql.functions._ | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.sources.StreamSourceProvider | ||
| import org.apache.spark.sql.types.{IntegerType, StructField, StructType} | ||
|
|
||
|
|
@@ -389,6 +392,102 @@ class StreamSuite extends StreamTest { | |
| query.stop() | ||
| assert(query.exception.isEmpty) | ||
| } | ||
|
|
||
| test("SPARK-19873: streaming aggregation with change in number of partitions") { | ||
| val inputData = MemoryStream[(Int, Int)] | ||
| val agg = inputData.toDS().groupBy("_1").count() | ||
|
|
||
| testStream(agg, OutputMode.Complete())( | ||
| AddData(inputData, (1, 0), (2, 0)), | ||
| StartStream(additionalConfs = Map(SQLConf.SHUFFLE_PARTITIONS.key -> "2")), | ||
| CheckAnswer((1, 1), (2, 1)), | ||
| StopStream, | ||
| AddData(inputData, (3, 0), (2, 0)), | ||
| StartStream(additionalConfs = Map(SQLConf.SHUFFLE_PARTITIONS.key -> "5")), | ||
| CheckAnswer((1, 1), (2, 2), (3, 1)), | ||
| StopStream, | ||
| AddData(inputData, (3, 0), (1, 0)), | ||
| StartStream(additionalConfs = Map(SQLConf.SHUFFLE_PARTITIONS.key -> "1")), | ||
| CheckAnswer((1, 2), (2, 2), (3, 2))) | ||
| } | ||
|
|
||
| test("recover from a Spark v2.1 checkpoint") { | ||
| var inputData: MemoryStream[Int] = null | ||
| var query: DataStreamWriter[Row] = null | ||
|
|
||
| def prepareMemoryStream(): Unit = { | ||
| inputData = MemoryStream[Int] | ||
| inputData.addData(1, 2, 3, 4) | ||
| inputData.addData(3, 4, 5, 6) | ||
| inputData.addData(5, 6, 7, 8) | ||
|
|
||
| query = inputData | ||
| .toDF() | ||
| .groupBy($"value") | ||
| .agg(count("*")) | ||
| .writeStream | ||
| .outputMode("complete") | ||
| .format("memory") | ||
| } | ||
|
|
||
| // Get an existing checkpoint generated by Spark v2.1. | ||
| // v2.1 does not record # shuffle partitions in the offset metadata. | ||
| val resourceUri = | ||
| this.getClass.getResource("/structured-streaming/checkpoint-version-2.1.0").toURI | ||
| val checkpointDir = new File(resourceUri) | ||
|
|
||
| // 1 - Test if recovery from the checkpoint is successful. | ||
| prepareMemoryStream() | ||
| withTempDir { dir => | ||
| // Copy the checkpoint to a temp dir to prevent changes to the original. | ||
| // Not doing this will lead to the test passing on the first run, but fail subsequent runs. | ||
| FileUtils.copyDirectory(checkpointDir, dir) | ||
|
|
||
| // Checkpoint data was generated by a query with 10 shuffle partitions. | ||
| // In order to test reading from the checkpoint, the checkpoint must have two or more batches, | ||
| // since the last batch may be rerun. | ||
|
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. |
||
| withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "10") { | ||
| var streamingQuery: StreamingQuery = null | ||
| try { | ||
| streamingQuery = | ||
| query.queryName("counts").option("checkpointLocation", dir.getCanonicalPath).start() | ||
| streamingQuery.processAllAvailable() | ||
| inputData.addData(9) | ||
| streamingQuery.processAllAvailable() | ||
|
|
||
| QueryTest.checkAnswer(spark.table("counts").toDF(), | ||
| Row("1", 1) :: Row("2", 1) :: Row("3", 2) :: Row("4", 2) :: | ||
| Row("5", 2) :: Row("6", 2) :: Row("7", 1) :: Row("8", 1) :: Row("9", 1) :: Nil) | ||
| } finally { | ||
| if (streamingQuery ne null) { | ||
| streamingQuery.stop() | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| // 2 - Check recovery with wrong num shuffle partitions | ||
| prepareMemoryStream() | ||
| withTempDir { dir => | ||
| FileUtils.copyDirectory(checkpointDir, dir) | ||
|
|
||
| // Since the number of partitions is greater than 10, should throw exception. | ||
| withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "15") { | ||
| var streamingQuery: StreamingQuery = null | ||
| try { | ||
| intercept[StreamingQueryException] { | ||
|
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. what is the error message?
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. |
||
| streamingQuery = | ||
| query.queryName("badQuery").option("checkpointLocation", dir.getCanonicalPath).start() | ||
| streamingQuery.processAllAvailable() | ||
| } | ||
| } finally { | ||
| if (streamingQuery ne null) { | ||
| streamingQuery.stop() | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| abstract class FakeSource extends StreamSourceProvider { | ||
|
|
||
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: remove line.
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.
Yeah, this should be kept. It should use the conf in the cloned session.