Github user zsxwing commented on a diff in the pull request: https://github.com/apache/spark/pull/16758#discussion_r99238895 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/KeyedState.scala --- @@ -0,0 +1,134 @@ +/* + * 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 + +import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.sql.catalyst.plans.logical.LogicalKeyedState + +/** + * :: Experimental :: + * + * Wrapper class for interacting with keyed state data in `mapGroupsWithState` and + * `flatMapGroupsWithState` operations on + * [[KeyValueGroupedDataset]]. + * + * Detail description on `[map/flatMap]GroupsWithState` operation + * ------------------------------------------------------------ + * Both, `mapGroupsWithState` and `flatMapGroupsWithState` in [[KeyValueGroupedDataset]] + * will invoke the user-given function on each group (defined by the grouping function in + * `Dataset.groupByKey()`) while maintaining user-defined per-group state between invocations. + * 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. + * That is, in every batch of the [[streaming.StreamingQuery StreamingQuery]], + * the function will be invoked once for each group that has data in the batch. + * + * The function is invoked with following parameters. + * - The key of the group. + * - An iterator containing all the values for this key. + * - A user-defined state object set by previous invocations of the given function. + * In case of a batch Dataset, there is only invocation and state object will be empty as + * there is no prior state. Essentially, for batch Datasets, `[map/flatMap]GroupsWithState` + * is equivalent to `[map/flatMap]Groups`. + * + * Important points to note about the function. + * - In a trigger, the function will be called only the groups present in the batch. So do not + * assume that the function will be called in every trigger for every group that has state. + * - There is no guaranteed ordering of values in the iterator in the function, neither with + * batch, nor with streaming Datasets. + * - All the data will be shuffled before applying the function. + * + * Important points to note about using KeyedState. + * - The value of the state cannot be null. So updating state with null is same as removing it. + * - Operations on `KeyedState` are not thread-safe. This is to avoid memory barriers. + * - If the `remove()` is called, then `exists()` will return `false`, and + * `getOption()` will return `None`. + * - After that `update(newState)` is called, then `exists()` will return `true`, + * and `getOption()` will return `Some(...)`. + * + * Scala example of using `KeyedState` in `mapGroupsWithState`: + * {{{ + * // A mapping function that maintains an integer state for string keys and returns a string. + * def mappingFunction(key: String, value: Iterable[Int], state: KeyedState[Int]): Option[String]= { + * // Check if state exists + * if (state.exists) { + * val existingState = state.get // Get the existing state + * val shouldRemove = ... // Decide whether to remove the state + * if (shouldRemove) { + * state.remove() // Remove the state + * } else { + * val newState = ... + * state.update(newState) // Set the new state + * } + * } else { + * val initialState = ... + * state.update(initialState) // Set the initial state + * } + * ... // return something + * } + * + * }}} + * + * Java example of using `KeyedState`: + * {{{ + * // A mapping function that maintains an integer state for string keys and returns a string. + * MapGroupsWithStateFunction<String, Integer, Integer, String> mappingFunction = + * new MapGroupsWithStateFunction<String, Integer, Integer, String>() { + * + * @Override + * public String call(String key, Optional<Integer> value, KeyedState<Integer> state) { --- End diff -- nit: `Optional<Integer>` -> `Iterator<Integer>`
--- 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