Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16758#discussion_r99747455
  
    --- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/MapGroupsWithStateSuite.scala
 ---
    @@ -0,0 +1,240 @@
    +/*
    + * 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.scalatest.BeforeAndAfterAll
    +
    +import org.apache.spark.SparkException
    +import org.apache.spark.sql.KeyedState
    +import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._
    +import org.apache.spark.sql.execution.streaming.{KeyedStateImpl, 
MemoryStream}
    +import org.apache.spark.sql.execution.streaming.state.StateStore
    +
    +/** Class to check custom state types */
    +case class RunningCount(count: Long)
    +
    +class MapGroupsWithStateSuite extends StreamTest with BeforeAndAfterAll {
    +
    +  import testImplicits._
    +
    +  override def afterAll(): Unit = {
    +    super.afterAll()
    +    StateStore.stop()
    +  }
    +
    +  test("state - get, exists, update, remove") {
    +    var state: KeyedStateImpl[String] = null
    +
    +    def testState(
    +        expectedData: Option[String],
    +        shouldBeUpdated: Boolean = false,
    +        shouldBeRemoved: Boolean = false
    +      ): Unit = {
    +      if (expectedData.isDefined) {
    +        assert(state.exists)
    +        assert(state.get === expectedData.get)
    +      } else {
    +        assert(!state.exists)
    +        assert(state.get === null)
    +      }
    +      assert(state.isUpdated === shouldBeUpdated)
    +      assert(state.isRemoved === shouldBeRemoved)
    +    }
    +
    +    // Updating empty state
    +    state = KeyedStateImpl[String](null)
    +    testState(None)
    +    state.update("")
    +    testState(Some(""), shouldBeUpdated = true)
    +
    +    // Updating exiting state
    +    state = KeyedStateImpl[String]("2")
    +    testState(Some("2"))
    +    state.update("3")
    +    testState(Some("3"), shouldBeUpdated = true)
    +
    +    // Removing state
    +    state.remove()
    +    testState(None, shouldBeRemoved = true, shouldBeUpdated = false)
    +    state.remove()      // should be still callable
    +    state.update("4")
    +    testState(Some("4"), shouldBeRemoved = false, shouldBeUpdated = true)
    +
    +    // Updating by null is same as remove
    +    state.update(null)
    +    testState(None, shouldBeRemoved = true, shouldBeUpdated = false)
    +  }
    +
    +  test("flatMapGroupsWithState - streaming") {
    +    // Function to maintain running count up to 2, and then remove the 
count
    +    // Returns the data and the count if state is defined, otherwise does 
not return anything
    +    val stateFunc = (key: String, values: Iterator[String], state: 
KeyedState[RunningCount]) => {
    +
    +      var count = Option(state.get).map(_.count).getOrElse(0L) + 
values.size
    +      if (count == 3) {
    +        state.remove()
    +        Iterator.empty
    +      } else {
    +        state.update(RunningCount(count))
    +        Iterator((key, count.toString))
    +      }
    +    }
    +
    +    val inputData = MemoryStream[String]
    +    val result =
    +      inputData.toDS()
    +        .groupByKey(x => x)
    +        .flatMapGroupsWithState(stateFunc) // State: Int, Out: (Str, Str)
    +
    +    testStream(result, Append)(
    +      AddData(inputData, "a"),
    +      CheckLastBatch(("a", "1")),
    +      assertNumStateRows(total = 1, updated = 1),
    +      AddData(inputData, "a", "b"),
    +      CheckLastBatch(("a", "2"), ("b", "1")),
    +      assertNumStateRows(total = 2, updated = 2),
    +      StopStream,
    +      StartStream(),
    +      AddData(inputData, "a", "b"), // should remove state for "a" and not 
return anything for a
    +      CheckLastBatch(("b", "2")),
    +      assertNumStateRows(total = 1, updated = 2),
    +      StopStream,
    +      StartStream(),
    +      AddData(inputData, "a", "c"), // should recreate state for "a" and 
return count as 1 and
    +      CheckLastBatch(("a", "1"), ("c", "1")),
    +      assertNumStateRows(total = 3, updated = 2)
    +    )
    +  }
    +
    +  test("flatMapGroupsWithState - batch") {
    +    // Function that returns running count only if its even, otherwise 
does not return
    +    val stateFunc = (key: String, values: Iterator[String], state: 
KeyedState[RunningCount]) => {
    +      if (state.exists) throw new IllegalArgumentException("state.exists 
should be false")
    +      if (state.exists) {
    --- End diff --
    
    done. not a nit! wrong test!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to