-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-54583][SS] Add SQLConf to enable use of OffsetMap #53311
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 3 commits
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 |
|---|---|---|
|
|
@@ -169,7 +169,7 @@ object OffsetSeqMetadata extends Logging { | |
| STREAMING_JOIN_STATE_FORMAT_VERSION, STATE_STORE_COMPRESSION_CODEC, | ||
| STATE_STORE_ROCKSDB_FORMAT_VERSION, STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION, | ||
| PRUNE_FILTERS_CAN_PRUNE_STREAMING_SUBPLAN, STREAMING_STATE_STORE_ENCODING_FORMAT, | ||
| STATE_STORE_ROW_CHECKSUM_ENABLED | ||
| STATE_STORE_ROW_CHECKSUM_ENABLED, STREAMING_OFFSET_LOG_FORMAT_VERSION | ||
| ) | ||
|
|
||
| /** | ||
|
|
@@ -244,7 +244,8 @@ object OffsetSeqMetadata extends Logging { | |
| case (confInSession, confInOffsetLog) => | ||
| confInOffsetLog.key -> sessionConf.get(confInSession.key) | ||
| }.toMap | ||
| OffsetSeqMetadata(batchWatermarkMs, batchTimestampMs, confs++ confsFromRebind) | ||
| val version = sessionConf.get(STREAMING_OFFSET_LOG_FORMAT_VERSION.key).toInt | ||
|
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. Does the version here refer to the OffsetSeqMetadata version or the OffsetSeqBase version?
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. OffsetSeqBase - we don't have multiple metadata versions here. |
||
| OffsetSeqMetadata(batchWatermarkMs, batchTimestampMs, confs++ confsFromRebind, version) | ||
| } | ||
|
|
||
| /** Set the SparkSession configuration with the values in the metadata */ | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,7 +21,7 @@ import java.io.File | |
|
|
||
| import org.apache.spark.sql.catalyst.util.stringToFile | ||
| import org.apache.spark.sql.execution.streaming.checkpointing.{OffsetMap, OffsetSeq, OffsetSeqBase, OffsetSeqLog, OffsetSeqMetadata} | ||
| import org.apache.spark.sql.execution.streaming.runtime.{LongOffset, SerializedOffset} | ||
| import org.apache.spark.sql.execution.streaming.runtime.{LongOffset, MemoryStream, SerializedOffset} | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.test.SharedSparkSession | ||
| import org.apache.spark.util.Utils | ||
|
|
@@ -237,4 +237,151 @@ class OffsetSeqLogSuite extends SharedSparkSession { | |
| assert(metadata.batchWatermarkMs === 0) | ||
| assert(metadata.batchTimestampMs === 1758651405232L) | ||
| } | ||
|
|
||
| test("STREAMING_OFFSET_LOG_FORMAT_VERSION config - new query with VERSION_2") { | ||
ericm-db marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| import testImplicits._ | ||
| withTempDir { checkpointDir => | ||
| withSQLConf(SQLConf.STREAMING_OFFSET_LOG_FORMAT_VERSION.key -> "2") { | ||
| val inputData = MemoryStream[Int] | ||
| val query = inputData.toDF() | ||
| .writeStream | ||
| .format("memory") | ||
| .queryName("offsetlog_v2_test") | ||
| .option("checkpointLocation", checkpointDir.getAbsolutePath) | ||
| .start() | ||
|
|
||
| try { | ||
| // Add data and process batches | ||
| inputData.addData(1, 2, 3) | ||
| query.processAllAvailable() | ||
| inputData.addData(4, 5) | ||
| query.processAllAvailable() | ||
|
|
||
| // Read the offset log from checkpoint | ||
| val offsetLog = new OffsetSeqLog(spark, s"${checkpointDir.getAbsolutePath}/offsets") | ||
| val latestBatch = offsetLog.getLatest() | ||
| assert(latestBatch.isDefined, "Offset log should have at least one entry") | ||
|
|
||
| val (batchId, offsetSeq) = latestBatch.get | ||
|
|
||
| // Verify it's OffsetMap (VERSION_2) | ||
| assert(offsetSeq.isInstanceOf[OffsetMap], | ||
| s"Expected OffsetMap but got ${offsetSeq.getClass.getSimpleName}") | ||
|
|
||
| // Verify metadata version is 2 | ||
| assert(offsetSeq.metadataOpt.isDefined) | ||
| val metadata = offsetSeq.metadataOpt.get | ||
| assert(metadata.version === 2, s"Expected version 2 but got ${metadata.version}") | ||
|
|
||
| // Verify the config is persisted | ||
| assert(metadata.conf.contains(SQLConf.STREAMING_OFFSET_LOG_FORMAT_VERSION.key)) | ||
| assert(metadata.conf(SQLConf.STREAMING_OFFSET_LOG_FORMAT_VERSION.key) === "2") | ||
|
|
||
| // Verify OffsetMap uses string keys ("0", "1", etc.) | ||
| val offsetMap = offsetSeq.asInstanceOf[OffsetMap] | ||
| assert(offsetMap.offsetsMap.keys.forall(_.forall(_.isDigit)), | ||
ericm-db marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| "OffsetMap keys should be string representations of ordinals (e.g., '0', '1')") | ||
| } finally { | ||
| query.stop() | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| test("STREAMING_OFFSET_LOG_FORMAT_VERSION config - default VERSION_1") { | ||
| import testImplicits._ | ||
|
||
| withTempDir { checkpointDir => | ||
| // Don't set the config, should default to VERSION_1 | ||
| val inputData = MemoryStream[Int] | ||
| val query = inputData.toDF() | ||
| .writeStream | ||
| .format("memory") | ||
| .queryName("offsetlog_v1_test") | ||
| .option("checkpointLocation", checkpointDir.getAbsolutePath) | ||
| .start() | ||
|
|
||
| try { | ||
| // Add data and process batches | ||
| inputData.addData(1, 2, 3) | ||
| query.processAllAvailable() | ||
|
|
||
| // Read the offset log from checkpoint | ||
| val offsetLog = new OffsetSeqLog(spark, s"${checkpointDir.getAbsolutePath}/offsets") | ||
| val latestBatch = offsetLog.getLatest() | ||
| assert(latestBatch.isDefined, "Offset log should have at least one entry") | ||
|
|
||
| val (batchId, offsetSeq) = latestBatch.get | ||
|
|
||
| // Verify it's OffsetSeq (VERSION_1) | ||
| assert(offsetSeq.isInstanceOf[OffsetSeq], | ||
| s"Expected OffsetSeq but got ${offsetSeq.getClass.getSimpleName}") | ||
|
|
||
| // Verify metadata version is 1 | ||
| assert(offsetSeq.metadataOpt.isDefined) | ||
| val metadata = offsetSeq.metadataOpt.get | ||
| assert(metadata.version === 1, s"Expected version 1 but got ${metadata.version}") | ||
| } finally { | ||
| query.stop() | ||
| } | ||
| } | ||
| } | ||
|
|
||
| test("STREAMING_OFFSET_LOG_FORMAT_VERSION config - checkpoint wins on restart") { | ||
|
||
| import testImplicits._ | ||
| withTempDir { checkpointDir => | ||
| withTempDir { outputDir => | ||
| val inputData = MemoryStream[Int] | ||
|
|
||
| // Start query with VERSION_1 (default) | ||
| val query1 = inputData.toDF() | ||
| .writeStream | ||
| .format("parquet") | ||
| .option("path", outputDir.getAbsolutePath) | ||
| .option("checkpointLocation", checkpointDir.getAbsolutePath) | ||
| .start() | ||
|
|
||
| inputData.addData(1, 2) | ||
| query1.processAllAvailable() | ||
| query1.stop() | ||
|
|
||
| // Verify VERSION_1 was used in the initial checkpoint | ||
| val offsetLog = new OffsetSeqLog(spark, s"${checkpointDir.getAbsolutePath}/offsets") | ||
| val batch1 = offsetLog.getLatest() | ||
| assert(batch1.isDefined) | ||
| assert(batch1.get._2.isInstanceOf[OffsetSeq], "Initial checkpoint should use VERSION_1") | ||
| assert(batch1.get._2.metadataOpt.get.version === 1) | ||
|
|
||
| // Restart query with VERSION_2 config - should still use VERSION_1 from checkpoint | ||
| withSQLConf(SQLConf.STREAMING_OFFSET_LOG_FORMAT_VERSION.key -> "2") { | ||
| val query2 = inputData.toDF() | ||
| .writeStream | ||
| .format("parquet") | ||
| .option("path", outputDir.getAbsolutePath) | ||
| .option("checkpointLocation", checkpointDir.getAbsolutePath) | ||
| .start() | ||
|
|
||
| try { | ||
| inputData.addData(3, 4) | ||
| query2.processAllAvailable() | ||
|
|
||
| // Read the latest offset log entry | ||
| val latestBatch = offsetLog.getLatest() | ||
| assert(latestBatch.isDefined) | ||
|
|
||
| val (batchId, offsetSeq) = latestBatch.get | ||
|
|
||
| // Should still be VERSION_1 because checkpoint was created with VERSION_1 | ||
| assert(offsetSeq.isInstanceOf[OffsetSeq], | ||
| "Query should continue using VERSION_1 format from checkpoint despite config change") | ||
|
|
||
| val metadata = offsetSeq.metadataOpt.get | ||
| assert(metadata.version === 1, | ||
| "Query should continue using version 1 from checkpoint despite config being set to 2") | ||
| } finally { | ||
| query2.stop() | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.