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

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


The following commit(s) were added to refs/heads/master by this push:
     new 61ac3421651c [SPARK-47805][SS] Implementing TTL for MapState
61ac3421651c is described below

commit 61ac3421651c2d42127a4b3f62c9df51e85cbb85
Author: Eric Marnadi <eric.marn...@databricks.com>
AuthorDate: Tue Apr 23 16:57:16 2024 +0900

    [SPARK-47805][SS] Implementing TTL for MapState
    
    ### What changes were proposed in this pull request?
    
    This PR adds support for expiring state based on TTL for MapState. Using 
this functionality, Spark users can specify a TTL Mode for transformWithState 
operator, and provide a ttlDuration for each value in MapState. Once the 
ttlDuration has expired, the value will not be returned as part of get() and 
would be cleaned up at the end of the micro-batch.
    
    ### Why are the changes needed?
    
    These changes are needed to support TTL for MapState. The PR supports 
specifying ttl for processing time.
    
    ### Does this PR introduce _any_ user-facing change?
    
    Yes, modifies the MapState interface for specifying ttlDuration
    
    ### How was this patch tested?
    
    Added the TransformWithMapStateTTLSuite, MapStateSuite, 
StatefulProcessorHandleSuite
    ### Was this patch authored or co-authored using generative AI tooling?
    
    No
    
    Closes #45991 from ericm-db/map-state-ttl.
    
    Authored-by: Eric Marnadi <eric.marn...@databricks.com>
    Signed-off-by: Jungtaek Lim <kabhwan.opensou...@gmail.com>
---
 .../sql/streaming/StatefulProcessorHandle.scala    |  23 ++
 .../execution/streaming/ListStateImplWithTTL.scala |   1 +
 .../execution/streaming/MapStateImplWithTTL.scala  | 252 ++++++++++++++++
 .../streaming/StateTypesEncoderUtils.scala         |  25 ++
 .../streaming/StatefulProcessorHandleImpl.scala    |  17 ++
 .../spark/sql/execution/streaming/TTLState.scala   | 151 ++++++++--
 .../streaming/TransformWithStateExec.scala         |   2 +
 .../streaming/ValueStateImplWithTTL.scala          |   1 +
 .../execution/streaming/state/MapStateSuite.scala  |  92 +++++-
 .../state/StatefulProcessorHandleSuite.scala       |  19 ++
 .../streaming/TransformWithMapStateTTLSuite.scala  | 322 +++++++++++++++++++++
 11 files changed, 886 insertions(+), 19 deletions(-)

diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
index f662b685c4e4..4dc2ca875ef0 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala
@@ -108,6 +108,29 @@ private[sql] trait StatefulProcessorHandle extends 
Serializable {
       userKeyEnc: Encoder[K],
       valEncoder: Encoder[V]): MapState[K, V]
 
