anishshri-db commented on code in PR #45932:
URL: https://github.com/apache/spark/pull/45932#discussion_r1563050397


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala:
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.sql.Encoder
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import 
org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchema.{KEY_ROW_SCHEMA,
 VALUE_ROW_SCHEMA_WITH_TTL}
+import 
org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, 
StateStore, StateStoreErrors}
+import org.apache.spark.sql.streaming.{ListState, TTLConfig}
+import org.apache.spark.util.NextIterator
+
+/**
+ * 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
+ * @param keyEnc - Spark SQL encoder for key
+ * @param valEncoder - Spark SQL encoder for value

Review Comment:
   Can we update the comment to match the args ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -75,7 +75,7 @@ trait TTLState {
    *
    * @return true if the state was cleared, false otherwise.

Review Comment:
   Could we update this too ?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala:
##########
@@ -237,13 +237,33 @@ class StatefulProcessorHandleSuite extends 
StateVariableSuiteBase {
     }
   }
 
+  test(s"ttl States are populated for listState and timeMode=ProcessingTime") {
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
+      val store = provider.getStore(0)
+      val handle = new StatefulProcessorHandleImpl(store,
+        UUID.randomUUID(), keyExprEncoder, TimeMode.ProcessingTime(),
+        batchTimestampMs = Some(10))
+
+      val listStateWithTTL = handle.getListState("testState",
+        Encoders.STRING, TTLConfig(Duration.ofHours(1)))
+
+      // create another state without TTL, this should not be captured in the 
handle
+      handle.getListState("testState", Encoders.STRING)
+
+      assert(handle.ttlStates.size() === 1)
+      assert(handle.ttlStates.get(0) === listStateWithTTL)
+    }
+  }
+
+

Review Comment:
   nit: extra newline ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala:
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.sql.Encoder
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import 
org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchema.{KEY_ROW_SCHEMA,
 VALUE_ROW_SCHEMA_WITH_TTL}
+import 
org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, 
StateStore, StateStoreErrors}
+import org.apache.spark.sql.streaming.{ListState, TTLConfig}
+import org.apache.spark.util.NextIterator
+
+/**
+ * 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
+ * @param keyEnc - Spark SQL encoder for key
+ * @param valEncoder - Spark SQL encoder for value
+ * @tparam S - data type of object that will be stored in the list
+ */
+class ListStateImplWithTTL[S](
+  store: StateStore,
+  stateName: String,
+  keyExprEnc: ExpressionEncoder[Any],
+  valEncoder: Encoder[S],
+  ttlConfig: TTLConfig,
+  batchTimestampMs: Long)
+  extends SingleKeyTTLStateImpl(stateName, store, batchTimestampMs) with 
ListState[S] {
+
+  private lazy val keySerializer = keyExprEnc.createSerializer()
+
+  private lazy val stateTypesEncoder = StateTypesEncoder(
+    keySerializer, valEncoder, stateName, hasTtl = true)
+
+  private lazy val ttlExpirationMs =
+    StateTTL.calculateExpirationTimeForDuration(ttlConfig.ttlDuration, 
batchTimestampMs)
+
+  initialize()
+
+  private def initialize(): Unit = {
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, 
VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA), useMultipleValuesPerKey = 
true)
+  }
+
+  /** Whether state exists or not. */
+  override def exists(): Boolean = {
+    get().nonEmpty
+  }
+
+  /**
+   * Get the state value if it exists. If the state does not exist in state 
store, an
+   * empty iterator is returned.
+   */
+  override def get(): Iterator[S] = {
+    val encodedKey = stateTypesEncoder.encodeGroupingKey()
+    val unsafeRowValuesIterator = store.valuesIterator(encodedKey, stateName)
+
+    new NextIterator[S] {
+
+      override protected def getNext(): S = {
+        val iter = unsafeRowValuesIterator.dropWhile { row =>
+          stateTypesEncoder.isExpired(row, batchTimestampMs)
+        }
+
+        if (iter.hasNext) {
+          val currentRow = iter.next()
+          stateTypesEncoder.decodeValue(currentRow)
+        } else {
+          finished = true
+          null.asInstanceOf[S]
+        }
+      }
+
+      override protected def close(): Unit = {}
+    }
+  }
+
+  /** Update the value of the list. */
+  override def put(newState: Array[S]): Unit = {
+    validateNewState(newState)
+
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    val encodedKey = 
stateTypesEncoder.encodeSerializedGroupingKey(serializedGroupingKey)
+    var isFirst = true
+
+    newState.foreach { v =>
+      val encodedValue = stateTypesEncoder.encodeValue(v, ttlExpirationMs)
+      if (isFirst) {
+        store.put(encodedKey, encodedValue, stateName)
+        isFirst = false
+      } else {
+        store.merge(encodedKey, encodedValue, stateName)
+      }
+    }
+    upsertTTLForStateKey(serializedGroupingKey)
+  }
+
+  /** Append an entry to the list. */
+  override def appendValue(newState: S): Unit = {
+    StateStoreErrors.requireNonNullStateValue(newState, stateName)
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    
store.merge(stateTypesEncoder.encodeSerializedGroupingKey(serializedGroupingKey),
+      stateTypesEncoder.encodeValue(newState, ttlExpirationMs), stateName)
+    upsertTTLForStateKey(serializedGroupingKey)
+  }
+
+  /** Append an entire list to the existing value. */
+  override def appendList(newState: Array[S]): Unit = {
+    validateNewState(newState)
+
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    val encodedKey = 
stateTypesEncoder.encodeSerializedGroupingKey(serializedGroupingKey)
+    newState.foreach { v =>
+      val encodedValue = stateTypesEncoder.encodeValue(v, ttlExpirationMs)
+      store.merge(encodedKey, encodedValue, stateName)
+    }
+    upsertTTLForStateKey(serializedGroupingKey)
+  }
+
+  /** Remove this state. */
+  override def clear(): Unit = {
+    store.remove(stateTypesEncoder.encodeGroupingKey(), stateName)
+  }
+
+  private def validateNewState(newState: Array[S]): Unit = {
+    StateStoreErrors.requireNonNullStateValue(newState, stateName)
+    StateStoreErrors.requireNonEmptyListStateValue(newState, stateName)
+
+    newState.foreach { v =>
+      StateStoreErrors.requireNonNullStateValue(v, stateName)
+    }
+  }
+
+  /**
+   *

Review Comment:
   nit: extra empty line ?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala:
##########
@@ -237,13 +237,33 @@ class StatefulProcessorHandleSuite extends 
StateVariableSuiteBase {
     }
   }
 
+  test(s"ttl States are populated for listState and timeMode=ProcessingTime") {
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
+      val store = provider.getStore(0)
+      val handle = new StatefulProcessorHandleImpl(store,
+        UUID.randomUUID(), keyExprEncoder, TimeMode.ProcessingTime(),
+        batchTimestampMs = Some(10))
+
+      val listStateWithTTL = handle.getListState("testState",
+        Encoders.STRING, TTLConfig(Duration.ofHours(1)))
+
+      // create another state without TTL, this should not be captured in the 
handle
+      handle.getListState("testState", Encoders.STRING)
+
+      assert(handle.ttlStates.size() === 1)
+      assert(handle.ttlStates.get(0) === listStateWithTTL)
+    }
+  }
+
+
   test(s"ttl States are not populated for timeMode=None") {
     tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
       val store = provider.getStore(0)
       val handle = new StatefulProcessorHandleImpl(store,
         UUID.randomUUID(), keyExprEncoder, TimeMode.None())
 
       handle.getValueState("testState", Encoders.STRING)
+      handle.getListState("testState", Encoders.STRING)

Review Comment:
   Could we use a different name for the state variable ?



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