jingz-db commented on code in PR #45341:
URL: https://github.com/apache/spark/pull/45341#discussion_r1520613982


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImpl.scala:
##########
@@ -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),

Review Comment:
   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.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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

Reply via email to