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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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
+
+/**
+ * Class that provides a concrete implementation for a list state state 
associated with state
+ * variables (with ttl expiration support) 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 keyExprEnc - Spark SQL encoder for key
+ * @param valEncoder - Spark SQL encoder for value
+ * @param ttlConfig  - TTL configuration for values  stored in this state
+ * @param batchTimestampMs - current batch processing timestamp.
+ * @tparam S - data type of object that will be stored
+ */
+class ListStateImplWithTTL[S](
+  store: StateStore,

Review Comment:
   nit: 2 more spaces (4 spaces)



##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala:
##########
@@ -72,6 +72,26 @@ private[sql] trait StatefulProcessorHandle extends 
Serializable {
    */
   def getListState[T](stateName: String, valEncoder: Encoder[T]): ListState[T]
 
+  /**
+   * Function to create new or return existing list state variable of given 
type
+   * with ttl. State values will not be returned past ttlDuration, and will be 
eventually removed
+   * from the state store. Any values in listState which have expired after 
ttlDuration will not
+   * be returned on get() and will be eventually removed from the state.
+   *
+   * The user must ensure to call this function only within the `init()` 
method of the
+   * StatefulProcessor.
+   *
+   * @param stateName  - name of the state variable
+   * @param valEncoder - SQL encoder for state variable
+   * @param ttlConfig  - the ttl configuration (time to live duration etc.)
+   * @tparam T - type of state variable
+   * @return - instance of ListState of type T that can be used to store state 
persistently
+   */
+  def getListState[T](
+     stateName: String,

Review Comment:
   nit: 1 more space



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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
+
+/**
+ * Class that provides a concrete implementation for a list state state 
associated with state
+ * variables (with ttl expiration support) 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 keyExprEnc - Spark SQL encoder for key
+ * @param valEncoder - Spark SQL encoder for value
+ * @param ttlConfig  - TTL configuration for values  stored in this state
+ * @param batchTimestampMs - current batch processing timestamp.
+ * @tparam S - data type of object that will be stored
+ */
+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 =>

Review Comment:
   Just to be 100% sure, we don't have a case where elements are stored in the 
direction which is out of sync with its appearance order. Do I understand 
correctly?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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
+
+/**
+ * Class that provides a concrete implementation for a list state state 
associated with state
+ * variables (with ttl expiration support) 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 keyExprEnc - Spark SQL encoder for key
+ * @param valEncoder - Spark SQL encoder for value
+ * @param ttlConfig  - TTL configuration for values  stored in this state
+ * @param batchTimestampMs - current batch processing timestamp.
+ * @tparam S - data type of object that will be stored
+ */
+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)

Review Comment:
   This sounds to me we support both element level TTL and TTL of the list 
itself. Do I understand correctly? If then looks OK.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -245,6 +245,38 @@ class StatefulProcessorHandleImpl(
     resultState
   }
 
+  /**
+   * Function to create new or return existing list state variable of given 
type
+   * with ttl. State values will not be returned past ttlDuration, and will be 
eventually removed
+   * from the state store. Any values in listState which have expired after 
ttlDuration will not
+   * returned on get() and will be eventually removed from the state.
+   *
+   * The user must ensure to call this function only within the `init()` 
method of the
+   * StatefulProcessor.
+   *
+   * @param stateName  - name of the state variable
+   * @param valEncoder - SQL encoder for state variable
+   * @param ttlConfig  - the ttl configuration (time to live duration etc.)
+   * @tparam T - type of state variable
+   * @return - instance of ListState of type T that can be used to store state 
persistently
+   */
+  override def getListState[T](
+    stateName: String,

Review Comment:
   nit: 2 more spaces (4 spaces)



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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
+
+/**
+ * Class that provides a concrete implementation for a list state state 
associated with state
+ * variables (with ttl expiration support) 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 keyExprEnc - Spark SQL encoder for key
+ * @param valEncoder - Spark SQL encoder for value
+ * @param ttlConfig  - TTL configuration for values  stored in this state
+ * @param batchTimestampMs - current batch processing timestamp.
+ * @tparam S - data type of object that will be stored
+ */
+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)
+    }
+  }
+
+  /**
+   * Loops through all the values associated with the grouping key, and removes
+   * the expired elements from the list.
+   * @param groupingKey grouping key for which cleanup should be performed.
+   */
+  override def clearIfExpired(groupingKey: Array[Byte]): Long = {
+    var numValuesExpired = 0L
+    val encodedGroupingKey = 
stateTypesEncoder.encodeSerializedGroupingKey(groupingKey)
+    val unsafeRowValuesIterator = store.valuesIterator(encodedGroupingKey, 
stateName)
+    // We clear the list, and use the iterator to put back all of the 
non-expired values
+    store.remove(encodedGroupingKey, stateName)
+    var isFirst = true
+    unsafeRowValuesIterator.foreach { encodedValue =>
+      if (!stateTypesEncoder.isExpired(encodedValue, batchTimestampMs)) {
+        if (isFirst) {
+          store.put(encodedGroupingKey, encodedValue, stateName)
+          isFirst = false
+        } else {
+          store.merge(encodedGroupingKey, encodedValue, stateName)
+        }
+      } else {
+        numValuesExpired += 1
+      }
+    }
+    numValuesExpired
+  }
+
+  private def upsertTTLForStateKey(serializedGroupingKey: Array[Byte]): Unit = 
{
+    upsertTTLForStateKey(ttlExpirationMs, serializedGroupingKey)
+  }
+
+  /*
+    * Internal methods to probe state for testing. The below methods exist for 
unit tests
+    * to read the state ttl values, and ensure that values are persisted 
correctly in
+    * the underlying state store.
+    */
+
+  /**
+   * Retrieves the value from State even if its expired. This method is used
+   * in tests to read the state store value, and ensure if its cleaned up at 
the
+   * end of the micro-batch.
+   */
+  private[sql] def getWithoutEnforcingTTL(): Iterator[S] = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val unsafeRowValuesIterator = store.valuesIterator(encodedGroupingKey, 
stateName)
+    unsafeRowValuesIterator.map{
+      valueUnsafeRow =>
+        stateTypesEncoder.decodeValue(valueUnsafeRow)
+    }
+  }
+
+  /**
+   * Read the ttl value associated with the grouping key.
+   */
+  private[sql] def getTTLValues(): Iterator[(S, Long)] = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val unsafeRowValuesIterator = store.valuesIterator(encodedGroupingKey, 
stateName)
+    unsafeRowValuesIterator.map{

Review Comment:
   nit: same style



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala:
##########
@@ -237,13 +237,32 @@ class StatefulProcessorHandleSuite extends 
StateVariableSuiteBase {
     }
   }
 
