HeartSaVioR commented on code in PR #44961:
URL: https://github.com/apache/spark/pull/44961#discussion_r1490075730


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala:
##########
@@ -75,6 +76,61 @@ private[sql] class RocksDBStateStoreProvider
       value
     }
 
+    /**
+     * Provides an iterator containing all values of a non-null key.
+     *
+     * Inside RocksDB, the values are merged together and stored as a byte 
Array.
+     * This operation relies on state store value encoder to be able to split 
the
+     * single array into multiple values.
+     *
+     * Also see [[MultiValuedStateEncoder]] which supports encoding/decoding 
multiple
+     * values per key.
+     */
+    override def valuesIterator(key: UnsafeRow, colFamilyName: String): 
Iterator[UnsafeRow] = {
+      verify(key != null, "Key cannot be null")
+
+      val kvEncoder = keyValueEncoderMap.get(colFamilyName)
+      val valueEncoder = kvEncoder._2
+      val keyEncoder = kvEncoder._1
+
+      verify(valueEncoder.supportsMultipleValuesPerKey, "valuesIterator 
requires a encoder " +
+      "that supports multiple values for a single key.")
+      val encodedKey = rocksDB.get(keyEncoder.encodeKey(key), colFamilyName)
+      val valueIterator = valueEncoder.decodeValues(encodedKey)
+
+      if (valueIterator.nonEmpty) {
+        new Iterator[UnsafeRow] {
+          override def hasNext: Boolean = {
+            valueIterator.hasNext
+          }
+
+          override def next(): UnsafeRow = {
+            val value = valueIterator.next()
+            if (value != null) {
+              StateStoreProvider.validateStateRowFormat(

Review Comment:
   This triggers overhead so we wouldn't like to do this for every value. 
Either we have to sample (that's why we only do for the first value) or not 
doing this at all. It's probably OK to do latter as get() also doesn't do this 
for the case where there are multiple CFs.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -77,9 +50,7 @@ class ValueStateImpl[S](
   override def getOption(): Option[S] = {
     val retRow = getImpl()

Review Comment:
   I know this is beyond the scope of PR, but while we are here, I don't see 
any difference with `Option(get())`.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala:
##########
@@ -131,6 +141,15 @@ trait StateStore extends ReadStateStore {
   def remove(key: UnsafeRow,
     colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit
 
+  /**
+   * Merges the provided value with existing values of a non-null key.

Review Comment:
   Maybe we want to explicitly document the contract on the behavior when 
performing merge against non-existing key, since all implementations should 
provide the same.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala:
##########
@@ -67,6 +67,15 @@ trait ReadStateStore {
   def get(key: UnsafeRow,
     colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): UnsafeRow
 
+  /**
+   * Provides an iterator containing all values of a non-null key.

Review Comment:
   Let's make a contract about expected return value, especially about 
clarification of value for non-existing key. Multiple implementations should 
follow the contract so this needs to be clearly explained.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImpl.scala:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.catalyst.encoders.ExpressionEncoder
+import 
org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema.{KEY_ROW_SCHEMA,
 VALUE_ROW_SCHEMA}
+import org.apache.spark.sql.execution.streaming.state.{StateStore, 
StateStoreErrors}
+import org.apache.spark.sql.streaming.ListState
+
+/**
+ * Provides concrete implementation for list of values associated with a state 
variable
+ * used in the streaming transformWithState operator.
+ *
+ * @param store - reference to the StateStore instance to be used for storing 
state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored in the list
+ */
+class ListStateImpl[S](
+     store: StateStore,
+     stateName: String,
+     keyExprEnc: ExpressionEncoder[Any])
+  extends ListState[S] with Logging {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+
+  store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, numColsPrefixKey = 
0,

Review Comment:
   Thinking out loud - it'd be ideal if we can move out the context of schema 
for state store key/value entirely to StateTypesEncoderUtils. State impls don't 
need to know about anything except that StateTypesEncoderUtils will give the 
UnsafeRow, and all context about serde will be put to StateTypesEncoderUtils. 
Just a 2 cents, not mandatory to address.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImpl.scala:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.catalyst.encoders.ExpressionEncoder
+import 
org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema.{KEY_ROW_SCHEMA,
 VALUE_ROW_SCHEMA}
+import org.apache.spark.sql.execution.streaming.state.{StateStore, 
StateStoreErrors}
+import org.apache.spark.sql.streaming.ListState
+
+/**
+ * Provides concrete implementation for list of values associated with a state 
variable
+ * used in the streaming transformWithState operator.
+ *
+ * @param store - reference to the StateStore instance to be used for storing 
state
+ * @param stateName - name of logical state partition
+ * @tparam S - data type of object that will be stored in the list
+ */
+class ListStateImpl[S](
+     store: StateStore,
+     stateName: String,
+     keyExprEnc: ExpressionEncoder[Any])
+  extends ListState[S] with Logging {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+
+  store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, numColsPrefixKey = 
0,

Review Comment:
   If we anticipate further changes to StateTypesEncoderUtils it's also OK to 
leave it as it is and make such change in later phase.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to