-
Notifications
You must be signed in to change notification settings - Fork 414
Expand file tree
/
Copy pathBenchmark.scala
More file actions
566 lines (490 loc) · 18.2 KB
/
Benchmark.scala
File metadata and controls
566 lines (490 loc) · 18.2 KB
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
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
/*
* Copyright 2015 Databricks Inc.
*
* Licensed 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 com.databricks.spark.sql.perf
import scala.collection.mutable.ArrayBuffer
import scala.concurrent.ExecutionContext.Implicits.global
import scala.concurrent._
import scala.concurrent.duration._
import scala.language.implicitConversions
import scala.util.{Success, Try, Failure => SFailure}
import scala.util.control.NonFatal
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{Dataset, DataFrame, SQLContext, SparkSession}
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.SparkContext
import com.databricks.spark.sql.perf.cpu._
/**
* A collection of queries that test a particular aspect of Spark SQL.
*
* @param sqlContext An existing SQLContext.
*/
abstract class Benchmark(
@transient val sqlContext: SQLContext)
extends Serializable {
import Benchmark._
def this() = this(SparkSession.builder.getOrCreate().sqlContext)
val resultsLocation =
sqlContext.getAllConfs.getOrElse(
"spark.sql.perf.results",
"/spark/sql/performance")
protected def sparkContext = sqlContext.sparkContext
protected implicit def toOption[A](a: A): Option[A] = Option(a)
val buildInfo = Try(getClass.getClassLoader.loadClass("org.apache.spark.BuildInfo")).map { cls =>
cls.getMethods
.filter(_.getReturnType == classOf[String])
.filterNot(_.getName == "toString")
.map(m => m.getName -> m.invoke(cls).asInstanceOf[String])
.toMap
}.getOrElse(Map.empty)
def currentConfiguration = BenchmarkConfiguration(
sqlConf = sqlContext.getAllConfs,
sparkConf = sparkContext.getConf.getAll.toMap,
defaultParallelism = sparkContext.defaultParallelism,
buildInfo = buildInfo)
val codegen = Variation("codegen", Seq("on", "off")) {
case "off" => sqlContext.setConf("spark.sql.codegen", "false")
case "on" => sqlContext.setConf("spark.sql.codegen", "true")
}
val unsafe = Variation("unsafe", Seq("on", "off")) {
case "off" => sqlContext.setConf("spark.sql.unsafe.enabled", "false")
case "on" => sqlContext.setConf("spark.sql.unsafe.enabled", "true")
}
val tungsten = Variation("tungsten", Seq("on", "off")) {
case "off" => sqlContext.setConf("spark.sql.tungsten.enabled", "false")
case "on" => sqlContext.setConf("spark.sql.tungsten.enabled", "true")
}
/**
* Starts an experiment run with a given set of executions to run.
*
* @param executionsToRun a list of executions to run.
* @param includeBreakdown If it is true, breakdown results of an execution will be recorded.
* Setting it to true may significantly increase the time used to
* run an execution.
* @param iterations The number of iterations to run of each execution.
* @param variations [[Variation]]s used in this run. The cross product of all variations will be
* run for each execution * iteration.
* @param tags Tags of this run.
* @param timeout wait at most timeout milliseconds for each query, 0 means wait forever
* @return It returns a ExperimentStatus object that can be used to
* track the progress of this experiment run.
*/
def runExperiment(
executionsToRun: Seq[Benchmarkable],
includeBreakdown: Boolean = false,
iterations: Int = 3,
variations: Seq[Variation[_]] = Seq(Variation("StandardRun", Seq("true")) { _ => {} }),
tags: Map[String, String] = Map.empty,
timeout: Long = 0L,
resultLocation: String = resultsLocation,
forkThread: Boolean = true,
prewarmQueryPlanning: Boolean = false) = {
new ExperimentStatus(
executionsToRun,
includeBreakdown,
iterations,
variations,
tags,
timeout,
resultLocation,
sqlContext,
allTables,
currentConfiguration,
forkThread = forkThread,
prewarmQueryPlanning = prewarmQueryPlanning)
}
import reflect.runtime._, universe._
import reflect.runtime._
import universe._
@transient
private val runtimeMirror = universe.runtimeMirror(getClass.getClassLoader)
@transient
val myType = runtimeMirror.classSymbol(getClass).toType
def singleTables =
myType.declarations
.filter(m => m.isMethod)
.map(_.asMethod)
.filter(_.asMethod.returnType =:= typeOf[Table])
.map(method => runtimeMirror.reflect(this).reflectMethod(method).apply().asInstanceOf[Table])
def groupedTables =
myType.declarations
.filter(m => m.isMethod)
.map(_.asMethod)
.filter(_.asMethod.returnType =:= typeOf[Seq[Table]])
.flatMap(method => runtimeMirror.reflect(this).reflectMethod(method).apply().asInstanceOf[Seq[Table]])
@transient
lazy val allTables: Seq[Table] = (singleTables ++ groupedTables).toSeq
def singleQueries =
myType.declarations
.filter(m => m.isMethod)
.map(_.asMethod)
.filter(_.asMethod.returnType =:= typeOf[Benchmarkable])
.map(method => runtimeMirror.reflect(this).reflectMethod(method).apply().asInstanceOf[Benchmarkable])
def groupedQueries =
myType.declarations
.filter(m => m.isMethod)
.map(_.asMethod)
.filter(_.asMethod.returnType =:= typeOf[Seq[Benchmarkable]])
.flatMap(method => runtimeMirror.reflect(this).reflectMethod(method).apply().asInstanceOf[Seq[Benchmarkable]])
@transient
lazy val allQueries = (singleQueries ++ groupedQueries).toSeq
def html: String = {
val singleQueries =
myType.declarations
.filter(m => m.isMethod)
.map(_.asMethod)
.filter(_.asMethod.returnType =:= typeOf[Query])
.map(method => runtimeMirror.reflect(this).reflectMethod(method).apply().asInstanceOf[Query])
.mkString(",")
val queries =
myType.declarations
.filter(m => m.isMethod)
.map(_.asMethod)
.filter(_.asMethod.returnType =:= typeOf[Seq[Query]])
.map { method =>
val queries = runtimeMirror.reflect(this).reflectMethod(method).apply().asInstanceOf[Seq[Query]]
val queryList = queries.map(_.name).mkString(", ")
s"""
|<h3>${method.name}</h3>
|<ul>$queryList</ul>
""".stripMargin
}.mkString("\n")
s"""
|<h1>Spark SQL Performance Benchmarking</h1>
|<h2>Available Queries</h2>
|$singleQueries
|$queries
""".stripMargin
}
/** Factory object for benchmark queries. */
case object Query {
def apply(
name: String,
sqlText: String,
description: String,
executionMode: ExecutionMode = ExecutionMode.ForeachResults): Query = {
new Query(name, sqlContext.sql(sqlText), description, Some(sqlText), executionMode)
}
def apply(
name: String,
dataFrameBuilder: => DataFrame,
description: String): Query = {
new Query(name, dataFrameBuilder, description, None, ExecutionMode.CollectResults)
}
}
object RDDCount {
def apply(
name: String,
rdd: RDD[_]) = {
new SparkPerfExecution(
name,
Map.empty,
() => Unit,
() => rdd.count(),
rdd.toDebugString)
}
}
/** A class for benchmarking Spark perf results. */
class SparkPerfExecution(
override val name: String,
parameters: Map[String, String],
prepare: () => Unit,
run: () => Unit,
description: String = "")
extends Benchmarkable {
override def toString: String =
s"""
|== $name ==
|$description
""".stripMargin
protected override val executionMode: ExecutionMode = ExecutionMode.SparkPerfResults
protected override def beforeBenchmark(): Unit = { prepare() }
protected override def doBenchmark(
includeBreakdown: Boolean,
description: String = "",
messages: ArrayBuffer[String],
listener: Option[BenchmarkableListener]): BenchmarkResult = {
try {
val timeMs = measureTimeMs(run())
BenchmarkResult(
name = name,
mode = executionMode.toString,
parameters = parameters,
executionTime = Some(timeMs))
} catch {
case e: Exception =>
BenchmarkResult(
name = name,
mode = executionMode.toString,
parameters = parameters,
failure = Some(Failure(e.getClass.getSimpleName, e.getMessage)))
}
}
}
}
/**
* A Variation represents a setting (e.g. the number of shuffle partitions or if tables
* are cached in memory) that we want to change in a experiment run.
* A Variation has three parts, `name`, `options`, and `setup`.
* The `name` is the identifier of a Variation. `options` is a Seq of options that
* will be used for a query. Basically, a query will be executed with every option
* defined in the list of `options`. `setup` defines the needed action for every
* option. For example, the following Variation is used to change the number of shuffle
* partitions of a query. The name of the Variation is "shufflePartitions". There are
* two options, 200 and 2000. The setup is used to set the value of property
* "spark.sql.shuffle.partitions".
*
* {{{
* Variation("shufflePartitions", Seq("200", "2000")) {
* case num => sqlContext.setConf("spark.sql.shuffle.partitions", num)
* }
* }}}
*/
case class Variation[T](name: String, options: Seq[T])(val setup: T => Unit)
case class Table(
name: String,
data: Dataset[_])
object Benchmark {
class ExperimentStatus(
executionsToRun: Seq[Benchmarkable],
includeBreakdown: Boolean,
iterations: Int,
variations: Seq[Variation[_]],
tags: Map[String, String],
timeout: Long,
resultsLocation: String,
sqlContext: SQLContext,
allTables: Seq[Table],
currentConfiguration: BenchmarkConfiguration,
forkThread: Boolean = true,
prewarmQueryPlanning: Boolean = false) {
val currentResults = new collection.mutable.ArrayBuffer[BenchmarkResult]()
val currentRuns = new collection.mutable.ArrayBuffer[ExperimentRun]()
val currentMessages = new collection.mutable.ArrayBuffer[String]()
def logMessage(msg: String) = {
println(msg)
currentMessages += msg
}
// Stats for HTML status message.
@volatile var currentExecution = ""
@volatile var currentPlan = "" // for queries only
@volatile var currentConfig = ""
@volatile var failures = 0
@volatile var startTime = 0L
/** An optional log collection task that will run after the experiment. */
@volatile var logCollection: () => Unit = () => {}
def cartesianProduct[T](xss: List[List[T]]): List[List[T]] = xss match {
case Nil => List(Nil)
case h :: t => for(xh <- h; xt <- cartesianProduct(t)) yield xh :: xt
}
val timestamp = System.currentTimeMillis()
val resultPath = s"$resultsLocation/timestamp=$timestamp"
val combinations = cartesianProduct(variations.map(l => (0 until l.options.size).toList).toList)
val resultsFuture = Future {
// If we're running queries, create tables for them
executionsToRun
.collect { case query: Query => query }
.flatMap { query =>
try {
query.newDataFrame().queryExecution.logical.collect {
case r: UnresolvedRelation => r.tableName
}
} catch {
// ignore the queries that can't be parsed
case e: Exception => Seq()
}
}
.distinct
.foreach { name =>
try {
sqlContext.table(name)
logMessage(s"Table $name exists.")
} catch {
case ae: Exception =>
val table = allTables
.find(_.name == name)
if (table.isDefined) {
logMessage(s"Creating table: $name")
table.get.data
.write
.mode("overwrite")
.saveAsTable(name)
} else {
// the table could be subquery
logMessage(s"Couldn't read table $name and its not defined as a Benchmark.Table.")
}
}
}
// Run the benchmarks!
val results: Seq[ExperimentRun] = (1 to iterations).flatMap { i =>
combinations.map { setup =>
val currentOptions = variations.asInstanceOf[Seq[Variation[Any]]].zip(setup).map {
case (v, idx) =>
v.setup(v.options(idx))
v.name -> v.options(idx).toString
}
currentConfig = currentOptions.map { case (k,v) => s"$k: $v" }.mkString(", ")
val res = executionsToRun.flatMap { q =>
val setup = s"iteration: $i, ${currentOptions.map { case (k, v) => s"$k=$v"}.mkString(", ")}"
logMessage(s"Running execution ${q.name} $setup")
currentExecution = q.name
currentPlan = "" // Clear previous plan if any
prewarmQueryPlanningIfEnabled(q)
startTime = System.currentTimeMillis()
val singleResultT = Try {
q.benchmark(includeBreakdown, setup, currentMessages, timeout,
forkThread=forkThread, listener = Some(queryListener))
}
singleResultT match {
case Success(singleResult) =>
singleResult.failure.foreach { f =>
failures += 1
logMessage(s"Execution '${q.name}' failed: ${f.message}")
}
singleResult.executionTime.foreach { time =>
logMessage(s"Execution time: ${time / 1000}s")
}
currentResults += singleResult
singleResult :: Nil
case SFailure(e) =>
failures += 1
logMessage(s"Execution '${q.name}' failed: ${e}")
Nil
}
}
val result = ExperimentRun(
timestamp = timestamp,
iteration = i,
tags = currentOptions.toMap ++ tags,
configuration = currentConfiguration,
res)
currentRuns += result
result
}
}
try {
val resultsTable = sqlContext.createDataFrame(results)
logMessage(s"Results written to table: 'sqlPerformance' at $resultPath")
resultsTable
.coalesce(1)
.write
.format("json")
.save(resultPath)
} catch {
case NonFatal(e) =>
logMessage(s"Failed to write data: $e")
throw e
}
logCollection()
}
private def prewarmQueryPlanningIfEnabled(benchmarkable: Benchmarkable): Unit = {
if (prewarmQueryPlanning) {
benchmarkable match {
case query: Query => query.newDataFrame().queryExecution.executedPlan
case _ =>
}
}
}
private val queryListener = new BenchmarkableListener {
override def onQueryPlanned(plan: SparkPlan): Unit = {
currentPlan = try {
plan.toString
} catch {
case NonFatal(e) => s"failed to parse: $e"
}
}
}
def scheduleCpuCollection(fs: FS) = {
logCollection = () => {
logMessage(s"Begining CPU log collection")
try {
val location = cpu.collectLogs(sqlContext, fs, timestamp)
logMessage(s"cpu results recorded to $location")
} catch {
case NonFatal(e) =>
logMessage(s"Error collecting logs: $e")
throw e
}
}
}
def cpuProfile = new Profile(sqlContext, sqlContext.read.json(getCpuLocation(timestamp)))
def cpuProfileHtml(fs: FS) = {
s"""
|<h1>CPU Profile</h1>
|<b>Permalink:</b> <tt>sqlContext.read.json("${getCpuLocation(timestamp)}")</tt></br>
|${cpuProfile.buildGraph(fs)}
""".stripMargin
}
/** Waits for the finish of the experiment. */
def waitForFinish(timeoutInSeconds: Int) = {
Await.result(resultsFuture, timeoutInSeconds.seconds)
}
/** Returns results from an actively running experiment. */
def getCurrentResults() = {
val tbl = sqlContext.createDataFrame(currentResults)
tbl.createOrReplaceTempView("currentResults")
tbl
}
/** Returns full iterations from an actively running experiment. */
def getCurrentRuns() = {
val tbl = sqlContext.createDataFrame(currentRuns)
tbl.createOrReplaceTempView("currentRuns")
tbl
}
def tail(n: Int = 20) = {
currentMessages.takeRight(n).mkString("\n")
}
def status =
if (resultsFuture.isCompleted) {
if (resultsFuture.value.get.isFailure) "Failed" else "Successful"
} else {
"Running"
}
override def toString =
s"""Permalink: table("sqlPerformance").where('timestamp === ${timestamp}L)"""
def html: String = {
val maybeQueryPlan: String =
if (currentPlan.nonEmpty) {
s"""
|<h3>QueryPlan</h3>
|<pre>
|${currentPlan.replaceAll("\n", "<br/>")}
|</pre>
""".stripMargin
} else {
""
}
s"""
|<h2>$status Experiment</h2>
|<b>Permalink:</b> <tt>sqlContext.read.json("$resultPath")</tt><br/>
|<b>Iterations complete:</b> ${currentRuns.size / combinations.size} / $iterations<br/>
|<b>Failures:</b> $failures<br/>
|<b>Executions run:</b> ${currentResults.size} / ${iterations * combinations.size * executionsToRun.size}
|<br/>
|<b>Run time:</b> ${(System.currentTimeMillis() - timestamp) / 1000}s<br/>
|
|<h2>Current Execution: $currentExecution</h2>
|Runtime: ${(System.currentTimeMillis() - startTime) / 1000}s<br/>
|$currentConfig<br/>
|$maybeQueryPlan
|<h2>Logs</h2>
|<pre>
|${tail()}
|</pre>
""".stripMargin
}
}
}