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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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)

Review Comment:
   This refactoring (COMPOSITE_KEY_ROW_SCHEMA) seems to be missed to be applied 
to MapStateImpl?
   
   Please revisit MapStateImpl and apply refactoring we missed so far. No need 
to include the change to this PR - MINOR follow-up PR would be fine.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -99,6 +98,21 @@ 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 = {

Review Comment:
   If this intends to remove all TTL states because we are removing state, why 
removing only expired entries? Sounds like we should remove all TTL entries - 
please correct me if I'm missing something.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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)
+
+  createColumnFamily()
+
+  private def createColumnFamily(): 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) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!stateTypesEncoder.isExpired(retRow, batchTimestampMs)) {
+        resState
+      } 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 encodedValue = stateTypesEncoder.encodeValue(value, ttlExpirationMs)
+    val encodedCompositeKey = stateTypesEncoder.encodeCompositeKey(key)
+    store.put(encodedCompositeKey, encodedValue, stateName)
+
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    val serializedUserKey = stateTypesEncoder.serializeUserKey(key)
+    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 = {
+    store.removeColFamilyIfExists(stateName)
+    createColumnFamily()
+    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 getValuesInTTLState(): Iterator[Long] = {

Review Comment:
   Do we need to implement both methods? This seems to be just 
`getKeyValuesInTTLState().map(_._2)`, and since it's only used for testing we 
don't expect these methods to be optimized at best.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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)
+
+  createColumnFamily()
+
+  private def createColumnFamily(): 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) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!stateTypesEncoder.isExpired(retRow, batchTimestampMs)) {
+        resState
+      } 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 encodedValue = stateTypesEncoder.encodeValue(value, ttlExpirationMs)
+    val encodedCompositeKey = stateTypesEncoder.encodeCompositeKey(key)
+    store.put(encodedCompositeKey, encodedValue, stateName)
+
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()

Review Comment:
   Here we've already done with serialization for two keys in 
encodeCompositeKey(). Shall we either require caller of encodeCompositeKey() to 
call serializeGroupingKey() & serializeUserKey() by itself before calling 
encodeCompositeKey(), or override encodeCompositeKey() to allow serialized 
grouping key and serialized user key?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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)
+
+  createColumnFamily()

Review Comment:
   nit: it's a minor but probably better to be consistent? initialize() in 
ListStateImplWithTTL vs createColumnFamily() in MapStateImplWithTTL.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -163,6 +177,115 @@ 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.takeWhile { kv =>
+      val expirationMs = kv.key.getLong(0)
+      StateTTL.isExpired(expirationMs, ttlExpirationMs)
+    }.foreach { kv =>
+      store.remove(kv.key, ttlColumnFamilyName)
+    }
+  }
+
+  def upsertTTLForStateKey(
+    expirationMs: Long,

Review Comment:
   nit: 2 more spaces



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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)
+
+  createColumnFamily()
+
+  private def createColumnFamily(): 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) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!stateTypesEncoder.isExpired(retRow, batchTimestampMs)) {

Review Comment:
   Why not defer decoding the value to the time when we know it's not expired?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -163,6 +177,115 @@ 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 = {

Review Comment:
   Same, if this is called when state is cleared, is there a reason we only 
clear TTL entries for expired ones?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MapStateImplWithTTL.scala:
##########
@@ -0,0 +1,280 @@
+/*
+ * 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)
+
+  createColumnFamily()
+
+  private def createColumnFamily(): 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) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!stateTypesEncoder.isExpired(retRow, batchTimestampMs)) {
+        resState
+      } 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 encodedValue = stateTypesEncoder.encodeValue(value, ttlExpirationMs)
+    val encodedCompositeKey = stateTypesEncoder.encodeCompositeKey(key)
+    store.put(encodedCompositeKey, encodedValue, stateName)
+
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    val serializedUserKey = stateTypesEncoder.serializeUserKey(key)
+    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 = {
+    store.removeColFamilyIfExists(stateName)

Review Comment:
   This is diverged with current clear() implementation of MapStateImpl. What 
would be the one to follow? Does changelog checkpoint handle the removal of 
column family and re-creation nicely in some edge-case e.g. one microbatch has 
both operations?



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