-
Notifications
You must be signed in to change notification settings - Fork 28k
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
[SPARK-47272][SS] Add MapState implementation for State API v2. #45341
Changes from 11 commits
997f6d9
8a13c71
7c32130
c413e8a
cbed9c6
fa10594
d6afc82
14e8fa6
c1797a3
f005f0b
d671cc3
6421952
6e3fafe
84f16c5
646d226
3678127
a87f7ae
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,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 getMap(): Map[K, V] | ||
|
||
/** Get the list of keys present in map associated with grouping key */ | ||
def getKeys(): Iterator[K] | ||
|
||
/** Get the list of values present in map associated with grouping key */ | ||
def getValues(): Iterator[V] | ||
|
||
/** Remove user key from map state */ | ||
def removeKey(key: K): Unit | ||
|
||
/** Remove this state. */ | ||
def clear(): Unit | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,109 @@ | ||
/* | ||
* 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], | ||
userKeyExprEnc: Encoder[K]) 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 = CompositeKeyStateEncoder( | ||
keySerializer, schemaForCompositeKeyRow, stateName, userKeyExprEnc) | ||
|
||
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 getMap(): Map[K, V] = { | ||
val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey() | ||
store.prefixScan(encodedGroupingKey, stateName) | ||
.map { | ||
case iter: UnsafeRowPair => | ||
(stateTypesEncoder.decodeCompositeKey(iter.key), | ||
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. Note: As you will rebase with #45447, UnsafeProjection will reuse the row instance, so we can't store the row persistently unless copying it. If we do copy, we probably want to reduce the scope for key-value vs key vs value. Maybe good to have a private method decoding key and value in iterator but not creating a map. Each method can get the iterator from the new method, and pick key / value / both, and then copy rows. 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. Thanks for the input! Not sure if I understand you correctly, are you trying to say that: we want to return 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. Hi @anishshri-db, need your input on this: Do we want to return 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. Yea sure - lets use the iterator approach. Thx |
||
stateTypesEncoder.decodeValue(iter.value)) | ||
}.toMap | ||
} | ||
|
||
/** Get the list of keys present in map associated with grouping key */ | ||
override def getKeys(): Iterator[K] = { | ||
getMap().keys.iterator | ||
} | ||
|
||
/** Get the list of values present in map associated with grouping key */ | ||
override def getValues(): Iterator[V] = { | ||
getMap().values.iterator | ||
} | ||
|
||
/** 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 = { | ||
getKeys().foreach { itr => | ||
removeKey(itr) | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,8 +19,10 @@ package org.apache.spark.sql.execution.streaming | |
|
||
import org.apache.commons.lang3.SerializationUtils | ||
|
||
import org.apache.spark.sql.Encoder | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.Serializer | ||
import org.apache.spark.sql.catalyst.encoders.encoderFor | ||
import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow} | ||
import org.apache.spark.sql.execution.streaming.state.StateStoreErrors | ||
import org.apache.spark.sql.types.{BinaryType, StructType} | ||
|
@@ -86,3 +88,60 @@ object StateTypesEncoder { | |
new StateTypesEncoder[GK](keySerializer, stateName) | ||
} | ||
} | ||
|
||
class CompositeKeyStateEncoder[GK, K]( | ||
keySerializer: Serializer[GK], | ||
schemaForCompositeKeyRow: StructType, | ||
stateName: String, | ||
userKeyEnc: Encoder[K]) | ||
extends StateTypesEncoder[GK](keySerializer: Serializer[GK], stateName: String) { | ||
jingz-db marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
private val compositeKeyProjection = UnsafeProjection.create(schemaForCompositeKeyRow) | ||
private val reuseRow = new UnsafeRow(userKeyEnc.schema.fields.length) | ||
private val userKeyExpressionEnc = encoderFor(userKeyEnc) | ||
|
||
private val userKeyRowToObjDeserializer = | ||
userKeyExpressionEnc.resolveAndBind().createDeserializer() | ||
private val userKeySerializer = encoderFor(userKeyEnc).createSerializer() | ||
|
||
/** | ||
* Grouping key and user key are encoded as a row of `schemaForCompositeKeyRow` schema. | ||
* Grouping key will be encoded in `RocksDBStateEncoder` as the prefix column. | ||
*/ | ||
def encodeCompositeKey(userKey: K): UnsafeRow = { | ||
val keyOption = ImplicitGroupingKeyTracker.getImplicitKeyOption | ||
if (keyOption.isEmpty) { | ||
throw StateStoreErrors.implicitKeyNotFound(stateName) | ||
} | ||
val groupingKey = keyOption.get.asInstanceOf[GK] | ||
// generate grouping key byte array | ||
val groupingKeyByteArr = keySerializer.apply(groupingKey).asInstanceOf[UnsafeRow].getBytes() | ||
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. I guess you can just directly call 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. Compiler will complain in the next line: |
||
// generate user key byte array | ||
val userKeyBytesArr = userKeySerializer.apply(userKey).asInstanceOf[UnsafeRow].getBytes() | ||
|
||
val compositeKeyRow = compositeKeyProjection(InternalRow(groupingKeyByteArr, userKeyBytesArr)) | ||
compositeKeyRow | ||
} | ||
|
||
/** | ||
* The input row is of composite Key schema. | ||
* Only user key is returned though grouping key also exist in the row. | ||
*/ | ||
def decodeCompositeKey(row: UnsafeRow): K = { | ||
val bytes = row.getBinary(1) | ||
reuseRow.pointTo(bytes, bytes.length) | ||
val userKey = userKeyRowToObjDeserializer.apply(reuseRow) | ||
userKey | ||
} | ||
} | ||
|
||
object CompositeKeyStateEncoder { | ||
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. Hmm - do we really need these singleton objects ? Could we just call 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. I was following Bhuwan's style in the base class. Maybe I am missing something but did not find anything useful in the style guide. 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. If the list of parameters are exactly the same with default constructor, let's just use new. |
||
def apply[GK, K]( | ||
keySerializer: Serializer[GK], | ||
schemaForCompositeKeyRow: StructType, | ||
stateName: String, | ||
userKeyEnc: Encoder[K]): CompositeKeyStateEncoder[GK, K] = { | ||
new CompositeKeyStateEncoder[GK, K]( | ||
keySerializer, schemaForCompositeKeyRow, stateName, userKeyEnc) | ||
} | ||
} |
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.
Just a friendly reminder: I expect value encoder will be in the parameter as well once this is rebased.