This is an automated email from the ASF dual-hosted git repository.

kabhwan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 29e91d0e48a3 [SPARK-47272][SS] Add MapState implementation for State 
API v2.
29e91d0e48a3 is described below

commit 29e91d0e48a342ceae1259a9b0d10cb27244a14a
Author: jingz-db <jing.z...@databricks.com>
AuthorDate: Wed Mar 13 07:18:25 2024 +0900

    [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 #45341 from jingz-db/map-state-impl.
    
    Lead-authored-by: jingz-db <jing.z...@databricks.com>
    Co-authored-by: Jing Zhan <135738831+jingz...@users.noreply.github.com>
    Signed-off-by: Jungtaek Lim <kabhwan.opensou...@gmail.com>
---
 .../org/apache/spark/sql/streaming/MapState.scala  |  54 +++++
 .../sql/streaming/StatefulProcessorHandle.scala    |  16 ++
 .../sql/execution/streaming/MapStateImpl.scala     | 110 ++++++++++
 .../streaming/StateTypesEncoderUtils.scala         |  53 ++++-
 .../streaming/StatefulProcessorHandleImpl.scala    |  12 +-
 .../execution/streaming/state/MapStateSuite.scala  | 170 ++++++++++++++++
 .../streaming/state/ValueStateSuite.scala          | 117 ++++++-----
 .../sql/streaming/TransformWithMapStateSuite.scala | 226 +++++++++++++++++++++
 8 files changed, 701 insertions(+), 57 deletions(-)

diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/MapState.scala 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/MapState.scala
new file mode 100644
index 000000000000..030c3ee989c6
--- /dev/null
+++ b/sql/api/src/main/scala/org/apache/spark/sql/streaming/MapState.scala
@@ -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
+}
diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
index 86bf1e85f90c..c26d0d806b86 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
@@ -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
 
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImpl.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImpl.scala
new file mode 100644
index 000000000000..91f6be4ddfd1
--- /dev/null
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImpl.scala
@@ -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)
+    }
+  }
+}
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala
index 36758eafa392..1d41db896cdf 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala
@@ -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.
@@ -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
   }
 }
