tdas commented on a change in pull request #32938:
URL: https://github.com/apache/spark/pull/32938#discussion_r655745832



##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,68 @@
+package org.apache.spark.sql.streaming
+
+import org.apache.spark.api.java.Optional
+import org.apache.spark.sql.execution.streaming.GroupStateImpl
+import org.apache.spark.sql.execution.streaming.GroupStateImpl._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * @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).
+ */
+trait TestGroupState[S] extends GroupState[S] {
+  /** Whether the state has been marked for removing */
+  def hasRemoved: Boolean

Review comment:
       I think isRemoved and isUpdated are better.

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,68 @@
+package org.apache.spark.sql.streaming
+
+import org.apache.spark.api.java.Optional
+import org.apache.spark.sql.execution.streaming.GroupStateImpl
+import org.apache.spark.sql.execution.streaming.GroupStateImpl._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * @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).
+ */

Review comment:
       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

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,161 @@
+/*
+ * 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._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * Scala example of using `TestGroupState`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState
+ * // other imports
+ *
+ * // test class setups
+ *
+ * test("Structured Streaming state update 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.
+ *   var prevState = TestGroupState.create[UserStatus](

Review comment:
       nit: why is this a var

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,161 @@
+/*
+ * 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._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * Scala example of using `TestGroupState`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState
+ * // other imports
+ *
+ * // test class setups
+ *
+ * test("Structured Streaming state update 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.
+ *   var prevState = TestGroupState.create[UserStatus](
+ *     optionalState = Optional.empty[UserStatus],
+ *     timeoutConf = EventTimeTimeout,

Review comment:
       lets keep this simple.. .to NoTimeout

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,161 @@
+/*
+ * 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._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * Scala example of using `TestGroupState`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState
+ * // other imports
+ *
+ * // test class setups
+ *
+ * test("Structured Streaming state update 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.
+ *   var prevState = TestGroupState.create[UserStatus](
+ *     optionalState = Optional.empty[UserStatus],
+ *     timeoutConf = EventTimeTimeout,

Review comment:
       lets keep this simple...NoTimeout

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,161 @@
+/*
+ * 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._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * Scala example of using `TestGroupState`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState
+ * // other imports
+ *
+ * // test class setups
+ *
+ * test("Structured Streaming state update 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.
+ *   var prevState = TestGroupState.create[UserStatus](
+ *     optionalState = Optional.empty[UserStatus],
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false)
+ *
+ *   val userId: String = ...
+ *   val actions: Iterator[UserAction] = ...
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assert(!prevState.hasUpdated)
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState)

Review comment:
       define updateState as the function that is going to be used with 
map/flatmapGroupsWithState

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,161 @@
+/*
+ * 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._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * Scala example of using `TestGroupState`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState
+ * // other imports
+ *
+ * // test class setups
+ *
+ * test("Structured Streaming state update 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.
+ *   var prevState = TestGroupState.create[UserStatus](
+ *     optionalState = Optional.empty[UserStatus],
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false)
+ *
+ *   val userId: String = ...
+ *   val actions: Iterator[UserAction] = ...
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assert(!prevState.hasUpdated)
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState)
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assert(prevState.hasUpdated)
+ * }
+ * }}}
+ *
+ * Java example of using `TestGroupSate`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState;
+ * // other imports
+ *
+ * // test class setups
+ *
+ * // test `flatMapGroupsWithState` state transition function `updateState()`
+ * public void testUpdateState() {
+ *   // 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 prevState = new TestGroupState<UserStatus>().create(
+ *     optionalState = Optional.<UserStatus>empty(),
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false);
+ *
+ *   String userId = ...;
+ *   ArrayList<UserAction> actions = ...;
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assertTrue(!prevState.hasUpdated());
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState);
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assertTrue(prevState.hasUpdated());
+ * }
+ * }}}
+ *
+ * @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

Review comment:
       same comment as above.

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,161 @@
+/*
+ * 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._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * Scala example of using `TestGroupState`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState
+ * // other imports
+ *
+ * // test class setups
+ *
+ * test("Structured Streaming state update 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.
+ *   var prevState = TestGroupState.create[UserStatus](
+ *     optionalState = Optional.empty[UserStatus],
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false)
+ *
+ *   val userId: String = ...
+ *   val actions: Iterator[UserAction] = ...
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assert(!prevState.hasUpdated)
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState)
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assert(prevState.hasUpdated)
+ * }
+ * }}}
+ *
+ * Java example of using `TestGroupSate`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState;
+ * // other imports
+ *
+ * // test class setups
+ *
+ * // test `flatMapGroupsWithState` state transition function `updateState()`
+ * public void testUpdateState() {
+ *   // 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 prevState = new TestGroupState<UserStatus>().create(
+ *     optionalState = Optional.<UserStatus>empty(),
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false);
+ *
+ *   String userId = ...;
+ *   ArrayList<UserAction> actions = ...;
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assertTrue(!prevState.hasUpdated());
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState);
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assertTrue(prevState.hasUpdated());
+ * }
+ * }}}
+ *
+ * @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
+ */
+@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.

