zhenlineo commented on code in PR #41558:
URL: https://github.com/apache/spark/pull/41558#discussion_r1227339536
##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -460,6 +461,159 @@ abstract class KeyValueGroupedDataset[K, V] private[sql]
() extends Serializable
cogroupSorted(other)(thisSortExprs: _*)(otherSortExprs: _*)(
UdfUtils.coGroupFunctionToScalaFunc(f))(encoder)
}
+
+ /**
+ * (Scala-specific) Applies the given function to each group of data, while
maintaining a
+ * user-defined per-group state. The result Dataset will represent the
objects returned by the
+ * function. For a static batch Dataset, the function will be invoked once
per group. For a
+ * streaming Dataset, the function will be invoked for each group repeatedly
in every trigger,
+ * and updates to each group's state will be saved across invocations. See
+ * [[org.apache.spark.sql.streaming.GroupState]] for more details.
+ *
+ * @tparam S
+ * The type of the user-defined state. Must be encodable to Spark SQL
types.
+ * @tparam U
+ * The type of the output objects. Must be encodable to Spark SQL types.
+ * @param func
+ * Function to be called on every group.
+ *
+ * See [[Encoder]] for more details on what types are encodable to Spark SQL.
+ * @since 3.5.0
+ */
+ def mapGroupsWithState[S: Encoder, U: Encoder](
+ func: (K, Iterator[V], GroupState[S]) => U): Dataset[U] = {
+ mapGroupsWithState(GroupStateTimeout.NoTimeout)(func)
+ }
+
+ /**
+ * (Scala-specific) Applies the given function to each group of data, while
maintaining a
+ * user-defined per-group state. The result Dataset will represent the
objects returned by the
+ * function. For a static batch Dataset, the function will be invoked once
per group. For a
+ * streaming Dataset, the function will be invoked for each group repeatedly
in every trigger,
+ * and updates to each group's state will be saved across invocations. See
+ * [[org.apache.spark.sql.streaming.GroupState]] for more details.
+ *
+ * @tparam S
+ * The type of the user-defined state. Must be encodable to Spark SQL
types.
+ * @tparam U
+ * The type of the output objects. Must be encodable to Spark SQL types.
+ * @param func
+ * Function to be called on every group.
+ * @param timeoutConf
+ * Timeout configuration for groups that do not receive data for a while.
+ *
+ * See [[Encoder]] for more details on what types are encodable to Spark SQL.
+ * @since 3.5.0
+ */
+ def mapGroupsWithState[S: Encoder, U: Encoder](timeoutConf:
GroupStateTimeout)(
+ func: (K, Iterator[V], GroupState[S]) => U): Dataset[U] = {
+ throw new UnsupportedOperationException
+ }
+
+ /**
+ * (Scala-specific) Applies the given function to each group of data, while
maintaining a
+ * user-defined per-group state. The result Dataset will represent the
objects returned by the
+ * function. For a static batch Dataset, the function will be invoked once
per group. For a
+ * streaming Dataset, the function will be invoked for each group repeatedly
in every trigger,
+ * and updates to each group's state will be saved across invocations. See
+ * [[org.apache.spark.sql.streaming.GroupState]] for more details.
+ *
+ * @tparam S
+ * The type of the user-defined state. Must be encodable to Spark SQL
types.
+ * @tparam U
+ * The type of the output objects. Must be encodable to Spark SQL types.
+ * @param func
+ * Function to be called on every group.
+ * @param timeoutConf
+ * Timeout Conf, see GroupStateTimeout for more details
+ * @param initialState
+ * The user provided state that will be initialized when the first batch
of data is processed
+ * in the streaming query. The user defined function will be called on the
state data even if
+ * there are no other values in the group. To convert a Dataset ds of type
Dataset[(K, S)] to
+ * a KeyValueGroupedDataset[K, S] do {{{ds.groupByKey(x =>
x._1).mapValues(_._2)}}}
+ *
+ * See [[Encoder]] for more details on what types are encodable to Spark SQL.
+ * @since 3.5.0
+ */
+ def mapGroupsWithState[S: Encoder, U: Encoder](
+ timeoutConf: GroupStateTimeout,
+ initialState: KeyValueGroupedDataset[K, S])(
+ func: (K, Iterator[V], GroupState[S]) => U): Dataset[U] = {
+ throw new UnsupportedOperationException
+ }
+
+ /**
+ * (Scala-specific) Applies the given function to each group of data, while
maintaining a
+ * user-defined per-group state. The result Dataset will represent the
objects returned by the
+ * function. For a static batch Dataset, the function will be invoked once
per group. For a
+ * streaming Dataset, the function will be invoked for each group repeatedly
in every trigger,
+ * and updates to each group's state will be saved across invocations. See
`GroupState` for more
+ * details.
+ *
+ * @tparam S
+ * The type of the user-defined state. Must be encodable to Spark SQL
types.
+ * @tparam U
+ * The type of the output objects. Must be encodable to Spark SQL types.
+ * @param func
+ * Function to be called on every group.
+ * @param outputMode
+ * The output mode of the function.
+ * @param timeoutConf
+ * Timeout configuration for groups that do not receive data for a while.
+ *
+ * See [[Encoder]] for more details on what types are encodable to Spark SQL.
+ * @since 3.5.0
+ */
+ def flatMapGroupsWithState[S: Encoder, U: Encoder](
+ outputMode: OutputMode,
+ timeoutConf: GroupStateTimeout)(
+ func: (K, Iterator[V], GroupState[S]) => Iterator[U]): Dataset[U] = {
+ throw new UnsupportedOperationException
+ }
+
+ /**
+ * (Scala-specific) Applies the given function to each group of data, while
maintaining a
+ * user-defined per-group state. The result Dataset will represent the
objects returned by the
+ * function. For a static batch Dataset, the function will be invoked once
per group. For a
+ * streaming Dataset, the function will be invoked for each group repeatedly
in every trigger,
+ * and updates to each group's state will be saved across invocations. See
`GroupState` for more
+ * details.
+ *
+ * @tparam S
+ * The type of the user-defined state. Must be encodable to Spark SQL
types.
+ * @tparam U
+ * The type of the output objects. Must be encodable to Spark SQL types.
+ * @param func
+ * Function to be called on every group.
+ * @param outputMode
+ * The output mode of the function.
+ * @param timeoutConf
+ * Timeout configuration for groups that do not receive data for a while.
+ * @param initialState
+ * The user provided state that will be initialized when the first batch
of data is processed
+ * in the streaming query. The user defined function will be called on the
state data even if
+ * there are no other values in the group. To covert a Dataset `ds` of
type of type
+ * `Dataset[(K, S)]` to a `KeyValueGroupedDataset[K, S]`, use
+ * {{{ds.groupByKey(x => x._1).mapValues(_._2)}}} See [[Encoder]] for more
details on what
+ * types are encodable to Spark SQL.
+ * @since 3.5.0
+ */
+ def flatMapGroupsWithState[S: Encoder, U: Encoder](
+ outputMode: OutputMode,
+ timeoutConf: GroupStateTimeout,
+ initialState: KeyValueGroupedDataset[K, S])(
+ func: (K, Iterator[V], GroupState[S]) => Iterator[U]): Dataset[U] = {
+ throw new UnsupportedOperationException
+ }
+
+ /**
+ * Returns the current java.util.List[proto.Expression].
+ *
+ * @since 3.5.0
+ */
+ def getGroupingExpressions: java.util.List[proto.Expression] = {
Review Comment:
This will go to the public API, do you intend to add this to public API?
##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -460,6 +461,159 @@ abstract class KeyValueGroupedDataset[K, V] private[sql]
() extends Serializable
cogroupSorted(other)(thisSortExprs: _*)(otherSortExprs: _*)(
UdfUtils.coGroupFunctionToScalaFunc(f))(encoder)
}
+
+ /**
+ * (Scala-specific) Applies the given function to each group of data, while
maintaining a
+ * user-defined per-group state. The result Dataset will represent the
objects returned by the
+ * function. For a static batch Dataset, the function will be invoked once
per group. For a
+ * streaming Dataset, the function will be invoked for each group repeatedly
in every trigger,
+ * and updates to each group's state will be saved across invocations. See
+ * [[org.apache.spark.sql.streaming.GroupState]] for more details.
+ *
+ * @tparam S
+ * The type of the user-defined state. Must be encodable to Spark SQL
types.
+ * @tparam U
+ * The type of the output objects. Must be encodable to Spark SQL types.
+ * @param func
+ * Function to be called on every group.
+ *
+ * See [[Encoder]] for more details on what types are encodable to Spark SQL.
+ * @since 3.5.0
+ */
+ def mapGroupsWithState[S: Encoder, U: Encoder](
+ func: (K, Iterator[V], GroupState[S]) => U): Dataset[U] = {
+ mapGroupsWithState(GroupStateTimeout.NoTimeout)(func)
+ }
+
+ /**
+ * (Scala-specific) Applies the given function to each group of data, while
maintaining a
+ * user-defined per-group state. The result Dataset will represent the
objects returned by the
+ * function. For a static batch Dataset, the function will be invoked once
per group. For a
+ * streaming Dataset, the function will be invoked for each group repeatedly
in every trigger,
+ * and updates to each group's state will be saved across invocations. See
+ * [[org.apache.spark.sql.streaming.GroupState]] for more details.
+ *
+ * @tparam S
+ * The type of the user-defined state. Must be encodable to Spark SQL
types.
+ * @tparam U
+ * The type of the output objects. Must be encodable to Spark SQL types.
+ * @param func
+ * Function to be called on every group.
+ * @param timeoutConf
+ * Timeout configuration for groups that do not receive data for a while.
+ *
+ * See [[Encoder]] for more details on what types are encodable to Spark SQL.
+ * @since 3.5.0
+ */
+ def mapGroupsWithState[S: Encoder, U: Encoder](timeoutConf:
GroupStateTimeout)(
+ func: (K, Iterator[V], GroupState[S]) => U): Dataset[U] = {
+ throw new UnsupportedOperationException
+ }
+
+ /**
+ * (Scala-specific) Applies the given function to each group of data, while
maintaining a
+ * user-defined per-group state. The result Dataset will represent the
objects returned by the
+ * function. For a static batch Dataset, the function will be invoked once
per group. For a
+ * streaming Dataset, the function will be invoked for each group repeatedly
in every trigger,
+ * and updates to each group's state will be saved across invocations. See
+ * [[org.apache.spark.sql.streaming.GroupState]] for more details.
+ *
+ * @tparam S
+ * The type of the user-defined state. Must be encodable to Spark SQL
types.
+ * @tparam U
+ * The type of the output objects. Must be encodable to Spark SQL types.
+ * @param func
+ * Function to be called on every group.
+ * @param timeoutConf
+ * Timeout Conf, see GroupStateTimeout for more details
+ * @param initialState
+ * The user provided state that will be initialized when the first batch
of data is processed
+ * in the streaming query. The user defined function will be called on the
state data even if
+ * there are no other values in the group. To convert a Dataset ds of type
Dataset[(K, S)] to
+ * a KeyValueGroupedDataset[K, S] do {{{ds.groupByKey(x =>
x._1).mapValues(_._2)}}}
+ *
+ * See [[Encoder]] for more details on what types are encodable to Spark SQL.
+ * @since 3.5.0
+ */
+ def mapGroupsWithState[S: Encoder, U: Encoder](
+ timeoutConf: GroupStateTimeout,
+ initialState: KeyValueGroupedDataset[K, S])(
+ func: (K, Iterator[V], GroupState[S]) => U): Dataset[U] = {
+ throw new UnsupportedOperationException
+ }
+
+ /**
+ * (Scala-specific) Applies the given function to each group of data, while
maintaining a
+ * user-defined per-group state. The result Dataset will represent the
objects returned by the
+ * function. For a static batch Dataset, the function will be invoked once
per group. For a
+ * streaming Dataset, the function will be invoked for each group repeatedly
in every trigger,
+ * and updates to each group's state will be saved across invocations. See
`GroupState` for more
+ * details.
+ *
+ * @tparam S
+ * The type of the user-defined state. Must be encodable to Spark SQL
types.
+ * @tparam U
+ * The type of the output objects. Must be encodable to Spark SQL types.
+ * @param func
+ * Function to be called on every group.
+ * @param outputMode
+ * The output mode of the function.
+ * @param timeoutConf
+ * Timeout configuration for groups that do not receive data for a while.
+ *
+ * See [[Encoder]] for more details on what types are encodable to Spark SQL.
+ * @since 3.5.0
+ */
+ def flatMapGroupsWithState[S: Encoder, U: Encoder](
Review Comment:
The only diff between this and the method with the same name is if
`initialState` is provided or not. Can you delegate the impl of these two
methods to one protected method internally?
```
protected def flatMapGroupsWithState( ..., optionalInitState...)(...) = {
throw new UnsupportedOperationException
}
```
And then impl in the child class? See `aggUntyped` as an example.
##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -905,6 +906,29 @@ message ApplyInPandasWithState {
string timeout_conf = 7;
}
+message FlatMapGroupsWithState {
Review Comment:
Is it the same as `GroupMap`? Any reason that you cannot reuse the same
message?
##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -460,6 +461,159 @@ abstract class KeyValueGroupedDataset[K, V] private[sql]
() extends Serializable
cogroupSorted(other)(thisSortExprs: _*)(otherSortExprs: _*)(
UdfUtils.coGroupFunctionToScalaFunc(f))(encoder)
}
+
+ /**
+ * (Scala-specific) Applies the given function to each group of data, while
maintaining a
+ * user-defined per-group state. The result Dataset will represent the
objects returned by the
+ * function. For a static batch Dataset, the function will be invoked once
per group. For a
+ * streaming Dataset, the function will be invoked for each group repeatedly
in every trigger,
+ * and updates to each group's state will be saved across invocations. See
+ * [[org.apache.spark.sql.streaming.GroupState]] for more details.
+ *
+ * @tparam S
+ * The type of the user-defined state. Must be encodable to Spark SQL
types.
+ * @tparam U
+ * The type of the output objects. Must be encodable to Spark SQL types.
+ * @param func
+ * Function to be called on every group.
+ *
+ * See [[Encoder]] for more details on what types are encodable to Spark SQL.
+ * @since 3.5.0
+ */
+ def mapGroupsWithState[S: Encoder, U: Encoder](
+ func: (K, Iterator[V], GroupState[S]) => U): Dataset[U] = {
+ mapGroupsWithState(GroupStateTimeout.NoTimeout)(func)
+ }
+
+ /**
+ * (Scala-specific) Applies the given function to each group of data, while
maintaining a
+ * user-defined per-group state. The result Dataset will represent the
objects returned by the
+ * function. For a static batch Dataset, the function will be invoked once
per group. For a
+ * streaming Dataset, the function will be invoked for each group repeatedly
in every trigger,
+ * and updates to each group's state will be saved across invocations. See
+ * [[org.apache.spark.sql.streaming.GroupState]] for more details.
+ *
+ * @tparam S
+ * The type of the user-defined state. Must be encodable to Spark SQL
types.
+ * @tparam U
+ * The type of the output objects. Must be encodable to Spark SQL types.
+ * @param func
+ * Function to be called on every group.
+ * @param timeoutConf
+ * Timeout configuration for groups that do not receive data for a while.
+ *
+ * See [[Encoder]] for more details on what types are encodable to Spark SQL.
+ * @since 3.5.0
+ */
+ def mapGroupsWithState[S: Encoder, U: Encoder](timeoutConf:
GroupStateTimeout)(
+ func: (K, Iterator[V], GroupState[S]) => U): Dataset[U] = {
Review Comment:
I feel `mapGroupsWithState` can be expressed using `flatMapGroupsWithState`.
Could you delegate the call to `flatMapGroupsWithState` to reduce the methods
that need to impl in the child class?
##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala:
##########
@@ -447,4 +449,20 @@ class KeyValueGroupedDatasetE2ETestSuite extends QueryTest
with SQLHelper {
checkDataset(keys, "1", "2", "10", "20")
}
+
+ test("flatMapGroupsWithState") {
+ val stateFunc = (key: String, values: Iterator[String], state:
GroupState[Int]) => {
+ if (state.exists) throw new IllegalArgumentException("state.exists
should be false")
+ Iterator((key, values.size))
+ }
+
+ val session: SparkSession = spark
+ import session.implicits._
+ val values = Seq("a", "a", "b", "c", "c", "c", "c").toDS()
+ .groupByKey(x => x)
+ .flatMapGroupsWithState(Append, GroupStateTimeout.NoTimeout)(stateFunc)
Review Comment:
Feels the conversion func is wrongly constructed. You can check out some
conversion code I wrote for `mapGroups -> flatMapGroups`?
```
def mapGroups[U: Encoder](f: (K, Iterator[V]) => U): Dataset[U] = {
flatMapGroups(UdfUtils.mapGroupsFuncToFlatMapAdaptor(f))
}
```
##########
connector/connect/common/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.api.java.function;
+
+import java.io.Serializable;
+import java.util.Iterator;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.streaming.GroupState;
+
+/**
+ * ::Experimental::
+ * Base interface for a map function used in
+ * {@code org.apache.spark.sql.KeyValueGroupedDataset.flatMapGroupsWithState(
+ * FlatMapGroupsWithStateFunction, org.apache.spark.sql.streaming.OutputMode,
+ * org.apache.spark.sql.Encoder, org.apache.spark.sql.Encoder)}
+ * @since 2.1.1
Review Comment:
This funcs need to be moved to the util package. I will leave it to you if
you want to directly depends on it or copy the files for now.
In the future we should refactoring with other such classes in the util
package.
--
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]