@@ -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()
+    // 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
+  }
+}
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
index d5dd9fcaf401..fde8d5c3c1e5 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
@@ -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
 
 /**
@@ -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
+  }
 }
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MapStateSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MapStateSuite.scala
new file mode 100644
index 000000000000..d3ed9b5824a1
--- /dev/null
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MapStateSuite.scala
@@ -0,0 +1,170 @@
+/*
+ * 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.state
+
+import java.util.UUID
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.execution.streaming.{ImplicitGroupingKeyTracker, 
StatefulProcessorHandleImpl}
+import org.apache.spark.sql.streaming.{ListState, MapState, ValueState}
+import org.apache.spark.sql.types.{BinaryType, StructType}
+
+/**
+ * Class that adds unit tests for MapState types used in arbitrary stateful
+ * operators such as transformWithState
+ */
+class MapStateSuite extends StateVariableSuiteBase {
+  // Overwrite Key schema as MapState use composite key
+  schemaForKeyRow = new StructType()
+    .add("key", BinaryType)
+    .add("userKey", BinaryType)
+
+  test("Map state operations for single instance") {
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
+      val store = provider.getStore(0)
+      val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
+        Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]])
+
+      val testState: MapState[String, Double] =
+        handle.getMapState[String, Double]("testState", Encoders.STRING, 
Encoders.scalaDouble)
+      ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+      // put initial value
+      testState.updateValue("k1", 1.0)
+      assert(testState.getValue("k1") === 1.0)
+      // update existing value, append new key-value pairs
+      testState.updateValue("k1", 1.0)
+      testState.updateValue("k2", 2.0)
+      assert(testState.getValue("k1") === 1.0)
+      assert(testState.getValue("k2") === 2.0)
+      testState.updateValue("k1", 3.0)
+      assert(testState.getValue("k1") === 3.0)
+
+      assert(testState.keys().toSeq === Seq("k1", "k2"))
+      assert(testState.values().toSeq === Seq(3.0, 2.0))
+
+      // test remove
+      testState.removeKey("k1")
+      assert(testState.getValue("k1") === null)
+      assert(!testState.containsKey("k1"))
+
+      testState.clear()
+      assert(!testState.exists())
+      assert(testState.iterator().hasNext === false)
+    }
+  }
+
+  test("Map state operations for multiple map instances") {
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
+      val store = provider.getStore(0)
+      val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
+        Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]])
+
+      val testState1: MapState[Long, Double] =
+        handle.getMapState[Long, Double]("testState1", Encoders.scalaLong, 
Encoders.scalaDouble)
+      val testState2: MapState[Long, Int] =
+        handle.getMapState[Long, Int]("testState2", Encoders.scalaLong, 
Encoders.scalaInt)
+      ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+      // put initial value
+      testState1.updateValue(1L, 1.0)
+      assert(testState1.getValue(1L) === 1.0)
+      assert(!testState2.containsKey(1L))
+      // update existing values, append new key-value pairs
+      testState1.updateValue(1L, 2.0)
+      testState2.updateValue(2L, 3)
+      assert(testState1.getValue(1L) === 2.0)
+      assert(testState2.getValue(2L) === 3)
+
+      assert(testState1.keys().toSeq === Seq(1L))
+      assert(testState2.keys().toSeq === Seq(2L))
+      assert(testState1.values().toSeq === Seq(2.0))
+      assert(testState2.values().toSeq === Seq(3))
+
+      // test remove
+      testState1.removeKey(1L)
+      assert(testState1.getValue(1L) === null)
+      assert(!testState1.containsKey(1L))
+
+      testState2.clear()
+      assert(!testState1.exists())
+      assert(!testState2.exists())
+      assert(testState1.iterator().hasNext === false)
+      assert(testState2.iterator().hasNext === false)
+    }
+  }
+
+  test("Map state operations with list, value, another map instances") {
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
+      val store = provider.getStore(0)
+      val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
+        Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]])
+
+      val mapTestState1: MapState[String, Int] =
+        handle.getMapState[String, Int]("mapTestState1", Encoders.STRING, 
Encoders.scalaInt)
+      val mapTestState2: MapState[String, Int] =
+        handle.getMapState[String, Int]("mapTestState2", Encoders.STRING, 
Encoders.scalaInt)
+      val valueTestState: ValueState[String] =
+        handle.getValueState[String]("valueTestState", Encoders.STRING)
+      val listTestState: ListState[String] =
+        handle.getListState[String]("listTestState", Encoders.STRING)
+
+      ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+      // put initial values
+      valueTestState.update("v1")
+      listTestState.put(Seq("v1").toArray)
+      mapTestState1.updateValue("k1", 1)
+      mapTestState2.updateValue("k2", 2)
+      assert(valueTestState.get() === "v1")
+      assert(listTestState.get().toSeq === Seq("v1"))
+      assert(mapTestState1.getValue("k1") === 1)
+      assert(mapTestState2.getValue("k2") === 2)
+      // update existing values, append
+      valueTestState.update("v2")
+      listTestState.appendValue("v3")
+      mapTestState1.updateValue("k1", 3)
+      mapTestState2.updateValue("k2", 4)
+
+      assert(valueTestState.get() === "v2")
+      assert(listTestState.get().toSeq === Seq("v1", "v3"))
+      assert(mapTestState1.getValue("k1") === 3)
+      assert(mapTestState2.getValue("k2") === 4)
+
+      // advanced append/get operations
+      listTestState.appendList(Seq("v4").toArray)
+      mapTestState1.updateValue("k3", 5)
+      mapTestState2.updateValue("k4", 6)
+
+      assert(valueTestState.get() === "v2")
+      assert(listTestState.get().toSeq === Seq("v1", "v3", "v4"))
+      assert(mapTestState1.keys().toSeq === Seq("k1", "k3"))
+      assert(mapTestState2.values().toSeq === Seq(4, 6))
+
+      // test remove
+      valueTestState.clear()
+      listTestState.clear()
+      mapTestState1.clear()
+      mapTestState2.removeKey("k4")
+
+      assert(!valueTestState.exists())
+      assert(!listTestState.exists())
+      assert(!mapTestState1.exists())
+      assert(mapTestState2.exists())
+      assert(mapTestState2.iterator().toList === List(("k2", 4)))
+    }
+  }
+}
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala
index 40e31239895c..0bf14037a2ea 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala
@@ -40,57 +40,12 @@ case class TestClass(var id: Long, var name: String)
  * Class that adds tests for single value ValueState types used in arbitrary 
stateful
  * operators such as transformWithState
  */