Review comment:
       Link to the corresponding functions in GroupState. If thats complicated, 
at least put the function names and codee snippets in backticks
   ```
   `setTimeoutTimestamp` 
   `Optional.empty`
   ```
   
   
   

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,161 @@
+/*
+ * 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._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * Scala example of using `TestGroupState`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState
+ * // other imports
+ *
+ * // test class setups
+ *
+ * test("Structured Streaming state update 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.
+ *   var prevState = TestGroupState.create[UserStatus](
+ *     optionalState = Optional.empty[UserStatus],
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false)
+ *
+ *   val userId: String = ...
+ *   val actions: Iterator[UserAction] = ...
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assert(!prevState.hasUpdated)
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState)
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assert(prevState.hasUpdated)
+ * }
+ * }}}
+ *
+ * Java example of using `TestGroupSate`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState;
+ * // other imports
+ *
+ * // test class setups
+ *
+ * // test `flatMapGroupsWithState` state transition function `updateState()`
+ * public void testUpdateState() {
+ *   // 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 prevState = new TestGroupState<UserStatus>().create(
+ *     optionalState = Optional.<UserStatus>empty(),
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false);
+ *
+ *   String userId = ...;
+ *   ArrayList<UserAction> actions = ...;
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assertTrue(!prevState.hasUpdated());
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState);
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assertTrue(prevState.hasUpdated());
+ * }
+ * }}}
+ *
+ * @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
+ */
+@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, batch processing time + the duration will be returned 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 None if
+   *                              watermark is not present. Otherwise, event 
time watermark
+   *                              should be a positive long and the timestampMs
+   *                              set through setTimeoutTimestamp.
+   *                              cannot be less than the set 
eventTimeWatermarkMs.
+   * @param hasTimedOut           Whether the key for which this state wrapped 
is being created is
+   *                              getting timed out or not.
+   * @return a [[TestGroupState]] instance that's built with the user 
specified configs.
+   */
+  @throws[IllegalArgumentException]("if 'batchProcessingTimeMs' is not 
positive")
+  @throws[IllegalArgumentException]("if 'eventTimeWatermarkMs' present but is 
not 0 or positive")

Review comment:
       is present

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,161 @@
+/*
+ * 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._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * Scala example of using `TestGroupState`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState
+ * // other imports
+ *
+ * // test class setups
+ *
+ * test("Structured Streaming state update 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.
+ *   var prevState = TestGroupState.create[UserStatus](
+ *     optionalState = Optional.empty[UserStatus],
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false)
+ *
+ *   val userId: String = ...
+ *   val actions: Iterator[UserAction] = ...
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assert(!prevState.hasUpdated)
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState)
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assert(prevState.hasUpdated)
+ * }
+ * }}}
+ *
+ * Java example of using `TestGroupSate`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState;
+ * // other imports
+ *
+ * // test class setups
+ *
+ * // test `flatMapGroupsWithState` state transition function `updateState()`
+ * public void testUpdateState() {
+ *   // 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 prevState = new TestGroupState<UserStatus>().create(
+ *     optionalState = Optional.<UserStatus>empty(),
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false);
+ *
+ *   String userId = ...;
+ *   ArrayList<UserAction> actions = ...;
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assertTrue(!prevState.hasUpdated());
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState);
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assertTrue(prevState.hasUpdated());
+ * }
+ * }}}
+ *
+ * @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
+ */
+@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, batch processing time + the duration will be returned 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 None if
+   *                              watermark is not present. Otherwise, event 
time watermark
+   *                              should be a positive long and the timestampMs
+   *                              set through setTimeoutTimestamp.
+   *                              cannot be less than the set 
eventTimeWatermarkMs.
+   * @param hasTimedOut           Whether the key for which this state wrapped 
is being created is
+   *                              getting timed out or not.
+   * @return a [[TestGroupState]] instance that's built with the user 
specified configs.
+   */
+  @throws[IllegalArgumentException]("if 'batchProcessingTimeMs' is not 
positive")
+  @throws[IllegalArgumentException]("if 'eventTimeWatermarkMs' present but is 
not 0 or positive")

Review comment:
       `is present but is < 0`
   
   `not X or Y` is a little ambiguous. is it `!x || y` or `!(x || y)`

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,161 @@
+/*
+ * 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._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * Scala example of using `TestGroupState`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState
+ * // other imports
+ *
+ * // test class setups
+ *
+ * test("Structured Streaming state update 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.
+ *   var prevState = TestGroupState.create[UserStatus](
+ *     optionalState = Optional.empty[UserStatus],
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false)
+ *
+ *   val userId: String = ...
+ *   val actions: Iterator[UserAction] = ...
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assert(!prevState.hasUpdated)
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState)
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assert(prevState.hasUpdated)
+ * }
+ * }}}
+ *
+ * Java example of using `TestGroupSate`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState;
+ * // other imports
+ *
+ * // test class setups
+ *
+ * // test `flatMapGroupsWithState` state transition function `updateState()`
+ * public void testUpdateState() {
+ *   // 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 prevState = new TestGroupState<UserStatus>().create(
+ *     optionalState = Optional.<UserStatus>empty(),
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false);
+ *
+ *   String userId = ...;
+ *   ArrayList<UserAction> actions = ...;
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assertTrue(!prevState.hasUpdated());
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState);
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assertTrue(prevState.hasUpdated());
+ * }
+ * }}}
+ *
+ * @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
+ */
+@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, batch processing time + the duration will be returned 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 None if
+   *                              watermark is not present. Otherwise, event 
time watermark
+   *                              should be a positive long and the timestampMs
+   *                              set through setTimeoutTimestamp.
+   *                              cannot be less than the set 
eventTimeWatermarkMs.
+   * @param hasTimedOut           Whether the key for which this state wrapped 
is being created is
+   *                              getting timed out or not.
+   * @return a [[TestGroupState]] instance that's built with the user 
specified configs.
+   */
+  @throws[IllegalArgumentException]("if 'batchProcessingTimeMs' is not 
positive")
+  @throws[IllegalArgumentException]("if 'eventTimeWatermarkMs' present but is 
not 0 or positive")

Review comment:
       `is present but is < 0`
   
   `not X or Y` is a sometimes a little ambiguous to parse in english. is it 
`!x || y` or `!(x || y)`

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,161 @@
+/*
+ * 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._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * Scala example of using `TestGroupState`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState
+ * // other imports
+ *
+ * // test class setups
+ *
+ * test("Structured Streaming state update 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.
+ *   var prevState = TestGroupState.create[UserStatus](
+ *     optionalState = Optional.empty[UserStatus],
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false)
+ *
+ *   val userId: String = ...
+ *   val actions: Iterator[UserAction] = ...
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assert(!prevState.hasUpdated)
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState)
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assert(prevState.hasUpdated)
+ * }
+ * }}}
+ *
+ * Java example of using `TestGroupSate`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState;
+ * // other imports
+ *
+ * // test class setups
+ *
+ * // test `flatMapGroupsWithState` state transition function `updateState()`
+ * public void testUpdateState() {
+ *   // 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 prevState = new TestGroupState<UserStatus>().create(
+ *     optionalState = Optional.<UserStatus>empty(),
+ *     timeoutConf = EventTimeTimeout,

Review comment:
       same comments as above.

##########
File path: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala
##########
@@ -1291,24 +1391,24 @@ class FlatMapGroupsWithStateSuite extends 
StateStoreMetricsTest {
       }.get
   }
 
-  def testTimeoutDurationNotAllowed[T <: Exception: Manifest](state: 
GroupStateImpl[_]): Unit = {
-    val prevTimestamp = state.getTimeoutTimestamp
+  def testTimeoutDurationNotAllowed[T <: Exception: Manifest](state: 
TestGroupState[_]): Unit = {
+    val prevTimestamp = state.getTimeoutTimestampMs
     intercept[T] { state.setTimeoutDuration(1000) }
-    assert(state.getTimeoutTimestamp === prevTimestamp)
+    assert(state.getTimeoutTimestampMs === prevTimestamp)
     intercept[T] { state.setTimeoutDuration("2 second") }
-    assert(state.getTimeoutTimestamp === prevTimestamp)
+    assert(state.getTimeoutTimestampMs === prevTimestamp)
   }
 
-  def testTimeoutTimestampNotAllowed[T <: Exception: Manifest](state: 
GroupStateImpl[_]): Unit = {
-    val prevTimestamp = state.getTimeoutTimestamp
+  def testTimeoutTimestampNotAllowed[T <: Exception: Manifest](state: 
TestGroupState[_]): Unit = {
+    val prevTimestamp = state.getTimeoutTimestampMs
     intercept[T] { state.setTimeoutTimestamp(2000) }
-    assert(state.getTimeoutTimestamp === prevTimestamp)
+    assert(state.getTimeoutTimestampMs === prevTimestamp)
     intercept[T] { state.setTimeoutTimestamp(2000, "1 second") }
-    assert(state.getTimeoutTimestamp === prevTimestamp)
+    assert(state.getTimeoutTimestampMs === prevTimestamp)
     intercept[T] { state.setTimeoutTimestamp(new Date(2000)) }
-    assert(state.getTimeoutTimestamp === prevTimestamp)
+    assert(state.getTimeoutTimestampMs === prevTimestamp)
     intercept[T] { state.setTimeoutTimestamp(new Date(2000), "1 second") }
-    assert(state.getTimeoutTimestamp === prevTimestamp)
+    assert(state.getTimeoutTimestampMs === prevTimestamp)
   }
 
   def newStateStore(): StateStore = new MemoryStateStore()

Review comment:
       We should add a Java unit test!

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,161 @@
+/*
+ * 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._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * Scala example of using `TestGroupState`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState
+ * // other imports
+ *
+ * // test class setups
+ *
+ * test("Structured Streaming state update 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.
+ *   var prevState = TestGroupState.create[UserStatus](
+ *     optionalState = Optional.empty[UserStatus],
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false)
+ *
+ *   val userId: String = ...
+ *   val actions: Iterator[UserAction] = ...
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assert(!prevState.hasUpdated)
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState)
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assert(prevState.hasUpdated)
+ * }
+ * }}}
+ *
+ * Java example of using `TestGroupSate`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState;
+ * // other imports
+ *
+ * // test class setups
+ *
+ * // test `flatMapGroupsWithState` state transition function `updateState()`
+ * public void testUpdateState() {
+ *   // 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 prevState = new TestGroupState<UserStatus>().create(
+ *     optionalState = Optional.<UserStatus>empty(),
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false);
+ *
+ *   String userId = ...;
+ *   ArrayList<UserAction> actions = ...;
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assertTrue(!prevState.hasUpdated());
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState);
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assertTrue(prevState.hasUpdated());
+ * }
+ * }}}
+ *
+ * @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
+ */
+@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, batch processing time + the duration will be returned 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 None if

Review comment:
       nit: `None` is not correct. its Optional.empty . isnt it?

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,161 @@
+/*
+ * 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._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * Scala example of using `TestGroupState`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState
+ * // other imports
+ *
+ * // test class setups
+ *
+ * test("Structured Streaming state update 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.
+ *   var prevState = TestGroupState.create[UserStatus](

Review comment:
       why not be consistent with the docs of GroupState by using the same 
names and types. like the function name over there is `mappingFunction` and 
data types are String, Int, and Int?

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,161 @@
+/*
+ * 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._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * Scala example of using `TestGroupState`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState
+ * // other imports
+ *
+ * // test class setups
+ *
+ * test("Structured Streaming state update 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.
+ *   var prevState = TestGroupState.create[UserStatus](
+ *     optionalState = Optional.empty[UserStatus],
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false)
+ *
+ *   val userId: String = ...
+ *   val actions: Iterator[UserAction] = ...
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assert(!prevState.hasUpdated)
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState)
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assert(prevState.hasUpdated)
+ * }
+ * }}}
+ *
+ * Java example of using `TestGroupSate`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState;
+ * // other imports
+ *
+ * // test class setups
+ *
+ * // test `flatMapGroupsWithState` state transition function `updateState()`
+ * public void testUpdateState() {
+ *   // 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 prevState = new TestGroupState<UserStatus>().create(
+ *     optionalState = Optional.<UserStatus>empty(),
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false);
+ *
+ *   String userId = ...;
+ *   ArrayList<UserAction> actions = ...;
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assertTrue(!prevState.hasUpdated());
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState);
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assertTrue(prevState.hasUpdated());
+ * }
+ * }}}
+ *
+ * @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

Review comment:
       same comment as above.

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,161 @@
+/*
+ * 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._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * Scala example of using `TestGroupState`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState
+ * // other imports
+ *
+ * // test class setups
+ *
+ * test("Structured Streaming state update 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.
+ *   var prevState = TestGroupState.create[UserStatus](
+ *     optionalState = Optional.empty[UserStatus],
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false)
+ *
+ *   val userId: String = ...
+ *   val actions: Iterator[UserAction] = ...
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assert(!prevState.hasUpdated)
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState)

Review comment:
       if you are sticking to new names, then at least define updateState as 
the function that is going to be used with map/flatmapGroupsWithState

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,161 @@
+/*
+ * 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._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields
+ * to improve testability of the [[GroupState]] implementations
+ * which inherit from the extended interface.
+ *
+ * Scala example of using `TestGroupState`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState
+ * // other imports
+ *
+ * // test class setups
+ *
+ * test("Structured Streaming state update 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.
+ *   var prevState = TestGroupState.create[UserStatus](
+ *     optionalState = Optional.empty[UserStatus],
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false)
+ *
+ *   val userId: String = ...
+ *   val actions: Iterator[UserAction] = ...
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assert(!prevState.hasUpdated)
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState)
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assert(prevState.hasUpdated)
+ * }
+ * }}}
+ *
+ * Java example of using `TestGroupSate`:
+ * {{{
+ * import org.apache.spark.sql.streaming.TestGroupState;
+ * // other imports
+ *
+ * // test class setups
+ *
+ * // test `flatMapGroupsWithState` state transition function `updateState()`
+ * public void testUpdateState() {
+ *   // 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 prevState = new TestGroupState<UserStatus>().create(
+ *     optionalState = Optional.<UserStatus>empty(),
+ *     timeoutConf = EventTimeTimeout,
+ *     batchProcessingTimeMs = 1L,
+ *     eventTimeWatermarkMs = Optional.of(1L),
+ *     hasTimedOut = false);
+ *
+ *   String userId = ...;
+ *   ArrayList<UserAction> actions = ...;
+ *
+ *   // Asserts the prevState is in init state without updates.
+ *   assertTrue(!prevState.hasUpdated());
+ *
+ *   // Calls the state transition function with the test previous state
+ *   //  with desired configs.
+ *   updateState(userId, actions, prevState);
+ *
+ *   // Asserts the test GroupState object has been updated after calling
+ *   // the state transition function
+ *   assertTrue(prevState.hasUpdated());
+ * }
+ * }}}
+ *
+ * @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
+ */
+@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.

Review comment:
       Link to the corresponding functions in GroupState. If thats complicated, 
at least put the function names and codee snippets in backticks
   ```
   `setTimeoutTimestamp` `setTimeoutDuration` 
   `Optional.empty`
   ```
   
   
   and `Or returns batch processing time + the duration...` 
   keeps the verb tense consistent with the previous "Returns the timestamp"
   
   
   

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/streaming/TestGroupState.scala
##########
@@ -0,0 +1,161 @@
+/*
+ * 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._
+
+/**
+ * The extended version of [[GroupState]] interface with extra getters of 
state machine fields

Review comment:
       I think you need htis as well 
   `  ::Experimental::`
   See GroupState
   




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

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



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

Reply via email to