Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-21977] SinglePartition optimizations break certain Streaming S…
…tateful Aggregation requirements ## What changes were proposed in this pull request? This is a bit hard to explain as there are several issues here, I'll try my best. Here are the requirements: 1. A StructuredStreaming Source that can generate empty RDDs with 0 partitions 2. A StructuredStreaming query that uses the above source, performs a stateful aggregation (mapGroupsWithState, groupBy.count, ...), and coalesce's by 1 The crux of the problem is that when a dataset has a `coalesce(1)` call, it receives a `SinglePartition` partitioning scheme. This scheme satisfies most required distributions used for aggregations such as HashAggregateExec. This causes a world of problems: Symptom 1. If the input RDD has 0 partitions, the whole lineage will receive 0 partitions, nothing will be executed, the state store will not create any delta files. When this happens, the next trigger fails, because the StateStore fails to load the delta file for the previous trigger Symptom 2. Let's say that there was data. Then in this case, if you stop your stream, and change `coalesce(1)` with `coalesce(2)`, then restart your stream, your stream will fail, because `spark.sql.shuffle.partitions - 1` number of StateStores will fail to find its delta files. To fix the issues above, we must check that the partitioning of the child of a `StatefulOperator` satisfies: If the grouping expressions are empty: a) AllTuple distribution b) Single physical partition If the grouping expressions are non empty: a) Clustered distribution b) spark.sql.shuffle.partition # of partitions whether or not `coalesce(1)` exists in the plan, and whether or not the input RDD for the trigger has any data. Once you fix the above problem by adding an Exchange to the plan, you come across the following bug: If you call `coalesce(1).groupBy().count()` on a Streaming DataFrame, and if you have a trigger with no data, `StateStoreRestoreExec` doesn't return the prior state. However, for this specific aggregation, `HashAggregateExec` after the restore returns a (0, 0) row, since we're performing a count, and there is no data. Then this data gets stored in `StateStoreSaveExec` causing the previous counts to be overwritten and lost. ## How was this patch tested? Regression tests Author: Burak Yavuz <brkyvz@gmail.com> Closes #19196 from brkyvz/sa-0.
- Loading branch information
Showing
6 changed files
with
395 additions
and
21 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
132 changes: 132 additions & 0 deletions
132
...ore/src/test/scala/org/apache/spark/sql/streaming/EnsureStatefulOpPartitioningSuite.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,132 @@ | ||
/* | ||
* 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.util.UUID | ||
|
||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute | ||
import org.apache.spark.sql.catalyst.expressions.Attribute | ||
import org.apache.spark.sql.catalyst.plans.physical._ | ||
import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest, UnaryExecNode} | ||
import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchange} | ||
import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata, StatefulOperator, StatefulOperatorStateInfo} | ||
import org.apache.spark.sql.test.SharedSQLContext | ||
|
||
class EnsureStatefulOpPartitioningSuite extends SparkPlanTest with SharedSQLContext { | ||
|
||
import testImplicits._ | ||
super.beforeAll() | ||
This comment has been minimized.
Sorry, something went wrong. |
||
|
||
private val baseDf = Seq((1, "A"), (2, "b")).toDF("num", "char") | ||
|
||
testEnsureStatefulOpPartitioning( | ||
"ClusteredDistribution generates Exchange with HashPartitioning", | ||
baseDf.queryExecution.sparkPlan, | ||
requiredDistribution = keys => ClusteredDistribution(keys), | ||
expectedPartitioning = | ||
keys => HashPartitioning(keys, spark.sessionState.conf.numShufflePartitions), | ||
expectShuffle = true) | ||
|
||
testEnsureStatefulOpPartitioning( | ||
"ClusteredDistribution with coalesce(1) generates Exchange with HashPartitioning", | ||
baseDf.coalesce(1).queryExecution.sparkPlan, | ||
requiredDistribution = keys => ClusteredDistribution(keys), | ||
expectedPartitioning = | ||
keys => HashPartitioning(keys, spark.sessionState.conf.numShufflePartitions), | ||
expectShuffle = true) | ||
|
||
testEnsureStatefulOpPartitioning( | ||
"AllTuples generates Exchange with SinglePartition", | ||
baseDf.queryExecution.sparkPlan, | ||
requiredDistribution = _ => AllTuples, | ||
expectedPartitioning = _ => SinglePartition, | ||
expectShuffle = true) | ||
|
||
testEnsureStatefulOpPartitioning( | ||
"AllTuples with coalesce(1) doesn't need Exchange", | ||
baseDf.coalesce(1).queryExecution.sparkPlan, | ||
requiredDistribution = _ => AllTuples, | ||
expectedPartitioning = _ => SinglePartition, | ||
expectShuffle = false) | ||
|
||
/** | ||
* For `StatefulOperator` with the given `requiredChildDistribution`, and child SparkPlan | ||
* `inputPlan`, ensures that the incremental planner adds exchanges, if required, in order to | ||
* ensure the expected partitioning. | ||
*/ | ||
private def testEnsureStatefulOpPartitioning( | ||
testName: String, | ||
inputPlan: SparkPlan, | ||
requiredDistribution: Seq[Attribute] => Distribution, | ||
expectedPartitioning: Seq[Attribute] => Partitioning, | ||
expectShuffle: Boolean): Unit = { | ||
test(testName) { | ||
val operator = TestStatefulOperator(inputPlan, requiredDistribution(inputPlan.output.take(1))) | ||
val executed = executePlan(operator, OutputMode.Complete()) | ||
if (expectShuffle) { | ||
val exchange = executed.children.find(_.isInstanceOf[Exchange]) | ||
if (exchange.isEmpty) { | ||
fail(s"Was expecting an exchange but didn't get one in:\n$executed") | ||
} | ||
assert(exchange.get === | ||
ShuffleExchange(expectedPartitioning(inputPlan.output.take(1)), inputPlan), | ||
s"Exchange didn't have expected properties:\n${exchange.get}") | ||
} else { | ||
assert(!executed.children.exists(_.isInstanceOf[Exchange]), | ||
s"Unexpected exchange found in:\n$executed") | ||
} | ||
} | ||
} | ||
|
||
/** Executes a SparkPlan using the IncrementalPlanner used for Structured Streaming. */ | ||
private def executePlan( | ||
p: SparkPlan, | ||
outputMode: OutputMode = OutputMode.Append()): SparkPlan = { | ||
val execution = new IncrementalExecution( | ||
spark, | ||
null, | ||
OutputMode.Complete(), | ||
"chk", | ||
UUID.randomUUID(), | ||
0L, | ||
OffsetSeqMetadata()) { | ||
override lazy val sparkPlan: SparkPlan = p transform { | ||
case plan: SparkPlan => | ||
val inputMap = plan.children.flatMap(_.output).map(a => (a.name, a)).toMap | ||
plan transformExpressions { | ||
case UnresolvedAttribute(Seq(u)) => | ||
inputMap.getOrElse(u, | ||
sys.error(s"Invalid Test: Cannot resolve $u given input $inputMap")) | ||
} | ||
} | ||
} | ||
execution.executedPlan | ||
} | ||
} | ||
|
||
/** Used to emulate a `StatefulOperator` with the given requiredDistribution. */ | ||
case class TestStatefulOperator( | ||
child: SparkPlan, | ||
requiredDist: Distribution) extends UnaryExecNode with StatefulOperator { | ||
override def output: Seq[Attribute] = child.output | ||
override def doExecute(): RDD[InternalRow] = child.execute() | ||
override def requiredChildDistribution: Seq[Distribution] = requiredDist :: Nil | ||
override def stateInfo: Option[StatefulOperatorStateInfo] = None | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
@brkyvz this test is actually failing consistently in master -- it's actually manually calling beforeAll and tests in the constructor. I have a fix I can submit