Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,220 @@
/*
* 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.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, TTLConfig, ValueState}

/** In-memory implementation of ValueState. */
class InMemoryValueState[T] extends ValueState[T] {
private val keyToStateValue = mutable.Map[Any, T]()

override def exists(): Boolean =
keyToStateValue.contains(ImplicitGroupingKeyTracker.getImplicitKeyOption.get)

override def get(): T =
keyToStateValue.getOrElse(
ImplicitGroupingKeyTracker.getImplicitKeyOption.get,
null.asInstanceOf[T]
)

override def update(newState: T): Unit =
keyToStateValue.put(ImplicitGroupingKeyTracker.getImplicitKeyOption.get, newState)

override def clear(): Unit =
keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get)
}

/** In-memory implementation of ListState. */
class InMemoryListState[T] extends ListState[T] {
private val keyToStateValue = mutable.Map[Any, mutable.ArrayBuffer[T]]()

override def exists(): Boolean =
keyToStateValue.contains(ImplicitGroupingKeyTracker.getImplicitKeyOption.get)

private def getList: mutable.ArrayBuffer[T] = {
if (!exists()) {
keyToStateValue.put(
ImplicitGroupingKeyTracker.getImplicitKeyOption.get,
mutable.ArrayBuffer.empty[T]
)
}
keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get).get
}

override def get(): Iterator[T] =
if (exists()) getList.iterator else Iterator.empty

override def put(newState: Array[T]): Unit =
keyToStateValue.put(
ImplicitGroupingKeyTracker.getImplicitKeyOption.get,
mutable.ArrayBuffer.empty[T] ++ newState
)

override def appendValue(newState: T): Unit = getList += newState

override def appendList(newState: Array[T]): Unit = getList ++= newState

override def clear(): Unit =
keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get)
}

/** In-memory implementation of MapState. */
class InMemoryMapState[K, V] extends MapState[K, V] {
private val keyToStateValue = mutable.Map[Any, mutable.HashMap[K, V]]()

override def exists(): Boolean =
keyToStateValue.contains(ImplicitGroupingKeyTracker.getImplicitKeyOption.get)

private def getMap: mutable.HashMap[K, V] = {
if (!exists()) {
keyToStateValue.put(
ImplicitGroupingKeyTracker.getImplicitKeyOption.get,
mutable.HashMap.empty[K, V]
)
}
keyToStateValue(ImplicitGroupingKeyTracker.getImplicitKeyOption.get)
}

private def getMapIfExists: Option[mutable.HashMap[K, V]] = {
keyToStateValue.get(ImplicitGroupingKeyTracker.getImplicitKeyOption.get)
}

override def getValue(key: K): V =
getMapIfExists.flatMap(_.get(key)).getOrElse(null.asInstanceOf[V])

override def containsKey(key: K): Boolean = getMapIfExists.exists(_.contains(key))

override def updateValue(key: K, value: V): Unit = getMap.put(key, value)

override def iterator(): Iterator[(K, V)] =
getMapIfExists.map(_.iterator).getOrElse(Iterator.empty)

override def keys(): Iterator[K] = getMapIfExists.map(_.keys.iterator).getOrElse(Iterator.empty)

override def values(): Iterator[V] =
getMapIfExists.map(_.values.iterator).getOrElse(Iterator.empty)

override def removeKey(key: K): Unit = getMapIfExists.foreach(_.remove(key))

override def clear(): Unit =
keyToStateValue.remove(ImplicitGroupingKeyTracker.getImplicitKeyOption.get)
}

/**
* In-memory implementation of StatefulProcessorHandle.
*
* Doesn't support timers and TTL. Supports directly accessing state.
*/
class InMemoryStatefulProcessorHandle() extends StatefulProcessorHandle {
private val states = mutable.Map[String, Any]()

override def getValueState[T](
stateName: String,
valEncoder: Encoder[T],
ttlConfig: TTLConfig
): ValueState[T] = {
states
.getOrElseUpdate(stateName, new InMemoryValueState[T]())
.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]())
.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]())
.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 =
throw new UnsupportedOperationException("Timers are not supported.")

override def deleteTimer(expiryTimestampMs: Long): Unit =
throw new UnsupportedOperationException("Timers are not supported.")

override def listTimers(): Iterator[Long] =
throw new UnsupportedOperationException("Timers are not supported.")

override def deleteIfExists(stateName: String): Unit = states.remove(stateName)

def setValueState[T](stateName: String, value: T): Unit = {
require(states.contains(stateName), s"State $stateName has not been initialized.")
states(stateName).asInstanceOf[InMemoryValueState[T]].update(value)
}

def peekValueState[T](stateName: String): Option[T] = {
require(states.contains(stateName), s"State $stateName has not been initialized.")
Option(states(stateName).asInstanceOf[InMemoryValueState[T]].get())
}

def setListState[T](stateName: String, value: List[T])(implicit ct: ClassTag[T]): Unit = {
require(states.contains(stateName), s"State $stateName has not been initialized.")
states(stateName).asInstanceOf[InMemoryListState[T]].put(value.toArray)
}

def peekListState[T](stateName: String): List[T] = {
require(states.contains(stateName), s"State $stateName has not been initialized.")
states(stateName).asInstanceOf[InMemoryListState[T]].get().toList
}

def setMapState[MK, MV](stateName: String, value: Map[MK, MV]): Unit = {
require(states.contains(stateName), s"State $stateName has not been initialized.")
val mapState = states(stateName).asInstanceOf[InMemoryMapState[MK, MV]]
mapState.clear()
value.foreach { case (k, v) => mapState.updateValue(k, v) }
}

def peekMapState[MK, MV](stateName: String): Map[MK, MV] = {
require(states.contains(stateName), s"State $stateName has not been initialized.")
states(stateName).asInstanceOf[InMemoryMapState[MK, MV]].iterator().toMap
}
}
131 changes: 131 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/streaming/TwsTester.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,131 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql.streaming

import scala.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

/**
* 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`.
*
* '''Not Supported:'''
* - '''Timers''': Only TimeMode.None is supported. If the processor attempts to register or
* use timers (as if in TimeMode.EventTime or TimeMode.ProcessingTime), a NullPointerException
* will be thrown.
* - '''TTL''': State TTL configurations are ignored. All state persists indefinitely.
*
* '''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
* @tparam K the type of grouping key
* @tparam I the type of input rows
* @tparam O the type of output rows
*/
class TwsTester[K, I, O](
val processor: StatefulProcessor[K, I, O],
val initialState: List[(K, Any)] = List()) {
private val handle = new InMemoryStatefulProcessorHandle()
processor.setHandle(handle)
processor.init(OutputMode.Append, TimeMode.None)
processor match {
case p: StatefulProcessorWithInitialState[K @unchecked, I @unchecked, O @unchecked, s] =>
handleInitialState[s]()
case _ =>
}

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], null)
}
}

/**
* Processes input rows through the stateful processor, grouped by key.
*
* This corresponds to processing one microbatch. {@code handleInputRows} will be called once for
* each key that appears in {@code input}.
*
* To simulate real-time mode, call this method repeatedly in a loop, passing a list with a single
* (key, input row) tuple per call.
*
* @param input list of (key, input row) tuples to process
* @return all output rows produced by the processor
*/
def test(input: List[(K, I)]): List[O] = {
var ans: List[O] = List()
for ((key, v) <- input.groupBy(_._1)) {
ImplicitGroupingKeyTracker.setImplicitKey(key)
ans = ans ++ processor.handleInputRows(key, v.map(_._2).iterator, null).toList
}
ans
}

/** 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)
}
}
Loading