forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-47272][SS] Add MapState implementation for State API v2.
### What changes were proposed in this pull request? This PR adds changes for MapState implementation in State Api v2. This implementation adds a new encoder/decoder to encode grouping key and user key into a composite key to be put into RocksDB so that we could retrieve key-value pair by user specified user key by one rocksdb get. ### Why are the changes needed? These changes are needed to support map values in the State Store. The changes are part of the work around adding new stateful streaming operator for arbitrary state mgmt that provides a bunch of new features listed in the SPIP JIRA here - https://issues.apache.org/jira/browse/SPARK-45939 ### Does this PR introduce _any_ user-facing change? Yes This PR introduces a new state type (MapState) that users can use in their Spark streaming queries. ### How was this patch tested? Unit tests in `TransforWithMapStateSuite`. ### Was this patch authored or co-authored using generative AI tooling? No Closes apache#45341 from jingz-db/map-state-impl. Lead-authored-by: jingz-db <[email protected]> Co-authored-by: Jing Zhan <[email protected]> Signed-off-by: Jungtaek Lim <[email protected]>
- Loading branch information
1 parent
621f2c8
commit 29e91d0
Showing
8 changed files
with
701 additions
and
57 deletions.
There are no files selected for viewing
54 changes: 54 additions & 0 deletions
54
sql/api/src/main/scala/org/apache/spark/sql/streaming/MapState.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,54 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.spark.sql.streaming | ||
|
||
import org.apache.spark.annotation.{Evolving, Experimental} | ||
|
||
@Experimental | ||
@Evolving | ||
/** | ||
* Interface used for arbitrary stateful operations with the v2 API to capture | ||
* map value state. | ||
*/ | ||
trait MapState[K, V] extends Serializable { | ||
/** Whether state exists or not. */ | ||
def exists(): Boolean | ||
|
||
/** Get the state value if it exists */ | ||
def getValue(key: K): V | ||
|
||
/** Check if the user key is contained in the map */ | ||
def containsKey(key: K): Boolean | ||
|
||
/** Update value for given user key */ | ||
def updateValue(key: K, value: V) : Unit | ||
|
||
/** Get the map associated with grouping key */ | ||
def iterator(): Iterator[(K, V)] | ||
|
||
/** Get the list of keys present in map associated with grouping key */ | ||
def keys(): Iterator[K] | ||
|
||
/** Get the list of values present in map associated with grouping key */ | ||
def values(): Iterator[V] | ||
|
||
/** Remove user key from map state */ | ||
def removeKey(key: K): Unit | ||
|
||
/** Remove this state. */ | ||
def clear(): Unit | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
110 changes: 110 additions & 0 deletions
110
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImpl.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,110 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.spark.sql.execution.streaming | ||
|
||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.sql.Encoder | ||
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder | ||
import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreErrors, UnsafeRowPair} | ||
import org.apache.spark.sql.streaming.MapState | ||
import org.apache.spark.sql.types.{BinaryType, StructType} | ||
|
||
class MapStateImpl[K, V]( | ||
store: StateStore, | ||
stateName: String, | ||
keyExprEnc: ExpressionEncoder[Any], | ||
userKeyEnc: Encoder[K], | ||
valEncoder: Encoder[V]) extends MapState[K, V] with Logging { | ||
|
||
// Pack grouping key and user key together as a prefixed composite key | ||
private val schemaForCompositeKeyRow: StructType = | ||
new StructType() | ||
.add("key", BinaryType) | ||
.add("userKey", BinaryType) | ||
private val schemaForValueRow: StructType = new StructType().add("value", BinaryType) | ||
private val keySerializer = keyExprEnc.createSerializer() | ||
private val stateTypesEncoder = new CompositeKeyStateEncoder( | ||
keySerializer, userKeyEnc, valEncoder, schemaForCompositeKeyRow, stateName) | ||
|
||
store.createColFamilyIfAbsent(stateName, schemaForCompositeKeyRow, numColsPrefixKey = 1, | ||
schemaForValueRow) | ||
|
||
/** Whether state exists or not. */ | ||
override def exists(): Boolean = { | ||
!store.prefixScan(stateTypesEncoder.encodeGroupingKey(), stateName).isEmpty | ||
} | ||
|
||
/** Get the state value if it exists */ | ||
override def getValue(key: K): V = { | ||
StateStoreErrors.requireNonNullStateValue(key, stateName) | ||
val encodedCompositeKey = stateTypesEncoder.encodeCompositeKey(key) | ||
val unsafeRowValue = store.get(encodedCompositeKey, stateName) | ||
|
||
if (unsafeRowValue == null) return null.asInstanceOf[V] | ||
stateTypesEncoder.decodeValue(unsafeRowValue) | ||
} | ||
|
||
/** Check if the user key is contained in the map */ | ||
override def containsKey(key: K): Boolean = { | ||
StateStoreErrors.requireNonNullStateValue(key, stateName) | ||
getValue(key) != null | ||
} | ||
|
||
/** Update value for given user key */ | ||
override def updateValue(key: K, value: V): Unit = { | ||
StateStoreErrors.requireNonNullStateValue(key, stateName) | ||
StateStoreErrors.requireNonNullStateValue(value, stateName) | ||
val encodedValue = stateTypesEncoder.encodeValue(value) | ||
val encodedCompositeKey = stateTypesEncoder.encodeCompositeKey(key) | ||
store.put(encodedCompositeKey, encodedValue, stateName) | ||
} | ||
|
||
/** Get the map associated with grouping key */ | ||
override def iterator(): Iterator[(K, V)] = { | ||
val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey() | ||
store.prefixScan(encodedGroupingKey, stateName) | ||
.map { | ||
case iter: UnsafeRowPair => | ||
(stateTypesEncoder.decodeCompositeKey(iter.key), | ||
stateTypesEncoder.decodeValue(iter.value)) | ||
} | ||
} | ||
|
||
/** Get the list of keys present in map associated with grouping key */ | ||
override def keys(): Iterator[K] = { | ||
iterator().map(_._1) | ||
} | ||
|
||
/** Get the list of values present in map associated with grouping key */ | ||
override def values(): Iterator[V] = { | ||
iterator().map(_._2) | ||
} | ||
|
||
/** Remove user key from map state */ | ||
override def removeKey(key: K): Unit = { | ||
StateStoreErrors.requireNonNullStateValue(key, stateName) | ||
val compositeKey = stateTypesEncoder.encodeCompositeKey(key) | ||
store.remove(compositeKey, stateName) | ||
} | ||
|
||
/** Remove this state. */ | ||
override def clear(): Unit = { | ||
keys().foreach { itr => | ||
removeKey(itr) | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.