diff --git a/docs/ss-migration-guide.md b/docs/ss-migration-guide.md index 480e5e2695a1..c28724576bc4 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 3.2 to 3.3 + +- Since Spark 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/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index edb388e21987..3611eea86c57 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 @@ -1773,6 +1773,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 " + + "StatefulOpClusteredDistribution 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.3.0") + .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 32db622c9f93..26161acae30b 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 @@ -45,8 +45,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, @@ -60,6 +80,8 @@ object AggUtils { if (useHash && !forceSortAggregate) { HashAggregateExec( requiredChildDistributionExpressions = requiredChildDistributionExpressions, + isStreaming = isStreaming, + numShufflePartitions = None, groupingExpressions = groupingExpressions, aggregateExpressions = mayRemoveAggFilters(aggregateExpressions), aggregateAttributes = aggregateAttributes, @@ -73,6 +95,8 @@ object AggUtils { if (objectHashEnabled && useObjectHash && !forceSortAggregate) { ObjectHashAggregateExec( requiredChildDistributionExpressions = requiredChildDistributionExpressions, + isStreaming = isStreaming, + numShufflePartitions = None, groupingExpressions = groupingExpressions, aggregateExpressions = mayRemoveAggFilters(aggregateExpressions), aggregateAttributes = aggregateAttributes, @@ -82,6 +106,8 @@ object AggUtils { } else { SortAggregateExec( requiredChildDistributionExpressions = requiredChildDistributionExpressions, + isStreaming = isStreaming, + numShufflePartitions = None, groupingExpressions = groupingExpressions, aggregateExpressions = mayRemoveAggFilters(aggregateExpressions), aggregateAttributes = aggregateAttributes, @@ -290,7 +316,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, @@ -302,7 +328,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, @@ -320,7 +346,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, @@ -348,7 +374,7 @@ object AggUtils { // projection: val finalAggregateAttributes = finalAggregateExpressions.map(_.resultAttribute) - createAggregate( + createStreamingAggregate( requiredChildDistributionExpressions = Some(groupingAttributes), groupingExpressions = groupingAttributes, aggregateExpressions = finalAggregateExpressions, @@ -407,7 +433,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, @@ -424,7 +450,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, @@ -447,8 +474,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, @@ -476,8 +505,8 @@ object AggUtils { // projection: val finalAggregateAttributes = finalAggregateExpressions.map(_.resultAttribute) - createAggregate( - requiredChildDistributionExpressions = Some(groupingAttributes), + createStreamingAggregate( + requiredChildDistributionExpressions = Some(groupingWithoutSessionAttributes), groupingExpressions = groupingAttributes, aggregateExpressions = finalAggregateExpressions, aggregateAttributes = finalAggregateAttributes, @@ -491,10 +520,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) @@ -506,7 +540,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)) @@ -519,7 +554,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 b709c8092e46..756b5eb09d0b 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] @@ -92,7 +95,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 } } @@ -102,7 +118,8 @@ trait BaseAggregateExec extends UnaryExecNode with AliasAwareOutputPartitioning */ def toSortAggregate: SortAggregateExec = { SortAggregateExec( - requiredChildDistributionExpressions, groupingExpressions, aggregateExpressions, - aggregateAttributes, initialInputBufferOffset, resultExpressions, child) + requiredChildDistributionExpressions, isStreaming, numShufflePartitions, groupingExpressions, + aggregateExpressions, aggregateAttributes, initialInputBufferOffset, resultExpressions, + child) } } 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 1b4f4be501cc..8be3a018cee5 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 08e8b59a1782..31245c545185 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 c98c9f42e69d..9da0ca93c181 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 a0557822795a..3cf63a5318dc 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 @@ -32,6 +32,8 @@ import org.apache.spark.sql.internal.SQLConf */ 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 f15a22403cfb..fee7e29f8add 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 dfcb70737666..973c06975357 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,13 +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] = { - // NOTE: Please read through the NOTE on the classdoc of StatefulOpClusteredDistribution - // before making any changes. - // TODO(SPARK-38204) - ClusteredDistribution( - groupingAttributes, requiredNumPartitions = stateInfo.map(_.numPartitions)) :: - ClusteredDistribution( - initialStateGroupAttrs, requiredNumPartitions = 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 3e772e104648..9670c774a74c 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 c08a14c65b77..913805d1a074 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 000000000000..527349201574 --- /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, StatefulOpClusteredDistribution} +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 + * [[StatefulOpClusteredDistribution]] before making any changes. Please refer SPARK-38204 + * for details. + * + * Do not use methods in this object for stateful operators which already uses + * [[StatefulOpClusteredDistribution]] 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)) { + StatefulOpClusteredDistribution(expressions, 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 45c6430f9642..bcfdeb4f85cd 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._ @@ -334,14 +334,11 @@ case class StateStoreRestoreExec( override def outputPartitioning: Partitioning = child.outputPartitioning override def requiredChildDistribution: Seq[Distribution] = { - // NOTE: Please read through the NOTE on the classdoc of StatefulOpClusteredDistribution - // before making any changes. - // TODO(SPARK-38204) if (keyExpressions.isEmpty) { AllTuples :: Nil } else { - ClusteredDistribution(keyExpressions, - requiredNumPartitions = stateInfo.map(_.numPartitions)) :: Nil + StatefulOperatorPartitioning.getCompatibleDistribution( + keyExpressions, getStateInfo, conf) :: Nil } } @@ -497,14 +494,11 @@ case class StateStoreSaveExec( override def outputPartitioning: Partitioning = child.outputPartitioning override def requiredChildDistribution: Seq[Distribution] = { - // NOTE: Please read through the NOTE on the classdoc of StatefulOpClusteredDistribution - // before making any changes. - // TODO(SPARK-38204) if (keyExpressions.isEmpty) { AllTuples :: Nil } else { - ClusteredDistribution(keyExpressions, - requiredNumPartitions = stateInfo.map(_.numPartitions)) :: Nil + StatefulOperatorPartitioning.getCompatibleDistribution( + keyExpressions, getStateInfo, conf) :: Nil } } @@ -581,11 +575,8 @@ case class SessionWindowStateStoreRestoreExec( } override def requiredChildDistribution: Seq[Distribution] = { - // NOTE: Please read through the NOTE on the classdoc of StatefulOpClusteredDistribution - // before making any changes. - // TODO(SPARK-38204) - ClusteredDistribution(keyWithoutSessionExpressions, - requiredNumPartitions = stateInfo.map(_.numPartitions)) :: Nil + StatefulOperatorPartitioning.getCompatibleDistribution( + keyWithoutSessionExpressions, getStateInfo, conf) :: Nil } override def requiredChildOrdering: Seq[Seq[SortOrder]] = { @@ -696,11 +687,8 @@ case class SessionWindowStateStoreSaveExec( override def outputPartitioning: Partitioning = child.outputPartitioning override def requiredChildDistribution: Seq[Distribution] = { - // NOTE: Please read through the NOTE on the classdoc of StatefulOpClusteredDistribution - // before making any changes. - // TODO(SPARK-38204) - ClusteredDistribution(keyExpressions, - requiredNumPartitions = stateInfo.map(_.numPartitions)) :: Nil + StatefulOperatorPartitioning.getCompatibleDistribution( + keyWithoutSessionExpressions, getStateInfo, conf) :: Nil } override def shouldRunAnotherBatch(newMetadata: OffsetSeqMetadata): Boolean = { @@ -758,11 +746,8 @@ case class StreamingDeduplicateExec( /** Distribute by grouping attributes */ override def requiredChildDistribution: Seq[Distribution] = { - // NOTE: Please read through the NOTE on the classdoc of StatefulOpClusteredDistribution - // before making any changes. - // TODO(SPARK-38204) - ClusteredDistribution(keyExpressions, - requiredNumPartitions = stateInfo.map(_.numPartitions)) :: Nil + StatefulOperatorPartitioning.getCompatibleDistribution( + keyExpressions, getStateInfo, conf) :: Nil } override protected def doExecute(): RDD[InternalRow] = { 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 000000000000..9c1e3021c3ea --- /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 000000000000..019111c30702 --- /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 000000000000..d00e8a5a4134 --- /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 000000000000..635297805184 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 000000000000..d3948722c325 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 000000000000..2639d3211dec 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 000000000000..635297805184 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 000000000000..635297805184 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 000000000000..635297805184 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 000000000000..1aee7033161e 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 000000000000..1aee7033161e 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 000000000000..9c1e3021c3ea --- /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 000000000000..9c1e3021c3ea --- /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 000000000000..81acb4439e8f --- /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 000000000000..b8a997658581 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 000000000000..81716485cf02 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 000000000000..852130a526e0 --- /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 000000000000..2d894644897b --- /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 000000000000..cf1d68e2acee 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 000000000000..cf1d68e2acee 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 000000000000..635297805184 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 000000000000..635297805184 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 000000000000..f03866c573c1 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 000000000000..e4695f58d7de 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 000000000000..cf1d68e2acee 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 000000000000..dc5c3a4905a5 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 000000000000..635297805184 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 000000000000..00c03b0f2aaa 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 000000000000..cf1d68e2acee 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 000000000000..0df89359466b 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 000000000000..635297805184 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 000000000000..0a0f74c94403 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 000000000000..fcb13666a0ad 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 000000000000..cf1d68e2acee 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 000000000000..4e033f8786aa 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 000000000000..635297805184 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 000000000000..eb2b6be4e5e5 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 000000000000..cf1d68e2acee 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 000000000000..7b6e9c175b8c 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 000000000000..635297805184 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 000000000000..1aee7033161e 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 000000000000..9c1e3021c3ea --- /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 000000000000..54698e5f8afa --- /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 000000000000..04523a6882fd 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 000000000000..321a56f4d370 --- /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 000000000000..cf1d68e2acee 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 000000000000..635297805184 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 000000000000..4d339e472ac2 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 000000000000..bf902e50cf26 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 000000000000..7029bc3ccdf2 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 000000000000..610e2c0250d4 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 000000000000..cf1d68e2acee 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 000000000000..635297805184 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 000000000000..cf1d68e2acee 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 000000000000..635297805184 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 000000000000..cf1d68e2acee 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 000000000000..635297805184 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 000000000000..1aee7033161e 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 000000000000..9c1e3021c3ea --- /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 000000000000..fa78985cb877 --- /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 000000000000..04523a6882fd 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 000000000000..321a56f4d370 --- /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 000000000000..cf1d68e2acee 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 000000000000..635297805184 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 000000000000..4d339e472ac2 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 000000000000..bf902e50cf26 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 000000000000..421f95ae9dd8 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 000000000000..2639d3211dec 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 000000000000..cf1d68e2acee 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 000000000000..635297805184 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 000000000000..cf1d68e2acee 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 000000000000..635297805184 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 000000000000..cf1d68e2acee 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 000000000000..635297805184 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 000000000000..ba56986ebd21 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 000000000000..ba56986ebd21 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 000000000000..00b8a64995dd --- /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 000000000000..00b8a64995dd --- /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 000000000000..879dac88e351 --- /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 000000000000..0d6e0a477850 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 000000000000..24dcb52ef609 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 000000000000..6f149ed4ec45 --- /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 000000000000..4a6194c2002b --- /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 000000000000..cf1d68e2acee 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 000000000000..cf1d68e2acee 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 000000000000..635297805184 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 000000000000..635297805184 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 000000000000..3f3804f1999c 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 000000000000..871586884066 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 000000000000..9e684a6792e5 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 000000000000..cf1d68e2acee 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 000000000000..73c35f68c2f2 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 000000000000..635297805184 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 000000000000..cf1d68e2acee 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 000000000000..cf1d68e2acee 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 000000000000..635297805184 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 000000000000..635297805184 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 000000000000..816cff99cd15 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 000000000000..cf1d68e2acee 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 000000000000..3c6d389f0426 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 000000000000..635297805184 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 000000000000..cf1d68e2acee 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 000000000000..cf1d68e2acee 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 000000000000..635297805184 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 000000000000..635297805184 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 000000000000..1aee7033161e 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 000000000000..1aee7033161e 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 000000000000..9c1e3021c3ea --- /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 000000000000..9c1e3021c3ea --- /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 000000000000..0831489d9d02 --- /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 000000000000..b1cf4d310b24 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 000000000000..cf958a5259df 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 000000000000..523d0ce69165 --- /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 000000000000..f69d320e37d2 --- /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 000000000000..cf1d68e2acee 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 000000000000..cf1d68e2acee 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 000000000000..635297805184 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 000000000000..635297805184 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 000000000000..701a0a87ad48 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 000000000000..08ee320ef242 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 000000000000..f712e4290ad3 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 000000000000..2a9f3595f24b 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 000000000000..f5faf01f4dc5 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 000000000000..ec3f1af46bd4 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 000000000000..cf1d68e2acee 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 000000000000..cf1d68e2acee 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 000000000000..635297805184 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 000000000000..635297805184 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 000000000000..cf1d68e2acee 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 000000000000..cf1d68e2acee 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 000000000000..635297805184 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 000000000000..635297805184 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 000000000000..cf1d68e2acee 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 000000000000..3ffbb7a9133b 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 000000000000..635297805184 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 000000000000..7c8834f659bd 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 f0533f89b63e..be0559721fb3 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 @@ -742,7 +742,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession assert( executedPlan.exists { case WholeStageCodegenExec( - HashAggregateExec(_, _, _, _, _, _, _: LocalTableScanExec)) => true + HashAggregateExec(_, _, _, _, _, _, _, _, _: LocalTableScanExec)) => true case _ => false }, "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 000000000000..f1578ae5df97 --- /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.{StatefulOpClusteredDistributionTestHelper, StreamManualClock} +import org.apache.spark.util.Utils + +class FlatMapGroupsWithStateDistributionSuite extends StreamTest + with StatefulOpClusteredDistributionTestHelper { + + import testImplicits._ + + test("SPARK-38204: flatMapGroupsWithState should require StatefulOpClusteredDistribution " + + "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(requireStatefulOpClusteredDistribution( + 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 StatefulOpClusteredDistribution " + + "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(requireStatefulOpClusteredDistribution( + 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 000000000000..615434f2edad --- /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.StatefulOpClusteredDistributionTestHelper +import org.apache.spark.util.Utils + +class StreamingAggregationDistributionSuite extends StreamTest + with StatefulOpClusteredDistributionTestHelper with Assertions { + + import testImplicits._ + + test("SPARK-38204: streaming aggregation should require StatefulOpClusteredDistribution " + + "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(requireStatefulOpClusteredDistribution(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 a183e6b4e395..64dffe7f571a 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 000000000000..8dbdb3620688 --- /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.StatefulOpClusteredDistributionTestHelper +import org.apache.spark.util.Utils + +class StreamingDeduplicationDistributionSuite extends StreamTest + with StatefulOpClusteredDistributionTestHelper { + + import testImplicits._ + + test("SPARK-38204: streaming deduplication should require StatefulOpClusteredDistribution " + + "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(requireStatefulOpClusteredDistribution( + 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 000000000000..bb7b9804105f --- /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.StatefulOpClusteredDistributionTestHelper +import org.apache.spark.util.Utils + +class StreamingSessionWindowDistributionSuite extends StreamTest + with StatefulOpClusteredDistributionTestHelper with Logging { + + import testImplicits._ + + test("SPARK-38204: session window aggregation should require StatefulOpClusteredDistribution " + + "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(requireStatefulOpClusteredDistribution(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/StatefulOpClusteredDistributionTestHelper.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/StatefulOpClusteredDistributionTestHelper.scala new file mode 100644 index 000000000000..f2684b8c39cd --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/StatefulOpClusteredDistributionTestHelper.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, HashPartitioning, StatefulOpClusteredDistribution} +import org.apache.spark.sql.execution.SparkPlan + +trait StatefulOpClusteredDistributionTestHelper 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 requireStatefulOpClusteredDistribution( + plan: SparkPlan, + desiredClusterColumns: Seq[Seq[String]], + desiredNumPartitions: Int): Boolean = { + assert(plan.requiredChildDistribution.length === desiredClusterColumns.length) + plan.requiredChildDistribution.zip(desiredClusterColumns).forall { + case (d: StatefulOpClusteredDistribution, clusterColumns: Seq[String]) + if partitionExpressionsColumns(d.expressions) == clusterColumns && + d._requiredNumPartitions == 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) + } + } +}