+  /**
+   * Function to create new or return existing map 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 mapState 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 userKeyEnc  - spark sql encoder for the map key
+   * @param valEncoder - SQL encoder for state variable
+   * @param ttlConfig  - the ttl configuration (time to live duration etc.)
+   * @tparam K - type of key for map state variable
+   * @tparam V - type of value for map state variable
+   * @return - instance of MapState of type [K,V] that can be used to store 
state persistently
+   */
+  def getMapState[K, V](
+     stateName: String,
+     userKeyEnc: Encoder[K],
+     valEncoder: Encoder[V],
+     ttlConfig: TTLConfig): MapState[K, V]
+
   /** Function to return queryInfo for currently running task */
   def getQueryInfo(): QueryInfo
 
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala
index 32bc21cea6ed..dc72f8bcd560 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ListStateImplWithTTL.scala
@@ -137,6 +137,7 @@ class ListStateImplWithTTL[S](
   /** Remove this state. */
   override def clear(): Unit = {
     store.remove(stateTypesEncoder.encodeGroupingKey(), stateName)
+    clearTTLState()
   }
 
   private def validateNewState(newState: Array[S]): Unit = {
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala
new file mode 100644
index 000000000000..2ab06f36dd5f
--- /dev/null
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution.streaming
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.Encoder
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import 
org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchema.{COMPOSITE_KEY_ROW_SCHEMA,
 VALUE_ROW_SCHEMA_WITH_TTL}
+import 
org.apache.spark.sql.execution.streaming.state.{PrefixKeyScanStateEncoderSpec, 
StateStore, StateStoreErrors}
+import org.apache.spark.sql.streaming.{MapState, TTLConfig}
+import org.apache.spark.util.NextIterator
+
+/**
+ * Class that provides a concrete implementation for map 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 the state variable
+ * @param keyExprEnc - Spark SQL encoder for key
+ * @param userKeyEnc  - Spark SQL encoder for the map key
+ * @param valEncoder - SQL encoder for state variable
+ * @param ttlConfig  - the ttl configuration (time to live duration etc.)
+ * @param batchTimestampMs - current batch processing timestamp.
+ * @tparam K - type of key for map state variable
+ * @tparam V - type of value for map state variable
+ * @return - instance of MapState of type [K,V] that can be used to store 
state persistently
+ */
+class MapStateImplWithTTL[K, V](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    userKeyEnc: Encoder[K],
+    valEncoder: Encoder[V],
+    ttlConfig: TTLConfig,
+    batchTimestampMs: Long) extends CompositeKeyTTLStateImpl(stateName, store, 
batchTimestampMs)
+  with MapState[K, V] with Logging {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = new CompositeKeyStateEncoder(
+    keySerializer, userKeyEnc, valEncoder, COMPOSITE_KEY_ROW_SCHEMA, 
stateName, hasTtl = true)
+
+  private val ttlExpirationMs =
+    StateTTL.calculateExpirationTimeForDuration(ttlConfig.ttlDuration, 
batchTimestampMs)
+
+  initialize()
+
+  private def initialize(): Unit = {
+    store.createColFamilyIfAbsent(stateName, COMPOSITE_KEY_ROW_SCHEMA, 
VALUE_ROW_SCHEMA_WITH_TTL,
+      PrefixKeyScanStateEncoderSpec(COMPOSITE_KEY_ROW_SCHEMA, 1))
+  }
+
+  /** Whether state exists or not. */
+  override def exists(): Boolean = {
+    iterator().nonEmpty
+  }
+
+  /** Get the state value if it exists */
+  override def getValue(key: K): V = {
+    StateStoreErrors.requireNonNullStateValue(key, stateName)
+    val encodedCompositeKey = stateTypesEncoder.encodeCompositeKey(key)
+    val retRow = store.get(encodedCompositeKey, stateName)
+
+    if (retRow != null) {
+      if (!stateTypesEncoder.isExpired(retRow, batchTimestampMs)) {
+        stateTypesEncoder.decodeValue(retRow)
+      } else {
+        null.asInstanceOf[V]
+      }
+    } else {
+      null.asInstanceOf[V]
+    }
+  }
+
+  /** Check if the user key is contained in the map */
+  override def containsKey(key: K): Boolean = {
+    StateStoreErrors.requireNonNullStateValue(key, stateName)
+    getValue(key) != null
+  }
+
+  /** Update value for given user key */
+  override def updateValue(key: K, value: V): Unit = {
+    StateStoreErrors.requireNonNullStateValue(key, stateName)
+    StateStoreErrors.requireNonNullStateValue(value, stateName)
+
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    val serializedUserKey = stateTypesEncoder.serializeUserKey(key)
+
+    val encodedValue = stateTypesEncoder.encodeValue(value, ttlExpirationMs)
+    val encodedCompositeKey = stateTypesEncoder.encodeCompositeKey(
+      serializedGroupingKey, serializedUserKey)
+    store.put(encodedCompositeKey, encodedValue, stateName)
+
+    upsertTTLForStateKey(ttlExpirationMs, serializedGroupingKey, 
serializedUserKey)
+  }
+
+  /** Get the map associated with grouping key */
+  override def iterator(): Iterator[(K, V)] = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val unsafeRowPairIterator = store.prefixScan(encodedGroupingKey, stateName)
+    new NextIterator[(K, V)] {
+      override protected def getNext(): (K, V) = {
+        val iter = unsafeRowPairIterator.dropWhile { rowPair =>
+          stateTypesEncoder.isExpired(rowPair.value, batchTimestampMs)
+        }
+        if (iter.hasNext) {
+          val currentRowPair = iter.next()
+          val key = stateTypesEncoder.decodeCompositeKey(currentRowPair.key)
+          val value = stateTypesEncoder.decodeValue(currentRowPair.value)
+          (key, value)
+        } else {
+          finished = true
+          null.asInstanceOf[(K, V)]
+        }
+      }
+
+      override protected def close(): Unit = {}
+    }
+  }
+
+  /** Get the list of keys present in map associated with grouping key */
+  override def keys(): Iterator[K] = {
+    iterator().map(_._1)
+  }
+
+  /** Get the list of values present in map associated with grouping key */
+  override def values(): Iterator[V] = {
+    iterator().map(_._2)
+  }
+
+  /** Remove user key from map state */
+  override def removeKey(key: K): Unit = {
+    StateStoreErrors.requireNonNullStateValue(key, stateName)
+    val compositeKey = stateTypesEncoder.encodeCompositeKey(key)
+    store.remove(compositeKey, stateName)
+  }
+
+  /** Remove this state. */
+  override def clear(): Unit = {
+    keys().foreach { itr =>
+      removeKey(itr)
+    }
+    clearTTLState()
+  }
+
+  /**
+   * Clears the user state associated with this grouping key
+   * if it has expired. This function is called by Spark to perform
+   * cleanup at the end of transformWithState processing.
+   *
+   * Spark uses a secondary index to determine if the user state for
+   * this grouping key has expired. However, its possible that the user
+   * has updated the TTL and secondary index is out of date. Implementations
+   * must validate that the user State has actually expired before cleanup 
based
+   * on their own State data.
+   *
+   * @param groupingKey grouping key for which cleanup should be performed.
+   * @param userKey     user key for which cleanup should be performed.
+   */
+  override def clearIfExpired(groupingKey: Array[Byte], userKey: Array[Byte]): 
Long = {
+    val encodedCompositeKey = 
stateTypesEncoder.encodeCompositeKey(groupingKey, userKey)
+    val retRow = store.get(encodedCompositeKey, stateName)
+    var numRemovedElements = 0L
+    if (retRow != null) {
+      if (stateTypesEncoder.isExpired(retRow, batchTimestampMs)) {
+        store.remove(encodedCompositeKey, stateName)
+        numRemovedElements += 1
+      }
+    }
+    numRemovedElements
+  }
+
+  /*
+   * 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(userKey: K): Option[V] = {
+    val encodedCompositeKey = stateTypesEncoder.encodeCompositeKey(userKey)
+    val retRow = store.get(encodedCompositeKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+      Some(resState)
+    } else {
+      None
+    }
+  }
+
+  /**
+   * Read the ttl value associated with the grouping and user key.
+   */
+  private[sql] def getTTLValue(userKey: K): Option[(V, Long)] = {
+    val encodedCompositeKey = stateTypesEncoder.encodeCompositeKey(userKey)
+    val retRow = store.get(encodedCompositeKey, stateName)
+
+    // if the returned row is not null, we want to return the value associated 
with the
+    // ttlExpiration
+    Option(retRow).flatMap { row =>
+      val ttlExpiration = stateTypesEncoder.decodeTtlExpirationMs(row)
+      ttlExpiration.map(expiration => (stateTypesEncoder.decodeValue(row), 
expiration))
+    }
+  }
+
+  /**
+   * Get all ttl values stored in ttl state for current implicit
+   * grouping key.
+   */
+  private[sql] def getKeyValuesInTTLState(): Iterator[(K, Long)] = {
+    val ttlIterator = ttlIndexIterator()
+    val implicitGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    var nextValue: Option[(K, Long)] = None
+
+    new Iterator[(K, Long)] {
+      override def hasNext: Boolean = {
+        while (nextValue.isEmpty && ttlIterator.hasNext) {
+          val nextTtlValue = ttlIterator.next()
+          val groupingKey = nextTtlValue.groupingKey
+          if (groupingKey sameElements implicitGroupingKey) {
+            val userKey = 
stateTypesEncoder.decodeUserKeyFromTTLRow(nextTtlValue)
+            nextValue = Some(userKey, nextTtlValue.expirationMs)
+          }
+        }
+        nextValue.isDefined
+      }
+
+      override def next(): (K, Long) = {
+        val result = nextValue.get
+        nextValue = None
+        result
+      }
+    }
+  }
+}
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala
index 56b0731e0db4..ed881b49ec1e 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala
@@ -27,6 +27,9 @@ import org.apache.spark.sql.types.{BinaryType, LongType, 
StructType}
 
 object TransformWithStateKeyValueRowSchema {
   val KEY_ROW_SCHEMA: StructType = new StructType().add("key", BinaryType)
+  val COMPOSITE_KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("key", BinaryType)
+    .add("userKey", BinaryType)
   val VALUE_ROW_SCHEMA: StructType = new StructType()
     .add("value", BinaryType)
   val VALUE_ROW_SCHEMA_WITH_TTL: StructType = new StructType()
@@ -192,6 +195,28 @@ class CompositeKeyStateEncoder[GK, K, V](
     compositeKeyRow
   }
 
+  def decodeUserKeyFromTTLRow(row: CompositeKeyTTLRow): K = {
+    val bytes = row.userKey
+    reusedKeyRow.pointTo(bytes, bytes.length)
+    val userKey = userKeyRowToObjDeserializer.apply(reusedKeyRow)
+    userKey
+  }
+
+  /**
+   * Grouping key and user key are encoded as a row of 
`schemaForCompositeKeyRow` schema.
+   * Grouping key will be encoded in `RocksDBStateEncoder` as the prefix 
column.
+   */
+  def encodeCompositeKey(
+      groupingKeyByteArr: Array[Byte],
+      userKeyByteArr: Array[Byte]): UnsafeRow = {
+    val compositeKeyRow = 
compositeKeyProjection(InternalRow(groupingKeyByteArr, userKeyByteArr))
+    compositeKeyRow
+  }
+
+  def serializeUserKey(userKey: K): Array[Byte] = {
+    userKeySerializer.apply(userKey).asInstanceOf[UnsafeRow].getBytes
+  }
+
   /**
    * The input row is of composite Key schema.
    * Only user key is returned though grouping key also exist in the row.
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
index 885df96a206a..dcc77e94de28 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala
@@ -287,6 +287,23 @@ class StatefulProcessorHandleImpl(
     resultState
   }
 
+  override def getMapState[K, V](
+      stateName: String,
+      userKeyEnc: Encoder[K],
+      valEncoder: Encoder[V],
+      ttlConfig: TTLConfig): MapState[K, V] = {
+    verifyStateVarOperations("get_map_state")
+    validateTTLConfig(ttlConfig, stateName)
+
+    assert(batchTimestampMs.isDefined)
+    val mapStateWithTTL = new MapStateImplWithTTL[K, V](store, stateName, 
keyEncoder, userKeyEnc,
+      valEncoder, ttlConfig, batchTimestampMs.get)
+    incrementMetric("numMapStateWithTTLVars")
+    ttlStates.add(mapStateWithTTL)
+
+    mapStateWithTTL
+  }
+
   private def validateTTLConfig(ttlConfig: TTLConfig, stateName: String): Unit 
= {
     val ttlDuration = ttlConfig.ttlDuration
     if (timeMode != TimeMode.ProcessingTime()) {
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala
index b245f8fc14d4..02efcefe19ca 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala
@@ -27,6 +27,10 @@ object StateTTLSchema {
   val TTL_KEY_ROW_SCHEMA: StructType = new StructType()
     .add("expirationMs", LongType)
     .add("groupingKey", BinaryType)
+  val TTL_COMPOSITE_KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+    .add("userKey", BinaryType)
   val TTL_VALUE_ROW_SCHEMA: StructType =
     StructType(Array(StructField("__dummy__", NullType)))
 }
@@ -41,6 +45,18 @@ case class SingleKeyTTLRow(
     groupingKey: Array[Byte],
     expirationMs: Long)
 
+/**
+ * Encapsulates the ttl row information stored in [[CompositeKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param userKey user key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class CompositeKeyTTLRow(
+   groupingKey: Array[Byte],
+   userKey: Array[Byte],
+   expirationMs: Long)
+
 /**
  * Represents the underlying state for secondary TTL Index for a user defined
  * state variable.
@@ -59,23 +75,6 @@ trait TTLState {
    * @return number of values cleaned up.
    */
   def clearExpiredState(): Long
-
-  /**
-   * Clears the user state associated with this grouping key
-   * if it has expired. This function is called by Spark to perform
-   * cleanup at the end of transformWithState processing.
-   *
-   * Spark uses a secondary index to determine if the user state for
-   * this grouping key has expired. However, its possible that the user
-   * has updated the TTL and secondary index is out of date. Implementations
-   * must validate that the user State has actually expired before cleanup 
based
-   * on their own State data.
-   *
-   * @param groupingKey grouping key for which cleanup should be performed.
-   *
-   * @return how many state objects were cleaned up.
-   */
-  def clearIfExpired(groupingKey: Array[Byte]): Long
 }
 
 /**
@@ -99,6 +98,18 @@ abstract class SingleKeyTTLStateImpl(
   store.createColFamilyIfAbsent(ttlColumnFamilyName, TTL_KEY_ROW_SCHEMA, 
TTL_VALUE_ROW_SCHEMA,
     RangeKeyScanStateEncoderSpec(TTL_KEY_ROW_SCHEMA, Seq(0)), isInternal = 
true)
 
+  /**
+   * This function will be called when clear() on State Variables
+   * with ttl enabled is called. This function should clear any
+   * associated ttlState, since we are clearing the user state.
+   */
+  def clearTTLState(): Unit = {
+    val iterator = store.iterator(ttlColumnFamilyName)
+    iterator.foreach { kv =>
+      store.remove(kv.key, ttlColumnFamilyName)
+    }
+  }
+
   def upsertTTLForStateKey(
       expirationMs: Long,
       groupingKey: Array[Byte]): Unit = {
@@ -163,6 +174,112 @@ abstract class SingleKeyTTLStateImpl(
       }
     }
   }
