fedimser commented on code in PR #53159:
URL: https://github.com/apache/spark/pull/53159#discussion_r2662908183


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TestProcessors.scala:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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 scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.Encoders
+
+/** Test processor that exercises all state methods for coverage testing. */
+class AllMethodsTestProcessor extends StatefulProcessor[String, String, 
(String, String)] {
+
+  @transient private var valueState: ValueState[Int] = _
+  @transient private var listState: ListState[String] = _
+  @transient private var mapState: MapState[String, Int] = _
+
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+    valueState = getHandle.getValueState[Int]("value", Encoders.scalaInt, 
TTLConfig.NONE)
+    listState = getHandle.getListState[String]("list", Encoders.STRING, 
TTLConfig.NONE)
+    mapState =
+      getHandle.getMapState[String, Int]("map", Encoders.STRING, 
Encoders.scalaInt, TTLConfig.NONE)
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[String],
+      timerValues: TimerValues
+  ): Iterator[(String, String)] = {
+    val results = ArrayBuffer[(String, String)]()
+
+    inputRows.foreach { cmd =>
+      cmd match {
+        case "value-exists" =>
+          results += ((key, s"value-exists:${valueState.exists()}"))
+        case "value-set" =>
+          valueState.update(42)
+          results += ((key, "value-set:done"))
+        case "value-clear" =>
+          valueState.clear()
+          results += ((key, "value-clear:done"))
+        case "list-exists" =>
+          results += ((key, s"list-exists:${listState.exists()}"))
+        case "list-append" =>
+          listState.appendValue("a")
+          listState.appendValue("b")
+          results += ((key, "list-append:done"))
+        case "list-append-array" =>
+          listState.appendList(Array("c", "d"))
+          results += ((key, "list-append-array:done"))
+        case "list-get" =>
+          val items = listState.get().toList.mkString(",")
+          results += ((key, s"list-get:$items"))
+        case "map-exists" =>
+          results += ((key, s"map-exists:${mapState.exists()}"))
+        case "map-add" =>
+          mapState.updateValue("x", 1)
+          mapState.updateValue("y", 2)
+          mapState.updateValue("z", 3)
+          results += ((key, "map-add:done"))
+        case "map-keys" =>
+          val keys = mapState.keys().toList.sorted.mkString(",")
+          results += ((key, s"map-keys:$keys"))
+        case "map-values" =>
+          val values = mapState.values().toList.sorted.mkString(",")
+          results += ((key, s"map-values:$values"))
+        case "map-iterator" =>
+          val pairs =
+            mapState.iterator().toList.sortBy(_._1).map(p => 
s"${p._1}=${p._2}").mkString(",")
+          results += ((key, s"map-iterator:$pairs"))
+        case "map-remove" =>
+          mapState.removeKey("y")
+          results += ((key, "map-remove:done"))
+        case "map-clear" =>
+          mapState.clear()
+          results += ((key, "map-clear:done"))
+      }
+    }
+
+    results.iterator
+  }
+}
+
+/** Test StatefulProcessor implementation that maintains a running count. */
+class RunningCountProcessor[T](ttl: TTLConfig = TTLConfig.NONE)
+    extends StatefulProcessorWithInitialState[String, T, (String, Long), Long] 
{
+
+  @transient private var countState: ValueState[Long] = _
+
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+    countState = getHandle.getValueState[Long]("count", Encoders.scalaLong, 
ttl)
+  }
+
+  override def handleInitialState(
+      key: String,
+      initialState: Long,
+      timerValues: TimerValues
+  ): Unit = {
+    countState.update(initialState)
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[T],
+      timerValues: TimerValues
+  ): Iterator[(String, Long)] = {
+    val incoming = inputRows.size
+    val current = countState.get()
+    val updated = current + incoming
+    countState.update(updated)
+    Iterator.single((key, updated))
+  }
+}
+
+/**
+ * Processor that registers a processing time timer on first input and emits a 
message on expiry.
+ */
+class SessionTimeoutProcessor extends StatefulProcessor[String, String, 
(String, String)] {
+
+  @transient private var lastSeenState: ValueState[Long] = _
+
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+    lastSeenState = getHandle.getValueState[Long]("lastSeen", 
Encoders.scalaLong, TTLConfig.NONE)
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[String],
+      timerValues: TimerValues
+  ): Iterator[(String, String)] = {
+    val currentTime = timerValues.getCurrentProcessingTimeInMs()
+
+    // Clear any existing timer if we have previous state
+    if (lastSeenState.exists()) {
+      val oldTimerTime = lastSeenState.get() + 10000 // old timeout was 10s 
after last seen
+      getHandle.deleteTimer(oldTimerTime)
+    }
+
+    // Update last seen time and register new timer
+    lastSeenState.update(currentTime)
+    getHandle.registerTimer(currentTime + 10000) // 10 second timeout
+
+    inputRows.map(value => (key, s"received:$value"))
+  }
+
+  override def handleExpiredTimer(
+      key: String,
+      timerValues: TimerValues,
+      expiredTimerInfo: ExpiredTimerInfo
+  ): Iterator[(String, String)] = {
+    lastSeenState.clear()
+    Iterator.single((key, "session-expired"))
+  }
+}
+
+/**
+ * Processor that registers an event time timer based on watermark.
+ * Input format: (eventTimeMs: Long, value: String)
+ * Registers a timer at eventTime + 5000ms. Timer fires when watermark passes 
that time.
+ */
+class EventTimeSessionProcessor
+    extends StatefulProcessor[String, (Long, String), (String, String)] {
+
+  @transient private var lastEventTimeState: ValueState[Long] = _
+
+  override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+    lastEventTimeState =
+      getHandle.getValueState[Long]("lastEventTime", Encoders.scalaLong, 
TTLConfig.NONE)
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[(Long, String)],
+      timerValues: TimerValues
+  ): Iterator[(String, String)] = {
+    val results = scala.collection.mutable.ArrayBuffer[(String, String)]()
+
+    inputRows.foreach {
+      case (eventTimeMs, value) =>
+        // Clear any existing timer if we have previous state
+        if (lastEventTimeState.exists()) {

Review Comment:
   I moved this code block out of the loop. This should be more efficient while 
having the same behaviour.



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to