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

2024-04-07 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-07 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-05 Thread via GitHub


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 + 6
-
-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()

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

2024-04-05 Thread via GitHub


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 + 6
-
-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()
+ 

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

2024-04-05 Thread via GitHub


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

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

2024-04-05 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-05 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-05 Thread via GitHub


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 + 6
-
-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()
+ 

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

2024-04-05 Thread via GitHub


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

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

2024-04-04 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-04 Thread via GitHub


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 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-unsubscr...@spark.apache.org

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


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



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

2024-04-04 Thread via GitHub


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 

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

2024-04-04 Thread via GitHub


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 

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

2024-04-04 Thread via GitHub


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 

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

2024-04-04 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-04 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-04 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-04 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-04 Thread via GitHub


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

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

2024-04-04 Thread via GitHub


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 

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

2024-04-03 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-03 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-03 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-03 Thread via GitHub


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

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

2024-04-02 Thread via GitHub


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

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

2024-04-02 Thread via GitHub


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 

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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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

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

2024-04-02 Thread via GitHub


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

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

2024-04-02 Thread via GitHub


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

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

2024-04-02 Thread via GitHub


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

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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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

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

2024-04-02 Thread via GitHub


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 + 6
+  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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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

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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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

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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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

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

2024-04-02 Thread via GitHub


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 + 6
+  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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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, 

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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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, 

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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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)
+  }
+

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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:

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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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 

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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

For queries about this service, 

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

2024-04-02 Thread via GitHub


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

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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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 

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

2024-04-02 Thread via GitHub


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 

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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-02 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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 newStatethe 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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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 

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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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= on state=.",
+  "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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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 newStatethe 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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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 

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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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= on state=.",
+  "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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



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

2024-04-01 Thread via GitHub


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-unsubscr...@spark.apache.org

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


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



  1   2   >