Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -46,8 +46,13 @@ object UnsupportedOperationChecker {
"Queries without streaming sources cannot be executed with writeStream.start()")(plan)
}

/** Collect all the streaming aggregates in a sub plan */
def collectStreamingAggregates(subplan: LogicalPlan): Seq[Aggregate] = {
subplan.collect { case a: Aggregate if a.isStreaming => a }
}

// Disallow multiple streaming aggregations
val aggregates = plan.collect { case a@Aggregate(_, _, _) if a.isStreaming => a }
val aggregates = collectStreamingAggregates(plan)

if (aggregates.size > 1) {
throwError(
Expand Down Expand Up @@ -111,6 +116,10 @@ object UnsupportedOperationChecker {
throwError("Commands like CreateTable*, AlterTable*, Show* are not supported with " +
"streaming DataFrames/Datasets")

case m: MapGroupsWithState if collectStreamingAggregates(m).nonEmpty =>
throwError("(map/flatMap)GroupsWithState is not supported after aggregation on a " +
"streaming DataFrame/Dataset")

case Join(left, right, joinType, _) =>

joinType match {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -313,6 +313,55 @@ case class MapGroups(
outputObjAttr: Attribute,
child: LogicalPlan) extends UnaryNode with ObjectProducer

/** Internal class representing State */
trait LogicalKeyedState[S]

/** Factory for constructing new `MapGroupsWithState` nodes. */
object MapGroupsWithState {
def apply[K: Encoder, V: Encoder, S: Encoder, U: Encoder](
func: (Any, Iterator[Any], LogicalKeyedState[Any]) => Iterator[Any],
groupingAttributes: Seq[Attribute],
dataAttributes: Seq[Attribute],
child: LogicalPlan): LogicalPlan = {
val mapped = new MapGroupsWithState(
func,
UnresolvedDeserializer(encoderFor[K].deserializer, groupingAttributes),
UnresolvedDeserializer(encoderFor[V].deserializer, dataAttributes),
groupingAttributes,
dataAttributes,
CatalystSerde.generateObjAttr[U],
encoderFor[S].resolveAndBind().deserializer,
encoderFor[S].namedExpressions,
child)
CatalystSerde.serialize[U](mapped)
}
}

/**
* Applies func to each unique group in `child`, based on the evaluation of `groupingAttributes`,
* while using state data.
* Func is invoked with an object representation of the grouping key an iterator containing the
* object representation of all the rows with that key.
*
* @param keyDeserializer used to extract the key object for each group.
* @param valueDeserializer used to extract the items in the iterator from an input row.
* @param groupingAttributes used to group the data
* @param dataAttributes used to read the data
* @param outputObjAttr used to define the output object
* @param stateDeserializer used to deserialize state before calling `func`
* @param stateSerializer used to serialize updated state after calling `func`
*/
case class MapGroupsWithState(
func: (Any, Iterator[Any], LogicalKeyedState[Any]) => Iterator[Any],
keyDeserializer: Expression,
valueDeserializer: Expression,
groupingAttributes: Seq[Attribute],
dataAttributes: Seq[Attribute],
outputObjAttr: Attribute,
stateDeserializer: Expression,
stateSerializer: Seq[NamedExpression],
child: LogicalPlan) extends UnaryNode with ObjectProducer

/** Factory for constructing new `FlatMapGroupsInR` nodes. */
object FlatMapGroupsInR {
def apply(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,13 +22,13 @@ import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, NamedExpression}
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Literal, NamedExpression}
import org.apache.spark.sql.catalyst.expressions.aggregate.Count
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.logical.{MapGroupsWithState, _}
import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._
import org.apache.spark.sql.streaming.OutputMode
import org.apache.spark.sql.types.IntegerType
import org.apache.spark.sql.types.{IntegerType, LongType}

/** A dummy command for testing unsupported operations. */
case class DummyCommand() extends Command
Expand Down Expand Up @@ -111,6 +111,24 @@ class UnsupportedOperationsSuite extends SparkFunSuite {
outputMode = Complete,
expectedMsgs = Seq("distinct aggregation"))

// MapGroupsWithState: Not supported after a streaming aggregation
val att = new AttributeReference(name = "a", dataType = LongType)()
assertSupportedInBatchPlan(
"mapGroupsWithState - mapGroupsWithState on batch relation",
MapGroupsWithState(null, att, att, Seq(att), Seq(att), att, att, Seq(att), batchRelation))

assertSupportedInStreamingPlan(
"mapGroupsWithState - mapGroupsWithState on streaming relation before aggregation",
MapGroupsWithState(null, att, att, Seq(att), Seq(att), att, att, Seq(att), streamRelation),
outputMode = Append)

assertNotSupportedInStreamingPlan(
"mapGroupsWithState - mapGroupsWithState on streaming relation after aggregation",
MapGroupsWithState(null, att, att, Seq(att), Seq(att), att, att, Seq(att),
Aggregate(Nil, aggExprs("c"), streamRelation)),
outputMode = Complete,
expectedMsgs = Seq("(map/flatMap)GroupsWithState"))

// Inner joins: Stream-stream not supported
testBinaryOperationInStreamingPlan(
"inner join",
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
/*
* 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.Experimental;
import org.apache.spark.annotation.InterfaceStability;
import org.apache.spark.sql.Encoder;
import org.apache.spark.sql.KeyedState;

/**
* ::Experimental::
* Base interface for a map function used in
* {@link org.apache.spark.sql.KeyValueGroupedDataset#flatMapGroupsWithState(FlatMapGroupsWithStateFunction, Encoder, Encoder)}.
* @since 2.1.1
*/
@Experimental
@InterfaceStability.Evolving
public interface FlatMapGroupsWithStateFunction<K, V, S, R> extends Serializable {
Iterator<R> call(K key, Iterator<V> values, KeyedState<S> state) throws Exception;
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
/*
* 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.Experimental;
import org.apache.spark.annotation.InterfaceStability;
import org.apache.spark.sql.Encoder;
import org.apache.spark.sql.KeyedState;

/**
* ::Experimental::
* Base interface for a map function used in
* {@link org.apache.spark.sql.KeyValueGroupedDataset#mapGroupsWithState(MapGroupsWithStateFunction, Encoder, Encoder)}
* @since 2.1.1
*/
@Experimental
@InterfaceStability.Evolving
public interface MapGroupsWithStateFunction<K, V, S, R> extends Serializable {
R call(K key, Iterator<V> values, KeyedState<S> state) throws Exception;
}
Original file line number Diff line number Diff line change
Expand Up @@ -218,6 +218,119 @@ class KeyValueGroupedDataset[K, V] private[sql](
mapGroups((key, data) => f.call(key, data.asJava))(encoder)
}

/**
* ::Experimental::
* (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 [[KeyedState]] 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.
*
* See [[Encoder]] for more details on what types are encodable to Spark SQL.
* @since 2.1.1
*/
@Experimental
@InterfaceStability.Evolving
def mapGroupsWithState[S: Encoder, U: Encoder](
func: (K, Iterator[V], KeyedState[S]) => U): Dataset[U] = {
flatMapGroupsWithState[S, U](
(key: K, it: Iterator[V], s: KeyedState[S]) => Iterator(func(key, it, s)))
}

/**
* ::Experimental::
* (Java-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 [[KeyedState]] 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 stateEncoder Encoder for the state type.
* @param outputEncoder Encoder for the output type.
*
* See [[Encoder]] for more details on what types are encodable to Spark SQL.
* @since 2.1.1
*/
@Experimental
@InterfaceStability.Evolving
def mapGroupsWithState[S, U](
func: MapGroupsWithStateFunction[K, V, S, U],
stateEncoder: Encoder[S],
outputEncoder: Encoder[U]): Dataset[U] = {
flatMapGroupsWithState[S, U](
(key: K, it: Iterator[V], s: KeyedState[S]) => Iterator(func.call(key, it.asJava, s))
)(stateEncoder, outputEncoder)
}

/**
* ::Experimental::
* (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 [[KeyedState]] 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.
*
* See [[Encoder]] for more details on what types are encodable to Spark SQL.
* @since 2.1.1
*/
@Experimental
@InterfaceStability.Evolving
def flatMapGroupsWithState[S: Encoder, U: Encoder](
func: (K, Iterator[V], KeyedState[S]) => Iterator[U]): Dataset[U] = {
Dataset[U](
sparkSession,
MapGroupsWithState[K, V, S, U](
func.asInstanceOf[(Any, Iterator[Any], LogicalKeyedState[Any]) => Iterator[Any]],
groupingAttributes,
dataAttributes,
logicalPlan))
}

/**
* ::Experimental::
* (Java-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 [[KeyedState]] 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 stateEncoder Encoder for the state type.
* @param outputEncoder Encoder for the output type.
*
* See [[Encoder]] for more details on what types are encodable to Spark SQL.
* @since 2.1.1
*/
@Experimental
@InterfaceStability.Evolving
def flatMapGroupsWithState[S, U](
func: FlatMapGroupsWithStateFunction[K, V, S, U],
stateEncoder: Encoder[S],
outputEncoder: Encoder[U]): Dataset[U] = {
flatMapGroupsWithState[S, U](
(key: K, it: Iterator[V], s: KeyedState[S]) => func.call(key, it.asJava, s).asScala
)(stateEncoder, outputEncoder)
}

/**
* (Scala-specific)
* Reduces the elements of each group of data using the specified binary function.
Expand Down
Loading