/
DataFrameCallbackSuite.scala
277 lines (232 loc) · 10.8 KB
/
DataFrameCallbackSuite.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
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
/*
* 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.util
import scala.collection.mutable.ArrayBuffer
import org.apache.spark._
import org.apache.spark.sql.{functions, AnalysisException, QueryTest, Row}
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, InsertIntoStatement, LogicalPlan, Project}
import org.apache.spark.sql.execution.{QueryExecution, WholeStageCodegenExec}
import org.apache.spark.sql.execution.datasources.{CreateTable, InsertIntoHadoopFsRelationCommand}
import org.apache.spark.sql.execution.datasources.json.JsonFileFormat
import org.apache.spark.sql.test.SharedSparkSession
class DataFrameCallbackSuite extends QueryTest with SharedSparkSession {
import testImplicits._
import functions._
test("execute callback functions when a DataFrame action finished successfully") {
val metrics = ArrayBuffer.empty[(String, QueryExecution, Long)]
val listener = new QueryExecutionListener {
// Only test successful case here, so no need to implement `onFailure`
override def onFailure(funcName: String, qe: QueryExecution, error: Throwable): Unit = {}
override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = {
metrics += ((funcName, qe, duration))
}
}
spark.listenerManager.register(listener)
val df = Seq(1 -> "a").toDF("i", "j")
df.select("i").collect()
df.filter($"i" > 0).count()
sparkContext.listenerBus.waitUntilEmpty()
assert(metrics.length == 2)
assert(metrics(0)._1 == "collect")
assert(metrics(0)._2.analyzed.isInstanceOf[Project])
assert(metrics(0)._3 > 0)
assert(metrics(1)._1 == "count")
assert(metrics(1)._2.analyzed.isInstanceOf[Aggregate])
assert(metrics(1)._3 > 0)
spark.listenerManager.unregister(listener)
}
testQuietly("execute callback functions when a DataFrame action failed") {
val metrics = ArrayBuffer.empty[(String, QueryExecution, Throwable)]
val listener = new QueryExecutionListener {
override def onFailure(funcName: String, qe: QueryExecution, error: Throwable): Unit = {
metrics += ((funcName, qe, error))
}
// Only test failed case here, so no need to implement `onSuccess`
override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = {}
}
spark.listenerManager.register(listener)
val errorUdf = udf[Int, Int] { _ => throw new RuntimeException("udf error") }
val df = sparkContext.makeRDD(Seq(1 -> "a")).toDF("i", "j")
val e = intercept[SparkException](df.select(errorUdf($"i")).collect())
sparkContext.listenerBus.waitUntilEmpty()
assert(metrics.length == 1)
assert(metrics(0)._1 == "collect")
assert(metrics(0)._2.analyzed.isInstanceOf[Project])
assert(metrics(0)._3.getMessage == e.getMessage)
spark.listenerManager.unregister(listener)
}
test("get numRows metrics by callback") {
val metrics = ArrayBuffer.empty[Long]
val listener = new QueryExecutionListener {
// Only test successful case here, so no need to implement `onFailure`
override def onFailure(funcName: String, qe: QueryExecution, error: Throwable): Unit = {}
override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = {
val metric = qe.executedPlan match {
case w: WholeStageCodegenExec => w.child.longMetric("numOutputRows")
case other => other.longMetric("numOutputRows")
}
metrics += metric.value
}
}
spark.listenerManager.register(listener)
val df = Seq(1 -> "a").toDF("i", "j").groupBy("i").count()
df.collect()
// Wait for the first `collect` to be caught by our listener. Otherwise the next `collect` will
// reset the plan metrics.
sparkContext.listenerBus.waitUntilEmpty()
df.collect()
Seq(1 -> "a", 2 -> "a").toDF("i", "j").groupBy("i").count().collect()
sparkContext.listenerBus.waitUntilEmpty()
assert(metrics.length == 3)
assert(metrics(0) === 1)
assert(metrics(1) === 1)
assert(metrics(2) === 2)
spark.listenerManager.unregister(listener)
}
// TODO: Currently some LongSQLMetric use -1 as initial value, so if the accumulator is never
// updated, we can filter it out later. However, when we aggregate(sum) accumulator values at
// driver side for SQL physical operators, these -1 values will make our result smaller.
// A easy fix is to create a new SQLMetric(including new MetricValue, MetricParam, etc.), but we
// can do it later because the impact is just too small (1048576 tasks for 1 MB).
ignore("get size metrics by callback") {
val metrics = ArrayBuffer.empty[Long]
val listener = new QueryExecutionListener {
// Only test successful case here, so no need to implement `onFailure`
override def onFailure(funcName: String, qe: QueryExecution, error: Throwable): Unit = {}
override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = {
metrics += qe.executedPlan.longMetric("dataSize").value
val bottomAgg = qe.executedPlan.children(0).children(0)
metrics += bottomAgg.longMetric("dataSize").value
}
}
spark.listenerManager.register(listener)
val sparkListener = new SaveInfoListener
spark.sparkContext.addSparkListener(sparkListener)
val df = (1 to 100).map(i => i -> i.toString).toDF("i", "j")
df.groupBy("i").count().collect()
def getPeakExecutionMemory(stageId: Int): Long = {
val peakMemoryAccumulator = sparkListener.getCompletedStageInfos(stageId).accumulables
.filter(_._2.name == Some(InternalAccumulator.PEAK_EXECUTION_MEMORY))
assert(peakMemoryAccumulator.size == 1)
peakMemoryAccumulator.head._2.value.get.asInstanceOf[Long]
}
assert(sparkListener.getCompletedStageInfos.length == 2)
val bottomAggDataSize = getPeakExecutionMemory(0)
val topAggDataSize = getPeakExecutionMemory(1)
// For this simple case, the peakExecutionMemory of a stage should be the data size of the
// aggregate operator, as we only have one memory consuming operator per stage.
sparkContext.listenerBus.waitUntilEmpty()
assert(metrics.length == 2)
assert(metrics(0) == topAggDataSize)
assert(metrics(1) == bottomAggDataSize)
spark.listenerManager.unregister(listener)
}
test("execute callback functions for DataFrameWriter") {
val commands = ArrayBuffer.empty[(String, LogicalPlan)]
val errors = ArrayBuffer.empty[(String, Throwable)]
val listener = new QueryExecutionListener {
override def onFailure(funcName: String, qe: QueryExecution, error: Throwable): Unit = {
errors += funcName -> error
}
override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = {
commands += funcName -> qe.logical
}
}
spark.listenerManager.register(listener)
withTempPath { path =>
spark.range(10).write.format("json").save(path.getCanonicalPath)
sparkContext.listenerBus.waitUntilEmpty()
assert(commands.length == 1)
assert(commands.head._1 == "save")
assert(commands.head._2.isInstanceOf[InsertIntoHadoopFsRelationCommand])
assert(commands.head._2.asInstanceOf[InsertIntoHadoopFsRelationCommand]
.fileFormat.isInstanceOf[JsonFileFormat])
}
withTable("tab") {
sql("CREATE TABLE tab(i long) using parquet") // adds commands(1) via onSuccess
spark.range(10).write.insertInto("tab")
sparkContext.listenerBus.waitUntilEmpty()
assert(commands.length == 3)
assert(commands(2)._1 == "insertInto")
assert(commands(2)._2.isInstanceOf[InsertIntoStatement])
assert(commands(2)._2.asInstanceOf[InsertIntoStatement].table
.asInstanceOf[UnresolvedRelation].multipartIdentifier == Seq("tab"))
}
// exiting withTable adds commands(3) via onSuccess (drops tab)
withTable("tab") {
spark.range(10).select($"id", $"id" % 5 as "p").write.partitionBy("p").saveAsTable("tab")
sparkContext.listenerBus.waitUntilEmpty()
assert(commands.length == 5)
assert(commands(4)._1 == "saveAsTable")
assert(commands(4)._2.isInstanceOf[CreateTable])
assert(commands(4)._2.asInstanceOf[CreateTable].tableDesc.partitionColumnNames == Seq("p"))
}
withTable("tab") {
sql("CREATE TABLE tab(i long) using parquet")
val e = intercept[AnalysisException] {
spark.range(10).select($"id", $"id").write.insertInto("tab")
}
sparkContext.listenerBus.waitUntilEmpty()
assert(errors.length == 1)
assert(errors.head._1 == "insertInto")
assert(errors.head._2 == e)
}
}
test("get observable metrics by callback") {
val metricMaps = ArrayBuffer.empty[Map[String, Row]]
val listener = new QueryExecutionListener {
override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = {
metricMaps += qe.observedMetrics
}
override def onFailure(funcName: String, qe: QueryExecution, exception: Throwable): Unit = {
// No-op
}
}
spark.listenerManager.register(listener)
try {
val df = spark.range(100)
.observe(
name = "my_event",
min($"id").as("min_val"),
max($"id").as("max_val"),
sum($"id").as("sum_val"),
count(when($"id" % 2 === 0, 1)).as("num_even"))
.observe(
name = "other_event",
avg($"id").cast("int").as("avg_val"))
def checkMetrics(metrics: Map[String, Row]): Unit = {
assert(metrics.size === 2)
assert(metrics("my_event") === Row(0L, 99L, 4950L, 50L))
assert(metrics("other_event") === Row(49))
}
// First run
df.collect()
sparkContext.listenerBus.waitUntilEmpty()
assert(metricMaps.size === 1)
checkMetrics(metricMaps.head)
metricMaps.clear()
// Second run should produce the same result as the first run.
df.collect()
sparkContext.listenerBus.waitUntilEmpty()
assert(metricMaps.size === 1)
checkMetrics(metricMaps.head)
} finally {
spark.listenerManager.unregister(listener)
}
}
}