pengzhon-db commented on code in PR #40959:
URL: https://github.com/apache/spark/pull/40959#discussion_r1198361197


##########
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
+ */
+@Experimental
+@Evolving
+public interface FlatMapGroupsWithStateFunction<K, V, S, R> extends 
Serializable {

Review Comment:
   Can we not reuse the existing one? Similar for MapGroupsWithStateFunction



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -312,6 +313,154 @@ 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 2.2.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 2.2.0

Review Comment:
   update the version



##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -904,6 +905,29 @@ message ApplyInPandasWithState {
   string timeout_conf = 7;
 }
 
+message FlatMapGroupsWithState {
+  // (Required) One input relation for CoGroup Map API - applyInPandas.
+  Relation input = 1;
+
+  // Expressions for grouping keys.
+  repeated Expression grouping_expressions = 2;
+
+  // Expressions for grouping keys of the initial input relation.
+  repeated Expression initial_grouping_expressions = 3;
+
+  // (Required) Input user-defined function.
+  CommonInlineUserDefinedFunction func = 4;
+
+  // (Required) Is MapGroupsWithState or not.
+  bool is_map_groups_with_state = 5;
+
+  // (Optional) The output mode of the function.
+  optional string output_mode = 6;

Review Comment:
   I don't think we need `optional`



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -312,6 +313,154 @@ 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 2.2.0

Review Comment:
   this version should be updated. Similar for the following methods



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -312,6 +313,154 @@ 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 2.2.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 2.2.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.2.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

Review Comment:
   why this is unsupported?



##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala:
##########
@@ -312,6 +313,154 @@ 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 2.2.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 2.2.0
+   */
+  def mapGroupsWithState[S: Encoder, U: Encoder](timeoutConf: 
GroupStateTimeout)(
+      func: (K, Iterator[V], GroupState[S]) => U): Dataset[U] = {
+    throw new UnsupportedOperationException

Review Comment:
   why this is unsupported?



##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -904,6 +905,29 @@ message ApplyInPandasWithState {
   string timeout_conf = 7;
 }
 
+message FlatMapGroupsWithState {
+  // (Required) One input relation for CoGroup Map API - applyInPandas.

Review Comment:
   the comment seems not correct



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

Reply via email to