-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-53294][SS] Enable StateDataSource with state checkpoint v2 (only batchId option) #52047
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
eaed968
188491d
dd1eb2a
e2104d6
03ca2ff
a3d3161
c286e97
3d1dfb2
9662334
cd6aac3
6b38181
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 |
|---|---|---|
|
|
@@ -370,7 +370,8 @@ case class StateSourceOptions( | |
| readChangeFeedOptions: Option[ReadChangeFeedOptions], | ||
| stateVarName: Option[String], | ||
| readRegisteredTimers: Boolean, | ||
| flattenCollectionTypes: Boolean) { | ||
| flattenCollectionTypes: Boolean, | ||
| operatorStateUniqueIds: Option[Array[Array[String]]] = None) { | ||
| def stateCheckpointLocation: Path = new Path(resolvedCpLocation, DIR_NAME_STATE) | ||
|
|
||
| override def toString: String = { | ||
|
|
@@ -567,10 +568,38 @@ object StateSourceOptions extends DataSourceOptions { | |
| } | ||
| } | ||
|
|
||
|
|
||
| val startBatchId = if (fromSnapshotOptions.isDefined) { | ||
| fromSnapshotOptions.get.snapshotStartBatchId | ||
| } else if (readChangeFeedOptions.isDefined) { | ||
| readChangeFeedOptions.get.changeStartBatchId | ||
| } else { | ||
| batchId.get | ||
| } | ||
|
|
||
| val operatorStateUniqueIds = getOperatorStateUniqueIds( | ||
| sparkSession, | ||
| startBatchId, | ||
| operatorId, | ||
| resolvedCpLocation) | ||
|
|
||
| if (operatorStateUniqueIds.isDefined) { | ||
| if (fromSnapshotOptions.isDefined) { | ||
| throw StateDataSourceErrors.invalidOptionValue( | ||
| SNAPSHOT_START_BATCH_ID, | ||
| "Snapshot reading is currently not supported with checkpoint v2.") | ||
| } | ||
| if (readChangeFeedOptions.isDefined) { | ||
| throw StateDataSourceErrors.invalidOptionValue( | ||
| READ_CHANGE_FEED, | ||
| "Read change feed is currently not supported with checkpoint v2.") | ||
| } | ||
| } | ||
|
|
||
| StateSourceOptions( | ||
| resolvedCpLocation, batchId.get, operatorId, storeName, joinSide, | ||
| readChangeFeed, fromSnapshotOptions, readChangeFeedOptions, | ||
| stateVarName, readRegisteredTimers, flattenCollectionTypes) | ||
| stateVarName, readRegisteredTimers, flattenCollectionTypes, operatorStateUniqueIds) | ||
| } | ||
|
|
||
| private def resolvedCheckpointLocation( | ||
|
|
@@ -589,6 +618,26 @@ object StateSourceOptions extends DataSourceOptions { | |
| } | ||
| } | ||
|
|
||
| private def getOperatorStateUniqueIds( | ||
| session: SparkSession, | ||
| batchId: Long, | ||
| operatorId: Long, | ||
| checkpointLocation: String): Option[Array[Array[String]]] = { | ||
| val commitLog = new StreamingQueryCheckpointMetadata(session, checkpointLocation).commitLog | ||
| val commitMetadata = commitLog.get(batchId) match { | ||
| case Some(commitMetadata) => commitMetadata | ||
| case None => throw StateDataSourceErrors.committedBatchUnavailable(checkpointLocation) | ||
| } | ||
|
|
||
| val operatorStateUniqueIds = if (commitMetadata.stateUniqueIds.isDefined) { | ||
dylanwong250 marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| Some(commitMetadata.stateUniqueIds.get(operatorId)) | ||
|
||
| } else { | ||
| None | ||
| } | ||
|
|
||
| operatorStateUniqueIds | ||
| } | ||
|
|
||
| // Modifies options due to external data. Returns modified options. | ||
| // If this is a join operator specifying a store name using state format v3, | ||
| // we need to modify the options. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,13 +21,24 @@ import org.apache.spark.sql.catalyst.InternalRow | |
| import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeRow} | ||
| import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} | ||
| import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil | ||
| import org.apache.spark.sql.execution.streaming.operators.stateful.join.SymmetricHashJoinStateManager | ||
| import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.{StateVariableType, TransformWithStateVariableInfo} | ||
| import org.apache.spark.sql.execution.streaming.state._ | ||
| import org.apache.spark.sql.execution.streaming.state.RecordType.{getRecordTypeAsString, RecordType} | ||
| import org.apache.spark.sql.types.{NullType, StructField, StructType} | ||
| import org.apache.spark.unsafe.types.UTF8String | ||
| import org.apache.spark.util.{NextIterator, SerializableConfiguration} | ||
|
|
||
| /** | ||
| * Constants for store names used in Stream-Stream joins. | ||
| */ | ||
| object StatePartitionReaderStoreNames { | ||
|
||
| val LEFT_KEY_TO_NUM_VALUES_STORE = "left-keyToNumValues" | ||
| val LEFT_KEY_WITH_INDEX_TO_VALUE_STORE = "left-keyWithIndexToValue" | ||
| val RIGHT_KEY_TO_NUM_VALUES_STORE = "right-keyToNumValues" | ||
| val RIGHT_KEY_WITH_INDEX_TO_VALUE_STORE = "right-keyWithIndexToValue" | ||
| } | ||
|
|
||
| /** | ||
| * An implementation of [[PartitionReaderFactory]] for State data source. This is used to support | ||
| * general read from a state store instance, rather than specific to the operator. | ||
|
|
@@ -95,6 +106,31 @@ abstract class StatePartitionReaderBase( | |
| schema, "value").asInstanceOf[StructType] | ||
| } | ||
|
|
||
| protected val getStoreUniqueId : Option[String] = { | ||
| val partitionStateUniqueIds = | ||
| partition.sourceOptions.operatorStateUniqueIds.map(_(partition.partition)) | ||
| if (partition.sourceOptions.storeName == StateStoreId.DEFAULT_STORE_NAME) { | ||
| partitionStateUniqueIds.map(_.head) | ||
| } else { | ||
| val stateStoreCheckpointIds = SymmetricHashJoinStateManager.getStateStoreCheckpointIds( | ||
| partition.partition, | ||
| partition.sourceOptions.operatorStateUniqueIds, | ||
| useColumnFamiliesForJoins = false) | ||
|
|
||
| partition.sourceOptions.storeName match { | ||
|
||
| case StatePartitionReaderStoreNames.LEFT_KEY_TO_NUM_VALUES_STORE => | ||
| stateStoreCheckpointIds.left.keyToNumValues | ||
| case StatePartitionReaderStoreNames.LEFT_KEY_WITH_INDEX_TO_VALUE_STORE => | ||
| stateStoreCheckpointIds.left.valueToNumKeys | ||
| case StatePartitionReaderStoreNames.RIGHT_KEY_TO_NUM_VALUES_STORE => | ||
| stateStoreCheckpointIds.right.keyToNumValues | ||
| case StatePartitionReaderStoreNames.RIGHT_KEY_WITH_INDEX_TO_VALUE_STORE => | ||
| stateStoreCheckpointIds.right.valueToNumKeys | ||
| case _ => None | ||
| } | ||
| } | ||
| } | ||
|
|
||
| protected lazy val provider: StateStoreProvider = { | ||
| val stateStoreId = StateStoreId(partition.sourceOptions.stateCheckpointLocation.toString, | ||
| partition.sourceOptions.operatorId, partition.partition, partition.sourceOptions.storeName) | ||
|
|
@@ -113,7 +149,9 @@ abstract class StatePartitionReaderBase( | |
| val isInternal = partition.sourceOptions.readRegisteredTimers | ||
|
|
||
| if (useColFamilies) { | ||
| val store = provider.getStore(partition.sourceOptions.batchId + 1) | ||
| val store = provider.getStore( | ||
| partition.sourceOptions.batchId + 1, | ||
| getStoreUniqueId) | ||
| require(stateStoreColFamilySchemaOpt.isDefined) | ||
| val stateStoreColFamilySchema = stateStoreColFamilySchemaOpt.get | ||
| require(stateStoreColFamilySchema.keyStateEncoderSpec.isDefined) | ||
|
|
@@ -171,7 +209,11 @@ class StatePartitionReader( | |
|
|
||
| private lazy val store: ReadStateStore = { | ||
| partition.sourceOptions.fromSnapshotOptions match { | ||
| case None => provider.getReadStore(partition.sourceOptions.batchId + 1) | ||
| case None => | ||
| provider.getReadStore( | ||
| partition.sourceOptions.batchId + 1, | ||
| getStoreUniqueId | ||
| ) | ||
|
|
||
| case Some(fromSnapshotOptions) => | ||
| if (!provider.isInstanceOf[SupportsFineGrainedReplay]) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -71,6 +71,28 @@ class StreamStreamJoinStatePartitionReader( | |
| throw StateDataSourceErrors.internalError("Unexpected join side for stream-stream read!") | ||
| } | ||
|
|
||
| private val usesVirtualColumnFamilies = StreamStreamJoinStateHelper.usesVirtualColumnFamilies( | ||
| hadoopConf.value, | ||
| partition.sourceOptions.stateCheckpointLocation.toString, | ||
| partition.sourceOptions.operatorId) | ||
|
|
||
| private val stateStoreCheckpointIds = SymmetricHashJoinStateManager.getStateStoreCheckpointIds( | ||
| partition.partition, | ||
| partition.sourceOptions.operatorStateUniqueIds, | ||
| usesVirtualColumnFamilies) | ||
|
|
||
| private val keyToNumValuesStateStoreCkptId = if (joinSide == LeftSide) { | ||
| stateStoreCheckpointIds.left.keyToNumValues | ||
| } else { | ||
| stateStoreCheckpointIds.right.keyToNumValues | ||
| } | ||
|
|
||
| private val keyWithIndexToValueStateStoreCkptId = if (joinSide == LeftSide) { | ||
| stateStoreCheckpointIds.left.valueToNumKeys | ||
|
||
| } else { | ||
| stateStoreCheckpointIds.right.valueToNumKeys | ||
| } | ||
|
|
||
| /* | ||
| * This is to handle the difference of schema across state format versions. The major difference | ||
| * is whether we have added new field(s) in addition to the fields from input schema. | ||
|
|
@@ -85,10 +107,7 @@ class StreamStreamJoinStatePartitionReader( | |
| // column from the value schema to get the actual fields. | ||
| if (maybeMatchedColumn.name == "matched" && maybeMatchedColumn.dataType == BooleanType) { | ||
| // If checkpoint is using one store and virtual column families, version is 3 | ||
| if (StreamStreamJoinStateHelper.usesVirtualColumnFamilies( | ||
| hadoopConf.value, | ||
| partition.sourceOptions.stateCheckpointLocation.toString, | ||
| partition.sourceOptions.operatorId)) { | ||
| if (usesVirtualColumnFamilies) { | ||
| (valueSchema.dropRight(1), 3) | ||
| } else { | ||
| (valueSchema.dropRight(1), 2) | ||
|
|
@@ -130,8 +149,8 @@ class StreamStreamJoinStatePartitionReader( | |
| storeConf = storeConf, | ||
| hadoopConf = hadoopConf.value, | ||
| partitionId = partition.partition, | ||
| keyToNumValuesStateStoreCkptId = None, | ||
| keyWithIndexToValueStateStoreCkptId = None, | ||
| keyToNumValuesStateStoreCkptId = keyToNumValuesStateStoreCkptId, | ||
| keyWithIndexToValueStateStoreCkptId = keyWithIndexToValueStateStoreCkptId, | ||
| formatVersion, | ||
| skippedNullValueCount = None, | ||
| useStateStoreCoordinator = false, | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -690,7 +690,7 @@ private[sql] class RocksDBStateStoreProvider | |
|
|
||
| rocksDB.load( | ||
| version, | ||
| stateStoreCkptId = if (storeConf.enableStateStoreCheckpointIds) uniqueId else None, | ||
| stateStoreCkptId = uniqueId, | ||
|
||
| readOnly = readOnly) | ||
|
|
||
| // Create or reuse store instance | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.