anishshri-db commented on code in PR #49488:
URL: https://github.com/apache/spark/pull/49488#discussion_r1924447293
##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -525,6 +551,71 @@ private class KeyValueGroupedDatasetImpl[K, V, IK, IV](
}
}
+ override protected[sql] def transformWithStateHelper[U: Encoder, S: Encoder](
+ statefulProcessor: StatefulProcessor[K, V, U],
+ timeMode: TimeMode,
+ outputMode: OutputMode,
+ initialState: Option[KeyValueGroupedDataset[K, S]] = None,
+ eventTimeColumnName: String = ""): Dataset[U] = {
+ val outputEncoder = agnosticEncoderFor[U]
+ val stateEncoder = agnosticEncoderFor[S]
+
+ val inputEncoders: Seq[AgnosticEncoder[_]] = Seq(kEncoder, stateEncoder,
ivEncoder)
+ val dummyGroupingFunc = SparkUserDefinedFunction(
+ function = UdfUtils.noOp[K, U](),
+ inputEncoders = inputEncoders,
+ outputEncoder = outputEncoder)
+ val udf = dummyGroupingFunc
+ .apply(inputEncoders.map(_ => col("*")): _*)
+ .expr
+ .getCommonInlineUserDefinedFunction
+
+ val initialStateImpl = if (initialState.isDefined) {
+ assert(initialState.get.isInstanceOf[KeyValueGroupedDatasetImpl[K, S, _,
_]])
+ initialState.get.asInstanceOf[KeyValueGroupedDatasetImpl[K, S, _, _]]
+ } else {
+ null
+ }
+ val statefulProcessorStr = if (!initialState.isDefined) {
Review Comment:
nit: pls add newline here
##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateConnectSuite.scala:
##########
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.streaming
+
+import java.io.{BufferedWriter, FileWriter}
+import java.nio.file.{Files, Paths}
+import java.sql.Timestamp
+
+import org.scalatest.concurrent.Eventually.eventually
+import org.scalatest.concurrent.Futures.timeout
+import org.scalatest.time.SpanSugar._
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{DataFrame, Dataset, Encoders, Row, SparkSession}
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.test.{QueryTest, RemoteSparkSession}
+import org.apache.spark.sql.types._
+
+// A basic stateful processor which will return the occurrences of key
+class BasicCountStatefulProcessor
+ extends StatefulProcessor[String, (String, String), (String, String)]
+ with Logging {
+ @transient protected var _countState: ValueState[Long] = _
+
+ override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+ _countState = getHandle.getValueState[Long]("countState",
Encoders.scalaLong, TTLConfig.NONE)
+ }
+
+ override def handleInputRows(
+ key: String,
+ inputRows: Iterator[(String, String)],
+ timerValues: TimerValues): Iterator[(String, String)] = {
+ val count = _countState.getOption().getOrElse(0L) + inputRows.toSeq.length
+ _countState.update(count)
+ Iterator((key, count.toString))
+ }
+}
+
+// A stateful processor with initial state which will return the occurrences
of key
+class TestInitialStatefulProcessor
+ extends StatefulProcessorWithInitialState[
+ String,
+ (String, String),
+ (String, String),
+ (String, String, String)]
+ with Logging {
+ @transient protected var _countState: ValueState[Long] = _
+
+ override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {
+ _countState = getHandle.getValueState[Long]("countState",
Encoders.scalaLong, TTLConfig.NONE)
+ }
+
+ override def handleInputRows(
+ key: String,
+ inputRows: Iterator[(String, String)],
+ timerValues: TimerValues): Iterator[(String, String)] = {
+ val count = _countState.getOption().getOrElse(0L) + inputRows.toSeq.length
+ _countState.update(count)
+ Iterator((key, count.toString))
+ }
+
+ override def handleInitialState(
+ key: String,
+ initialState: (String, String, String),
+ timerValues: TimerValues): Unit = {
+ val count = _countState.getOption().getOrElse(0L) + 1
+ _countState.update(count)
+ }
+}
+
+case class OutputEventTimeRow(key: String, outputTimestamp: Timestamp)
+
+// A stateful processor which will return timestamp of the first item from
input rows
+class ChainingOfOpsStatefulProcessor
+ extends StatefulProcessor[String, (String, Timestamp), OutputEventTimeRow]
{
+ override def init(outputMode: OutputMode, timeMode: TimeMode): Unit = {}
+
+ override def handleInputRows(
+ key: String,
+ inputRows: Iterator[(String, Timestamp)],
+ timerValues: TimerValues): Iterator[OutputEventTimeRow] = {
+ val timestamp = inputRows.next()._2
+ Iterator(OutputEventTimeRow(key, timestamp))
+ }
+}
+
+class TransformWithStateConnectSuite extends QueryTest with RemoteSparkSession
with Logging {
+ val testData: Seq[(String, String)] = Seq(("a", "1"), ("b", "1"), ("a", "2"))
+ val twsAdditionalSQLConf = Seq(
+ "spark.sql.streaming.stateStore.providerClass" ->
+
"org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider",
+ "spark.sql.shuffle.partitions" -> "5",
+ "spark.sql.session.timeZone" -> "UTC")
+
+ test("transformWithState - streaming with state variable") {
Review Comment:
Are we covering all composite types ? Also, are we covering TTL ? Also -
cases where the state is written using `case class` to make sure that the serde
is working as expected ?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]