You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "sahnib (via GitHub)" <gi...@apache.org> on 2024/03/22 20:34:49 UTC

[PR] [WIP] State TTL support for ValueState TTL [spark]

sahnib opened a new pull request, #45674:
URL: https://github.com/apache/spark/pull/45674

   Note: This change has been co-authored by @ericm-db  and @sahnib 
   
   **Authors: @ericm-db @sahnib ** 
   
   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'common/utils/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   This PR adds support for expiring state based on TTL for ValueState. This functionality is needed as part of supporting Arbitrary State Management State API. 
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   Yes
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   
   
   ### Was this patch authored or co-authored using generative AI tooling?
   <!--
   If generative AI tooling has been used in the process of authoring this patch, please include the
   phrase: 'Generated-by: ' followed by the name of the tool and its version.
   If no, write 'No'.
   Please refer to the [ASF Generative Tooling Guidance](https://www.apache.org/legal/generative-tooling.html) for details.
   -->
   


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547142418


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/ValueState.scala:
##########
@@ -42,8 +43,22 @@ private[sql] trait ValueState[S] extends Serializable {
   /** Get the state if it exists as an option and None otherwise */
   def getOption(): Option[S]
 
-  /** Update the value of the state. */
-  def update(newState: S): Unit
+  /**
+   * Update the value of the state.
+   * @param newState the new value
+   * @param ttlDuration set the ttl to current batch processing time
+   *                    (for processing time TTL mode) plus ttlDuration
+   */
+  def update(newState: S, ttlDuration: Duration = Duration.ZERO): Unit
+
+
+  /**
+   * Update the value of the state.
+   *
+   * @param newState    the new value
+   * @param expirationMs set the ttl to expirationMs (processingTime or eventTime)

Review Comment:
   done.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548810035


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -103,22 +113,35 @@ class StatefulProcessorHandleImpl(
 
   private var currState: StatefulProcessorHandleState = CREATED
 
-  private def verify(condition: => Boolean, msg: String): Unit = {
-    if (!condition) {
-      throw new IllegalStateException(msg)
+  private val ttlExpirationMs =
+    if (ttlMode == TTLMode.ProcessingTimeTTL()) {
+      batchTimestampMs.get
+    } else if (ttlMode == TTLMode.EventTimeTTL()) {
+      eventTimeWatermarkMs.get
+    } else {
+    -1

Review Comment:
   Discussed offline and we will convert this to `Option[Long]` and assert below while creating the `stateImplWithTTL` 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548554766


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val TTL_KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val TTL_VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+class SingleKeyTTLStateImpl(
+    ttlMode: TTLMode,
+    stateName: String,
+    store: StateStore,
+    batchTimestampMs: Option[Long],

Review Comment:
   updated to pass the ttlExpirationMs. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548554581


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/ValueState.scala:
##########
@@ -42,8 +43,26 @@ private[sql] trait ValueState[S] extends Serializable {
   /** Get the state if it exists as an option and None otherwise */
   def getOption(): Option[S]
 
-  /** Update the value of the state. */
-  def update(newState: S): Unit
+  /**
+   * Update the value of the state.
+   *
+   * @param newState the new value
+   * @param ttlDuration set the ttl to current batch processing time
+   *                    (for processing time TTL mode) plus ttlDuration
+   *
+   * Note: This method only allows to set ttl for Processing Time.
+   * For setting ttl for eventTime, use [[update(newState, expirationTimeInMs)]] method.
+   */
+  def update(newState: S, ttlDuration: Duration = Duration.ZERO): Unit
+
+

Review Comment:
   Removed.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548787953


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala:
##########
@@ -303,6 +311,244 @@ class ValueStateSuite extends StateVariableSuiteBase {
       assert(testState.get() === null)
     }
   }
+
+  test("test Value state TTL for processing time") {
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
+      val store = provider.getStore(0)
+      val batchTimestampMs = 10
+      val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
+        Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
+        TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
+        batchTimestampMs = Some(batchTimestampMs))
+
+      val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
+        Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
+      ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+      testState.update("v1")
+      assert(testState.get() === "v1")
+      assert(testState.getWithoutEnforcingTTL().get === "v1")
+
+      var ttlValue = testState.getTTLValue()
+      assert(ttlValue.isEmpty)
+      var ttlStateValueIterator = testState.getValuesInTTLState()
+      assert(ttlStateValueIterator.isEmpty)
+
+      testState.clear()
+      assert(!testState.exists())
+      assert(testState.get() === null)
+
+      testState.update("v1", Duration.ofMinutes(1))
+      assert(testState.get() === "v1")
+      assert(testState.getWithoutEnforcingTTL().get === "v1")
+
+      val expectedTtlExpirationMs = batchTimestampMs + 60000
+      ttlValue = testState.getTTLValue()
+      assert(ttlValue.isDefined)
+      assert(ttlValue.get === expectedTtlExpirationMs)
+      ttlStateValueIterator = testState.getValuesInTTLState()
+      assert(ttlStateValueIterator.hasNext)
+      assert(ttlStateValueIterator.next() === expectedTtlExpirationMs)
+      assert(ttlStateValueIterator.isEmpty)
+
+      // increment batchProcessingTime and ensure expired value is not returned
+      val nextBatchHandle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
+        Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
+        TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
+        batchTimestampMs = Some(expectedTtlExpirationMs))
+
+      val nextBatchTestState: ValueStateImplWithTTL[String] = nextBatchHandle
+        .getValueState[String]("testState", Encoders.STRING)
+        .asInstanceOf[ValueStateImplWithTTL[String]]
+      ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+
+      // ensure get does not return the expired value
+      assert(!nextBatchTestState.exists())
+      assert(nextBatchTestState.get() === null)
+
+      // ttl value should still exist in state
+      ttlValue = nextBatchTestState.getTTLValue()
+      assert(ttlValue.isDefined)
+      assert(ttlValue.get === expectedTtlExpirationMs)
+      ttlStateValueIterator = nextBatchTestState.getValuesInTTLState()
+      assert(ttlStateValueIterator.hasNext)
+      assert(ttlStateValueIterator.next() === expectedTtlExpirationMs)
+      assert(ttlStateValueIterator.isEmpty)
+
+      // getWithoutTTL should still return the expired value
+      assert(nextBatchTestState.getWithoutEnforcingTTL().get === "v1")
+
+      nextBatchTestState.clear()
+      assert(!nextBatchTestState.exists())
+      assert(nextBatchTestState.get() === null)
+
+      nextBatchTestState.clear()
+      assert(!nextBatchTestState.exists())
+      assert(nextBatchTestState.get() === null)
+    }
+  }
+
+  test("test Value state TTL for event time") {

Review Comment:
   Done



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1552912988


##########
sql/core/src/test/java/test/org/apache/spark/sql/TestStatefulProcessor.java:
##########
@@ -85,7 +89,7 @@ public scala.collection.Iterator<String> handleInputRows(
       }
 
       count += numRows;
-      countState.update(count);
+      countState.update(count, Duration.ZERO);

Review Comment:
   As discussed offline, removed ttlDuration per update, and added ttlConfig for state variable. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1541502063


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -23,11 +23,19 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.Serializer
 import org.apache.spark.sql.catalyst.encoders.encoderFor
 import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
 import org.apache.spark.sql.execution.streaming.state.StateStoreErrors
-import org.apache.spark.sql.types.{BinaryType, StructType}
+import org.apache.spark.sql.types.{BinaryType, LongType, StructType}
 
 object StateKeyValueRowSchema {
   val KEY_ROW_SCHEMA: StructType = new StructType().add("key", BinaryType)
-  val VALUE_ROW_SCHEMA: StructType = new StructType().add("value", BinaryType)
+  val VALUE_ROW_SCHEMA: StructType = new StructType()
+    .add("value", BinaryType)
+    .add("ttlExpirationMs", LongType)

Review Comment:
   Would we add a long even if ttl is disabled ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1546663211


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -3565,6 +3571,19 @@
     ],
     "sqlState" : "42802"
   },