+  test(s"ttl States are populated for listState and timeMode=ProcessingTime") {

Review Comment:
   nit: same



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{ListStateImplWithTTL, 
MemoryStream}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.util.StreamManualClock
+
+class ListStateTTLProcessor(ttlConfig: TTLConfig)
+  extends StatefulProcessor[String, InputEvent, OutputEvent] {
+
+  @transient private var _listState: ListStateImplWithTTL[Int] = _
+
+  override def init(
+   outputMode: OutputMode,
+   timeMode: TimeMode): Unit = {
+    _listState = getHandle
+      .getListState("listState", Encoders.scalaInt, ttlConfig)
+      .asInstanceOf[ListStateImplWithTTL[Int]]
+  }
+
+  override def handleInputRows(
+    key: String,
+    inputRows: Iterator[InputEvent],
+    timerValues: TimerValues,
+    expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    inputRows.foreach { row =>
+      val resultIter = processRow(row, _listState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+
+  def processRow(
+    row: InputEvent,
+    listState: ListStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = listState.get()
+      currState.foreach { v =>
+        results = OutputEvent(key, v, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = listState.getWithoutEnforcingTTL()
+      currState.foreach { v =>
+        results = OutputEvent(key, v, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlValues = listState.getTTLValues()
+      ttlValues.foreach { ttlValue =>
+        results = OutputEvent(key, ttlValue._1, isTTLValue = true, 
ttlValue._2) :: results
+      }
+    } else if (row.action == "put") {
+      listState.put(Array(row.value))
+    } else if (row.action == "append") {
+      listState.appendValue(row.value)
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = listState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: 
results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+/**
+ * Test suite for testing list state with TTL.
+ * We use the base TTL suite with a list state processor.
+ */
+class TransformWithListStateTTLSuite extends TransformWithStateTTLTest {
+
+  import testImplicits._
+
+  override def getProcessor(ttlConfig: TTLConfig):
+    StatefulProcessor[String, InputEvent, OutputEvent] = {
+      new ListStateTTLProcessor(ttlConfig)
+  }
+
+  override def getStateTTLMetricName: String = "numListStateWithTTLVars"
+
+  test("verify iterator works with expired values in middle of list") {

Review Comment:
   The test is hard to understand as we change TTL config back and forth and 
clock is reused. Could you please add more informative comment about expected 
batch timestamp for the batch?



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{ListStateImplWithTTL, 
MemoryStream}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.util.StreamManualClock
+
+class ListStateTTLProcessor(ttlConfig: TTLConfig)
+  extends StatefulProcessor[String, InputEvent, OutputEvent] {
+
+  @transient private var _listState: ListStateImplWithTTL[Int] = _
+
+  override def init(
+   outputMode: OutputMode,
+   timeMode: TimeMode): Unit = {
+    _listState = getHandle
+      .getListState("listState", Encoders.scalaInt, ttlConfig)
+      .asInstanceOf[ListStateImplWithTTL[Int]]
+  }
+
+  override def handleInputRows(
+    key: String,
+    inputRows: Iterator[InputEvent],
+    timerValues: TimerValues,
+    expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    inputRows.foreach { row =>
+      val resultIter = processRow(row, _listState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+
+  def processRow(
+    row: InputEvent,
+    listState: ListStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = listState.get()
+      currState.foreach { v =>
+        results = OutputEvent(key, v, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = listState.getWithoutEnforcingTTL()
+      currState.foreach { v =>
+        results = OutputEvent(key, v, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlValues = listState.getTTLValues()
+      ttlValues.foreach { ttlValue =>
+        results = OutputEvent(key, ttlValue._1, isTTLValue = true, 
ttlValue._2) :: results
+      }
+    } else if (row.action == "put") {
+      listState.put(Array(row.value))
+    } else if (row.action == "append") {
+      listState.appendValue(row.value)
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = listState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: 
results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+/**
+ * Test suite for testing list state with TTL.
+ * We use the base TTL suite with a list state processor.
+ */
+class TransformWithListStateTTLSuite extends TransformWithStateTTLTest {
+
+  import testImplicits._
+
+  override def getProcessor(ttlConfig: TTLConfig):
+    StatefulProcessor[String, InputEvent, OutputEvent] = {
+      new ListStateTTLProcessor(ttlConfig)
+  }
+
+  override def getStateTTLMetricName: String = "numListStateWithTTLVars"
+
+  test("verify iterator works with expired values in middle of list") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName,
+      SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
+      withTempDir { checkpointLocation =>
+        val ttlConfig1 = TTLConfig(ttlDuration = Duration.ofMinutes(3))
+        val inputStream = MemoryStream[InputEvent]
+        val result1 = inputStream.toDS()
+          .groupByKey(x => x.key)
+          .transformWithState(
+            getProcessor(ttlConfig1),
+            TimeMode.ProcessingTime(),
+            OutputMode.Append())
+
+        val clock = new StreamManualClock
+        // add 3 elements with a duration of a minute
+        testStream(result1)(
+          StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock,
+            checkpointLocation = checkpointLocation.getAbsolutePath),
+          AddData(inputStream, InputEvent("k1", "put", 1)),
+          AdvanceManualClock(1 * 1000),
+          AddData(inputStream, InputEvent("k1", "append", 2)),
+          AddData(inputStream, InputEvent("k1", "append", 3)),
+          // advance clock to trigger processing
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(),
+          // get ttl values
+          AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", 
-1, null)),
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(
+            OutputEvent("k1", 1, isTTLValue = true, 181000),
+            OutputEvent("k1", 2, isTTLValue = true, 182000),
+            OutputEvent("k1", 3, isTTLValue = true, 182000)
+          ),
+          AddData(inputStream, InputEvent("k1", "get", -1, null)),
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(
+            OutputEvent("k1", 1, isTTLValue = false, -1),
+            OutputEvent("k1", 2, isTTLValue = false, -1),
+            OutputEvent("k1", 3, isTTLValue = false, -1)
+          ),
+          StopStream
+        )
+
+        val ttlConfig2 = TTLConfig(ttlDuration = Duration.ofSeconds(15))
+        val result2 = inputStream.toDS()
+          .groupByKey(x => x.key)
+          .transformWithState(
+            getProcessor(ttlConfig2),
+            TimeMode.ProcessingTime(),
+            OutputMode.Append())
+        // add 3 elements with a duration of 15 seconds
+        testStream(result2)(
+          StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock,
+            checkpointLocation = checkpointLocation.getAbsolutePath),
+          AddData(inputStream, InputEvent("k1", "append", 4)),
+          AddData(inputStream, InputEvent("k1", "append", 5)),
+          AddData(inputStream, InputEvent("k1", "append", 6)),
+          // advance clock to trigger processing
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(),
+          // get all elements without enforcing ttl
+          AddData(inputStream, InputEvent("k1", "get_without_enforcing_ttl", 
-1, null)),
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(
+            OutputEvent("k1", 1, isTTLValue = false, -1),
+            OutputEvent("k1", 2, isTTLValue = false, -1),
+            OutputEvent("k1", 3, isTTLValue = false, -1),
+            OutputEvent("k1", 4, isTTLValue = false, -1),
+            OutputEvent("k1", 5, isTTLValue = false, -1),
+            OutputEvent("k1", 6, isTTLValue = false, -1)
+          ),
+          StopStream
+        )
+        // add 3 more elements with a duration of a minute

Review Comment:
   Wait, if we allow the change of TTL during query restart, it is possible 
that earlier elements in the list has later expiration time. 
   
   How this will work with `ListStateImplWithTTL.get()`? dropWhile would not 
work.
   
   E.g. some elements in the middle weren't expired at batch N but going to be 
expired at batch N + 1, and earlier elements are somehow having later 
expiration time and do not expire at batch N + 1. Calling get() in batch N + 1 
will produce all elements including the elements which are going to be expired 
in batch N + 1, because earlier elements will terminate dropWhile.



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{ListStateImplWithTTL, 
MemoryStream}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.util.StreamManualClock
+
+class ListStateTTLProcessor(ttlConfig: TTLConfig)
+  extends StatefulProcessor[String, InputEvent, OutputEvent] {
+
+  @transient private var _listState: ListStateImplWithTTL[Int] = _
+
+  override def init(
+   outputMode: OutputMode,
+   timeMode: TimeMode): Unit = {
+    _listState = getHandle
+      .getListState("listState", Encoders.scalaInt, ttlConfig)
+      .asInstanceOf[ListStateImplWithTTL[Int]]
+  }
+
+  override def handleInputRows(
+    key: String,
+    inputRows: Iterator[InputEvent],
+    timerValues: TimerValues,
+    expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    inputRows.foreach { row =>
+      val resultIter = processRow(row, _listState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+
+  def processRow(
+    row: InputEvent,
+    listState: ListStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = listState.get()
+      currState.foreach { v =>
+        results = OutputEvent(key, v, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = listState.getWithoutEnforcingTTL()
+      currState.foreach { v =>
+        results = OutputEvent(key, v, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlValues = listState.getTTLValues()
+      ttlValues.foreach { ttlValue =>
+        results = OutputEvent(key, ttlValue._1, isTTLValue = true, 
ttlValue._2) :: results
+      }
+    } else if (row.action == "put") {
+      listState.put(Array(row.value))
+    } else if (row.action == "append") {
+      listState.appendValue(row.value)
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = listState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: 
results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+/**
+ * Test suite for testing list state with TTL.
+ * We use the base TTL suite with a list state processor.
+ */
+class TransformWithListStateTTLSuite extends TransformWithStateTTLTest {
+
+  import testImplicits._
+
+  override def getProcessor(ttlConfig: TTLConfig):
+    StatefulProcessor[String, InputEvent, OutputEvent] = {
+      new ListStateTTLProcessor(ttlConfig)
+  }
+
+  override def getStateTTLMetricName: String = "numListStateWithTTLVars"
+
+  test("verify iterator works with expired values in middle of list") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName,
+      SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
+      withTempDir { checkpointLocation =>
+        val ttlConfig1 = TTLConfig(ttlDuration = Duration.ofMinutes(3))
+        val inputStream = MemoryStream[InputEvent]
+        val result1 = inputStream.toDS()
+          .groupByKey(x => x.key)
+          .transformWithState(
+            getProcessor(ttlConfig1),
+            TimeMode.ProcessingTime(),
+            OutputMode.Append())
+
+        val clock = new StreamManualClock
+        // add 3 elements with a duration of a minute
+        testStream(result1)(
+          StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock,
+            checkpointLocation = checkpointLocation.getAbsolutePath),
+          AddData(inputStream, InputEvent("k1", "put", 1)),
+          AdvanceManualClock(1 * 1000),
+          AddData(inputStream, InputEvent("k1", "append", 2)),
+          AddData(inputStream, InputEvent("k1", "append", 3)),
+          // advance clock to trigger processing
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(),
+          // get ttl values
+          AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", 
-1, null)),
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(
+            OutputEvent("k1", 1, isTTLValue = true, 181000),
+            OutputEvent("k1", 2, isTTLValue = true, 182000),
+            OutputEvent("k1", 3, isTTLValue = true, 182000)
+          ),
+          AddData(inputStream, InputEvent("k1", "get", -1, null)),
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(
+            OutputEvent("k1", 1, isTTLValue = false, -1),
+            OutputEvent("k1", 2, isTTLValue = false, -1),
+            OutputEvent("k1", 3, isTTLValue = false, -1)
+          ),
+          StopStream
+        )
+
+        val ttlConfig2 = TTLConfig(ttlDuration = Duration.ofSeconds(15))
+        val result2 = inputStream.toDS()
+          .groupByKey(x => x.key)
+          .transformWithState(
+            getProcessor(ttlConfig2),
+            TimeMode.ProcessingTime(),
+            OutputMode.Append())
+        // add 3 elements with a duration of 15 seconds
+        testStream(result2)(
+          StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock,
+            checkpointLocation = checkpointLocation.getAbsolutePath),
+          AddData(inputStream, InputEvent("k1", "append", 4)),
+          AddData(inputStream, InputEvent("k1", "append", 5)),
+          AddData(inputStream, InputEvent("k1", "append", 6)),
+          // advance clock to trigger processing
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(),
+          // get all elements without enforcing ttl
+          AddData(inputStream, InputEvent("k1", "get_without_enforcing_ttl", 
-1, null)),
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(
+            OutputEvent("k1", 1, isTTLValue = false, -1),
+            OutputEvent("k1", 2, isTTLValue = false, -1),
+            OutputEvent("k1", 3, isTTLValue = false, -1),
+            OutputEvent("k1", 4, isTTLValue = false, -1),
+            OutputEvent("k1", 5, isTTLValue = false, -1),
+            OutputEvent("k1", 6, isTTLValue = false, -1)
+          ),
+          StopStream
+        )
+        // add 3 more elements with a duration of a minute
+        testStream(result1)(
+          StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock,
+            checkpointLocation = checkpointLocation.getAbsolutePath),
+          AddData(inputStream, InputEvent("k1", "append", 7)),
+          AddData(inputStream, InputEvent("k1", "append", 8)),
+          AddData(inputStream, InputEvent("k1", "append", 9)),
+          // advance clock to trigger processing
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(),
+          // advance clock to expire the middle three elements
+          AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, 
null)),
+          AdvanceManualClock(1 * 1000),
+          // validate that the expired elements are not returned

Review Comment:
   This is not checking the expired elements, right? Comment confuses me.



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{ListStateImplWithTTL, 
MemoryStream}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.util.StreamManualClock
+
+class ListStateTTLProcessor(ttlConfig: TTLConfig)
+  extends StatefulProcessor[String, InputEvent, OutputEvent] {
+
+  @transient private var _listState: ListStateImplWithTTL[Int] = _
+
+  override def init(
+   outputMode: OutputMode,

Review Comment:
   nit: indentation is off (3 more spaces)



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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
+
+/**
+ * Class that provides a concrete implementation for a list state state 
associated with state
+ * variables (with ttl expiration support) 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 keyExprEnc - Spark SQL encoder for key
+ * @param valEncoder - Spark SQL encoder for value
+ * @param ttlConfig  - TTL configuration for values  stored in this state
+ * @param batchTimestampMs - current batch processing timestamp.
+ * @tparam S - data type of object that will be stored
+ */
+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)
+    }
+  }
+
+  /**
+   * Loops through all the values associated with the grouping key, and removes
+   * the expired elements from the list.
+   * @param groupingKey grouping key for which cleanup should be performed.
+   */
+  override def clearIfExpired(groupingKey: Array[Byte]): Long = {
+    var numValuesExpired = 0L
+    val encodedGroupingKey = 
stateTypesEncoder.encodeSerializedGroupingKey(groupingKey)
+    val unsafeRowValuesIterator = store.valuesIterator(encodedGroupingKey, 
stateName)
+    // We clear the list, and use the iterator to put back all of the 
non-expired values
+    store.remove(encodedGroupingKey, stateName)
+    var isFirst = true
+    unsafeRowValuesIterator.foreach { encodedValue =>
+      if (!stateTypesEncoder.isExpired(encodedValue, batchTimestampMs)) {
+        if (isFirst) {
+          store.put(encodedGroupingKey, encodedValue, stateName)
+          isFirst = false
+        } else {
+          store.merge(encodedGroupingKey, encodedValue, stateName)
+        }
+      } else {
+        numValuesExpired += 1
+      }
+    }
+    numValuesExpired
+  }
+
+  private def upsertTTLForStateKey(serializedGroupingKey: Array[Byte]): Unit = 
{
+    upsertTTLForStateKey(ttlExpirationMs, serializedGroupingKey)
+  }
+
+  /*
+    * Internal methods to probe state for testing. The below methods exist for 
unit tests
+    * to read the state ttl values, and ensure that values are persisted 
correctly in
+    * the underlying state store.
+    */
+
+  /**
+   * Retrieves the value from State even if its expired. This method is used
+   * in tests to read the state store value, and ensure if its cleaned up at 
the
+   * end of the micro-batch.
+   */
+  private[sql] def getWithoutEnforcingTTL(): Iterator[S] = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val unsafeRowValuesIterator = store.valuesIterator(encodedGroupingKey, 
stateName)
+    unsafeRowValuesIterator.map{
+      valueUnsafeRow =>
+        stateTypesEncoder.decodeValue(valueUnsafeRow)
+    }
+  }
+
+  /**
+   * Read the ttl value associated with the grouping key.
+   */
+  private[sql] def getTTLValues(): Iterator[(S, Long)] = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val unsafeRowValuesIterator = store.valuesIterator(encodedGroupingKey, 
stateName)
+    unsafeRowValuesIterator.map{
+      valueUnsafeRow =>
+        (stateTypesEncoder.decodeValue(valueUnsafeRow),
+          stateTypesEncoder.decodeTtlExpirationMs(valueUnsafeRow).get)
+    }
+  }
+  /**

Review Comment:
   nit: empty line between two



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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
+
+/**
+ * Class that provides a concrete implementation for a list state state 
associated with state
+ * variables (with ttl expiration support) 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 keyExprEnc - Spark SQL encoder for key
+ * @param valEncoder - Spark SQL encoder for value
+ * @param ttlConfig  - TTL configuration for values  stored in this state
+ * @param batchTimestampMs - current batch processing timestamp.
+ * @tparam S - data type of object that will be stored
+ */
+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)
+    }
+  }
+
+  /**
+   * Loops through all the values associated with the grouping key, and removes
+   * the expired elements from the list.
+   * @param groupingKey grouping key for which cleanup should be performed.
+   */
+  override def clearIfExpired(groupingKey: Array[Byte]): Long = {
+    var numValuesExpired = 0L
+    val encodedGroupingKey = 
stateTypesEncoder.encodeSerializedGroupingKey(groupingKey)
+    val unsafeRowValuesIterator = store.valuesIterator(encodedGroupingKey, 
stateName)
+    // We clear the list, and use the iterator to put back all of the 
non-expired values
+    store.remove(encodedGroupingKey, stateName)
+    var isFirst = true
+    unsafeRowValuesIterator.foreach { encodedValue =>
+      if (!stateTypesEncoder.isExpired(encodedValue, batchTimestampMs)) {
+        if (isFirst) {
+          store.put(encodedGroupingKey, encodedValue, stateName)
+          isFirst = false
+        } else {
+          store.merge(encodedGroupingKey, encodedValue, stateName)
+        }
+      } else {
+        numValuesExpired += 1
+      }
+    }
+    numValuesExpired
+  }
+
+  private def upsertTTLForStateKey(serializedGroupingKey: Array[Byte]): Unit = 
{
+    upsertTTLForStateKey(ttlExpirationMs, serializedGroupingKey)
+  }
+
+  /*
+    * Internal methods to probe state for testing. The below methods exist for 
unit tests
+    * to read the state ttl values, and ensure that values are persisted 
correctly in
+    * the underlying state store.
+    */
+
+  /**
+   * Retrieves the value from State even if its expired. This method is used
+   * in tests to read the state store value, and ensure if its cleaned up at 
the
+   * end of the micro-batch.
+   */
+  private[sql] def getWithoutEnforcingTTL(): Iterator[S] = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val unsafeRowValuesIterator = store.valuesIterator(encodedGroupingKey, 
stateName)
+    unsafeRowValuesIterator.map{

Review Comment:
   nit: space between p and {, also valueUnsafeRow => to be after {. Saving one 
line and also one indentation.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ListStateSuite.scala:
##########
@@ -160,4 +161,60 @@ class ListStateSuite extends StateVariableSuiteBase {
       assert(listState1.get().toSeq === Seq.empty[Long])
     }
   }
+
+  test(s"test List state TTL") {
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
+      val store = provider.getStore(0)
+      val timestampMs = 10
+      val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
+        Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
+        TimeMode.ProcessingTime(), batchTimestampMs = Some(timestampMs))
+
+      val ttlConfig = TTLConfig(ttlDuration = Duration.ofMinutes(1))
+      val testState: ListStateImplWithTTL[String] = 
handle.getListState[String]("testState",
+        Encoders.STRING, ttlConfig).asInstanceOf[ListStateImplWithTTL[String]]
+      ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+      testState.put(Array("v1", "v2", "v3"))
+      assert(testState.get().toSeq === Seq("v1", "v2", "v3"))
+      assert(testState.getWithoutEnforcingTTL().toSeq === Seq("v1", "v2", 
"v3"))
+
+      val ttlExpirationMs = timestampMs + 60000
+      var ttlValues = testState.getTTLValues()
+      assert(ttlValues.nonEmpty)
+      assert(ttlValues.forall(_._2 === ttlExpirationMs))
+      var ttlStateValueIterator = testState.getValuesInTTLState()
+      assert(ttlStateValueIterator.hasNext)
+
+      // increment batchProcessingTime, or watermark and ensure expired value 
is not returned
+      val nextBatchHandle = new StatefulProcessorHandleImpl(store, 
UUID.randomUUID(),
+        Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
+        TimeMode.ProcessingTime(), batchTimestampMs = Some(ttlExpirationMs))
+
+      val nextBatchTestState: ListStateImplWithTTL[String] =
+        nextBatchHandle.getListState[String]("testState", Encoders.STRING, 
ttlConfig)
+          .asInstanceOf[ListStateImplWithTTL[String]]
+
+      ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+
+      // ensure get does not return the expired value
+      assert(!nextBatchTestState.exists())
+      assert(nextBatchTestState.get().isEmpty)
+
+      // ttl value should still exist in state
+      ttlValues = nextBatchTestState.getTTLValues()
+      assert(ttlValues.nonEmpty)
+      assert(ttlValues.forall(_._2 === ttlExpirationMs))
+      ttlStateValueIterator = nextBatchTestState.getValuesInTTLState()
+      assert(ttlStateValueIterator.hasNext)
+      assert(ttlStateValueIterator.next() === ttlExpirationMs)
+      assert(ttlStateValueIterator.isEmpty)
+
+      // getWithoutTTL should still return the expired value
+      assert(nextBatchTestState.getWithoutEnforcingTTL().toSeq === Seq("v1", 
"v2", "v3"))
+
+      nextBatchTestState.clear()
+      assert(!nextBatchTestState.exists())
+      assert(nextBatchTestState.get().isEmpty)
+    }
+  }

Review Comment:
   Do we want to replicate the test case for duration to be 0 or negative? See 
ValueStateSuite.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala:
##########
@@ -219,7 +219,7 @@ class StatefulProcessorHandleSuite extends 
StateVariableSuiteBase {
     }
   }
 
-  test(s"ttl States are populated for timeMode=ProcessingTime") {
+  test(s"ttl States are populated for valueState and timeMode=ProcessingTime") 
{

Review Comment:
   nit: (while we are here) `s` before "" is not needed



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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 java.sql.Timestamp
+import java.time.Duration
+
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.util.StreamManualClock
+
+case class InputEvent(
+    key: String,
+    action: String,
+    value: Int,
+    eventTime: Timestamp = null)
+
+case class OutputEvent(
+    key: String,
+    value: Int,
+    isTTLValue: Boolean,
+    ttlValue: Long)
+/**
+ * Tests that ttl works as expected for Value State for

Review Comment:
   The comment does not seem valid - there's no event time based TTL (for now), 
and also I guess you'll reuse this with list state as well. If you don't intend 
to place the comment here, please move this back to 
TransformWithValueStateTTLSuite.



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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 java.sql.Timestamp
+import java.time.Duration
+
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.util.StreamManualClock
+
+case class InputEvent(
+    key: String,
+    action: String,
+    value: Int,
+    eventTime: Timestamp = null)
+
+case class OutputEvent(
+    key: String,
+    value: Int,
+    isTTLValue: Boolean,
+    ttlValue: Long)
+/**
+ * Tests that ttl works as expected for Value State for
+ * processing time and event time based ttl.
+ */
+abstract class TransformWithStateTTLTest
+  extends StreamTest {
+  import testImplicits._
+
+  def getProcessor(ttlConfig: TTLConfig): StatefulProcessor[String, 
InputEvent, OutputEvent]
+
+  def getStateTTLMetricName: String
+
+  test("validate state is evicted at ttl expiry") {

Review Comment:
   I expect tests in here aren't changed from TransformWithValueStateTTLSuite 
and just extracted out. Please let me know if that's not the case, and also 
point out all the places for the change.



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{ListStateImplWithTTL, 
MemoryStream}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.util.StreamManualClock
+
+class ListStateTTLProcessor(ttlConfig: TTLConfig)
+  extends StatefulProcessor[String, InputEvent, OutputEvent] {
+
+  @transient private var _listState: ListStateImplWithTTL[Int] = _
+
+  override def init(
+   outputMode: OutputMode,
+   timeMode: TimeMode): Unit = {
+    _listState = getHandle
+      .getListState("listState", Encoders.scalaInt, ttlConfig)
+      .asInstanceOf[ListStateImplWithTTL[Int]]
+  }
+
+  override def handleInputRows(
+    key: String,
+    inputRows: Iterator[InputEvent],
+    timerValues: TimerValues,
+    expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    inputRows.foreach { row =>
+      val resultIter = processRow(row, _listState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+
+  def processRow(
+    row: InputEvent,

Review Comment:
   nit: 2 more spaces



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLTest.scala:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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 java.sql.Timestamp
+import java.time.Duration
+
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.util.StreamManualClock
+
+case class InputEvent(
+    key: String,
+    action: String,
+    value: Int,
+    eventTime: Timestamp = null)
+
+case class OutputEvent(
+    key: String,
+    value: Int,
+    isTTLValue: Boolean,
+    ttlValue: Long)
+/**

Review Comment:
   nit: empty line between two



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{ListStateImplWithTTL, 
MemoryStream}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.util.StreamManualClock
+
+class ListStateTTLProcessor(ttlConfig: TTLConfig)
+  extends StatefulProcessor[String, InputEvent, OutputEvent] {
+
+  @transient private var _listState: ListStateImplWithTTL[Int] = _
+
+  override def init(
+   outputMode: OutputMode,
+   timeMode: TimeMode): Unit = {
+    _listState = getHandle
+      .getListState("listState", Encoders.scalaInt, ttlConfig)
+      .asInstanceOf[ListStateImplWithTTL[Int]]
+  }
+
+  override def handleInputRows(
+    key: String,
+    inputRows: Iterator[InputEvent],
+    timerValues: TimerValues,
+    expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    inputRows.foreach { row =>
+      val resultIter = processRow(row, _listState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+
+  def processRow(
+    row: InputEvent,
+    listState: ListStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = listState.get()
+      currState.foreach { v =>
+        results = OutputEvent(key, v, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = listState.getWithoutEnforcingTTL()
+      currState.foreach { v =>
+        results = OutputEvent(key, v, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlValues = listState.getTTLValues()
+      ttlValues.foreach { ttlValue =>
+        results = OutputEvent(key, ttlValue._1, isTTLValue = true, 
ttlValue._2) :: results
+      }
+    } else if (row.action == "put") {
+      listState.put(Array(row.value))
+    } else if (row.action == "append") {
+      listState.appendValue(row.value)
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = listState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: 
results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+/**
+ * Test suite for testing list state with TTL.
+ * We use the base TTL suite with a list state processor.
+ */
+class TransformWithListStateTTLSuite extends TransformWithStateTTLTest {
+
+  import testImplicits._
+
+  override def getProcessor(ttlConfig: TTLConfig):
+    StatefulProcessor[String, InputEvent, OutputEvent] = {
+      new ListStateTTLProcessor(ttlConfig)
+  }
+
+  override def getStateTTLMetricName: String = "numListStateWithTTLVars"
+
+  test("verify iterator works with expired values in middle of list") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName,
+      SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
+      withTempDir { checkpointLocation =>
+        val ttlConfig1 = TTLConfig(ttlDuration = Duration.ofMinutes(3))
+        val inputStream = MemoryStream[InputEvent]
+        val result1 = inputStream.toDS()
+          .groupByKey(x => x.key)
+          .transformWithState(
+            getProcessor(ttlConfig1),
+            TimeMode.ProcessingTime(),
+            OutputMode.Append())
+
+        val clock = new StreamManualClock
+        // add 3 elements with a duration of a minute
+        testStream(result1)(
+          StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock,
+            checkpointLocation = checkpointLocation.getAbsolutePath),
+          AddData(inputStream, InputEvent("k1", "put", 1)),
+          AdvanceManualClock(1 * 1000),
+          AddData(inputStream, InputEvent("k1", "append", 2)),
+          AddData(inputStream, InputEvent("k1", "append", 3)),
+          // advance clock to trigger processing
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(),
+          // get ttl values
+          AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", 
-1, null)),
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(
+            OutputEvent("k1", 1, isTTLValue = true, 181000),
+            OutputEvent("k1", 2, isTTLValue = true, 182000),
+            OutputEvent("k1", 3, isTTLValue = true, 182000)
+          ),
+          AddData(inputStream, InputEvent("k1", "get", -1, null)),
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(
+            OutputEvent("k1", 1, isTTLValue = false, -1),
+            OutputEvent("k1", 2, isTTLValue = false, -1),
+            OutputEvent("k1", 3, isTTLValue = false, -1)
+          ),
+          StopStream
+        )
+
+        val ttlConfig2 = TTLConfig(ttlDuration = Duration.ofSeconds(15))
+        val result2 = inputStream.toDS()
+          .groupByKey(x => x.key)
+          .transformWithState(
+            getProcessor(ttlConfig2),
+            TimeMode.ProcessingTime(),
+            OutputMode.Append())
+        // add 3 elements with a duration of 15 seconds
+        testStream(result2)(
+          StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock,
+            checkpointLocation = checkpointLocation.getAbsolutePath),
+          AddData(inputStream, InputEvent("k1", "append", 4)),
+          AddData(inputStream, InputEvent("k1", "append", 5)),
+          AddData(inputStream, InputEvent("k1", "append", 6)),
+          // advance clock to trigger processing
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(),
+          // get all elements without enforcing ttl
+          AddData(inputStream, InputEvent("k1", "get_without_enforcing_ttl", 
-1, null)),
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(
+            OutputEvent("k1", 1, isTTLValue = false, -1),
+            OutputEvent("k1", 2, isTTLValue = false, -1),
+            OutputEvent("k1", 3, isTTLValue = false, -1),
+            OutputEvent("k1", 4, isTTLValue = false, -1),
+            OutputEvent("k1", 5, isTTLValue = false, -1),
+            OutputEvent("k1", 6, isTTLValue = false, -1)
+          ),
+          StopStream
+        )
+        // add 3 more elements with a duration of a minute

Review Comment:
   Also, it's going back to 3 minutes, not 1 minute.



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{ListStateImplWithTTL, 
MemoryStream}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.util.StreamManualClock
+
+class ListStateTTLProcessor(ttlConfig: TTLConfig)
+  extends StatefulProcessor[String, InputEvent, OutputEvent] {
+
+  @transient private var _listState: ListStateImplWithTTL[Int] = _
+
+  override def init(
+   outputMode: OutputMode,
+   timeMode: TimeMode): Unit = {
+    _listState = getHandle
+      .getListState("listState", Encoders.scalaInt, ttlConfig)
+      .asInstanceOf[ListStateImplWithTTL[Int]]
+  }
+
+  override def handleInputRows(
+    key: String,

Review Comment:
   nit: 2 more spaces



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{ListStateImplWithTTL, 
MemoryStream}
+import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.util.StreamManualClock
+
+class ListStateTTLProcessor(ttlConfig: TTLConfig)
+  extends StatefulProcessor[String, InputEvent, OutputEvent] {
+
+  @transient private var _listState: ListStateImplWithTTL[Int] = _
+
+  override def init(
+   outputMode: OutputMode,
+   timeMode: TimeMode): Unit = {
+    _listState = getHandle
+      .getListState("listState", Encoders.scalaInt, ttlConfig)
+      .asInstanceOf[ListStateImplWithTTL[Int]]
+  }
+
+  override def handleInputRows(
+    key: String,
+    inputRows: Iterator[InputEvent],
+    timerValues: TimerValues,
+    expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    inputRows.foreach { row =>
+      val resultIter = processRow(row, _listState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+
+  def processRow(
+    row: InputEvent,
+    listState: ListStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = listState.get()
+      currState.foreach { v =>
+        results = OutputEvent(key, v, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = listState.getWithoutEnforcingTTL()
+      currState.foreach { v =>
+        results = OutputEvent(key, v, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlValues = listState.getTTLValues()
+      ttlValues.foreach { ttlValue =>
+        results = OutputEvent(key, ttlValue._1, isTTLValue = true, 
ttlValue._2) :: results
+      }
+    } else if (row.action == "put") {
+      listState.put(Array(row.value))
+    } else if (row.action == "append") {
+      listState.appendValue(row.value)
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = listState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: 
results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+/**
+ * Test suite for testing list state with TTL.
+ * We use the base TTL suite with a list state processor.
+ */
+class TransformWithListStateTTLSuite extends TransformWithStateTTLTest {
+
+  import testImplicits._
+
+  override def getProcessor(ttlConfig: TTLConfig):
+    StatefulProcessor[String, InputEvent, OutputEvent] = {
+      new ListStateTTLProcessor(ttlConfig)
+  }
+
+  override def getStateTTLMetricName: String = "numListStateWithTTLVars"
+
+  test("verify iterator works with expired values in middle of list") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName,
+      SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
+      withTempDir { checkpointLocation =>
+        val ttlConfig1 = TTLConfig(ttlDuration = Duration.ofMinutes(3))
+        val inputStream = MemoryStream[InputEvent]
+        val result1 = inputStream.toDS()
+          .groupByKey(x => x.key)
+          .transformWithState(
+            getProcessor(ttlConfig1),
+            TimeMode.ProcessingTime(),
+            OutputMode.Append())
+
+        val clock = new StreamManualClock
+        // add 3 elements with a duration of a minute
+        testStream(result1)(
+          StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock,
+            checkpointLocation = checkpointLocation.getAbsolutePath),
+          AddData(inputStream, InputEvent("k1", "put", 1)),
+          AdvanceManualClock(1 * 1000),
+          AddData(inputStream, InputEvent("k1", "append", 2)),
+          AddData(inputStream, InputEvent("k1", "append", 3)),
+          // advance clock to trigger processing
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(),
+          // get ttl values
+          AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", 
-1, null)),
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(
+            OutputEvent("k1", 1, isTTLValue = true, 181000),
+            OutputEvent("k1", 2, isTTLValue = true, 182000),
+            OutputEvent("k1", 3, isTTLValue = true, 182000)
+          ),
+          AddData(inputStream, InputEvent("k1", "get", -1, null)),
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(
+            OutputEvent("k1", 1, isTTLValue = false, -1),
+            OutputEvent("k1", 2, isTTLValue = false, -1),
+            OutputEvent("k1", 3, isTTLValue = false, -1)
+          ),
+          StopStream
+        )
+
+        val ttlConfig2 = TTLConfig(ttlDuration = Duration.ofSeconds(15))
+        val result2 = inputStream.toDS()
+          .groupByKey(x => x.key)
+          .transformWithState(
+            getProcessor(ttlConfig2),
+            TimeMode.ProcessingTime(),
+            OutputMode.Append())
+        // add 3 elements with a duration of 15 seconds
+        testStream(result2)(
+          StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock,
+            checkpointLocation = checkpointLocation.getAbsolutePath),
+          AddData(inputStream, InputEvent("k1", "append", 4)),
+          AddData(inputStream, InputEvent("k1", "append", 5)),
+          AddData(inputStream, InputEvent("k1", "append", 6)),
+          // advance clock to trigger processing
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(),
+          // get all elements without enforcing ttl
+          AddData(inputStream, InputEvent("k1", "get_without_enforcing_ttl", 
-1, null)),
+          AdvanceManualClock(1 * 1000),
+          CheckNewAnswer(
+            OutputEvent("k1", 1, isTTLValue = false, -1),
+            OutputEvent("k1", 2, isTTLValue = false, -1),
+            OutputEvent("k1", 3, isTTLValue = false, -1),
+            OutputEvent("k1", 4, isTTLValue = false, -1),
+            OutputEvent("k1", 5, isTTLValue = false, -1),
+            OutputEvent("k1", 6, isTTLValue = false, -1)
+          ),

Review Comment:
   Could you please add a comment to get TTL values for 4 to 6 here? Should be 
much helpful for reviewer to go through.



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