Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-35800][SS] Improving GroupState testability by introducing TestGroupState #32938

Original file line number Diff line number Diff line change
Expand Up @@ -58,8 +58,8 @@ case class FlatMapGroupsWithStateExec(
child: SparkPlan
) extends UnaryExecNode with ObjectProducerExec with StateStoreWriter with WatermarkSupport {

import GroupStateImpl._
import FlatMapGroupsWithStateExecHelper._
import GroupStateImpl._

private val isTimeoutEnabled = timeoutConf != NoTimeout
private val watermarkPresent = child.output.exists {
Expand Down Expand Up @@ -226,13 +226,13 @@ case class FlatMapGroupsWithStateExec(

// When the iterator is consumed, then write changes to state
def onIteratorCompletion: Unit = {
if (groupState.hasRemoved && groupState.getTimeoutTimestamp == NO_TIMESTAMP) {
if (groupState.isRemoved && !groupState.getTimeoutTimestampMs.isPresent()) {
stateManager.removeState(store, stateData.keyRow)
numUpdatedStateRows += 1
} else {
val currentTimeoutTimestamp = groupState.getTimeoutTimestamp
val currentTimeoutTimestamp = groupState.getTimeoutTimestampMs.orElse(NO_TIMESTAMP)
val hasTimeoutChanged = currentTimeoutTimestamp != stateData.timeoutTimestamp
val shouldWriteState = groupState.hasUpdated || groupState.hasRemoved || hasTimeoutChanged
val shouldWriteState = groupState.isUpdated || groupState.isRemoved || hasTimeoutChanged

if (shouldWriteState) {
val updatedStateObj = if (groupState.exists) groupState.get else null
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,16 +20,16 @@ package org.apache.spark.sql.execution.streaming
import java.sql.Date
import java.util.concurrent.TimeUnit

import org.apache.spark.sql.catalyst.plans.logical.{EventTimeTimeout, ProcessingTimeTimeout}
import org.apache.spark.api.java.Optional
import org.apache.spark.sql.catalyst.plans.logical.{EventTimeTimeout, NoTimeout, ProcessingTimeTimeout}
import org.apache.spark.sql.catalyst.util.IntervalUtils
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.execution.streaming.GroupStateImpl._
import org.apache.spark.sql.streaming.{GroupState, GroupStateTimeout}
import org.apache.spark.sql.streaming.{GroupStateTimeout, TestGroupState}
import org.apache.spark.unsafe.types.UTF8String


/**
* Internal implementation of the [[GroupState]] interface. Methods are not thread-safe.
* Internal implementation of the [[TestGroupState]] interface. Methods are not thread-safe.
*
* @param optionalValue Optional value of the state
* @param batchProcessingTimeMs Processing time of current batch, used to calculate timestamp
Expand All @@ -45,7 +45,7 @@ private[sql] class GroupStateImpl[S] private(
eventTimeWatermarkMs: Long,
timeoutConf: GroupStateTimeout,
override val hasTimedOut: Boolean,
watermarkPresent: Boolean) extends GroupState[S] {
watermarkPresent: Boolean) extends TestGroupState[S] {

private var value: S = optionalValue.getOrElse(null.asInstanceOf[S])
private var defined: Boolean = optionalValue.isDefined
Expand Down Expand Up @@ -147,14 +147,17 @@ private[sql] class GroupStateImpl[S] private(

// ========= Internal API =========

/** Whether the state has been marked for removing */
def hasRemoved: Boolean = removed
override def isRemoved: Boolean = removed

/** Whether the state has been updated */
def hasUpdated: Boolean = updated
override def isUpdated: Boolean = updated

/** Return timeout timestamp or `TIMEOUT_TIMESTAMP_NOT_SET` if not set */
def getTimeoutTimestamp: Long = timeoutTimestamp
override def getTimeoutTimestampMs: Optional[Long] = {
if (timeoutTimestamp != NO_TIMESTAMP) {
Optional.of(timeoutTimestamp)
} else {
Optional.empty[Long]
}
}

private def parseDuration(duration: String): Long = {
val cal = IntervalUtils.stringToInterval(UTF8String.fromString(duration))
Expand Down Expand Up @@ -184,6 +187,17 @@ private[sql] object GroupStateImpl {
timeoutConf: GroupStateTimeout,
hasTimedOut: Boolean,
watermarkPresent: Boolean): GroupStateImpl[S] = {
if (batchProcessingTimeMs < 0) {
throw new IllegalArgumentException("batchProcessingTimeMs must be 0 or positive")
}
if (watermarkPresent && eventTimeWatermarkMs < 0) {
throw new IllegalArgumentException("eventTimeWatermarkMs must be 0 or positive if present")
}
if (hasTimedOut && timeoutConf == NoTimeout) {
throw new UnsupportedOperationException(
"hasTimedOut is true however there's no timeout configured")
}

new GroupStateImpl[S](
optionalValue, batchProcessingTimeMs, eventTimeWatermarkMs,
timeoutConf, hasTimedOut, watermarkPresent)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.streaming.state
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.execution.ObjectOperator
import org.apache.spark.sql.execution.streaming.GroupStateImpl
import org.apache.spark.sql.execution.streaming.GroupStateImpl.NO_TIMESTAMP
import org.apache.spark.sql.types._

Expand Down Expand Up @@ -168,7 +167,7 @@ object FlatMapGroupsWithStateExecHelper {
override val stateSerializerExprs: Seq[Expression] = {
val encoderSerializer = stateEncoder.namedExpressions
if (shouldStoreTimestamp) {
encoderSerializer :+ Literal(GroupStateImpl.NO_TIMESTAMP)
encoderSerializer :+ Literal(NO_TIMESTAMP)
} else {
encoderSerializer
}
Expand Down Expand Up @@ -226,7 +225,7 @@ object FlatMapGroupsWithStateExecHelper {
}

if (shouldStoreTimestamp) {
Seq(nullSafeNestedStateSerExpr, Literal(GroupStateImpl.NO_TIMESTAMP))
Seq(nullSafeNestedStateSerExpr, Literal(NO_TIMESTAMP))
} else {
Seq(nullSafeNestedStateSerExpr)
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,173 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.streaming

import org.apache.spark.annotation.{Evolving, Experimental}
import org.apache.spark.api.java.Optional
import org.apache.spark.sql.execution.streaming.GroupStateImpl
import org.apache.spark.sql.execution.streaming.GroupStateImpl._

/**
* :: Experimental ::
*
* The extended version of [[GroupState]] interface with extra getters of state machine fields
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think you need htis as well
::Experimental::
See GroupState

* to improve testability of the [[GroupState]] implementations
* which inherit from the extended interface.
*
* Scala example of using `TestGroupState`:
* {{{
* // Please refer to ScalaDoc of `GroupState` for the Scala definition of `mappingFunction()`
*
* import org.apache.spark.api.java.Optional
* import org.apache.spark.sql.streaming.GroupStateTimeout
* import org.apache.spark.sql.streaming.TestGroupState
* // other imports
*
* // test class setups
*
* test("MapGroupsWithState state transition function") {
* // Creates the prevState input for the state transition function
* // with desired configs. The `create()` API would guarantee that
* // the generated instance has the same behavior as the one built by
* // engine with the same configs.
* val prevState = TestGroupState.create[Int](
* optionalState = Optional.empty[Int],
* timeoutConf = NoTimeout,
* batchProcessingTimeMs = 1L,
* eventTimeWatermarkMs = Optional.of(1L),
* hasTimedOut = false)
*
* val key: String = ...
* val values: Iterator[Int] = ...
*
* // Asserts the prevState is in init state without updates.
* assert(!prevState.isUpdated)
*
* // Calls the state transition function with the test previous state
* // with desired configs.
* mappingFunction(key, values, prevState)
*
* // Asserts the test GroupState object has been updated but not removed
* // after calling the state transition function
* assert(prevState.isUpdated)
* assert(!prevState.isRemoved)
* }
* }}}
*
* Java example of using `TestGroupSate`:
* {{{
* // Please refer to ScalaDoc of `GroupState` for the Java definition of `mappingFunction()`
*
* import org.apache.spark.api.java.Optional;
* import org.apache.spark.sql.streaming.GroupStateTimeout;
* import org.apache.spark.sql.streaming.TestGroupState;
* // other imports
*
* // test class setups
*
* // test `MapGroupsWithState` state transition function `mappingFunction()`
* public void testMappingFunctionWithTestGroupState() {
* // Creates the prevState input for the state transition function
* // with desired configs. The `create()` API would guarantee that
* // the generated instance has the same behavior as the one built by
* // engine with the same configs.
* TestGroupState<Int> prevState = TestGroupState.create(
* Optional.empty(),
* GroupStateTimeout.NoTimeout(),
* 1L,
* Optional.of(1L),
* false);
*
* String key = ...;
* Integer[] values = ...;
*
* // Asserts the prevState is in init state without updates.
* Assert.assertFalse(prevState.isUpdated());
*
* // Calls the state transition function with the test previous state
* // with desired configs.
* mappingFunction.call(key, Arrays.asList(values).iterator(), prevState);
*
* // Asserts the test GroupState object has been updated but not removed
* // after calling the state transition function
* Assert.assertTrue(prevState.isUpdated());
* Assert.assertFalse(prevState.isRemoved());
* }
* }}}
*
* @tparam S User-defined type of the state to be stored for each group. Must be encodable into
* Spark SQL types (see `Encoder` for more details).
* @since 3.2.0
*/
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since the original GroupState API is still marked as experimental (I know it does make sense, but that's for a different discussion), lets be consistent and add the same annotations to this as well. similarly, also add @since 3.2.0

and would be nice if you could add Java and Scala code snippets of how to use these traits. It's not entirely obvious how to use TestGroupState. See GroupState to find out how to add code snippets. This is how it looks later - https://spark.apache.org/docs/2.4.0/api/scala/index.html#org.apache.spark.sql.streaming.GroupState

@Experimental
@Evolving
trait TestGroupState[S] extends GroupState[S] {
/** Whether the state has been marked for removing */
def isRemoved: Boolean

/** Whether the state has been updated but not removed */
def isUpdated: Boolean

/**
* Returns the timestamp if `setTimeoutTimestamp()` is called.
* Or, returns batch processing time + the duration when
* `setTimeoutDuration()` is called.
*
* Otherwise, returns `Optional.empty` if not set.
*/
def getTimeoutTimestampMs: Optional[Long]
}

object TestGroupState {

/**
* Creates TestGroupState instances for general testing purposes.
*
* @param optionalState Optional value of the state.
* @param timeoutConf Type of timeout configured. Based on this, different operations
* will be supported.
* @param batchProcessingTimeMs Processing time of current batch, used to calculate timestamp
* for processing time timeouts.
* @param eventTimeWatermarkMs Optional value of event time watermark in ms. Set as
* `Optional.empty` if watermark is not present.
* Otherwise, event time watermark should be a positive long
* and the timestampMs set through `setTimeoutTimestamp()`
* cannot be less than `eventTimeWatermarkMs`.
* @param hasTimedOut Whether the key for which this state wrapped is being created is
* getting timed out or not.
* @return a [[TestGroupState]] instance built with the user specified configs.
*/
@throws[IllegalArgumentException]("if 'batchProcessingTimeMs' is less than 0")
@throws[IllegalArgumentException]("if 'eventTimeWatermarkMs' is present but less than 0")
@throws[UnsupportedOperationException](
"if 'hasTimedOut' is true however there's no timeout configured")
def create[S](
optionalState: Optional[S],
timeoutConf: GroupStateTimeout,
batchProcessingTimeMs: Long,
eventTimeWatermarkMs: Optional[Long],
hasTimedOut: Boolean): TestGroupState[S] = {
GroupStateImpl.createForStreaming[S](
Option(optionalState.orNull),
batchProcessingTimeMs,
eventTimeWatermarkMs.orElse(NO_TIMESTAMP),
timeoutConf,
hasTimedOut,
eventTimeWatermarkMs.isPresent())
}
}
Loading