-class ValueStateSuite extends SharedSparkSession
-  with BeforeAndAfter {
-
-  before {
-    StateStore.stop()
-    require(!StateStore.isMaintenanceRunning)
-  }
-
-  after {
-    StateStore.stop()
-    require(!StateStore.isMaintenanceRunning)
-  }
+class ValueStateSuite extends StateVariableSuiteBase {
 
   import StateStoreTestsHelper._
 
-  val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
-
-  val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
-
-  private def newStoreProviderWithValueState(useColumnFamilies: Boolean):
-    RocksDBStateStoreProvider = {
-    newStoreProviderWithValueState(StateStoreId(newDir(), Random.nextInt(), 0),
-      numColsPrefixKey = 0,
-      useColumnFamilies = useColumnFamilies)
-  }
-
-  private def newStoreProviderWithValueState(
-      storeId: StateStoreId,
-      numColsPrefixKey: Int,
-      sqlConf: SQLConf = SQLConf.get,
-      conf: Configuration = new Configuration,
-      useColumnFamilies: Boolean = false): RocksDBStateStoreProvider = {
-    val provider = new RocksDBStateStoreProvider()
-    provider.init(
-      storeId, schemaForKeyRow, schemaForValueRow, numColsPrefixKey = 
numColsPrefixKey,
-      useColumnFamilies,
-      new StateStoreConf(sqlConf), conf)
-    provider
-  }
-
-  private def tryWithProviderResource[T](
-      provider: StateStoreProvider)(f: StateStoreProvider => T): T = {
-    try {
-      f(provider)
-    } finally {
-      provider.close()
-    }
-  }
-
   test("Implicit key operations") {
-    tryWithProviderResource(newStoreProviderWithValueState(true)) { provider =>
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
       val store = provider.getStore(0)
       val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]])
@@ -134,7 +89,7 @@ class ValueStateSuite extends SharedSparkSession
   }
 
   test("Value state operations for single instance") {
-    tryWithProviderResource(newStoreProviderWithValueState(true)) { provider =>
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
       val store = provider.getStore(0)
       val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]])
@@ -160,7 +115,7 @@ class ValueStateSuite extends SharedSparkSession
   }
 
   test("Value state operations for multiple instances") {
-    tryWithProviderResource(newStoreProviderWithValueState(true)) { provider =>
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
       val store = provider.getStore(0)
       val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]])
@@ -224,7 +179,7 @@ class ValueStateSuite extends SharedSparkSession
   }
 
   test("test SQL encoder - Value state operations for Primitive(Double) 
instances") {
-    tryWithProviderResource(newStoreProviderWithValueState(true)) { provider =>
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
       val store = provider.getStore(0)
       val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]])
@@ -250,7 +205,7 @@ class ValueStateSuite extends SharedSparkSession
   }
 
   test("test SQL encoder - Value state operations for Primitive(Long) 
instances") {
-    tryWithProviderResource(newStoreProviderWithValueState(true)) { provider =>
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
       val store = provider.getStore(0)
       val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]])
@@ -276,7 +231,7 @@ class ValueStateSuite extends SharedSparkSession
   }
 
   test("test SQL encoder - Value state operations for case class instances") {
-    tryWithProviderResource(newStoreProviderWithValueState(true)) { provider =>
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
       val store = provider.getStore(0)
       val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]])
@@ -302,7 +257,7 @@ class ValueStateSuite extends SharedSparkSession
   }
 
   test("test SQL encoder - Value state operations for POJO instances") {
-    tryWithProviderResource(newStoreProviderWithValueState(true)) { provider =>
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
       val store = provider.getStore(0)
       val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]])
@@ -327,3 +282,59 @@ class ValueStateSuite extends SharedSparkSession
     }
   }
 }
