Skip to content
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

Closed
wants to merge 17 commits into from
Closed
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
}
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,22 @@ private[sql] trait StatefulProcessorHandle extends Serializable {
*/
def getListState[T](stateName: String, valEncoder: Encoder[T]): ListState[T]

/**
* Creates new or returns existing map state associated with stateName.
* The MapState persists Key-Value pairs of type [K, V].
*
* @param stateName - name of the state variable
* @param userKeyEnc - spark sql encoder for the map key
* @param valEncoder - spark sql encoder for the map value
* @tparam K - type of key for map state variable
* @tparam V - type of value for map state variable
* @return - instance of MapState of type [K,V] that can be used to store state persistently
*/
def getMapState[K, V](
stateName: String,
userKeyEnc: Encoder[K],
valEncoder: Encoder[V]): MapState[K, V]

/** Function to return queryInfo for currently running task */
def getQueryInfo(): QueryInfo

Expand Down
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),
Copy link
Contributor

Choose a reason for hiding this comment

The 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.

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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 returnIterator instead of Map to reduce the copy, and we need to use different reused rows for key/value in StateTypesEncoder?

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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 Map type or Iterator type for getMap function?
Talked with Jungtaek on Slack, if we decide to return Map type, we'll probably need to materialize the map and copy everything in map into memory (because we reuse UnsafeRow in StateTypeEncoder). So Jungtaek is concerning about the case where we have a large map. I also feel like returning Iterator type makes more sense, because for ListState we also return Iterator for get list function.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yea sure - lets use the iterator approach. Thx

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)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ class StateTypesEncoder[GK, V](
private val valExpressionEnc = encoderFor(valEncoder)
private val objToRowSerializer = valExpressionEnc.createSerializer()
private val rowToObjDeserializer = valExpressionEnc.resolveAndBind().createDeserializer()
private val reuseRow = new UnsafeRow(valEncoder.schema.fields.length)
private val reusedValRow = new UnsafeRow(valEncoder.schema.fields.length)

// TODO: validate places that are trying to encode the key and check if we can eliminate/
// add caching for some of these calls.
Expand All @@ -85,8 +85,8 @@ class StateTypesEncoder[GK, V](

def decodeValue(row: UnsafeRow): V = {
val bytes = row.getBinary(0)
reuseRow.pointTo(bytes, bytes.length)
val value = rowToObjDeserializer.apply(reuseRow)
reusedValRow.pointTo(bytes, bytes.length)
val value = rowToObjDeserializer.apply(reusedValRow)
value
}
}
Expand All @@ -99,3 +99,50 @@ object StateTypesEncoder {
new StateTypesEncoder[GK, V](keySerializer, valEncoder, stateName)
}
}

class CompositeKeyStateEncoder[GK, K, V](
keySerializer: Serializer[GK],
userKeyEnc: Encoder[K],
valEncoder: Encoder[V],
schemaForCompositeKeyRow: StructType,
stateName: String)
extends StateTypesEncoder[GK, V](keySerializer, valEncoder, stateName) {

private val compositeKeyProjection = UnsafeProjection.create(schemaForCompositeKeyRow)
private val reusedKeyRow = 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()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I guess you can just directly call keyOption.get here ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Compiler will complain in the next line:
keySerializer.apply(groupingKey) where groupingKey will be of Any type if we directly call keyOption.get

// 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)
reusedKeyRow.pointTo(bytes, bytes.length)
val userKey = userKeyRowToObjDeserializer.apply(reusedKeyRow)
userKey
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ 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
import org.apache.spark.sql.streaming.{ListState, QueryInfo, StatefulProcessorHandle, ValueState}
import org.apache.spark.sql.streaming.{ListState, MapState, QueryInfo, StatefulProcessorHandle, ValueState}
import org.apache.spark.util.Utils

/**
Expand Down Expand Up @@ -139,4 +139,14 @@ class StatefulProcessorHandleImpl(
val resultState = new ListStateImpl[T](store, stateName, keyEncoder, valEncoder)
resultState
}

override def getMapState[K, V](
stateName: String,
userKeyEnc: Encoder[K],
valEncoder: Encoder[V]): MapState[K, V] = {
verify(currState == CREATED, s"Cannot create state variable with name=$stateName after " +
"initialization is complete")
val resultState = new MapStateImpl[K, V](store, stateName, keyEncoder, userKeyEnc, valEncoder)
resultState
}
}