HeartSaVioR commented on code in PR #53159: URL: https://github.com/apache/spark/pull/53159#discussion_r2647587362
########## sql/core/src/main/scala/org/apache/spark/sql/streaming/TwsTester.scala: ########## @@ -0,0 +1,286 @@ +/* + * 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.{Clock, Instant, ZoneId} + +import scala.reflect.ClassTag + +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.testing.InMemoryStatefulProcessorHandle +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl} + +/** + * Testing utility for transformWithState stateful processors. + * + * This class enables unit testing of StatefulProcessor business logic by simulating the + * behavior of transformWithState. It processes input rows and returns output rows equivalent + * to those that would be produced by the processor in an actual Spark streaming query. + * + * '''Supported:''' + * - Processing input rows and producing output rows via `test()`. + * - Initial state setup via constructor parameter. + * - Direct state manipulation via `setValueState`, `setListState`, `setMapState`. + * - Direct state inspection via `peekValueState`, `peekListState`, `peekMapState`. + * - Timers in ProcessingTime mode (use `advanceProcessingTime` to fire timers). + * - Timers in EventTime mode (use `eventTimeExtractor` and `watermarkDelayMs` to configure; + * watermark advances automatically based on event times, or use `advanceWatermark` manually). + * - Late event filtering in EventTime mode (events older than the current watermark are dropped). + * - TTL for ValueState, ListState, and MapState (use ProcessingTime mode and + * `advanceProcessingTime` to test expiry). + * + * '''Testing EventTime Mode:''' + * To test with EventTime, provide `eventTimeExtractor` (a function extracting the event + * timestamp from each input row) and `watermarkDelayMs` (the watermark delay in milliseconds). + * The watermark is computed as `max(event_time_seen) - watermarkDelayMs` and is updated + * automatically after each `test()` call. Late events (with event time older than the current + * watermark) are filtered out before processing, matching production behavior. Timers with + * expiry time <= watermark will fire. You can also manually advance the watermark using + * `advanceWatermark()`. + * + * '''Use Cases:''' + * - '''Primary''': Unit testing business logic in `handleInputRows` implementations. + * - '''Not recommended''': End-to-end testing or performance testing - use actual Spark + * streaming queries for those scenarios. + * + * @param processor the StatefulProcessor to test. + * @param initialState initial state for each key as a list of (key, state) tuples. + * @param timeMode time mode (None, ProcessingTime or EventTime). + * @param outputMode output mode (Append, Update, or Complete). + * @param realTimeMode whether input rows should be processed one-by-one (separate call to + * handleInputRows) for each input row. + * @param eventTimeExtractor function to extract event time from input rows. Required if and + * only if timeMode is EventTime. + * @param watermarkDelayMs watermark delay in milliseconds. The watermark is computed as + * `max(event_time) - watermarkDelayMs`. Required if and only if timeMode is EventTime. + * @tparam K the type of grouping key. + * @tparam I the type of input rows. + * @tparam O the type of output rows. + * @since 4.0.2 Review Comment: nit: 4.2.0 ########## sql/core/src/main/scala/org/apache/spark/sql/streaming/TwsTester.scala: ########## @@ -0,0 +1,286 @@ +/* + * 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.{Clock, Instant, ZoneId} + +import scala.reflect.ClassTag + +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.testing.InMemoryStatefulProcessorHandle +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl} + +/** + * Testing utility for transformWithState stateful processors. + * + * This class enables unit testing of StatefulProcessor business logic by simulating the + * behavior of transformWithState. It processes input rows and returns output rows equivalent + * to those that would be produced by the processor in an actual Spark streaming query. + * + * '''Supported:''' + * - Processing input rows and producing output rows via `test()`. + * - Initial state setup via constructor parameter. + * - Direct state manipulation via `setValueState`, `setListState`, `setMapState`. + * - Direct state inspection via `peekValueState`, `peekListState`, `peekMapState`. + * - Timers in ProcessingTime mode (use `advanceProcessingTime` to fire timers). + * - Timers in EventTime mode (use `eventTimeExtractor` and `watermarkDelayMs` to configure; + * watermark advances automatically based on event times, or use `advanceWatermark` manually). + * - Late event filtering in EventTime mode (events older than the current watermark are dropped). + * - TTL for ValueState, ListState, and MapState (use ProcessingTime mode and + * `advanceProcessingTime` to test expiry). + * + * '''Testing EventTime Mode:''' Review Comment: Same here. ########## sql/core/src/main/scala/org/apache/spark/sql/streaming/TwsTester.scala: ########## @@ -0,0 +1,286 @@ +/* + * 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.{Clock, Instant, ZoneId} + +import scala.reflect.ClassTag + +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.testing.InMemoryStatefulProcessorHandle +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl} + +/** + * Testing utility for transformWithState stateful processors. + * + * This class enables unit testing of StatefulProcessor business logic by simulating the + * behavior of transformWithState. It processes input rows and returns output rows equivalent + * to those that would be produced by the processor in an actual Spark streaming query. + * + * '''Supported:''' + * - Processing input rows and producing output rows via `test()`. + * - Initial state setup via constructor parameter. + * - Direct state manipulation via `setValueState`, `setListState`, `setMapState`. + * - Direct state inspection via `peekValueState`, `peekListState`, `peekMapState`. + * - Timers in ProcessingTime mode (use `advanceProcessingTime` to fire timers). + * - Timers in EventTime mode (use `eventTimeExtractor` and `watermarkDelayMs` to configure; + * watermark advances automatically based on event times, or use `advanceWatermark` manually). + * - Late event filtering in EventTime mode (events older than the current watermark are dropped). + * - TTL for ValueState, ListState, and MapState (use ProcessingTime mode and + * `advanceProcessingTime` to test expiry). + * + * '''Testing EventTime Mode:''' + * To test with EventTime, provide `eventTimeExtractor` (a function extracting the event + * timestamp from each input row) and `watermarkDelayMs` (the watermark delay in milliseconds). + * The watermark is computed as `max(event_time_seen) - watermarkDelayMs` and is updated + * automatically after each `test()` call. Late events (with event time older than the current + * watermark) are filtered out before processing, matching production behavior. Timers with + * expiry time <= watermark will fire. You can also manually advance the watermark using + * `advanceWatermark()`. + * + * '''Use Cases:''' + * - '''Primary''': Unit testing business logic in `handleInputRows` implementations. + * - '''Not recommended''': End-to-end testing or performance testing - use actual Spark + * streaming queries for those scenarios. + * + * @param processor the StatefulProcessor to test. + * @param initialState initial state for each key as a list of (key, state) tuples. + * @param timeMode time mode (None, ProcessingTime or EventTime). + * @param outputMode output mode (Append, Update, or Complete). + * @param realTimeMode whether input rows should be processed one-by-one (separate call to + * handleInputRows) for each input row. + * @param eventTimeExtractor function to extract event time from input rows. Required if and + * only if timeMode is EventTime. + * @param watermarkDelayMs watermark delay in milliseconds. The watermark is computed as + * `max(event_time) - watermarkDelayMs`. Required if and only if timeMode is EventTime. + * @tparam K the type of grouping key. + * @tparam I the type of input rows. + * @tparam O the type of output rows. + * @since 4.0.2 + */ +class TwsTester[K, I, O]( + val processor: StatefulProcessor[K, I, O], Review Comment: nit: Will the fields be referenced by other class? Otherwise I'd rather hide them by default. ########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/testing/InMemoryStatefulProcessorHandle.scala: ########## @@ -0,0 +1,332 @@ +/* + * 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.operators.stateful.transformwithstate.testing + +import java.time.{Clock, Instant} +import java.util.UUID + +import scala.collection.mutable +import scala.reflect.ClassTag + +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.QueryInfoImpl +import org.apache.spark.sql.streaming.{ListState, MapState, QueryInfo, StatefulProcessorHandle, TimeMode, TTLConfig, ValueState} + +/** Helper to track expired keys. */ +class TtlTracker(val clock: Clock, ttl: TTLConfig) { + require(!ttl.ttlDuration.isNegative()) + private val keyToLastUpdatedTime = mutable.Map[Any, Instant]() + + def isKeyExpired(): Boolean = { + if (ttl.ttlDuration.isZero()) { + return false + } + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (!keyToLastUpdatedTime.contains(key)) { + return false + } + val expiration: Instant = keyToLastUpdatedTime.get(key).get.plus(ttl.ttlDuration) + return expiration.isBefore(clock.instant()) + } + + def onKeyUpdated(): Unit = { + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + keyToLastUpdatedTime.put(key, clock.instant()) + } +} + +/** In-memory implementation of ValueState. */ +class InMemoryValueState[T](clock: Clock, ttl: TTLConfig) extends ValueState[T] { + private val keyToStateValue = mutable.Map[Any, T]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getValue: Option[T] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + override def exists(): Boolean = getValue.isDefined + + override def get(): T = getValue.getOrElse(null.asInstanceOf[T]) + + override def update(newState: T): Unit = { + ttlTracker.onKeyUpdated() + keyToStateValue.put(ImplicitGroupingKeyTracker.getImplicitKeyOption.get, newState) + } + + override def clear(): Unit = + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) +} + +/** In-memory implementation of ListState. */ +class InMemoryListState[T](clock: Clock, ttl: TTLConfig) extends ListState[T] { + private val keyToStateValue = mutable.Map[Any, mutable.ArrayBuffer[T]]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getList: Option[mutable.ArrayBuffer[T]] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + + override def exists(): Boolean = getList.isDefined + + override def get(): Iterator[T] = { + getList.orElse(Some(mutable.ArrayBuffer.empty[T])).get.iterator + } + + override def put(newState: Array[T]): Unit = { + ttlTracker.onKeyUpdated() + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] ++ newState + ) + } + + override def appendValue(newState: T): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] + ) + } + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) += newState + } + + override def appendList(newState: Array[T]): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] + ) + } + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) ++= newState + } + + override def clear(): Unit = { + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } +} + +/** In-memory implementation of MapState. */ +class InMemoryMapState[K, V](clock: Clock, ttl: TTLConfig) extends MapState[K, V] { + private val keyToStateValue = mutable.Map[Any, mutable.HashMap[K, V]]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getMap: Option[mutable.HashMap[K, V]] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + + override def exists(): Boolean = getMap.isDefined + + override def getValue(key: K): V = { + getMap.flatMap(_.get(key)).getOrElse(null.asInstanceOf[V]) + } + + override def containsKey(key: K): Boolean = { + getMap.exists(_.contains(key)) + } + + override def updateValue(key: K, value: V): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.HashMap.empty[K, V] + ) + } + + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) += (key -> value) + } + + override def iterator(): Iterator[(K, V)] = { + getMap.map(_.iterator).getOrElse(Iterator.empty) + } + + override def keys(): Iterator[K] = { + getMap.map(_.keys.iterator).getOrElse(Iterator.empty) + } + + override def values(): Iterator[V] = { + getMap.map(_.values.iterator).getOrElse(Iterator.empty) + } + + override def removeKey(key: K): Unit = { + getMap.foreach(_.remove(key)) + } + + override def clear(): Unit = { + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } +} + +/** In-memory timers. */ +class InMemoryTimers { + private val keyToTimers = mutable.Map[Any, mutable.TreeSet[Long]]() + + def registerTimer(expiryTimestampMs: Long): Unit = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (!keyToTimers.contains(groupingKey)) { + keyToTimers.put(groupingKey, mutable.TreeSet[Long]()) + } + keyToTimers(groupingKey).add(expiryTimestampMs) + } + + def deleteTimer(expiryTimestampMs: Long): Unit = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (keyToTimers.contains(groupingKey)) { + keyToTimers(groupingKey).remove(expiryTimestampMs) + if (keyToTimers(groupingKey).isEmpty) { + keyToTimers.remove(groupingKey) + } + } + } + + def listTimers(): Iterator[Long] = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + keyToTimers.get(groupingKey) match { + case Some(timers) => timers.iterator + case None => Iterator.empty + } + } + + def getAllKeysWithTimers[K](): Iterator[K] = { + keyToTimers.keys.iterator.map(_.asInstanceOf[K]) + } +} + +/** + * In-memory implementation of StatefulProcessorHandle. + * + * Supports TTL and directly accessing state. + */ +class InMemoryStatefulProcessorHandle(val timeMode: TimeMode, val clock: Clock) + extends StatefulProcessorHandle { + private val states = mutable.Map[String, Any]() + val timers = new InMemoryTimers() + + override def getValueState[T]( + stateName: String, + valEncoder: Encoder[T], + ttlConfig: TTLConfig + ): ValueState[T] = { Review Comment: nit: one line above, after TTLConfig https://github.com/databricks/scala-style-guide ########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/testing/InMemoryStatefulProcessorHandle.scala: ########## @@ -0,0 +1,332 @@ +/* + * 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.operators.stateful.transformwithstate.testing + +import java.time.{Clock, Instant} +import java.util.UUID + +import scala.collection.mutable +import scala.reflect.ClassTag + +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.QueryInfoImpl +import org.apache.spark.sql.streaming.{ListState, MapState, QueryInfo, StatefulProcessorHandle, TimeMode, TTLConfig, ValueState} + +/** Helper to track expired keys. */ +class TtlTracker(val clock: Clock, ttl: TTLConfig) { + require(!ttl.ttlDuration.isNegative()) + private val keyToLastUpdatedTime = mutable.Map[Any, Instant]() + + def isKeyExpired(): Boolean = { + if (ttl.ttlDuration.isZero()) { + return false + } + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (!keyToLastUpdatedTime.contains(key)) { + return false + } + val expiration: Instant = keyToLastUpdatedTime.get(key).get.plus(ttl.ttlDuration) + return expiration.isBefore(clock.instant()) + } + + def onKeyUpdated(): Unit = { + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + keyToLastUpdatedTime.put(key, clock.instant()) + } +} + +/** In-memory implementation of ValueState. */ +class InMemoryValueState[T](clock: Clock, ttl: TTLConfig) extends ValueState[T] { + private val keyToStateValue = mutable.Map[Any, T]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getValue: Option[T] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + override def exists(): Boolean = getValue.isDefined + + override def get(): T = getValue.getOrElse(null.asInstanceOf[T]) + + override def update(newState: T): Unit = { + ttlTracker.onKeyUpdated() + keyToStateValue.put(ImplicitGroupingKeyTracker.getImplicitKeyOption.get, newState) + } + + override def clear(): Unit = + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) +} + +/** In-memory implementation of ListState. */ +class InMemoryListState[T](clock: Clock, ttl: TTLConfig) extends ListState[T] { + private val keyToStateValue = mutable.Map[Any, mutable.ArrayBuffer[T]]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getList: Option[mutable.ArrayBuffer[T]] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + + override def exists(): Boolean = getList.isDefined + + override def get(): Iterator[T] = { + getList.orElse(Some(mutable.ArrayBuffer.empty[T])).get.iterator + } + + override def put(newState: Array[T]): Unit = { + ttlTracker.onKeyUpdated() + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] ++ newState + ) + } + + override def appendValue(newState: T): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] + ) + } + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) += newState + } + + override def appendList(newState: Array[T]): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] + ) + } + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) ++= newState + } + + override def clear(): Unit = { + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } +} + +/** In-memory implementation of MapState. */ +class InMemoryMapState[K, V](clock: Clock, ttl: TTLConfig) extends MapState[K, V] { + private val keyToStateValue = mutable.Map[Any, mutable.HashMap[K, V]]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getMap: Option[mutable.HashMap[K, V]] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + + override def exists(): Boolean = getMap.isDefined + + override def getValue(key: K): V = { + getMap.flatMap(_.get(key)).getOrElse(null.asInstanceOf[V]) + } + + override def containsKey(key: K): Boolean = { + getMap.exists(_.contains(key)) + } + + override def updateValue(key: K, value: V): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.HashMap.empty[K, V] + ) + } + + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) += (key -> value) + } + + override def iterator(): Iterator[(K, V)] = { + getMap.map(_.iterator).getOrElse(Iterator.empty) + } + + override def keys(): Iterator[K] = { + getMap.map(_.keys.iterator).getOrElse(Iterator.empty) + } + + override def values(): Iterator[V] = { + getMap.map(_.values.iterator).getOrElse(Iterator.empty) + } + + override def removeKey(key: K): Unit = { + getMap.foreach(_.remove(key)) + } + + override def clear(): Unit = { + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } +} + +/** In-memory timers. */ +class InMemoryTimers { + private val keyToTimers = mutable.Map[Any, mutable.TreeSet[Long]]() Review Comment: The actual implementation of timers is more likely `mutable.Map[Long, mutable.HashSet[Any]]`, so the evaluation of expired timers is ordered by timestamp. There is no contract of which timers should fire earlier if wall clock or watermark advance triggers multiple timers to expire, so this implementation doesn't break the thing. It's slightly different in terms of the actual behavior, but it's optional to make it be sync with the actual impl. I'll leave this to you. ########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/testing/InMemoryStatefulProcessorHandle.scala: ########## @@ -0,0 +1,332 @@ +/* + * 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.operators.stateful.transformwithstate.testing + +import java.time.{Clock, Instant} +import java.util.UUID + +import scala.collection.mutable +import scala.reflect.ClassTag + +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.QueryInfoImpl +import org.apache.spark.sql.streaming.{ListState, MapState, QueryInfo, StatefulProcessorHandle, TimeMode, TTLConfig, ValueState} + +/** Helper to track expired keys. */ +class TtlTracker(val clock: Clock, ttl: TTLConfig) { + require(!ttl.ttlDuration.isNegative()) + private val keyToLastUpdatedTime = mutable.Map[Any, Instant]() + + def isKeyExpired(): Boolean = { + if (ttl.ttlDuration.isZero()) { + return false + } + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (!keyToLastUpdatedTime.contains(key)) { + return false + } + val expiration: Instant = keyToLastUpdatedTime.get(key).get.plus(ttl.ttlDuration) + return expiration.isBefore(clock.instant()) + } + + def onKeyUpdated(): Unit = { + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + keyToLastUpdatedTime.put(key, clock.instant()) + } +} + +/** In-memory implementation of ValueState. */ +class InMemoryValueState[T](clock: Clock, ttl: TTLConfig) extends ValueState[T] { + private val keyToStateValue = mutable.Map[Any, T]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getValue: Option[T] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + override def exists(): Boolean = getValue.isDefined + + override def get(): T = getValue.getOrElse(null.asInstanceOf[T]) + + override def update(newState: T): Unit = { + ttlTracker.onKeyUpdated() + keyToStateValue.put(ImplicitGroupingKeyTracker.getImplicitKeyOption.get, newState) + } + + override def clear(): Unit = + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) +} + +/** In-memory implementation of ListState. */ +class InMemoryListState[T](clock: Clock, ttl: TTLConfig) extends ListState[T] { + private val keyToStateValue = mutable.Map[Any, mutable.ArrayBuffer[T]]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getList: Option[mutable.ArrayBuffer[T]] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + + override def exists(): Boolean = getList.isDefined + + override def get(): Iterator[T] = { + getList.orElse(Some(mutable.ArrayBuffer.empty[T])).get.iterator + } + + override def put(newState: Array[T]): Unit = { + ttlTracker.onKeyUpdated() + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] ++ newState + ) + } + + override def appendValue(newState: T): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] + ) + } + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) += newState + } + + override def appendList(newState: Array[T]): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] + ) + } + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) ++= newState + } + + override def clear(): Unit = { + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } +} + +/** In-memory implementation of MapState. */ +class InMemoryMapState[K, V](clock: Clock, ttl: TTLConfig) extends MapState[K, V] { + private val keyToStateValue = mutable.Map[Any, mutable.HashMap[K, V]]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getMap: Option[mutable.HashMap[K, V]] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + + override def exists(): Boolean = getMap.isDefined + + override def getValue(key: K): V = { + getMap.flatMap(_.get(key)).getOrElse(null.asInstanceOf[V]) + } + + override def containsKey(key: K): Boolean = { + getMap.exists(_.contains(key)) + } + + override def updateValue(key: K, value: V): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.HashMap.empty[K, V] + ) + } + + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) += (key -> value) + } + + override def iterator(): Iterator[(K, V)] = { + getMap.map(_.iterator).getOrElse(Iterator.empty) + } + + override def keys(): Iterator[K] = { + getMap.map(_.keys.iterator).getOrElse(Iterator.empty) + } + + override def values(): Iterator[V] = { + getMap.map(_.values.iterator).getOrElse(Iterator.empty) + } + + override def removeKey(key: K): Unit = { + getMap.foreach(_.remove(key)) + } + + override def clear(): Unit = { + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } +} + +/** In-memory timers. */ +class InMemoryTimers { + private val keyToTimers = mutable.Map[Any, mutable.TreeSet[Long]]() + + def registerTimer(expiryTimestampMs: Long): Unit = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (!keyToTimers.contains(groupingKey)) { + keyToTimers.put(groupingKey, mutable.TreeSet[Long]()) + } + keyToTimers(groupingKey).add(expiryTimestampMs) + } + + def deleteTimer(expiryTimestampMs: Long): Unit = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (keyToTimers.contains(groupingKey)) { + keyToTimers(groupingKey).remove(expiryTimestampMs) + if (keyToTimers(groupingKey).isEmpty) { + keyToTimers.remove(groupingKey) + } + } + } + + def listTimers(): Iterator[Long] = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + keyToTimers.get(groupingKey) match { + case Some(timers) => timers.iterator + case None => Iterator.empty + } + } + + def getAllKeysWithTimers[K](): Iterator[K] = { + keyToTimers.keys.iterator.map(_.asInstanceOf[K]) + } +} + +/** + * In-memory implementation of StatefulProcessorHandle. + * + * Supports TTL and directly accessing state. + */ +class InMemoryStatefulProcessorHandle(val timeMode: TimeMode, val clock: Clock) + extends StatefulProcessorHandle { + private val states = mutable.Map[String, Any]() + val timers = new InMemoryTimers() + + override def getValueState[T]( + stateName: String, + valEncoder: Encoder[T], + ttlConfig: TTLConfig + ): ValueState[T] = { + states + .getOrElseUpdate(stateName, new InMemoryValueState[T](clock, ttlConfig)) + .asInstanceOf[InMemoryValueState[T]] + } + + override def getValueState[T: Encoder](stateName: String, ttlConfig: TTLConfig): ValueState[T] = + getValueState(stateName, implicitly[Encoder[T]], ttlConfig) + + override def getListState[T]( + stateName: String, + valEncoder: Encoder[T], + ttlConfig: TTLConfig + ): ListState[T] = { + states + .getOrElseUpdate(stateName, new InMemoryListState[T](clock, ttlConfig)) + .asInstanceOf[InMemoryListState[T]] + } + + override def getListState[T: Encoder](stateName: String, ttlConfig: TTLConfig): ListState[T] = + getListState(stateName, implicitly[Encoder[T]], ttlConfig) + + override def getMapState[K, V]( + stateName: String, + userKeyEnc: Encoder[K], + valEncoder: Encoder[V], + ttlConfig: TTLConfig + ): MapState[K, V] = { + states + .getOrElseUpdate(stateName, new InMemoryMapState[K, V](clock, ttlConfig)) + .asInstanceOf[InMemoryMapState[K, V]] + } + + override def getMapState[K: Encoder, V: Encoder]( + stateName: String, + ttlConfig: TTLConfig): MapState[K, V] = + getMapState(stateName, implicitly[Encoder[K]], implicitly[Encoder[V]], ttlConfig) + + override def getQueryInfo(): QueryInfo = Review Comment: I understand this won't be used actively on user's state processor implementation, but maybe making this to be "val" to make the value of QueryInfo be consistent? ########## sql/core/src/main/scala/org/apache/spark/sql/streaming/TwsTester.scala: ########## @@ -0,0 +1,286 @@ +/* + * 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.{Clock, Instant, ZoneId} + +import scala.reflect.ClassTag + +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.testing.InMemoryStatefulProcessorHandle +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl} + +/** + * Testing utility for transformWithState stateful processors. + * + * This class enables unit testing of StatefulProcessor business logic by simulating the + * behavior of transformWithState. It processes input rows and returns output rows equivalent + * to those that would be produced by the processor in an actual Spark streaming query. + * + * '''Supported:''' + * - Processing input rows and producing output rows via `test()`. + * - Initial state setup via constructor parameter. + * - Direct state manipulation via `setValueState`, `setListState`, `setMapState`. + * - Direct state inspection via `peekValueState`, `peekListState`, `peekMapState`. + * - Timers in ProcessingTime mode (use `advanceProcessingTime` to fire timers). + * - Timers in EventTime mode (use `eventTimeExtractor` and `watermarkDelayMs` to configure; + * watermark advances automatically based on event times, or use `advanceWatermark` manually). + * - Late event filtering in EventTime mode (events older than the current watermark are dropped). + * - TTL for ValueState, ListState, and MapState (use ProcessingTime mode and + * `advanceProcessingTime` to test expiry). + * + * '''Testing EventTime Mode:''' + * To test with EventTime, provide `eventTimeExtractor` (a function extracting the event + * timestamp from each input row) and `watermarkDelayMs` (the watermark delay in milliseconds). + * The watermark is computed as `max(event_time_seen) - watermarkDelayMs` and is updated + * automatically after each `test()` call. Late events (with event time older than the current + * watermark) are filtered out before processing, matching production behavior. Timers with + * expiry time <= watermark will fire. You can also manually advance the watermark using + * `advanceWatermark()`. + * + * '''Use Cases:''' + * - '''Primary''': Unit testing business logic in `handleInputRows` implementations. + * - '''Not recommended''': End-to-end testing or performance testing - use actual Spark + * streaming queries for those scenarios. + * + * @param processor the StatefulProcessor to test. + * @param initialState initial state for each key as a list of (key, state) tuples. + * @param timeMode time mode (None, ProcessingTime or EventTime). + * @param outputMode output mode (Append, Update, or Complete). + * @param realTimeMode whether input rows should be processed one-by-one (separate call to + * handleInputRows) for each input row. + * @param eventTimeExtractor function to extract event time from input rows. Required if and + * only if timeMode is EventTime. + * @param watermarkDelayMs watermark delay in milliseconds. The watermark is computed as + * `max(event_time) - watermarkDelayMs`. Required if and only if timeMode is EventTime. + * @tparam K the type of grouping key. + * @tparam I the type of input rows. + * @tparam O the type of output rows. + * @since 4.0.2 + */ +class TwsTester[K, I, O]( + val processor: StatefulProcessor[K, I, O], + val initialState: List[(K, Any)] = List(), + val timeMode: TimeMode = TimeMode.None, + val outputMode: OutputMode = OutputMode.Append, + val realTimeMode: Boolean = false, + val eventTimeExtractor: I => Timestamp = null, + val watermarkDelayMs: Long = 0L) { + val clock: Clock = new Clock { + override def instant(): Instant = Instant.ofEpochMilli(currentProcessingTimeMs) + override def getZone: ZoneId = ZoneId.systemDefault() + override def withZone(zone: ZoneId): Clock = this + } + + private val handle = new InMemoryStatefulProcessorHandle(timeMode, clock) Review Comment: nit: We have similar name of the class for benchmarking. Probably good to have different name - maybe simply follow TwsTester since it's purposed to be only used for this test class? ########## sql/core/src/main/scala/org/apache/spark/sql/streaming/TwsTester.scala: ########## @@ -0,0 +1,286 @@ +/* + * 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.{Clock, Instant, ZoneId} + +import scala.reflect.ClassTag + +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.testing.InMemoryStatefulProcessorHandle +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl} + +/** + * Testing utility for transformWithState stateful processors. + * + * This class enables unit testing of StatefulProcessor business logic by simulating the + * behavior of transformWithState. It processes input rows and returns output rows equivalent + * to those that would be produced by the processor in an actual Spark streaming query. + * + * '''Supported:''' + * - Processing input rows and producing output rows via `test()`. + * - Initial state setup via constructor parameter. + * - Direct state manipulation via `setValueState`, `setListState`, `setMapState`. + * - Direct state inspection via `peekValueState`, `peekListState`, `peekMapState`. + * - Timers in ProcessingTime mode (use `advanceProcessingTime` to fire timers). + * - Timers in EventTime mode (use `eventTimeExtractor` and `watermarkDelayMs` to configure; + * watermark advances automatically based on event times, or use `advanceWatermark` manually). + * - Late event filtering in EventTime mode (events older than the current watermark are dropped). + * - TTL for ValueState, ListState, and MapState (use ProcessingTime mode and + * `advanceProcessingTime` to test expiry). + * + * '''Testing EventTime Mode:''' + * To test with EventTime, provide `eventTimeExtractor` (a function extracting the event + * timestamp from each input row) and `watermarkDelayMs` (the watermark delay in milliseconds). + * The watermark is computed as `max(event_time_seen) - watermarkDelayMs` and is updated + * automatically after each `test()` call. Late events (with event time older than the current + * watermark) are filtered out before processing, matching production behavior. Timers with + * expiry time <= watermark will fire. You can also manually advance the watermark using + * `advanceWatermark()`. + * + * '''Use Cases:''' + * - '''Primary''': Unit testing business logic in `handleInputRows` implementations. + * - '''Not recommended''': End-to-end testing or performance testing - use actual Spark + * streaming queries for those scenarios. + * + * @param processor the StatefulProcessor to test. + * @param initialState initial state for each key as a list of (key, state) tuples. + * @param timeMode time mode (None, ProcessingTime or EventTime). + * @param outputMode output mode (Append, Update, or Complete). + * @param realTimeMode whether input rows should be processed one-by-one (separate call to + * handleInputRows) for each input row. + * @param eventTimeExtractor function to extract event time from input rows. Required if and + * only if timeMode is EventTime. + * @param watermarkDelayMs watermark delay in milliseconds. The watermark is computed as + * `max(event_time) - watermarkDelayMs`. Required if and only if timeMode is EventTime. + * @tparam K the type of grouping key. + * @tparam I the type of input rows. + * @tparam O the type of output rows. + * @since 4.0.2 + */ +class TwsTester[K, I, O]( + val processor: StatefulProcessor[K, I, O], + val initialState: List[(K, Any)] = List(), Review Comment: Is it possible we have S in the type list and have `List[(K, S])` as the type of initialState, and provide the default value to `List[(K, Any])()`? I haven't tried so just to see the possibility of enforcing the type for initial state data as well. ########## sql/core/src/main/scala/org/apache/spark/sql/streaming/TwsTester.scala: ########## @@ -0,0 +1,286 @@ +/* + * 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.{Clock, Instant, ZoneId} + +import scala.reflect.ClassTag + +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.testing.InMemoryStatefulProcessorHandle +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl} + +/** + * Testing utility for transformWithState stateful processors. + * + * This class enables unit testing of StatefulProcessor business logic by simulating the + * behavior of transformWithState. It processes input rows and returns output rows equivalent + * to those that would be produced by the processor in an actual Spark streaming query. + * + * '''Supported:''' + * - Processing input rows and producing output rows via `test()`. + * - Initial state setup via constructor parameter. + * - Direct state manipulation via `setValueState`, `setListState`, `setMapState`. + * - Direct state inspection via `peekValueState`, `peekListState`, `peekMapState`. + * - Timers in ProcessingTime mode (use `advanceProcessingTime` to fire timers). + * - Timers in EventTime mode (use `eventTimeExtractor` and `watermarkDelayMs` to configure; + * watermark advances automatically based on event times, or use `advanceWatermark` manually). + * - Late event filtering in EventTime mode (events older than the current watermark are dropped). + * - TTL for ValueState, ListState, and MapState (use ProcessingTime mode and + * `advanceProcessingTime` to test expiry). + * + * '''Testing EventTime Mode:''' + * To test with EventTime, provide `eventTimeExtractor` (a function extracting the event + * timestamp from each input row) and `watermarkDelayMs` (the watermark delay in milliseconds). + * The watermark is computed as `max(event_time_seen) - watermarkDelayMs` and is updated + * automatically after each `test()` call. Late events (with event time older than the current + * watermark) are filtered out before processing, matching production behavior. Timers with + * expiry time <= watermark will fire. You can also manually advance the watermark using + * `advanceWatermark()`. + * + * '''Use Cases:''' + * - '''Primary''': Unit testing business logic in `handleInputRows` implementations. + * - '''Not recommended''': End-to-end testing or performance testing - use actual Spark + * streaming queries for those scenarios. + * + * @param processor the StatefulProcessor to test. + * @param initialState initial state for each key as a list of (key, state) tuples. + * @param timeMode time mode (None, ProcessingTime or EventTime). + * @param outputMode output mode (Append, Update, or Complete). + * @param realTimeMode whether input rows should be processed one-by-one (separate call to + * handleInputRows) for each input row. + * @param eventTimeExtractor function to extract event time from input rows. Required if and + * only if timeMode is EventTime. + * @param watermarkDelayMs watermark delay in milliseconds. The watermark is computed as + * `max(event_time) - watermarkDelayMs`. Required if and only if timeMode is EventTime. + * @tparam K the type of grouping key. + * @tparam I the type of input rows. + * @tparam O the type of output rows. + * @since 4.0.2 + */ +class TwsTester[K, I, O]( + val processor: StatefulProcessor[K, I, O], + val initialState: List[(K, Any)] = List(), + val timeMode: TimeMode = TimeMode.None, + val outputMode: OutputMode = OutputMode.Append, + val realTimeMode: Boolean = false, + val eventTimeExtractor: I => Timestamp = null, + val watermarkDelayMs: Long = 0L) { + val clock: Clock = new Clock { + override def instant(): Instant = Instant.ofEpochMilli(currentProcessingTimeMs) + override def getZone: ZoneId = ZoneId.systemDefault() + override def withZone(zone: ZoneId): Clock = this + } + + private val handle = new InMemoryStatefulProcessorHandle(timeMode, clock) + + if (timeMode == TimeMode.EventTime) { + require( + eventTimeExtractor != null, + "eventTimeExtractor must be provided when timeMode is EventTime." + ) + } + + processor.setHandle(handle) + processor.init(outputMode, timeMode) + processor match { + case p: StatefulProcessorWithInitialState[K @unchecked, I @unchecked, O @unchecked, s] => Review Comment: Good to know the type can be captured and be passed into the type info again. TIL. ########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/testing/InMemoryStatefulProcessorHandle.scala: ########## @@ -0,0 +1,332 @@ +/* + * 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.operators.stateful.transformwithstate.testing + +import java.time.{Clock, Instant} +import java.util.UUID + +import scala.collection.mutable +import scala.reflect.ClassTag + +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.QueryInfoImpl +import org.apache.spark.sql.streaming.{ListState, MapState, QueryInfo, StatefulProcessorHandle, TimeMode, TTLConfig, ValueState} + +/** Helper to track expired keys. */ +class TtlTracker(val clock: Clock, ttl: TTLConfig) { + require(!ttl.ttlDuration.isNegative()) + private val keyToLastUpdatedTime = mutable.Map[Any, Instant]() + + def isKeyExpired(): Boolean = { + if (ttl.ttlDuration.isZero()) { + return false + } + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (!keyToLastUpdatedTime.contains(key)) { + return false + } + val expiration: Instant = keyToLastUpdatedTime.get(key).get.plus(ttl.ttlDuration) + return expiration.isBefore(clock.instant()) + } + + def onKeyUpdated(): Unit = { + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + keyToLastUpdatedTime.put(key, clock.instant()) + } +} + +/** In-memory implementation of ValueState. */ +class InMemoryValueState[T](clock: Clock, ttl: TTLConfig) extends ValueState[T] { + private val keyToStateValue = mutable.Map[Any, T]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getValue: Option[T] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + override def exists(): Boolean = getValue.isDefined + + override def get(): T = getValue.getOrElse(null.asInstanceOf[T]) + + override def update(newState: T): Unit = { + ttlTracker.onKeyUpdated() + keyToStateValue.put(ImplicitGroupingKeyTracker.getImplicitKeyOption.get, newState) + } + + override def clear(): Unit = + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) +} + +/** In-memory implementation of ListState. */ +class InMemoryListState[T](clock: Clock, ttl: TTLConfig) extends ListState[T] { + private val keyToStateValue = mutable.Map[Any, mutable.ArrayBuffer[T]]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getList: Option[mutable.ArrayBuffer[T]] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + + override def exists(): Boolean = getList.isDefined + + override def get(): Iterator[T] = { + getList.orElse(Some(mutable.ArrayBuffer.empty[T])).get.iterator Review Comment: ```suggestion getList.getOrElse(mutable.ArrayBuffer.empty[T]).iterator ``` I guess this would work? ########## sql/core/src/main/scala/org/apache/spark/sql/streaming/TwsTester.scala: ########## @@ -0,0 +1,286 @@ +/* + * 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.{Clock, Instant, ZoneId} + +import scala.reflect.ClassTag + +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.testing.InMemoryStatefulProcessorHandle +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl} + +/** + * Testing utility for transformWithState stateful processors. + * + * This class enables unit testing of StatefulProcessor business logic by simulating the + * behavior of transformWithState. It processes input rows and returns output rows equivalent + * to those that would be produced by the processor in an actual Spark streaming query. + * + * '''Supported:''' + * - Processing input rows and producing output rows via `test()`. + * - Initial state setup via constructor parameter. + * - Direct state manipulation via `setValueState`, `setListState`, `setMapState`. + * - Direct state inspection via `peekValueState`, `peekListState`, `peekMapState`. + * - Timers in ProcessingTime mode (use `advanceProcessingTime` to fire timers). + * - Timers in EventTime mode (use `eventTimeExtractor` and `watermarkDelayMs` to configure; + * watermark advances automatically based on event times, or use `advanceWatermark` manually). Review Comment: > watermark advances automatically based on event times, or use `advanceWatermark` manually Both conditions taking effect to watermark lead confusion. Why not just picking the latter one? Is the former coupled with the implementation? ########## sql/core/src/main/scala/org/apache/spark/sql/streaming/TwsTester.scala: ########## @@ -0,0 +1,286 @@ +/* + * 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.{Clock, Instant, ZoneId} + +import scala.reflect.ClassTag + +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.testing.InMemoryStatefulProcessorHandle +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl} + +/** + * Testing utility for transformWithState stateful processors. + * + * This class enables unit testing of StatefulProcessor business logic by simulating the + * behavior of transformWithState. It processes input rows and returns output rows equivalent + * to those that would be produced by the processor in an actual Spark streaming query. + * + * '''Supported:''' + * - Processing input rows and producing output rows via `test()`. + * - Initial state setup via constructor parameter. + * - Direct state manipulation via `setValueState`, `setListState`, `setMapState`. + * - Direct state inspection via `peekValueState`, `peekListState`, `peekMapState`. + * - Timers in ProcessingTime mode (use `advanceProcessingTime` to fire timers). + * - Timers in EventTime mode (use `eventTimeExtractor` and `watermarkDelayMs` to configure; + * watermark advances automatically based on event times, or use `advanceWatermark` manually). + * - Late event filtering in EventTime mode (events older than the current watermark are dropped). + * - TTL for ValueState, ListState, and MapState (use ProcessingTime mode and + * `advanceProcessingTime` to test expiry). + * + * '''Testing EventTime Mode:''' + * To test with EventTime, provide `eventTimeExtractor` (a function extracting the event + * timestamp from each input row) and `watermarkDelayMs` (the watermark delay in milliseconds). + * The watermark is computed as `max(event_time_seen) - watermarkDelayMs` and is updated + * automatically after each `test()` call. Late events (with event time older than the current + * watermark) are filtered out before processing, matching production behavior. Timers with + * expiry time <= watermark will fire. You can also manually advance the watermark using + * `advanceWatermark()`. + * + * '''Use Cases:''' + * - '''Primary''': Unit testing business logic in `handleInputRows` implementations. + * - '''Not recommended''': End-to-end testing or performance testing - use actual Spark + * streaming queries for those scenarios. + * + * @param processor the StatefulProcessor to test. + * @param initialState initial state for each key as a list of (key, state) tuples. + * @param timeMode time mode (None, ProcessingTime or EventTime). + * @param outputMode output mode (Append, Update, or Complete). + * @param realTimeMode whether input rows should be processed one-by-one (separate call to + * handleInputRows) for each input row. + * @param eventTimeExtractor function to extract event time from input rows. Required if and + * only if timeMode is EventTime. + * @param watermarkDelayMs watermark delay in milliseconds. The watermark is computed as Review Comment: This is not needed if we just leave users to advance watermark by themselves. ########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/testing/InMemoryStatefulProcessorHandle.scala: ########## @@ -0,0 +1,332 @@ +/* + * 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.operators.stateful.transformwithstate.testing + +import java.time.{Clock, Instant} +import java.util.UUID + +import scala.collection.mutable +import scala.reflect.ClassTag + +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.QueryInfoImpl +import org.apache.spark.sql.streaming.{ListState, MapState, QueryInfo, StatefulProcessorHandle, TimeMode, TTLConfig, ValueState} + +/** Helper to track expired keys. */ +class TtlTracker(val clock: Clock, ttl: TTLConfig) { + require(!ttl.ttlDuration.isNegative()) + private val keyToLastUpdatedTime = mutable.Map[Any, Instant]() + + def isKeyExpired(): Boolean = { + if (ttl.ttlDuration.isZero()) { + return false + } + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (!keyToLastUpdatedTime.contains(key)) { + return false + } + val expiration: Instant = keyToLastUpdatedTime.get(key).get.plus(ttl.ttlDuration) + return expiration.isBefore(clock.instant()) + } + + def onKeyUpdated(): Unit = { + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + keyToLastUpdatedTime.put(key, clock.instant()) + } +} + +/** In-memory implementation of ValueState. */ +class InMemoryValueState[T](clock: Clock, ttl: TTLConfig) extends ValueState[T] { + private val keyToStateValue = mutable.Map[Any, T]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getValue: Option[T] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + override def exists(): Boolean = getValue.isDefined + + override def get(): T = getValue.getOrElse(null.asInstanceOf[T]) + + override def update(newState: T): Unit = { + ttlTracker.onKeyUpdated() + keyToStateValue.put(ImplicitGroupingKeyTracker.getImplicitKeyOption.get, newState) + } + + override def clear(): Unit = + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) +} + +/** In-memory implementation of ListState. */ Review Comment: For TTL, ListState and MapState track every "element" so the way we implement the TTL is not the same with the behavior of TWS. That said, I'm open to leave the TTL as unsupported - the testing tool is mostly to test the implementation of users' stateful processor, not the TWS itself. Still, we could check whether the TTL config is set properly or not from implementing TTL, so let's evaluate the complexity with this and make a decision. ########## sql/core/src/main/scala/org/apache/spark/sql/streaming/TwsTester.scala: ########## @@ -0,0 +1,286 @@ +/* + * 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.{Clock, Instant, ZoneId} + +import scala.reflect.ClassTag + +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.testing.InMemoryStatefulProcessorHandle +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl} + +/** + * Testing utility for transformWithState stateful processors. + * + * This class enables unit testing of StatefulProcessor business logic by simulating the + * behavior of transformWithState. It processes input rows and returns output rows equivalent + * to those that would be produced by the processor in an actual Spark streaming query. + * + * '''Supported:''' + * - Processing input rows and producing output rows via `test()`. + * - Initial state setup via constructor parameter. + * - Direct state manipulation via `setValueState`, `setListState`, `setMapState`. + * - Direct state inspection via `peekValueState`, `peekListState`, `peekMapState`. + * - Timers in ProcessingTime mode (use `advanceProcessingTime` to fire timers). + * - Timers in EventTime mode (use `eventTimeExtractor` and `watermarkDelayMs` to configure; + * watermark advances automatically based on event times, or use `advanceWatermark` manually). + * - Late event filtering in EventTime mode (events older than the current watermark are dropped). + * - TTL for ValueState, ListState, and MapState (use ProcessingTime mode and + * `advanceProcessingTime` to test expiry). + * + * '''Testing EventTime Mode:''' + * To test with EventTime, provide `eventTimeExtractor` (a function extracting the event + * timestamp from each input row) and `watermarkDelayMs` (the watermark delay in milliseconds). + * The watermark is computed as `max(event_time_seen) - watermarkDelayMs` and is updated + * automatically after each `test()` call. Late events (with event time older than the current + * watermark) are filtered out before processing, matching production behavior. Timers with + * expiry time <= watermark will fire. You can also manually advance the watermark using + * `advanceWatermark()`. + * + * '''Use Cases:''' + * - '''Primary''': Unit testing business logic in `handleInputRows` implementations. + * - '''Not recommended''': End-to-end testing or performance testing - use actual Spark + * streaming queries for those scenarios. + * + * @param processor the StatefulProcessor to test. + * @param initialState initial state for each key as a list of (key, state) tuples. + * @param timeMode time mode (None, ProcessingTime or EventTime). + * @param outputMode output mode (Append, Update, or Complete). + * @param realTimeMode whether input rows should be processed one-by-one (separate call to + * handleInputRows) for each input row. + * @param eventTimeExtractor function to extract event time from input rows. Required if and + * only if timeMode is EventTime. + * @param watermarkDelayMs watermark delay in milliseconds. The watermark is computed as + * `max(event_time) - watermarkDelayMs`. Required if and only if timeMode is EventTime. + * @tparam K the type of grouping key. + * @tparam I the type of input rows. + * @tparam O the type of output rows. + * @since 4.0.2 + */ +class TwsTester[K, I, O]( + val processor: StatefulProcessor[K, I, O], + val initialState: List[(K, Any)] = List(), + val timeMode: TimeMode = TimeMode.None, + val outputMode: OutputMode = OutputMode.Append, + val realTimeMode: Boolean = false, + val eventTimeExtractor: I => Timestamp = null, + val watermarkDelayMs: Long = 0L) { + val clock: Clock = new Clock { + override def instant(): Instant = Instant.ofEpochMilli(currentProcessingTimeMs) + override def getZone: ZoneId = ZoneId.systemDefault() + override def withZone(zone: ZoneId): Clock = this + } + + private val handle = new InMemoryStatefulProcessorHandle(timeMode, clock) + + if (timeMode == TimeMode.EventTime) { + require( + eventTimeExtractor != null, + "eventTimeExtractor must be provided when timeMode is EventTime." + ) + } + + processor.setHandle(handle) + processor.init(outputMode, timeMode) + processor match { + case p: StatefulProcessorWithInitialState[K @unchecked, I @unchecked, O @unchecked, s] => + handleInitialState[s]() + case _ => + require( + initialState.isEmpty, + "Passed initial state, but the stateful processor doesn't support initial state." + ) + } + + private def handleInitialState[S](): Unit = { + val p = processor.asInstanceOf[StatefulProcessorWithInitialState[K, I, O, S]] + initialState.foreach { + case (key, state) => + ImplicitGroupingKeyTracker.setImplicitKey(key) + p.handleInitialState(key, state.asInstanceOf[S], getTimerValues()) + } + } + + /** + * Processes input rows for a single key through the stateful processor. + * + * In EventTime mode, late events (with event time older than the current watermark) are + * filtered out before processing. After processing, the watermark is updated based on + * the maximum event time seen (using `eventTimeExtractor`) minus `watermarkDelayMs`, + * and any expired timers are fired. + * + * @param key the grouping key + * @param values input rows to process + * @return all output rows produced by the processor (including any from expired timers + * in EventTime mode) + */ + def test(key: K, values: List[I]): List[O] = { + if (realTimeMode) { + values.flatMap(value => testInternal(key, List(value))).toList + } else { + testInternal(key, values) + } + } + + private def testInternal(key: K, values: List[I]): List[O] = { + ImplicitGroupingKeyTracker.setImplicitKey(key) + val timerValues = getTimerValues() + val filteredValues = filterLateEvents(values) + var result: List[O] = + processor.handleInputRows(key, filteredValues.iterator, timerValues).toList + if (timeMode == TimeMode.EventTime()) { + updateWatermarkFromEventTime(values) + result ++= handleExpiredTimers() Review Comment: advancing watermark manually will separate the output for processing inputs and processing timers clearly and help users to verify the output easier. ########## sql/core/src/main/scala/org/apache/spark/sql/streaming/TwsTester.scala: ########## @@ -0,0 +1,286 @@ +/* + * 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.{Clock, Instant, ZoneId} + +import scala.reflect.ClassTag + +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.testing.InMemoryStatefulProcessorHandle +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl} + +/** + * Testing utility for transformWithState stateful processors. + * + * This class enables unit testing of StatefulProcessor business logic by simulating the + * behavior of transformWithState. It processes input rows and returns output rows equivalent + * to those that would be produced by the processor in an actual Spark streaming query. + * + * '''Supported:''' + * - Processing input rows and producing output rows via `test()`. + * - Initial state setup via constructor parameter. + * - Direct state manipulation via `setValueState`, `setListState`, `setMapState`. + * - Direct state inspection via `peekValueState`, `peekListState`, `peekMapState`. + * - Timers in ProcessingTime mode (use `advanceProcessingTime` to fire timers). + * - Timers in EventTime mode (use `eventTimeExtractor` and `watermarkDelayMs` to configure; + * watermark advances automatically based on event times, or use `advanceWatermark` manually). + * - Late event filtering in EventTime mode (events older than the current watermark are dropped). + * - TTL for ValueState, ListState, and MapState (use ProcessingTime mode and + * `advanceProcessingTime` to test expiry). + * + * '''Testing EventTime Mode:''' + * To test with EventTime, provide `eventTimeExtractor` (a function extracting the event + * timestamp from each input row) and `watermarkDelayMs` (the watermark delay in milliseconds). + * The watermark is computed as `max(event_time_seen) - watermarkDelayMs` and is updated + * automatically after each `test()` call. Late events (with event time older than the current + * watermark) are filtered out before processing, matching production behavior. Timers with + * expiry time <= watermark will fire. You can also manually advance the watermark using + * `advanceWatermark()`. + * + * '''Use Cases:''' + * - '''Primary''': Unit testing business logic in `handleInputRows` implementations. + * - '''Not recommended''': End-to-end testing or performance testing - use actual Spark + * streaming queries for those scenarios. + * + * @param processor the StatefulProcessor to test. + * @param initialState initial state for each key as a list of (key, state) tuples. + * @param timeMode time mode (None, ProcessingTime or EventTime). + * @param outputMode output mode (Append, Update, or Complete). + * @param realTimeMode whether input rows should be processed one-by-one (separate call to + * handleInputRows) for each input row. + * @param eventTimeExtractor function to extract event time from input rows. Required if and + * only if timeMode is EventTime. + * @param watermarkDelayMs watermark delay in milliseconds. The watermark is computed as + * `max(event_time) - watermarkDelayMs`. Required if and only if timeMode is EventTime. + * @tparam K the type of grouping key. + * @tparam I the type of input rows. + * @tparam O the type of output rows. + * @since 4.0.2 + */ +class TwsTester[K, I, O]( + val processor: StatefulProcessor[K, I, O], + val initialState: List[(K, Any)] = List(), + val timeMode: TimeMode = TimeMode.None, + val outputMode: OutputMode = OutputMode.Append, + val realTimeMode: Boolean = false, + val eventTimeExtractor: I => Timestamp = null, + val watermarkDelayMs: Long = 0L) { + val clock: Clock = new Clock { + override def instant(): Instant = Instant.ofEpochMilli(currentProcessingTimeMs) + override def getZone: ZoneId = ZoneId.systemDefault() + override def withZone(zone: ZoneId): Clock = this + } + + private val handle = new InMemoryStatefulProcessorHandle(timeMode, clock) + + if (timeMode == TimeMode.EventTime) { + require( + eventTimeExtractor != null, + "eventTimeExtractor must be provided when timeMode is EventTime." + ) + } + + processor.setHandle(handle) + processor.init(outputMode, timeMode) + processor match { + case p: StatefulProcessorWithInitialState[K @unchecked, I @unchecked, O @unchecked, s] => + handleInitialState[s]() + case _ => + require( + initialState.isEmpty, + "Passed initial state, but the stateful processor doesn't support initial state." Review Comment: nit: initial state is given (or provided) ########## sql/core/src/main/scala/org/apache/spark/sql/streaming/TwsTester.scala: ########## @@ -0,0 +1,286 @@ +/* + * 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.{Clock, Instant, ZoneId} + +import scala.reflect.ClassTag + +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.testing.InMemoryStatefulProcessorHandle +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl} + +/** + * Testing utility for transformWithState stateful processors. + * + * This class enables unit testing of StatefulProcessor business logic by simulating the + * behavior of transformWithState. It processes input rows and returns output rows equivalent + * to those that would be produced by the processor in an actual Spark streaming query. + * + * '''Supported:''' + * - Processing input rows and producing output rows via `test()`. + * - Initial state setup via constructor parameter. + * - Direct state manipulation via `setValueState`, `setListState`, `setMapState`. + * - Direct state inspection via `peekValueState`, `peekListState`, `peekMapState`. + * - Timers in ProcessingTime mode (use `advanceProcessingTime` to fire timers). + * - Timers in EventTime mode (use `eventTimeExtractor` and `watermarkDelayMs` to configure; + * watermark advances automatically based on event times, or use `advanceWatermark` manually). + * - Late event filtering in EventTime mode (events older than the current watermark are dropped). Review Comment: While I think this is still needed for correctness perspective, we need to be very clear about "what" users will test with this class. It's not a TWS operator impl - it's their state processor. ########## sql/core/src/main/scala/org/apache/spark/sql/streaming/TwsTester.scala: ########## @@ -0,0 +1,286 @@ +/* + * 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.{Clock, Instant, ZoneId} + +import scala.reflect.ClassTag + +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.testing.InMemoryStatefulProcessorHandle +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl} + +/** + * Testing utility for transformWithState stateful processors. + * + * This class enables unit testing of StatefulProcessor business logic by simulating the + * behavior of transformWithState. It processes input rows and returns output rows equivalent + * to those that would be produced by the processor in an actual Spark streaming query. + * + * '''Supported:''' + * - Processing input rows and producing output rows via `test()`. + * - Initial state setup via constructor parameter. + * - Direct state manipulation via `setValueState`, `setListState`, `setMapState`. + * - Direct state inspection via `peekValueState`, `peekListState`, `peekMapState`. + * - Timers in ProcessingTime mode (use `advanceProcessingTime` to fire timers). + * - Timers in EventTime mode (use `eventTimeExtractor` and `watermarkDelayMs` to configure; + * watermark advances automatically based on event times, or use `advanceWatermark` manually). + * - Late event filtering in EventTime mode (events older than the current watermark are dropped). + * - TTL for ValueState, ListState, and MapState (use ProcessingTime mode and + * `advanceProcessingTime` to test expiry). + * + * '''Testing EventTime Mode:''' + * To test with EventTime, provide `eventTimeExtractor` (a function extracting the event + * timestamp from each input row) and `watermarkDelayMs` (the watermark delay in milliseconds). + * The watermark is computed as `max(event_time_seen) - watermarkDelayMs` and is updated + * automatically after each `test()` call. Late events (with event time older than the current + * watermark) are filtered out before processing, matching production behavior. Timers with + * expiry time <= watermark will fire. You can also manually advance the watermark using + * `advanceWatermark()`. + * + * '''Use Cases:''' + * - '''Primary''': Unit testing business logic in `handleInputRows` implementations. + * - '''Not recommended''': End-to-end testing or performance testing - use actual Spark + * streaming queries for those scenarios. + * + * @param processor the StatefulProcessor to test. + * @param initialState initial state for each key as a list of (key, state) tuples. + * @param timeMode time mode (None, ProcessingTime or EventTime). + * @param outputMode output mode (Append, Update, or Complete). Review Comment: @anishshri-db Just to confirm, do we leave complete mode as a supported list in TWS? ########## sql/core/src/main/scala/org/apache/spark/sql/streaming/TwsTester.scala: ########## @@ -0,0 +1,286 @@ +/* + * 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.{Clock, Instant, ZoneId} + +import scala.reflect.ClassTag + +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.testing.InMemoryStatefulProcessorHandle +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl} + +/** + * Testing utility for transformWithState stateful processors. + * + * This class enables unit testing of StatefulProcessor business logic by simulating the + * behavior of transformWithState. It processes input rows and returns output rows equivalent + * to those that would be produced by the processor in an actual Spark streaming query. + * + * '''Supported:''' + * - Processing input rows and producing output rows via `test()`. + * - Initial state setup via constructor parameter. + * - Direct state manipulation via `setValueState`, `setListState`, `setMapState`. + * - Direct state inspection via `peekValueState`, `peekListState`, `peekMapState`. + * - Timers in ProcessingTime mode (use `advanceProcessingTime` to fire timers). + * - Timers in EventTime mode (use `eventTimeExtractor` and `watermarkDelayMs` to configure; + * watermark advances automatically based on event times, or use `advanceWatermark` manually). + * - Late event filtering in EventTime mode (events older than the current watermark are dropped). + * - TTL for ValueState, ListState, and MapState (use ProcessingTime mode and + * `advanceProcessingTime` to test expiry). + * + * '''Testing EventTime Mode:''' + * To test with EventTime, provide `eventTimeExtractor` (a function extracting the event + * timestamp from each input row) and `watermarkDelayMs` (the watermark delay in milliseconds). + * The watermark is computed as `max(event_time_seen) - watermarkDelayMs` and is updated + * automatically after each `test()` call. Late events (with event time older than the current + * watermark) are filtered out before processing, matching production behavior. Timers with + * expiry time <= watermark will fire. You can also manually advance the watermark using + * `advanceWatermark()`. + * + * '''Use Cases:''' + * - '''Primary''': Unit testing business logic in `handleInputRows` implementations. + * - '''Not recommended''': End-to-end testing or performance testing - use actual Spark + * streaming queries for those scenarios. + * + * @param processor the StatefulProcessor to test. + * @param initialState initial state for each key as a list of (key, state) tuples. + * @param timeMode time mode (None, ProcessingTime or EventTime). + * @param outputMode output mode (Append, Update, or Complete). + * @param realTimeMode whether input rows should be processed one-by-one (separate call to + * handleInputRows) for each input row. + * @param eventTimeExtractor function to extract event time from input rows. Required if and + * only if timeMode is EventTime. + * @param watermarkDelayMs watermark delay in milliseconds. The watermark is computed as + * `max(event_time) - watermarkDelayMs`. Required if and only if timeMode is EventTime. + * @tparam K the type of grouping key. + * @tparam I the type of input rows. + * @tparam O the type of output rows. + * @since 4.0.2 + */ +class TwsTester[K, I, O]( + val processor: StatefulProcessor[K, I, O], + val initialState: List[(K, Any)] = List(), + val timeMode: TimeMode = TimeMode.None, + val outputMode: OutputMode = OutputMode.Append, + val realTimeMode: Boolean = false, + val eventTimeExtractor: I => Timestamp = null, + val watermarkDelayMs: Long = 0L) { + val clock: Clock = new Clock { + override def instant(): Instant = Instant.ofEpochMilli(currentProcessingTimeMs) + override def getZone: ZoneId = ZoneId.systemDefault() + override def withZone(zone: ZoneId): Clock = this + } + + private val handle = new InMemoryStatefulProcessorHandle(timeMode, clock) + + if (timeMode == TimeMode.EventTime) { + require( + eventTimeExtractor != null, + "eventTimeExtractor must be provided when timeMode is EventTime." + ) + } + + processor.setHandle(handle) + processor.init(outputMode, timeMode) + processor match { + case p: StatefulProcessorWithInitialState[K @unchecked, I @unchecked, O @unchecked, s] => + handleInitialState[s]() + case _ => + require( + initialState.isEmpty, + "Passed initial state, but the stateful processor doesn't support initial state." + ) + } + + private def handleInitialState[S](): Unit = { + val p = processor.asInstanceOf[StatefulProcessorWithInitialState[K, I, O, S]] + initialState.foreach { + case (key, state) => + ImplicitGroupingKeyTracker.setImplicitKey(key) + p.handleInitialState(key, state.asInstanceOf[S], getTimerValues()) + } + } + + /** + * Processes input rows for a single key through the stateful processor. + * + * In EventTime mode, late events (with event time older than the current watermark) are + * filtered out before processing. After processing, the watermark is updated based on + * the maximum event time seen (using `eventTimeExtractor`) minus `watermarkDelayMs`, + * and any expired timers are fired. + * + * @param key the grouping key + * @param values input rows to process + * @return all output rows produced by the processor (including any from expired timers + * in EventTime mode) + */ + def test(key: K, values: List[I]): List[O] = { + if (realTimeMode) { + values.flatMap(value => testInternal(key, List(value))).toList + } else { + testInternal(key, values) + } + } + + private def testInternal(key: K, values: List[I]): List[O] = { + ImplicitGroupingKeyTracker.setImplicitKey(key) + val timerValues = getTimerValues() + val filteredValues = filterLateEvents(values) + var result: List[O] = + processor.handleInputRows(key, filteredValues.iterator, timerValues).toList + if (timeMode == TimeMode.EventTime()) { + updateWatermarkFromEventTime(values) + result ++= handleExpiredTimers() + } + result + } + + /** Sets the value state for a given key. */ + def setValueState[T](stateName: String, key: K, value: T): Unit = { + ImplicitGroupingKeyTracker.setImplicitKey(key) + handle.setValueState[T](stateName, value) + } + + /** Retrieves the value state for a given key. */ + def peekValueState[T](stateName: String, key: K): Option[T] = { + ImplicitGroupingKeyTracker.setImplicitKey(key) + handle.peekValueState[T](stateName) + } + + /** Sets the list state for a given key. */ + def setListState[T](stateName: String, key: K, value: List[T])(implicit ct: ClassTag[T]): Unit = { + ImplicitGroupingKeyTracker.setImplicitKey(key) + handle.setListState[T](stateName, value) + } + + /** Retrieves the list state for a given key. */ + def peekListState[T](stateName: String, key: K): List[T] = { + ImplicitGroupingKeyTracker.setImplicitKey(key) + handle.peekListState[T](stateName) + } + + /** Sets the map state for a given key. */ + def setMapState[MK, MV](stateName: String, key: K, value: Map[MK, MV]): Unit = { + ImplicitGroupingKeyTracker.setImplicitKey(key) + handle.setMapState[MK, MV](stateName, value) + } + + /** Retrieves the map state for a given key. */ + def peekMapState[MK, MV](stateName: String, key: K): Map[MK, MV] = { + ImplicitGroupingKeyTracker.setImplicitKey(key) + handle.peekMapState[MK, MV](stateName) + } + + /** Deletes state for a given key. */ + def deleteState(stateName: String, key: K): Unit = { + ImplicitGroupingKeyTracker.setImplicitKey(key) + handle.deleteState(stateName) + } + + // Logic for dealing with timers. + private var currentProcessingTimeMs: Long = 0 + private var currentWatermarkMs: Long = 0 + + private def handleExpiredTimers(): List[O] = { + if (timeMode == TimeMode.None) { + return List() + } + val timerValues = getTimerValues() + val expiryThreshold = if (timeMode == TimeMode.ProcessingTime()) { + currentProcessingTimeMs + } else if (timeMode == TimeMode.EventTime()) { + currentWatermarkMs + } else { + 0L + } + + var ans: List[O] = List() + for (key <- handle.timers.getAllKeysWithTimers[K]()) { + ImplicitGroupingKeyTracker.setImplicitKey(key) + val expiredTimers: List[Long] = handle.listTimers().filter(_ <= expiryThreshold).toList + for (timerExpiryTimeMs <- expiredTimers) { + val expiredTimerInfo = new ExpiredTimerInfoImpl(Some(timerExpiryTimeMs)) + ans = ans ++ processor.handleExpiredTimer(key, timerValues, expiredTimerInfo).toList + handle.deleteTimer(timerExpiryTimeMs) + } + } + ans + } + + /** Review Comment: Maybe good to have the method to provide the current time? Or probably make this and the below method to handle absolute time instead of time diff. ########## sql/core/src/main/scala/org/apache/spark/sql/streaming/TwsTester.scala: ########## @@ -0,0 +1,286 @@ +/* + * 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.{Clock, Instant, ZoneId} + +import scala.reflect.ClassTag + +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.testing.InMemoryStatefulProcessorHandle +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl} + +/** + * Testing utility for transformWithState stateful processors. + * + * This class enables unit testing of StatefulProcessor business logic by simulating the + * behavior of transformWithState. It processes input rows and returns output rows equivalent + * to those that would be produced by the processor in an actual Spark streaming query. + * + * '''Supported:''' + * - Processing input rows and producing output rows via `test()`. + * - Initial state setup via constructor parameter. + * - Direct state manipulation via `setValueState`, `setListState`, `setMapState`. + * - Direct state inspection via `peekValueState`, `peekListState`, `peekMapState`. + * - Timers in ProcessingTime mode (use `advanceProcessingTime` to fire timers). + * - Timers in EventTime mode (use `eventTimeExtractor` and `watermarkDelayMs` to configure; + * watermark advances automatically based on event times, or use `advanceWatermark` manually). Review Comment: It's not just about confusion. Since we only support test() with a single key, it's not going to test the case where the watermark is advanced "after" multiple grouping keys have been processed, which is the default behavior of microbatch mode. I'd rather leave users to advance watermark for both 1) consistency with processing time timer and 2) closer to actual behavior. ########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/testing/InMemoryStatefulProcessorHandle.scala: ########## @@ -0,0 +1,332 @@ +/* + * 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.operators.stateful.transformwithstate.testing + +import java.time.{Clock, Instant} +import java.util.UUID + +import scala.collection.mutable +import scala.reflect.ClassTag + +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.QueryInfoImpl +import org.apache.spark.sql.streaming.{ListState, MapState, QueryInfo, StatefulProcessorHandle, TimeMode, TTLConfig, ValueState} + +/** Helper to track expired keys. */ +class TtlTracker(val clock: Clock, ttl: TTLConfig) { + require(!ttl.ttlDuration.isNegative()) + private val keyToLastUpdatedTime = mutable.Map[Any, Instant]() + + def isKeyExpired(): Boolean = { + if (ttl.ttlDuration.isZero()) { + return false + } + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (!keyToLastUpdatedTime.contains(key)) { + return false + } + val expiration: Instant = keyToLastUpdatedTime.get(key).get.plus(ttl.ttlDuration) + return expiration.isBefore(clock.instant()) + } + + def onKeyUpdated(): Unit = { + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + keyToLastUpdatedTime.put(key, clock.instant()) + } +} + +/** In-memory implementation of ValueState. */ +class InMemoryValueState[T](clock: Clock, ttl: TTLConfig) extends ValueState[T] { + private val keyToStateValue = mutable.Map[Any, T]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getValue: Option[T] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + override def exists(): Boolean = getValue.isDefined + + override def get(): T = getValue.getOrElse(null.asInstanceOf[T]) + + override def update(newState: T): Unit = { + ttlTracker.onKeyUpdated() + keyToStateValue.put(ImplicitGroupingKeyTracker.getImplicitKeyOption.get, newState) + } + + override def clear(): Unit = + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) +} + +/** In-memory implementation of ListState. */ +class InMemoryListState[T](clock: Clock, ttl: TTLConfig) extends ListState[T] { + private val keyToStateValue = mutable.Map[Any, mutable.ArrayBuffer[T]]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getList: Option[mutable.ArrayBuffer[T]] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + + override def exists(): Boolean = getList.isDefined + + override def get(): Iterator[T] = { + getList.orElse(Some(mutable.ArrayBuffer.empty[T])).get.iterator + } + + override def put(newState: Array[T]): Unit = { + ttlTracker.onKeyUpdated() + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] ++ newState + ) + } + + override def appendValue(newState: T): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] + ) + } + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) += newState + } + + override def appendList(newState: Array[T]): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] + ) + } + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) ++= newState + } + + override def clear(): Unit = { + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } +} + +/** In-memory implementation of MapState. */ +class InMemoryMapState[K, V](clock: Clock, ttl: TTLConfig) extends MapState[K, V] { + private val keyToStateValue = mutable.Map[Any, mutable.HashMap[K, V]]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getMap: Option[mutable.HashMap[K, V]] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + + override def exists(): Boolean = getMap.isDefined + + override def getValue(key: K): V = { + getMap.flatMap(_.get(key)).getOrElse(null.asInstanceOf[V]) + } + + override def containsKey(key: K): Boolean = { + getMap.exists(_.contains(key)) + } + + override def updateValue(key: K, value: V): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.HashMap.empty[K, V] + ) + } + + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) += (key -> value) + } + + override def iterator(): Iterator[(K, V)] = { + getMap.map(_.iterator).getOrElse(Iterator.empty) + } + + override def keys(): Iterator[K] = { + getMap.map(_.keys.iterator).getOrElse(Iterator.empty) + } + + override def values(): Iterator[V] = { + getMap.map(_.values.iterator).getOrElse(Iterator.empty) + } + + override def removeKey(key: K): Unit = { + getMap.foreach(_.remove(key)) + } + + override def clear(): Unit = { + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } +} + +/** In-memory timers. */ +class InMemoryTimers { + private val keyToTimers = mutable.Map[Any, mutable.TreeSet[Long]]() + + def registerTimer(expiryTimestampMs: Long): Unit = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (!keyToTimers.contains(groupingKey)) { + keyToTimers.put(groupingKey, mutable.TreeSet[Long]()) + } + keyToTimers(groupingKey).add(expiryTimestampMs) + } + + def deleteTimer(expiryTimestampMs: Long): Unit = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (keyToTimers.contains(groupingKey)) { + keyToTimers(groupingKey).remove(expiryTimestampMs) + if (keyToTimers(groupingKey).isEmpty) { + keyToTimers.remove(groupingKey) + } + } + } + + def listTimers(): Iterator[Long] = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + keyToTimers.get(groupingKey) match { + case Some(timers) => timers.iterator + case None => Iterator.empty + } + } + + def getAllKeysWithTimers[K](): Iterator[K] = { + keyToTimers.keys.iterator.map(_.asInstanceOf[K]) + } +} + +/** + * In-memory implementation of StatefulProcessorHandle. + * + * Supports TTL and directly accessing state. + */ +class InMemoryStatefulProcessorHandle(val timeMode: TimeMode, val clock: Clock) + extends StatefulProcessorHandle { + private val states = mutable.Map[String, Any]() + val timers = new InMemoryTimers() + + override def getValueState[T]( + stateName: String, + valEncoder: Encoder[T], + ttlConfig: TTLConfig + ): ValueState[T] = { + states + .getOrElseUpdate(stateName, new InMemoryValueState[T](clock, ttlConfig)) + .asInstanceOf[InMemoryValueState[T]] + } + + override def getValueState[T: Encoder](stateName: String, ttlConfig: TTLConfig): ValueState[T] = + getValueState(stateName, implicitly[Encoder[T]], ttlConfig) + + override def getListState[T]( + stateName: String, + valEncoder: Encoder[T], + ttlConfig: TTLConfig + ): ListState[T] = { + states + .getOrElseUpdate(stateName, new InMemoryListState[T](clock, ttlConfig)) + .asInstanceOf[InMemoryListState[T]] + } + + override def getListState[T: Encoder](stateName: String, ttlConfig: TTLConfig): ListState[T] = + getListState(stateName, implicitly[Encoder[T]], ttlConfig) + + override def getMapState[K, V]( + stateName: String, + userKeyEnc: Encoder[K], + valEncoder: Encoder[V], + ttlConfig: TTLConfig + ): MapState[K, V] = { Review Comment: ditto ########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/testing/InMemoryStatefulProcessorHandle.scala: ########## @@ -0,0 +1,332 @@ +/* + * 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.operators.stateful.transformwithstate.testing + +import java.time.{Clock, Instant} +import java.util.UUID + +import scala.collection.mutable +import scala.reflect.ClassTag + +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.QueryInfoImpl +import org.apache.spark.sql.streaming.{ListState, MapState, QueryInfo, StatefulProcessorHandle, TimeMode, TTLConfig, ValueState} + +/** Helper to track expired keys. */ +class TtlTracker(val clock: Clock, ttl: TTLConfig) { + require(!ttl.ttlDuration.isNegative()) + private val keyToLastUpdatedTime = mutable.Map[Any, Instant]() + + def isKeyExpired(): Boolean = { + if (ttl.ttlDuration.isZero()) { + return false + } + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (!keyToLastUpdatedTime.contains(key)) { + return false + } + val expiration: Instant = keyToLastUpdatedTime.get(key).get.plus(ttl.ttlDuration) + return expiration.isBefore(clock.instant()) + } + + def onKeyUpdated(): Unit = { + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + keyToLastUpdatedTime.put(key, clock.instant()) + } +} + +/** In-memory implementation of ValueState. */ +class InMemoryValueState[T](clock: Clock, ttl: TTLConfig) extends ValueState[T] { + private val keyToStateValue = mutable.Map[Any, T]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getValue: Option[T] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + override def exists(): Boolean = getValue.isDefined + + override def get(): T = getValue.getOrElse(null.asInstanceOf[T]) + + override def update(newState: T): Unit = { + ttlTracker.onKeyUpdated() + keyToStateValue.put(ImplicitGroupingKeyTracker.getImplicitKeyOption.get, newState) + } + + override def clear(): Unit = + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) +} + +/** In-memory implementation of ListState. */ +class InMemoryListState[T](clock: Clock, ttl: TTLConfig) extends ListState[T] { + private val keyToStateValue = mutable.Map[Any, mutable.ArrayBuffer[T]]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getList: Option[mutable.ArrayBuffer[T]] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + + override def exists(): Boolean = getList.isDefined + + override def get(): Iterator[T] = { + getList.orElse(Some(mutable.ArrayBuffer.empty[T])).get.iterator + } + + override def put(newState: Array[T]): Unit = { + ttlTracker.onKeyUpdated() + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] ++ newState + ) + } + + override def appendValue(newState: T): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] + ) + } + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) += newState + } + + override def appendList(newState: Array[T]): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] + ) + } + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) ++= newState + } + + override def clear(): Unit = { + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } +} + +/** In-memory implementation of MapState. */ +class InMemoryMapState[K, V](clock: Clock, ttl: TTLConfig) extends MapState[K, V] { + private val keyToStateValue = mutable.Map[Any, mutable.HashMap[K, V]]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getMap: Option[mutable.HashMap[K, V]] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + + override def exists(): Boolean = getMap.isDefined + + override def getValue(key: K): V = { + getMap.flatMap(_.get(key)).getOrElse(null.asInstanceOf[V]) + } + + override def containsKey(key: K): Boolean = { + getMap.exists(_.contains(key)) + } + + override def updateValue(key: K, value: V): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.HashMap.empty[K, V] + ) + } + + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) += (key -> value) + } + + override def iterator(): Iterator[(K, V)] = { + getMap.map(_.iterator).getOrElse(Iterator.empty) + } + + override def keys(): Iterator[K] = { + getMap.map(_.keys.iterator).getOrElse(Iterator.empty) + } + + override def values(): Iterator[V] = { + getMap.map(_.values.iterator).getOrElse(Iterator.empty) + } + + override def removeKey(key: K): Unit = { + getMap.foreach(_.remove(key)) + } + + override def clear(): Unit = { + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } +} + +/** In-memory timers. */ +class InMemoryTimers { + private val keyToTimers = mutable.Map[Any, mutable.TreeSet[Long]]() + + def registerTimer(expiryTimestampMs: Long): Unit = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (!keyToTimers.contains(groupingKey)) { + keyToTimers.put(groupingKey, mutable.TreeSet[Long]()) + } + keyToTimers(groupingKey).add(expiryTimestampMs) + } + + def deleteTimer(expiryTimestampMs: Long): Unit = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (keyToTimers.contains(groupingKey)) { + keyToTimers(groupingKey).remove(expiryTimestampMs) + if (keyToTimers(groupingKey).isEmpty) { + keyToTimers.remove(groupingKey) + } + } + } + + def listTimers(): Iterator[Long] = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + keyToTimers.get(groupingKey) match { + case Some(timers) => timers.iterator + case None => Iterator.empty + } + } + + def getAllKeysWithTimers[K](): Iterator[K] = { + keyToTimers.keys.iterator.map(_.asInstanceOf[K]) + } +} + +/** + * In-memory implementation of StatefulProcessorHandle. + * + * Supports TTL and directly accessing state. + */ +class InMemoryStatefulProcessorHandle(val timeMode: TimeMode, val clock: Clock) + extends StatefulProcessorHandle { + private val states = mutable.Map[String, Any]() + val timers = new InMemoryTimers() + + override def getValueState[T]( + stateName: String, + valEncoder: Encoder[T], + ttlConfig: TTLConfig + ): ValueState[T] = { + states + .getOrElseUpdate(stateName, new InMemoryValueState[T](clock, ttlConfig)) + .asInstanceOf[InMemoryValueState[T]] + } + + override def getValueState[T: Encoder](stateName: String, ttlConfig: TTLConfig): ValueState[T] = + getValueState(stateName, implicitly[Encoder[T]], ttlConfig) + + override def getListState[T]( + stateName: String, + valEncoder: Encoder[T], + ttlConfig: TTLConfig + ): ListState[T] = { + states + .getOrElseUpdate(stateName, new InMemoryListState[T](clock, ttlConfig)) + .asInstanceOf[InMemoryListState[T]] + } + + override def getListState[T: Encoder](stateName: String, ttlConfig: TTLConfig): ListState[T] = + getListState(stateName, implicitly[Encoder[T]], ttlConfig) + + override def getMapState[K, V]( + stateName: String, + userKeyEnc: Encoder[K], + valEncoder: Encoder[V], + ttlConfig: TTLConfig + ): MapState[K, V] = { + states + .getOrElseUpdate(stateName, new InMemoryMapState[K, V](clock, ttlConfig)) + .asInstanceOf[InMemoryMapState[K, V]] + } + + override def getMapState[K: Encoder, V: Encoder]( + stateName: String, + ttlConfig: TTLConfig): MapState[K, V] = + getMapState(stateName, implicitly[Encoder[K]], implicitly[Encoder[V]], ttlConfig) + + override def getQueryInfo(): QueryInfo = + new QueryInfoImpl(UUID.randomUUID(), UUID.randomUUID(), 0L) + + override def registerTimer(expiryTimestampMs: Long): Unit = { + require(timeMode != TimeMode.None, "Timers are not supported with TimeMode.None.") + timers.registerTimer(expiryTimestampMs) + } + + override def deleteTimer(expiryTimestampMs: Long): Unit = { + require(timeMode != TimeMode.None, "Timers are not supported with TimeMode.None.") + timers.deleteTimer(expiryTimestampMs) + } + + override def listTimers(): Iterator[Long] = { + require(timeMode != TimeMode.None, "Timers are not supported with TimeMode.None.") + timers.listTimers() + } + + override def deleteIfExists(stateName: String): Unit = states.remove(stateName) + + def setValueState[T](stateName: String, value: T): Unit = { Review Comment: nit: I feel like `updateValueState` would be slightly better to avoid being symmetric with getValueState. ########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/testing/InMemoryStatefulProcessorHandle.scala: ########## @@ -0,0 +1,332 @@ +/* + * 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.operators.stateful.transformwithstate.testing + +import java.time.{Clock, Instant} +import java.util.UUID + +import scala.collection.mutable +import scala.reflect.ClassTag + +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.QueryInfoImpl +import org.apache.spark.sql.streaming.{ListState, MapState, QueryInfo, StatefulProcessorHandle, TimeMode, TTLConfig, ValueState} + +/** Helper to track expired keys. */ +class TtlTracker(val clock: Clock, ttl: TTLConfig) { + require(!ttl.ttlDuration.isNegative()) + private val keyToLastUpdatedTime = mutable.Map[Any, Instant]() + + def isKeyExpired(): Boolean = { + if (ttl.ttlDuration.isZero()) { + return false + } + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (!keyToLastUpdatedTime.contains(key)) { + return false + } + val expiration: Instant = keyToLastUpdatedTime.get(key).get.plus(ttl.ttlDuration) + return expiration.isBefore(clock.instant()) + } + + def onKeyUpdated(): Unit = { + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + keyToLastUpdatedTime.put(key, clock.instant()) + } +} + +/** In-memory implementation of ValueState. */ +class InMemoryValueState[T](clock: Clock, ttl: TTLConfig) extends ValueState[T] { + private val keyToStateValue = mutable.Map[Any, T]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getValue: Option[T] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + override def exists(): Boolean = getValue.isDefined + + override def get(): T = getValue.getOrElse(null.asInstanceOf[T]) + + override def update(newState: T): Unit = { + ttlTracker.onKeyUpdated() + keyToStateValue.put(ImplicitGroupingKeyTracker.getImplicitKeyOption.get, newState) + } + + override def clear(): Unit = + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) +} + +/** In-memory implementation of ListState. */ +class InMemoryListState[T](clock: Clock, ttl: TTLConfig) extends ListState[T] { + private val keyToStateValue = mutable.Map[Any, mutable.ArrayBuffer[T]]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getList: Option[mutable.ArrayBuffer[T]] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + + override def exists(): Boolean = getList.isDefined + + override def get(): Iterator[T] = { + getList.orElse(Some(mutable.ArrayBuffer.empty[T])).get.iterator + } + + override def put(newState: Array[T]): Unit = { + ttlTracker.onKeyUpdated() + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] ++ newState + ) + } + + override def appendValue(newState: T): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] + ) + } + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) += newState + } + + override def appendList(newState: Array[T]): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] + ) + } + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) ++= newState + } + + override def clear(): Unit = { + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } +} + +/** In-memory implementation of MapState. */ +class InMemoryMapState[K, V](clock: Clock, ttl: TTLConfig) extends MapState[K, V] { + private val keyToStateValue = mutable.Map[Any, mutable.HashMap[K, V]]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getMap: Option[mutable.HashMap[K, V]] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + + override def exists(): Boolean = getMap.isDefined + + override def getValue(key: K): V = { + getMap.flatMap(_.get(key)).getOrElse(null.asInstanceOf[V]) + } + + override def containsKey(key: K): Boolean = { + getMap.exists(_.contains(key)) + } + + override def updateValue(key: K, value: V): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.HashMap.empty[K, V] + ) + } + + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) += (key -> value) + } + + override def iterator(): Iterator[(K, V)] = { + getMap.map(_.iterator).getOrElse(Iterator.empty) + } + + override def keys(): Iterator[K] = { + getMap.map(_.keys.iterator).getOrElse(Iterator.empty) + } + + override def values(): Iterator[V] = { + getMap.map(_.values.iterator).getOrElse(Iterator.empty) + } + + override def removeKey(key: K): Unit = { + getMap.foreach(_.remove(key)) + } + + override def clear(): Unit = { + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } +} + +/** In-memory timers. */ +class InMemoryTimers { + private val keyToTimers = mutable.Map[Any, mutable.TreeSet[Long]]() + + def registerTimer(expiryTimestampMs: Long): Unit = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (!keyToTimers.contains(groupingKey)) { + keyToTimers.put(groupingKey, mutable.TreeSet[Long]()) + } + keyToTimers(groupingKey).add(expiryTimestampMs) + } + + def deleteTimer(expiryTimestampMs: Long): Unit = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (keyToTimers.contains(groupingKey)) { + keyToTimers(groupingKey).remove(expiryTimestampMs) + if (keyToTimers(groupingKey).isEmpty) { + keyToTimers.remove(groupingKey) + } + } + } + + def listTimers(): Iterator[Long] = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + keyToTimers.get(groupingKey) match { + case Some(timers) => timers.iterator + case None => Iterator.empty + } + } + + def getAllKeysWithTimers[K](): Iterator[K] = { + keyToTimers.keys.iterator.map(_.asInstanceOf[K]) + } +} + +/** + * In-memory implementation of StatefulProcessorHandle. + * + * Supports TTL and directly accessing state. + */ +class InMemoryStatefulProcessorHandle(val timeMode: TimeMode, val clock: Clock) + extends StatefulProcessorHandle { Review Comment: nit: 2 spaces https://github.com/databricks/scala-style-guide ########## sql/core/src/main/scala/org/apache/spark/sql/streaming/TwsTester.scala: ########## @@ -0,0 +1,286 @@ +/* + * 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.{Clock, Instant, ZoneId} + +import scala.reflect.ClassTag + +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.testing.InMemoryStatefulProcessorHandle +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.timers.{ExpiredTimerInfoImpl, TimerValuesImpl} + +/** + * Testing utility for transformWithState stateful processors. + * + * This class enables unit testing of StatefulProcessor business logic by simulating the + * behavior of transformWithState. It processes input rows and returns output rows equivalent + * to those that would be produced by the processor in an actual Spark streaming query. + * + * '''Supported:''' + * - Processing input rows and producing output rows via `test()`. + * - Initial state setup via constructor parameter. + * - Direct state manipulation via `setValueState`, `setListState`, `setMapState`. + * - Direct state inspection via `peekValueState`, `peekListState`, `peekMapState`. + * - Timers in ProcessingTime mode (use `advanceProcessingTime` to fire timers). + * - Timers in EventTime mode (use `eventTimeExtractor` and `watermarkDelayMs` to configure; + * watermark advances automatically based on event times, or use `advanceWatermark` manually). + * - Late event filtering in EventTime mode (events older than the current watermark are dropped). + * - TTL for ValueState, ListState, and MapState (use ProcessingTime mode and + * `advanceProcessingTime` to test expiry). + * + * '''Testing EventTime Mode:''' + * To test with EventTime, provide `eventTimeExtractor` (a function extracting the event + * timestamp from each input row) and `watermarkDelayMs` (the watermark delay in milliseconds). + * The watermark is computed as `max(event_time_seen) - watermarkDelayMs` and is updated + * automatically after each `test()` call. Late events (with event time older than the current + * watermark) are filtered out before processing, matching production behavior. Timers with + * expiry time <= watermark will fire. You can also manually advance the watermark using + * `advanceWatermark()`. + * + * '''Use Cases:''' + * - '''Primary''': Unit testing business logic in `handleInputRows` implementations. + * - '''Not recommended''': End-to-end testing or performance testing - use actual Spark + * streaming queries for those scenarios. + * + * @param processor the StatefulProcessor to test. + * @param initialState initial state for each key as a list of (key, state) tuples. + * @param timeMode time mode (None, ProcessingTime or EventTime). + * @param outputMode output mode (Append, Update, or Complete). + * @param realTimeMode whether input rows should be processed one-by-one (separate call to + * handleInputRows) for each input row. + * @param eventTimeExtractor function to extract event time from input rows. Required if and + * only if timeMode is EventTime. + * @param watermarkDelayMs watermark delay in milliseconds. The watermark is computed as + * `max(event_time) - watermarkDelayMs`. Required if and only if timeMode is EventTime. + * @tparam K the type of grouping key. + * @tparam I the type of input rows. + * @tparam O the type of output rows. + * @since 4.0.2 + */ +class TwsTester[K, I, O]( + val processor: StatefulProcessor[K, I, O], + val initialState: List[(K, Any)] = List(), + val timeMode: TimeMode = TimeMode.None, + val outputMode: OutputMode = OutputMode.Append, + val realTimeMode: Boolean = false, + val eventTimeExtractor: I => Timestamp = null, + val watermarkDelayMs: Long = 0L) { + val clock: Clock = new Clock { + override def instant(): Instant = Instant.ofEpochMilli(currentProcessingTimeMs) + override def getZone: ZoneId = ZoneId.systemDefault() + override def withZone(zone: ZoneId): Clock = this + } + + private val handle = new InMemoryStatefulProcessorHandle(timeMode, clock) Review Comment: Or if you think we can consolidate two, that's even better. ########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/operators/stateful/transformwithstate/testing/InMemoryStatefulProcessorHandle.scala: ########## @@ -0,0 +1,332 @@ +/* + * 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.operators.stateful.transformwithstate.testing + +import java.time.{Clock, Instant} +import java.util.UUID + +import scala.collection.mutable +import scala.reflect.ClassTag + +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.ImplicitGroupingKeyTracker +import org.apache.spark.sql.execution.streaming.operators.stateful.transformwithstate.statefulprocessor.QueryInfoImpl +import org.apache.spark.sql.streaming.{ListState, MapState, QueryInfo, StatefulProcessorHandle, TimeMode, TTLConfig, ValueState} + +/** Helper to track expired keys. */ +class TtlTracker(val clock: Clock, ttl: TTLConfig) { + require(!ttl.ttlDuration.isNegative()) + private val keyToLastUpdatedTime = mutable.Map[Any, Instant]() + + def isKeyExpired(): Boolean = { + if (ttl.ttlDuration.isZero()) { + return false + } + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (!keyToLastUpdatedTime.contains(key)) { + return false + } + val expiration: Instant = keyToLastUpdatedTime.get(key).get.plus(ttl.ttlDuration) + return expiration.isBefore(clock.instant()) + } + + def onKeyUpdated(): Unit = { + val key = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + keyToLastUpdatedTime.put(key, clock.instant()) + } +} + +/** In-memory implementation of ValueState. */ +class InMemoryValueState[T](clock: Clock, ttl: TTLConfig) extends ValueState[T] { + private val keyToStateValue = mutable.Map[Any, T]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getValue: Option[T] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + override def exists(): Boolean = getValue.isDefined + + override def get(): T = getValue.getOrElse(null.asInstanceOf[T]) + + override def update(newState: T): Unit = { + ttlTracker.onKeyUpdated() + keyToStateValue.put(ImplicitGroupingKeyTracker.getImplicitKeyOption.get, newState) + } + + override def clear(): Unit = + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) +} + +/** In-memory implementation of ListState. */ +class InMemoryListState[T](clock: Clock, ttl: TTLConfig) extends ListState[T] { + private val keyToStateValue = mutable.Map[Any, mutable.ArrayBuffer[T]]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getList: Option[mutable.ArrayBuffer[T]] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + + override def exists(): Boolean = getList.isDefined + + override def get(): Iterator[T] = { + getList.orElse(Some(mutable.ArrayBuffer.empty[T])).get.iterator + } + + override def put(newState: Array[T]): Unit = { + ttlTracker.onKeyUpdated() + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] ++ newState + ) + } + + override def appendValue(newState: T): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] + ) + } + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) += newState + } + + override def appendList(newState: Array[T]): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.ArrayBuffer.empty[T] + ) + } + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) ++= newState + } + + override def clear(): Unit = { + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } +} + +/** In-memory implementation of MapState. */ +class InMemoryMapState[K, V](clock: Clock, ttl: TTLConfig) extends MapState[K, V] { + private val keyToStateValue = mutable.Map[Any, mutable.HashMap[K, V]]() + private val ttlTracker = new TtlTracker(clock, ttl) + + private def getMap: Option[mutable.HashMap[K, V]] = { + if (ttlTracker.isKeyExpired()) { + return None + } + keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } + + override def exists(): Boolean = getMap.isDefined + + override def getValue(key: K): V = { + getMap.flatMap(_.get(key)).getOrElse(null.asInstanceOf[V]) + } + + override def containsKey(key: K): Boolean = { + getMap.exists(_.contains(key)) + } + + override def updateValue(key: K, value: V): Unit = { + ttlTracker.onKeyUpdated() + if (!exists()) { + keyToStateValue.put( + ImplicitGroupingKeyTracker.getImplicitKeyOption.get, + mutable.HashMap.empty[K, V] + ) + } + + keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) += (key -> value) + } + + override def iterator(): Iterator[(K, V)] = { + getMap.map(_.iterator).getOrElse(Iterator.empty) + } + + override def keys(): Iterator[K] = { + getMap.map(_.keys.iterator).getOrElse(Iterator.empty) + } + + override def values(): Iterator[V] = { + getMap.map(_.values.iterator).getOrElse(Iterator.empty) + } + + override def removeKey(key: K): Unit = { + getMap.foreach(_.remove(key)) + } + + override def clear(): Unit = { + keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get) + } +} + +/** In-memory timers. */ +class InMemoryTimers { + private val keyToTimers = mutable.Map[Any, mutable.TreeSet[Long]]() + + def registerTimer(expiryTimestampMs: Long): Unit = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (!keyToTimers.contains(groupingKey)) { + keyToTimers.put(groupingKey, mutable.TreeSet[Long]()) + } + keyToTimers(groupingKey).add(expiryTimestampMs) + } + + def deleteTimer(expiryTimestampMs: Long): Unit = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + if (keyToTimers.contains(groupingKey)) { + keyToTimers(groupingKey).remove(expiryTimestampMs) + if (keyToTimers(groupingKey).isEmpty) { + keyToTimers.remove(groupingKey) + } + } + } + + def listTimers(): Iterator[Long] = { + val groupingKey = ImplicitGroupingKeyTracker.getImplicitKeyOption.get + keyToTimers.get(groupingKey) match { + case Some(timers) => timers.iterator + case None => Iterator.empty + } + } + + def getAllKeysWithTimers[K](): Iterator[K] = { + keyToTimers.keys.iterator.map(_.asInstanceOf[K]) + } +} + +/** + * In-memory implementation of StatefulProcessorHandle. + * + * Supports TTL and directly accessing state. + */ +class InMemoryStatefulProcessorHandle(val timeMode: TimeMode, val clock: Clock) + extends StatefulProcessorHandle { + private val states = mutable.Map[String, Any]() + val timers = new InMemoryTimers() + + override def getValueState[T]( + stateName: String, + valEncoder: Encoder[T], + ttlConfig: TTLConfig + ): ValueState[T] = { + states + .getOrElseUpdate(stateName, new InMemoryValueState[T](clock, ttlConfig)) + .asInstanceOf[InMemoryValueState[T]] + } + + override def getValueState[T: Encoder](stateName: String, ttlConfig: TTLConfig): ValueState[T] = + getValueState(stateName, implicitly[Encoder[T]], ttlConfig) + + override def getListState[T]( + stateName: String, + valEncoder: Encoder[T], + ttlConfig: TTLConfig + ): ListState[T] = { Review Comment: ditto -- 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]