+
+/**
+ * Abstract Base Class that provides test utilities for different state 
variable
+ * types (ValueState, ListState, MapState) used in arbitrary stateful 
operators.
+ */
+abstract class StateVariableSuiteBase extends SharedSparkSession
+  with BeforeAndAfter {
+
+  before {
+    StateStore.stop()
+    require(!StateStore.isMaintenanceRunning)
+  }
+
+  after {
+    ImplicitGroupingKeyTracker.removeImplicitKey()
+    require(ImplicitGroupingKeyTracker.getImplicitKeyOption.isEmpty)
+    StateStore.stop()
+    require(!StateStore.isMaintenanceRunning)
+  }
+
+  import StateStoreTestsHelper._
+
+  protected var schemaForKeyRow: StructType = new StructType().add("key", 
BinaryType)
+  protected var schemaForValueRow: StructType = new StructType().add("value", 
BinaryType)
+
+  protected def newStoreProviderWithStateVariable(
+      useColumnFamilies: Boolean): RocksDBStateStoreProvider = {
+    newStoreProviderWithStateVariable(StateStoreId(newDir(), Random.nextInt(), 
0),
+      numColsPrefixKey = 0,
+      useColumnFamilies = useColumnFamilies)
+  }
+
+  protected def newStoreProviderWithStateVariable(
+      storeId: StateStoreId,
+      numColsPrefixKey: Int,
+      sqlConf: SQLConf = SQLConf.get,
+      conf: Configuration = new Configuration,
+      useColumnFamilies: Boolean = false): RocksDBStateStoreProvider = {
+    val provider = new RocksDBStateStoreProvider()
+    provider.init(
+      storeId, schemaForKeyRow, schemaForValueRow, numColsPrefixKey = 
numColsPrefixKey,
+      useColumnFamilies,
+      new StateStoreConf(sqlConf), conf)
+    provider
+  }
+
+  protected def tryWithProviderResource[T](
+      provider: StateStoreProvider)(f: StateStoreProvider => T): T = {
+    try {
+      f(provider)
+    } finally {
+      provider.close()
+    }
+  }
+}
+
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala
new file mode 100644
index 000000000000..a0576497f399
--- /dev/null
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateSuite.scala
@@ -0,0 +1,226 @@
+/*
+ * 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.SparkIllegalArgumentException
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.internal.SQLConf
+
+case class InputMapRow(key: String, action: String, value: (String, String))
+
+class TestMapStateProcessor
+  extends StatefulProcessor[String, InputMapRow, (String, String, String)] {
+
+  @transient var _mapState: MapState[String, String] = _
+
+  override def init(outputMode: OutputMode): Unit = {
+    _mapState = getHandle.getMapState("sessionState", Encoders.STRING, 
Encoders.STRING)
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputMapRow],
+      timerValues: TimerValues): Iterator[(String, String, String)] = {
+
+    var output = List[(String, String, String)]()
+
+    for (row <- inputRows) {
+      if (row.action == "exists") {
+        output = (key, "exists", _mapState.exists().toString) :: output
+      } else if (row.action == "getValue") {
+        output = (key, row.value._1, _mapState.getValue(row.value._1)) :: 
output
+      } else if (row.action == "containsKey") {
+        output = (key, row.value._1,
+          if (_mapState.containsKey(row.value._1)) "true" else "false") :: 
output
+      } else if (row.action == "updateValue") {
+        _mapState.updateValue(row.value._1, row.value._2)
+      } else if (row.action == "iterator") {
+        _mapState.iterator().foreach { pair =>
+          output = (key, pair._1, pair._2) :: output
+        }
+      } else if (row.action == "keys") {
+        _mapState.keys().foreach { key =>
+          output = (row.key, key, row.value._2) :: output
+        }
+      } else if (row.action == "values") {
+        _mapState.values().foreach { value =>
+          output = (row.key, row.value._1, value) :: output
+        }
+      } else if (row.action == "removeKey") {
+        _mapState.removeKey(row.value._1)
+      } else if (row.action == "clear") {
+        _mapState.clear()
+      }
+    }
+    output.iterator
+  }
+
+  override def close(): Unit = {}
+}
+
+/**
+ * Class that adds integration tests for MapState types used in arbitrary 
stateful
+ * operators such as transformWithState.
+ */
+class TransformWithMapStateSuite extends StreamTest {
+  import testImplicits._
+
+  private def testMapStateWithNullUserKey(inputMapRow: InputMapRow): Unit = {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+
+      val inputData = MemoryStream[InputMapRow]
+      val result = inputData.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(new TestMapStateProcessor(),
+          TimeoutMode.NoTimeouts(),
+          OutputMode.Update())
+
+
+      testStream(result, OutputMode.Update())(
+        AddData(inputData, inputMapRow),
+        ExpectFailure[SparkIllegalArgumentException] { e => {
+          checkError(
+            exception = e.asInstanceOf[SparkIllegalArgumentException],
+            errorClass = "ILLEGAL_STATE_STORE_VALUE.NULL_VALUE",
+            sqlState = Some("42601"),
+            parameters = Map("stateName" -> "sessionState")
+          )
+        }}
+      )
+    }
+  }
+
+  test("Test retrieving value with non-existing user key") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+
+      val inputData = MemoryStream[InputMapRow]
+      val result = inputData.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(new TestMapStateProcessor(),
+          TimeoutMode.NoTimeouts(),
+          OutputMode.Update())
+
+      testStream(result, OutputMode.Update())(
+        AddData(inputData, InputMapRow("k1", "getValue", ("v1", ""))),
+        CheckAnswer(("k1", "v1", null))
+      )
+    }
+  }
+
+  Seq("getValue", "containsKey", "updateValue", "removeKey").foreach { 
mapImplFunc =>
+    test(s"Test $mapImplFunc with null user key") {
+      testMapStateWithNullUserKey(InputMapRow("k1", mapImplFunc, (null, "")))
+    }
+  }
+
+  test("Test put value with null value") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+
+      val inputData = MemoryStream[InputMapRow]
+      val result = inputData.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(new TestMapStateProcessor(),
+          TimeoutMode.NoTimeouts(),
+          OutputMode.Update())
+
+      testStream(result, OutputMode.Update())(
+        AddData(inputData, InputMapRow("k1", "updateValue", ("k1", null))),
+        ExpectFailure[SparkIllegalArgumentException] { e => {
+          checkError(
+            exception = e.asInstanceOf[SparkIllegalArgumentException],
+            errorClass = "ILLEGAL_STATE_STORE_VALUE.NULL_VALUE",
+            sqlState = Some("42601"),
+            parameters = Map("stateName" -> "sessionState"))
+        }}
+      )
+    }
+  }
+
+  test("Test map state correctness") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputData = MemoryStream[InputMapRow]
+      val result = inputData.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(new TestMapStateProcessor(),
+          TimeoutMode.NoTimeouts(),
+          OutputMode.Append())
+      testStream(result, OutputMode.Append())(
+        // Test exists()
+        AddData(inputData, InputMapRow("k1", "updateValue", ("v1", "10"))),
+        AddData(inputData, InputMapRow("k1", "exists", ("", ""))),
+        AddData(inputData, InputMapRow("k2", "exists", ("", ""))),
+        CheckNewAnswer(("k1", "exists", "true"), ("k2", "exists", "false")),
+
+        // Test get and put with composite key
+        AddData(inputData, InputMapRow("k1", "updateValue", ("v2", "5"))),
+
+        AddData(inputData, InputMapRow("k2", "updateValue", ("v2", "3"))),
+        AddData(inputData, InputMapRow("k2", "updateValue", ("v2", "12"))),
+        AddData(inputData, InputMapRow("k2", "updateValue", ("v4", "1"))),
+
+        // Different grouping key, same user key
+        AddData(inputData, InputMapRow("k1", "getValue", ("v2", ""))),
+        CheckNewAnswer(("k1", "v2", "5")),
+        // Same grouping key, same user key, update value should reflect
+        AddData(inputData, InputMapRow("k2", "getValue", ("v2", ""))),
+        CheckNewAnswer(("k2", "v2", "12")),
+
+        // Test get full map for a given grouping key - prefixScan
+        AddData(inputData, InputMapRow("k2", "iterator", ("", ""))),
+        CheckNewAnswer(("k2", "v2", "12"), ("k2", "v4", "1")),
+
+        AddData(inputData, InputMapRow("k2", "keys", ("", ""))),
+        CheckNewAnswer(("k2", "v2", ""), ("k2", "v4", "")),
+
+        AddData(inputData, InputMapRow("k2", "values", ("", ""))),
+        CheckNewAnswer(("k2", "", "12"), ("k2", "", "1")),
+
+        // Test remove functionalities
+        AddData(inputData, InputMapRow("k1", "removeKey", ("v2", ""))),
+        AddData(inputData, InputMapRow("k1", "containsKey", ("v2", ""))),
+        CheckNewAnswer(("k1", "v2", "false")),
+
+        AddData(inputData, InputMapRow("k2", "clear", ("", ""))),
+        AddData(inputData, InputMapRow("k2", "iterator", ("", ""))),
+        CheckNewAnswer(),
+        AddData(inputData, InputMapRow("k2", "exists", ("", ""))),
+        CheckNewAnswer(("k2", "exists", "false"))
+      )
+    }
+  }
+
+  test("transformWithMapState - batch should succeed") {
+    val inputData = Seq(
+      InputMapRow("k1", "updateValue", ("v1", "10")),
+      InputMapRow("k1", "getValue", ("v1", "")))
+    val result = inputData.toDS()
+      .groupByKey(x => x.key)
+      .transformWithState(new TestMapStateProcessor(),
+        TimeoutMode.NoTimeouts(),
+        OutputMode.Append())
+
+    val df = result.toDF()
+    checkAnswer(df, Seq(("k1", "v1", "10")).toDF())
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org


Reply via email to