+  "STATEFUL_PROCESSOR_CANNOT_USE_TTL_DURATION_IN_EVENT_TIME_TTL_MODE" : {
+    "message" : [
+      "TTL duration is not allowed for event time ttl expiration on State store operation=<operationType> on state=<stateName>.",
+      "Use absolute expiration time instead."
+    ],
+    "sqlState" : "42802"
+  },
+  "STATEFUL_PROCESSOR_TTL_CANNOT_BE_NEGATIVE" : {

Review Comment:
   nit: `TTL_VALUE_CANNOT_BE` ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1542070936


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -90,16 +92,13 @@ case class TransformWithStateExec(
 
   override def keyExpressions: Seq[Attribute] = groupingAttributes
 
-  protected val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
-
-  protected val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
-
   override def requiredChildDistribution: Seq[Distribution] = {
     StatefulOperatorPartitioning.getCompatibleDistribution(groupingAttributes,
       getStateInfo, conf) ::
       Nil
   }
 
+

Review Comment:
   Removed. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on PR #45674:
URL: https://github.com/apache/spark/pull/45674#issuecomment-2038922607

   > Reviewed code changes. Reviewing tests.
   > 
   > In overall, I'd like to understand the use case where we need to set different TTL per update. My gut feeling of main use case of state TTL was that they just consider the grouping key A to never appear again after TTL has expired (so they set an enough TTL value), which actually doesn't need such flexible TTL setup.
   
   Discussed offline, the reason we decided on two APIs because ttlDuration does not make sense for event time ttlMode. In eventTime, user might want to decide TTL based on event time column value (of row being processed), or add value to watermark (however watermark for first batch is always zero and then jumps significantly as we process first batch). Having such a interface however complicates the API. 
   Its hard to decipher at this stage if Spark users would ever want to use eventTime ttl. If its needed, we should understand exact use-cases (how should ttl be calculated in event time) and then support this mode. 
   
   Based on this discussion, we have decided to remove EventTimeTTL for now. Furthermore, to simplify the API - we accept a ttlConfig per state variable which sets ttlDuration at variable level. 


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1554101379


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala:
##########
@@ -227,19 +218,17 @@ class StatefulProcessorHandleSuite extends StateVariableSuiteBase {
     }
   }
 
-  Seq("ProcessingTime", "EventTime").foreach { ttlMode =>
-    test(s"ttl States are populated for ttlMode=$ttlMode") {
-      tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
-        val store = provider.getStore(0)
-        val handle = new StatefulProcessorHandleImpl(store,
-          UUID.randomUUID(), keyExprEncoder, getTtlMode(ttlMode), TimeoutMode.NoTimeouts(),
-          batchTimestampMs = Some(10), eventTimeWatermarkMs = Some(100))
+  test(s"ttl States are populated for ttlMode=ProcessingTime") {
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
+      val store = provider.getStore(0)
+      val handle = new StatefulProcessorHandleImpl(store,
+        UUID.randomUUID(), keyExprEncoder, TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
+        batchTimestampMs = Some(10))
 
-        val valueState = handle.getValueState("testState", Encoders.STRING)
+      val valueState = handle.getValueState("testState", Encoders.STRING)

Review Comment:
   It would not be, I have captured both scenarios in the test now (created a ttl and non-ttl state). 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1546642636


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -17,17 +17,22 @@
 
 package org.apache.spark.sql.execution.streaming
 
+import org.apache.spark.internal.Logging

Review Comment:
   same here ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -49,12 +54,17 @@ object StateKeyValueRowSchema {
 class StateTypesEncoder[GK, V](
     keySerializer: Serializer[GK],
     valEncoder: Encoder[V],
-    stateName: String) {
-  import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema._
+    stateName: String,
+    hasTtl: Boolean) extends Logging {
+  import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchema._
 
   /** Variables reused for conversions between byte array and UnsafeRow */
   private val keyProjection = UnsafeProjection.create(KEY_ROW_SCHEMA)
-  private val valueProjection = UnsafeProjection.create(VALUE_ROW_SCHEMA)
+  private val valueProjection = if (hasTtl) {
+      UnsafeProjection.create(VALUE_ROW_SCHEMA_WITH_TTL)

Review Comment:
   nit: indent seems off ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -49,12 +54,17 @@ object StateKeyValueRowSchema {
 class StateTypesEncoder[GK, V](
     keySerializer: Serializer[GK],
     valEncoder: Encoder[V],
-    stateName: String) {
-  import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema._
+    stateName: String,
+    hasTtl: Boolean) extends Logging {

Review Comment:
   nit: do we use `Logging` somewhere ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -65,22 +75,49 @@ class StateTypesEncoder[GK, V](
   // TODO: validate places that are trying to encode the key and check if we can eliminate/
   // add caching for some of these calls.
   def encodeGroupingKey(): UnsafeRow = {
+    val keyRow = keyProjection(InternalRow(serializeGroupingKey()))
+    keyRow
+  }
+
+  /**
+   * Encodes the provided grouping key into Spark UnsafeRow.
+   *
+   * @param groupingKeyBytes serialized grouping key byte array
+   * @return encoded UnsafeRow
+   */
+  def encodeSerializedGroupingKey(
+      groupingKeyBytes: Array[Byte]): UnsafeRow = {

Review Comment:
   nit: could we move to same line above ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548555204


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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 java.time.Duration
+
+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.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTimestampMs - processing timestamp of the current batch.
+ * @param eventTimeWatermarkMs - event time watermark for streaming query
+ *                               (same as watermark for state eviction)
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends ValueState[S] with Logging with StateVariableWithTTLSupport {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+  private[sql] var ttlState: SingleKeyTTLStateImpl = _
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+
+    ttlState = new SingleKeyTTLStateImpl(ttlMode, stateName, store,

Review Comment:
   Modified to use one trait `TTLState` and extend it from state variable. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547135608


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -16,39 +16,46 @@
  */
 package org.apache.spark.sql.execution.streaming
 
+import java.time.Duration
+
 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.catalyst.expressions.UnsafeRow
-import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema.{KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA}
-import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore}
+import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchema.{KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA}
+import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore, StateStoreErrors}
 import org.apache.spark.sql.streaming.ValueState
 
 /**
  * Class that provides a concrete implementation for a single value state associated with state
  * variables used in the streaming transformWithState operator.
  * @param store - reference to the StateStore instance to be used for storing state
  * @param stateName - name of logical state partition
- * @param keyEnc - Spark SQL encoder for key
+ * @param keyExprEnc - Spark SQL encoder for key
  * @param valEncoder - Spark SQL encoder for value
  * @tparam S - data type of object that will be stored
  */
 class ValueStateImpl[S](
     store: StateStore,
     stateName: String,
     keyExprEnc: ExpressionEncoder[Any],
-    valEncoder: Encoder[S]) extends ValueState[S] with Logging {
+    valEncoder: Encoder[S])

Review Comment:
   done



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547134168


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -49,12 +54,17 @@ object StateKeyValueRowSchema {
 class StateTypesEncoder[GK, V](
     keySerializer: Serializer[GK],
     valEncoder: Encoder[V],
-    stateName: String) {
-  import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema._
+    stateName: String,
+    hasTtl: Boolean) extends Logging {
+  import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchema._
 
   /** Variables reused for conversions between byte array and UnsafeRow */
   private val keyProjection = UnsafeProjection.create(KEY_ROW_SCHEMA)
-  private val valueProjection = UnsafeProjection.create(VALUE_ROW_SCHEMA)
+  private val valueProjection = if (hasTtl) {
+      UnsafeProjection.create(VALUE_ROW_SCHEMA_WITH_TTL)

Review Comment:
   I indented it with 2 spaces as it was easier to read that way. I did not find any specific guidance for class variable declarations with `if .. else` in https://github.com/databricks/scala-style-guide?tab=readme-ov-file#indent. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548203583


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -65,22 +75,49 @@ class StateTypesEncoder[GK, V](
   // TODO: validate places that are trying to encode the key and check if we can eliminate/
   // add caching for some of these calls.
   def encodeGroupingKey(): UnsafeRow = {
+    val keyRow = keyProjection(InternalRow(serializeGroupingKey()))
+    keyRow
+  }
+
+  /**
+   * Encodes the provided grouping key into Spark UnsafeRow.
+   *
+   * @param groupingKeyBytes serialized grouping key byte array
+   * @return encoded UnsafeRow
+   */
+  def encodeSerializedGroupingKey(
+      groupingKeyBytes: Array[Byte]): UnsafeRow = {

Review Comment:
   yes, done. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1546640629


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/ValueState.scala:
##########
@@ -42,8 +43,22 @@ private[sql] trait ValueState[S] extends Serializable {
   /** Get the state if it exists as an option and None otherwise */
   def getOption(): Option[S]
 
-  /** Update the value of the state. */
-  def update(newState: S): Unit
+  /**
+   * Update the value of the state.
+   * @param newState the new value
+   * @param ttlDuration set the ttl to current batch processing time
+   *                    (for processing time TTL mode) plus ttlDuration

Review Comment:
   Should we note whether we throw an exception for event time based TTL mode here ? i.e. explicitly note that only processing time TTL mode is supported for this API ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1542064062


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -77,14 +78,20 @@ class StatefulProcessorHandleImpl(
     store: StateStore,
     runId: UUID,
     keyEncoder: ExpressionEncoder[Any],
+    ttlMode: TTLMode,
     timeoutMode: TimeoutMode,
-    isStreaming: Boolean = true)
+    isStreaming: Boolean = true,
+    batchTimestampMs: Option[Long] = None,
+    eventTimeWatermarkMs: Option[Long] = None)
   extends StatefulProcessorHandle with Logging {
   import StatefulProcessorHandleState._
 
+  private val ttlStates: util.List[TTLState] = new util.ArrayList[TTLState]()
+
   private val BATCH_QUERY_ID = "00000000-0000-0000-0000-000000000000"
-  private def buildQueryInfo(): QueryInfo = {
+  logInfo(s"Created StatefulProcessorHandle")

Review Comment:
   Removed. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1542070036


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -26,10 +26,10 @@ import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expressi
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.plans.physical.Distribution
 import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema.{KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA}

Review Comment:
   Yeah, makes sense. Renamed this to `TransformWithStateKeyValueRowSchema`. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548590607


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -103,22 +113,35 @@ class StatefulProcessorHandleImpl(
 
   private var currState: StatefulProcessorHandleState = CREATED
 
-  private def verify(condition: => Boolean, msg: String): Unit = {
-    if (!condition) {
-      throw new IllegalStateException(msg)
+  private val ttlExpirationMs =
+    if (ttlMode == TTLMode.ProcessingTimeTTL()) {
+      batchTimestampMs.get
+    } else if (ttlMode == TTLMode.EventTimeTTL()) {
+      eventTimeWatermarkMs.get
+    } else {
+    -1

Review Comment:
   nit: indent seems iff again ?
   
   Btw - should we just throw an exception here ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -103,22 +113,35 @@ class StatefulProcessorHandleImpl(
 
   private var currState: StatefulProcessorHandleState = CREATED
 
-  private def verify(condition: => Boolean, msg: String): Unit = {
-    if (!condition) {
-      throw new IllegalStateException(msg)
+  private val ttlExpirationMs =
+    if (ttlMode == TTLMode.ProcessingTimeTTL()) {
+      batchTimestampMs.get
+    } else if (ttlMode == TTLMode.EventTimeTTL()) {
+      eventTimeWatermarkMs.get
+    } else {
+    -1

Review Comment:
   nit: indent seems off again ?
   
   Btw - should we just throw an exception here ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1550585362


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -78,17 +80,15 @@ case class TransformWithStateExec(
   override def shortName: String = "transformWithStateExec"
 
   override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = {
-    timeoutMode match {
+    if (ttlMode == TTLMode.ProcessingTimeTTL() || timeoutMode == TimeoutMode.ProcessingTime()) {
       // TODO: check if we can return true only if actual timers are registered
-      case ProcessingTime =>
-        true
-
-      case EventTime =>
-        eventTimeWatermarkForEviction.isDefined &&
-          newInputWatermark > eventTimeWatermarkForEviction.get
-
-      case _ =>
-        false
+      logWarning(s"returning true from shouldRunAnotherBatch - $newInputWatermark")

Review Comment:
   nit: intentional ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on PR #45674:
URL: https://github.com/apache/spark/pull/45674#issuecomment-2035807653

   @HeartSaVioR PTAL, thanks. 


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548788984


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLSuite.scala:
##########
@@ -0,0 +1,579 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{MemoryStream, ValueStateImplWithTTL}
+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,
+    ttl: Duration,
+    eventTime: Timestamp = null,
+    eventTimeTtl: Timestamp = null)
+
+case class OutputEvent(
+    key: String,
+    value: Int,
+    isTTLValue: Boolean,
+    ttlValue: Long)
+
+object TTLInputProcessFunction {
+  def processRow(
+      ttlMode: TTLMode,
+      row: InputEvent,
+      valueState: ValueStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = valueState.getOption()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = valueState.getWithoutEnforcingTTL()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlExpiration = valueState.getTTLValue()
+      if (ttlExpiration.isDefined) {
+        results = OutputEvent(key, -1, isTTLValue = true, ttlExpiration.get) :: results
+      }
+    } else if (row.action == "put") {
+      if (ttlMode == TTLMode.EventTimeTTL() && row.eventTimeTtl != null) {
+        valueState.update(row.value, row.eventTimeTtl.getTime)
+      } else if (ttlMode == TTLMode.EventTimeTTL()) {
+        valueState.update(row.value)
+      } else {
+        valueState.update(row.value, row.ttl)
+      }
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = valueState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+class ValueStateTTLProcessor
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+  with Logging {
+
+  @transient private var _valueState: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueState = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    for (row <- inputRows) {

Review Comment:
   Sure



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548555378


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val TTL_KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val TTL_VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+class SingleKeyTTLStateImpl(
+    ttlMode: TTLMode,
+    stateName: String,
+    store: StateStore,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends TTLState {
+
+  import org.apache.spark.sql.execution.streaming.StateTTLSchema._
+
+  private val ttlColumnFamilyName = s"_ttl_$stateName"
+  private val ttlKeyEncoder = UnsafeProjection.create(TTL_KEY_ROW_SCHEMA)
+  private var state: StateVariableWithTTLSupport = _
+
+  // empty row used for values
+  private val EMPTY_ROW =
+    UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
+
+  store.createColFamilyIfAbsent(ttlColumnFamilyName, TTL_KEY_ROW_SCHEMA, TTL_VALUE_ROW_SCHEMA,
+    RangeKeyScanStateEncoderSpec(TTL_KEY_ROW_SCHEMA, 1), isInternal = true)
+
+  def upsertTTLForStateKey(
+      expirationMs: Long,
+      groupingKey: Array[Byte]): Unit = {
+    val encodedTtlKey = ttlKeyEncoder(InternalRow(expirationMs, groupingKey))
+    store.put(encodedTtlKey, EMPTY_ROW, ttlColumnFamilyName)
+  }
+
+  override def clearExpiredState(): Unit = {
+    val iterator = store.iterator(ttlColumnFamilyName)
+
+    iterator.takeWhile { kv =>
+      val expirationMs = kv.key.getLong(0)
+      StateTTL.isExpired(ttlMode, expirationMs,
+        batchTimestampMs, eventTimeWatermarkMs)
+    }.foreach { kv =>
+      val groupingKey = kv.key.getBinary(1)
+      state.clearIfExpired(groupingKey)
+      store.remove(kv.key, ttlColumnFamilyName)
+    }
+  }
+
+  private[sql] def setStateVariable(
+      state: StateVariableWithTTLSupport): Unit = {
+    this.state = state
+  }
+
+  private[sql] def iterator(): Iterator[SingleKeyTTLRow] = {
+    val ttlIterator = store.iterator(ttlColumnFamilyName)
+
+    new Iterator[SingleKeyTTLRow] {
+      override def hasNext: Boolean = ttlIterator.hasNext
+
+      override def next(): SingleKeyTTLRow = {
+        val kv = ttlIterator.next()
+        SingleKeyTTLRow(
+          expirationMs = kv.key.getLong(0),
+          groupingKey = kv.key.getBinary(1)
+        )
+      }
+    }
+  }
+}
+
+/**
+ * Helper methods for user State TTL.
+ */
+object StateTTL {
+  def calculateExpirationTimeForDuration(
+      ttlMode: TTLMode,
+      ttlDuration: Duration,
+      batchTimestampMs: Option[Long],
+      eventTimeWatermarkMs: Option[Long]): Long = {
+    if (ttlMode == TTLMode.ProcessingTimeTTL()) {
+      batchTimestampMs.get + ttlDuration.toMillis
+    } else if (ttlMode == TTLMode.EventTimeTTL()) {
+      eventTimeWatermarkMs.get + ttlDuration.toMillis
+    } else {
+      throw new IllegalStateException(s"cannot calculate expiration time for" +

Review Comment:
   This code path is no longer needed. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1552907749


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoder
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTtlExpirationMs - ttl expiration for the current batch.
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTtlExpirationMs: Long)
+  extends SingleKeyTTLStateImpl(stateName, store, batchTtlExpirationMs) with ValueState[S] {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    get() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    Option(get())
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!isExpired(retRow)) {
+        resState
+      } else {
+        null.asInstanceOf[S]
+      }
+    } else {
+      null.asInstanceOf[S]
+    }
+  }
+
+  /** Function to update and overwrite state associated with given key */
+  override def update(
+      newState: S,
+      ttlDuration: Duration = Duration.ZERO): Unit = {
+
+    if (ttlMode == TTLMode.EventTimeTTL() && ttlDuration != Duration.ZERO) {
+      throw StateStoreErrors.cannotProvideTTLDurationForEventTimeTTLMode("update", stateName)
+    }
+
+    if (ttlDuration != null && ttlDuration.isNegative) {
+      throw StateStoreErrors.ttlCannotBeNegative("update", stateName)
+    }
+
+    val expirationTimeInMs =
+      if (ttlDuration != null && ttlDuration != Duration.ZERO) {
+        StateTTL.calculateExpirationTimeForDuration(ttlDuration, batchTtlExpirationMs)
+      } else {
+        -1
+      }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  override def update(
+      newState: S,
+      expirationTimeInMs: Long): Unit = {
+
+    if (expirationTimeInMs < 0) {
+      throw StateStoreErrors.ttlCannotBeNegative(
+        "update", stateName)
+    }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  private def doUpdate(newState: S,
+      expirationTimeInMs: Long): Unit = {
+    val encodedValue = stateTypesEncoder.encodeValue(newState, expirationTimeInMs)
+
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    store.put(stateTypesEncoder.encodeSerializedGroupingKey(serializedGroupingKey),
+      encodedValue, stateName)
+
+    if (expirationTimeInMs != -1) {
+      upsertTTLForStateKey(expirationTimeInMs, serializedGroupingKey)
+    }
+  }
+
+  /** Function to remove state for given key */
+  override def clear(): Unit = {
+    store.remove(stateTypesEncoder.encodeGroupingKey(), stateName)
+  }
+
+  def clearIfExpired(groupingKey: Array[Byte]): Unit = {
+    val encodedGroupingKey = stateTypesEncoder.encodeSerializedGroupingKey(groupingKey)
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      if (isExpired(retRow)) {
+        store.remove(encodedGroupingKey, stateName)
+      }
+    }
+  }
+
+  private def isExpired(valueRow: UnsafeRow): Boolean = {
+    val expirationMs = stateTypesEncoder.decodeTtlExpirationMs(valueRow)
+    val isExpired = expirationMs.map(
+      StateTTL.isExpired(_, batchTtlExpirationMs))
+
+    isExpired.isDefined && isExpired.get
+  }
+
+  /*
+   * 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.

Review Comment:
   Fixed. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR closed pull request #45674: [SPARK-47558][SS] State TTL support for ValueState
URL: https://github.com/apache/spark/pull/45674


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1554103768


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala:
##########
@@ -312,187 +312,109 @@ class ValueStateSuite extends StateVariableSuiteBase {
     }
   }
 
-  Seq(TTLMode.ProcessingTimeTTL(), TTLMode.EventTimeTTL()).foreach { ttlMode =>
-    test(s"test Value state TTL for $ttlMode") {
-      tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
-        val store = provider.getStore(0)
-        val timestampMs = 10
-        val handle = createHandleForTtlMode(ttlMode, store, timestampMs)
-
-        val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-          Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
-        ImplicitGroupingKeyTracker.setImplicitKey("test_key")
-        testState.update("v1")
-        assert(testState.get() === "v1")
-        assert(testState.getWithoutEnforcingTTL().get === "v1")
-
-        var ttlValue = testState.getTTLValue()
-        assert(ttlValue.isEmpty)
-        var ttlStateValueIterator = testState.getValuesInTTLState()
-        assert(ttlStateValueIterator.isEmpty)
-
-        testState.clear()
-        assert(!testState.exists())
-        assert(testState.get() === null)
-
-        val ttlExpirationMs = timestampMs + 60000
-
-        if (ttlMode == TTLMode.ProcessingTimeTTL()) {
-          testState.update("v1", Duration.ofMinutes(1))
-        } else {
-          testState.update("v1", ttlExpirationMs)
-        }
-        assert(testState.get() === "v1")
-        assert(testState.getWithoutEnforcingTTL().get === "v1")
-
-        ttlValue = testState.getTTLValue()
-        assert(ttlValue.isDefined)
-        assert(ttlValue.get === ttlExpirationMs)
-        ttlStateValueIterator = testState.getValuesInTTLState()
-        assert(ttlStateValueIterator.hasNext)
-        assert(ttlStateValueIterator.next() === ttlExpirationMs)
-        assert(ttlStateValueIterator.isEmpty)
-
-        // increment batchProcessingTime, or watermark and ensure expired value is not returned
-        val nextBatchHandle = createHandleForTtlMode(ttlMode, store, ttlExpirationMs)
-
-        val nextBatchTestState: ValueStateImplWithTTL[String] = nextBatchHandle
-          .getValueState[String]("testState", Encoders.STRING)
-          .asInstanceOf[ValueStateImplWithTTL[String]]
-        ImplicitGroupingKeyTracker.setImplicitKey("test_key")
-
-        // ensure get does not return the expired value
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-
-        // ttl value should still exist in state
-        ttlValue = nextBatchTestState.getTTLValue()
-        assert(ttlValue.isDefined)
-        assert(ttlValue.get === ttlExpirationMs)
-        ttlStateValueIterator = nextBatchTestState.getValuesInTTLState()
-        assert(ttlStateValueIterator.hasNext)
-        assert(ttlStateValueIterator.next() === ttlExpirationMs)
-        assert(ttlStateValueIterator.isEmpty)
-
-        // getWithoutTTL should still return the expired value
-        assert(nextBatchTestState.getWithoutEnforcingTTL().get === "v1")
-
-        nextBatchTestState.clear()
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-
-        nextBatchTestState.clear()
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-      }
-    }
-  }
 
-  test("test TTL duration throws error for event time") {
+  test(s"test Value state TTL") {
     tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
       val store = provider.getStore(0)
-      val eventTimeWatermarkMs = 10
+      val timestampMs = 10
       val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
-        TTLMode.EventTimeTTL(), TimeoutMode.NoTimeouts(),
-        eventTimeWatermarkMs = Some(eventTimeWatermarkMs))
+        TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
+        batchTimestampMs = Some(timestampMs))
 
+      val ttlConfig = TTLConfig(ttlDuration = Duration.ofMinutes(1))
       val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-        Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
+        Encoders.STRING, ttlConfig).asInstanceOf[ValueStateImplWithTTL[String]]
       ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+      testState.update("v1")

Review Comment:
   it should not be. I missed updating this test, somehow local run was not initially showing this as failed. Once I cleaned everything and rebuilt, failure showed up. Fixed the testcase. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1554249774


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala:
##########
@@ -312,187 +312,109 @@ class ValueStateSuite extends StateVariableSuiteBase {
     }
   }
 
-  Seq(TTLMode.ProcessingTimeTTL(), TTLMode.EventTimeTTL()).foreach { ttlMode =>
-    test(s"test Value state TTL for $ttlMode") {
-      tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
-        val store = provider.getStore(0)
-        val timestampMs = 10
-        val handle = createHandleForTtlMode(ttlMode, store, timestampMs)
-
-        val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-          Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
-        ImplicitGroupingKeyTracker.setImplicitKey("test_key")
-        testState.update("v1")
-        assert(testState.get() === "v1")
-        assert(testState.getWithoutEnforcingTTL().get === "v1")
-
-        var ttlValue = testState.getTTLValue()
-        assert(ttlValue.isEmpty)
-        var ttlStateValueIterator = testState.getValuesInTTLState()
-        assert(ttlStateValueIterator.isEmpty)
-
-        testState.clear()
-        assert(!testState.exists())
-        assert(testState.get() === null)
-
-        val ttlExpirationMs = timestampMs + 60000
-
-        if (ttlMode == TTLMode.ProcessingTimeTTL()) {
-          testState.update("v1", Duration.ofMinutes(1))
-        } else {
-          testState.update("v1", ttlExpirationMs)
-        }
-        assert(testState.get() === "v1")
-        assert(testState.getWithoutEnforcingTTL().get === "v1")
-
-        ttlValue = testState.getTTLValue()
-        assert(ttlValue.isDefined)
-        assert(ttlValue.get === ttlExpirationMs)
-        ttlStateValueIterator = testState.getValuesInTTLState()
-        assert(ttlStateValueIterator.hasNext)
-        assert(ttlStateValueIterator.next() === ttlExpirationMs)
-        assert(ttlStateValueIterator.isEmpty)
-
-        // increment batchProcessingTime, or watermark and ensure expired value is not returned
-        val nextBatchHandle = createHandleForTtlMode(ttlMode, store, ttlExpirationMs)
-
-        val nextBatchTestState: ValueStateImplWithTTL[String] = nextBatchHandle
-          .getValueState[String]("testState", Encoders.STRING)
-          .asInstanceOf[ValueStateImplWithTTL[String]]
-        ImplicitGroupingKeyTracker.setImplicitKey("test_key")
-
-        // ensure get does not return the expired value
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-
-        // ttl value should still exist in state
-        ttlValue = nextBatchTestState.getTTLValue()
-        assert(ttlValue.isDefined)
-        assert(ttlValue.get === ttlExpirationMs)
-        ttlStateValueIterator = nextBatchTestState.getValuesInTTLState()
-        assert(ttlStateValueIterator.hasNext)
-        assert(ttlStateValueIterator.next() === ttlExpirationMs)
-        assert(ttlStateValueIterator.isEmpty)
-
-        // getWithoutTTL should still return the expired value
-        assert(nextBatchTestState.getWithoutEnforcingTTL().get === "v1")
-
-        nextBatchTestState.clear()
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-
-        nextBatchTestState.clear()
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-      }
-    }
-  }
 
-  test("test TTL duration throws error for event time") {
+  test(s"test Value state TTL") {
     tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
       val store = provider.getStore(0)
-      val eventTimeWatermarkMs = 10
+      val timestampMs = 10
       val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
-        TTLMode.EventTimeTTL(), TimeoutMode.NoTimeouts(),
-        eventTimeWatermarkMs = Some(eventTimeWatermarkMs))
+        TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
+        batchTimestampMs = Some(timestampMs))
 
+      val ttlConfig = TTLConfig(ttlDuration = Duration.ofMinutes(1))
       val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-        Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
+        Encoders.STRING, ttlConfig).asInstanceOf[ValueStateImplWithTTL[String]]
       ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+      testState.update("v1")
+      assert(testState.get() === "v1")
+      assert(testState.getWithoutEnforcingTTL().get === "v1")
 
-      val ex = intercept[SparkUnsupportedOperationException] {
-        testState.update("v1", Duration.ofMinutes(1))
-      }
+      var ttlValue = testState.getTTLValue()
+      assert(ttlValue.isEmpty)
+      var ttlStateValueIterator = testState.getValuesInTTLState()
+      assert(ttlStateValueIterator.isEmpty)
 
-      checkError(
-        ex,
-        errorClass = "STATEFUL_PROCESSOR_CANNOT_USE_TTL_DURATION_IN_EVENT_TIME_TTL_MODE",
-        parameters = Map(
-          "operationType" -> "update",
-          "stateName" -> "testState"
-        ),
-        matchPVals = true
-      )
-    }
-  }
+      testState.clear()
+      assert(!testState.exists())
+      assert(testState.get() === null)
 
-  test("test negative TTL duration throws error") {
-    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
-      val store = provider.getStore(0)
-      val batchTimestampMs = 10
-      val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
+      val ttlExpirationMs = timestampMs + 60000
+
+      testState.update("v1")
+      assert(testState.get() === "v1")
+      assert(testState.getWithoutEnforcingTTL().get === "v1")
+
+      ttlValue = testState.getTTLValue()
+      assert(ttlValue.isDefined)
+      assert(ttlValue.get === ttlExpirationMs)
+      ttlStateValueIterator = testState.getValuesInTTLState()
+      assert(ttlStateValueIterator.hasNext)
+      assert(ttlStateValueIterator.next() === ttlExpirationMs)
+      assert(ttlStateValueIterator.isEmpty)
+
+      // increment batchProcessingTime, or watermark and ensure expired value is not returned
+      val nextBatchHandle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
         TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
-        batchTimestampMs = Some(batchTimestampMs))
+        batchTimestampMs = Some(timestampMs))
 
-      val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-        Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
+      val nextBatchTestState: ValueStateImplWithTTL[String] = nextBatchHandle

Review Comment:
   Yup, I actually expected that behavior. It is probably good to add or remove TTL functionality against existing state, but it's just good to have and maybe it's even arguable whether it's really good to have. So OK.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1541504006


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -77,14 +78,20 @@ class StatefulProcessorHandleImpl(
     store: StateStore,
     runId: UUID,
     keyEncoder: ExpressionEncoder[Any],
+    ttlMode: TTLMode,
     timeoutMode: TimeoutMode,
-    isStreaming: Boolean = true)
+    isStreaming: Boolean = true,
+    batchTimestampMs: Option[Long] = None,
+    eventTimeWatermarkMs: Option[Long] = None)
   extends StatefulProcessorHandle with Logging {
   import StatefulProcessorHandleState._
 
+  private val ttlStates: util.List[TTLState] = new util.ArrayList[TTLState]()

Review Comment:
   Could we add a comment for what this list is storing ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548810035


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -103,22 +113,35 @@ class StatefulProcessorHandleImpl(
 
   private var currState: StatefulProcessorHandleState = CREATED
 
-  private def verify(condition: => Boolean, msg: String): Unit = {
-    if (!condition) {
-      throw new IllegalStateException(msg)
+  private val ttlExpirationMs =
+    if (ttlMode == TTLMode.ProcessingTimeTTL()) {
+      batchTimestampMs.get
+    } else if (ttlMode == TTLMode.EventTimeTTL()) {
+      eventTimeWatermarkMs.get
+    } else {
+    -1

Review Comment:
   Discussed offline and we will convert this to `Option[Long]` and assert below while creating the `valueStateWithTTL` 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547138251


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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 java.time.Duration
+
+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.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTimestampMs - processing timestamp of the current batch.
+ * @param eventTimeWatermarkMs - event time watermark for streaming query
+ *                               (same as watermark for state eviction)
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends ValueState[S]
+    with Logging

Review Comment:
   Hmm. I was under the impression that multiple with needs to be intended in separate line. Moved it to one line. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547135223


##########
sql/api/src/main/java/org/apache/spark/sql/streaming/TTLMode.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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 org.apache.spark.annotation.Evolving;
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.catalyst.plans.logical.*;
+
+/**
+ * Represents the type of ttl modes possible for the Dataset operations
+ * {@code transformWithState}.
+ */
+@Experimental
+@Evolving
+public class TTLMode {
+
+    /**

Review Comment:
   Hmm, should be 2 spaces. Fixed. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547200600


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala:
##########
@@ -48,7 +49,7 @@ class StatefulProcessorHandleSuite extends StateVariableSuiteBase {
       tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
         val store = provider.getStore(0)
         val handle = new StatefulProcessorHandleImpl(store,
-          UUID.randomUUID(), keyExprEncoder, getTimeoutMode(timeoutMode))
+          UUID.randomUUID(), keyExprEncoder, TTLMode.NoTTL(), getTimeoutMode(timeoutMode))

Review Comment:
   Is it possible to add unit testing to `StatefulProcessorHandleSuite` or `ValueSuite` to cover the TTL functionality at a UT level ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548293264


##########
sql/api/src/main/java/org/apache/spark/sql/streaming/TTLMode.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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 org.apache.spark.annotation.Evolving;
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.catalyst.plans.logical.*;
+
+/**
+ * Represents the type of ttl modes possible for the Dataset operations
+ * {@code transformWithState}.
+ */
+@Experimental
+@Evolving
+public class TTLMode {
+
+  /**
+   * Specifies that there is no TTL for the user state. User state would not
+   * be cleaned up by Spark automatically.
+   */
+  public static final TTLMode NoTTL() {
+        return NoTTL$.MODULE$;

Review Comment:
   maybe follow same format as lines below ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548242612


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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 java.time.Duration
+
+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.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTimestampMs - processing timestamp of the current batch.
+ * @param eventTimeWatermarkMs - event time watermark for streaming query
+ *                               (same as watermark for state eviction)
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends ValueState[S] with Logging with StateVariableWithTTLSupport {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+  private[sql] var ttlState: SingleKeyTTLStateImpl = _
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+
+    ttlState = new SingleKeyTTLStateImpl(ttlMode, stateName, store,
+      batchTimestampMs, eventTimeWatermarkMs)
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {

Review Comment:
   `ValueStateImplWithTTL` and ` ValueStateImpl` do not have a common base class, so we need to implement all functions from ValueState trait. The only common functionality was clear and exists, so I kept them separate. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1546653301


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -16,39 +16,46 @@
  */
 package org.apache.spark.sql.execution.streaming
 
+import java.time.Duration
+
 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.catalyst.expressions.UnsafeRow
-import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema.{KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA}
-import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore}
+import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchema.{KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA}
+import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore, StateStoreErrors}
 import org.apache.spark.sql.streaming.ValueState
 
 /**
  * Class that provides a concrete implementation for a single value state associated with state
  * variables used in the streaming transformWithState operator.
  * @param store - reference to the StateStore instance to be used for storing state
  * @param stateName - name of logical state partition
- * @param keyEnc - Spark SQL encoder for key
+ * @param keyExprEnc - Spark SQL encoder for key
  * @param valEncoder - Spark SQL encoder for value
  * @tparam S - data type of object that will be stored
  */
 class ValueStateImpl[S](
     store: StateStore,
     stateName: String,
     keyExprEnc: ExpressionEncoder[Any],
-    valEncoder: Encoder[S]) extends ValueState[S] with Logging {
+    valEncoder: Encoder[S])

Review Comment:
   We can retain the older formatting ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1546648688


##########
sql/api/src/main/java/org/apache/spark/sql/streaming/TTLMode.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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 org.apache.spark.annotation.Evolving;
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.catalyst.plans.logical.*;
+
+/**
+ * Represents the type of ttl modes possible for the Dataset operations
+ * {@code transformWithState}.
+ */
+@Experimental
+@Evolving
+public class TTLMode {
+
+    /**

Review Comment:
   nit: not sure if indent is off here ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547143112


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+class SingleKeyTTLStateImpl(
+    ttlMode: TTLMode,
+    stateName: String,
+    store: StateStore,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends TTLState
+  with Logging {
+
+  import org.apache.spark.sql.execution.streaming.StateTTLSchema._
+
+  private val ttlColumnFamilyName = s"_ttl_$stateName"
+  private val ttlKeyEncoder = UnsafeProjection.create(KEY_ROW_SCHEMA)

Review Comment:
   Done. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547138671


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -3565,6 +3571,19 @@
     ],
     "sqlState" : "42802"
   },
+  "STATEFUL_PROCESSOR_CANNOT_USE_TTL_DURATION_IN_EVENT_TIME_TTL_MODE" : {
+    "message" : [
+      "TTL duration is not allowed for event time ttl expiration on State store operation=<operationType> on state=<stateName>.",
+      "Use absolute expiration time instead."
+    ],
+    "sqlState" : "42802"
+  },
+  "STATEFUL_PROCESSOR_TTL_CANNOT_BE_NEGATIVE" : {

Review Comment:
   Done



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547198717


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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 java.time.Duration
+
+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.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTimestampMs - processing timestamp of the current batch.
+ * @param eventTimeWatermarkMs - event time watermark for streaming query
+ *                               (same as watermark for state eviction)
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends ValueState[S] with Logging with StateVariableWithTTLSupport {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+  private[sql] var ttlState: SingleKeyTTLStateImpl = _
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+
+    ttlState = new SingleKeyTTLStateImpl(ttlMode, stateName, store,
+      batchTimestampMs, eventTimeWatermarkMs)
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    get() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    Option(get())
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!isExpired(retRow)) {
+        resState
+      } else {
+        null.asInstanceOf[S]
+      }
+    } else {
+      null.asInstanceOf[S]
+    }
+  }
+
+  /** Function to update and overwrite state associated with given key */
+  override def update(
+      newState: S,
+      ttlDuration: Duration = Duration.ZERO): Unit = {
+
+    if (ttlMode == TTLMode.EventTimeTTL() && ttlDuration != Duration.ZERO) {
+      throw StateStoreErrors.cannotProvideTTLDurationForEventTimeTTLMode(
+        "update", stateName)
+    }
+
+    if (ttlDuration != null && ttlDuration.isNegative) {
+      throw StateStoreErrors.ttlCannotBeNegative(
+        "update", stateName)

Review Comment:
   Could prob move to line above ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547198338


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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 java.time.Duration
+
+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.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTimestampMs - processing timestamp of the current batch.
+ * @param eventTimeWatermarkMs - event time watermark for streaming query
+ *                               (same as watermark for state eviction)
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends ValueState[S] with Logging with StateVariableWithTTLSupport {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+  private[sql] var ttlState: SingleKeyTTLStateImpl = _
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+
+    ttlState = new SingleKeyTTLStateImpl(ttlMode, stateName, store,
+      batchTimestampMs, eventTimeWatermarkMs)
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {

Review Comment:
   Do we need to override this ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548744341


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -103,22 +113,35 @@ class StatefulProcessorHandleImpl(
 
   private var currState: StatefulProcessorHandleState = CREATED
 
-  private def verify(condition: => Boolean, msg: String): Unit = {
-    if (!condition) {
-      throw new IllegalStateException(msg)
+  private val ttlExpirationMs =
+    if (ttlMode == TTLMode.ProcessingTimeTTL()) {
+      batchTimestampMs.get
+    } else if (ttlMode == TTLMode.EventTimeTTL()) {
+      eventTimeWatermarkMs.get
+    } else {
+    -1

Review Comment:
   We do have ttlMode validation in TransformWithStateExec, so -1 value should never be used. 
   
   We can throw an exeception, but then we need to make this a function I think. Right now, ttlExpiraionMs is created once and used across all state variables. We cannot throw the exception while initializing the class variable. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548788722


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLSuite.scala:
##########
@@ -0,0 +1,579 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{MemoryStream, ValueStateImplWithTTL}
+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,
+    ttl: Duration,
+    eventTime: Timestamp = null,
+    eventTimeTtl: Timestamp = null)
+
+case class OutputEvent(
+    key: String,
+    value: Int,
+    isTTLValue: Boolean,
+    ttlValue: Long)
+
+object TTLInputProcessFunction {
+  def processRow(
+      ttlMode: TTLMode,
+      row: InputEvent,
+      valueState: ValueStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = valueState.getOption()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = valueState.getWithoutEnforcingTTL()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlExpiration = valueState.getTTLValue()
+      if (ttlExpiration.isDefined) {
+        results = OutputEvent(key, -1, isTTLValue = true, ttlExpiration.get) :: results
+      }
+    } else if (row.action == "put") {
+      if (ttlMode == TTLMode.EventTimeTTL() && row.eventTimeTtl != null) {
+        valueState.update(row.value, row.eventTimeTtl.getTime)
+      } else if (ttlMode == TTLMode.EventTimeTTL()) {
+        valueState.update(row.value)
+      } else {
+        valueState.update(row.value, row.ttl)
+      }
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = valueState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+class ValueStateTTLProcessor
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+  with Logging {
+
+  @transient private var _valueState: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueState = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    for (row <- inputRows) {
+      val resultIter = TTLInputProcessFunction.processRow(_ttlMode, row, _valueState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+case class MultipleValueStatesTTLProcessor(
+    ttlKey: String,
+    noTtlKey: String)
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+    with Logging {
+
+  @transient private var _valueStateWithTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _valueStateWithoutTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueStateWithTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _valueStateWithoutTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val state = if (key == ttlKey) {
+      _valueStateWithTTL
+    } else {
+      _valueStateWithoutTTL
+    }
+
+    for (row <- inputRows) {
+      val resultIterator = TTLInputProcessFunction.processRow(_ttlMode, row, state)
+      resultIterator.foreach { r =>
+        results = r :: results
+      }
+    }
+    results.iterator
+  }
+}
+
+class TransformWithStateTTLSuite
+  extends StreamTest {
+  import testImplicits._
+
+  test("validate state is evicted at ttl expiry - processing time ttl") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputStream = MemoryStream[InputEvent]
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new ValueStateTTLProcessor(),
+          TimeoutMode.NoTimeouts(),
+          TTLMode.ProcessingTimeTTL())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, InputEvent("k1", "put", 1, Duration.ofMinutes(1))),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // get this state, and make sure we get unexpired value
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // ensure ttl values were added correctly
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        // advance clock so that state expires
+        AdvanceManualClock(60 * 1000),
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        // validate expired value is not returned
+        CheckNewAnswer(),
+        // ensure this state does not exist any longer in State
+        AddData(inputStream, InputEvent("k1", "get_without_enforcing_ttl", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer()
+      )
+    }
+  }
+
+  test("validate ttl update updates the expiration timestamp - processing time ttl") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputStream = MemoryStream[InputEvent]
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new ValueStateTTLProcessor(),
+          TimeoutMode.NoTimeouts(),
+          TTLMode.ProcessingTimeTTL())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, InputEvent("k1", "put", 1, Duration.ofMinutes(1))),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // get this state, and make sure we get unexpired value
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // ensure ttl values were added correctly
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),

Review Comment:
   Is it possible to simulate a no data batch that might still trigger state eviction for expired state ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548791096


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLSuite.scala:
##########
@@ -0,0 +1,579 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{MemoryStream, ValueStateImplWithTTL}
+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,
+    ttl: Duration,
+    eventTime: Timestamp = null,
+    eventTimeTtl: Timestamp = null)
+
+case class OutputEvent(
+    key: String,
+    value: Int,
+    isTTLValue: Boolean,
+    ttlValue: Long)
+
+object TTLInputProcessFunction {
+  def processRow(
+      ttlMode: TTLMode,
+      row: InputEvent,
+      valueState: ValueStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = valueState.getOption()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = valueState.getWithoutEnforcingTTL()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlExpiration = valueState.getTTLValue()
+      if (ttlExpiration.isDefined) {
+        results = OutputEvent(key, -1, isTTLValue = true, ttlExpiration.get) :: results
+      }
+    } else if (row.action == "put") {
+      if (ttlMode == TTLMode.EventTimeTTL() && row.eventTimeTtl != null) {
+        valueState.update(row.value, row.eventTimeTtl.getTime)
+      } else if (ttlMode == TTLMode.EventTimeTTL()) {
+        valueState.update(row.value)
+      } else {
+        valueState.update(row.value, row.ttl)
+      }
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = valueState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+class ValueStateTTLProcessor
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+  with Logging {
+
+  @transient private var _valueState: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueState = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    for (row <- inputRows) {
+      val resultIter = TTLInputProcessFunction.processRow(_ttlMode, row, _valueState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+case class MultipleValueStatesTTLProcessor(
+    ttlKey: String,
+    noTtlKey: String)
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+    with Logging {
+
+  @transient private var _valueStateWithTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _valueStateWithoutTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueStateWithTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _valueStateWithoutTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val state = if (key == ttlKey) {
+      _valueStateWithTTL
+    } else {
+      _valueStateWithoutTTL
+    }
+
+    for (row <- inputRows) {
+      val resultIterator = TTLInputProcessFunction.processRow(_ttlMode, row, state)
+      resultIterator.foreach { r =>
+        results = r :: results
+      }
+    }
+    results.iterator
+  }
+}
+
+class TransformWithStateTTLSuite

Review Comment:
   Done.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548339181


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/ValueState.scala:
##########
@@ -42,8 +43,26 @@ private[sql] trait ValueState[S] extends Serializable {
   /** Get the state if it exists as an option and None otherwise */
   def getOption(): Option[S]
 
-  /** Update the value of the state. */
-  def update(newState: S): Unit
+  /**
+   * Update the value of the state.
+   *
+   * @param newState the new value
+   * @param ttlDuration set the ttl to current batch processing time
+   *                    (for processing time TTL mode) plus ttlDuration
+   *
+   * Note: This method only allows to set ttl for Processing Time.
+   * For setting ttl for eventTime, use [[update(newState, expirationTimeInMs)]] method.
+   */
+  def update(newState: S, ttlDuration: Duration = Duration.ZERO): Unit
+
+

Review Comment:
   nit: extra newline ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1554099900


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala:
##########
@@ -41,6 +41,12 @@ private[sql] trait StatefulProcessorHandle extends Serializable {
    */
   def getValueState[T](stateName: String, valEncoder: Encoder[T]): ValueState[T]
 
+  def getValueState[T](
+      stateName: String,
+      valEncoder: Encoder[T],
+      tTLConfig: TTLConfig): ValueState[T]

Review Comment:
   Fixed. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1551120394


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoder
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTtlExpirationMs - ttl expiration for the current batch.
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTtlExpirationMs: Long)
+  extends SingleKeyTTLStateImpl(stateName, store, batchTtlExpirationMs) with ValueState[S] {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    get() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    Option(get())
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!isExpired(retRow)) {
+        resState
+      } else {
+        null.asInstanceOf[S]
+      }
+    } else {
+      null.asInstanceOf[S]
+    }
+  }
+
+  /** Function to update and overwrite state associated with given key */
+  override def update(
+      newState: S,
+      ttlDuration: Duration = Duration.ZERO): Unit = {
+
+    if (ttlMode == TTLMode.EventTimeTTL() && ttlDuration != Duration.ZERO) {
+      throw StateStoreErrors.cannotProvideTTLDurationForEventTimeTTLMode("update", stateName)
+    }
+
+    if (ttlDuration != null && ttlDuration.isNegative) {
+      throw StateStoreErrors.ttlCannotBeNegative("update", stateName)
+    }
+
+    val expirationTimeInMs =
+      if (ttlDuration != null && ttlDuration != Duration.ZERO) {
+        StateTTL.calculateExpirationTimeForDuration(ttlDuration, batchTtlExpirationMs)
+      } else {
+        -1
+      }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  override def update(
+      newState: S,
+      expirationTimeInMs: Long): Unit = {
+
+    if (expirationTimeInMs < 0) {
+      throw StateStoreErrors.ttlCannotBeNegative(
+        "update", stateName)
+    }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  private def doUpdate(newState: S,
+      expirationTimeInMs: Long): Unit = {
+    val encodedValue = stateTypesEncoder.encodeValue(newState, expirationTimeInMs)
+
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    store.put(stateTypesEncoder.encodeSerializedGroupingKey(serializedGroupingKey),
+      encodedValue, stateName)
+
+    if (expirationTimeInMs != -1) {
+      upsertTTLForStateKey(expirationTimeInMs, serializedGroupingKey)
+    }
+  }
+
+  /** Function to remove state for given key */
+  override def clear(): Unit = {
+    store.remove(stateTypesEncoder.encodeGroupingKey(), stateName)
+  }
+
+  def clearIfExpired(groupingKey: Array[Byte]): Unit = {
+    val encodedGroupingKey = stateTypesEncoder.encodeSerializedGroupingKey(groupingKey)
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      if (isExpired(retRow)) {
+        store.remove(encodedGroupingKey, stateName)
+      }
+    }
+  }
+
+  private def isExpired(valueRow: UnsafeRow): Boolean = {
+    val expirationMs = stateTypesEncoder.decodeTtlExpirationMs(valueRow)
+    val isExpired = expirationMs.map(

Review Comment:
   This is equivalent one-liner:
   `expirationMs.filter(StateTTL.isExpired(_, batchTtlExpirationMs)).isDefined`
   
   The above is `true` only when expirationMs is `Some(x)` and `StateTTL.isExpired(x, batchTtlExpirationMs)` is `true`.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1541540421


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateVariableWithTTLSupport.scala:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLState]].
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean yp based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+class SingleKeyTTLState(

Review Comment:
   nit: should we call this `SingleKeyTTLStateImpl` to be consistent ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1543448561


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateVariableWithTTLSupport.scala:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLState]].
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean yp based on ttl values stored in

Review Comment:
   Done



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547187515


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+class SingleKeyTTLStateImpl(
+    ttlMode: TTLMode,
+    stateName: String,
+    store: StateStore,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends TTLState
+  with Logging {
+
+  import org.apache.spark.sql.execution.streaming.StateTTLSchema._
+
+  private val ttlColumnFamilyName = s"_ttl_$stateName"
+  private val ttlKeyEncoder = UnsafeProjection.create(KEY_ROW_SCHEMA)
+  private var state: StateVariableWithTTLSupport = _
+
+  // empty row used for values
+  private val EMPTY_ROW =
+    UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
+
+  store.createColFamilyIfAbsent(ttlColumnFamilyName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA,
+    RangeKeyScanStateEncoderSpec(KEY_ROW_SCHEMA, 1), isInternal = true)
+
+  def upsertTTLForStateKey(
+      expirationMs: Long,
+      groupingKey: Array[Byte]): Unit = {
+    val encodedTtlKey = ttlKeyEncoder(InternalRow(expirationMs, groupingKey))
+    store.put(encodedTtlKey, EMPTY_ROW, ttlColumnFamilyName)
+  }
+
+  override def clearExpiredState(): Unit = {
+    val iterator = store.iterator(ttlColumnFamilyName)
+    var reachedPastExpirationTime = false
+
+    while (iterator.hasNext && !reachedPastExpirationTime) {

Review Comment:
   Right - wondering if there is a more functional way to write this.AFAIK - we tend to not use `while` loops unless absolutely required. But in this case - I guess you want to terminate on condition - so could we do something like
   
   `takeWhile(...).foreach(...)` ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547134400


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -49,12 +54,17 @@ object StateKeyValueRowSchema {
 class StateTypesEncoder[GK, V](
     keySerializer: Serializer[GK],
     valEncoder: Encoder[V],
-    stateName: String) {
-  import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema._
+    stateName: String,
+    hasTtl: Boolean) extends Logging {

Review Comment:
   removed, thanks. 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -17,17 +17,22 @@
 
 package org.apache.spark.sql.execution.streaming
 
+import org.apache.spark.internal.Logging

Review Comment:
   removed, thanks. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548243090


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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 java.time.Duration
+
+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.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTimestampMs - processing timestamp of the current batch.
+ * @param eventTimeWatermarkMs - event time watermark for streaming query
+ *                               (same as watermark for state eviction)
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends ValueState[S] with Logging with StateVariableWithTTLSupport {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+  private[sql] var ttlState: SingleKeyTTLStateImpl = _
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+
+    ttlState = new SingleKeyTTLStateImpl(ttlMode, stateName, store,
+      batchTimestampMs, eventTimeWatermarkMs)
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    get() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    Option(get())
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!isExpired(retRow)) {
+        resState
+      } else {
+        null.asInstanceOf[S]
+      }
+    } else {
+      null.asInstanceOf[S]
+    }
+  }
+
+  /** Function to update and overwrite state associated with given key */
+  override def update(
+      newState: S,
+      ttlDuration: Duration = Duration.ZERO): Unit = {
+
+    if (ttlMode == TTLMode.EventTimeTTL() && ttlDuration != Duration.ZERO) {
+      throw StateStoreErrors.cannotProvideTTLDurationForEventTimeTTLMode(
+        "update", stateName)
+    }
+
+    if (ttlDuration != null && ttlDuration.isNegative) {
+      throw StateStoreErrors.ttlCannotBeNegative(
+        "update", stateName)

Review Comment:
   Done



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548265031


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala:
##########
@@ -48,7 +49,7 @@ class StatefulProcessorHandleSuite extends StateVariableSuiteBase {
       tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
         val store = provider.getStore(0)
         val handle = new StatefulProcessorHandleImpl(store,
-          UUID.randomUUID(), keyExprEncoder, getTimeoutMode(timeoutMode))
+          UUID.randomUUID(), keyExprEncoder, TTLMode.NoTTL(), getTimeoutMode(timeoutMode))

Review Comment:
   I added testcases for `StatefulProcessorHandleSuite`. Will add few more testcases for ValueStateImplWithTTL, and update the PR. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547189404


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -16,39 +16,45 @@
  */
 package org.apache.spark.sql.execution.streaming
 
+import java.time.Duration
+
 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.catalyst.expressions.UnsafeRow
-import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema.{KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA}
-import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore}
+import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchema.{KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA}
+import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore, StateStoreErrors}
 import org.apache.spark.sql.streaming.ValueState
 
 /**
  * Class that provides a concrete implementation for a single value state associated with state
  * variables used in the streaming transformWithState operator.
  * @param store - reference to the StateStore instance to be used for storing state
  * @param stateName - name of logical state partition
- * @param keyEnc - Spark SQL encoder for key
+ * @param keyExprEnc - Spark SQL encoder for key
  * @param valEncoder - Spark SQL encoder for value
  * @tparam S - data type of object that will be stored
  */
 class ValueStateImpl[S](
     store: StateStore,
     stateName: String,
     keyExprEnc: ExpressionEncoder[Any],
-    valEncoder: Encoder[S]) extends ValueState[S] with Logging {
+    valEncoder: Encoder[S])
+  extends ValueState[S] with Logging {
 
   private val keySerializer = keyExprEnc.createSerializer()
-
   private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder, stateName)
+  private[sql] var ttlState: Option[SingleKeyTTLStateImpl] = None
+
+  initialize()

Review Comment:
   Is this change just part of refactoring ? or is it required for some reason ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548202686


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -49,12 +54,17 @@ object StateKeyValueRowSchema {
 class StateTypesEncoder[GK, V](
     keySerializer: Serializer[GK],
     valEncoder: Encoder[V],
-    stateName: String) {
-  import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema._
+    stateName: String,
+    hasTtl: Boolean) extends Logging {
+  import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchema._
 
   /** Variables reused for conversions between byte array and UnsafeRow */
   private val keyProjection = UnsafeProjection.create(KEY_ROW_SCHEMA)
-  private val valueProjection = UnsafeProjection.create(VALUE_ROW_SCHEMA)
+  private val valueProjection = if (hasTtl) {
+      UnsafeProjection.create(VALUE_ROW_SCHEMA_WITH_TTL)

Review Comment:
   I think its because `UnsafeProjection.create` is inside the if part. The curly brace is next line is indented with 2 spaces. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548371855


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val TTL_KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val TTL_VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+class SingleKeyTTLStateImpl(
+    ttlMode: TTLMode,
+    stateName: String,
+    store: StateStore,
+    batchTimestampMs: Option[Long],

Review Comment:
   Should we just pass a single value here ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548382462


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,237 @@
+/*
+ * 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 java.time.Duration
+
+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.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTimestampMs - processing timestamp of the current batch.
+ * @param eventTimeWatermarkMs - event time watermark for streaming query
+ *                               (same as watermark for state eviction)
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends ValueState[S] with Logging with StateVariableWithTTLSupport {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+  private[sql] var ttlState: SingleKeyTTLStateImpl = _
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+
+    ttlState = new SingleKeyTTLStateImpl(ttlMode, stateName, store,

Review Comment:
   Could we pass `this` here ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548292843


##########
sql/api/src/main/java/org/apache/spark/sql/streaming/TTLMode.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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 org.apache.spark.annotation.Evolving;
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.catalyst.plans.logical.*;
+
+/**
+ * Represents the type of ttl modes possible for the Dataset operations
+ * {@code transformWithState}.
+ */
+@Experimental
+@Evolving
+public class TTLMode {
+
+  /**
+   * Specifies that there is no TTL for the user state. User state would not
+   * be cleaned up by Spark automatically.
+   */
+  public static final TTLMode NoTTL() {
+        return NoTTL$.MODULE$;

Review Comment:
   Indent seems off here again ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1552905463


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoder
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTtlExpirationMs - ttl expiration for the current batch.
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTtlExpirationMs: Long)
+  extends SingleKeyTTLStateImpl(stateName, store, batchTtlExpirationMs) with ValueState[S] {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    get() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    Option(get())
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!isExpired(retRow)) {
+        resState
+      } else {
+        null.asInstanceOf[S]
+      }
+    } else {
+      null.asInstanceOf[S]
+    }
+  }
+
+  /** Function to update and overwrite state associated with given key */
+  override def update(
+      newState: S,
+      ttlDuration: Duration = Duration.ZERO): Unit = {
+
+    if (ttlMode == TTLMode.EventTimeTTL() && ttlDuration != Duration.ZERO) {
+      throw StateStoreErrors.cannotProvideTTLDurationForEventTimeTTLMode("update", stateName)
+    }
+
+    if (ttlDuration != null && ttlDuration.isNegative) {
+      throw StateStoreErrors.ttlCannotBeNegative("update", stateName)
+    }
+
+    val expirationTimeInMs =
+      if (ttlDuration != null && ttlDuration != Duration.ZERO) {
+        StateTTL.calculateExpirationTimeForDuration(ttlDuration, batchTtlExpirationMs)
+      } else {
+        -1
+      }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  override def update(
+      newState: S,
+      expirationTimeInMs: Long): Unit = {
+
+    if (expirationTimeInMs < 0) {
+      throw StateStoreErrors.ttlCannotBeNegative(
+        "update", stateName)
+    }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  private def doUpdate(newState: S,
+      expirationTimeInMs: Long): Unit = {
+    val encodedValue = stateTypesEncoder.encodeValue(newState, expirationTimeInMs)
+
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    store.put(stateTypesEncoder.encodeSerializedGroupingKey(serializedGroupingKey),
+      encodedValue, stateName)
+
+    if (expirationTimeInMs != -1) {
+      upsertTTLForStateKey(expirationTimeInMs, serializedGroupingKey)
+    }
+  }
+
+  /** Function to remove state for given key */
+  override def clear(): Unit = {
+    store.remove(stateTypesEncoder.encodeGroupingKey(), stateName)
+  }
+
+  def clearIfExpired(groupingKey: Array[Byte]): Unit = {
+    val encodedGroupingKey = stateTypesEncoder.encodeSerializedGroupingKey(groupingKey)
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      if (isExpired(retRow)) {
+        store.remove(encodedGroupingKey, stateName)
+      }
+    }
+  }
+
+  private def isExpired(valueRow: UnsafeRow): Boolean = {
+    val expirationMs = stateTypesEncoder.decodeTtlExpirationMs(valueRow)
+    val isExpired = expirationMs.map(

Review Comment:
   Actually, we could even do `expirationMs.exists(StateTTL.isExpired(_, batchTimestampMs))`. Incorporated the change, 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1552920644


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -89,14 +124,29 @@ class StateTypesEncoder[GK, V](
     val value = rowToObjDeserializer.apply(reusedValRow)
     value
   }
+
+  /**
+   * Decode the ttl information out of Value row. If the ttl has
+   * not been set (-1L specifies no user defined value), the API will
+   * return None.
+   */
+  def decodeTtlExpirationMs(row: UnsafeRow): Option[Long] = {
+    val expirationMs = row.getLong(1)

Review Comment:
   I have added an assert for `hasTTL`. However, I want to note that at the end of microbatch (before `store.commit`), we will call this method multiple times for each candidate row which has expired ttl in the secondary index.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1543447959


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateVariableWithTTLSupport.scala:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLState]].
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean yp based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+class SingleKeyTTLState(

Review Comment:
   Done



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1540204683


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -77,14 +78,20 @@ class StatefulProcessorHandleImpl(
     store: StateStore,
     runId: UUID,
     keyEncoder: ExpressionEncoder[Any],
+    ttlMode: TTLMode,
     timeoutMode: TimeoutMode,
-    isStreaming: Boolean = true)
+    isStreaming: Boolean = true,
+    batchTimestampMs: Option[Long] = None,
+    eventTimeWatermarkMs: Option[Long] = None)
   extends StatefulProcessorHandle with Logging {
   import StatefulProcessorHandleState._
 
+  private val ttlStates: util.List[TTLState] = new util.ArrayList[TTLState]()
+
   private val BATCH_QUERY_ID = "00000000-0000-0000-0000-000000000000"
-  private def buildQueryInfo(): QueryInfo = {
+  logInfo(s"Created StatefulProcessorHandle")

Review Comment:
   nit: intentional ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547132684


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/ValueState.scala:
##########
@@ -42,8 +43,22 @@ private[sql] trait ValueState[S] extends Serializable {
   /** Get the state if it exists as an option and None otherwise */
   def getOption(): Option[S]
 
-  /** Update the value of the state. */
-  def update(newState: S): Unit
+  /**
+   * Update the value of the state.
+   * @param newState the new value
+   * @param ttlDuration set the ttl to current batch processing time
+   *                    (for processing time TTL mode) plus ttlDuration

Review Comment:
   Done. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547185002


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -77,14 +78,23 @@ class StatefulProcessorHandleImpl(
     store: StateStore,
     runId: UUID,
     keyEncoder: ExpressionEncoder[Any],
+    ttlMode: TTLMode,
     timeoutMode: TimeoutMode,
-    isStreaming: Boolean = true)
+    isStreaming: Boolean = true,
+    batchTimestampMs: Option[Long] = None,
+    eventTimeWatermarkMs: Option[Long] = None)
   extends StatefulProcessorHandle with Logging {
   import StatefulProcessorHandleState._
 
+  /**
+   * Stores all the active ttl states, and is used to cleanup expired values
+   * in [[doTtlCleanup()]] function.
+   */
+  private val ttlStates: util.List[TTLState] = new util.ArrayList[TTLState]()

Review Comment:
   Is there any reason to store this as Java list vs Scala seq (or other equivalent data structures) ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547199470


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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 java.time.Duration
+
+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.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTimestampMs - processing timestamp of the current batch.
+ * @param eventTimeWatermarkMs - event time watermark for streaming query
+ *                               (same as watermark for state eviction)
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends ValueState[S] with Logging with StateVariableWithTTLSupport {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+  private[sql] var ttlState: SingleKeyTTLStateImpl = _
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+
+    ttlState = new SingleKeyTTLStateImpl(ttlMode, stateName, store,
+      batchTimestampMs, eventTimeWatermarkMs)
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    get() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    Option(get())
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!isExpired(retRow)) {
+        resState
+      } else {
+        null.asInstanceOf[S]
+      }
+    } else {
+      null.asInstanceOf[S]
+    }
+  }
+
+  /** Function to update and overwrite state associated with given key */
+  override def update(
+      newState: S,
+      ttlDuration: Duration = Duration.ZERO): Unit = {
+
+    if (ttlMode == TTLMode.EventTimeTTL() && ttlDuration != Duration.ZERO) {
+      throw StateStoreErrors.cannotProvideTTLDurationForEventTimeTTLMode(
+        "update", stateName)
+    }
+
+    if (ttlDuration != null && ttlDuration.isNegative) {
+      throw StateStoreErrors.ttlCannotBeNegative(
+        "update", stateName)
+    }
+
+    val expirationTimeInMs =
+      if (ttlDuration != null && ttlDuration != Duration.ZERO) {
+        StateTTL.calculateExpirationTimeForDuration(
+          ttlMode, ttlDuration, batchTimestampMs, eventTimeWatermarkMs)
+      } else {
+        -1
+      }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  override def update(
+      newState: S,
+      expirationTimeInMs: Long): Unit = {
+
+    if (expirationTimeInMs < 0) {
+      throw StateStoreErrors.ttlCannotBeNegative(
+        "update", stateName)
+    }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  private def doUpdate(newState: S,
+      expirationTimeInMs: Long): Unit = {
+    val encodedValue = stateTypesEncoder.encodeValue(newState, expirationTimeInMs)
+
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    store.put(stateTypesEncoder.encodeSerializedGroupingKey(serializedGroupingKey),
+      encodedValue, stateName)
+
+    if (expirationTimeInMs != -1) {
+      ttlState.upsertTTLForStateKey(expirationTimeInMs, serializedGroupingKey)
+    }
+  }
+
+  /** Function to remove state for given key */
+  override def clear(): Unit = {
+    store.remove(stateTypesEncoder.encodeGroupingKey(), stateName)
+  }
+
+  def clearIfExpired(groupingKey: Array[Byte]): Unit = {
+    val encodedGroupingKey = stateTypesEncoder.encodeSerializedGroupingKey(groupingKey)
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      if (isExpired(retRow)) {
+        store.remove(encodedGroupingKey, stateName)
+      }
+    }
+  }
+
+  private def isExpired(valueRow: UnsafeRow): Boolean = {
+    val expirationMs = stateTypesEncoder.decodeTtlExpirationMs(valueRow)
+    val isExpired = expirationMs.map(
+      StateTTL.isExpired(ttlMode, _, batchTimestampMs, eventTimeWatermarkMs))
+
+    isExpired.isDefined && isExpired.get
+  }
+
+  /*
+   * 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(): Option[S] = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+      Some(resState)
+    } else {
+      None
+    }
+  }
+
+  /**
+   * Read the ttl value associated with the grouping key.
+   */
+  private[sql] def getTTLValue(): Option[Long] = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      stateTypesEncoder.decodeTtlExpirationMs(retRow)
+    } else {
+      None
+    }
+  }
+
+  /**
+   * Get all ttl values stored in ttl state for current implicit
+   * grouping key.
+   */
+  private[sql] def getValuesInTTLState(): Iterator[Long] = {
+    val ttlIterator = ttlState.iterator()
+    val implicitGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    var nextValue: Option[Long] = None
+
+    new Iterator[Long] {
+      override def hasNext: Boolean = {
+        while (nextValue.isEmpty && ttlIterator.hasNext) {
+          val nextTtlValue = ttlIterator.next()
+          val groupingKey = nextTtlValue.groupingKey
+
+          if (groupingKey sameElements implicitGroupingKey) {
+            nextValue = Some(nextTtlValue.expirationMs)
+          }
+        }
+
+        nextValue.isDefined
+      }
+
+      override def next(): Long = {
+        val result = nextValue.get
+        nextValue = None
+

Review Comment:
   nit: we could remove extra line ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1546653808


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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 java.time.Duration
+
+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.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTimestampMs - processing timestamp of the current batch.
+ * @param eventTimeWatermarkMs - event time watermark for streaming query
+ *                               (same as watermark for state eviction)
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends ValueState[S]
+    with Logging

Review Comment:
   same here ? move on line above ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1546695010


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+class SingleKeyTTLStateImpl(
+    ttlMode: TTLMode,
+    stateName: String,
+    store: StateStore,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends TTLState
+  with Logging {
+
+  import org.apache.spark.sql.execution.streaming.StateTTLSchema._
+
+  private val ttlColumnFamilyName = s"_ttl_$stateName"
+  private val ttlKeyEncoder = UnsafeProjection.create(KEY_ROW_SCHEMA)

Review Comment:
   Could we prefix with the object name for the key and value schema ? it seems that we use the same name in a bunch of places - which might be confusing ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548785634


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLSuite.scala:
##########
@@ -0,0 +1,579 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{MemoryStream, ValueStateImplWithTTL}
+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,
+    ttl: Duration,
+    eventTime: Timestamp = null,
+    eventTimeTtl: Timestamp = null)
+
+case class OutputEvent(
+    key: String,
+    value: Int,
+    isTTLValue: Boolean,
+    ttlValue: Long)
+
+object TTLInputProcessFunction {
+  def processRow(
+      ttlMode: TTLMode,
+      row: InputEvent,
+      valueState: ValueStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = valueState.getOption()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = valueState.getWithoutEnforcingTTL()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlExpiration = valueState.getTTLValue()
+      if (ttlExpiration.isDefined) {
+        results = OutputEvent(key, -1, isTTLValue = true, ttlExpiration.get) :: results
+      }
+    } else if (row.action == "put") {
+      if (ttlMode == TTLMode.EventTimeTTL() && row.eventTimeTtl != null) {
+        valueState.update(row.value, row.eventTimeTtl.getTime)
+      } else if (ttlMode == TTLMode.EventTimeTTL()) {
+        valueState.update(row.value)
+      } else {
+        valueState.update(row.value, row.ttl)
+      }
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = valueState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+class ValueStateTTLProcessor
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+  with Logging {
+
+  @transient private var _valueState: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueState = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    for (row <- inputRows) {
+      val resultIter = TTLInputProcessFunction.processRow(_ttlMode, row, _valueState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+case class MultipleValueStatesTTLProcessor(
+    ttlKey: String,
+    noTtlKey: String)
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+    with Logging {
+
+  @transient private var _valueStateWithTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _valueStateWithoutTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueStateWithTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _valueStateWithoutTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val state = if (key == ttlKey) {
+      _valueStateWithTTL
+    } else {
+      _valueStateWithoutTTL
+    }
+
+    for (row <- inputRows) {

Review Comment:
   Same here ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548795527


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLSuite.scala:
##########
@@ -0,0 +1,579 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{MemoryStream, ValueStateImplWithTTL}
+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,
+    ttl: Duration,
+    eventTime: Timestamp = null,
+    eventTimeTtl: Timestamp = null)
+
+case class OutputEvent(
+    key: String,
+    value: Int,
+    isTTLValue: Boolean,
+    ttlValue: Long)
+
+object TTLInputProcessFunction {
+  def processRow(
+      ttlMode: TTLMode,
+      row: InputEvent,
+      valueState: ValueStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = valueState.getOption()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = valueState.getWithoutEnforcingTTL()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlExpiration = valueState.getTTLValue()
+      if (ttlExpiration.isDefined) {
+        results = OutputEvent(key, -1, isTTLValue = true, ttlExpiration.get) :: results
+      }
+    } else if (row.action == "put") {
+      if (ttlMode == TTLMode.EventTimeTTL() && row.eventTimeTtl != null) {
+        valueState.update(row.value, row.eventTimeTtl.getTime)
+      } else if (ttlMode == TTLMode.EventTimeTTL()) {
+        valueState.update(row.value)
+      } else {
+        valueState.update(row.value, row.ttl)
+      }
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = valueState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+class ValueStateTTLProcessor
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+  with Logging {
+
+  @transient private var _valueState: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueState = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    for (row <- inputRows) {
+      val resultIter = TTLInputProcessFunction.processRow(_ttlMode, row, _valueState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+case class MultipleValueStatesTTLProcessor(
+    ttlKey: String,
+    noTtlKey: String)
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+    with Logging {
+
+  @transient private var _valueStateWithTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _valueStateWithoutTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueStateWithTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _valueStateWithoutTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val state = if (key == ttlKey) {
+      _valueStateWithTTL
+    } else {
+      _valueStateWithoutTTL
+    }
+
+    for (row <- inputRows) {
+      val resultIterator = TTLInputProcessFunction.processRow(_ttlMode, row, state)
+      resultIterator.foreach { r =>
+        results = r :: results
+      }
+    }
+    results.iterator
+  }
+}
+
+class TransformWithStateTTLSuite
+  extends StreamTest {
+  import testImplicits._
+
+  test("validate state is evicted at ttl expiry - processing time ttl") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputStream = MemoryStream[InputEvent]
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new ValueStateTTLProcessor(),
+          TimeoutMode.NoTimeouts(),
+          TTLMode.ProcessingTimeTTL())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, InputEvent("k1", "put", 1, Duration.ofMinutes(1))),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // get this state, and make sure we get unexpired value
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // ensure ttl values were added correctly
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        // advance clock so that state expires
+        AdvanceManualClock(60 * 1000),
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        // validate expired value is not returned
+        CheckNewAnswer(),
+        // ensure this state does not exist any longer in State
+        AddData(inputStream, InputEvent("k1", "get_without_enforcing_ttl", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer()
+      )
+    }
+  }
+
+  test("validate ttl update updates the expiration timestamp - processing time ttl") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputStream = MemoryStream[InputEvent]
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new ValueStateTTLProcessor(),
+          TimeoutMode.NoTimeouts(),
+          TTLMode.ProcessingTimeTTL())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, InputEvent("k1", "put", 1, Duration.ofMinutes(1))),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // get this state, and make sure we get unexpired value
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // ensure ttl values were added correctly
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        // advance clock and update expiration time
+        AdvanceManualClock(30 * 1000),
+        AddData(inputStream, InputEvent("k1", "put", 1, Duration.ofMinutes(1))),
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        // validate value is not expired
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // validate ttl value is updated in the state
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 95000)),
+        // validate ttl state has both ttl values present
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000),
+          OutputEvent("k1", -1, isTTLValue = true, 95000)
+        ),
+        // advance clock after older expiration value
+        AdvanceManualClock(30 * 1000),
+        // ensure unexpired value is still present in the state
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // validate that the older expiration value is removed from ttl state
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 95000))
+      )
+    }
+  }
+
+  test("validate ttl removal keeps value in state - processing time ttl") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputStream = MemoryStream[InputEvent]
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new ValueStateTTLProcessor(),
+          TimeoutMode.NoTimeouts(),
+          TTLMode.ProcessingTimeTTL())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, InputEvent("k1", "put", 1, Duration.ofMinutes(1))),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // get this state, and make sure we get unexpired value
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // ensure ttl values were added correctly
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        // advance clock and update state to remove ttl
+        AdvanceManualClock(30 * 1000),
+        AddData(inputStream, InputEvent("k1", "put", 1, null)),
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        // validate value is not expired
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // validate ttl value is removed in the value state column family
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // validate ttl state still has old ttl value present
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        // advance clock after older expiration value
+        AdvanceManualClock(30 * 1000),
+        // ensure unexpired value is still present in the state
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // validate that the older expiration value is removed from ttl state
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer()
+      )
+    }
+  }
+
+  test("validate multiple value states - processing time ttl") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val ttlKey = "k1"
+      val noTtlKey = "k2"
+
+      val inputStream = MemoryStream[InputEvent]
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          MultipleValueStatesTTLProcessor(ttlKey, noTtlKey),
+          TimeoutMode.NoTimeouts(),
+          TTLMode.ProcessingTimeTTL())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, InputEvent(ttlKey, "put", 1, Duration.ofMinutes(1))),
+        AddData(inputStream, InputEvent(noTtlKey, "put", 2, Duration.ZERO)),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // get both state values, and make sure we get unexpired value
+        AddData(inputStream, InputEvent(ttlKey, "get", -1, null)),
+        AddData(inputStream, InputEvent(noTtlKey, "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(
+          OutputEvent(ttlKey, 1, isTTLValue = false, -1),
+          OutputEvent(noTtlKey, 2, isTTLValue = false, -1)
+        ),
+        // ensure ttl values were added correctly, and noTtlKey has no ttl values
+        AddData(inputStream, InputEvent(ttlKey, "get_ttl_value_from_state", -1, null)),
+        AddData(inputStream, InputEvent(noTtlKey, "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent(ttlKey, -1, isTTLValue = true, 61000)),
+        AddData(inputStream, InputEvent(ttlKey, "get_values_in_ttl_state", -1, null)),
+        AddData(inputStream, InputEvent(noTtlKey, "get_values_in_ttl_state", -1, null)),

Review Comment:
   Added Stop/StartStream for existing tests to test this scenario. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548791295


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLSuite.scala:
##########
@@ -0,0 +1,579 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{MemoryStream, ValueStateImplWithTTL}
+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,
+    ttl: Duration,
+    eventTime: Timestamp = null,
+    eventTimeTtl: Timestamp = null)
+
+case class OutputEvent(
+    key: String,
+    value: Int,
+    isTTLValue: Boolean,
+    ttlValue: Long)
+
+object TTLInputProcessFunction {
+  def processRow(
+      ttlMode: TTLMode,
+      row: InputEvent,
+      valueState: ValueStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = valueState.getOption()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = valueState.getWithoutEnforcingTTL()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlExpiration = valueState.getTTLValue()
+      if (ttlExpiration.isDefined) {
+        results = OutputEvent(key, -1, isTTLValue = true, ttlExpiration.get) :: results
+      }
+    } else if (row.action == "put") {
+      if (ttlMode == TTLMode.EventTimeTTL() && row.eventTimeTtl != null) {
+        valueState.update(row.value, row.eventTimeTtl.getTime)
+      } else if (ttlMode == TTLMode.EventTimeTTL()) {
+        valueState.update(row.value)
+      } else {
+        valueState.update(row.value, row.ttl)
+      }
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = valueState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+class ValueStateTTLProcessor
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+  with Logging {
+
+  @transient private var _valueState: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueState = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    for (row <- inputRows) {
+      val resultIter = TTLInputProcessFunction.processRow(_ttlMode, row, _valueState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+case class MultipleValueStatesTTLProcessor(
+    ttlKey: String,
+    noTtlKey: String)
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+    with Logging {
+
+  @transient private var _valueStateWithTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _valueStateWithoutTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueStateWithTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _valueStateWithoutTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val state = if (key == ttlKey) {
+      _valueStateWithTTL
+    } else {
+      _valueStateWithoutTTL
+    }
+
+    for (row <- inputRows) {

Review Comment:
   Done



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1552891419


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala:
##########
@@ -925,15 +926,15 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
           hasInitialState, planLater(initialState), planLater(child)
         ) :: Nil
       case logical.TransformWithState(keyDeserializer, valueDeserializer, groupingAttributes,
-          dataAttributes, statefulProcessor, timeoutMode, outputMode, keyEncoder,
+          dataAttributes, statefulProcessor, ttlMode, timeoutMode, outputMode, keyEncoder,
           outputObjAttr, child, hasInitialState,
           initialStateGroupingAttrs, initialStateDataAttrs,
           initialStateDeserializer, initialState) =>
         TransformWithStateExec.generateSparkPlanForBatchQueries(keyDeserializer, valueDeserializer,
-          groupingAttributes, dataAttributes, statefulProcessor, timeoutMode, outputMode,
+          groupingAttributes, dataAttributes, statefulProcessor, ttlMode, timeoutMode, outputMode,
           keyEncoder, outputObjAttr, planLater(child), hasInitialState,
           initialStateGroupingAttrs, initialStateDataAttrs,
-          initialStateDeserializer, planLater(initialState)) :: Nil
+          initialStateDeserializer, planLater (initialState)) :: Nil

Review Comment:
   Removed. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1554114021


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala:
##########
@@ -312,187 +312,109 @@ class ValueStateSuite extends StateVariableSuiteBase {
     }
   }
 
-  Seq(TTLMode.ProcessingTimeTTL(), TTLMode.EventTimeTTL()).foreach { ttlMode =>
-    test(s"test Value state TTL for $ttlMode") {
-      tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
-        val store = provider.getStore(0)
-        val timestampMs = 10
-        val handle = createHandleForTtlMode(ttlMode, store, timestampMs)
-
-        val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-          Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
-        ImplicitGroupingKeyTracker.setImplicitKey("test_key")
-        testState.update("v1")
-        assert(testState.get() === "v1")
-        assert(testState.getWithoutEnforcingTTL().get === "v1")
-
-        var ttlValue = testState.getTTLValue()
-        assert(ttlValue.isEmpty)
-        var ttlStateValueIterator = testState.getValuesInTTLState()
-        assert(ttlStateValueIterator.isEmpty)
-
-        testState.clear()
-        assert(!testState.exists())
-        assert(testState.get() === null)
-
-        val ttlExpirationMs = timestampMs + 60000
-
-        if (ttlMode == TTLMode.ProcessingTimeTTL()) {
-          testState.update("v1", Duration.ofMinutes(1))
-        } else {
-          testState.update("v1", ttlExpirationMs)
-        }
-        assert(testState.get() === "v1")
-        assert(testState.getWithoutEnforcingTTL().get === "v1")
-
-        ttlValue = testState.getTTLValue()
-        assert(ttlValue.isDefined)
-        assert(ttlValue.get === ttlExpirationMs)
-        ttlStateValueIterator = testState.getValuesInTTLState()
-        assert(ttlStateValueIterator.hasNext)
-        assert(ttlStateValueIterator.next() === ttlExpirationMs)
-        assert(ttlStateValueIterator.isEmpty)
-
-        // increment batchProcessingTime, or watermark and ensure expired value is not returned
-        val nextBatchHandle = createHandleForTtlMode(ttlMode, store, ttlExpirationMs)
-
-        val nextBatchTestState: ValueStateImplWithTTL[String] = nextBatchHandle
-          .getValueState[String]("testState", Encoders.STRING)
-          .asInstanceOf[ValueStateImplWithTTL[String]]
-        ImplicitGroupingKeyTracker.setImplicitKey("test_key")
-
-        // ensure get does not return the expired value
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-
-        // ttl value should still exist in state
-        ttlValue = nextBatchTestState.getTTLValue()
-        assert(ttlValue.isDefined)
-        assert(ttlValue.get === ttlExpirationMs)
-        ttlStateValueIterator = nextBatchTestState.getValuesInTTLState()
-        assert(ttlStateValueIterator.hasNext)
-        assert(ttlStateValueIterator.next() === ttlExpirationMs)
-        assert(ttlStateValueIterator.isEmpty)
-
-        // getWithoutTTL should still return the expired value
-        assert(nextBatchTestState.getWithoutEnforcingTTL().get === "v1")
-
-        nextBatchTestState.clear()
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-
-        nextBatchTestState.clear()
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-      }
-    }
-  }
 
-  test("test TTL duration throws error for event time") {
+  test(s"test Value state TTL") {
     tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
       val store = provider.getStore(0)
-      val eventTimeWatermarkMs = 10
+      val timestampMs = 10
       val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
-        TTLMode.EventTimeTTL(), TimeoutMode.NoTimeouts(),
-        eventTimeWatermarkMs = Some(eventTimeWatermarkMs))
+        TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
+        batchTimestampMs = Some(timestampMs))
 
+      val ttlConfig = TTLConfig(ttlDuration = Duration.ofMinutes(1))
       val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-        Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
+        Encoders.STRING, ttlConfig).asInstanceOf[ValueStateImplWithTTL[String]]
       ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+      testState.update("v1")
+      assert(testState.get() === "v1")
+      assert(testState.getWithoutEnforcingTTL().get === "v1")
 
-      val ex = intercept[SparkUnsupportedOperationException] {
-        testState.update("v1", Duration.ofMinutes(1))
-      }
+      var ttlValue = testState.getTTLValue()
+      assert(ttlValue.isEmpty)
+      var ttlStateValueIterator = testState.getValuesInTTLState()
+      assert(ttlStateValueIterator.isEmpty)
 
-      checkError(
-        ex,
-        errorClass = "STATEFUL_PROCESSOR_CANNOT_USE_TTL_DURATION_IN_EVENT_TIME_TTL_MODE",
-        parameters = Map(
-          "operationType" -> "update",
-          "stateName" -> "testState"
-        ),
-        matchPVals = true
-      )
-    }
-  }
+      testState.clear()
+      assert(!testState.exists())
+      assert(testState.get() === null)
 
-  test("test negative TTL duration throws error") {
-    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
-      val store = provider.getStore(0)
-      val batchTimestampMs = 10
-      val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
+      val ttlExpirationMs = timestampMs + 60000
+
+      testState.update("v1")
+      assert(testState.get() === "v1")
+      assert(testState.getWithoutEnforcingTTL().get === "v1")
+
+      ttlValue = testState.getTTLValue()
+      assert(ttlValue.isDefined)
+      assert(ttlValue.get === ttlExpirationMs)
+      ttlStateValueIterator = testState.getValuesInTTLState()
+      assert(ttlStateValueIterator.hasNext)
+      assert(ttlStateValueIterator.next() === ttlExpirationMs)
+      assert(ttlStateValueIterator.isEmpty)
+
+      // increment batchProcessingTime, or watermark and ensure expired value is not returned
+      val nextBatchHandle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
         TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
-        batchTimestampMs = Some(batchTimestampMs))
+        batchTimestampMs = Some(timestampMs))
 
-      val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-        Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
+      val nextBatchTestState: ValueStateImplWithTTL[String] = nextBatchHandle

Review Comment:
   Okay, on this part - we would not allow removing ttlConfig from a state variable, or adding to a variable which did not have ttl previously. (This will be enforced as part of state metadata - cc: @anishshri-db ). 
   
   The reason to disallow is that schema is different between ValueState/ValueStateWithTTL (ttlExpiration column wont exist in first one), and mixing these 2 schemas in StateStore for a column family complicates things significantly. If a user wants to remove ttl, they can simply create a new State Variable to store future state without enforcing ttl. 
   
   Let me know if this makes sense @HeartSaVioR 
   



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1553078246


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala:
##########
@@ -41,6 +41,12 @@ private[sql] trait StatefulProcessorHandle extends Serializable {
    */
   def getValueState[T](stateName: String, valEncoder: Encoder[T]): ValueState[T]
 
+  def getValueState[T](
+      stateName: String,
+      valEncoder: Encoder[T],
+      tTLConfig: TTLConfig): ValueState[T]

Review Comment:
   nit: ttlConfig. I get the reason of doing this as tTL but it's less weird to just use ttl.
   nit2: 2 empty lines (while we are here)



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -89,14 +124,29 @@ class StateTypesEncoder[GK, V](
     val value = rowToObjDeserializer.apply(reusedValRow)
     value
   }
+
+  /**
+   * Decode the ttl information out of Value row. If the ttl has
+   * not been set (-1L specifies no user defined value), the API will
+   * return None.
+   */
+  def decodeTtlExpirationMs(row: UnsafeRow): Option[Long] = {
+    val expirationMs = row.getLong(1)

Review Comment:
   If you feel like there are many rows to be matched, I'm OK with removing this. Actually I see the same necessity for `encodeValue(value: V)` and `encodeValue(value: V, expirationMs: Long)`. They depend on hasTtl flag to work properly. I'm fine to defer the caller to do the right thing.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala:
##########
@@ -312,187 +312,109 @@ class ValueStateSuite extends StateVariableSuiteBase {
     }
   }
 
-  Seq(TTLMode.ProcessingTimeTTL(), TTLMode.EventTimeTTL()).foreach { ttlMode =>
-    test(s"test Value state TTL for $ttlMode") {
-      tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
-        val store = provider.getStore(0)
-        val timestampMs = 10
-        val handle = createHandleForTtlMode(ttlMode, store, timestampMs)
-
-        val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-          Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
-        ImplicitGroupingKeyTracker.setImplicitKey("test_key")
-        testState.update("v1")
-        assert(testState.get() === "v1")
-        assert(testState.getWithoutEnforcingTTL().get === "v1")
-
-        var ttlValue = testState.getTTLValue()
-        assert(ttlValue.isEmpty)
-        var ttlStateValueIterator = testState.getValuesInTTLState()
-        assert(ttlStateValueIterator.isEmpty)
-
-        testState.clear()
-        assert(!testState.exists())
-        assert(testState.get() === null)
-
-        val ttlExpirationMs = timestampMs + 60000
-
-        if (ttlMode == TTLMode.ProcessingTimeTTL()) {
-          testState.update("v1", Duration.ofMinutes(1))
-        } else {
-          testState.update("v1", ttlExpirationMs)
-        }
-        assert(testState.get() === "v1")
-        assert(testState.getWithoutEnforcingTTL().get === "v1")
-
-        ttlValue = testState.getTTLValue()
-        assert(ttlValue.isDefined)
-        assert(ttlValue.get === ttlExpirationMs)
-        ttlStateValueIterator = testState.getValuesInTTLState()
-        assert(ttlStateValueIterator.hasNext)
-        assert(ttlStateValueIterator.next() === ttlExpirationMs)
-        assert(ttlStateValueIterator.isEmpty)
-
-        // increment batchProcessingTime, or watermark and ensure expired value is not returned
-        val nextBatchHandle = createHandleForTtlMode(ttlMode, store, ttlExpirationMs)
-
-        val nextBatchTestState: ValueStateImplWithTTL[String] = nextBatchHandle
-          .getValueState[String]("testState", Encoders.STRING)
-          .asInstanceOf[ValueStateImplWithTTL[String]]
-        ImplicitGroupingKeyTracker.setImplicitKey("test_key")
-
-        // ensure get does not return the expired value
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-
-        // ttl value should still exist in state
-        ttlValue = nextBatchTestState.getTTLValue()
-        assert(ttlValue.isDefined)
-        assert(ttlValue.get === ttlExpirationMs)
-        ttlStateValueIterator = nextBatchTestState.getValuesInTTLState()
-        assert(ttlStateValueIterator.hasNext)
-        assert(ttlStateValueIterator.next() === ttlExpirationMs)
-        assert(ttlStateValueIterator.isEmpty)
-
-        // getWithoutTTL should still return the expired value
-        assert(nextBatchTestState.getWithoutEnforcingTTL().get === "v1")
-
-        nextBatchTestState.clear()
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-
-        nextBatchTestState.clear()
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-      }
-    }
-  }
 
-  test("test TTL duration throws error for event time") {
+  test(s"test Value state TTL") {
     tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
       val store = provider.getStore(0)
-      val eventTimeWatermarkMs = 10
+      val timestampMs = 10
       val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
-        TTLMode.EventTimeTTL(), TimeoutMode.NoTimeouts(),
-        eventTimeWatermarkMs = Some(eventTimeWatermarkMs))
+        TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
+        batchTimestampMs = Some(timestampMs))
 
+      val ttlConfig = TTLConfig(ttlDuration = Duration.ofMinutes(1))
       val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-        Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
+        Encoders.STRING, ttlConfig).asInstanceOf[ValueStateImplWithTTL[String]]
       ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+      testState.update("v1")

Review Comment:
   Wait, why this is different from the below update after clearing state? I'd expect TTL to be set when this update has happened.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala:
##########
@@ -227,19 +218,17 @@ class StatefulProcessorHandleSuite extends StateVariableSuiteBase {
     }
   }
 
-  Seq("ProcessingTime", "EventTime").foreach { ttlMode =>
-    test(s"ttl States are populated for ttlMode=$ttlMode") {
-      tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
-        val store = provider.getStore(0)
-        val handle = new StatefulProcessorHandleImpl(store,
-          UUID.randomUUID(), keyExprEncoder, getTtlMode(ttlMode), TimeoutMode.NoTimeouts(),
-          batchTimestampMs = Some(10), eventTimeWatermarkMs = Some(100))
+  test(s"ttl States are populated for ttlMode=ProcessingTime") {
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
+      val store = provider.getStore(0)
+      val handle = new StatefulProcessorHandleImpl(store,
+        UUID.randomUUID(), keyExprEncoder, TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
+        batchTimestampMs = Some(10))
 
-        val valueState = handle.getValueState("testState", Encoders.STRING)
+      val valueState = handle.getValueState("testState", Encoders.STRING)

Review Comment:
   Just wanted to confirm - here we don't add configuration of TTL for this state. Will this state still be a part of TTL states?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala:
##########
@@ -312,187 +312,109 @@ class ValueStateSuite extends StateVariableSuiteBase {
     }
   }
 
-  Seq(TTLMode.ProcessingTimeTTL(), TTLMode.EventTimeTTL()).foreach { ttlMode =>
-    test(s"test Value state TTL for $ttlMode") {
-      tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
-        val store = provider.getStore(0)
-        val timestampMs = 10
-        val handle = createHandleForTtlMode(ttlMode, store, timestampMs)
-
-        val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-          Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
-        ImplicitGroupingKeyTracker.setImplicitKey("test_key")
-        testState.update("v1")
-        assert(testState.get() === "v1")
-        assert(testState.getWithoutEnforcingTTL().get === "v1")
-
-        var ttlValue = testState.getTTLValue()
-        assert(ttlValue.isEmpty)
-        var ttlStateValueIterator = testState.getValuesInTTLState()
-        assert(ttlStateValueIterator.isEmpty)
-
-        testState.clear()
-        assert(!testState.exists())
-        assert(testState.get() === null)
-
-        val ttlExpirationMs = timestampMs + 60000
-
-        if (ttlMode == TTLMode.ProcessingTimeTTL()) {
-          testState.update("v1", Duration.ofMinutes(1))
-        } else {
-          testState.update("v1", ttlExpirationMs)
-        }
-        assert(testState.get() === "v1")
-        assert(testState.getWithoutEnforcingTTL().get === "v1")
-
-        ttlValue = testState.getTTLValue()
-        assert(ttlValue.isDefined)
-        assert(ttlValue.get === ttlExpirationMs)
-        ttlStateValueIterator = testState.getValuesInTTLState()
-        assert(ttlStateValueIterator.hasNext)
-        assert(ttlStateValueIterator.next() === ttlExpirationMs)
-        assert(ttlStateValueIterator.isEmpty)
-
-        // increment batchProcessingTime, or watermark and ensure expired value is not returned
-        val nextBatchHandle = createHandleForTtlMode(ttlMode, store, ttlExpirationMs)
-
-        val nextBatchTestState: ValueStateImplWithTTL[String] = nextBatchHandle
-          .getValueState[String]("testState", Encoders.STRING)
-          .asInstanceOf[ValueStateImplWithTTL[String]]
-        ImplicitGroupingKeyTracker.setImplicitKey("test_key")
-
-        // ensure get does not return the expired value
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-
-        // ttl value should still exist in state
-        ttlValue = nextBatchTestState.getTTLValue()
-        assert(ttlValue.isDefined)
-        assert(ttlValue.get === ttlExpirationMs)
-        ttlStateValueIterator = nextBatchTestState.getValuesInTTLState()
-        assert(ttlStateValueIterator.hasNext)
-        assert(ttlStateValueIterator.next() === ttlExpirationMs)
-        assert(ttlStateValueIterator.isEmpty)
-
-        // getWithoutTTL should still return the expired value
-        assert(nextBatchTestState.getWithoutEnforcingTTL().get === "v1")
-
-        nextBatchTestState.clear()
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-
-        nextBatchTestState.clear()
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-      }
-    }
-  }
 
-  test("test TTL duration throws error for event time") {
+  test(s"test Value state TTL") {
     tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
       val store = provider.getStore(0)
-      val eventTimeWatermarkMs = 10
+      val timestampMs = 10
       val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
-        TTLMode.EventTimeTTL(), TimeoutMode.NoTimeouts(),
-        eventTimeWatermarkMs = Some(eventTimeWatermarkMs))
+        TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
+        batchTimestampMs = Some(timestampMs))
 
+      val ttlConfig = TTLConfig(ttlDuration = Duration.ofMinutes(1))
       val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-        Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
+        Encoders.STRING, ttlConfig).asInstanceOf[ValueStateImplWithTTL[String]]
       ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+      testState.update("v1")
+      assert(testState.get() === "v1")
+      assert(testState.getWithoutEnforcingTTL().get === "v1")
 
-      val ex = intercept[SparkUnsupportedOperationException] {
-        testState.update("v1", Duration.ofMinutes(1))
-      }
+      var ttlValue = testState.getTTLValue()
+      assert(ttlValue.isEmpty)
+      var ttlStateValueIterator = testState.getValuesInTTLState()
+      assert(ttlStateValueIterator.isEmpty)
 
-      checkError(
-        ex,
-        errorClass = "STATEFUL_PROCESSOR_CANNOT_USE_TTL_DURATION_IN_EVENT_TIME_TTL_MODE",
-        parameters = Map(
-          "operationType" -> "update",
-          "stateName" -> "testState"
-        ),
-        matchPVals = true
-      )
-    }
-  }
+      testState.clear()
+      assert(!testState.exists())
+      assert(testState.get() === null)
 
-  test("test negative TTL duration throws error") {
-    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
-      val store = provider.getStore(0)
-      val batchTimestampMs = 10
-      val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
+      val ttlExpirationMs = timestampMs + 60000
+
+      testState.update("v1")
+      assert(testState.get() === "v1")
+      assert(testState.getWithoutEnforcingTTL().get === "v1")
+
+      ttlValue = testState.getTTLValue()
+      assert(ttlValue.isDefined)
+      assert(ttlValue.get === ttlExpirationMs)
+      ttlStateValueIterator = testState.getValuesInTTLState()
+      assert(ttlStateValueIterator.hasNext)
+      assert(ttlStateValueIterator.next() === ttlExpirationMs)
+      assert(ttlStateValueIterator.isEmpty)
+
+      // increment batchProcessingTime, or watermark and ensure expired value is not returned
+      val nextBatchHandle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
         TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
-        batchTimestampMs = Some(batchTimestampMs))
+        batchTimestampMs = Some(timestampMs))
 
-      val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-        Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
+      val nextBatchTestState: ValueStateImplWithTTL[String] = nextBatchHandle
+        .getValueState[String]("testState", Encoders.STRING)
+        .asInstanceOf[ValueStateImplWithTTL[String]]
       ImplicitGroupingKeyTracker.setImplicitKey("test_key")
 
-      val ex = intercept[SparkUnsupportedOperationException] {
-        testState.update("v1", Duration.ofMinutes(-1))
-      }
-
-      checkError(
-        ex,
-        errorClass = "STATEFUL_PROCESSOR_TTL_VALUE_CANNOT_BE_NEGATIVE",
-        parameters = Map(
-          "operationType" -> "update",
-          "stateName" -> "testState"
-        ),
-        matchPVals = true
-      )
+      // ensure get does not return the expired value
+      assert(!nextBatchTestState.exists())
+      assert(nextBatchTestState.get() === null)
+
+      // ttl value should still exist in state
+      ttlValue = nextBatchTestState.getTTLValue()
+      assert(ttlValue.isDefined)
+      assert(ttlValue.get === ttlExpirationMs)
+      ttlStateValueIterator = nextBatchTestState.getValuesInTTLState()
+      assert(ttlStateValueIterator.hasNext)
+      assert(ttlStateValueIterator.next() === ttlExpirationMs)
+      assert(ttlStateValueIterator.isEmpty)
+
+      // getWithoutTTL should still return the expired value
+      assert(nextBatchTestState.getWithoutEnforcingTTL().get === "v1")
+
+      nextBatchTestState.clear()
+      assert(!nextBatchTestState.exists())
+      assert(nextBatchTestState.get() === null)
+
+      nextBatchTestState.clear()

Review Comment:
   nit: redundant? is there an intention to do this twice?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala:
##########
@@ -312,187 +312,109 @@ class ValueStateSuite extends StateVariableSuiteBase {
     }
   }
 
-  Seq(TTLMode.ProcessingTimeTTL(), TTLMode.EventTimeTTL()).foreach { ttlMode =>
-    test(s"test Value state TTL for $ttlMode") {
-      tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
-        val store = provider.getStore(0)
-        val timestampMs = 10
-        val handle = createHandleForTtlMode(ttlMode, store, timestampMs)
-
-        val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-          Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
-        ImplicitGroupingKeyTracker.setImplicitKey("test_key")
-        testState.update("v1")
-        assert(testState.get() === "v1")
-        assert(testState.getWithoutEnforcingTTL().get === "v1")
-
-        var ttlValue = testState.getTTLValue()
-        assert(ttlValue.isEmpty)
-        var ttlStateValueIterator = testState.getValuesInTTLState()
-        assert(ttlStateValueIterator.isEmpty)
-
-        testState.clear()
-        assert(!testState.exists())
-        assert(testState.get() === null)
-
-        val ttlExpirationMs = timestampMs + 60000
-
-        if (ttlMode == TTLMode.ProcessingTimeTTL()) {
-          testState.update("v1", Duration.ofMinutes(1))
-        } else {
-          testState.update("v1", ttlExpirationMs)
-        }
-        assert(testState.get() === "v1")
-        assert(testState.getWithoutEnforcingTTL().get === "v1")
-
-        ttlValue = testState.getTTLValue()
-        assert(ttlValue.isDefined)
-        assert(ttlValue.get === ttlExpirationMs)
-        ttlStateValueIterator = testState.getValuesInTTLState()
-        assert(ttlStateValueIterator.hasNext)
-        assert(ttlStateValueIterator.next() === ttlExpirationMs)
-        assert(ttlStateValueIterator.isEmpty)
-
-        // increment batchProcessingTime, or watermark and ensure expired value is not returned
-        val nextBatchHandle = createHandleForTtlMode(ttlMode, store, ttlExpirationMs)
-
-        val nextBatchTestState: ValueStateImplWithTTL[String] = nextBatchHandle
-          .getValueState[String]("testState", Encoders.STRING)
-          .asInstanceOf[ValueStateImplWithTTL[String]]
-        ImplicitGroupingKeyTracker.setImplicitKey("test_key")
-
-        // ensure get does not return the expired value
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-
-        // ttl value should still exist in state
-        ttlValue = nextBatchTestState.getTTLValue()
-        assert(ttlValue.isDefined)
-        assert(ttlValue.get === ttlExpirationMs)
-        ttlStateValueIterator = nextBatchTestState.getValuesInTTLState()
-        assert(ttlStateValueIterator.hasNext)
-        assert(ttlStateValueIterator.next() === ttlExpirationMs)
-        assert(ttlStateValueIterator.isEmpty)
-
-        // getWithoutTTL should still return the expired value
-        assert(nextBatchTestState.getWithoutEnforcingTTL().get === "v1")
-
-        nextBatchTestState.clear()
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-
-        nextBatchTestState.clear()
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-      }
-    }
-  }
 
-  test("test TTL duration throws error for event time") {
+  test(s"test Value state TTL") {
     tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
       val store = provider.getStore(0)
-      val eventTimeWatermarkMs = 10
+      val timestampMs = 10
       val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
-        TTLMode.EventTimeTTL(), TimeoutMode.NoTimeouts(),
-        eventTimeWatermarkMs = Some(eventTimeWatermarkMs))
+        TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
+        batchTimestampMs = Some(timestampMs))
 
+      val ttlConfig = TTLConfig(ttlDuration = Duration.ofMinutes(1))
       val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-        Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
+        Encoders.STRING, ttlConfig).asInstanceOf[ValueStateImplWithTTL[String]]
       ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+      testState.update("v1")
+      assert(testState.get() === "v1")
+      assert(testState.getWithoutEnforcingTTL().get === "v1")
 
-      val ex = intercept[SparkUnsupportedOperationException] {
-        testState.update("v1", Duration.ofMinutes(1))
-      }
+      var ttlValue = testState.getTTLValue()
+      assert(ttlValue.isEmpty)
+      var ttlStateValueIterator = testState.getValuesInTTLState()
+      assert(ttlStateValueIterator.isEmpty)
 
-      checkError(
-        ex,
-        errorClass = "STATEFUL_PROCESSOR_CANNOT_USE_TTL_DURATION_IN_EVENT_TIME_TTL_MODE",
-        parameters = Map(
-          "operationType" -> "update",
-          "stateName" -> "testState"
-        ),
-        matchPVals = true
-      )
-    }
-  }
+      testState.clear()
+      assert(!testState.exists())
+      assert(testState.get() === null)
 
-  test("test negative TTL duration throws error") {
-    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
-      val store = provider.getStore(0)
-      val batchTimestampMs = 10
-      val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
+      val ttlExpirationMs = timestampMs + 60000
+
+      testState.update("v1")
+      assert(testState.get() === "v1")
+      assert(testState.getWithoutEnforcingTTL().get === "v1")
+
+      ttlValue = testState.getTTLValue()
+      assert(ttlValue.isDefined)
+      assert(ttlValue.get === ttlExpirationMs)
+      ttlStateValueIterator = testState.getValuesInTTLState()
+      assert(ttlStateValueIterator.hasNext)
+      assert(ttlStateValueIterator.next() === ttlExpirationMs)
+      assert(ttlStateValueIterator.isEmpty)
+
+      // increment batchProcessingTime, or watermark and ensure expired value is not returned
+      val nextBatchHandle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
         TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
-        batchTimestampMs = Some(batchTimestampMs))
+        batchTimestampMs = Some(timestampMs))
 
-      val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-        Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
+      val nextBatchTestState: ValueStateImplWithTTL[String] = nextBatchHandle

Review Comment:
   This seems to test the case where a state is defined with TTL once, and definition of TTL is removed after restarting the query. It's great we have a test to make sure flipping this does not bring up correctness issue.
   
   While we are here, we do not support opposite direction, right? Defining the state without TTL in the first query run, and defining TTL to that state during the query restart. I expect  schema compatibility error, but wanted to double confirm. If I'm missing something and we support the case, let's have a test for that as well.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1540204564


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -103,21 +110,26 @@ class StatefulProcessorHandleImpl(
 
   private var currState: StatefulProcessorHandleState = CREATED
 
-  private def verify(condition: => Boolean, msg: String): Unit = {

Review Comment:
   Thx for removing this 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547142763


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -185,6 +202,16 @@ class StatefulProcessorHandleImpl(
     timerState.listTimers()
   }
 
+  /**
+   * Performs the user state cleanup based on assigned TTl values. Any state
+   * which is expired will be cleaned up from StateStore.
+   */
+  def doTtlCleanup(): Unit = {
+    ttlStates.forEach { s =>

Review Comment:
   correct. All types should have a associated ttlState. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548593408


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala:
##########
@@ -303,6 +311,244 @@ class ValueStateSuite extends StateVariableSuiteBase {
       assert(testState.get() === null)
     }
   }
+
+  test("test Value state TTL for processing time") {
+    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
+      val store = provider.getStore(0)
+      val batchTimestampMs = 10
+      val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
+        Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
+        TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
+        batchTimestampMs = Some(batchTimestampMs))
+
+      val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
+        Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
+      ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+      testState.update("v1")
+      assert(testState.get() === "v1")
+      assert(testState.getWithoutEnforcingTTL().get === "v1")
+
+      var ttlValue = testState.getTTLValue()
+      assert(ttlValue.isEmpty)
+      var ttlStateValueIterator = testState.getValuesInTTLState()
+      assert(ttlStateValueIterator.isEmpty)
+
+      testState.clear()
+      assert(!testState.exists())
+      assert(testState.get() === null)
+
+      testState.update("v1", Duration.ofMinutes(1))
+      assert(testState.get() === "v1")
+      assert(testState.getWithoutEnforcingTTL().get === "v1")
+
+      val expectedTtlExpirationMs = batchTimestampMs + 60000
+      ttlValue = testState.getTTLValue()
+      assert(ttlValue.isDefined)
+      assert(ttlValue.get === expectedTtlExpirationMs)
+      ttlStateValueIterator = testState.getValuesInTTLState()
+      assert(ttlStateValueIterator.hasNext)
+      assert(ttlStateValueIterator.next() === expectedTtlExpirationMs)
+      assert(ttlStateValueIterator.isEmpty)
+
+      // increment batchProcessingTime and ensure expired value is not returned
+      val nextBatchHandle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
+        Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
+        TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
+        batchTimestampMs = Some(expectedTtlExpirationMs))
+
+      val nextBatchTestState: ValueStateImplWithTTL[String] = nextBatchHandle
+        .getValueState[String]("testState", Encoders.STRING)
+        .asInstanceOf[ValueStateImplWithTTL[String]]
+      ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+
+      // ensure get does not return the expired value
+      assert(!nextBatchTestState.exists())
+      assert(nextBatchTestState.get() === null)
+
+      // ttl value should still exist in state
+      ttlValue = nextBatchTestState.getTTLValue()
+      assert(ttlValue.isDefined)
+      assert(ttlValue.get === expectedTtlExpirationMs)
+      ttlStateValueIterator = nextBatchTestState.getValuesInTTLState()
+      assert(ttlStateValueIterator.hasNext)
+      assert(ttlStateValueIterator.next() === expectedTtlExpirationMs)
+      assert(ttlStateValueIterator.isEmpty)
+
+      // getWithoutTTL should still return the expired value
+      assert(nextBatchTestState.getWithoutEnforcingTTL().get === "v1")
+
+      nextBatchTestState.clear()
+      assert(!nextBatchTestState.exists())
+      assert(nextBatchTestState.get() === null)
+
+      nextBatchTestState.clear()
+      assert(!nextBatchTestState.exists())
+      assert(nextBatchTestState.get() === null)
+    }
+  }
+
+  test("test Value state TTL for event time") {

Review Comment:
   Could we combine with test above ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547140114


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -77,14 +78,23 @@ class StatefulProcessorHandleImpl(
     store: StateStore,
     runId: UUID,
     keyEncoder: ExpressionEncoder[Any],
+    ttlMode: TTLMode,
     timeoutMode: TimeoutMode,
-    isStreaming: Boolean = true)
+    isStreaming: Boolean = true,
+    batchTimestampMs: Option[Long] = None,
+    eventTimeWatermarkMs: Option[Long] = None)
   extends StatefulProcessorHandle with Logging {
   import StatefulProcessorHandleState._
 
+  /**
+   * Stores all the active ttl states, and is used to cleanup expired values
+   * in [[doTtlCleanup()]] function.
+   */
+  private val ttlStates: util.List[TTLState] = new util.ArrayList[TTLState]()

Review Comment:
   I dont think there is a usecase to access ttl state by name. We can modify it in future if needed. 
   
   I dont think this needs to be thread safe, as all access happens only in Task thread. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548243931


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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 java.time.Duration
+
+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.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTimestampMs - processing timestamp of the current batch.
+ * @param eventTimeWatermarkMs - event time watermark for streaming query
+ *                               (same as watermark for state eviction)
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends ValueState[S] with Logging with StateVariableWithTTLSupport {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+  private[sql] var ttlState: SingleKeyTTLStateImpl = _
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+
+    ttlState = new SingleKeyTTLStateImpl(ttlMode, stateName, store,
+      batchTimestampMs, eventTimeWatermarkMs)
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    get() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    Option(get())
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!isExpired(retRow)) {
+        resState
+      } else {
+        null.asInstanceOf[S]
+      }
+    } else {
+      null.asInstanceOf[S]
+    }
+  }
+
+  /** Function to update and overwrite state associated with given key */
+  override def update(
+      newState: S,
+      ttlDuration: Duration = Duration.ZERO): Unit = {
+
+    if (ttlMode == TTLMode.EventTimeTTL() && ttlDuration != Duration.ZERO) {
+      throw StateStoreErrors.cannotProvideTTLDurationForEventTimeTTLMode(
+        "update", stateName)
+    }
+
+    if (ttlDuration != null && ttlDuration.isNegative) {
+      throw StateStoreErrors.ttlCannotBeNegative(
+        "update", stateName)
+    }
+
+    val expirationTimeInMs =
+      if (ttlDuration != null && ttlDuration != Duration.ZERO) {
+        StateTTL.calculateExpirationTimeForDuration(
+          ttlMode, ttlDuration, batchTimestampMs, eventTimeWatermarkMs)
+      } else {
+        -1
+      }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  override def update(
+      newState: S,
+      expirationTimeInMs: Long): Unit = {
+
+    if (expirationTimeInMs < 0) {
+      throw StateStoreErrors.ttlCannotBeNegative(
+        "update", stateName)
+    }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  private def doUpdate(newState: S,
+      expirationTimeInMs: Long): Unit = {
+    val encodedValue = stateTypesEncoder.encodeValue(newState, expirationTimeInMs)
+
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    store.put(stateTypesEncoder.encodeSerializedGroupingKey(serializedGroupingKey),
+      encodedValue, stateName)
+
+    if (expirationTimeInMs != -1) {
+      ttlState.upsertTTLForStateKey(expirationTimeInMs, serializedGroupingKey)
+    }
+  }
+
+  /** Function to remove state for given key */
+  override def clear(): Unit = {
+    store.remove(stateTypesEncoder.encodeGroupingKey(), stateName)
+  }
+
+  def clearIfExpired(groupingKey: Array[Byte]): Unit = {
+    val encodedGroupingKey = stateTypesEncoder.encodeSerializedGroupingKey(groupingKey)
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      if (isExpired(retRow)) {
+        store.remove(encodedGroupingKey, stateName)
+      }
+    }
+  }
+
+  private def isExpired(valueRow: UnsafeRow): Boolean = {
+    val expirationMs = stateTypesEncoder.decodeTtlExpirationMs(valueRow)
+    val isExpired = expirationMs.map(
+      StateTTL.isExpired(ttlMode, _, batchTimestampMs, eventTimeWatermarkMs))
+
+    isExpired.isDefined && isExpired.get
+  }
+
+  /*
+   * 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(): Option[S] = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+      Some(resState)
+    } else {
+      None
+    }
+  }
+
+  /**
+   * Read the ttl value associated with the grouping key.
+   */
+  private[sql] def getTTLValue(): Option[Long] = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      stateTypesEncoder.decodeTtlExpirationMs(retRow)
+    } else {
+      None
+    }
+  }
+
+  /**
+   * Get all ttl values stored in ttl state for current implicit
+   * grouping key.
+   */
+  private[sql] def getValuesInTTLState(): Iterator[Long] = {
+    val ttlIterator = ttlState.iterator()
+    val implicitGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    var nextValue: Option[Long] = None
+
+    new Iterator[Long] {
+      override def hasNext: Boolean = {
+        while (nextValue.isEmpty && ttlIterator.hasNext) {
+          val nextTtlValue = ttlIterator.next()
+          val groupingKey = nextTtlValue.groupingKey
+
+          if (groupingKey sameElements implicitGroupingKey) {
+            nextValue = Some(nextTtlValue.expirationMs)
+          }
+        }
+
+        nextValue.isDefined
+      }
+
+      override def next(): Long = {
+        val result = nextValue.get
+        nextValue = None
+

Review Comment:
   Done



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on PR #45674:
URL: https://github.com/apache/spark/pull/45674#issuecomment-2030261383

   Should we update the PR description for this - ```event time ttl is calculated by adding ttlDuration to watermarkForEviction.``` ?


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548402002


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val TTL_KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val TTL_VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+class SingleKeyTTLStateImpl(
+    ttlMode: TTLMode,
+    stateName: String,
+    store: StateStore,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends TTLState {
+
+  import org.apache.spark.sql.execution.streaming.StateTTLSchema._
+
+  private val ttlColumnFamilyName = s"_ttl_$stateName"
+  private val ttlKeyEncoder = UnsafeProjection.create(TTL_KEY_ROW_SCHEMA)
+  private var state: StateVariableWithTTLSupport = _
+
+  // empty row used for values
+  private val EMPTY_ROW =
+    UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
+
+  store.createColFamilyIfAbsent(ttlColumnFamilyName, TTL_KEY_ROW_SCHEMA, TTL_VALUE_ROW_SCHEMA,
+    RangeKeyScanStateEncoderSpec(TTL_KEY_ROW_SCHEMA, 1), isInternal = true)
+
+  def upsertTTLForStateKey(
+      expirationMs: Long,
+      groupingKey: Array[Byte]): Unit = {
+    val encodedTtlKey = ttlKeyEncoder(InternalRow(expirationMs, groupingKey))
+    store.put(encodedTtlKey, EMPTY_ROW, ttlColumnFamilyName)
+  }
+
+  override def clearExpiredState(): Unit = {
+    val iterator = store.iterator(ttlColumnFamilyName)
+
+    iterator.takeWhile { kv =>
+      val expirationMs = kv.key.getLong(0)
+      StateTTL.isExpired(ttlMode, expirationMs,
+        batchTimestampMs, eventTimeWatermarkMs)
+    }.foreach { kv =>
+      val groupingKey = kv.key.getBinary(1)
+      state.clearIfExpired(groupingKey)
+      store.remove(kv.key, ttlColumnFamilyName)
+    }
+  }
+
+  private[sql] def setStateVariable(
+      state: StateVariableWithTTLSupport): Unit = {
+    this.state = state
+  }
+
+  private[sql] def iterator(): Iterator[SingleKeyTTLRow] = {
+    val ttlIterator = store.iterator(ttlColumnFamilyName)
+
+    new Iterator[SingleKeyTTLRow] {
+      override def hasNext: Boolean = ttlIterator.hasNext
+
+      override def next(): SingleKeyTTLRow = {
+        val kv = ttlIterator.next()
+        SingleKeyTTLRow(
+          expirationMs = kv.key.getLong(0),
+          groupingKey = kv.key.getBinary(1)
+        )
+      }
+    }
+  }
+}
+
+/**
+ * Helper methods for user State TTL.
+ */
+object StateTTL {
+  def calculateExpirationTimeForDuration(
+      ttlMode: TTLMode,
+      ttlDuration: Duration,
+      batchTimestampMs: Option[Long],
+      eventTimeWatermarkMs: Option[Long]): Long = {
+    if (ttlMode == TTLMode.ProcessingTimeTTL()) {
+      batchTimestampMs.get + ttlDuration.toMillis
+    } else if (ttlMode == TTLMode.EventTimeTTL()) {
+      eventTimeWatermarkMs.get + ttlDuration.toMillis
+    } else {
+      throw new IllegalStateException(s"cannot calculate expiration time for" +

Review Comment:
   Could we throw a NERF error if required ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548807708


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val TTL_KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val TTL_VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+abstract class SingleKeyTTLStateImpl(
+    stateName: String,
+    store: StateStore,
+    batchTtlExpirationMs: Long)
+  extends TTLState {
+
+  import org.apache.spark.sql.execution.streaming.StateTTLSchema._
+
+  private val ttlColumnFamilyName = s"_ttl_$stateName"
+  private val ttlKeyEncoder = UnsafeProjection.create(TTL_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_KEY_ROW_SCHEMA, TTL_VALUE_ROW_SCHEMA,
+    RangeKeyScanStateEncoderSpec(TTL_KEY_ROW_SCHEMA, 1), isInternal = true)
+
+  def upsertTTLForStateKey(
+      expirationMs: Long,
+      groupingKey: Array[Byte]): Unit = {
+    val encodedTtlKey = ttlKeyEncoder(InternalRow(expirationMs, groupingKey))
+    store.put(encodedTtlKey, EMPTY_ROW, ttlColumnFamilyName)
+  }
+
+  override def clearExpiredState(): Unit = {

Review Comment:
   Maybe add a small comment here ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1546668926


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -103,22 +113,29 @@ class StatefulProcessorHandleImpl(
 
   private var currState: StatefulProcessorHandleState = CREATED
 
-  private def verify(condition: => Boolean, msg: String): Unit = {
-    if (!condition) {
-      throw new IllegalStateException(msg)
-    }
-  }
-
   def setHandleState(newState: StatefulProcessorHandleState): Unit = {
     currState = newState
   }
 
   def getHandleState: StatefulProcessorHandleState = currState
 
-  override def getValueState[T](stateName: String, valEncoder: Encoder[T]): ValueState[T] = {
+  override def getValueState[T](
+      stateName: String,
+      valEncoder: Encoder[T]): ValueState[T] = {
     verifyStateVarOperations("get_value_state")
-    val resultState = new ValueStateImpl[T](store, stateName, keyEncoder, valEncoder)
-    resultState
+
+    if (ttlMode == TTLMode.NoTTL()) {
+      new ValueStateImpl[T](store, stateName, keyEncoder, valEncoder)
+    } else {
+      val valueStateWithTTL = new ValueStateImplWithTTL[T](store, stateName,
+        keyEncoder, valEncoder, ttlMode, batchTimestampMs, eventTimeWatermarkMs)
+
+      val ttlState = valueStateWithTTL.ttlState

Review Comment:
   Not sure what we want to do here ? could we just add the `valueStateWithTTL` instance to the list of `ttlStates` ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1546669757


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -77,14 +78,23 @@ class StatefulProcessorHandleImpl(
     store: StateStore,
     runId: UUID,
     keyEncoder: ExpressionEncoder[Any],
+    ttlMode: TTLMode,
     timeoutMode: TimeoutMode,
-    isStreaming: Boolean = true)
+    isStreaming: Boolean = true,
+    batchTimestampMs: Option[Long] = None,
+    eventTimeWatermarkMs: Option[Long] = None)
   extends StatefulProcessorHandle with Logging {
   import StatefulProcessorHandleState._
 
+  /**
+   * Stores all the active ttl states, and is used to cleanup expired values
+   * in [[doTtlCleanup()]] function.
+   */
+  private val ttlStates: util.List[TTLState] = new util.ArrayList[TTLState]()

Review Comment:
   Probably not - but does this need to be thread-safe ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548717721


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLSuite.scala:
##########
@@ -0,0 +1,579 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{MemoryStream, ValueStateImplWithTTL}
+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,
+    ttl: Duration,
+    eventTime: Timestamp = null,
+    eventTimeTtl: Timestamp = null)
+
+case class OutputEvent(
+    key: String,
+    value: Int,
+    isTTLValue: Boolean,
+    ttlValue: Long)
+
+object TTLInputProcessFunction {
+  def processRow(
+      ttlMode: TTLMode,
+      row: InputEvent,
+      valueState: ValueStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = valueState.getOption()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = valueState.getWithoutEnforcingTTL()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlExpiration = valueState.getTTLValue()
+      if (ttlExpiration.isDefined) {
+        results = OutputEvent(key, -1, isTTLValue = true, ttlExpiration.get) :: results
+      }
+    } else if (row.action == "put") {
+      if (ttlMode == TTLMode.EventTimeTTL() && row.eventTimeTtl != null) {
+        valueState.update(row.value, row.eventTimeTtl.getTime)
+      } else if (ttlMode == TTLMode.EventTimeTTL()) {
+        valueState.update(row.value)
+      } else {
+        valueState.update(row.value, row.ttl)
+      }
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = valueState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+class ValueStateTTLProcessor
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+  with Logging {
+
+  @transient private var _valueState: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueState = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    for (row <- inputRows) {

Review Comment:
   Could we do `foreach` here too ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1552897542


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -65,22 +74,48 @@ class StateTypesEncoder[GK, V](
   // TODO: validate places that are trying to encode the key and check if we can eliminate/
   // add caching for some of these calls.
   def encodeGroupingKey(): UnsafeRow = {
+    val keyRow = keyProjection(InternalRow(serializeGroupingKey()))
+    keyRow
+  }
+
+  /**
+   * Encodes the provided grouping key into Spark UnsafeRow.
+   *
+   * @param groupingKeyBytes serialized grouping key byte array
+   * @return encoded UnsafeRow
+   */
+  def encodeSerializedGroupingKey(groupingKeyBytes: Array[Byte]): UnsafeRow = {
+    val keyRow = keyProjection(InternalRow(groupingKeyBytes))
+    keyRow
+  }
+
+  def serializeGroupingKey(): Array[Byte] = {
     val keyOption = ImplicitGroupingKeyTracker.getImplicitKeyOption
     if (keyOption.isEmpty) {
       throw StateStoreErrors.implicitKeyNotFound(stateName)
     }
-
     val groupingKey = keyOption.get.asInstanceOf[GK]
-    val keyByteArr = keySerializer.apply(groupingKey).asInstanceOf[UnsafeRow].getBytes()
-    val keyRow = keyProjection(InternalRow(keyByteArr))
-    keyRow
+    keySerializer.apply(groupingKey).asInstanceOf[UnsafeRow].getBytes()
   }
 
+  /**
+   * Encode the specified value in Spark UnsafeRow with no ttl.
+   * The ttl expiration will be set to -1, specifying no TTL.
+   */
   def encodeValue(value: V): UnsafeRow = {

Review Comment:
   The comment is outdated. I have removed it from here. Also, we do not need a default value for `expirationMs: Long` anymore. Its the whole reason I split it into 2 methods. Not sure why Scala compiler did not catch this. I have removed the default value from second parameter. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1546680781


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+class SingleKeyTTLStateImpl(
+    ttlMode: TTLMode,
+    stateName: String,
+    store: StateStore,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends TTLState
+  with Logging {

Review Comment:
   Lets move to line aboe ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1546693200


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -185,6 +202,16 @@ class StatefulProcessorHandleImpl(
     timerState.listTimers()
   }
 
+  /**
+   * Performs the user state cleanup based on assigned TTl values. Any state
+   * which is expired will be cleaned up from StateStore.
+   */
+  def doTtlCleanup(): Unit = {
+    ttlStates.forEach { s =>

Review Comment:
   Just curious - this would be used to perform eviction for all types in the future - correct ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1551021910


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -65,22 +74,48 @@ class StateTypesEncoder[GK, V](
   // TODO: validate places that are trying to encode the key and check if we can eliminate/
   // add caching for some of these calls.
   def encodeGroupingKey(): UnsafeRow = {
+    val keyRow = keyProjection(InternalRow(serializeGroupingKey()))
+    keyRow
+  }
+
+  /**
+   * Encodes the provided grouping key into Spark UnsafeRow.
+   *
+   * @param groupingKeyBytes serialized grouping key byte array
+   * @return encoded UnsafeRow
+   */
+  def encodeSerializedGroupingKey(groupingKeyBytes: Array[Byte]): UnsafeRow = {
+    val keyRow = keyProjection(InternalRow(groupingKeyBytes))
+    keyRow
+  }
+
+  def serializeGroupingKey(): Array[Byte] = {
     val keyOption = ImplicitGroupingKeyTracker.getImplicitKeyOption
     if (keyOption.isEmpty) {
       throw StateStoreErrors.implicitKeyNotFound(stateName)
     }
-
     val groupingKey = keyOption.get.asInstanceOf[GK]
-    val keyByteArr = keySerializer.apply(groupingKey).asInstanceOf[UnsafeRow].getBytes()
-    val keyRow = keyProjection(InternalRow(keyByteArr))
-    keyRow
+    keySerializer.apply(groupingKey).asInstanceOf[UnsafeRow].getBytes()
   }
 
+  /**
+   * Encode the specified value in Spark UnsafeRow with no ttl.
+   * The ttl expiration will be set to -1, specifying no TTL.
+   */
   def encodeValue(value: V): UnsafeRow = {

Review Comment:
   I'm surprised this is ever possible, given the below method signature.
   `def encodeValue(value: V, expirationMs: Long = -1): UnsafeRow`
   How two methods are not ambiguous? Looks like an edge case of Scala compiler, otherwise I don't get how this could be accepted in language spec.
   
   Also the code comment is more proper to the latter method. Maybe you missed to remove the method and updated the method instead?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val TTL_KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val TTL_VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+
+  /**
+   * 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

Review Comment:
   Do we anticipate a possible bug, or this is expected, e.g. we don't remove the old entry 
    of secondary index but just add the new entry when the value is updated? Yet to read the remaining part of the code.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoder
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTtlExpirationMs - ttl expiration for the current batch.
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTtlExpirationMs: Long)
+  extends SingleKeyTTLStateImpl(stateName, store, batchTtlExpirationMs) with ValueState[S] {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    get() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    Option(get())
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!isExpired(retRow)) {
+        resState
+      } else {
+        null.asInstanceOf[S]
+      }
+    } else {
+      null.asInstanceOf[S]
+    }
+  }
+
+  /** Function to update and overwrite state associated with given key */
+  override def update(
+      newState: S,
+      ttlDuration: Duration = Duration.ZERO): Unit = {
+
+    if (ttlMode == TTLMode.EventTimeTTL() && ttlDuration != Duration.ZERO) {
+      throw StateStoreErrors.cannotProvideTTLDurationForEventTimeTTLMode("update", stateName)
+    }
+
+    if (ttlDuration != null && ttlDuration.isNegative) {
+      throw StateStoreErrors.ttlCannotBeNegative("update", stateName)
+    }
+
+    val expirationTimeInMs =
+      if (ttlDuration != null && ttlDuration != Duration.ZERO) {
+        StateTTL.calculateExpirationTimeForDuration(ttlDuration, batchTtlExpirationMs)
+      } else {
+        -1
+      }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  override def update(
+      newState: S,
+      expirationTimeInMs: Long): Unit = {
+
+    if (expirationTimeInMs < 0) {
+      throw StateStoreErrors.ttlCannotBeNegative(
+        "update", stateName)
+    }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  private def doUpdate(newState: S,
+      expirationTimeInMs: Long): Unit = {
+    val encodedValue = stateTypesEncoder.encodeValue(newState, expirationTimeInMs)
+
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    store.put(stateTypesEncoder.encodeSerializedGroupingKey(serializedGroupingKey),
+      encodedValue, stateName)
+
+    if (expirationTimeInMs != -1) {
+      upsertTTLForStateKey(expirationTimeInMs, serializedGroupingKey)
+    }
+  }
+
+  /** Function to remove state for given key */
+  override def clear(): Unit = {
+    store.remove(stateTypesEncoder.encodeGroupingKey(), stateName)
+  }
+
+  def clearIfExpired(groupingKey: Array[Byte]): Unit = {
+    val encodedGroupingKey = stateTypesEncoder.encodeSerializedGroupingKey(groupingKey)
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      if (isExpired(retRow)) {
+        store.remove(encodedGroupingKey, stateName)
+      }
+    }
+  }
+
+  private def isExpired(valueRow: UnsafeRow): Boolean = {
+    val expirationMs = stateTypesEncoder.decodeTtlExpirationMs(valueRow)
+    val isExpired = expirationMs.map(
+      StateTTL.isExpired(_, batchTtlExpirationMs))
+
+    isExpired.isDefined && isExpired.get
+  }
+
+  /*
+   * 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.

Review Comment:
   nit: 2 spaces are used



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoder
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTtlExpirationMs - ttl expiration for the current batch.
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTtlExpirationMs: Long)
+  extends SingleKeyTTLStateImpl(stateName, store, batchTtlExpirationMs) with ValueState[S] {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    get() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    Option(get())
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!isExpired(retRow)) {
+        resState
+      } else {
+        null.asInstanceOf[S]
+      }
+    } else {
+      null.asInstanceOf[S]
+    }
+  }
+
+  /** Function to update and overwrite state associated with given key */
+  override def update(
+      newState: S,
+      ttlDuration: Duration = Duration.ZERO): Unit = {
+
+    if (ttlMode == TTLMode.EventTimeTTL() && ttlDuration != Duration.ZERO) {
+      throw StateStoreErrors.cannotProvideTTLDurationForEventTimeTTLMode("update", stateName)
+    }
+
+    if (ttlDuration != null && ttlDuration.isNegative) {
+      throw StateStoreErrors.ttlCannotBeNegative("update", stateName)
+    }
+
+    val expirationTimeInMs =
+      if (ttlDuration != null && ttlDuration != Duration.ZERO) {
+        StateTTL.calculateExpirationTimeForDuration(ttlDuration, batchTtlExpirationMs)
+      } else {
+        -1
+      }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  override def update(
+      newState: S,
+      expirationTimeInMs: Long): Unit = {
+
+    if (expirationTimeInMs < 0) {
+      throw StateStoreErrors.ttlCannotBeNegative(
+        "update", stateName)
+    }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  private def doUpdate(newState: S,
+      expirationTimeInMs: Long): Unit = {
+    val encodedValue = stateTypesEncoder.encodeValue(newState, expirationTimeInMs)
+
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    store.put(stateTypesEncoder.encodeSerializedGroupingKey(serializedGroupingKey),
+      encodedValue, stateName)
+
+    if (expirationTimeInMs != -1) {
+      upsertTTLForStateKey(expirationTimeInMs, serializedGroupingKey)
+    }
+  }
+
+  /** Function to remove state for given key */
+  override def clear(): Unit = {
+    store.remove(stateTypesEncoder.encodeGroupingKey(), stateName)
+  }
+
+  def clearIfExpired(groupingKey: Array[Byte]): Unit = {
+    val encodedGroupingKey = stateTypesEncoder.encodeSerializedGroupingKey(groupingKey)
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      if (isExpired(retRow)) {
+        store.remove(encodedGroupingKey, stateName)
+      }
+    }
+  }
+
+  private def isExpired(valueRow: UnsafeRow): Boolean = {
+    val expirationMs = stateTypesEncoder.decodeTtlExpirationMs(valueRow)
+    val isExpired = expirationMs.map(

Review Comment:
   `expirationMs.filter(StateTTL.isExpired(_, batchTtlExpirationMs)).isDefined`
   
   The above is `true` only when expirationMs is `Some(x)` and `StateTTL.isExpired(x, batchTtlExpirationMs)` is `true`.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoder
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTtlExpirationMs - ttl expiration for the current batch.
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTtlExpirationMs: Long)
+  extends SingleKeyTTLStateImpl(stateName, store, batchTtlExpirationMs) with ValueState[S] {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    get() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    Option(get())
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!isExpired(retRow)) {
+        resState
+      } else {
+        null.asInstanceOf[S]
+      }
+    } else {
+      null.asInstanceOf[S]
+    }
+  }
+
+  /** Function to update and overwrite state associated with given key */
+  override def update(
+      newState: S,
+      ttlDuration: Duration = Duration.ZERO): Unit = {
+
+    if (ttlMode == TTLMode.EventTimeTTL() && ttlDuration != Duration.ZERO) {
+      throw StateStoreErrors.cannotProvideTTLDurationForEventTimeTTLMode("update", stateName)
+    }
+
+    if (ttlDuration != null && ttlDuration.isNegative) {
+      throw StateStoreErrors.ttlCannotBeNegative("update", stateName)
+    }
+
+    val expirationTimeInMs =
+      if (ttlDuration != null && ttlDuration != Duration.ZERO) {
+        StateTTL.calculateExpirationTimeForDuration(ttlDuration, batchTtlExpirationMs)
+      } else {
+        -1
+      }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  override def update(
+      newState: S,
+      expirationTimeInMs: Long): Unit = {
+
+    if (expirationTimeInMs < 0) {
+      throw StateStoreErrors.ttlCannotBeNegative(
+        "update", stateName)
+    }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  private def doUpdate(newState: S,
+      expirationTimeInMs: Long): Unit = {
+    val encodedValue = stateTypesEncoder.encodeValue(newState, expirationTimeInMs)
+
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    store.put(stateTypesEncoder.encodeSerializedGroupingKey(serializedGroupingKey),
+      encodedValue, stateName)
+
+    if (expirationTimeInMs != -1) {
+      upsertTTLForStateKey(expirationTimeInMs, serializedGroupingKey)
+    }
+  }
+
+  /** Function to remove state for given key */
+  override def clear(): Unit = {
+    store.remove(stateTypesEncoder.encodeGroupingKey(), stateName)
+  }
+
+  def clearIfExpired(groupingKey: Array[Byte]): Unit = {
+    val encodedGroupingKey = stateTypesEncoder.encodeSerializedGroupingKey(groupingKey)
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      if (isExpired(retRow)) {
+        store.remove(encodedGroupingKey, stateName)
+      }
+    }
+  }
+
+  private def isExpired(valueRow: UnsafeRow): Boolean = {
+    val expirationMs = stateTypesEncoder.decodeTtlExpirationMs(valueRow)
+    val isExpired = expirationMs.map(
+      StateTTL.isExpired(_, batchTtlExpirationMs))
+
+    isExpired.isDefined && isExpired.get
+  }
+
+  /*
+   * 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(): Option[S] = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+      Some(resState)
+    } else {
+      None
+    }
+  }
+
+  /**
+   * Read the ttl value associated with the grouping key.
+   */
+  private[sql] def getTTLValue(): Option[Long] = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      stateTypesEncoder.decodeTtlExpirationMs(retRow)
+    } else {
+      None
+    }
+  }
+
+  /**
+   * Get all ttl values stored in ttl state for current implicit
+   * grouping key.
+   */
+  private[sql] def getValuesInTTLState(): Iterator[Long] = {
+    val ttlIterator = ttlIndexIterator()
+    val implicitGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    var nextValue: Option[Long] = None
+
+    new Iterator[Long] {

Review Comment:
   FYI, NextIterator provides this pattern of implementation, though it's simpler enough so that it's good as it is.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val TTL_KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val TTL_VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+
+  /**
+   * 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

Review Comment:
   I see the semantic of upsertTTLForStateKey - so this is necessary.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoder
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTtlExpirationMs - ttl expiration for the current batch.
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTtlExpirationMs: Long)
+  extends SingleKeyTTLStateImpl(stateName, store, batchTtlExpirationMs) with ValueState[S] {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    get() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    Option(get())
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!isExpired(retRow)) {
+        resState
+      } else {
+        null.asInstanceOf[S]
+      }
+    } else {
+      null.asInstanceOf[S]
+    }
+  }
+
+  /** Function to update and overwrite state associated with given key */
+  override def update(
+      newState: S,
+      ttlDuration: Duration = Duration.ZERO): Unit = {
+
+    if (ttlMode == TTLMode.EventTimeTTL() && ttlDuration != Duration.ZERO) {
+      throw StateStoreErrors.cannotProvideTTLDurationForEventTimeTTLMode("update", stateName)
+    }
+
+    if (ttlDuration != null && ttlDuration.isNegative) {
+      throw StateStoreErrors.ttlCannotBeNegative("update", stateName)
+    }
+
+    val expirationTimeInMs =
+      if (ttlDuration != null && ttlDuration != Duration.ZERO) {
+        StateTTL.calculateExpirationTimeForDuration(ttlDuration, batchTtlExpirationMs)
+      } else {
+        -1
+      }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  override def update(
+      newState: S,
+      expirationTimeInMs: Long): Unit = {
+
+    if (expirationTimeInMs < 0) {
+      throw StateStoreErrors.ttlCannotBeNegative(
+        "update", stateName)
+    }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  private def doUpdate(newState: S,

Review Comment:
   nit: newState to be next line



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -89,14 +124,29 @@ class StateTypesEncoder[GK, V](
     val value = rowToObjDeserializer.apply(reusedValRow)
     value
   }
+
+  /**
+   * Decode the ttl information out of Value row. If the ttl has
+   * not been set (-1L specifies no user defined value), the API will
+   * return None.
+   */
+  def decodeTtlExpirationMs(row: UnsafeRow): Option[Long] = {
+    val expirationMs = row.getLong(1)

Review Comment:
   nit: Maybe check hasTtl for safety guard? I'm fine to assume that the caller knows the flag and has a responsibility, but if it's not a hot codepath, I'd feel comfortable to do the check.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala:
##########
@@ -925,15 +926,15 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
           hasInitialState, planLater(initialState), planLater(child)
         ) :: Nil
       case logical.TransformWithState(keyDeserializer, valueDeserializer, groupingAttributes,
-          dataAttributes, statefulProcessor, timeoutMode, outputMode, keyEncoder,
+          dataAttributes, statefulProcessor, ttlMode, timeoutMode, outputMode, keyEncoder,
           outputObjAttr, child, hasInitialState,
           initialStateGroupingAttrs, initialStateDataAttrs,
           initialStateDeserializer, initialState) =>
         TransformWithStateExec.generateSparkPlanForBatchQueries(keyDeserializer, valueDeserializer,
-          groupingAttributes, dataAttributes, statefulProcessor, timeoutMode, outputMode,
+          groupingAttributes, dataAttributes, statefulProcessor, ttlMode, timeoutMode, outputMode,
           keyEncoder, outputObjAttr, planLater(child), hasInitialState,
           initialStateGroupingAttrs, initialStateDataAttrs,
-          initialStateDeserializer, planLater(initialState)) :: Nil
+          initialStateDeserializer, planLater (initialState)) :: Nil

Review Comment:
   nit: unnecessary space?



##########
sql/core/src/test/java/test/org/apache/spark/sql/TestStatefulProcessor.java:
##########
@@ -85,7 +89,7 @@ public scala.collection.Iterator<String> handleInputRows(
       }
 
       count += numRows;
-      countState.update(count);
+      countState.update(count, Duration.ZERO);

Review Comment:
   Probably I had to comment this in API side but I just realized this in here.
   
   I feel like this may be an overkill to allow flexible TTL per every update. My understanding is that Flink allows users to set the TTL config as a spec of state, not something users can define for every update. Do we have a specific use case in mind?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoder
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTtlExpirationMs - ttl expiration for the current batch.
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTtlExpirationMs: Long)
+  extends SingleKeyTTLStateImpl(stateName, store, batchTtlExpirationMs) with ValueState[S] {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    get() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    Option(get())
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!isExpired(retRow)) {
+        resState
+      } else {
+        null.asInstanceOf[S]
+      }
+    } else {
+      null.asInstanceOf[S]
+    }
+  }
+
+  /** Function to update and overwrite state associated with given key */
+  override def update(
+      newState: S,
+      ttlDuration: Duration = Duration.ZERO): Unit = {
+
+    if (ttlMode == TTLMode.EventTimeTTL() && ttlDuration != Duration.ZERO) {
+      throw StateStoreErrors.cannotProvideTTLDurationForEventTimeTTLMode("update", stateName)
+    }
+
+    if (ttlDuration != null && ttlDuration.isNegative) {
+      throw StateStoreErrors.ttlCannotBeNegative("update", stateName)
+    }
+
+    val expirationTimeInMs =
+      if (ttlDuration != null && ttlDuration != Duration.ZERO) {
+        StateTTL.calculateExpirationTimeForDuration(ttlDuration, batchTtlExpirationMs)
+      } else {
+        -1
+      }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  override def update(
+      newState: S,
+      expirationTimeInMs: Long): Unit = {
+
+    if (expirationTimeInMs < 0) {
+      throw StateStoreErrors.ttlCannotBeNegative(
+        "update", stateName)
+    }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  private def doUpdate(newState: S,
+      expirationTimeInMs: Long): Unit = {
+    val encodedValue = stateTypesEncoder.encodeValue(newState, expirationTimeInMs)
+
+    val serializedGroupingKey = stateTypesEncoder.serializeGroupingKey()
+    store.put(stateTypesEncoder.encodeSerializedGroupingKey(serializedGroupingKey),
+      encodedValue, stateName)
+
+    if (expirationTimeInMs != -1) {
+      upsertTTLForStateKey(expirationTimeInMs, serializedGroupingKey)

Review Comment:
   The method name is actually confusing - it makes me expect that we find the old value of TTL for this grouping key, and remove it if any, and put the new TTL. But in reality we seem to leave the old TTL entry as it is. What to upsert seems to be ambiguious.
   
   I understand we don't have cross reference so uneasy to remove the old entry, but maybe we could make it clear that the entry is only removed from the expiration (there's no case where the update of value will "replace" the TTL entry) so that we shouldn't rely on expiration of TTL entry.
   
   I see the remaining code part is accounting this, but the method name still feels me in that way. putNewTTLEntryForStateKey to be super clear, or method doc to clarify?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala:
##########
@@ -78,7 +80,7 @@ class ValueStateSuite extends StateVariableSuiteBase {
         testState.update(123)
       }
       checkError(
-        ex.asInstanceOf[SparkException],
+        ex1.asInstanceOf[SparkException],

Review Comment:
   Nice finding :) 



##########
sql/core/src/test/java/test/org/apache/spark/sql/TestStatefulProcessor.java:
##########
@@ -85,7 +89,7 @@ public scala.collection.Iterator<String> handleInputRows(
       }
 
       count += numRows;
-      countState.update(count);
+      countState.update(count, Duration.ZERO);

Review Comment:
   It looks like regressing the UX on non-TTL case.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1554104398


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/ValueStateSuite.scala:
##########
@@ -312,187 +312,109 @@ class ValueStateSuite extends StateVariableSuiteBase {
     }
   }
 
-  Seq(TTLMode.ProcessingTimeTTL(), TTLMode.EventTimeTTL()).foreach { ttlMode =>
-    test(s"test Value state TTL for $ttlMode") {
-      tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
-        val store = provider.getStore(0)
-        val timestampMs = 10
-        val handle = createHandleForTtlMode(ttlMode, store, timestampMs)
-
-        val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-          Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
-        ImplicitGroupingKeyTracker.setImplicitKey("test_key")
-        testState.update("v1")
-        assert(testState.get() === "v1")
-        assert(testState.getWithoutEnforcingTTL().get === "v1")
-
-        var ttlValue = testState.getTTLValue()
-        assert(ttlValue.isEmpty)
-        var ttlStateValueIterator = testState.getValuesInTTLState()
-        assert(ttlStateValueIterator.isEmpty)
-
-        testState.clear()
-        assert(!testState.exists())
-        assert(testState.get() === null)
-
-        val ttlExpirationMs = timestampMs + 60000
-
-        if (ttlMode == TTLMode.ProcessingTimeTTL()) {
-          testState.update("v1", Duration.ofMinutes(1))
-        } else {
-          testState.update("v1", ttlExpirationMs)
-        }
-        assert(testState.get() === "v1")
-        assert(testState.getWithoutEnforcingTTL().get === "v1")
-
-        ttlValue = testState.getTTLValue()
-        assert(ttlValue.isDefined)
-        assert(ttlValue.get === ttlExpirationMs)
-        ttlStateValueIterator = testState.getValuesInTTLState()
-        assert(ttlStateValueIterator.hasNext)
-        assert(ttlStateValueIterator.next() === ttlExpirationMs)
-        assert(ttlStateValueIterator.isEmpty)
-
-        // increment batchProcessingTime, or watermark and ensure expired value is not returned
-        val nextBatchHandle = createHandleForTtlMode(ttlMode, store, ttlExpirationMs)
-
-        val nextBatchTestState: ValueStateImplWithTTL[String] = nextBatchHandle
-          .getValueState[String]("testState", Encoders.STRING)
-          .asInstanceOf[ValueStateImplWithTTL[String]]
-        ImplicitGroupingKeyTracker.setImplicitKey("test_key")
-
-        // ensure get does not return the expired value
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-
-        // ttl value should still exist in state
-        ttlValue = nextBatchTestState.getTTLValue()
-        assert(ttlValue.isDefined)
-        assert(ttlValue.get === ttlExpirationMs)
-        ttlStateValueIterator = nextBatchTestState.getValuesInTTLState()
-        assert(ttlStateValueIterator.hasNext)
-        assert(ttlStateValueIterator.next() === ttlExpirationMs)
-        assert(ttlStateValueIterator.isEmpty)
-
-        // getWithoutTTL should still return the expired value
-        assert(nextBatchTestState.getWithoutEnforcingTTL().get === "v1")
-
-        nextBatchTestState.clear()
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-
-        nextBatchTestState.clear()
-        assert(!nextBatchTestState.exists())
-        assert(nextBatchTestState.get() === null)
-      }
-    }
-  }
 
-  test("test TTL duration throws error for event time") {
+  test(s"test Value state TTL") {
     tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
       val store = provider.getStore(0)
-      val eventTimeWatermarkMs = 10
+      val timestampMs = 10
       val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
-        TTLMode.EventTimeTTL(), TimeoutMode.NoTimeouts(),
-        eventTimeWatermarkMs = Some(eventTimeWatermarkMs))
+        TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
+        batchTimestampMs = Some(timestampMs))
 
+      val ttlConfig = TTLConfig(ttlDuration = Duration.ofMinutes(1))
       val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-        Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
+        Encoders.STRING, ttlConfig).asInstanceOf[ValueStateImplWithTTL[String]]
       ImplicitGroupingKeyTracker.setImplicitKey("test_key")
+      testState.update("v1")
+      assert(testState.get() === "v1")
+      assert(testState.getWithoutEnforcingTTL().get === "v1")
 
-      val ex = intercept[SparkUnsupportedOperationException] {
-        testState.update("v1", Duration.ofMinutes(1))
-      }
+      var ttlValue = testState.getTTLValue()
+      assert(ttlValue.isEmpty)
+      var ttlStateValueIterator = testState.getValuesInTTLState()
+      assert(ttlStateValueIterator.isEmpty)
 
-      checkError(
-        ex,
-        errorClass = "STATEFUL_PROCESSOR_CANNOT_USE_TTL_DURATION_IN_EVENT_TIME_TTL_MODE",
-        parameters = Map(
-          "operationType" -> "update",
-          "stateName" -> "testState"
-        ),
-        matchPVals = true
-      )
-    }
-  }
+      testState.clear()
+      assert(!testState.exists())
+      assert(testState.get() === null)
 
-  test("test negative TTL duration throws error") {
-    tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
-      val store = provider.getStore(0)
-      val batchTimestampMs = 10
-      val handle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
+      val ttlExpirationMs = timestampMs + 60000
+
+      testState.update("v1")
+      assert(testState.get() === "v1")
+      assert(testState.getWithoutEnforcingTTL().get === "v1")
+
+      ttlValue = testState.getTTLValue()
+      assert(ttlValue.isDefined)
+      assert(ttlValue.get === ttlExpirationMs)
+      ttlStateValueIterator = testState.getValuesInTTLState()
+      assert(ttlStateValueIterator.hasNext)
+      assert(ttlStateValueIterator.next() === ttlExpirationMs)
+      assert(ttlStateValueIterator.isEmpty)
+
+      // increment batchProcessingTime, or watermark and ensure expired value is not returned
+      val nextBatchHandle = new StatefulProcessorHandleImpl(store, UUID.randomUUID(),
         Encoders.STRING.asInstanceOf[ExpressionEncoder[Any]],
         TTLMode.ProcessingTimeTTL(), TimeoutMode.NoTimeouts(),
-        batchTimestampMs = Some(batchTimestampMs))
+        batchTimestampMs = Some(timestampMs))
 
-      val testState: ValueStateImplWithTTL[String] = handle.getValueState[String]("testState",
-        Encoders.STRING).asInstanceOf[ValueStateImplWithTTL[String]]
+      val nextBatchTestState: ValueStateImplWithTTL[String] = nextBatchHandle
+        .getValueState[String]("testState", Encoders.STRING)
+        .asInstanceOf[ValueStateImplWithTTL[String]]
       ImplicitGroupingKeyTracker.setImplicitKey("test_key")
 
-      val ex = intercept[SparkUnsupportedOperationException] {
-        testState.update("v1", Duration.ofMinutes(-1))
-      }
-
-      checkError(
-        ex,
-        errorClass = "STATEFUL_PROCESSOR_TTL_VALUE_CANNOT_BE_NEGATIVE",
-        parameters = Map(
-          "operationType" -> "update",
-          "stateName" -> "testState"
-        ),
-        matchPVals = true
-      )
+      // ensure get does not return the expired value
+      assert(!nextBatchTestState.exists())
+      assert(nextBatchTestState.get() === null)
+
+      // ttl value should still exist in state
+      ttlValue = nextBatchTestState.getTTLValue()
+      assert(ttlValue.isDefined)
+      assert(ttlValue.get === ttlExpirationMs)
+      ttlStateValueIterator = nextBatchTestState.getValuesInTTLState()
+      assert(ttlStateValueIterator.hasNext)
+      assert(ttlStateValueIterator.next() === ttlExpirationMs)
+      assert(ttlStateValueIterator.isEmpty)
+
+      // getWithoutTTL should still return the expired value
+      assert(nextBatchTestState.getWithoutEnforcingTTL().get === "v1")
+
+      nextBatchTestState.clear()
+      assert(!nextBatchTestState.exists())
+      assert(nextBatchTestState.get() === null)
+
+      nextBatchTestState.clear()

Review Comment:
   removed the redundant check. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #45674:
URL: https://github.com/apache/spark/pull/45674#issuecomment-2041792044

   Thanks! Merging to master.


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547187515


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+class SingleKeyTTLStateImpl(
+    ttlMode: TTLMode,
+    stateName: String,
+    store: StateStore,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends TTLState
+  with Logging {
+
+  import org.apache.spark.sql.execution.streaming.StateTTLSchema._
+
+  private val ttlColumnFamilyName = s"_ttl_$stateName"
+  private val ttlKeyEncoder = UnsafeProjection.create(KEY_ROW_SCHEMA)
+  private var state: StateVariableWithTTLSupport = _
+
+  // empty row used for values
+  private val EMPTY_ROW =
+    UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
+
+  store.createColFamilyIfAbsent(ttlColumnFamilyName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA,
+    RangeKeyScanStateEncoderSpec(KEY_ROW_SCHEMA, 1), isInternal = true)
+
+  def upsertTTLForStateKey(
+      expirationMs: Long,
+      groupingKey: Array[Byte]): Unit = {
+    val encodedTtlKey = ttlKeyEncoder(InternalRow(expirationMs, groupingKey))
+    store.put(encodedTtlKey, EMPTY_ROW, ttlColumnFamilyName)
+  }
+
+  override def clearExpiredState(): Unit = {
+    val iterator = store.iterator(ttlColumnFamilyName)
+    var reachedPastExpirationTime = false
+
+    while (iterator.hasNext && !reachedPastExpirationTime) {

Review Comment:
   Right - wondering if there is a more functional way to write this.AFAIK - we tend to not use `while` loops unless absolutely required. But in this case - I guess you want to terminate on condition - so could we do something like
   
   `takeWhile(...).foreach(...)` ?
   
   maybe there is an even better way. cc - @HeartSaVioR - thoughts ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548228995


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+class SingleKeyTTLStateImpl(
+    ttlMode: TTLMode,
+    stateName: String,
+    store: StateStore,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends TTLState
+  with Logging {
+
+  import org.apache.spark.sql.execution.streaming.StateTTLSchema._
+
+  private val ttlColumnFamilyName = s"_ttl_$stateName"
+  private val ttlKeyEncoder = UnsafeProjection.create(KEY_ROW_SCHEMA)
+  private var state: StateVariableWithTTLSupport = _
+
+  // empty row used for values
+  private val EMPTY_ROW =
+    UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
+
+  store.createColFamilyIfAbsent(ttlColumnFamilyName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA,
+    RangeKeyScanStateEncoderSpec(KEY_ROW_SCHEMA, 1), isInternal = true)
+
+  def upsertTTLForStateKey(
+      expirationMs: Long,
+      groupingKey: Array[Byte]): Unit = {
+    val encodedTtlKey = ttlKeyEncoder(InternalRow(expirationMs, groupingKey))
+    store.put(encodedTtlKey, EMPTY_ROW, ttlColumnFamilyName)
+  }
+
+  override def clearExpiredState(): Unit = {
+    val iterator = store.iterator(ttlColumnFamilyName)
+    var reachedPastExpirationTime = false
+
+    while (iterator.hasNext && !reachedPastExpirationTime) {

Review Comment:
   Changed the iteration to use takeWhile and foreach. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547141805


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+class SingleKeyTTLStateImpl(
+    ttlMode: TTLMode,
+    stateName: String,
+    store: StateStore,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends TTLState
+  with Logging {
+
+  import org.apache.spark.sql.execution.streaming.StateTTLSchema._
+
+  private val ttlColumnFamilyName = s"_ttl_$stateName"
+  private val ttlKeyEncoder = UnsafeProjection.create(KEY_ROW_SCHEMA)
+  private var state: StateVariableWithTTLSupport = _
+
+  // empty row used for values
+  private val EMPTY_ROW =
+    UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
+
+  store.createColFamilyIfAbsent(ttlColumnFamilyName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA,
+    RangeKeyScanStateEncoderSpec(KEY_ROW_SCHEMA, 1), isInternal = true)
+
+  def upsertTTLForStateKey(
+      expirationMs: Long,
+      groupingKey: Array[Byte]): Unit = {
+    val encodedTtlKey = ttlKeyEncoder(InternalRow(expirationMs, groupingKey))
+    store.put(encodedTtlKey, EMPTY_ROW, ttlColumnFamilyName)
+  }
+
+  override def clearExpiredState(): Unit = {
+    val iterator = store.iterator(ttlColumnFamilyName)
+    var reachedPastExpirationTime = false
+
+    while (iterator.hasNext && !reachedPastExpirationTime) {

Review Comment:
   Maybe I am not following the suggestion. This method does not return an iterator, it instead goes through the items in ttl column family and deletes the expired values from ttl state. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547183958


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -49,12 +54,17 @@ object StateKeyValueRowSchema {
 class StateTypesEncoder[GK, V](
     keySerializer: Serializer[GK],
     valEncoder: Encoder[V],
-    stateName: String) {
-  import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema._
+    stateName: String,
+    hasTtl: Boolean) extends Logging {
+  import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchema._
 
   /** Variables reused for conversions between byte array and UnsafeRow */
   private val keyProjection = UnsafeProjection.create(KEY_ROW_SCHEMA)
-  private val valueProjection = UnsafeProjection.create(VALUE_ROW_SCHEMA)
+  private val valueProjection = if (hasTtl) {
+      UnsafeProjection.create(VALUE_ROW_SCHEMA_WITH_TTL)

Review Comment:
   Hmm - maybe its just a GH UI thing - but it appears like 4 spaces to me ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548332977


##########
sql/api/src/main/java/org/apache/spark/sql/streaming/TTLMode.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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 org.apache.spark.annotation.Evolving;
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.catalyst.plans.logical.*;
+
+/**
+ * Represents the type of ttl modes possible for the Dataset operations
+ * {@code transformWithState}.
+ */
+@Experimental
+@Evolving
+public class TTLMode {
+
+  /**
+   * Specifies that there is no TTL for the user state. User state would not
+   * be cleaned up by Spark automatically.
+   */
+  public static final TTLMode NoTTL() {
+        return NoTTL$.MODULE$;

Review Comment:
   Fixed. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548800286


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLSuite.scala:
##########
@@ -0,0 +1,579 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{MemoryStream, ValueStateImplWithTTL}
+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,
+    ttl: Duration,
+    eventTime: Timestamp = null,
+    eventTimeTtl: Timestamp = null)
+
+case class OutputEvent(
+    key: String,
+    value: Int,
+    isTTLValue: Boolean,
+    ttlValue: Long)
+
+object TTLInputProcessFunction {
+  def processRow(
+      ttlMode: TTLMode,
+      row: InputEvent,
+      valueState: ValueStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = valueState.getOption()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = valueState.getWithoutEnforcingTTL()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlExpiration = valueState.getTTLValue()
+      if (ttlExpiration.isDefined) {
+        results = OutputEvent(key, -1, isTTLValue = true, ttlExpiration.get) :: results
+      }
+    } else if (row.action == "put") {
+      if (ttlMode == TTLMode.EventTimeTTL() && row.eventTimeTtl != null) {
+        valueState.update(row.value, row.eventTimeTtl.getTime)
+      } else if (ttlMode == TTLMode.EventTimeTTL()) {
+        valueState.update(row.value)
+      } else {
+        valueState.update(row.value, row.ttl)
+      }
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = valueState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+class ValueStateTTLProcessor
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+  with Logging {
+
+  @transient private var _valueState: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueState = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    for (row <- inputRows) {
+      val resultIter = TTLInputProcessFunction.processRow(_ttlMode, row, _valueState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+case class MultipleValueStatesTTLProcessor(
+    ttlKey: String,
+    noTtlKey: String)
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+    with Logging {
+
+  @transient private var _valueStateWithTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _valueStateWithoutTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueStateWithTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _valueStateWithoutTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val state = if (key == ttlKey) {
+      _valueStateWithTTL
+    } else {
+      _valueStateWithoutTTL
+    }
+
+    for (row <- inputRows) {
+      val resultIterator = TTLInputProcessFunction.processRow(_ttlMode, row, state)
+      resultIterator.foreach { r =>
+        results = r :: results
+      }
+    }
+    results.iterator
+  }
+}
+
+class TransformWithStateTTLSuite
+  extends StreamTest {
+  import testImplicits._
+
+  test("validate state is evicted at ttl expiry - processing time ttl") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputStream = MemoryStream[InputEvent]
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new ValueStateTTLProcessor(),
+          TimeoutMode.NoTimeouts(),
+          TTLMode.ProcessingTimeTTL())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, InputEvent("k1", "put", 1, Duration.ofMinutes(1))),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // get this state, and make sure we get unexpired value
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // ensure ttl values were added correctly
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        // advance clock so that state expires
+        AdvanceManualClock(60 * 1000),
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        // validate expired value is not returned
+        CheckNewAnswer(),
+        // ensure this state does not exist any longer in State
+        AddData(inputStream, InputEvent("k1", "get_without_enforcing_ttl", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer()
+      )
+    }
+  }
+
+  test("validate ttl update updates the expiration timestamp - processing time ttl") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputStream = MemoryStream[InputEvent]
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new ValueStateTTLProcessor(),
+          TimeoutMode.NoTimeouts(),
+          TTLMode.ProcessingTimeTTL())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, InputEvent("k1", "put", 1, Duration.ofMinutes(1))),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // get this state, and make sure we get unexpired value
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // ensure ttl values were added correctly
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),

Review Comment:
   So this is tricky. In order to run no dataBatches, we need to return true from `shouldRunAnotherBatch` if ttlMode is set to ProcessingTime/EventTime to enable noDataBatches. I am still working on this, might be able to simulate this with a combination of timers and ttl. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1540206734


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -90,16 +92,13 @@ case class TransformWithStateExec(
 
   override def keyExpressions: Seq[Attribute] = groupingAttributes
 
-  protected val schemaForKeyRow: StructType = new StructType().add("key", BinaryType)
-
-  protected val schemaForValueRow: StructType = new StructType().add("value", BinaryType)
-
   override def requiredChildDistribution: Seq[Distribution] = {
     StatefulOperatorPartitioning.getCompatibleDistribution(groupingAttributes,
       getStateInfo, conf) ::
       Nil
   }
 
+

Review Comment:
   nit: extra newline ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1543444350


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -23,11 +23,19 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.Serializer
 import org.apache.spark.sql.catalyst.encoders.encoderFor
 import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow}
 import org.apache.spark.sql.execution.streaming.state.StateStoreErrors
-import org.apache.spark.sql.types.{BinaryType, StructType}
+import org.apache.spark.sql.types.{BinaryType, LongType, StructType}
 
 object StateKeyValueRowSchema {
   val KEY_ROW_SCHEMA: StructType = new StructType().add("key", BinaryType)
-  val VALUE_ROW_SCHEMA: StructType = new StructType().add("value", BinaryType)
+  val VALUE_ROW_SCHEMA: StructType = new StructType()
+    .add("value", BinaryType)
+    .add("ttlExpirationMs", LongType)

Review Comment:
   Removed logging for ttlExpiration if ttl is disabled. In order to support this functionality, I have split ValueState into ValueStateImpl and ValueStateImplWithTTL - to prevent un-necessary ttlMode checks for each function. 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -62,33 +70,79 @@ class StateTypesEncoder[GK, V](
   private val rowToObjDeserializer = valExpressionEnc.resolveAndBind().createDeserializer()
   private val reusedValRow = new UnsafeRow(valEncoder.schema.fields.length)
 
+  private val NO_TTL_ENCODED_VALUE: Long = -1L

Review Comment:
   Done. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548800286


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLSuite.scala:
##########
@@ -0,0 +1,579 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{MemoryStream, ValueStateImplWithTTL}
+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,
+    ttl: Duration,
+    eventTime: Timestamp = null,
+    eventTimeTtl: Timestamp = null)
+
+case class OutputEvent(
+    key: String,
+    value: Int,
+    isTTLValue: Boolean,
+    ttlValue: Long)
+
+object TTLInputProcessFunction {
+  def processRow(
+      ttlMode: TTLMode,
+      row: InputEvent,
+      valueState: ValueStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = valueState.getOption()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = valueState.getWithoutEnforcingTTL()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlExpiration = valueState.getTTLValue()
+      if (ttlExpiration.isDefined) {
+        results = OutputEvent(key, -1, isTTLValue = true, ttlExpiration.get) :: results
+      }
+    } else if (row.action == "put") {
+      if (ttlMode == TTLMode.EventTimeTTL() && row.eventTimeTtl != null) {
+        valueState.update(row.value, row.eventTimeTtl.getTime)
+      } else if (ttlMode == TTLMode.EventTimeTTL()) {
+        valueState.update(row.value)
+      } else {
+        valueState.update(row.value, row.ttl)
+      }
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = valueState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+class ValueStateTTLProcessor
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+  with Logging {
+
+  @transient private var _valueState: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueState = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    for (row <- inputRows) {
+      val resultIter = TTLInputProcessFunction.processRow(_ttlMode, row, _valueState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+case class MultipleValueStatesTTLProcessor(
+    ttlKey: String,
+    noTtlKey: String)
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+    with Logging {
+
+  @transient private var _valueStateWithTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _valueStateWithoutTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueStateWithTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _valueStateWithoutTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val state = if (key == ttlKey) {
+      _valueStateWithTTL
+    } else {
+      _valueStateWithoutTTL
+    }
+
+    for (row <- inputRows) {
+      val resultIterator = TTLInputProcessFunction.processRow(_ttlMode, row, state)
+      resultIterator.foreach { r =>
+        results = r :: results
+      }
+    }
+    results.iterator
+  }
+}
+
+class TransformWithStateTTLSuite
+  extends StreamTest {
+  import testImplicits._
+
+  test("validate state is evicted at ttl expiry - processing time ttl") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputStream = MemoryStream[InputEvent]
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new ValueStateTTLProcessor(),
+          TimeoutMode.NoTimeouts(),
+          TTLMode.ProcessingTimeTTL())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, InputEvent("k1", "put", 1, Duration.ofMinutes(1))),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // get this state, and make sure we get unexpired value
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // ensure ttl values were added correctly
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        // advance clock so that state expires
+        AdvanceManualClock(60 * 1000),
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        // validate expired value is not returned
+        CheckNewAnswer(),
+        // ensure this state does not exist any longer in State
+        AddData(inputStream, InputEvent("k1", "get_without_enforcing_ttl", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer()
+      )
+    }
+  }
+
+  test("validate ttl update updates the expiration timestamp - processing time ttl") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputStream = MemoryStream[InputEvent]
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new ValueStateTTLProcessor(),
+          TimeoutMode.NoTimeouts(),
+          TTLMode.ProcessingTimeTTL())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, InputEvent("k1", "put", 1, Duration.ofMinutes(1))),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // get this state, and make sure we get unexpired value
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // ensure ttl values were added correctly
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),

Review Comment:
   So this is tricky. In order to run no dataBatches, we need to return true from `shouldRunAnotherBatch` if ttlMode is set to ProcessingTime/EventTime to enable noDataBatches. I am still working on this, might be able to simulate this with a combination of timers and ttl. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1550645031


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -78,17 +80,15 @@ case class TransformWithStateExec(
   override def shortName: String = "transformWithStateExec"
 
   override def shouldRunAnotherBatch(newInputWatermark: Long): Boolean = {
-    timeoutMode match {
+    if (ttlMode == TTLMode.ProcessingTimeTTL() || timeoutMode == TimeoutMode.ProcessingTime()) {
       // TODO: check if we can return true only if actual timers are registered
-      case ProcessingTime =>
-        true
-
-      case EventTime =>
-        eventTimeWatermarkForEviction.isDefined &&
-          newInputWatermark > eventTimeWatermarkForEviction.get
-
-      case _ =>
-        false
+      logWarning(s"returning true from shouldRunAnotherBatch - $newInputWatermark")

Review Comment:
   Removed. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1552901740


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImplWithTTL.scala:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.Encoder
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions.UnsafeRow
+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.{TTLMode, ValueState}
+
+/**
+ * Class that provides a concrete implementation for a single value 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 ttlMode    - TTL Mode for values  stored in this state
+ * @param batchTtlExpirationMs - ttl expiration for the current batch.
+ * @tparam S - data type of object that will be stored
+ */
+class ValueStateImplWithTTL[S](
+    store: StateStore,
+    stateName: String,
+    keyExprEnc: ExpressionEncoder[Any],
+    valEncoder: Encoder[S],
+    ttlMode: TTLMode,
+    batchTtlExpirationMs: Long)
+  extends SingleKeyTTLStateImpl(stateName, store, batchTtlExpirationMs) with ValueState[S] {
+
+  private val keySerializer = keyExprEnc.createSerializer()
+  private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder,
+    stateName, hasTtl = true)
+
+  initialize()
+
+  private def initialize(): Unit = {
+    assert(ttlMode != TTLMode.NoTTL())
+
+    store.createColFamilyIfAbsent(stateName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL,
+      NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA))
+  }
+
+  /** Function to check if state exists. Returns true if present and false otherwise */
+  override def exists(): Boolean = {
+    get() != null
+  }
+
+  /** Function to return Option of value if exists and None otherwise */
+  override def getOption(): Option[S] = {
+    Option(get())
+  }
+
+  /** Function to return associated value with key if exists and null otherwise */
+  override def get(): S = {
+    val encodedGroupingKey = stateTypesEncoder.encodeGroupingKey()
+    val retRow = store.get(encodedGroupingKey, stateName)
+
+    if (retRow != null) {
+      val resState = stateTypesEncoder.decodeValue(retRow)
+
+      if (!isExpired(retRow)) {
+        resState
+      } else {
+        null.asInstanceOf[S]
+      }
+    } else {
+      null.asInstanceOf[S]
+    }
+  }
+
+  /** Function to update and overwrite state associated with given key */
+  override def update(
+      newState: S,
+      ttlDuration: Duration = Duration.ZERO): Unit = {
+
+    if (ttlMode == TTLMode.EventTimeTTL() && ttlDuration != Duration.ZERO) {
+      throw StateStoreErrors.cannotProvideTTLDurationForEventTimeTTLMode("update", stateName)
+    }
+
+    if (ttlDuration != null && ttlDuration.isNegative) {
+      throw StateStoreErrors.ttlCannotBeNegative("update", stateName)
+    }
+
+    val expirationTimeInMs =
+      if (ttlDuration != null && ttlDuration != Duration.ZERO) {
+        StateTTL.calculateExpirationTimeForDuration(ttlDuration, batchTtlExpirationMs)
+      } else {
+        -1
+      }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  override def update(
+      newState: S,
+      expirationTimeInMs: Long): Unit = {
+
+    if (expirationTimeInMs < 0) {
+      throw StateStoreErrors.ttlCannotBeNegative(
+        "update", stateName)
+    }
+
+    doUpdate(newState, expirationTimeInMs)
+  }
+
+  private def doUpdate(newState: S,

Review Comment:
   removed this method as ttlConfig is now on state variable level. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1552897542


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -65,22 +74,48 @@ class StateTypesEncoder[GK, V](
   // TODO: validate places that are trying to encode the key and check if we can eliminate/
   // add caching for some of these calls.
   def encodeGroupingKey(): UnsafeRow = {
+    val keyRow = keyProjection(InternalRow(serializeGroupingKey()))
+    keyRow
+  }
+
+  /**
+   * Encodes the provided grouping key into Spark UnsafeRow.
+   *
+   * @param groupingKeyBytes serialized grouping key byte array
+   * @return encoded UnsafeRow
+   */
+  def encodeSerializedGroupingKey(groupingKeyBytes: Array[Byte]): UnsafeRow = {
+    val keyRow = keyProjection(InternalRow(groupingKeyBytes))
+    keyRow
+  }
+
+  def serializeGroupingKey(): Array[Byte] = {
     val keyOption = ImplicitGroupingKeyTracker.getImplicitKeyOption
     if (keyOption.isEmpty) {
       throw StateStoreErrors.implicitKeyNotFound(stateName)
     }
-
     val groupingKey = keyOption.get.asInstanceOf[GK]
-    val keyByteArr = keySerializer.apply(groupingKey).asInstanceOf[UnsafeRow].getBytes()
-    val keyRow = keyProjection(InternalRow(keyByteArr))
-    keyRow
+    keySerializer.apply(groupingKey).asInstanceOf[UnsafeRow].getBytes()
   }
 
+  /**
+   * Encode the specified value in Spark UnsafeRow with no ttl.
+   * The ttl expiration will be set to -1, specifying no TTL.
+   */
   def encodeValue(value: V): UnsafeRow = {

Review Comment:
   The comment is outdated. I have removed it from here. Also, we do not need a default value for `expirationMs: Long` anymore. Its the whole reason I split it into 2 methods. Not sure why Scala compiler did not catch this. I have removed the default value from second parameter. Thanks for catching this. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1543446564


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -77,14 +78,20 @@ class StatefulProcessorHandleImpl(
     store: StateStore,
     runId: UUID,
     keyEncoder: ExpressionEncoder[Any],
+    ttlMode: TTLMode,
     timeoutMode: TimeoutMode,
-    isStreaming: Boolean = true)
+    isStreaming: Boolean = true,
+    batchTimestampMs: Option[Long] = None,
+    eventTimeWatermarkMs: Option[Long] = None)
   extends StatefulProcessorHandle with Logging {
   import StatefulProcessorHandleState._
 
+  private val ttlStates: util.List[TTLState] = new util.ArrayList[TTLState]()

Review Comment:
   Done



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1540206480


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala:
##########
@@ -26,10 +26,10 @@ import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expressi
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.plans.physical.Distribution
 import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema.{KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA}

Review Comment:
   wonder whether we should clarify this a bit ? `StateKeyValueRowSchema` currently is only used with TWS right - should we rename the file/class to reflect that ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1541540920


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateVariableWithTTLSupport.scala:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLState]].
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean yp based on ttl values stored in

Review Comment:
   nit: `Perform the user state clean up based`



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1541502777


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StateTypesEncoderUtils.scala:
##########
@@ -62,33 +70,79 @@ class StateTypesEncoder[GK, V](
   private val rowToObjDeserializer = valExpressionEnc.resolveAndBind().createDeserializer()
   private val reusedValRow = new UnsafeRow(valEncoder.schema.fields.length)
 
+  private val NO_TTL_ENCODED_VALUE: Long = -1L

Review Comment:
   Could we avoid recording this explicitly ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1546669377


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -77,14 +78,23 @@ class StatefulProcessorHandleImpl(
     store: StateStore,
     runId: UUID,
     keyEncoder: ExpressionEncoder[Any],
+    ttlMode: TTLMode,
     timeoutMode: TimeoutMode,
-    isStreaming: Boolean = true)
+    isStreaming: Boolean = true,
+    batchTimestampMs: Option[Long] = None,
+    eventTimeWatermarkMs: Option[Long] = None)
   extends StatefulProcessorHandle with Logging {
   import StatefulProcessorHandleState._
 
+  /**
+   * Stores all the active ttl states, and is used to cleanup expired values
+   * in [[doTtlCleanup()]] function.
+   */
+  private val ttlStates: util.List[TTLState] = new util.ArrayList[TTLState]()

Review Comment:
   is there any use-case for accessing a TTL state by name ? i.e. just wondering whether a map is better here ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1546685853


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+class SingleKeyTTLStateImpl(
+    ttlMode: TTLMode,
+    stateName: String,
+    store: StateStore,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends TTLState
+  with Logging {
+
+  import org.apache.spark.sql.execution.streaming.StateTTLSchema._
+
+  private val ttlColumnFamilyName = s"_ttl_$stateName"
+  private val ttlKeyEncoder = UnsafeProjection.create(KEY_ROW_SCHEMA)
+  private var state: StateVariableWithTTLSupport = _
+
+  // empty row used for values
+  private val EMPTY_ROW =
+    UnsafeProjection.create(Array[DataType](NullType)).apply(InternalRow.apply(null))
+
+  store.createColFamilyIfAbsent(ttlColumnFamilyName, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA,
+    RangeKeyScanStateEncoderSpec(KEY_ROW_SCHEMA, 1), isInternal = true)
+
+  def upsertTTLForStateKey(
+      expirationMs: Long,
+      groupingKey: Array[Byte]): Unit = {
+    val encodedTtlKey = ttlKeyEncoder(InternalRow(expirationMs, groupingKey))
+    store.put(encodedTtlKey, EMPTY_ROW, ttlColumnFamilyName)
+  }
+
+  override def clearExpiredState(): Unit = {
+    val iterator = store.iterator(ttlColumnFamilyName)
+    var reachedPastExpirationTime = false
+
+    while (iterator.hasNext && !reachedPastExpirationTime) {

Review Comment:
   Could we remove `while` and model as a `Iterator` - something similar to https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TimerStateImpl.scala#L206 ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1546691740


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/ValueState.scala:
##########
@@ -42,8 +43,22 @@ private[sql] trait ValueState[S] extends Serializable {
   /** Get the state if it exists as an option and None otherwise */
   def getOption(): Option[S]
 
-  /** Update the value of the state. */
-  def update(newState: S): Unit
+  /**
+   * Update the value of the state.
+   * @param newState the new value
+   * @param ttlDuration set the ttl to current batch processing time
+   *                    (for processing time TTL mode) plus ttlDuration
+   */
+  def update(newState: S, ttlDuration: Duration = Duration.ZERO): Unit
+
+
+  /**
+   * Update the value of the state.
+   *
+   * @param newState    the new value
+   * @param expirationMs set the ttl to expirationMs (processingTime or eventTime)

Review Comment:
   nit: should we rename to `expirationTimeInMs` ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547140572


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.streaming.TTLMode
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents a State variable which supports TTL.
+ */
+trait StateVariableWithTTLSupport {
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+class SingleKeyTTLStateImpl(
+    ttlMode: TTLMode,
+    stateName: String,
+    store: StateStore,
+    batchTimestampMs: Option[Long],
+    eventTimeWatermarkMs: Option[Long])
+  extends TTLState
+  with Logging {

Review Comment:
   Logging is not needed here, removed. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1547139448


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -103,22 +113,29 @@ class StatefulProcessorHandleImpl(
 
   private var currState: StatefulProcessorHandleState = CREATED
 
-  private def verify(condition: => Boolean, msg: String): Unit = {
-    if (!condition) {
-      throw new IllegalStateException(msg)
-    }
-  }
-
   def setHandleState(newState: StatefulProcessorHandleState): Unit = {
     currState = newState
   }
 
   def getHandleState: StatefulProcessorHandleState = currState
 
-  override def getValueState[T](stateName: String, valEncoder: Encoder[T]): ValueState[T] = {
+  override def getValueState[T](
+      stateName: String,
+      valEncoder: Encoder[T]): ValueState[T] = {
     verifyStateVarOperations("get_value_state")
-    val resultState = new ValueStateImpl[T](store, stateName, keyEncoder, valEncoder)
-    resultState
+
+    if (ttlMode == TTLMode.NoTTL()) {
+      new ValueStateImpl[T](store, stateName, keyEncoder, valEncoder)
+    } else {
+      val valueStateWithTTL = new ValueStateImplWithTTL[T](store, stateName,
+        keyEncoder, valEncoder, ttlMode, batchTimestampMs, eventTimeWatermarkMs)
+
+      val ttlState = valueStateWithTTL.ttlState

Review Comment:
   Maybe I am not following your question. We want to add the ttlState instead of the actual state variable state to `ttlStates`. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548239064


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -77,14 +78,23 @@ class StatefulProcessorHandleImpl(
     store: StateStore,
     runId: UUID,
     keyEncoder: ExpressionEncoder[Any],
+    ttlMode: TTLMode,
     timeoutMode: TimeoutMode,
-    isStreaming: Boolean = true)
+    isStreaming: Boolean = true,
+    batchTimestampMs: Option[Long] = None,
+    eventTimeWatermarkMs: Option[Long] = None)
   extends StatefulProcessorHandle with Logging {
   import StatefulProcessorHandleState._
 
+  /**
+   * Stores all the active ttl states, and is used to cleanup expired values
+   * in [[doTtlCleanup()]] function.
+   */
+  private val ttlStates: util.List[TTLState] = new util.ArrayList[TTLState]()

Review Comment:
   No particular reason. I wanted to use a mutable data structure. Scala List would work fine, and we can append at the front. I dont think we need to use a Seq, as append would be less performant on it. Thoughts? 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ValueStateImpl.scala:
##########
@@ -16,39 +16,45 @@
  */
 package org.apache.spark.sql.execution.streaming
 
+import java.time.Duration
+
 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.catalyst.expressions.UnsafeRow
-import org.apache.spark.sql.execution.streaming.StateKeyValueRowSchema.{KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA}
-import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore}
+import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchema.{KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA}
+import org.apache.spark.sql.execution.streaming.state.{NoPrefixKeyStateEncoderSpec, StateStore, StateStoreErrors}
 import org.apache.spark.sql.streaming.ValueState
 
 /**
  * Class that provides a concrete implementation for a single value state associated with state
  * variables used in the streaming transformWithState operator.
  * @param store - reference to the StateStore instance to be used for storing state
  * @param stateName - name of logical state partition
- * @param keyEnc - Spark SQL encoder for key
+ * @param keyExprEnc - Spark SQL encoder for key
  * @param valEncoder - Spark SQL encoder for value
  * @tparam S - data type of object that will be stored
  */
 class ValueStateImpl[S](
     store: StateStore,
     stateName: String,
     keyExprEnc: ExpressionEncoder[Any],
-    valEncoder: Encoder[S]) extends ValueState[S] with Logging {
+    valEncoder: Encoder[S])
+  extends ValueState[S] with Logging {
 
   private val keySerializer = keyExprEnc.createSerializer()
-
   private val stateTypesEncoder = StateTypesEncoder(keySerializer, valEncoder, stateName)
+  private[sql] var ttlState: Option[SingleKeyTTLStateImpl] = None
+
+  initialize()

Review Comment:
   Just part of refactoring. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548334799


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StatefulProcessorHandleSuite.scala:
##########
@@ -48,7 +49,7 @@ class StatefulProcessorHandleSuite extends StateVariableSuiteBase {
       tryWithProviderResource(newStoreProviderWithStateVariable(true)) { provider =>
         val store = provider.getStore(0)
         val handle = new StatefulProcessorHandleImpl(store,
-          UUID.randomUUID(), keyExprEncoder, getTimeoutMode(timeoutMode))
+          UUID.randomUUID(), keyExprEncoder, TTLMode.NoTTL(), getTimeoutMode(timeoutMode))

Review Comment:
   Added UT testcases for `ValueStateImplWithTTL` in `ValueStateSuite`. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548594047


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLSuite.scala:
##########
@@ -0,0 +1,579 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{MemoryStream, ValueStateImplWithTTL}
+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,
+    ttl: Duration,
+    eventTime: Timestamp = null,
+    eventTimeTtl: Timestamp = null)
+
+case class OutputEvent(
+    key: String,
+    value: Int,
+    isTTLValue: Boolean,
+    ttlValue: Long)
+
+object TTLInputProcessFunction {
+  def processRow(
+      ttlMode: TTLMode,
+      row: InputEvent,
+      valueState: ValueStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = valueState.getOption()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = valueState.getWithoutEnforcingTTL()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlExpiration = valueState.getTTLValue()
+      if (ttlExpiration.isDefined) {
+        results = OutputEvent(key, -1, isTTLValue = true, ttlExpiration.get) :: results
+      }
+    } else if (row.action == "put") {
+      if (ttlMode == TTLMode.EventTimeTTL() && row.eventTimeTtl != null) {
+        valueState.update(row.value, row.eventTimeTtl.getTime)
+      } else if (ttlMode == TTLMode.EventTimeTTL()) {
+        valueState.update(row.value)
+      } else {
+        valueState.update(row.value, row.ttl)
+      }
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = valueState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+class ValueStateTTLProcessor
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+  with Logging {
+
+  @transient private var _valueState: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueState = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    for (row <- inputRows) {
+      val resultIter = TTLInputProcessFunction.processRow(_ttlMode, row, _valueState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+case class MultipleValueStatesTTLProcessor(
+    ttlKey: String,
+    noTtlKey: String)
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+    with Logging {
+
+  @transient private var _valueStateWithTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _valueStateWithoutTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueStateWithTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _valueStateWithoutTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val state = if (key == ttlKey) {
+      _valueStateWithTTL
+    } else {
+      _valueStateWithoutTTL
+    }
+
+    for (row <- inputRows) {
+      val resultIterator = TTLInputProcessFunction.processRow(_ttlMode, row, state)
+      resultIterator.foreach { r =>
+        results = r :: results
+      }
+    }
+    results.iterator
+  }
+}
+
+class TransformWithStateTTLSuite

Review Comment:
   Could we add a suite level comment for this ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548718711


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLSuite.scala:
##########
@@ -0,0 +1,579 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{MemoryStream, ValueStateImplWithTTL}
+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,
+    ttl: Duration,
+    eventTime: Timestamp = null,
+    eventTimeTtl: Timestamp = null)
+
+case class OutputEvent(
+    key: String,
+    value: Int,
+    isTTLValue: Boolean,
+    ttlValue: Long)
+
+object TTLInputProcessFunction {
+  def processRow(
+      ttlMode: TTLMode,
+      row: InputEvent,
+      valueState: ValueStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = valueState.getOption()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = valueState.getWithoutEnforcingTTL()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlExpiration = valueState.getTTLValue()
+      if (ttlExpiration.isDefined) {
+        results = OutputEvent(key, -1, isTTLValue = true, ttlExpiration.get) :: results
+      }
+    } else if (row.action == "put") {
+      if (ttlMode == TTLMode.EventTimeTTL() && row.eventTimeTtl != null) {
+        valueState.update(row.value, row.eventTimeTtl.getTime)
+      } else if (ttlMode == TTLMode.EventTimeTTL()) {
+        valueState.update(row.value)
+      } else {
+        valueState.update(row.value, row.ttl)
+      }
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = valueState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+class ValueStateTTLProcessor
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+  with Logging {
+
+  @transient private var _valueState: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueState = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    for (row <- inputRows) {
+      val resultIter = TTLInputProcessFunction.processRow(_ttlMode, row, _valueState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+case class MultipleValueStatesTTLProcessor(
+    ttlKey: String,
+    noTtlKey: String)
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+    with Logging {
+
+  @transient private var _valueStateWithTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _valueStateWithoutTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueStateWithTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _valueStateWithoutTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val state = if (key == ttlKey) {
+      _valueStateWithTTL
+    } else {
+      _valueStateWithoutTTL
+    }
+
+    for (row <- inputRows) {
+      val resultIterator = TTLInputProcessFunction.processRow(_ttlMode, row, state)
+      resultIterator.foreach { r =>
+        results = r :: results
+      }
+    }
+    results.iterator
+  }
+}
+
+class TransformWithStateTTLSuite
+  extends StreamTest {
+  import testImplicits._
+
+  test("validate state is evicted at ttl expiry - processing time ttl") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputStream = MemoryStream[InputEvent]
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new ValueStateTTLProcessor(),
+          TimeoutMode.NoTimeouts(),
+          TTLMode.ProcessingTimeTTL())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, InputEvent("k1", "put", 1, Duration.ofMinutes(1))),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // get this state, and make sure we get unexpired value
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // ensure ttl values were added correctly
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        // advance clock so that state expires
+        AdvanceManualClock(60 * 1000),
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        // validate expired value is not returned
+        CheckNewAnswer(),
+        // ensure this state does not exist any longer in State
+        AddData(inputStream, InputEvent("k1", "get_without_enforcing_ttl", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer()
+      )
+    }
+  }
+
+  test("validate ttl update updates the expiration timestamp - processing time ttl") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputStream = MemoryStream[InputEvent]
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new ValueStateTTLProcessor(),
+          TimeoutMode.NoTimeouts(),
+          TTLMode.ProcessingTimeTTL())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, InputEvent("k1", "put", 1, Duration.ofMinutes(1))),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // get this state, and make sure we get unexpired value
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // ensure ttl values were added correctly
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        // advance clock and update expiration time
+        AdvanceManualClock(30 * 1000),
+        AddData(inputStream, InputEvent("k1", "put", 1, Duration.ofMinutes(1))),
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        // validate value is not expired
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // validate ttl value is updated in the state
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 95000)),
+        // validate ttl state has both ttl values present
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000),
+          OutputEvent("k1", -1, isTTLValue = true, 95000)
+        ),
+        // advance clock after older expiration value
+        AdvanceManualClock(30 * 1000),
+        // ensure unexpired value is still present in the state
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // validate that the older expiration value is removed from ttl state
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 95000))
+      )
+    }
+  }
+
+  test("validate ttl removal keeps value in state - processing time ttl") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputStream = MemoryStream[InputEvent]
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new ValueStateTTLProcessor(),
+          TimeoutMode.NoTimeouts(),
+          TTLMode.ProcessingTimeTTL())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, InputEvent("k1", "put", 1, Duration.ofMinutes(1))),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // get this state, and make sure we get unexpired value
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // ensure ttl values were added correctly
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        // advance clock and update state to remove ttl
+        AdvanceManualClock(30 * 1000),
+        AddData(inputStream, InputEvent("k1", "put", 1, null)),
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        // validate value is not expired
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // validate ttl value is removed in the value state column family
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // validate ttl state still has old ttl value present
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        // advance clock after older expiration value
+        AdvanceManualClock(30 * 1000),
+        // ensure unexpired value is still present in the state
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // validate that the older expiration value is removed from ttl state
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer()
+      )
+    }
+  }
+
+  test("validate multiple value states - processing time ttl") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val ttlKey = "k1"
+      val noTtlKey = "k2"
+
+      val inputStream = MemoryStream[InputEvent]
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          MultipleValueStatesTTLProcessor(ttlKey, noTtlKey),
+          TimeoutMode.NoTimeouts(),
+          TTLMode.ProcessingTimeTTL())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, InputEvent(ttlKey, "put", 1, Duration.ofMinutes(1))),
+        AddData(inputStream, InputEvent(noTtlKey, "put", 2, Duration.ZERO)),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // get both state values, and make sure we get unexpired value
+        AddData(inputStream, InputEvent(ttlKey, "get", -1, null)),
+        AddData(inputStream, InputEvent(noTtlKey, "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(
+          OutputEvent(ttlKey, 1, isTTLValue = false, -1),
+          OutputEvent(noTtlKey, 2, isTTLValue = false, -1)
+        ),
+        // ensure ttl values were added correctly, and noTtlKey has no ttl values
+        AddData(inputStream, InputEvent(ttlKey, "get_ttl_value_from_state", -1, null)),
+        AddData(inputStream, InputEvent(noTtlKey, "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent(ttlKey, -1, isTTLValue = true, 61000)),
+        AddData(inputStream, InputEvent(ttlKey, "get_values_in_ttl_state", -1, null)),
+        AddData(inputStream, InputEvent(noTtlKey, "get_values_in_ttl_state", -1, null)),

Review Comment:
   Can we also add some tests that are stopping and starting the streaming query ?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548800286


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateTTLSuite.scala:
##########
@@ -0,0 +1,579 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.execution.streaming.{MemoryStream, ValueStateImplWithTTL}
+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,
+    ttl: Duration,
+    eventTime: Timestamp = null,
+    eventTimeTtl: Timestamp = null)
+
+case class OutputEvent(
+    key: String,
+    value: Int,
+    isTTLValue: Boolean,
+    ttlValue: Long)
+
+object TTLInputProcessFunction {
+  def processRow(
+      ttlMode: TTLMode,
+      row: InputEvent,
+      valueState: ValueStateImplWithTTL[Int]): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val key = row.key
+    if (row.action == "get") {
+      val currState = valueState.getOption()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_without_enforcing_ttl") {
+      val currState = valueState.getWithoutEnforcingTTL()
+      if (currState.isDefined) {
+        results = OutputEvent(key, currState.get, isTTLValue = false, -1) :: results
+      }
+    } else if (row.action == "get_ttl_value_from_state") {
+      val ttlExpiration = valueState.getTTLValue()
+      if (ttlExpiration.isDefined) {
+        results = OutputEvent(key, -1, isTTLValue = true, ttlExpiration.get) :: results
+      }
+    } else if (row.action == "put") {
+      if (ttlMode == TTLMode.EventTimeTTL() && row.eventTimeTtl != null) {
+        valueState.update(row.value, row.eventTimeTtl.getTime)
+      } else if (ttlMode == TTLMode.EventTimeTTL()) {
+        valueState.update(row.value)
+      } else {
+        valueState.update(row.value, row.ttl)
+      }
+    } else if (row.action == "get_values_in_ttl_state") {
+      val ttlValues = valueState.getValuesInTTLState()
+      ttlValues.foreach { v =>
+        results = OutputEvent(key, -1, isTTLValue = true, ttlValue = v) :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+class ValueStateTTLProcessor
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+  with Logging {
+
+  @transient private var _valueState: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueState = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+
+    for (row <- inputRows) {
+      val resultIter = TTLInputProcessFunction.processRow(_ttlMode, row, _valueState)
+      resultIter.foreach { r =>
+        results = r :: results
+      }
+    }
+
+    results.iterator
+  }
+}
+
+case class MultipleValueStatesTTLProcessor(
+    ttlKey: String,
+    noTtlKey: String)
+  extends StatefulProcessor[String, InputEvent, OutputEvent]
+    with Logging {
+
+  @transient private var _valueStateWithTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _valueStateWithoutTTL: ValueStateImplWithTTL[Int] = _
+  @transient private var _ttlMode: TTLMode = _
+
+  override def init(
+      outputMode: OutputMode,
+      timeoutMode: TimeoutMode,
+      ttlMode: TTLMode): Unit = {
+    _valueStateWithTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _valueStateWithoutTTL = getHandle
+      .getValueState("valueState", Encoders.scalaInt)
+      .asInstanceOf[ValueStateImplWithTTL[Int]]
+    _ttlMode = ttlMode
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[InputEvent],
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo): Iterator[OutputEvent] = {
+    var results = List[OutputEvent]()
+    val state = if (key == ttlKey) {
+      _valueStateWithTTL
+    } else {
+      _valueStateWithoutTTL
+    }
+
+    for (row <- inputRows) {
+      val resultIterator = TTLInputProcessFunction.processRow(_ttlMode, row, state)
+      resultIterator.foreach { r =>
+        results = r :: results
+      }
+    }
+    results.iterator
+  }
+}
+
+class TransformWithStateTTLSuite
+  extends StreamTest {
+  import testImplicits._
+
+  test("validate state is evicted at ttl expiry - processing time ttl") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputStream = MemoryStream[InputEvent]
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new ValueStateTTLProcessor(),
+          TimeoutMode.NoTimeouts(),
+          TTLMode.ProcessingTimeTTL())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, InputEvent("k1", "put", 1, Duration.ofMinutes(1))),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // get this state, and make sure we get unexpired value
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // ensure ttl values were added correctly
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        // advance clock so that state expires
+        AdvanceManualClock(60 * 1000),
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        // validate expired value is not returned
+        CheckNewAnswer(),
+        // ensure this state does not exist any longer in State
+        AddData(inputStream, InputEvent("k1", "get_without_enforcing_ttl", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer()
+      )
+    }
+  }
+
+  test("validate ttl update updates the expiration timestamp - processing time ttl") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName) {
+      val inputStream = MemoryStream[InputEvent]
+      val result = inputStream.toDS()
+        .groupByKey(x => x.key)
+        .transformWithState(
+          new ValueStateTTLProcessor(),
+          TimeoutMode.NoTimeouts(),
+          TTLMode.ProcessingTimeTTL())
+
+      val clock = new StreamManualClock
+      testStream(result)(
+        StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock),
+        AddData(inputStream, InputEvent("k1", "put", 1, Duration.ofMinutes(1))),
+        // advance clock to trigger processing
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(),
+        // get this state, and make sure we get unexpired value
+        AddData(inputStream, InputEvent("k1", "get", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", 1, isTTLValue = false, -1)),
+        // ensure ttl values were added correctly
+        AddData(inputStream, InputEvent("k1", "get_ttl_value_from_state", -1, null)),
+        AdvanceManualClock(1 * 1000),
+        CheckNewAnswer(OutputEvent("k1", -1, isTTLValue = true, 61000)),
+        AddData(inputStream, InputEvent("k1", "get_values_in_ttl_state", -1, null)),
+        AdvanceManualClock(1 * 1000),

Review Comment:
   Added. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-47558][SS] State TTL support for ValueState [spark]

Posted by "sahnib (via GitHub)" <gi...@apache.org>.
sahnib commented on code in PR #45674:
URL: https://github.com/apache/spark/pull/45674#discussion_r1548864182


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TTLState.scala:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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 java.time.Duration
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.execution.streaming.state.{RangeKeyScanStateEncoderSpec, StateStore}
+import org.apache.spark.sql.types.{BinaryType, DataType, LongType, NullType, StructField, StructType}
+
+object StateTTLSchema {
+  val TTL_KEY_ROW_SCHEMA: StructType = new StructType()
+    .add("expirationMs", LongType)
+    .add("groupingKey", BinaryType)
+  val TTL_VALUE_ROW_SCHEMA: StructType =
+    StructType(Array(StructField("__dummy__", NullType)))
+}
+
+/**
+ * Encapsulates the ttl row information stored in [[SingleKeyTTLStateImpl]].
+ *
+ * @param groupingKey grouping key for which ttl is set
+ * @param expirationMs expiration time for the grouping key
+ */
+case class SingleKeyTTLRow(
+    groupingKey: Array[Byte],
+    expirationMs: Long)
+
+/**
+ * Represents the underlying state for secondary TTL Index for a user defined
+ * state variable.
+ *
+ * This state allows Spark to query ttl values based on expiration time
+ * allowing efficient ttl cleanup.
+ */
+trait TTLState {
+
+  /**
+   * Perform the user state clean up based on ttl values stored in
+   * this state. NOTE that its not safe to call this operation concurrently
+   * when the user can also modify the underlying State. Cleanup should be initiated
+   * after arbitrary state operations are completed by the user.
+   */
+  def clearExpiredState(): Unit
+
+  /**
+   * 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.
+   */
+  def clearIfExpired(groupingKey: Array[Byte]): Unit
+}
+
+/**
+ * Manages the ttl information for user state keyed with a single key (grouping key).
+ */
+abstract class SingleKeyTTLStateImpl(
+    stateName: String,
+    store: StateStore,
+    batchTtlExpirationMs: Long)
+  extends TTLState {
+
+  import org.apache.spark.sql.execution.streaming.StateTTLSchema._
+
+  private val ttlColumnFamilyName = s"_ttl_$stateName"
+  private val ttlKeyEncoder = UnsafeProjection.create(TTL_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_KEY_ROW_SCHEMA, TTL_VALUE_ROW_SCHEMA,
+    RangeKeyScanStateEncoderSpec(TTL_KEY_ROW_SCHEMA, 1), isInternal = true)
+
+  def upsertTTLForStateKey(
+      expirationMs: Long,
+      groupingKey: Array[Byte]): Unit = {
+    val encodedTtlKey = ttlKeyEncoder(InternalRow(expirationMs, groupingKey))
+    store.put(encodedTtlKey, EMPTY_ROW, ttlColumnFamilyName)
+  }
+
+  override def clearExpiredState(): Unit = {

Review Comment:
   Done.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -103,22 +113,35 @@ class StatefulProcessorHandleImpl(
 
   private var currState: StatefulProcessorHandleState = CREATED
 
-  private def verify(condition: => Boolean, msg: String): Unit = {
-    if (!condition) {
-      throw new IllegalStateException(msg)
+  private val ttlExpirationMs =
+    if (ttlMode == TTLMode.ProcessingTimeTTL()) {
+      batchTimestampMs.get
+    } else if (ttlMode == TTLMode.EventTimeTTL()) {
+      eventTimeWatermarkMs.get
+    } else {
+    -1

Review Comment:
   Done. 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org