+
+  /**
+   * Clears the user state associated with this grouping key
+   * if it has expired. This function is called by Spark to perform
+   * cleanup at the end of transformWithState processing.
+   *
+   * Spark uses a secondary index to determine if the user state for
+   * this grouping key has expired. However, its possible that the user
+   * has updated the TTL and secondary index is out of date. Implementations
+   * must validate that the user State has actually expired before cleanup 
based
+   * on their own State data.
+   *
+   * @param groupingKey grouping key for which cleanup should be performed.
+   *
+   * @return true if the state was cleared, false otherwise.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Long
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key 
(grouping key).
+ */
+abstract class CompositeKeyTTLStateImpl(
+    stateName: String,
+    store: StateStore,
+    ttlExpirationMs: Long)
+  extends TTLState {
+
+  import org.apache.spark.sql.execution.streaming.StateTTLSchema._
+
+  private val ttlColumnFamilyName = s"_ttl_$stateName"
+  private val ttlKeyEncoder = 
UnsafeProjection.create(TTL_COMPOSITE_KEY_ROW_SCHEMA)
+
+  // empty row used for values
+  private val EMPTY_ROW =
+    
UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
+
+  store.createColFamilyIfAbsent(ttlColumnFamilyName, 
TTL_COMPOSITE_KEY_ROW_SCHEMA,
+    TTL_VALUE_ROW_SCHEMA, 
RangeKeyScanStateEncoderSpec(TTL_COMPOSITE_KEY_ROW_SCHEMA,
+      Seq(0)), isInternal = true)
+
+  def clearTTLState(): Unit = {
+    val iterator = store.iterator(ttlColumnFamilyName)
+    iterator.foreach { kv =>
+      store.remove(kv.key, ttlColumnFamilyName)
+    }
+  }
+
+  def upsertTTLForStateKey(
+      expirationMs: Long,
+      groupingKey: Array[Byte],
+      userKey: Array[Byte]): Unit = {
+    val encodedTtlKey = ttlKeyEncoder(InternalRow(expirationMs, groupingKey, 
userKey))
+    store.put(encodedTtlKey, EMPTY_ROW, ttlColumnFamilyName)
+  }
+
+  /**
+   * Clears any state which has ttl older than [[ttlExpirationMs]].
+   */
+  override def clearExpiredState(): Long = {
+    val iterator = store.iterator(ttlColumnFamilyName)
+    var numRemovedElements = 0L
+    iterator.takeWhile { kv =>
+      val expirationMs = kv.key.getLong(0)
+      StateTTL.isExpired(expirationMs, ttlExpirationMs)
+    }.foreach { kv =>
+      val groupingKey = kv.key.getBinary(1)
+      val userKey = kv.key.getBinary(2)
+      numRemovedElements += clearIfExpired(groupingKey, userKey)
+      store.remove(kv.key, ttlColumnFamilyName)
+    }
+    numRemovedElements
+  }
+
+  private[sql] def ttlIndexIterator(): Iterator[CompositeKeyTTLRow] = {
+    val ttlIterator = store.iterator(ttlColumnFamilyName)
+
+    new Iterator[CompositeKeyTTLRow] {
+      override def hasNext: Boolean = ttlIterator.hasNext
+
+      override def next(): CompositeKeyTTLRow = {
+        val kv = ttlIterator.next()
+        CompositeKeyTTLRow(
+          expirationMs = kv.key.getLong(0),
+          groupingKey = kv.key.getBinary(1),
+          userKey = kv.key.getBinary(2)
+        )
+      }
+    }
+  }
+
+  /**
+   * Clears the user state associated with this grouping key
+   * if it has expired. This function is called by Spark to perform
+   * cleanup at the end of transformWithState processing.
+   *
+   * Spark uses a secondary index to determine if the user state for
+   * this grouping key has expired. However, its possible that the user
+   * has updated the TTL and secondary index is out of date. Implementations
+   * must validate that the user State has actually expired before cleanup 
based
+   * on their own State data.
+   *
+   * @param groupingKey grouping key for which cleanup should be performed.
+   * @param userKey user key for which cleanup should be performed.
+   */
+  def clearIfExpired(groupingKey: Array[Byte], userKey: Array[Byte]): Long
 }
 
 /**
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala
index f5d2610d78d9..840e065b5e78 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala
@@ -313,6 +313,8 @@ case class TransformWithStateExec(
         "Number of value state variables with TTL"),
       StatefulOperatorCustomSumMetric("numListStateWithTTLVars",
         "Number of list state variables with TTL"),
+      StatefulOperatorCustomSumMetric("numMapStateWithTTLVars",
+        "Number of map state variables with TTL"),
       StatefulOperatorCustomSumMetric("numValuesRemovedDueToTTLExpiry",
         "Number of values removed due to TTL expiry")
     )
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala
index dbfa4586dc0a..0ed5a6f29a98 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala
@@ -96,6 +96,7 @@ class ValueStateImplWithTTL[S](
   /** Function to remove state for given key */
   override def clear(): Unit = {
     store.remove(stateTypesEncoder.encodeGroupingKey(), stateName)
+    clearTTLState()
   }
 
   def clearIfExpired(groupingKey: Array[Byte]): Long = {
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MapStateSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MapStateSuite.scala
index 572fc2429273..5b304c55dd5a 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MapStateSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MapStateSuite.scala
@@ -17,12 +17,14 @@
 
 package org.apache.spark.sql.execution.streaming.state
 
+import java.time.Duration
 import java.util.UUID
 
+import org.apache.spark.SparkUnsupportedOperationException
 import org.apache.spark.sql.Encoders
 import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
-import org.apache.spark.sql.execution.streaming.{ImplicitGroupingKeyTracker, 
StatefulProcessorHandleImpl}
-import org.apache.spark.sql.streaming.{ListState, MapState, TimeMode, 
ValueState}
+import org.apache.spark.sql.execution.streaming.{ImplicitGroupingKeyTracker, 
MapStateImplWithTTL, StatefulProcessorHandleImpl}
+import org.apache.spark.sql.streaming.{ListState, MapState, TimeMode, 
TTLConfig, ValueState}
 import org.apache.spark.sql.types.{BinaryType, StructType}
 
 /**
@@ -167,4 +169,90 @@ class MapStateSuite extends StateVariableSuiteBase {
       assert(mapTestState2.iterator().toList === List(("k2", 4)))
     }
   }
+
+  test("test Map 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: MapStateImplWithTTL[String, String] =
+        handle.getMapState[String, String]("testState", Encoders.STRING,
+          Encoders.STRING, ttlConfig).asInstanceOf[MapStateImplWithTTL[String, 
String]]
+      ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+      testState.updateValue("k1", "v1")
+      assert(testState.getValue("k1") === "v1")
+      assert(testState.getWithoutEnforcingTTL("k1").get === "v1")
+
+      val ttlExpirationMs = timestampMs + 60000
+      var ttlValue = testState.getTTLValue("k1")
+      assert(ttlValue.isDefined)
+      assert(ttlValue.get._2 === ttlExpirationMs)
+      var ttlStateValueIterator = testState.getKeyValuesInTTLState().map(_._2)
+      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: MapStateImplWithTTL[String, String] =
+        nextBatchHandle.getMapState[String, String](
+            "testState", Encoders.STRING, Encoders.STRING, ttlConfig)
+            .asInstanceOf[MapStateImplWithTTL[String, String]]
+
+      ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+
+      // ensure get does not return the expired value
+      assert(!nextBatchTestState.exists())
+      assert(nextBatchTestState.getValue("k1") === null)
+
+      // ttl value should still exist in state
+      ttlValue = nextBatchTestState.getTTLValue("k1")
+      assert(ttlValue.isDefined)
+      assert(ttlValue.get._2 === ttlExpirationMs)
+      ttlStateValueIterator = 
nextBatchTestState.getKeyValuesInTTLState().map(_._2)
+      assert(ttlStateValueIterator.hasNext)
+      assert(ttlStateValueIterator.next() === ttlExpirationMs)
+      assert(ttlStateValueIterator.isEmpty)
+
+      // getWithoutTTL should still return the expired value
+      assert(nextBatchTestState.getWithoutEnforcingTTL("k1").get === "v1")
+
+      nextBatchTestState.clear()
+      assert(!nextBatchTestState.exists())
+      assert(nextBatchTestState.getValue("k1") === null)
+    }
+  }
+
+  test("test negative or zero TTL duration throws error") {
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
+      val store = provider.getStore(0)
+      val batchTimestampMs = 10
+      val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
+        Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
+        TimeMode.ProcessingTime(), batchTimestampMs = Some(batchTimestampMs))
+
+      Seq(null, Duration.ZERO, Duration.ofMinutes(-1)).foreach { ttlDuration =>
+        val ttlConfig = TTLConfig(ttlDuration)
+        val ex = intercept[SparkUnsupportedOperationException] {
+          handle.getMapState[String, String](
+            "testState", Encoders.STRING, Encoders.STRING, ttlConfig)
+        }
+
+        checkError(
+          ex,
+          errorClass = "STATEFUL_PROCESSOR_TTL_DURATION_MUST_BE_POSITIVE",
+          parameters = Map(
+            "operationType" -> "update",
+            "stateName" -> "testState"
+          ),
+          matchPVals = true
+        )
+      }
+    }
+  }
 }
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala
index aafbf4df60af..52bdb0213c7e 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala
@@ -255,6 +255,24 @@ class StatefulProcessorHandleSuite extends 
StateVariableSuiteBase {
     }
   }
 
+  test("ttl States are populated for mapState 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 mapStateWithTTL = handle.getMapState("testState",
+        Encoders.STRING, Encoders.STRING, TTLConfig(Duration.ofHours(1)))
+
+      // create another state without TTL, this should not be captured in the 
handle
+      handle.getMapState("testState", Encoders.STRING, Encoders.STRING)
+
+      assert(handle.ttlStates.size() === 1)
+      assert(handle.ttlStates.get(0) === mapStateWithTTL)
+    }
+  }
+
   test("ttl States are not populated for timeMode=None") {
     tryWithProviderResource(newStoreProviderWithStateVariable(true)) { 
provider =>
       val store = provider.getStore(0)
@@ -263,6 +281,7 @@ class StatefulProcessorHandleSuite extends 
StateVariableSuiteBase {
 
       handle.getValueState("testValueState", Encoders.STRING)
       handle.getListState("testListState", Encoders.STRING)
+      handle.getMapState("testMapState", Encoders.STRING, Encoders.STRING)
 
       assert(handle.ttlStates.isEmpty)
     }
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala
new file mode 100644
index 000000000000..bf46c802fdea
--- /dev/null
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithMapStateTTLSuite.scala
@@ -0,0 +1,322 @@
+/*
+ * 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.{MapStateImplWithTTL, 
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 MapStateSingleKeyTTLProcessor(ttlConfig: TTLConfig)
+  extends StatefulProcessor[String, InputEvent, OutputEvent] {
+
+  @transient private var _mapState: MapStateImplWithTTL[String, Int] = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeMode: TimeMode): Unit = {
+    _mapState = getHandle
+      .getMapState("mapState", Encoders.STRING, Encoders.scalaInt, ttlConfig)
+      .asInstanceOf[MapStateImplWithTTL[String, Int]]
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    for (row <- inputRows) {
+      val resultIter = processRow(row, _mapState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+
+  def processRow(
+      row: InputEvent,
+      mapState: MapStateImplWithTTL[String, Int]): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val key = row.key
+    val userKey = "key"
+    if (row.action == "get") {
+      if (mapState.containsKey(userKey)) {
+        results = OutputEvent(key, mapState.getValue(userKey), isTTLValue = 
false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = mapState.getWithoutEnforcingTTL(userKey)
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: 
results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlValue = mapState.getTTLValue(userKey)
+      if (ttlValue.isDefined) {
+        val value = ttlValue.get._1
+        val ttlExpiration = ttlValue.get._2
+        results = OutputEvent(key, value, isTTLValue = true, ttlExpiration) :: 
results
+      }
+    } else if (row.action == "put") {
+      mapState.updateValue(userKey, row.value)
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = mapState.getKeyValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v._2) :: 
results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+case class MapInputEvent(
+    key: String,
+    userKey: String,
+    action: String,
+    value: Int)
+
+case class MapOutputEvent(
+    key: String,
+    userKey: String,
+    value: Int,
+    isTTLValue: Boolean,
+    ttlValue: Long)
+
+class MapStateTTLProcessor(ttlConfig: TTLConfig)
+  extends StatefulProcessor[String, MapInputEvent, MapOutputEvent] {
+
+  @transient private var _mapState: MapStateImplWithTTL[String, Int] = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeMode: TimeMode): Unit = {
+    _mapState = getHandle
+      .getMapState("mapState", Encoders.STRING, Encoders.scalaInt, ttlConfig)
+      .asInstanceOf[MapStateImplWithTTL[String, Int]]
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[MapInputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[MapOutputEvent] = {
+    var results = List[MapOutputEvent]()
+
+    for (row <- inputRows) {
+      val resultIter = processRow(row, _mapState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+
+  def processRow(
+      row: MapInputEvent,
+      mapState: MapStateImplWithTTL[String, Int]): Iterator[MapOutputEvent] = {
+    var results = List[MapOutputEvent]()
+    val key = row.key
+    val userKey = row.userKey
+    if (row.action == "get") {
+      if (mapState.containsKey(userKey)) {
+        results = MapOutputEvent(key, userKey, mapState.getValue(userKey),
+          isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = mapState.getWithoutEnforcingTTL(userKey)
+      if (currState.isDefined) {
+        results = MapOutputEvent(key, userKey, currState.get, isTTLValue = 
false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlValue = mapState.getTTLValue(userKey)
+      if (ttlValue.isDefined) {
+        val value = ttlValue.get._1
+        val ttlExpiration = ttlValue.get._2
+        results = MapOutputEvent(key, userKey, value, isTTLValue = true, 
ttlExpiration) :: results
+      }
+    } else if (row.action == "put") {
+      mapState.updateValue(userKey, row.value)
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = mapState.getKeyValuesInTTLState()
+      ttlValues.foreach { elem =>
+        results = MapOutputEvent(key, elem._1, -1, isTTLValue = true, ttlValue 
= elem._2) :: results
+      }
+    } else if (row.action == "iterator") {
+      val iter = mapState.iterator()
+      iter.foreach { elem =>
+        results = MapOutputEvent(key, elem._1, elem._2, isTTLValue = false, 
-1) :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+class TransformWithMapStateTTLSuite extends TransformWithStateTTLTest {
+
+  import testImplicits._
+  override def getProcessor(ttlConfig: TTLConfig):
+      StatefulProcessor[String, InputEvent, OutputEvent] = {
+    new MapStateSingleKeyTTLProcessor(ttlConfig)
+  }
+
+  override def getStateTTLMetricName: String = "numMapStateWithTTLVars"
+
+  test("validate state is evicted with multiple user keys") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName,
+      SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
+
+      val inputStream = MemoryStream[MapInputEvent]
+      val ttlConfig = TTLConfig(ttlDuration = Duration.ofMinutes(1))
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new MapStateTTLProcessor(ttlConfig),
+          TimeMode.ProcessingTime(),
+          OutputMode.Append())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, MapInputEvent("k1", "key1", "put", 1)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        AddData(inputStream, MapInputEvent("k1", "key1", "get", -1)),
+        AdvanceManualClock(30 * 1000),
+        CheckNewAnswer(MapOutputEvent("k1", "key1", 1, isTTLValue = false, 
-1)),
+        AddData(inputStream, MapInputEvent("k1", "key2", "put", 2)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // advance clock to expire first key
+        AdvanceManualClock(30 * 1000),
+        AddData(inputStream, MapInputEvent("k1", "key1", "get", -1),
+          MapInputEvent("k1", "key2", "get", -1)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(MapOutputEvent("k1", "key2", 2, isTTLValue = false, 
-1)),
+        StopStream
+      )
+    }
+  }
+
+  test("verify iterator doesn't return expired keys") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName,
+      SQLConf.SHUFFLE_PARTITIONS.key -> "1") {
+
+      val inputStream = MemoryStream[MapInputEvent]
+      val ttlConfig = TTLConfig(ttlDuration = Duration.ofMinutes(1))
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new MapStateTTLProcessor(ttlConfig),
+          TimeMode.ProcessingTime(),
+          OutputMode.Append())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream,
+          MapInputEvent("k1", "key1", "put", 1),
+          MapInputEvent("k1", "key2", "put", 2)
+        ),
+        AdvanceManualClock(1 * 1000), // batch timestamp: 1000
+        CheckNewAnswer(),
+        AddData(inputStream,
+          MapInputEvent("k1", "key1", "get", -1),
+          MapInputEvent("k1", "key2", "get", -1)
+        ),
+        AdvanceManualClock(30 * 1000), // batch timestamp: 31000
+        CheckNewAnswer(
+          MapOutputEvent("k1", "key1", 1, isTTLValue = false, -1),
+          MapOutputEvent("k1", "key2", 2, isTTLValue = false, -1)
+        ),
+        // get values from ttl state
+        AddData(inputStream,
+          MapInputEvent("k1", "", "get_values_in_ttl_state", -1)
+        ),
+        AdvanceManualClock(1 * 1000), // batch timestamp: 32000
+        CheckNewAnswer(
+          MapOutputEvent("k1", "key1", -1, isTTLValue = true, 61000),
+          MapOutputEvent("k1", "key2", -1, isTTLValue = true, 61000)
+        ),
+        // advance clock to expire first two values
+        AdvanceManualClock(30 * 1000), // batch timestamp: 62000
+        AddData(inputStream,
+          MapInputEvent("k1", "key3", "put", 3),
+          MapInputEvent("k1", "key4", "put", 4),
+          MapInputEvent("k1", "key5", "put", 5),
+          MapInputEvent("k1", "", "iterator", -1)
+        ),
+        AdvanceManualClock(1 * 1000), // batch timestamp: 63000
+        CheckNewAnswer(
+          MapOutputEvent("k1", "key3", 3, isTTLValue = false, -1),
+          MapOutputEvent("k1", "key4", 4, isTTLValue = false, -1),
+          MapOutputEvent("k1", "key5", 5, isTTLValue = false, -1)
+        ),
+        AddData(inputStream,
+          MapInputEvent("k1", "", "get_values_in_ttl_state", -1)
+        ),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(
+          MapOutputEvent("k1", "key3", -1, isTTLValue = true, 123000),
+          MapOutputEvent("k1", "key4", -1, isTTLValue = true, 123000),
+          MapOutputEvent("k1", "key5", -1, isTTLValue = true, 123000)
+        ),
+        // get all values without enforcing ttl
+        AddData(inputStream,
+          MapInputEvent("k1", "key1", "get_without_enforcing_ttl", -1),
+          MapInputEvent("k1", "key2", "get_without_enforcing_ttl", -1),
+          MapInputEvent("k1", "key3", "get_without_enforcing_ttl", -1),
+          MapInputEvent("k1", "key4", "get_without_enforcing_ttl", -1),
+          MapInputEvent("k1", "key5", "get_without_enforcing_ttl", -1)
+        ),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(
+          MapOutputEvent("k1", "key3", 3, isTTLValue = false, -1),
+          MapOutputEvent("k1", "key4", 4, isTTLValue = false, -1),
+          MapOutputEvent("k1", "key5", 5, isTTLValue = false, -1)
+        ),
+        // check that updating a key updates its TTL
+        AddData(inputStream, MapInputEvent("k1", "key3", "put", 3)),
+        AdvanceManualClock(1 * 1000),
+        AddData(inputStream, MapInputEvent("k1", "", 
"get_values_in_ttl_state", -1)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(
+          MapOutputEvent("k1", "key3", -1, isTTLValue = true, 123000),
+          MapOutputEvent("k1", "key3", -1, isTTLValue = true, 126000),
+          MapOutputEvent("k1", "key4", -1, isTTLValue = true, 123000),
+          MapOutputEvent("k1", "key5", -1, isTTLValue = true, 123000)
+        ),
+        AddData(inputStream, MapInputEvent("k1", "key3", 
"get_ttl_value_from_state", -1)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(
+          MapOutputEvent("k1", "key3", 3, isTTLValue = true, 126000)
+        ),
+        StopStream
+      )
+    }
+  }
+}


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


Reply via email to