diff --git a/docs/ss-migration-guide.md b/docs/ss-migration-guide.md index 480e5e2695a16..ba50b506892b9 100644 --- a/docs/ss-migration-guide.md +++ b/docs/ss-migration-guide.md @@ -26,6 +26,10 @@ Note that this migration guide describes the items specific to Structured Stream Many items of SQL migration can be applied when migrating Structured Streaming to higher versions. Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.html). +## Upgrading from Structured Streaming to 3.2.2 + +- Since Spark 3.2.2 (and 3.3), all stateful operators require hash partitioning with exact grouping keys. In previous versions, all stateful operators except stream-stream join require loose partitioning criteria which opens the possibility on correctness issue. (See [SPARK-38204](https://issues.apache.org/jira/browse/SPARK-38204) for more details.) To ensure backward compatibility, we retain the old behavior with the checkpoint built from older versions. + ## Upgrading from Structured Streaming 3.0 to 3.1 - In Spark 3.0 and before, for the queries that have stateful operation which can emit rows older than the current watermark plus allowed late record delay, which are "late rows" in downstream stateful operations and these rows can be discarded, Spark only prints a warning message. Since Spark 3.1, Spark will check for such queries with possible correctness issue and throw AnalysisException for it by default. For the users who understand the possible risk of correctness issue and still decide to run the query, please disable this check by setting the config `spark.sql.streaming.statefulOperator.checkCorrectness.enabled` to false. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index fb7089c6aec9f..f1d9a0ca0ac74 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -89,11 +89,19 @@ case class ClusteredDistribution( /** * Represents data where tuples have been clustered according to the hash of the given - * `expressions`. The hash function is defined as `HashPartitioning.partitionIdExpression`, so only - * [[HashPartitioning]] can satisfy this distribution. + * `expressions`. Since this distribution relies on [[HashPartitioning]] on the physical + * partitioning, only [[HashPartitioning]] (and HashPartitioning in [[PartitioningCollection]]) + * can satisfy this distribution. When `requiredNumPartitions` is Some(1), [[SinglePartition]] + * is essentially same as [[HashPartitioning]], so it can satisfy this distribution as well. * - * This is a strictly stronger guarantee than [[ClusteredDistribution]]. Given a tuple and the - * number of partitions, this distribution strictly requires which partition the tuple should be in. + * This distribution is used majorly to represent the requirement of distribution on the stateful + * operator in Structured Streaming, but this can be used for other cases as well. + * + * NOTE: Each partition in stateful operator initializes state store(s), which are independent + * with state store(s) in other partitions. Since it is not possible to repartition the data in + * state store, Spark should make sure the physical partitioning of the stateful operator is + * unchanged across Spark versions. Violation of this requirement may bring silent correctness + * issue. */ case class HashClusteredDistribution( expressions: Seq[Expression], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index bd990d3499f8e..67f26c1a93968 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1656,6 +1656,23 @@ object SQLConf { .booleanConf .createWithDefault(true) + val STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION = + buildConf("spark.sql.streaming.statefulOperator.useStrictDistribution") + .internal() + .doc("The purpose of this config is only compatibility; DO NOT MANUALLY CHANGE THIS!!! " + + "When true, the stateful operator for streaming query will use " + + "HashClusteredDistribution which guarantees stable state partitioning as long as " + + "the operator provides consistent grouping keys across the lifetime of query. " + + "When false, the stateful operator for streaming query will use ClusteredDistribution " + + "which is not sufficient to guarantee stable state partitioning despite the operator " + + "provides consistent grouping keys across the lifetime of query. " + + "This config will be set to true for new streaming queries to guarantee stable state " + + "partitioning, and set to false for existing streaming queries to not break queries " + + "which are restored from existing checkpoints. Please refer SPARK-38204 for details.") + .version("3.2.2") + .booleanConf + .createWithDefault(true) + val FILESTREAM_SINK_METADATA_IGNORED = buildConf("spark.sql.streaming.fileStreamSink.ignoreMetadata") .internal() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala index 0f239b457fd14..fc3e1106b43cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala @@ -43,8 +43,28 @@ object AggUtils { } } + private def createStreamingAggregate( + requiredChildDistributionExpressions: Option[Seq[Expression]] = None, + groupingExpressions: Seq[NamedExpression] = Nil, + aggregateExpressions: Seq[AggregateExpression] = Nil, + aggregateAttributes: Seq[Attribute] = Nil, + initialInputBufferOffset: Int = 0, + resultExpressions: Seq[NamedExpression] = Nil, + child: SparkPlan): SparkPlan = { + createAggregate( + requiredChildDistributionExpressions, + isStreaming = true, + groupingExpressions = groupingExpressions, + aggregateExpressions = aggregateExpressions, + aggregateAttributes = aggregateAttributes, + initialInputBufferOffset = initialInputBufferOffset, + resultExpressions = resultExpressions, + child = child) + } + private def createAggregate( requiredChildDistributionExpressions: Option[Seq[Expression]] = None, + isStreaming: Boolean = false, groupingExpressions: Seq[NamedExpression] = Nil, aggregateExpressions: Seq[AggregateExpression] = Nil, aggregateAttributes: Seq[Attribute] = Nil, @@ -56,6 +76,8 @@ object AggUtils { if (useHash) { HashAggregateExec( requiredChildDistributionExpressions = requiredChildDistributionExpressions, + isStreaming = isStreaming, + numShufflePartitions = None, groupingExpressions = groupingExpressions, aggregateExpressions = mayRemoveAggFilters(aggregateExpressions), aggregateAttributes = aggregateAttributes, @@ -69,6 +91,8 @@ object AggUtils { if (objectHashEnabled && useObjectHash) { ObjectHashAggregateExec( requiredChildDistributionExpressions = requiredChildDistributionExpressions, + isStreaming = isStreaming, + numShufflePartitions = None, groupingExpressions = groupingExpressions, aggregateExpressions = mayRemoveAggFilters(aggregateExpressions), aggregateAttributes = aggregateAttributes, @@ -78,6 +102,8 @@ object AggUtils { } else { SortAggregateExec( requiredChildDistributionExpressions = requiredChildDistributionExpressions, + isStreaming = isStreaming, + numShufflePartitions = None, groupingExpressions = groupingExpressions, aggregateExpressions = mayRemoveAggFilters(aggregateExpressions), aggregateAttributes = aggregateAttributes, @@ -286,7 +312,7 @@ object AggUtils { val partialAggregate: SparkPlan = { val aggregateExpressions = functionsWithoutDistinct.map(_.copy(mode = Partial)) val aggregateAttributes = aggregateExpressions.map(_.resultAttribute) - createAggregate( + createStreamingAggregate( groupingExpressions = groupingExpressions, aggregateExpressions = aggregateExpressions, aggregateAttributes = aggregateAttributes, @@ -298,7 +324,7 @@ object AggUtils { val partialMerged1: SparkPlan = { val aggregateExpressions = functionsWithoutDistinct.map(_.copy(mode = PartialMerge)) val aggregateAttributes = aggregateExpressions.map(_.resultAttribute) - createAggregate( + createStreamingAggregate( requiredChildDistributionExpressions = Some(groupingAttributes), groupingExpressions = groupingAttributes, @@ -316,7 +342,7 @@ object AggUtils { val partialMerged2: SparkPlan = { val aggregateExpressions = functionsWithoutDistinct.map(_.copy(mode = PartialMerge)) val aggregateAttributes = aggregateExpressions.map(_.resultAttribute) - createAggregate( + createStreamingAggregate( requiredChildDistributionExpressions = Some(groupingAttributes), groupingExpressions = groupingAttributes, @@ -344,7 +370,7 @@ object AggUtils { // projection: val finalAggregateAttributes = finalAggregateExpressions.map(_.resultAttribute) - createAggregate( + createStreamingAggregate( requiredChildDistributionExpressions = Some(groupingAttributes), groupingExpressions = groupingAttributes, aggregateExpressions = finalAggregateExpressions, @@ -403,7 +429,7 @@ object AggUtils { val partialAggregate: SparkPlan = { val aggregateExpressions = functionsWithoutDistinct.map(_.copy(mode = Partial)) val aggregateAttributes = aggregateExpressions.map(_.resultAttribute) - createAggregate( + createStreamingAggregate( groupingExpressions = groupingExpressions, aggregateExpressions = aggregateExpressions, aggregateAttributes = aggregateAttributes, @@ -420,7 +446,8 @@ object AggUtils { // this is to reduce amount of rows to shuffle MergingSessionsExec( requiredChildDistributionExpressions = None, - requiredChildDistributionOption = None, + isStreaming = true, + numShufflePartitions = None, groupingExpressions = groupingAttributes, sessionExpression = sessionExpression, aggregateExpressions = aggregateExpressions, @@ -443,8 +470,10 @@ object AggUtils { val aggregateExpressions = functionsWithoutDistinct.map(_.copy(mode = PartialMerge)) val aggregateAttributes = aggregateExpressions.map(_.resultAttribute) MergingSessionsExec( - requiredChildDistributionExpressions = None, - requiredChildDistributionOption = Some(restored.requiredChildDistribution), + requiredChildDistributionExpressions = Some(groupingWithoutSessionAttributes), + isStreaming = true, + // This will be replaced with actual value in state rule. + numShufflePartitions = None, groupingExpressions = groupingAttributes, sessionExpression = sessionExpression, aggregateExpressions = aggregateExpressions, @@ -472,8 +501,8 @@ object AggUtils { // projection: val finalAggregateAttributes = finalAggregateExpressions.map(_.resultAttribute) - createAggregate( - requiredChildDistributionExpressions = Some(groupingAttributes), + createStreamingAggregate( + requiredChildDistributionExpressions = Some(groupingWithoutSessionAttributes), groupingExpressions = groupingAttributes, aggregateExpressions = finalAggregateExpressions, aggregateAttributes = finalAggregateAttributes, @@ -487,10 +516,15 @@ object AggUtils { private def mayAppendUpdatingSessionExec( groupingExpressions: Seq[NamedExpression], - maybeChildPlan: SparkPlan): SparkPlan = { + maybeChildPlan: SparkPlan, + isStreaming: Boolean = false): SparkPlan = { groupingExpressions.find(_.metadata.contains(SessionWindow.marker)) match { case Some(sessionExpression) => UpdatingSessionsExec( + isStreaming = isStreaming, + // numShufflePartitions will be set to None, and replaced to the actual value in the + // state rule if the query is streaming. + numShufflePartitions = None, groupingExpressions.map(_.toAttribute), sessionExpression.toAttribute, maybeChildPlan) @@ -502,7 +536,8 @@ object AggUtils { private def mayAppendMergingSessionExec( groupingExpressions: Seq[NamedExpression], aggregateExpressions: Seq[AggregateExpression], - partialAggregate: SparkPlan): SparkPlan = { + partialAggregate: SparkPlan, + isStreaming: Boolean = false): SparkPlan = { groupingExpressions.find(_.metadata.contains(SessionWindow.marker)) match { case Some(sessionExpression) => val aggExpressions = aggregateExpressions.map(_.copy(mode = PartialMerge)) @@ -515,7 +550,10 @@ object AggUtils { MergingSessionsExec( requiredChildDistributionExpressions = Some(groupingWithoutSessionsAttributes), - requiredChildDistributionOption = None, + isStreaming = isStreaming, + // numShufflePartitions will be set to None, and replaced to the actual value in the + // state rule if the query is streaming. + numShufflePartitions = None, groupingExpressions = groupingAttributes, sessionExpression = sessionExpression, aggregateExpressions = aggExpressions, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala index c676609bc37e4..d4d7845eb9e0d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/BaseAggregateExec.scala @@ -21,12 +21,15 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Final, PartialMerge} import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, UnspecifiedDistribution} import org.apache.spark.sql.execution.{AliasAwareOutputPartitioning, ExplainUtils, UnaryExecNode} +import org.apache.spark.sql.execution.streaming.StatefulOperatorPartitioning /** * Holds common logic for aggregate operators */ trait BaseAggregateExec extends UnaryExecNode with AliasAwareOutputPartitioning { def requiredChildDistributionExpressions: Option[Seq[Expression]] + def isStreaming: Boolean + def numShufflePartitions: Option[Int] def groupingExpressions: Seq[NamedExpression] def aggregateExpressions: Seq[AggregateExpression] def aggregateAttributes: Seq[Attribute] @@ -91,7 +94,20 @@ trait BaseAggregateExec extends UnaryExecNode with AliasAwareOutputPartitioning override def requiredChildDistribution: List[Distribution] = { requiredChildDistributionExpressions match { case Some(exprs) if exprs.isEmpty => AllTuples :: Nil - case Some(exprs) => ClusteredDistribution(exprs) :: Nil + case Some(exprs) => + if (isStreaming) { + numShufflePartitions match { + case Some(parts) => + StatefulOperatorPartitioning.getCompatibleDistribution( + exprs, parts, conf) :: Nil + + case _ => + throw new IllegalStateException("Expected to set the number of partitions before " + + "constructing required child distribution!") + } + } else { + ClusteredDistribution(exprs) :: Nil + } case None => UnspecifiedDistribution :: Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 8545154028602..e75b5c6e46853 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -45,6 +45,8 @@ import org.apache.spark.util.Utils */ case class HashAggregateExec( requiredChildDistributionExpressions: Option[Seq[Expression]], + isStreaming: Boolean, + numShufflePartitions: Option[Int], groupingExpressions: Seq[NamedExpression], aggregateExpressions: Seq[AggregateExpression], aggregateAttributes: Seq[Attribute], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/MergingSessionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/MergingSessionsExec.scala index 08e8b59a17828..31245c5451857 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/MergingSessionsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/MergingSessionsExec.scala @@ -21,7 +21,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expression, MutableProjection, NamedExpression, SortOrder, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression -import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.SQLMetrics @@ -41,7 +40,8 @@ import org.apache.spark.sql.execution.metric.SQLMetrics */ case class MergingSessionsExec( requiredChildDistributionExpressions: Option[Seq[Expression]], - requiredChildDistributionOption: Option[Seq[Distribution]], + isStreaming: Boolean, + numShufflePartitions: Option[Int], groupingExpressions: Seq[NamedExpression], sessionExpression: NamedExpression, aggregateExpressions: Seq[AggregateExpression], @@ -59,17 +59,6 @@ case class MergingSessionsExec( override def outputOrdering: Seq[SortOrder] = child.outputOrdering - override def requiredChildDistribution: List[Distribution] = { - requiredChildDistributionExpressions match { - case Some(exprs) if exprs.isEmpty => AllTuples :: Nil - case Some(exprs) => ClusteredDistribution(exprs) :: Nil - case None => requiredChildDistributionOption match { - case Some(distributions) => distributions.toList - case None => UnspecifiedDistribution :: Nil - } - } - } - override def requiredChildOrdering: Seq[Seq[SortOrder]] = { Seq((keyWithoutSessionExpressions ++ Seq(sessionExpression)).map(SortOrder(_, Ascending))) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala index c98c9f42e69da..9da0ca93c1819 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala @@ -59,6 +59,8 @@ import org.apache.spark.sql.execution.metric.SQLMetrics */ case class ObjectHashAggregateExec( requiredChildDistributionExpressions: Option[Seq[Expression]], + isStreaming: Boolean, + numShufflePartitions: Option[Int], groupingExpressions: Seq[NamedExpression], aggregateExpressions: Seq[AggregateExpression], aggregateAttributes: Seq[Attribute], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index 4fb0f44db81c7..e9cba96fc3f2d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -30,6 +30,8 @@ import org.apache.spark.sql.execution.metric.SQLMetrics */ case class SortAggregateExec( requiredChildDistributionExpressions: Option[Seq[Expression]], + isStreaming: Boolean, + numShufflePartitions: Option[Int], groupingExpressions: Seq[NamedExpression], aggregateExpressions: Seq[AggregateExpression], aggregateAttributes: Seq[Attribute], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UpdatingSessionsExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UpdatingSessionsExec.scala index f15a22403cfb4..fee7e29f8add1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UpdatingSessionsExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UpdatingSessionsExec.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.streaming.StatefulOperatorPartitioning /** * This node updates the session window spec of each input rows via analyzing neighbor rows and @@ -35,6 +36,8 @@ import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} * Refer [[UpdatingSessionsIterator]] for more details. */ case class UpdatingSessionsExec( + isStreaming: Boolean, + numShufflePartitions: Option[Int], groupingExpression: Seq[Attribute], sessionExpression: Attribute, child: SparkPlan) extends UnaryExecNode { @@ -63,7 +66,20 @@ case class UpdatingSessionsExec( if (groupingWithoutSessionExpression.isEmpty) { AllTuples :: Nil } else { - ClusteredDistribution(groupingWithoutSessionExpression) :: Nil + if (isStreaming) { + numShufflePartitions match { + case Some(parts) => + StatefulOperatorPartitioning.getCompatibleDistribution( + groupingWithoutSessionExpression, parts, conf) :: Nil + + case _ => + throw new IllegalStateException("Expected to set the number of partitions before " + + "constructing required child distribution!") + } + + } else { + ClusteredDistribution(groupingWithoutSessionExpression) :: Nil + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala index 4c6197538d89e..3ff539b9ef32b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, Expression, SortOrder, UnsafeRow} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution} +import org.apache.spark.sql.catalyst.plans.physical.Distribution import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper._ import org.apache.spark.sql.execution.streaming.state._ @@ -93,8 +93,10 @@ case class FlatMapGroupsWithStateExec( * to have the same grouping so that the data are co-lacated on the same task. */ override def requiredChildDistribution: Seq[Distribution] = { - ClusteredDistribution(groupingAttributes, stateInfo.map(_.numPartitions)) :: - ClusteredDistribution(initialStateGroupAttrs, stateInfo.map(_.numPartitions)) :: + StatefulOperatorPartitioning.getCompatibleDistribution( + groupingAttributes, getStateInfo, conf) :: + StatefulOperatorPartitioning.getCompatibleDistribution( + initialStateGroupAttrs, getStateInfo, conf) :: Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index 3e772e104648b..9670c774a74c1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern._ import org.apache.spark.sql.execution.{LocalLimitExec, QueryExecution, SparkPlan, SparkPlanner, UnaryExecNode} +import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, MergingSessionsExec, ObjectHashAggregateExec, SortAggregateExec, UpdatingSessionsExec} import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.OutputMode @@ -132,6 +133,22 @@ class IncrementalExecution( } override def apply(plan: SparkPlan): SparkPlan = plan transform { + // NOTE: we should include all aggregate execs here which are used in streaming aggregations + case a: SortAggregateExec if a.isStreaming => + a.copy(numShufflePartitions = Some(numStateStores)) + + case a: HashAggregateExec if a.isStreaming => + a.copy(numShufflePartitions = Some(numStateStores)) + + case a: ObjectHashAggregateExec if a.isStreaming => + a.copy(numShufflePartitions = Some(numStateStores)) + + case a: MergingSessionsExec if a.isStreaming => + a.copy(numShufflePartitions = Some(numStateStores)) + + case a: UpdatingSessionsExec if a.isStreaming => + a.copy(numShufflePartitions = Some(numStateStores)) + case StateStoreSaveExec(keys, None, None, None, stateFormatVersion, UnaryExecNode(agg, StateStoreRestoreExec(_, None, _, child))) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala index c08a14c65b772..913805d1a074d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala @@ -98,7 +98,7 @@ object OffsetSeqMetadata extends Logging { SHUFFLE_PARTITIONS, STATE_STORE_PROVIDER_CLASS, STREAMING_MULTIPLE_WATERMARK_POLICY, FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION, STREAMING_AGGREGATION_STATE_FORMAT_VERSION, STREAMING_JOIN_STATE_FORMAT_VERSION, STATE_STORE_COMPRESSION_CODEC, - STATE_STORE_ROCKSDB_FORMAT_VERSION) + STATE_STORE_ROCKSDB_FORMAT_VERSION, STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION) /** * Default values of relevant configurations that are used for backward compatibility. @@ -118,7 +118,8 @@ object OffsetSeqMetadata extends Logging { StreamingAggregationStateManager.legacyVersion.toString, STREAMING_JOIN_STATE_FORMAT_VERSION.key -> SymmetricHashJoinStateManager.legacyVersion.toString, - STATE_STORE_COMPRESSION_CODEC.key -> "lz4" + STATE_STORE_COMPRESSION_CODEC.key -> "lz4", + STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION.key -> "false" ) def apply(json: String): OffsetSeqMetadata = Serialization.read[OffsetSeqMetadata](json) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulOperatorPartitioning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulOperatorPartitioning.scala new file mode 100644 index 0000000000000..c31d574880e21 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulOperatorPartitioning.scala @@ -0,0 +1,53 @@ +/* + * 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.execution.streaming + +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution, HashClusteredDistribution} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION + +/** + * This object is to provide clustered distribution for stateful operator with ensuring backward + * compatibility. Please read through the NOTE on the classdoc of + * [[HashClusteredDistribution]] before making any changes. Please refer SPARK-38204 + * for details. + * + * Do not use methods in this object for stateful operators which already uses + * [[HashClusteredDistribution]] as its required child distribution. + */ +object StatefulOperatorPartitioning { + + def getCompatibleDistribution( + expressions: Seq[Expression], + stateInfo: StatefulOperatorStateInfo, + conf: SQLConf): Distribution = { + getCompatibleDistribution(expressions, stateInfo.numPartitions, conf) + } + + def getCompatibleDistribution( + expressions: Seq[Expression], + numPartitions: Int, + conf: SQLConf): Distribution = { + if (conf.getConf(STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION)) { + HashClusteredDistribution(expressions, Some(numPartitions)) + } else { + ClusteredDistribution(expressions, requiredNumPartitions = Some(numPartitions)) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala index 3431823765c1b..bcfdeb4f85cdf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark -import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} +import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, Distribution, Partitioning} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution._ @@ -337,7 +337,8 @@ case class StateStoreRestoreExec( if (keyExpressions.isEmpty) { AllTuples :: Nil } else { - ClusteredDistribution(keyExpressions, stateInfo.map(_.numPartitions)) :: Nil + StatefulOperatorPartitioning.getCompatibleDistribution( + keyExpressions, getStateInfo, conf) :: Nil } } @@ -496,7 +497,8 @@ case class StateStoreSaveExec( if (keyExpressions.isEmpty) { AllTuples :: Nil } else { - ClusteredDistribution(keyExpressions, stateInfo.map(_.numPartitions)) :: Nil + StatefulOperatorPartitioning.getCompatibleDistribution( + keyExpressions, getStateInfo, conf) :: Nil } } @@ -573,7 +575,8 @@ case class SessionWindowStateStoreRestoreExec( } override def requiredChildDistribution: Seq[Distribution] = { - ClusteredDistribution(keyWithoutSessionExpressions, stateInfo.map(_.numPartitions)) :: Nil + StatefulOperatorPartitioning.getCompatibleDistribution( + keyWithoutSessionExpressions, getStateInfo, conf) :: Nil } override def requiredChildOrdering: Seq[Seq[SortOrder]] = { @@ -684,7 +687,8 @@ case class SessionWindowStateStoreSaveExec( override def outputPartitioning: Partitioning = child.outputPartitioning override def requiredChildDistribution: Seq[Distribution] = { - ClusteredDistribution(keyExpressions, stateInfo.map(_.numPartitions)) :: Nil + StatefulOperatorPartitioning.getCompatibleDistribution( + keyWithoutSessionExpressions, getStateInfo, conf) :: Nil } override def shouldRunAnotherBatch(newMetadata: OffsetSeqMetadata): Boolean = { @@ -741,8 +745,10 @@ case class StreamingDeduplicateExec( extends UnaryExecNode with StateStoreWriter with WatermarkSupport { /** Distribute by grouping attributes */ - override def requiredChildDistribution: Seq[Distribution] = - ClusteredDistribution(keyExpressions, stateInfo.map(_.numPartitions)) :: Nil + override def requiredChildDistribution: Seq[Distribution] = { + StatefulOperatorPartitioning.getCompatibleDistribution( + keyExpressions, getStateInfo, conf) :: Nil + } override protected def doExecute(): RDD[InternalRow] = { metrics // force lazy init at driver diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/metadata new file mode 100644 index 0000000000000..019111c307024 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/metadata @@ -0,0 +1 @@ +{"id":"dc9af96e-870c-4dc6-ad09-1b84b62caac3"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/offsets/0 new file mode 100644 index 0000000000000..d00e8a5a4134a --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1000,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/0/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/0/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/0/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/0/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/0/_metadata/schema new file mode 100644 index 0000000000000..d3948722c3258 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/0/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/1/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/1/1.delta new file mode 100644 index 0000000000000..2639d3211decf Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/1/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/2/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/2/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/2/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/3/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/3/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/3/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/4/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/4/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/state/0/4/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/commits/.0.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/commits/.0.crc new file mode 100644 index 0000000000000..1aee7033161ec Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/commits/.0.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/commits/.1.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/commits/.1.crc new file mode 100644 index 0000000000000..1aee7033161ec Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/commits/.1.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/commits/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/commits/1 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/commits/1 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/metadata new file mode 100644 index 0000000000000..81acb4439e8f5 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/metadata @@ -0,0 +1 @@ +{"id":"9538ada3-a233-4697-8b02-cc66250189a3"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/offsets/.0.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/offsets/.0.crc new file mode 100644 index 0000000000000..b8a9976585811 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/offsets/.0.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/offsets/.1.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/offsets/.1.crc new file mode 100644 index 0000000000000..81716485cf023 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/offsets/.1.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/offsets/0 new file mode 100644 index 0000000000000..852130a526e08 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1645693797622,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/offsets/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/offsets/1 new file mode 100644 index 0000000000000..2d894644897bf --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/offsets/1 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1645693802625,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5"}} +1 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/.2.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/.2.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/.2.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/_metadata/.schema.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/_metadata/.schema.crc new file mode 100644 index 0000000000000..f03866c573c15 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/_metadata/.schema.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/_metadata/schema new file mode 100644 index 0000000000000..e4695f58d7de9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/0/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/1/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/1/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/1/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/1/.2.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/1/.2.delta.crc new file mode 100644 index 0000000000000..dc5c3a4905a5b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/1/.2.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/1/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/1/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/1/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/1/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/1/2.delta new file mode 100644 index 0000000000000..00c03b0f2aaa5 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/1/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/2/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/2/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/2/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/2/.2.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/2/.2.delta.crc new file mode 100644 index 0000000000000..0df89359466b4 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/2/.2.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/2/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/2/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/2/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/2/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/2/2.delta new file mode 100644 index 0000000000000..0a0f74c944036 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/2/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/3/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/3/.1.delta.crc new file mode 100644 index 0000000000000..fcb13666a0ad8 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/3/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/3/.2.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/3/.2.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/3/.2.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/3/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/3/1.delta new file mode 100644 index 0000000000000..4e033f8786aa8 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/3/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/3/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/3/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/3/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/4/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/4/.1.delta.crc new file mode 100644 index 0000000000000..eb2b6be4e5e55 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/4/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/4/.2.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/4/.2.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/4/.2.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/4/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/4/1.delta new file mode 100644 index 0000000000000..7b6e9c175b8cf Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/4/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/4/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/4/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/state/0/4/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/commits/.0.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/commits/.0.crc new file mode 100644 index 0000000000000..1aee7033161ec Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/commits/.0.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/metadata new file mode 100644 index 0000000000000..54698e5f8afa9 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/metadata @@ -0,0 +1 @@ +{"id":"b36205c7-696a-4fe9-86d4-a4efdf05795b"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/offsets/.0.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/offsets/.0.crc new file mode 100644 index 0000000000000..04523a6882fdb Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/offsets/.0.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/offsets/0 new file mode 100644 index 0000000000000..321a56f4d3707 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1000,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/0/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/0/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/0/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/0/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/0/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/0/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/0/_metadata/.schema.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/0/_metadata/.schema.crc new file mode 100644 index 0000000000000..4d339e472ac25 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/0/_metadata/.schema.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/0/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/0/_metadata/schema new file mode 100644 index 0000000000000..bf902e50cf260 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/0/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/1/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/1/.1.delta.crc new file mode 100644 index 0000000000000..7029bc3ccdf2f Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/1/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/1/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/1/1.delta new file mode 100644 index 0000000000000..610e2c0250d4e Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/1/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/2/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/2/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/2/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/2/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/2/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/2/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/3/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/3/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/3/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/3/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/3/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/3/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/4/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/4/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/4/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/4/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/4/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/state/0/4/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/commits/.0.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/commits/.0.crc new file mode 100644 index 0000000000000..1aee7033161ec Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/commits/.0.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/metadata new file mode 100644 index 0000000000000..fa78985cb8778 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/metadata @@ -0,0 +1 @@ +{"id":"f4795695-2b3e-4864-983a-f7bf52c0e29d"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/offsets/.0.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/offsets/.0.crc new file mode 100644 index 0000000000000..04523a6882fdb Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/offsets/.0.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/offsets/0 new file mode 100644 index 0000000000000..321a56f4d3707 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1000,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/0/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/0/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/0/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/0/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/0/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/0/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/0/_metadata/.schema.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/0/_metadata/.schema.crc new file mode 100644 index 0000000000000..4d339e472ac25 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/0/_metadata/.schema.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/0/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/0/_metadata/schema new file mode 100644 index 0000000000000..bf902e50cf260 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/0/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/1/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/1/.1.delta.crc new file mode 100644 index 0000000000000..421f95ae9dd8b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/1/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/1/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/1/1.delta new file mode 100644 index 0000000000000..2639d3211decf Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/1/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/2/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/2/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/2/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/2/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/2/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/2/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/3/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/3/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/3/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/3/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/3/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/3/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/4/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/4/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/4/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/4/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/4/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/state/0/4/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/commits/.0.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/commits/.0.crc new file mode 100644 index 0000000000000..ba56986ebd219 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/commits/.0.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/commits/.1.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/commits/.1.crc new file mode 100644 index 0000000000000..ba56986ebd219 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/commits/.1.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/commits/0 new file mode 100644 index 0000000000000..00b8a64995dde --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":11000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/commits/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/commits/1 new file mode 100644 index 0000000000000..00b8a64995dde --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/commits/1 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":11000} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/metadata new file mode 100644 index 0000000000000..879dac88e351a --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/metadata @@ -0,0 +1 @@ +{"id":"c3d27d93-536b-49ce-a62f-f2777855a1fb"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/offsets/.0.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/offsets/.0.crc new file mode 100644 index 0000000000000..0d6e0a4778504 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/offsets/.0.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/offsets/.1.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/offsets/.1.crc new file mode 100644 index 0000000000000..24dcb52ef6098 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/offsets/.1.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/offsets/0 new file mode 100644 index 0000000000000..6f149ed4ec45c --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1645760172709,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/offsets/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/offsets/1 new file mode 100644 index 0000000000000..4a6194c2002bd --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/offsets/1 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":11000,"batchTimestampMs":1645760174214,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/.2.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/.2.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/.2.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/_metadata/.schema.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/_metadata/.schema.crc new file mode 100644 index 0000000000000..3f3804f1999c0 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/_metadata/.schema.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/_metadata/schema new file mode 100644 index 0000000000000..871586884066b Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/0/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/1/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/1/.1.delta.crc new file mode 100644 index 0000000000000..9e684a6792e54 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/1/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/1/.2.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/1/.2.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/1/.2.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/1/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/1/1.delta new file mode 100644 index 0000000000000..73c35f68c2f28 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/1/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/1/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/1/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/1/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/2/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/2/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/2/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/2/.2.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/2/.2.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/2/.2.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/2/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/2/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/2/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/2/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/2/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/2/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/3/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/3/.1.delta.crc new file mode 100644 index 0000000000000..816cff99cd156 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/3/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/3/.2.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/3/.2.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/3/.2.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/3/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/3/1.delta new file mode 100644 index 0000000000000..3c6d389f04264 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/3/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/3/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/3/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/3/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/4/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/4/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/4/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/4/.2.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/4/.2.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/4/.2.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/4/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/4/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/4/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/4/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/4/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/state/0/4/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/commits/.0.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/commits/.0.crc new file mode 100644 index 0000000000000..1aee7033161ec Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/commits/.0.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/commits/.1.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/commits/.1.crc new file mode 100644 index 0000000000000..1aee7033161ec Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/commits/.1.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/commits/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/commits/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/commits/1 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/commits/1 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/metadata b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/metadata new file mode 100644 index 0000000000000..0831489d9d02d --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/metadata @@ -0,0 +1 @@ +{"id":"cd462130-c8fb-4212-8b08-4e1b9e10dbcf"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/offsets/.0.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/offsets/.0.crc new file mode 100644 index 0000000000000..b1cf4d310b245 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/offsets/.0.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/offsets/.1.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/offsets/.1.crc new file mode 100644 index 0000000000000..cf958a5259df3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/offsets/.1.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/offsets/0 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/offsets/0 new file mode 100644 index 0000000000000..523d0ce69165e --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1645692626085,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/offsets/1 b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/offsets/1 new file mode 100644 index 0000000000000..f69d320e37d2f --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/offsets/1 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1645692630152,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.join.stateFormatVersion":"2","spark.sql.streaming.stateStore.compression.codec":"lz4","spark.sql.streaming.stateStore.rocksdb.formatVersion":"5","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"5"}} +1 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/.2.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/.2.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/.2.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/_metadata/.schema.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/_metadata/.schema.crc new file mode 100644 index 0000000000000..701a0a87ad48a Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/_metadata/.schema.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/_metadata/schema b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/_metadata/schema new file mode 100644 index 0000000000000..08ee320ef2421 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/0/_metadata/schema differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/1/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/1/.1.delta.crc new file mode 100644 index 0000000000000..f712e4290ad37 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/1/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/1/.2.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/1/.2.delta.crc new file mode 100644 index 0000000000000..2a9f3595f24b5 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/1/.2.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/1/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/1/1.delta new file mode 100644 index 0000000000000..f5faf01f4dc5c Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/1/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/1/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/1/2.delta new file mode 100644 index 0000000000000..ec3f1af46bd49 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/1/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/2/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/2/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/2/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/2/.2.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/2/.2.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/2/.2.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/2/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/2/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/2/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/2/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/2/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/2/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/3/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/3/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/3/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/3/.2.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/3/.2.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/3/.2.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/3/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/3/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/3/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/3/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/3/2.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/3/2.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/4/.1.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/4/.1.delta.crc new file mode 100644 index 0000000000000..cf1d68e2acee3 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/4/.1.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/4/.2.delta.crc b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/4/.2.delta.crc new file mode 100644 index 0000000000000..3ffbb7a9133b5 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/4/.2.delta.crc differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/4/1.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/4/1.delta new file mode 100644 index 0000000000000..6352978051846 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/4/1.delta differ diff --git a/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/4/2.delta b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/4/2.delta new file mode 100644 index 0000000000000..7c8834f659bd9 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/state/0/4/2.delta differ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 6cc6e33dd688a..c2b22125cb148 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -681,7 +681,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession assert( executedPlan.find { case WholeStageCodegenExec( - HashAggregateExec(_, _, _, _, _, _, _: LocalTableScanExec)) => true + HashAggregateExec(_, _, _, _, _, _, _, _, _: LocalTableScanExec)) => true case _ => false }.isDefined, "LocalTableScanExec should be within a WholeStageCodegen domain.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateDistributionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateDistributionSuite.scala new file mode 100644 index 0000000000000..019d4ada0914b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateDistributionSuite.scala @@ -0,0 +1,455 @@ +/* + * 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.File + +import org.apache.commons.io.FileUtils + +import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Update +import org.apache.spark.sql.execution.streaming.{FlatMapGroupsWithStateExec, MemoryStream} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.GroupStateTimeout.ProcessingTimeTimeout +import org.apache.spark.sql.streaming.util.{HashClusteredDistributionTestHelper, StreamManualClock} +import org.apache.spark.util.Utils + +class FlatMapGroupsWithStateDistributionSuite extends StreamTest + with HashClusteredDistributionTestHelper { + + import testImplicits._ + + test("SPARK-38204: flatMapGroupsWithState should require HashClusteredDistribution " + + "from children - with initial state") { + // function will return -1 on timeout and returns count of the state otherwise + val stateFunc = + (key: (String, String), values: Iterator[(String, String, Long)], + state: GroupState[RunningCount]) => { + + if (state.hasTimedOut) { + state.remove() + Iterator((key, "-1")) + } else { + val count = state.getOption.map(_.count).getOrElse(0L) + values.size + state.update(RunningCount(count)) + state.setTimeoutDuration("10 seconds") + Iterator((key, count.toString)) + } + } + + val clock = new StreamManualClock + val inputData = MemoryStream[(String, String, Long)] + val initialState = Seq(("c", "c", new RunningCount(2))) + .toDS() + .repartition($"_2") + .groupByKey(a => (a._1, a._2)).mapValues(_._3) + val result = + inputData.toDF().toDF("key1", "key2", "time") + .selectExpr("key1", "key2", "timestamp_seconds(time) as timestamp") + .withWatermark("timestamp", "10 second") + .as[(String, String, Long)] + .repartition($"_1") + .groupByKey(x => (x._1, x._2)) + .flatMapGroupsWithState(Update, ProcessingTimeTimeout(), initialState)(stateFunc) + .select($"_1._1".as("key1"), $"_1._2".as("key2"), $"_2".as("cnt")) + + testStream(result, Update)( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock), + AddData(inputData, ("a", "a", 1L)), + AdvanceManualClock(1 * 1000), // a and c are processed here for the first time. + CheckNewAnswer(("a", "a", "1"), ("c", "c", "2")), + Execute { query => + val numPartitions = query.lastExecution.numStateStores + + val flatMapGroupsWithStateExecs = query.lastExecution.executedPlan.collect { + case f: FlatMapGroupsWithStateExec => f + } + + assert(flatMapGroupsWithStateExecs.length === 1) + assert(requireHashClusteredDistribution( + flatMapGroupsWithStateExecs.head, Seq(Seq("_1", "_2"), Seq("_1", "_2")), numPartitions)) + assert(hasDesiredHashPartitioningInChildren( + flatMapGroupsWithStateExecs.head, Seq(Seq("_1", "_2"), Seq("_1", "_2")), numPartitions)) + } + ) + } + + test("SPARK-38204: flatMapGroupsWithState should require HashClusteredDistribution " + + "from children - without initial state") { + // function will return -1 on timeout and returns count of the state otherwise + val stateFunc = + (key: (String, String), values: Iterator[(String, String, Long)], + state: GroupState[RunningCount]) => { + + if (state.hasTimedOut) { + state.remove() + Iterator((key, "-1")) + } else { + val count = state.getOption.map(_.count).getOrElse(0L) + values.size + state.update(RunningCount(count)) + state.setTimeoutDuration("10 seconds") + Iterator((key, count.toString)) + } + } + + val clock = new StreamManualClock + val inputData = MemoryStream[(String, String, Long)] + val result = + inputData.toDF().toDF("key1", "key2", "time") + .selectExpr("key1", "key2", "timestamp_seconds(time) as timestamp") + .withWatermark("timestamp", "10 second") + .as[(String, String, Long)] + .repartition($"_1") + .groupByKey(x => (x._1, x._2)) + .flatMapGroupsWithState(Update, ProcessingTimeTimeout())(stateFunc) + .select($"_1._1".as("key1"), $"_1._2".as("key2"), $"_2".as("cnt")) + + testStream(result, Update)( + StartStream(Trigger.ProcessingTime("1 second"), triggerClock = clock), + AddData(inputData, ("a", "a", 1L)), + AdvanceManualClock(1 * 1000), // a is processed here for the first time. + CheckNewAnswer(("a", "a", "1")), + Execute { query => + val numPartitions = query.lastExecution.numStateStores + + val flatMapGroupsWithStateExecs = query.lastExecution.executedPlan.collect { + case f: FlatMapGroupsWithStateExec => f + } + + assert(flatMapGroupsWithStateExecs.length === 1) + assert(requireHashClusteredDistribution( + flatMapGroupsWithStateExecs.head, Seq(Seq("_1", "_2"), Seq("_1", "_2")), numPartitions)) + assert(hasDesiredHashPartitioningInChildren( + flatMapGroupsWithStateExecs.head, Seq(Seq("_1", "_2"), Seq("_1", "_2")), numPartitions)) + } + ) + } + + test("SPARK-38204: flatMapGroupsWithState should require ClusteredDistribution " + + "from children if the query starts from checkpoint in 3.2.x - with initial state") { + // function will return -1 on timeout and returns count of the state otherwise + val stateFunc = + (key: (String, String), values: Iterator[(String, String, Long)], + state: GroupState[RunningCount]) => { + + if (state.hasTimedOut) { + state.remove() + Iterator((key, "-1")) + } else { + val count = state.getOption.map(_.count).getOrElse(0L) + values.size + state.update(RunningCount(count)) + state.setTimeoutDuration("10 seconds") + Iterator((key, count.toString)) + } + } + + val clock = new StreamManualClock + val inputData = MemoryStream[(String, String, Long)] + val initialState = Seq(("c", "c", new RunningCount(2))) + .toDS() + .repartition($"_2") + .groupByKey(a => (a._1, a._2)).mapValues(_._3) + val result = + inputData.toDF().toDF("key1", "key2", "time") + .selectExpr("key1", "key2", "timestamp_seconds(time) as timestamp") + .withWatermark("timestamp", "10 second") + .as[(String, String, Long)] + .repartition($"_1") + .groupByKey(x => (x._1, x._2)) + .flatMapGroupsWithState(Update, ProcessingTimeTimeout(), initialState)(stateFunc) + .select($"_1._1".as("key1"), $"_1._2".as("key2"), $"_2".as("cnt")) + + val resourceUri = this.getClass.getResource( + "/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate1-repartition/").toURI + + val checkpointDir = Utils.createTempDir().getCanonicalFile + // Copy the checkpoint to a temp dir to prevent changes to the original. + // Not doing this will lead to the test passing on the first run, but fail subsequent runs. + FileUtils.copyDirectory(new File(resourceUri), checkpointDir) + + inputData.addData(("a", "a", 1L)) + + testStream(result, Update)( + StartStream(Trigger.ProcessingTime("1 second"), + checkpointLocation = checkpointDir.getAbsolutePath, + triggerClock = clock, + additionalConfs = Map(SQLConf.STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION.key -> "true")), + + // scalastyle:off line.size.limit + /* + Note: The checkpoint was generated using the following input in Spark version 3.2.0 + AddData(inputData, ("a", "a", 1L)), + AdvanceManualClock(1 * 1000), // a and c are processed here for the first time. + CheckNewAnswer(("a", "a", "1"), ("c", "c", "2")), + + Note2: The following is the physical plan of the query in Spark version 3.2.0. + + WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@253dd5ad, org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2214/0x0000000840ead440@6ede0d42 + +- *(6) Project [_1#58._1 AS key1#63, _1#58._2 AS key2#64, _2#59 AS cnt#65] + +- *(6) SerializeFromObject [if (isnull(knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1)) null else named_struct(_1, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1)._1, true, false), _2, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1)._2, true, false)) AS _1#58, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, true, false) AS _2#59] + +- FlatMapGroupsWithState org.apache.spark.sql.streaming.FlatMapGroupsWithStateSuite$$Lambda$1067/0x0000000840770440@3f2e51a9, newInstance(class scala.Tuple2), newInstance(class scala.Tuple3), newInstance(class org.apache.spark.sql.streaming.RunningCount), [_1#52, _2#53], [_1#22, _2#23], [key1#29, key2#30, timestamp#35-T10000ms], [count#25L], obj#57: scala.Tuple2, state info [ checkpoint = file:/tmp/streaming.metadata-d4f0d156-78b5-4129-97fb-361241ab03d8/state, runId = eb107298-692d-4336-bb76-6b11b34a0753, opId = 0, ver = 0, numPartitions = 5], class[count[0]: bigint], 2, Update, ProcessingTimeTimeout, 1000, 0, true + :- *(3) Sort [_1#52 ASC NULLS FIRST, _2#53 ASC NULLS FIRST], false, 0 + : +- Exchange hashpartitioning(_1#52, _2#53, 5), ENSURE_REQUIREMENTS, [id=#78] + : +- AppendColumns org.apache.spark.sql.streaming.FlatMapGroupsWithStateSuite$$Lambda$1751/0x0000000840ccc040@41d4c0d8, newInstance(class scala.Tuple3), [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#52, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, true, false) AS _2#53] + : +- *(2) Project [key1#29, key2#30, timestamp#35-T10000ms] + : +- Exchange hashpartitioning(_1#3, 5), REPARTITION_BY_COL, [id=#73] + : +- EventTimeWatermark timestamp#35: timestamp, 10 seconds + : +- *(1) Project [_1#3 AS key1#29, _2#4 AS key2#30, timestamp_seconds(_3#5L) AS timestamp#35, _1#3] + : +- MicroBatchScan[_1#3, _2#4, _3#5L] MemoryStreamDataSource + +- *(5) Sort [_1#22 ASC NULLS FIRST, _2#23 ASC NULLS FIRST], false, 0 + +- Exchange hashpartitioning(_1#22, _2#23, 5), ENSURE_REQUIREMENTS, [id=#85] + +- *(4) Project [count#25L, _1#22, _2#23] + +- AppendColumns org.apache.spark.sql.streaming.FlatMapGroupsWithStateSuite$$Lambda$1686/0x0000000840c9b840@6bb881d0, newInstance(class scala.Tuple3), [knownnotnull(assertnotnull(input[0, org.apache.spark.sql.streaming.RunningCount, true])).count AS count#25L] + +- AppendColumns org.apache.spark.sql.streaming.FlatMapGroupsWithStateSuite$$Lambda$1681/0x0000000840c98840@11355c7b, newInstance(class scala.Tuple3), [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#22, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, true, false) AS _2#23] + +- Exchange hashpartitioning(_1#9, 5), REPARTITION_BY_COL, [id=#43] + +- LocalTableScan [_1#9, _2#10, _3#11] + */ + // scalastyle:on line.size.limit + + AddData(inputData, ("a", "b", 1L)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer(("a", "b", "1")), + + Execute { query => + val numPartitions = query.lastExecution.numStateStores + + val flatMapGroupsWithStateExecs = query.lastExecution.executedPlan.collect { + case f: FlatMapGroupsWithStateExec => f + } + + assert(flatMapGroupsWithStateExecs.length === 1) + assert(requireClusteredDistribution(flatMapGroupsWithStateExecs.head, + Seq(Seq("_1", "_2"), Seq("_1", "_2")), Some(numPartitions))) + assert(hasDesiredHashPartitioningInChildren( + flatMapGroupsWithStateExecs.head, Seq(Seq("_1", "_2"), Seq("_1", "_2")), numPartitions)) + } + ) + } + + test("SPARK-38204: flatMapGroupsWithState should require ClusteredDistribution " + + "from children if the query starts from checkpoint in 3.2.x - without initial state") { + // function will return -1 on timeout and returns count of the state otherwise + val stateFunc = + (key: (String, String), values: Iterator[(String, String, Long)], + state: GroupState[RunningCount]) => { + + if (state.hasTimedOut) { + state.remove() + Iterator((key, "-1")) + } else { + val count = state.getOption.map(_.count).getOrElse(0L) + values.size + state.update(RunningCount(count)) + state.setTimeoutDuration("10 seconds") + Iterator((key, count.toString)) + } + } + + val clock = new StreamManualClock + val inputData = MemoryStream[(String, String, Long)] + val result = + inputData.toDF().toDF("key1", "key2", "time") + .selectExpr("key1", "key2", "timestamp_seconds(time) as timestamp") + .withWatermark("timestamp", "10 second") + .as[(String, String, Long)] + .repartition($"_1") + .groupByKey(x => (x._1, x._2)) + .flatMapGroupsWithState(Update, ProcessingTimeTimeout())(stateFunc) + .select($"_1._1".as("key1"), $"_1._2".as("key2"), $"_2".as("cnt")) + + val resourceUri = this.getClass.getResource( + "/structured-streaming/checkpoint-version-3.2.0-flatmapgroupswithstate2-repartition/").toURI + + val checkpointDir = Utils.createTempDir().getCanonicalFile + // Copy the checkpoint to a temp dir to prevent changes to the original. + // Not doing this will lead to the test passing on the first run, but fail subsequent runs. + FileUtils.copyDirectory(new File(resourceUri), checkpointDir) + + inputData.addData(("a", "a", 1L)) + + testStream(result, Update)( + StartStream(Trigger.ProcessingTime("1 second"), + checkpointLocation = checkpointDir.getAbsolutePath, + triggerClock = clock, + additionalConfs = Map(SQLConf.STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION.key -> "true")), + + // scalastyle:off line.size.limit + /* + Note: The checkpoint was generated using the following input in Spark version 3.2.0 + AddData(inputData, ("a", "a", 1L)), + AdvanceManualClock(1 * 1000), // a is processed here for the first time. + CheckNewAnswer(("a", "a", "1")), + + Note2: The following is the physical plan of the query in Spark version 3.2.0 (convenience for checking backward compatibility) + WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@20732f1b, org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2205/0x0000000840ea5440@48e6c016 + +- *(5) Project [_1#39._1 AS key1#44, _1#39._2 AS key2#45, _2#40 AS cnt#46] + +- *(5) SerializeFromObject [if (isnull(knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1)) null else named_struct(_1, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1)._1, true, false), _2, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1)._2, true, false)) AS _1#39, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, true, false) AS _2#40] + +- FlatMapGroupsWithState org.apache.spark.sql.streaming.FlatMapGroupsWithStateSuite$$Lambda$1065/0x0000000840770040@240e41f8, newInstance(class scala.Tuple2), newInstance(class scala.Tuple3), newInstance(class scala.Tuple2), [_1#32, _2#33], [_1#32, _2#33], [key1#9, key2#10, timestamp#15-T10000ms], [key1#9, key2#10, timestamp#15-T10000ms], obj#37: scala.Tuple2, state info [ checkpoint = file:/tmp/spark-6619d285-b0ca-42ab-8284-723a564e13b6/state, runId = b3383a6c-9976-483c-a463-7fc9e9ae3e1a, opId = 0, ver = 0, numPartitions = 5], class[count[0]: bigint], 2, Update, ProcessingTimeTimeout, 1000, 0, false + :- *(3) Sort [_1#32 ASC NULLS FIRST, _2#33 ASC NULLS FIRST], false, 0 + : +- Exchange hashpartitioning(_1#32, _2#33, 5), ENSURE_REQUIREMENTS, [id=#62] + : +- AppendColumns org.apache.spark.sql.streaming.FlatMapGroupsWithStateSuite$$Lambda$1709/0x0000000840ca7040@351810cb, newInstance(class scala.Tuple3), [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#32, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, true, false) AS _2#33] + : +- *(2) Project [key1#9, key2#10, timestamp#15-T10000ms] + : +- Exchange hashpartitioning(_1#3, 5), REPARTITION_BY_COL, [id=#57] + : +- EventTimeWatermark timestamp#15: timestamp, 10 seconds + : +- *(1) Project [_1#3 AS key1#9, _2#4 AS key2#10, timestamp_seconds(_3#5L) AS timestamp#15, _1#3] + : +- MicroBatchScan[_1#3, _2#4, _3#5L] MemoryStreamDataSource + +- *(4) !Sort [_1#32 ASC NULLS FIRST, _2#33 ASC NULLS FIRST], false, 0 + +- !Exchange hashpartitioning(_1#32, _2#33, 5), ENSURE_REQUIREMENTS, [id=#46] + +- LocalTableScan , [count#38L] + */ + // scalastyle:on line.size.limit + + AddData(inputData, ("a", "b", 1L)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer(("a", "b", "1")), + + Execute { query => + val numPartitions = query.lastExecution.numStateStores + + val flatMapGroupsWithStateExecs = query.lastExecution.executedPlan.collect { + case f: FlatMapGroupsWithStateExec => f + } + + assert(flatMapGroupsWithStateExecs.length === 1) + assert(requireClusteredDistribution(flatMapGroupsWithStateExecs.head, + Seq(Seq("_1", "_2"), Seq("_1", "_2")), Some(numPartitions))) + assert(hasDesiredHashPartitioningInChildren( + flatMapGroupsWithStateExecs.head, Seq(Seq("_1", "_2"), Seq("_1", "_2")), numPartitions)) + } + ) + } + + test("SPARK-38204: flatMapGroupsWithState should require ClusteredDistribution " + + "from children if the query starts from checkpoint in prior to 3.2") { + // function will return -1 on timeout and returns count of the state otherwise + val stateFunc = + (key: (String, String), values: Iterator[(String, String, Long)], + state: GroupState[RunningCount]) => { + + if (state.hasTimedOut) { + state.remove() + Iterator((key, "-1")) + } else { + val count = state.getOption.map(_.count).getOrElse(0L) + values.size + state.update(RunningCount(count)) + state.setTimeoutDuration("10 seconds") + Iterator((key, count.toString)) + } + } + + val clock = new StreamManualClock + val inputData = MemoryStream[(String, String, Long)] + val result = + inputData.toDF().toDF("key1", "key2", "time") + .selectExpr("key1", "key2", "timestamp_seconds(time) as timestamp") + .withWatermark("timestamp", "10 second") + .as[(String, String, Long)] + .repartition($"_1") + .groupByKey(x => (x._1, x._2)) + .flatMapGroupsWithState(Update, ProcessingTimeTimeout())(stateFunc) + .select($"_1._1".as("key1"), $"_1._2".as("key2"), $"_2".as("cnt")) + + val resourceUri = this.getClass.getResource( + "/structured-streaming/checkpoint-version-3.1.0-flatmapgroupswithstate-repartition/").toURI + + val checkpointDir = Utils.createTempDir().getCanonicalFile + // Copy the checkpoint to a temp dir to prevent changes to the original. + // Not doing this will lead to the test passing on the first run, but fail subsequent runs. + FileUtils.copyDirectory(new File(resourceUri), checkpointDir) + + inputData.addData(("a", "a", 1L)) + + testStream(result, Update)( + StartStream(Trigger.ProcessingTime("1 second"), + checkpointLocation = checkpointDir.getAbsolutePath, + triggerClock = clock, + additionalConfs = Map(SQLConf.STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION.key -> "true")), + + // scalastyle:off line.size.limit + /* + Note: The checkpoint was generated using the following input in Spark version 3.2.0 + AddData(inputData, ("a", "a", 1L)), + AdvanceManualClock(1 * 1000), // a is processed here for the first time. + CheckNewAnswer(("a", "a", "1")), + + Note2: The following plans are the physical plans of the query in older Spark versions + The physical plans around FlatMapGroupsWithStateExec are quite similar, especially + shuffles being injected are same. That said, verifying with checkpoint being built with + Spark 3.1.0 would verify the following versions as well. + + A. Spark 3.1.0 + WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@4505821b + +- *(3) Project [_1#38._1 AS key1#43, _1#38._2 AS key2#44, _2#39 AS cnt#45] + +- *(3) SerializeFromObject [if (isnull(knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1)) null else named_struct(_1, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1)._1, true, false), _2, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1)._2, true, false)) AS _1#38, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, true, false) AS _2#39] + +- FlatMapGroupsWithState org.apache.spark.sql.streaming.FlatMapGroupsWithStateSuite$$Lambda$1035/0x0000000840721840@64351072, newInstance(class scala.Tuple2), newInstance(class scala.Tuple3), [_1#32, _2#33], [key1#9, key2#10, timestamp#15-T10000ms], obj#37: scala.Tuple2, state info [ checkpoint = file:/tmp/spark-56397379-d014-48e0-a002-448c0621cfe8/state, runId = 4f9a129f-2b0c-4838-9d26-18171d94be7d, opId = 0, ver = 0, numPartitions = 5], class[count[0]: bigint], 2, Update, ProcessingTimeTimeout, 1000, 0 + +- *(2) Sort [_1#32 ASC NULLS FIRST, _2#33 ASC NULLS FIRST], false, 0 + +- Exchange hashpartitioning(_1#32, _2#33, 5), ENSURE_REQUIREMENTS, [id=#54] + +- AppendColumns org.apache.spark.sql.streaming.FlatMapGroupsWithStateSuite$$Lambda$1594/0x0000000840bc8840@857c80d, newInstance(class scala.Tuple3), [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#32, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, true, false) AS _2#33] + +- Exchange hashpartitioning(key1#9, 5), REPARTITION, [id=#52] + +- EventTimeWatermark timestamp#15: timestamp, 10 seconds + +- *(1) Project [_1#3 AS key1#9, _2#4 AS key2#10, timestamp_seconds(_3#5L) AS timestamp#15] + +- *(1) Project [_1#3, _2#4, _3#5L] + +- MicroBatchScan[_1#3, _2#4, _3#5L] MemoryStreamDataSource + + B. Spark 3.0.0 + WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@32ae8206 + +- *(3) Project [_1#38._1 AS key1#43, _1#38._2 AS key2#44, _2#39 AS cnt#45] + +- *(3) SerializeFromObject [if (isnull(knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1)) null else named_struct(_1, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1)._1, true, false), _2, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1)._2, true, false)) AS _1#38, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, true, false) AS _2#39] + +- FlatMapGroupsWithState org.apache.spark.sql.streaming.FlatMapGroupsWithStateSuite$$Lambda$972/0x0000000840721c40@3e8c825d, newInstance(class scala.Tuple2), newInstance(class scala.Tuple3), [_1#32, _2#33], [key1#9, key2#10, timestamp#15-T10000ms], obj#37: scala.Tuple2, state info [ checkpoint = file:/tmp/spark-dcd6753e-54c7-481c-aa21-f7fc677a29a4/state, runId = 4854d427-436c-4f4e-9e1d-577bcd9cc890, opId = 0, ver = 0, numPartitions = 5], class[count[0]: bigint], 2, Update, ProcessingTimeTimeout, 1000, 0 + +- *(2) Sort [_1#32 ASC NULLS FIRST, _2#33 ASC NULLS FIRST], false, 0 + +- Exchange hashpartitioning(_1#32, _2#33, 5), true, [id=#54] + +- AppendColumns org.apache.spark.sql.streaming.FlatMapGroupsWithStateSuite$$Lambda$1477/0x0000000840bb6040@627623e, newInstance(class scala.Tuple3), [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#32, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, knownnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, true, false) AS _2#33] + +- Exchange hashpartitioning(key1#9, 5), false, [id=#52] + +- EventTimeWatermark timestamp#15: timestamp, 10 seconds + +- *(1) Project [_1#3 AS key1#9, _2#4 AS key2#10, cast(_3#5L as timestamp) AS timestamp#15] + +- *(1) Project [_1#3, _2#4, _3#5L] + +- MicroBatchScan[_1#3, _2#4, _3#5L] MemoryStreamDataSource + + C. Spark 2.4.0 + *(3) Project [_1#32._1 AS key1#35, _1#32._2 AS key2#36, _2#33 AS cnt#37] + +- *(3) SerializeFromObject [if (isnull(assertnotnull(input[0, scala.Tuple2, true])._1)) null else named_struct(_1, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, scala.Tuple2, true])._1)._1, true, false), _2, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, scala.Tuple2, true])._1)._2, true, false)) AS _1#32, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._2, true, false) AS _2#33] + +- FlatMapGroupsWithState , newInstance(class scala.Tuple2), newInstance(class scala.Tuple3), [_1#26, _2#27], [key1#9, key2#10, timestamp#15-T10000ms], obj#31: scala.Tuple2, state info [ checkpoint = file:/tmp/spark-634482c9-a55a-4f4e-b352-babec98fb4fc/state, runId = dd65fff0-d901-4e0b-a1ad-8c09b69f33ba, opId = 0, ver = 0, numPartitions = 5], class[count[0]: bigint], 2, Update, ProcessingTimeTimeout, 1000, 0 + +- *(2) Sort [_1#26 ASC NULLS FIRST, _2#27 ASC NULLS FIRST], false, 0 + +- Exchange hashpartitioning(_1#26, _2#27, 5) + +- AppendColumns , newInstance(class scala.Tuple3), [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._1, true, false) AS _1#26, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._2, true, false) AS _2#27] + +- Exchange hashpartitioning(key1#9, 5) + +- EventTimeWatermark timestamp#15: timestamp, interval 10 seconds + +- *(1) Project [_1#56 AS key1#9, _2#57 AS key2#10, cast(_3#58L as timestamp) AS timestamp#15] + +- *(1) Project [_1#56, _2#57, _3#58L] + +- *(1) ScanV2 MemoryStreamDataSource$[_1#56, _2#57, _3#58L] + */ + // scalastyle:on line.size.limit + + AddData(inputData, ("a", "b", 1L)), + AdvanceManualClock(1 * 1000), + CheckNewAnswer(("a", "b", "1")), + + Execute { query => + val numPartitions = query.lastExecution.numStateStores + + val flatMapGroupsWithStateExecs = query.lastExecution.executedPlan.collect { + case f: FlatMapGroupsWithStateExec => f + } + + assert(flatMapGroupsWithStateExecs.length === 1) + assert(requireClusteredDistribution(flatMapGroupsWithStateExecs.head, + Seq(Seq("_1", "_2"), Seq("_1", "_2")), Some(numPartitions))) + assert(hasDesiredHashPartitioningInChildren( + flatMapGroupsWithStateExecs.head, Seq(Seq("_1", "_2"), Seq("_1", "_2")), numPartitions)) + } + ) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationDistributionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationDistributionSuite.scala new file mode 100644 index 0000000000000..41e984deedc91 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationDistributionSuite.scala @@ -0,0 +1,223 @@ +/* + * 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.File + +import org.apache.commons.io.FileUtils +import org.scalatest.Assertions + +import org.apache.spark.sql.catalyst.plans.physical.UnspecifiedDistribution +import org.apache.spark.sql.execution.aggregate.BaseAggregateExec +import org.apache.spark.sql.execution.streaming.{MemoryStream, StateStoreRestoreExec, StateStoreSaveExec} +import org.apache.spark.sql.functions.count +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.OutputMode.Update +import org.apache.spark.sql.streaming.util.HashClusteredDistributionTestHelper +import org.apache.spark.util.Utils + +class StreamingAggregationDistributionSuite extends StreamTest + with HashClusteredDistributionTestHelper with Assertions { + + import testImplicits._ + + test("SPARK-38204: streaming aggregation should require HashClusteredDistribution " + + "from children") { + + val input = MemoryStream[Int] + val df1 = input.toDF().select('value as 'key1, 'value * 2 as 'key2, 'value * 3 as 'value) + val agg = df1.repartition('key1).groupBy('key1, 'key2).agg(count('*)) + + testStream(agg, OutputMode.Update())( + AddData(input, 1, 1, 2, 3, 4), + CheckAnswer((1, 2, 2), (2, 4, 1), (3, 6, 1), (4, 8, 1)), + Execute { query => + val numPartitions = query.lastExecution.numStateStores + + // verify state store restore/save + val stateStoreOps = query.lastExecution.executedPlan.collect { + case s: StateStoreRestoreExec => s + case s: StateStoreSaveExec => s + } + + assert(stateStoreOps.nonEmpty) + stateStoreOps.foreach { stateOp => + assert(requireHashClusteredDistribution(stateOp, Seq(Seq("key1", "key2")), + numPartitions)) + assert(hasDesiredHashPartitioningInChildren(stateOp, Seq(Seq("key1", "key2")), + numPartitions)) + } + + // verify aggregations in between, except partial aggregation + val allAggregateExecs = query.lastExecution.executedPlan.collect { + case a: BaseAggregateExec => a + } + + val aggregateExecsWithoutPartialAgg = allAggregateExecs.filter { + _.requiredChildDistribution.head != UnspecifiedDistribution + } + + // We expect single partial aggregation - remaining agg execs should have child producing + // expected output partitioning. + assert(allAggregateExecs.length - 1 === aggregateExecsWithoutPartialAgg.length) + + // For aggregate execs, we make sure output partitioning of the children is same as + // we expect, HashPartitioning with clustering keys & number of partitions. + aggregateExecsWithoutPartialAgg.foreach { aggr => + assert(hasDesiredHashPartitioningInChildren(aggr, Seq(Seq("key1", "key2")), + numPartitions)) + } + } + ) + } + + test("SPARK-38204: streaming aggregation should require ClusteredDistribution " + + "from children if the query starts from checkpoint in prior to 3.3") { + + val inputData = MemoryStream[Int] + val df1 = inputData.toDF().select('value as 'key1, 'value * 2 as 'key2, 'value * 3 as 'value) + val agg = df1.repartition('key1).groupBy('key1, 'key2).agg(count('*)) + + val resourceUri = this.getClass.getResource( + "/structured-streaming/checkpoint-version-3.2.0-streaming-aggregate-with-repartition/").toURI + + val checkpointDir = Utils.createTempDir().getCanonicalFile + // Copy the checkpoint to a temp dir to prevent changes to the original. + // Not doing this will lead to the test passing on the first run, but fail subsequent runs. + FileUtils.copyDirectory(new File(resourceUri), checkpointDir) + + inputData.addData(3) + inputData.addData(3, 2) + + testStream(agg, Update)( + StartStream(checkpointLocation = checkpointDir.getAbsolutePath, + additionalConfs = Map(SQLConf.STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION.key -> "true")), + + // scalastyle:off line.size.limit + /* + Note: The checkpoint was generated using the following input in Spark version 3.2.0 + AddData(inputData, 3), + CheckLastBatch((3, 6, 1)), + AddData(inputData, 3, 2), + CheckLastBatch((3, 6, 2), (2, 4, 1)) + + Note2: The following plans are the physical plans of the query in older Spark versions + The physical plans around StateStoreRestore and StateStoreSave are quite similar, + especially shuffles being injected are same. That said, verifying with checkpoint being + built with Spark 3.2.0 would verify the following versions as well. + + A. Spark 3.2.0 + WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@61a581c0, org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$1968/1468582588@325b0006 + +- *(4) HashAggregate(keys=[key1#3, key2#4], functions=[count(1)], output=[key1#3, key2#4, count(1)#13L]) + +- StateStoreSave [key1#3, key2#4], state info [ checkpoint = file:/blabla/state, runId = 2bd7d18c-73b2-49a2-b2aa-1835162f9186, opId = 0, ver = 1, numPartitions = 5], Update, 0, 2 + +- *(3) HashAggregate(keys=[key1#3, key2#4], functions=[merge_count(1)], output=[key1#3, key2#4, count#47L]) + +- StateStoreRestore [key1#3, key2#4], state info [ checkpoint = file:/blabla/state, runId = 2bd7d18c-73b2-49a2-b2aa-1835162f9186, opId = 0, ver = 1, numPartitions = 5], 2 + +- *(2) HashAggregate(keys=[key1#3, key2#4], functions=[merge_count(1)], output=[key1#3, key2#4, count#47L]) + +- *(2) HashAggregate(keys=[key1#3, key2#4], functions=[partial_count(1)], output=[key1#3, key2#4, count#47L]) + +- Exchange hashpartitioning(key1#3, 5), REPARTITION_BY_COL, [id=#220] + +- *(1) Project [value#1 AS key1#3, (value#1 * 2) AS key2#4] + +- MicroBatchScan[value#1] MemoryStreamDataSource + + B. Spark 3.1.0 + WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@53602363 + +- *(4) HashAggregate(keys=[key1#3, key2#4], functions=[count(1)], output=[key1#3, key2#4, count(1)#13L]) + +- StateStoreSave [key1#3, key2#4], state info [ checkpoint = file:/tmp/spark-178e9eaf-b527-499c-8eb6-c9e734f9fdfc/state, runId = 9c7e8635-41ab-4141-9f46-7ab473c58560, opId = 0, ver = 1, numPartitions = 5], Update, 0, 2 + +- *(3) HashAggregate(keys=[key1#3, key2#4], functions=[merge_count(1)], output=[key1#3, key2#4, count#47L]) + +- StateStoreRestore [key1#3, key2#4], state info [ checkpoint = file:/tmp/spark-178e9eaf-b527-499c-8eb6-c9e734f9fdfc/state, runId = 9c7e8635-41ab-4141-9f46-7ab473c58560, opId = 0, ver = 1, numPartitions = 5], 2 + +- *(2) HashAggregate(keys=[key1#3, key2#4], functions=[merge_count(1)], output=[key1#3, key2#4, count#47L]) + +- *(2) HashAggregate(keys=[key1#3, key2#4], functions=[partial_count(1)], output=[key1#3, key2#4, count#47L]) + +- Exchange hashpartitioning(key1#3, 5), REPARTITION, [id=#222] + +- *(1) Project [value#1 AS key1#3, (value#1 * 2) AS key2#4] + +- *(1) Project [value#1] + +- MicroBatchScan[value#1] MemoryStreamDataSource + + C. Spark 3.0.0 + WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@33379044 + +- *(4) HashAggregate(keys=[key1#3, key2#4], functions=[count(1)], output=[key1#3, key2#4, count(1)#13L]) + +- StateStoreSave [key1#3, key2#4], state info [ checkpoint = file:/tmp/spark-83497e04-657c-4cad-b532-f433b1532302/state, runId = 1a650994-486f-4f32-92d9-f7c05d49d0a0, opId = 0, ver = 1, numPartitions = 5], Update, 0, 2 + +- *(3) HashAggregate(keys=[key1#3, key2#4], functions=[merge_count(1)], output=[key1#3, key2#4, count#47L]) + +- StateStoreRestore [key1#3, key2#4], state info [ checkpoint = file:/tmp/spark-83497e04-657c-4cad-b532-f433b1532302/state, runId = 1a650994-486f-4f32-92d9-f7c05d49d0a0, opId = 0, ver = 1, numPartitions = 5], 2 + +- *(2) HashAggregate(keys=[key1#3, key2#4], functions=[merge_count(1)], output=[key1#3, key2#4, count#47L]) + +- *(2) HashAggregate(keys=[key1#3, key2#4], functions=[partial_count(1)], output=[key1#3, key2#4, count#47L]) + +- Exchange hashpartitioning(key1#3, 5), false, [id=#104] + +- *(1) Project [value#1 AS key1#3, (value#1 * 2) AS key2#4] + +- *(1) Project [value#1] + +- MicroBatchScan[value#1] MemoryStreamDataSource + + D. Spark 2.4.0 + *(4) HashAggregate(keys=[key1#3, key2#4], functions=[count(1)], output=[key1#3, key2#4, count(1)#13L]) + +- StateStoreSave [key1#3, key2#4], state info [ checkpoint = file:/tmp/spark-c4fd5b1f-18e0-4433-ac7a-00df93464b49/state, runId = 89bfe27b-da33-4a75-9f36-97717c137b2a, opId = 0, ver = 1, numPartitions = 5], Update, 0, 2 + +- *(3) HashAggregate(keys=[key1#3, key2#4], functions=[merge_count(1)], output=[key1#3, key2#4, count#42L]) + +- StateStoreRestore [key1#3, key2#4], state info [ checkpoint = file:/tmp/spark-c4fd5b1f-18e0-4433-ac7a-00df93464b49/state, runId = 89bfe27b-da33-4a75-9f36-97717c137b2a, opId = 0, ver = 1, numPartitions = 5], 2 + +- *(2) HashAggregate(keys=[key1#3, key2#4], functions=[merge_count(1)], output=[key1#3, key2#4, count#42L]) + +- *(2) HashAggregate(keys=[key1#3, key2#4], functions=[partial_count(1)], output=[key1#3, key2#4, count#42L]) + +- Exchange hashpartitioning(key1#3, 5) + +- *(1) Project [value#47 AS key1#3, (value#47 * 2) AS key2#4] + +- *(1) Project [value#47] + +- *(1) ScanV2 MemoryStreamDataSource$[value#47] + */ + // scalastyle:on line.size.limit + + AddData(inputData, 3, 2, 1), + CheckLastBatch((3, 6, 3), (2, 4, 2), (1, 2, 1)), + + Execute { query => + val executedPlan = query.lastExecution.executedPlan + assert(!executedPlan.conf.getConf(SQLConf.STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION)) + + val numPartitions = query.lastExecution.numStateStores + + // verify state store restore/save + val stateStoreOps = executedPlan.collect { + case s: StateStoreRestoreExec => s + case s: StateStoreSaveExec => s + } + + assert(stateStoreOps.nonEmpty) + stateStoreOps.foreach { stateOp => + assert(requireClusteredDistribution(stateOp, Seq(Seq("key1", "key2")), + Some(numPartitions))) + assert(hasDesiredHashPartitioningInChildren(stateOp, Seq(Seq("key1")), + numPartitions)) + } + + // verify aggregations in between, except partial aggregation + val allAggregateExecs = executedPlan.collect { + case a: BaseAggregateExec => a + } + + val aggregateExecsWithoutPartialAgg = allAggregateExecs.filter { + _.requiredChildDistribution.head != UnspecifiedDistribution + } + + // We expect single partial aggregation - remaining agg execs should have child producing + // expected output partitioning. + assert(allAggregateExecs.length - 1 === aggregateExecsWithoutPartialAgg.length) + + // For aggregate execs, we make sure output partitioning of the children is same as + // we expect, HashPartitioning with sub-clustering keys & number of partitions. + aggregateExecsWithoutPartialAgg.foreach { aggr => + assert(requireClusteredDistribution(aggr, Seq(Seq("key1", "key2")), + Some(numPartitions))) + assert(hasDesiredHashPartitioningInChildren(aggr, Seq(Seq("key1")), + numPartitions)) + } + } + ) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 8a7bb8b60c878..d5e6231484f74 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -30,6 +30,7 @@ import org.apache.spark.rdd.BlockRDD import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.logical.Aggregate +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, SinglePartition} import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.exchange.Exchange @@ -542,8 +543,8 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { /** * This method verifies certain properties in the SparkPlan of a streaming aggregation. * First of all, it checks that the child of a `StateStoreRestoreExec` creates the desired - * data distribution, where the child could be an Exchange, or a `HashAggregateExec` which already - * provides the expected data distribution. + * data distribution, where the child is a `HashAggregateExec` which already provides + * the expected data distribution. * * The second thing it checks that the child provides the expected number of partitions. * @@ -552,7 +553,6 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { */ private def checkAggregationChain( se: StreamExecution, - expectShuffling: Boolean, expectedPartition: Int): Boolean = { val executedPlan = se.lastExecution.executedPlan val restore = executedPlan @@ -560,12 +560,17 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { .head restore.child match { case node: UnaryExecNode => - assert(node.outputPartitioning.numPartitions === expectedPartition, - "Didn't get the expected number of partitions.") - if (expectShuffling) { - assert(node.isInstanceOf[Exchange], s"Expected a shuffle, got: ${node.child}") - } else { - assert(!node.isInstanceOf[Exchange], "Didn't expect a shuffle") + node.outputPartitioning match { + case HashPartitioning(_, numPartitions) => + assert(numPartitions === expectedPartition, + "Didn't get the expected number of partitions.") + + // below case should only applied to no grouping key which leads to AllTuples + case SinglePartition if expectedPartition == 1 => // OK + + case p => + fail("Expected a hash partitioning for child output partitioning, but has " + + s"$p instead.") } case _ => fail("Expected no shuffling") @@ -605,12 +610,12 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { AddBlockData(inputSource, Seq(1)), CheckLastBatch(1), AssertOnQuery("Verify no shuffling") { se => - checkAggregationChain(se, expectShuffling = false, 1) + checkAggregationChain(se, 1) }, AddBlockData(inputSource), // create an empty trigger CheckLastBatch(1), AssertOnQuery("Verify that no exchange is required") { se => - checkAggregationChain(se, expectShuffling = false, 1) + checkAggregationChain(se, 1) }, AddBlockData(inputSource, Seq(2, 3)), CheckLastBatch(3), @@ -647,10 +652,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { AddBlockData(inputSource, Seq(1)), CheckLastBatch((0L, 1L)), AssertOnQuery("Verify addition of exchange operator") { se => - checkAggregationChain( - se, - expectShuffling = true, - spark.sessionState.conf.numShufflePartitions) + checkAggregationChain(se, spark.sessionState.conf.numShufflePartitions) }, StopStream ) @@ -661,10 +663,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { AddBlockData(inputSource, Seq(2), Seq(3), Seq(4)), CheckLastBatch((0L, 4L)), AssertOnQuery("Verify no exchange added") { se => - checkAggregationChain( - se, - expectShuffling = false, - spark.sessionState.conf.numShufflePartitions) + checkAggregationChain(se, spark.sessionState.conf.numShufflePartitions) }, AddBlockData(inputSource), CheckLastBatch((0L, 4L)), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationDistributionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationDistributionSuite.scala new file mode 100644 index 0000000000000..b5db782d1c686 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationDistributionSuite.scala @@ -0,0 +1,148 @@ +/* + * 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.File + +import org.apache.commons.io.FileUtils + +import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Update +import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamingDeduplicateExec} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.util.HashClusteredDistributionTestHelper +import org.apache.spark.util.Utils + +class StreamingDeduplicationDistributionSuite extends StreamTest + with HashClusteredDistributionTestHelper { + + import testImplicits._ + + test("SPARK-38204: streaming deduplication should require HashClusteredDistribution " + + "from children") { + + val input = MemoryStream[Int] + val df1 = input.toDF().select('value as 'key1, 'value * 2 as 'key2, 'value * 3 as 'value) + val dedup = df1.repartition('key1).dropDuplicates("key1", "key2") + + testStream(dedup, OutputMode.Update())( + AddData(input, 1, 1, 2, 3, 4), + CheckAnswer((1, 2, 3), (2, 4, 6), (3, 6, 9), (4, 8, 12)), + Execute { query => + val numPartitions = query.lastExecution.numStateStores + + val dedupExecs = query.lastExecution.executedPlan.collect { + case d: StreamingDeduplicateExec => d + } + + assert(dedupExecs.length === 1) + assert(requireHashClusteredDistribution( + dedupExecs.head, Seq(Seq("key1", "key2")), numPartitions)) + assert(hasDesiredHashPartitioningInChildren( + dedupExecs.head, Seq(Seq("key1", "key2")), numPartitions)) + } + ) + } + + test("SPARK-38204: streaming deduplication should require ClusteredDistribution " + + "from children if the query starts from checkpoint in prior to 3.3") { + + val inputData = MemoryStream[Int] + val df1 = inputData.toDF().select('value as 'key1, 'value * 2 as 'key2, 'value * 3 as 'value) + val dedup = df1.repartition('key1).dropDuplicates("key1", "key2") + + val resourceUri = this.getClass.getResource( + "/structured-streaming/checkpoint-version-3.2.0-deduplication-with-repartition/").toURI + + val checkpointDir = Utils.createTempDir().getCanonicalFile + // Copy the checkpoint to a temp dir to prevent changes to the original. + // Not doing this will lead to the test passing on the first run, but fail subsequent runs. + FileUtils.copyDirectory(new File(resourceUri), checkpointDir) + + inputData.addData(1, 1, 2) + inputData.addData(3, 4) + + testStream(dedup, Update)( + StartStream(checkpointLocation = checkpointDir.getAbsolutePath, + additionalConfs = Map(SQLConf.STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION.key -> "true")), + + // scalastyle:off line.size.limit + /* + Note: The checkpoint was generated using the following input in Spark version 3.2.0 + AddData(inputData, 1, 1, 2), + CheckLastBatch((1, 2, 3), (2, 4, 6)), + AddData(inputData, 3, 4), + CheckLastBatch((3, 6, 9), (4, 8, 12)) + + Note2: The following plans are the physical plans of the query in older Spark versions + The physical plans around StreamingDeduplicate are quite similar, especially shuffles + being injected are same. That said, verifying with checkpoint being built with + Spark 3.2.0 would verify the following versions as well. + + A. Spark 3.2.0 + WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@76467fb2, org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$1900/1334867523@32b72162 + +- StreamingDeduplicate [key1#3, key2#4], state info [ checkpoint = file:/blabla/state, runId = bf82c05e-4031-4421-89e0-28fd9127eb5b, opId = 0, ver = 1, numPartitions = 5], 0 + +- Exchange hashpartitioning(key1#3, 5), REPARTITION_BY_COL, [id=#115] + +- *(1) Project [value#1 AS key1#3, (value#1 * 2) AS key2#4, (value#1 * 3) AS value#5] + +- MicroBatchScan[value#1] MemoryStreamDataSource + + B. Spark 3.1.0 + WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@133d8337 + +- StreamingDeduplicate [key1#3, key2#4], state info [ checkpoint = file:/tmp/spark-c0b73191-75ec-4a54-89b7-368fbbc4b2a8/state, runId = 9b2baaee-1147-4faf-98b4-3c3d8ee34966, opId = 0, ver = 1, numPartitions = 5], 0 + +- Exchange hashpartitioning(key1#3, 5), REPARTITION, [id=#117] + +- *(1) Project [value#1 AS key1#3, (value#1 * 2) AS key2#4, (value#1 * 3) AS value#5] + +- *(1) Project [value#1] + +- MicroBatchScan[value#1] MemoryStreamDataSource + + C. Spark 3.0.0 + WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@bb06c00 + +- StreamingDeduplicate [key1#3, key2#4], state info [ checkpoint = file:/tmp/spark-6f8a96c7-2af5-4952-a1b4-c779766334ef/state, runId = 9a208eb0-d915-46dd-a0fd-23b1df82b951, opId = 0, ver = 1, numPartitions = 5], 0 + +- Exchange hashpartitioning(key1#3, 5), false, [id=#57] + +- *(1) Project [value#1 AS key1#3, (value#1 * 2) AS key2#4, (value#1 * 3) AS value#5] + +- *(1) Project [value#1] + +- MicroBatchScan[value#1] MemoryStreamDataSource + + D. Spark 2.4.0 + StreamingDeduplicate [key1#3, key2#4], state info [ checkpoint = file:/tmp/spark-d8a684a0-5623-4739-85e8-e45b99768aa7/state, runId = 85bd75bd-3d45-4d42-aeac-9e45fc559ee9, opId = 0, ver = 1, numPartitions = 5], 0 + +- Exchange hashpartitioning(key1#3, 5) + +- *(1) Project [value#37 AS key1#3, (value#37 * 2) AS key2#4, (value#37 * 3) AS value#5] + +- *(1) Project [value#37] + +- *(1) ScanV2 MemoryStreamDataSource$[value#37] + */ + // scalastyle:on line.size.limit + + AddData(inputData, 2, 3, 4, 5), + CheckLastBatch((5, 10, 15)), + Execute { query => + val executedPlan = query.lastExecution.executedPlan + assert(!executedPlan.conf.getConf(SQLConf.STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION)) + + val numPartitions = query.lastExecution.numStateStores + + val dedupExecs = executedPlan.collect { + case d: StreamingDeduplicateExec => d + } + + assert(dedupExecs.length === 1) + assert(requireClusteredDistribution( + dedupExecs.head, Seq(Seq("key1", "key2")), Some(numPartitions))) + assert(hasDesiredHashPartitioningInChildren( + dedupExecs.head, Seq(Seq("key1")), numPartitions)) + } + ) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowDistributionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowDistributionSuite.scala new file mode 100644 index 0000000000000..91bee130c21b8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSessionWindowDistributionSuite.scala @@ -0,0 +1,225 @@ +/* + * 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.File + +import org.apache.commons.io.FileUtils + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.plans.physical.UnspecifiedDistribution +import org.apache.spark.sql.execution.aggregate.BaseAggregateExec +import org.apache.spark.sql.execution.streaming.{MemoryStream, SessionWindowStateStoreRestoreExec, SessionWindowStateStoreSaveExec} +import org.apache.spark.sql.functions.{count, session_window} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.util.HashClusteredDistributionTestHelper +import org.apache.spark.util.Utils + +class StreamingSessionWindowDistributionSuite extends StreamTest + with HashClusteredDistributionTestHelper with Logging { + + import testImplicits._ + + test("SPARK-38204: session window aggregation should require HashClusteredDistribution " + + "from children") { + + withSQLConf( + // exclude partial merging session to simplify test + SQLConf.STREAMING_SESSION_WINDOW_MERGE_SESSIONS_IN_LOCAL_PARTITION.key -> "false") { + + val inputData = MemoryStream[(String, String, Long)] + + // Split the lines into words, treat words as sessionId of events + val events = inputData.toDF() + .select($"_1".as("value"), $"_2".as("userId"), $"_3".as("timestamp")) + .withColumn("eventTime", $"timestamp".cast("timestamp")) + .withWatermark("eventTime", "30 seconds") + .selectExpr("explode(split(value, ' ')) AS sessionId", "userId", "eventTime") + + val sessionUpdates = events + .repartition($"userId") + .groupBy(session_window($"eventTime", "10 seconds") as 'session, 'sessionId, 'userId) + .agg(count("*").as("numEvents")) + .selectExpr("sessionId", "userId", "CAST(session.start AS LONG)", + "CAST(session.end AS LONG)", + "CAST(session.end AS LONG) - CAST(session.start AS LONG) AS durationMs", + "numEvents") + + testStream(sessionUpdates, OutputMode.Append())( + AddData(inputData, + ("hello world spark streaming", "key1", 40L), + ("world hello structured streaming", "key2", 41L) + ), + + // skip checking the result, since we focus to verify the physical plan + ProcessAllAvailable(), + Execute { query => + val numPartitions = query.lastExecution.numStateStores + + val operators = query.lastExecution.executedPlan.collect { + case s: SessionWindowStateStoreRestoreExec => s + case s: SessionWindowStateStoreSaveExec => s + } + + assert(operators.nonEmpty) + operators.foreach { stateOp => + assert(requireHashClusteredDistribution(stateOp, Seq(Seq("sessionId", "userId")), + numPartitions)) + assert(hasDesiredHashPartitioningInChildren(stateOp, Seq(Seq("sessionId", "userId")), + numPartitions)) + } + + // Verify aggregations in between, except partial aggregation. + // This includes MergingSessionsExec. + val allAggregateExecs = query.lastExecution.executedPlan.collect { + case a: BaseAggregateExec => a + } + + val aggregateExecsWithoutPartialAgg = allAggregateExecs.filter { + _.requiredChildDistribution.head != UnspecifiedDistribution + } + + // We expect single partial aggregation since we disable partial merging sessions. + // Remaining agg execs should have child producing expected output partitioning. + assert(allAggregateExecs.length - 1 === aggregateExecsWithoutPartialAgg.length) + + // For aggregate execs, we make sure output partitioning of the children is same as + // we expect, HashPartitioning with clustering keys & number of partitions. + aggregateExecsWithoutPartialAgg.foreach { aggr => + assert(hasDesiredHashPartitioningInChildren(aggr, Seq(Seq("sessionId", "userId")), + numPartitions)) + } + } + ) + } + } + + test("SPARK-38204: session window aggregation should require ClusteredDistribution " + + "from children if the query starts from checkpoint in 3.2") { + + withSQLConf( + // exclude partial merging session to simplify test + SQLConf.STREAMING_SESSION_WINDOW_MERGE_SESSIONS_IN_LOCAL_PARTITION.key -> "false") { + + val inputData = MemoryStream[(String, String, Long)] + + // Split the lines into words, treat words as sessionId of events + val events = inputData.toDF() + .select($"_1".as("value"), $"_2".as("userId"), $"_3".as("timestamp")) + .withColumn("eventTime", $"timestamp".cast("timestamp")) + .withWatermark("eventTime", "30 seconds") + .selectExpr("explode(split(value, ' ')) AS sessionId", "userId", "eventTime") + + val sessionUpdates = events + .repartition($"userId") + .groupBy(session_window($"eventTime", "10 seconds") as 'session, 'sessionId, 'userId) + .agg(count("*").as("numEvents")) + .selectExpr("sessionId", "userId", "CAST(session.start AS LONG)", + "CAST(session.end AS LONG)", + "CAST(session.end AS LONG) - CAST(session.start AS LONG) AS durationMs", + "numEvents") + + val resourceUri = this.getClass.getResource( + "/structured-streaming/checkpoint-version-3.2.0-session-window-with-repartition/").toURI + + val checkpointDir = Utils.createTempDir().getCanonicalFile + // Copy the checkpoint to a temp dir to prevent changes to the original. + // Not doing this will lead to the test passing on the first run, but fail subsequent runs. + FileUtils.copyDirectory(new File(resourceUri), checkpointDir) + + inputData.addData( + ("hello world spark streaming", "key1", 40L), + ("world hello structured streaming", "key2", 41L)) + + testStream(sessionUpdates, OutputMode.Append())( + StartStream(checkpointLocation = checkpointDir.getAbsolutePath, + additionalConfs = Map(SQLConf.STATEFUL_OPERATOR_USE_STRICT_DISTRIBUTION.key -> "true")), + + // scalastyle:off line.size.limit + /* + Note: The checkpoint was generated using the following input in Spark version 3.2.0 + AddData(inputData, + ("hello world spark streaming", "key1", 40L), + ("world hello structured streaming", "key2", 41L)), + // skip checking the result, since we focus to verify the physical plan + ProcessAllAvailable() + + Note2: The following is the physical plan of the query in Spark version 3.2.0. + + WriteToDataSourceV2 org.apache.spark.sql.execution.streaming.sources.MicroBatchWrite@6649ee50, org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$2209/0x0000000840ebd440@f9f45c6 + +- *(3) HashAggregate(keys=[session_window#33-T30000ms, sessionId#21, userId#10], functions=[count(1)], output=[sessionId#21, userId#10, CAST(session.start AS BIGINT)#43L, CAST(session.end AS BIGINT)#44L, durationMs#38L, numEvents#32L]) + +- SessionWindowStateStoreSave [sessionId#21, userId#10], session_window#33: struct, state info [ checkpoint = file:/tmp/spark-f8a951f5-c7c1-43b0-883d-9b893d672ee5/state, runId = 92681f36-1f0d-434e-8492-897e4e988bb3, opId = 0, ver = 1, numPartitions = 5], Append, 11000, 1 + +- MergingSessions List(ClusteredDistribution(ArrayBuffer(sessionId#21, userId#10),None)), [session_window#33-T30000ms, sessionId#21, userId#10], session_window#33: struct, [merge_count(1)], [count(1)#30L], 3, [session_window#33-T30000ms, sessionId#21, userId#10, count#58L] + +- SessionWindowStateStoreRestore [sessionId#21, userId#10], session_window#33: struct, state info [ checkpoint = file:/tmp/spark-f8a951f5-c7c1-43b0-883d-9b893d672ee5/state, runId = 92681f36-1f0d-434e-8492-897e4e988bb3, opId = 0, ver = 1, numPartitions = 5], 11000, 1 + +- *(2) Sort [sessionId#21 ASC NULLS FIRST, userId#10 ASC NULLS FIRST, session_window#33-T30000ms ASC NULLS FIRST], false, 0 + +- *(2) HashAggregate(keys=[session_window#33-T30000ms, sessionId#21, userId#10], functions=[partial_count(1)], output=[session_window#33-T30000ms, sessionId#21, userId#10, count#58L]) + +- *(2) Project [named_struct(start, precisetimestampconversion(precisetimestampconversion(eventTime#15-T30000ms, TimestampType, LongType), LongType, TimestampType), end, precisetimestampconversion(precisetimestampconversion(eventTime#15-T30000ms + 10 seconds, TimestampType, LongType), LongType, TimestampType)) AS session_window#33-T30000ms, sessionId#21, userId#10] + +- Exchange hashpartitioning(userId#10, 5), REPARTITION_BY_COL, [id=#372] + +- *(1) Project [sessionId#21, userId#10, eventTime#15-T30000ms] + +- *(1) Generate explode(split(value#9, , -1)), [userId#10, eventTime#15-T30000ms], false, [sessionId#21] + +- *(1) Filter (precisetimestampconversion(precisetimestampconversion(eventTime#15-T30000ms + 10 seconds, TimestampType, LongType), LongType, TimestampType) > precisetimestampconversion(precisetimestampconversion(eventTime#15-T30000ms, TimestampType, LongType), LongType, TimestampType)) + +- EventTimeWatermark eventTime#15: timestamp, 30 seconds + +- LocalTableScan , [value#9, userId#10, eventTime#15] + */ + // scalastyle:on line.size.limit + + AddData(inputData, ("spark streaming", "key1", 25L)), + // skip checking the result, since we focus to verify the physical plan + ProcessAllAvailable(), + + Execute { query => + val numPartitions = query.lastExecution.numStateStores + + val operators = query.lastExecution.executedPlan.collect { + case s: SessionWindowStateStoreRestoreExec => s + case s: SessionWindowStateStoreSaveExec => s + } + + assert(operators.nonEmpty) + operators.foreach { stateOp => + assert(requireClusteredDistribution(stateOp, Seq(Seq("sessionId", "userId")), + Some(numPartitions))) + assert(hasDesiredHashPartitioningInChildren(stateOp, Seq(Seq("userId")), + numPartitions)) + } + + // Verify aggregations in between, except partial aggregation. + // This includes MergingSessionsExec. + val allAggregateExecs = query.lastExecution.executedPlan.collect { + case a: BaseAggregateExec => a + } + + val aggregateExecsWithoutPartialAgg = allAggregateExecs.filter { + _.requiredChildDistribution.head != UnspecifiedDistribution + } + + // We expect single partial aggregation since we disable partial merging sessions. + // Remaining agg execs should have child producing expected output partitioning. + assert(allAggregateExecs.length - 1 === aggregateExecsWithoutPartialAgg.length) + + // For aggregate execs, we make sure output partitioning of the children is same as + // we expect, HashPartitioning with sub-clustering keys & number of partitions. + aggregateExecsWithoutPartialAgg.foreach { aggr => + assert(hasDesiredHashPartitioningInChildren(aggr, Seq(Seq("userId")), + numPartitions)) + } + } + ) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/HashClusteredDistributionTestHelper.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/HashClusteredDistributionTestHelper.scala new file mode 100644 index 0000000000000..3279b7985de11 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/HashClusteredDistributionTestHelper.scala @@ -0,0 +1,80 @@ +/* + * 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.util + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} +import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, HashClusteredDistribution, HashPartitioning} +import org.apache.spark.sql.execution.SparkPlan + +trait HashClusteredDistributionTestHelper extends SparkFunSuite { + protected def requireClusteredDistribution( + plan: SparkPlan, + desiredClusterColumns: Seq[Seq[String]], + desiredNumPartitions: Option[Int]): Boolean = { + assert(plan.requiredChildDistribution.length === desiredClusterColumns.length) + plan.requiredChildDistribution.zip(desiredClusterColumns).forall { + case (d: ClusteredDistribution, clusterColumns: Seq[String]) + if partitionExpressionsColumns(d.clustering) == clusterColumns && + d.requiredNumPartitions == desiredNumPartitions => true + + case _ => false + } + } + + protected def requireHashClusteredDistribution( + plan: SparkPlan, + desiredClusterColumns: Seq[Seq[String]], + desiredNumPartitions: Int): Boolean = { + assert(plan.requiredChildDistribution.length === desiredClusterColumns.length) + plan.requiredChildDistribution.zip(desiredClusterColumns).forall { + case (d: HashClusteredDistribution, clusterColumns: Seq[String]) + if partitionExpressionsColumns(d.expressions) == clusterColumns && + d.requiredNumPartitions == Some(desiredNumPartitions) => true + + case _ => false + } + } + + protected def hasDesiredHashPartitioning( + plan: SparkPlan, + desiredClusterColumns: Seq[String], + desiredNumPartitions: Int): Boolean = { + plan.outputPartitioning match { + case HashPartitioning(expressions, numPartitions) + if partitionExpressionsColumns(expressions) == desiredClusterColumns && + numPartitions == desiredNumPartitions => true + + case _ => false + } + } + + protected def hasDesiredHashPartitioningInChildren( + plan: SparkPlan, + desiredClusterColumns: Seq[Seq[String]], + desiredNumPartitions: Int): Boolean = { + plan.children.zip(desiredClusterColumns).forall { case (child, clusterColumns) => + hasDesiredHashPartitioning(child, clusterColumns, desiredNumPartitions) + } + } + + private def partitionExpressionsColumns(expressions: Seq[Expression]): Seq[String] = { + expressions.flatMap { + case ref: AttributeReference => Some(ref.name) + } + } +}