/
EnsureStatefulOpPartitioningSuite.scala
132 lines (119 loc) · 5.33 KB
/
EnsureStatefulOpPartitioningSuite.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
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()
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
}