From 649f2965188efcfa0b1d2b5acb4c0f057ecd3788 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 20 Sep 2018 15:23:46 +0800 Subject: [PATCH 01/10] Refactor AggregateBenchmark --- .../benchmarks/AggregateBenchmark-results.txt | 154 +++ .../benchmark/AggregateBenchmark.scala | 908 ++++++++---------- 2 files changed, 563 insertions(+), 499 deletions(-) create mode 100644 sql/core/benchmarks/AggregateBenchmark-results.txt diff --git a/sql/core/benchmarks/AggregateBenchmark-results.txt b/sql/core/benchmarks/AggregateBenchmark-results.txt new file mode 100644 index 000000000000..11df00c680ee --- /dev/null +++ b/sql/core/benchmarks/AggregateBenchmark-results.txt @@ -0,0 +1,154 @@ +================================================================================================ +aggregate without grouping +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 +Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz + +agg w/o group: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +agg w/o group wholestage off 39650 / 46049 52.9 18.9 1.0X +agg w/o group wholestage on 1224 / 1413 1713.5 0.6 32.4X + + +================================================================================================ +stat functions +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 +Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz + +stddev: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +stddev wholestage off 6149 / 6366 17.1 58.6 1.0X +stddev wholestage on 871 / 881 120.4 8.3 7.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 +Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz + +kurtosis: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +kurtosis wholestage off 28822 / 29231 3.6 274.9 1.0X +kurtosis wholestage on 929 / 944 112.9 8.9 31.0X + + +================================================================================================ +aggregate with linear keys +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 +Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz + +Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +codegen = F 7956 / 7967 10.5 94.8 1.0X +codegen = T hashmap = F 3872 / 4049 21.7 46.2 2.1X +codegen = T hashmap = T 872 / 883 96.3 10.4 9.1X + + +================================================================================================ +aggregate with randomized keys +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 +Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz + +Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +codegen = F 9088 / 9240 9.2 108.3 1.0X +codegen = T hashmap = F 5065 / 5238 16.6 60.4 1.8X +codegen = T hashmap = T 1722 / 1768 48.7 20.5 5.3X + + +================================================================================================ +aggregate with string key +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 +Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz + +Aggregate w string key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +codegen = F 3666 / 3704 5.7 174.8 1.0X +codegen = T hashmap = F 2322 / 2357 9.0 110.7 1.6X +codegen = T hashmap = T 1643 / 1676 12.8 78.3 2.2X + + +================================================================================================ +aggregate with decimal key +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 +Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz + +Aggregate w decimal key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +codegen = F 2688 / 2704 7.8 128.2 1.0X +codegen = T hashmap = F 1401 / 1430 15.0 66.8 1.9X +codegen = T hashmap = T 394 / 415 53.2 18.8 6.8X + + +================================================================================================ +aggregate with multiple key types +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 +Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz + +Aggregate w multiple keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +codegen = F 5380 / 5437 3.9 256.5 1.0X +codegen = T hashmap = F 3554 / 3648 5.9 169.5 1.5X +codegen = T hashmap = T 2687 / 2719 7.8 128.1 2.0X + + +================================================================================================ +max function bytecode size of wholestagecodegen +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 +Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz + +max function bytecode size: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +codegen = F 375 / 416 1.7 572.0 1.0X +codegen = T hugeMethodLimit = 10000 231 / 245 2.8 352.0 1.6X +codegen = T hugeMethodLimit = 1500 383 / 412 1.7 583.7 1.0X + + +================================================================================================ +cube +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 +Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz + +cube: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +cube wholestage off 2250 / 2266 2.3 429.1 1.0X +cube wholestage on 907 / 945 5.8 173.0 2.5X + + +================================================================================================ +hash and BytesToBytesMap +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 +Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz + +BytesToBytesMap: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +UnsafeRowhash 205 / 215 102.4 9.8 1.0X +murmur3 hash 104 / 111 202.0 4.9 2.0X +fast hash 55 / 60 381.2 2.6 3.7X +arrayEqual 132 / 139 158.9 6.3 1.6X +Java HashMap (Long) 89 / 103 235.9 4.2 2.3X +Java HashMap (two ints) 91 / 107 229.2 4.4 2.2X +Java HashMap (UnsafeRow) 759 / 772 27.6 36.2 0.3X +LongToUnsafeRowMap (opt=false) 384 / 406 54.7 18.3 0.5X +LongToUnsafeRowMap (opt=true) 82 / 88 256.5 3.9 2.5X +BytesToBytesMap (off Heap) 753 / 811 27.8 35.9 0.3X +BytesToBytesMap (on Heap) 765 / 784 27.4 36.5 0.3X +Aggregate HashMap 35 / 39 591.4 1.7 5.8X + + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 8f4ee8533e59..37aa4b36ea9c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -22,6 +22,7 @@ import java.util.HashMap import org.apache.spark.SparkConf import org.apache.spark.internal.config._ import org.apache.spark.memory.{StaticMemoryManager, TaskMemoryManager} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.execution.joins.LongToUnsafeRowMap import org.apache.spark.sql.execution.vectorized.AggregateHashMap @@ -30,625 +31,534 @@ import org.apache.spark.sql.types.{LongType, StructType} import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.hash.Murmur3_x86_32 import org.apache.spark.unsafe.map.BytesToBytesMap -import org.apache.spark.util.Benchmark +import org.apache.spark.util.{Benchmark, BenchmarkBase => FileBenchmarkBase} /** * Benchmark to measure performance for aggregate primitives. - * To run this: - * build/sbt "sql/test-only *benchmark.AggregateBenchmark" - * - * Benchmarks in this file are skipped in normal builds. + * To run this benchmark: + * 1. without sbt: bin/spark-submit --class + * 2. build/sbt "sql/test:runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " + * Results will be written to "benchmarks/AggregateBenchmark-results.txt". */ -class AggregateBenchmark extends BenchmarkBase { +object AggregateBenchmark extends FileBenchmarkBase { - ignore("aggregate without grouping") { - val N = 500L << 22 - val benchmark = new Benchmark("agg without grouping", N) - runBenchmark("agg w/o group", N) { - sparkSession.range(N).selectExpr("sum(id)").collect() - } - /* - agg w/o group: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - agg w/o group wholestage off 30136 / 31885 69.6 14.4 1.0X - agg w/o group wholestage on 1851 / 1860 1132.9 0.9 16.3X - */ - } + lazy val sparkSession = SparkSession.builder + .master("local[1]") + .appName(this.getClass.getSimpleName) + .config("spark.sql.shuffle.partitions", 1) + .config("spark.sql.autoBroadcastJoinThreshold", 1) + .getOrCreate() - ignore("stat functions") { - val N = 100L << 20 + /** Runs function `f` with whole stage codegen on and off. */ + def runBenchmark(name: String, cardinality: Long)(f: => Unit): Unit = { + val benchmark = new Benchmark(name, cardinality, output = output) - runBenchmark("stddev", N) { - sparkSession.range(N).groupBy().agg("id" -> "stddev").collect() + benchmark.addCase(s"$name wholestage off", numIters = 2) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) + f } - runBenchmark("kurtosis", N) { - sparkSession.range(N).groupBy().agg("id" -> "kurtosis").collect() + benchmark.addCase(s"$name wholestage on", numIters = 5) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) + f } - /* - Using ImperativeAggregate (as implemented in Spark 1.6): - - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - stddev: Avg Time(ms) Avg Rate(M/s) Relative Rate - ------------------------------------------------------------------------------- - stddev w/o codegen 2019.04 10.39 1.00 X - stddev w codegen 2097.29 10.00 0.96 X - kurtosis w/o codegen 2108.99 9.94 0.96 X - kurtosis w codegen 2090.69 10.03 0.97 X - - Using DeclarativeAggregate: - - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - stddev: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - stddev codegen=false 5630 / 5776 18.0 55.6 1.0X - stddev codegen=true 1259 / 1314 83.0 12.0 4.5X - - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - kurtosis: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - kurtosis codegen=false 14847 / 15084 7.0 142.9 1.0X - kurtosis codegen=true 1652 / 2124 63.0 15.9 9.0X - */ + benchmark.run() } - ignore("aggregate with linear keys") { - val N = 20 << 22 - - val benchmark = new Benchmark("Aggregate w keys", N) - def f(): Unit = { - sparkSession.range(N).selectExpr("(id & 65535) as k").groupBy("k").sum().collect() + override def benchmark(): Unit = { + runBenchmark("aggregate without grouping") { + val N = 500L << 22 + runBenchmark("agg w/o group", N) { + sparkSession.range(N).selectExpr("sum(id)").collect() + } } - benchmark.addCase(s"codegen = F", numIters = 2) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") - f() - } + runBenchmark("stat functions") { + val N = 100L << 20 - benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") - f() - } + runBenchmark("stddev", N) { + sparkSession.range(N).groupBy().agg("id" -> "stddev").collect() + } - benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") - f() + runBenchmark("kurtosis", N) { + sparkSession.range(N).groupBy().agg("id" -> "kurtosis").collect() + } } - benchmark.run() + runBenchmark("aggregate with linear keys") { + val N = 20 << 22 - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - codegen = F 6619 / 6780 12.7 78.9 1.0X - codegen = T hashmap = F 3935 / 4059 21.3 46.9 1.7X - codegen = T hashmap = T 897 / 971 93.5 10.7 7.4X - */ - } + val benchmark = new Benchmark("Aggregate w keys", N, output = output) - ignore("aggregate with randomized keys") { - val N = 20 << 22 + def f(): Unit = { + sparkSession.range(N).selectExpr("(id & 65535) as k").groupBy("k").sum().collect() + } - val benchmark = new Benchmark("Aggregate w keys", N) - sparkSession.range(N).selectExpr("id", "floor(rand() * 10000) as k") - .createOrReplaceTempView("test") + benchmark.addCase(s"codegen = F", numIters = 2) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") + f() + } - def f(): Unit = sparkSession.sql("select k, k, sum(id) from test group by k, k").collect() + benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") + f() + } - benchmark.addCase(s"codegen = F", numIters = 2) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) - f() - } + benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + f() + } - benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") - f() + benchmark.run() } - benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") - f() - } + runBenchmark("aggregate with randomized keys") { + val N = 20 << 22 - benchmark.run() + val benchmark = new Benchmark("Aggregate w keys", N, output = output) + sparkSession.range(N).selectExpr("id", "floor(rand() * 10000) as k") + .createOrReplaceTempView("test") - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Mac OS X 10.11 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz + def f(): Unit = sparkSession.sql("select k, k, sum(id) from test group by k, k").collect() - Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - codegen = F 7445 / 7517 11.3 88.7 1.0X - codegen = T hashmap = F 4672 / 4703 18.0 55.7 1.6X - codegen = T hashmap = T 1764 / 1958 47.6 21.0 4.2X - */ - } + benchmark.addCase(s"codegen = F", numIters = 2) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) + f() + } - ignore("aggregate with string key") { - val N = 20 << 20 + benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") + f() + } - val benchmark = new Benchmark("Aggregate w string key", N) - def f(): Unit = sparkSession.range(N).selectExpr("id", "cast(id & 1023 as string) as k") - .groupBy("k").count().collect() + benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + f() + } - benchmark.addCase(s"codegen = F", numIters = 2) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") - f() + benchmark.run() } - benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") - f() - } + runBenchmark("aggregate with string key") { + val N = 20 << 20 - benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") - f() - } + val benchmark = new Benchmark("Aggregate w string key", N, output = output) - benchmark.run() + def f(): Unit = sparkSession.range(N).selectExpr("id", "cast(id & 1023 as string) as k") + .groupBy("k").count().collect() - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - Aggregate w string key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - codegen = F 3307 / 3376 6.3 157.7 1.0X - codegen = T hashmap = F 2364 / 2471 8.9 112.7 1.4X - codegen = T hashmap = T 1740 / 1841 12.0 83.0 1.9X - */ - } + benchmark.addCase(s"codegen = F", numIters = 2) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") + f() + } - ignore("aggregate with decimal key") { - val N = 20 << 20 + benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") + f() + } - val benchmark = new Benchmark("Aggregate w decimal key", N) - def f(): Unit = sparkSession.range(N).selectExpr("id", "cast(id & 65535 as decimal) as k") - .groupBy("k").count().collect() + benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + f() + } - benchmark.addCase(s"codegen = F") { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") - f() + benchmark.run() } - benchmark.addCase(s"codegen = T hashmap = F") { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") - f() - } + runBenchmark("aggregate with decimal key") { + val N = 20 << 20 - benchmark.addCase(s"codegen = T hashmap = T") { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") - f() - } + val benchmark = new Benchmark("Aggregate w decimal key", N, output = output) - benchmark.run() + def f(): Unit = sparkSession.range(N).selectExpr("id", "cast(id & 65535 as decimal) as k") + .groupBy("k").count().collect() - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - Aggregate w decimal key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - codegen = F 2756 / 2817 7.6 131.4 1.0X - codegen = T hashmap = F 1580 / 1647 13.3 75.4 1.7X - codegen = T hashmap = T 641 / 662 32.7 30.6 4.3X - */ - } + benchmark.addCase(s"codegen = F") { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") + f() + } - ignore("aggregate with multiple key types") { - val N = 20 << 20 - - val benchmark = new Benchmark("Aggregate w multiple keys", N) - def f(): Unit = sparkSession.range(N) - .selectExpr( - "id", - "(id & 1023) as k1", - "cast(id & 1023 as string) as k2", - "cast(id & 1023 as int) as k3", - "cast(id & 1023 as double) as k4", - "cast(id & 1023 as float) as k5", - "id > 1023 as k6") - .groupBy("k1", "k2", "k3", "k4", "k5", "k6") - .sum() - .collect() - - benchmark.addCase(s"codegen = F") { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") - f() - } + benchmark.addCase(s"codegen = T hashmap = F") { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") + f() + } - benchmark.addCase(s"codegen = T hashmap = F") { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") - f() - } + benchmark.addCase(s"codegen = T hashmap = T") { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + f() + } - benchmark.addCase(s"codegen = T hashmap = T") { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") - f() + benchmark.run() } - benchmark.run() + runBenchmark("aggregate with multiple key types") { + val N = 20 << 20 - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_73-b02 on Mac OS X 10.11.4 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - Aggregate w decimal key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - codegen = F 5885 / 6091 3.6 280.6 1.0X - codegen = T hashmap = F 3625 / 4009 5.8 172.8 1.6X - codegen = T hashmap = T 3204 / 3271 6.5 152.8 1.8X - */ - } + val benchmark = new Benchmark("Aggregate w multiple keys", N, output = output) - ignore("max function bytecode size of wholestagecodegen") { - val N = 20 << 15 - - val benchmark = new Benchmark("max function bytecode size", N) - def f(): Unit = sparkSession.range(N) - .selectExpr( - "id", - "(id & 1023) as k1", - "cast(id & 1023 as double) as k2", - "cast(id & 1023 as int) as k3", - "case when id > 100 and id <= 200 then 1 else 0 end as v1", - "case when id > 200 and id <= 300 then 1 else 0 end as v2", - "case when id > 300 and id <= 400 then 1 else 0 end as v3", - "case when id > 400 and id <= 500 then 1 else 0 end as v4", - "case when id > 500 and id <= 600 then 1 else 0 end as v5", - "case when id > 600 and id <= 700 then 1 else 0 end as v6", - "case when id > 700 and id <= 800 then 1 else 0 end as v7", - "case when id > 800 and id <= 900 then 1 else 0 end as v8", - "case when id > 900 and id <= 1000 then 1 else 0 end as v9", - "case when id > 1000 and id <= 1100 then 1 else 0 end as v10", - "case when id > 1100 and id <= 1200 then 1 else 0 end as v11", - "case when id > 1200 and id <= 1300 then 1 else 0 end as v12", - "case when id > 1300 and id <= 1400 then 1 else 0 end as v13", - "case when id > 1400 and id <= 1500 then 1 else 0 end as v14", - "case when id > 1500 and id <= 1600 then 1 else 0 end as v15", - "case when id > 1600 and id <= 1700 then 1 else 0 end as v16", - "case when id > 1700 and id <= 1800 then 1 else 0 end as v17", - "case when id > 1800 and id <= 1900 then 1 else 0 end as v18") - .groupBy("k1", "k2", "k3") - .sum() - .collect() - - benchmark.addCase("codegen = F") { iter => - sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "false") - f() - } - - benchmark.addCase("codegen = T hugeMethodLimit = 10000") { iter => - sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") - sparkSession.conf.set(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key, "10000") - f() - } + def f(): Unit = sparkSession.range(N) + .selectExpr( + "id", + "(id & 1023) as k1", + "cast(id & 1023 as string) as k2", + "cast(id & 1023 as int) as k3", + "cast(id & 1023 as double) as k4", + "cast(id & 1023 as float) as k5", + "id > 1023 as k6") + .groupBy("k1", "k2", "k3", "k4", "k5", "k6") + .sum() + .collect() - benchmark.addCase("codegen = T hugeMethodLimit = 1500") { iter => - sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") - sparkSession.conf.set(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key, "1500") - f() - } + benchmark.addCase(s"codegen = F") { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") + f() + } - benchmark.run() + benchmark.addCase(s"codegen = T hashmap = F") { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") + f() + } - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_31-b13 on Mac OS X 10.10.2 - Intel(R) Core(TM) i7-4578U CPU @ 3.00GHz + benchmark.addCase(s"codegen = T hashmap = T") { iter => + sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") + sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + f() + } - max function bytecode size: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - codegen = F 709 / 803 0.9 1082.1 1.0X - codegen = T hugeMethodLimit = 10000 3485 / 3548 0.2 5317.7 0.2X - codegen = T hugeMethodLimit = 1500 636 / 701 1.0 969.9 1.1X - */ - } + benchmark.run() + } + + runBenchmark("max function bytecode size of wholestagecodegen") { + val N = 20 << 15 + + val benchmark = new Benchmark("max function bytecode size", N, output = output) + + def f(): Unit = sparkSession.range(N) + .selectExpr( + "id", + "(id & 1023) as k1", + "cast(id & 1023 as double) as k2", + "cast(id & 1023 as int) as k3", + "case when id > 100 and id <= 200 then 1 else 0 end as v1", + "case when id > 200 and id <= 300 then 1 else 0 end as v2", + "case when id > 300 and id <= 400 then 1 else 0 end as v3", + "case when id > 400 and id <= 500 then 1 else 0 end as v4", + "case when id > 500 and id <= 600 then 1 else 0 end as v5", + "case when id > 600 and id <= 700 then 1 else 0 end as v6", + "case when id > 700 and id <= 800 then 1 else 0 end as v7", + "case when id > 800 and id <= 900 then 1 else 0 end as v8", + "case when id > 900 and id <= 1000 then 1 else 0 end as v9", + "case when id > 1000 and id <= 1100 then 1 else 0 end as v10", + "case when id > 1100 and id <= 1200 then 1 else 0 end as v11", + "case when id > 1200 and id <= 1300 then 1 else 0 end as v12", + "case when id > 1300 and id <= 1400 then 1 else 0 end as v13", + "case when id > 1400 and id <= 1500 then 1 else 0 end as v14", + "case when id > 1500 and id <= 1600 then 1 else 0 end as v15", + "case when id > 1600 and id <= 1700 then 1 else 0 end as v16", + "case when id > 1700 and id <= 1800 then 1 else 0 end as v17", + "case when id > 1800 and id <= 1900 then 1 else 0 end as v18") + .groupBy("k1", "k2", "k3") + .sum() + .collect() + + benchmark.addCase("codegen = F") { iter => + sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "false") + f() + } + benchmark.addCase("codegen = T hugeMethodLimit = 10000") { iter => + sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + sparkSession.conf.set(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key, "10000") + f() + } - ignore("cube") { - val N = 5 << 20 + benchmark.addCase("codegen = T hugeMethodLimit = 1500") { iter => + sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + sparkSession.conf.set(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key, "1500") + f() + } - runBenchmark("cube", N) { - sparkSession.range(N).selectExpr("id", "id % 1000 as k1", "id & 256 as k2") - .cube("k1", "k2").sum("id").collect() + benchmark.run() } - /** - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - cube: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - cube codegen=false 3188 / 3392 1.6 608.2 1.0X - cube codegen=true 1239 / 1394 4.2 236.3 2.6X - */ - } - ignore("hash and BytesToBytesMap") { - val N = 20 << 20 + runBenchmark("cube") { + val N = 5 << 20 - val benchmark = new Benchmark("BytesToBytesMap", N) - - benchmark.addCase("UnsafeRowhash") { iter => - var i = 0 - val keyBytes = new Array[Byte](16) - val key = new UnsafeRow(1) - key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) - var s = 0 - while (i < N) { - key.setInt(0, i % 1000) - val h = Murmur3_x86_32.hashUnsafeWords( - key.getBaseObject, key.getBaseOffset, key.getSizeInBytes, 42) - s += h - i += 1 + runBenchmark("cube", N) { + sparkSession.range(N).selectExpr("id", "id % 1000 as k1", "id & 256 as k2") + .cube("k1", "k2").sum("id").collect() } } - benchmark.addCase("murmur3 hash") { iter => - var i = 0 - val keyBytes = new Array[Byte](16) - val key = new UnsafeRow(1) - key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) - var p = 524283 - var s = 0 - while (i < N) { - var h = Murmur3_x86_32.hashLong(i, 42) - key.setInt(0, h) - s += h - i += 1 - } - } + runBenchmark("hash and BytesToBytesMap") { + val N = 20 << 20 - benchmark.addCase("fast hash") { iter => - var i = 0 - val keyBytes = new Array[Byte](16) - val key = new UnsafeRow(1) - key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) - var p = 524283 - var s = 0 - while (i < N) { - var h = i % p - if (h < 0) { - h += p + val benchmark = new Benchmark("BytesToBytesMap", N, output = output) + + benchmark.addCase("UnsafeRowhash") { iter => + var i = 0 + val keyBytes = new Array[Byte](16) + val key = new UnsafeRow(1) + key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) + var s = 0 + while (i < N) { + key.setInt(0, i % 1000) + val h = Murmur3_x86_32.hashUnsafeWords( + key.getBaseObject, key.getBaseOffset, key.getSizeInBytes, 42) + s += h + i += 1 } - key.setInt(0, h) - s += h - i += 1 } - } - benchmark.addCase("arrayEqual") { iter => - var i = 0 - val keyBytes = new Array[Byte](16) - val valueBytes = new Array[Byte](16) - val key = new UnsafeRow(1) - key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) - val value = new UnsafeRow(1) - value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) - value.setInt(0, 555) - var s = 0 - while (i < N) { - key.setInt(0, i % 1000) - if (key.equals(value)) { - s += 1 + benchmark.addCase("murmur3 hash") { iter => + var i = 0 + val keyBytes = new Array[Byte](16) + val key = new UnsafeRow(1) + key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) + var p = 524283 + var s = 0 + while (i < N) { + var h = Murmur3_x86_32.hashLong(i, 42) + key.setInt(0, h) + s += h + i += 1 } - i += 1 } - } - benchmark.addCase("Java HashMap (Long)") { iter => - var i = 0 - val keyBytes = new Array[Byte](16) - val valueBytes = new Array[Byte](16) - val value = new UnsafeRow(1) - value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) - value.setInt(0, 555) - val map = new HashMap[Long, UnsafeRow]() - while (i < 65536) { - value.setInt(0, i) - map.put(i.toLong, value) - i += 1 - } - var s = 0 - i = 0 - while (i < N) { - if (map.get(i % 100000) != null) { - s += 1 + benchmark.addCase("fast hash") { iter => + var i = 0 + val keyBytes = new Array[Byte](16) + val key = new UnsafeRow(1) + key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) + var p = 524283 + var s = 0 + while (i < N) { + var h = i % p + if (h < 0) { + h += p + } + key.setInt(0, h) + s += h + i += 1 } - i += 1 } - } - benchmark.addCase("Java HashMap (two ints) ") { iter => - var i = 0 - val valueBytes = new Array[Byte](16) - val value = new UnsafeRow(1) - value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) - value.setInt(0, 555) - val map = new HashMap[Long, UnsafeRow]() - while (i < 65536) { - value.setInt(0, i) - val key = (i.toLong << 32) + Integer.rotateRight(i, 15) - map.put(key, value) - i += 1 - } - var s = 0 - i = 0 - while (i < N) { - val key = ((i & 100000).toLong << 32) + Integer.rotateRight(i & 100000, 15) - if (map.get(key) != null) { - s += 1 + benchmark.addCase("arrayEqual") { iter => + var i = 0 + val keyBytes = new Array[Byte](16) + val valueBytes = new Array[Byte](16) + val key = new UnsafeRow(1) + key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) + val value = new UnsafeRow(1) + value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) + value.setInt(0, 555) + var s = 0 + while (i < N) { + key.setInt(0, i % 1000) + if (key.equals(value)) { + s += 1 + } + i += 1 } - i += 1 } - } - benchmark.addCase("Java HashMap (UnsafeRow)") { iter => - var i = 0 - val keyBytes = new Array[Byte](16) - val valueBytes = new Array[Byte](16) - val key = new UnsafeRow(1) - key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) - val value = new UnsafeRow(1) - value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) - value.setInt(0, 555) - val map = new HashMap[UnsafeRow, UnsafeRow]() - while (i < 65536) { - key.setInt(0, i) - value.setInt(0, i) - map.put(key, value.copy()) - i += 1 - } - var s = 0 - i = 0 - while (i < N) { - key.setInt(0, i % 100000) - if (map.get(key) != null) { - s += 1 + benchmark.addCase("Java HashMap (Long)") { iter => + var i = 0 + val keyBytes = new Array[Byte](16) + val valueBytes = new Array[Byte](16) + val value = new UnsafeRow(1) + value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) + value.setInt(0, 555) + val map = new HashMap[Long, UnsafeRow]() + while (i < 65536) { + value.setInt(0, i) + map.put(i.toLong, value) + i += 1 + } + var s = 0 + i = 0 + while (i < N) { + if (map.get(i % 100000) != null) { + s += 1 + } + i += 1 } - i += 1 } - } - Seq(false, true).foreach { optimized => - benchmark.addCase(s"LongToUnsafeRowMap (opt=$optimized)") { iter => + benchmark.addCase("Java HashMap (two ints) ") { iter => var i = 0 val valueBytes = new Array[Byte](16) val value = new UnsafeRow(1) value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) value.setInt(0, 555) - val taskMemoryManager = new TaskMemoryManager( - new StaticMemoryManager( - new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), - Long.MaxValue, - Long.MaxValue, - 1), - 0) - val map = new LongToUnsafeRowMap(taskMemoryManager, 64) + val map = new HashMap[Long, UnsafeRow]() while (i < 65536) { value.setInt(0, i) - val key = i % 100000 - map.append(key, value) + val key = (i.toLong << 32) + Integer.rotateRight(i, 15) + map.put(key, value) i += 1 } - if (optimized) { - map.optimize() - } var s = 0 i = 0 while (i < N) { - val key = i % 100000 - if (map.getValue(key, value) != null) { + val key = ((i & 100000).toLong << 32) + Integer.rotateRight(i & 100000, 15) + if (map.get(key) != null) { s += 1 } i += 1 } } - } - Seq("off", "on").foreach { heap => - benchmark.addCase(s"BytesToBytesMap ($heap Heap)") { iter => - val taskMemoryManager = new TaskMemoryManager( - new StaticMemoryManager( - new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, s"${heap == "off"}") - .set(MEMORY_OFFHEAP_SIZE.key, "102400000"), - Long.MaxValue, - Long.MaxValue, - 1), - 0) - val map = new BytesToBytesMap(taskMemoryManager, 1024, 64L<<20) + benchmark.addCase("Java HashMap (UnsafeRow)") { iter => + var i = 0 val keyBytes = new Array[Byte](16) val valueBytes = new Array[Byte](16) val key = new UnsafeRow(1) key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) val value = new UnsafeRow(1) value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) - var i = 0 - val numKeys = 65536 - while (i < numKeys) { - key.setInt(0, i % 65536) - val loc = map.lookup(key.getBaseObject, key.getBaseOffset, key.getSizeInBytes, - Murmur3_x86_32.hashLong(i % 65536, 42)) - if (!loc.isDefined) { - loc.append(key.getBaseObject, key.getBaseOffset, key.getSizeInBytes, - value.getBaseObject, value.getBaseOffset, value.getSizeInBytes) - } + value.setInt(0, 555) + val map = new HashMap[UnsafeRow, UnsafeRow]() + while (i < 65536) { + key.setInt(0, i) + value.setInt(0, i) + map.put(key, value.copy()) i += 1 } - i = 0 var s = 0 + i = 0 while (i < N) { key.setInt(0, i % 100000) - val loc = map.lookup(key.getBaseObject, key.getBaseOffset, key.getSizeInBytes, - Murmur3_x86_32.hashLong(i % 100000, 42)) - if (loc.isDefined) { + if (map.get(key) != null) { s += 1 } i += 1 } } - } - benchmark.addCase("Aggregate HashMap") { iter => - var i = 0 - val numKeys = 65536 - val schema = new StructType() - .add("key", LongType) - .add("value", LongType) - val map = new AggregateHashMap(schema) - while (i < numKeys) { - val row = map.findOrInsert(i.toLong) - row.setLong(1, row.getLong(1) + 1) - i += 1 - } - var s = 0 - i = 0 - while (i < N) { - if (map.find(i % 100000) != -1) { - s += 1 + Seq(false, true).foreach { optimized => + benchmark.addCase(s"LongToUnsafeRowMap (opt=$optimized)") { iter => + var i = 0 + val valueBytes = new Array[Byte](16) + val value = new UnsafeRow(1) + value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) + value.setInt(0, 555) + val taskMemoryManager = new TaskMemoryManager( + new StaticMemoryManager( + new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), + Long.MaxValue, + Long.MaxValue, + 1), + 0) + val map = new LongToUnsafeRowMap(taskMemoryManager, 64) + while (i < 65536) { + value.setInt(0, i) + val key = i % 100000 + map.append(key, value) + i += 1 + } + if (optimized) { + map.optimize() + } + var s = 0 + i = 0 + while (i < N) { + val key = i % 100000 + if (map.getValue(key, value) != null) { + s += 1 + } + i += 1 + } } - i += 1 } - } - /* - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - BytesToBytesMap: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - UnsafeRow hash 267 / 284 78.4 12.8 1.0X - murmur3 hash 102 / 129 205.5 4.9 2.6X - fast hash 79 / 96 263.8 3.8 3.4X - arrayEqual 164 / 172 128.2 7.8 1.6X - Java HashMap (Long) 321 / 399 65.4 15.3 0.8X - Java HashMap (two ints) 328 / 363 63.9 15.7 0.8X - Java HashMap (UnsafeRow) 1140 / 1200 18.4 54.3 0.2X - LongToUnsafeRowMap (opt=false) 378 / 400 55.5 18.0 0.7X - LongToUnsafeRowMap (opt=true) 144 / 152 145.2 6.9 1.9X - BytesToBytesMap (off Heap) 1300 / 1616 16.1 62.0 0.2X - BytesToBytesMap (on Heap) 1165 / 1202 18.0 55.5 0.2X - Aggregate HashMap 121 / 131 173.3 5.8 2.2X - */ - benchmark.run() - } + Seq("off", "on").foreach { heap => + benchmark.addCase(s"BytesToBytesMap ($heap Heap)") { iter => + val taskMemoryManager = new TaskMemoryManager( + new StaticMemoryManager( + new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, s"${heap == "off"}") + .set(MEMORY_OFFHEAP_SIZE.key, "102400000"), + Long.MaxValue, + Long.MaxValue, + 1), + 0) + val map = new BytesToBytesMap(taskMemoryManager, 1024, 64L << 20) + val keyBytes = new Array[Byte](16) + val valueBytes = new Array[Byte](16) + val key = new UnsafeRow(1) + key.pointTo(keyBytes, Platform.BYTE_ARRAY_OFFSET, 16) + val value = new UnsafeRow(1) + value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) + var i = 0 + val numKeys = 65536 + while (i < numKeys) { + key.setInt(0, i % 65536) + val loc = map.lookup(key.getBaseObject, key.getBaseOffset, key.getSizeInBytes, + Murmur3_x86_32.hashLong(i % 65536, 42)) + if (!loc.isDefined) { + loc.append(key.getBaseObject, key.getBaseOffset, key.getSizeInBytes, + value.getBaseObject, value.getBaseOffset, value.getSizeInBytes) + } + i += 1 + } + i = 0 + var s = 0 + while (i < N) { + key.setInt(0, i % 100000) + val loc = map.lookup(key.getBaseObject, key.getBaseOffset, key.getSizeInBytes, + Murmur3_x86_32.hashLong(i % 100000, 42)) + if (loc.isDefined) { + s += 1 + } + i += 1 + } + } + } + benchmark.addCase("Aggregate HashMap") { iter => + var i = 0 + val numKeys = 65536 + val schema = new StructType() + .add("key", LongType) + .add("value", LongType) + val map = new AggregateHashMap(schema) + while (i < numKeys) { + val row = map.findOrInsert(i.toLong) + row.setLong(1, row.getLong(1) + 1) + i += 1 + } + var s = 0 + i = 0 + while (i < N) { + if (map.find(i % 100000) != -1) { + s += 1 + } + i += 1 + } + } + benchmark.run() + } + } } From 42230b6e3edb731eb69b3b8800805805e2234d10 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sat, 22 Sep 2018 23:37:12 +0800 Subject: [PATCH 02/10] Add RunBenchmarkWithCodegen --- .../benchmarks/AggregateBenchmark-results.txt | 76 +++++----- .../benchmark/AggregateBenchmark.scala | 131 +++++++----------- .../benchmark/RunBenchmarkWithCodegen.scala | 58 ++++++++ 3 files changed, 149 insertions(+), 116 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RunBenchmarkWithCodegen.scala diff --git a/sql/core/benchmarks/AggregateBenchmark-results.txt b/sql/core/benchmarks/AggregateBenchmark-results.txt index 11df00c680ee..b0ccef02cb47 100644 --- a/sql/core/benchmarks/AggregateBenchmark-results.txt +++ b/sql/core/benchmarks/AggregateBenchmark-results.txt @@ -7,8 +7,8 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz agg w/o group: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -agg w/o group wholestage off 39650 / 46049 52.9 18.9 1.0X -agg w/o group wholestage on 1224 / 1413 1713.5 0.6 32.4X +agg w/o group wholestage off 40454 / 44424 51.8 19.3 1.0X +agg w/o group wholestage on 907 / 929 2312.8 0.4 44.6X ================================================================================================ @@ -20,16 +20,16 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz stddev: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -stddev wholestage off 6149 / 6366 17.1 58.6 1.0X -stddev wholestage on 871 / 881 120.4 8.3 7.1X +stddev wholestage off 6281 / 6626 16.7 59.9 1.0X +stddev wholestage on 909 / 987 115.4 8.7 6.9X Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz kurtosis: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -kurtosis wholestage off 28822 / 29231 3.6 274.9 1.0X -kurtosis wholestage on 929 / 944 112.9 8.9 31.0X +kurtosis wholestage off 32819 / 33789 3.2 313.0 1.0X +kurtosis wholestage on 988 / 1046 106.1 9.4 33.2X ================================================================================================ @@ -41,9 +41,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 7956 / 7967 10.5 94.8 1.0X -codegen = T hashmap = F 3872 / 4049 21.7 46.2 2.1X -codegen = T hashmap = T 872 / 883 96.3 10.4 9.1X +codegen = F 8490 / 8568 9.9 101.2 1.0X +codegen = T hashmap = F 5985 / 6916 14.0 71.4 1.4X +codegen = T hashmap = T 943 / 977 88.9 11.2 9.0X ================================================================================================ @@ -55,9 +55,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 9088 / 9240 9.2 108.3 1.0X -codegen = T hashmap = F 5065 / 5238 16.6 60.4 1.8X -codegen = T hashmap = T 1722 / 1768 48.7 20.5 5.3X +codegen = F 9509 / 9893 8.8 113.4 1.0X +codegen = T hashmap = F 5381 / 5715 15.6 64.1 1.8X +codegen = T hashmap = T 1792 / 1874 46.8 21.4 5.3X ================================================================================================ @@ -69,9 +69,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz Aggregate w string key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 3666 / 3704 5.7 174.8 1.0X -codegen = T hashmap = F 2322 / 2357 9.0 110.7 1.6X -codegen = T hashmap = T 1643 / 1676 12.8 78.3 2.2X +codegen = F 4372 / 4440 4.8 208.5 1.0X +codegen = T hashmap = F 2850 / 2940 7.4 135.9 1.5X +codegen = T hashmap = T 2033 / 2175 10.3 96.9 2.2X ================================================================================================ @@ -83,9 +83,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz Aggregate w decimal key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 2688 / 2704 7.8 128.2 1.0X -codegen = T hashmap = F 1401 / 1430 15.0 66.8 1.9X -codegen = T hashmap = T 394 / 415 53.2 18.8 6.8X +codegen = F 3819 / 4047 5.5 182.1 1.0X +codegen = T hashmap = F 2080 / 2122 10.1 99.2 1.8X +codegen = T hashmap = T 500 / 523 41.9 23.9 7.6X ================================================================================================ @@ -97,9 +97,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz Aggregate w multiple keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 5380 / 5437 3.9 256.5 1.0X -codegen = T hashmap = F 3554 / 3648 5.9 169.5 1.5X -codegen = T hashmap = T 2687 / 2719 7.8 128.1 2.0X +codegen = F 5815 / 6015 3.6 277.3 1.0X +codegen = T hashmap = F 3934 / 3955 5.3 187.6 1.5X +codegen = T hashmap = T 3196 / 3307 6.6 152.4 1.8X ================================================================================================ @@ -111,9 +111,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz max function bytecode size: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 375 / 416 1.7 572.0 1.0X -codegen = T hugeMethodLimit = 10000 231 / 245 2.8 352.0 1.6X -codegen = T hugeMethodLimit = 1500 383 / 412 1.7 583.7 1.0X +codegen = F 415 / 511 1.6 632.8 1.0X +codegen = T hugeMethodLimit = 10000 246 / 281 2.7 375.5 1.7X +codegen = T hugeMethodLimit = 1500 416 / 472 1.6 635.2 1.0X ================================================================================================ @@ -125,8 +125,8 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz cube: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -cube wholestage off 2250 / 2266 2.3 429.1 1.0X -cube wholestage on 907 / 945 5.8 173.0 2.5X +cube wholestage off 2192 / 2270 2.4 418.1 1.0X +cube wholestage on 910 / 1001 5.8 173.5 2.4X ================================================================================================ @@ -138,17 +138,17 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz BytesToBytesMap: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -UnsafeRowhash 205 / 215 102.4 9.8 1.0X -murmur3 hash 104 / 111 202.0 4.9 2.0X -fast hash 55 / 60 381.2 2.6 3.7X -arrayEqual 132 / 139 158.9 6.3 1.6X -Java HashMap (Long) 89 / 103 235.9 4.2 2.3X -Java HashMap (two ints) 91 / 107 229.2 4.4 2.2X -Java HashMap (UnsafeRow) 759 / 772 27.6 36.2 0.3X -LongToUnsafeRowMap (opt=false) 384 / 406 54.7 18.3 0.5X -LongToUnsafeRowMap (opt=true) 82 / 88 256.5 3.9 2.5X -BytesToBytesMap (off Heap) 753 / 811 27.8 35.9 0.3X -BytesToBytesMap (on Heap) 765 / 784 27.4 36.5 0.3X -Aggregate HashMap 35 / 39 591.4 1.7 5.8X +UnsafeRowhash 218 / 225 96.4 10.4 1.0X +murmur3 hash 114 / 121 183.3 5.5 1.9X +fast hash 59 / 64 356.3 2.8 3.7X +arrayEqual 143 / 151 146.6 6.8 1.5X +Java HashMap (Long) 99 / 127 211.0 4.7 2.2X +Java HashMap (two ints) 101 / 136 207.2 4.8 2.1X +Java HashMap (UnsafeRow) 781 / 938 26.9 37.2 0.3X +LongToUnsafeRowMap (opt=false) 398 / 412 52.7 19.0 0.5X +LongToUnsafeRowMap (opt=true) 88 / 95 238.1 4.2 2.5X +BytesToBytesMap (off Heap) 1091 / 1113 19.2 52.0 0.2X +BytesToBytesMap (on Heap) 792 / 835 26.5 37.8 0.3X +Aggregate HashMap 36 / 41 587.9 1.7 6.1X diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index b9abb1e1a940..8fef8998eaa7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -20,10 +20,9 @@ package org.apache.spark.sql.execution.benchmark import java.util.HashMap import org.apache.spark.SparkConf -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.benchmark.Benchmark import org.apache.spark.internal.config._ import org.apache.spark.memory.{StaticMemoryManager, TaskMemoryManager} -import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.execution.joins.LongToUnsafeRowMap import org.apache.spark.sql.execution.vectorized.AggregateHashMap @@ -43,37 +42,13 @@ import org.apache.spark.unsafe.map.BytesToBytesMap * Results will be written to "benchmarks/AggregateBenchmark-results.txt". * }}} */ -object AggregateBenchmark extends BenchmarkBase { - - lazy val sparkSession = SparkSession.builder - .master("local[1]") - .appName(this.getClass.getSimpleName) - .config("spark.sql.shuffle.partitions", 1) - .config("spark.sql.autoBroadcastJoinThreshold", 1) - .getOrCreate() - - /** Runs function `f` with whole stage codegen on and off. */ - def runBenchmark(name: String, cardinality: Long)(f: => Unit): Unit = { - val benchmark = new Benchmark(name, cardinality, output = output) - - benchmark.addCase(s"$name wholestage off", numIters = 2) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) - f - } - - benchmark.addCase(s"$name wholestage on", numIters = 5) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) - f - } - - benchmark.run() - } +object AggregateBenchmark extends RunBenchmarkWithCodegen { override def benchmark(): Unit = { runBenchmark("aggregate without grouping") { val N = 500L << 22 runBenchmark("agg w/o group", N) { - sparkSession.range(N).selectExpr("sum(id)").collect() + spark.range(N).selectExpr("sum(id)").collect() } } @@ -81,11 +56,11 @@ object AggregateBenchmark extends BenchmarkBase { val N = 100L << 20 runBenchmark("stddev", N) { - sparkSession.range(N).groupBy().agg("id" -> "stddev").collect() + spark.range(N).groupBy().agg("id" -> "stddev").collect() } runBenchmark("kurtosis", N) { - sparkSession.range(N).groupBy().agg("id" -> "kurtosis").collect() + spark.range(N).groupBy().agg("id" -> "kurtosis").collect() } } @@ -95,25 +70,25 @@ object AggregateBenchmark extends BenchmarkBase { val benchmark = new Benchmark("Aggregate w keys", N, output = output) def f(): Unit = { - sparkSession.range(N).selectExpr("(id & 65535) as k").groupBy("k").sum().collect() + spark.range(N).selectExpr("(id & 65535) as k").groupBy("k").sum().collect() } benchmark.addCase(s"codegen = F", numIters = 2) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") + spark.conf.set("spark.sql.codegen.wholeStage", "false") f() } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") + spark.conf.set("spark.sql.codegen.wholeStage", "true") + spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") + spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") f() } benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + spark.conf.set("spark.sql.codegen.wholeStage", "true") + spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") + spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") f() } @@ -124,27 +99,27 @@ object AggregateBenchmark extends BenchmarkBase { val N = 20 << 22 val benchmark = new Benchmark("Aggregate w keys", N, output = output) - sparkSession.range(N).selectExpr("id", "floor(rand() * 10000) as k") + spark.range(N).selectExpr("id", "floor(rand() * 10000) as k") .createOrReplaceTempView("test") - def f(): Unit = sparkSession.sql("select k, k, sum(id) from test group by k, k").collect() + def f(): Unit = spark.sql("select k, k, sum(id) from test group by k, k").collect() benchmark.addCase(s"codegen = F", numIters = 2) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) + spark.conf.set("spark.sql.codegen.wholeStage", value = false) f() } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") + spark.conf.set("spark.sql.codegen.wholeStage", value = true) + spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") + spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") f() } benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + spark.conf.set("spark.sql.codegen.wholeStage", value = true) + spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") + spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") f() } @@ -156,25 +131,25 @@ object AggregateBenchmark extends BenchmarkBase { val benchmark = new Benchmark("Aggregate w string key", N, output = output) - def f(): Unit = sparkSession.range(N).selectExpr("id", "cast(id & 1023 as string) as k") + def f(): Unit = spark.range(N).selectExpr("id", "cast(id & 1023 as string) as k") .groupBy("k").count().collect() benchmark.addCase(s"codegen = F", numIters = 2) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") + spark.conf.set("spark.sql.codegen.wholeStage", "false") f() } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") + spark.conf.set("spark.sql.codegen.wholeStage", "true") + spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") + spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") f() } benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + spark.conf.set("spark.sql.codegen.wholeStage", "true") + spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") + spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") f() } @@ -186,25 +161,25 @@ object AggregateBenchmark extends BenchmarkBase { val benchmark = new Benchmark("Aggregate w decimal key", N, output = output) - def f(): Unit = sparkSession.range(N).selectExpr("id", "cast(id & 65535 as decimal) as k") + def f(): Unit = spark.range(N).selectExpr("id", "cast(id & 65535 as decimal) as k") .groupBy("k").count().collect() benchmark.addCase(s"codegen = F") { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") + spark.conf.set("spark.sql.codegen.wholeStage", "false") f() } benchmark.addCase(s"codegen = T hashmap = F") { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") + spark.conf.set("spark.sql.codegen.wholeStage", "true") + spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") + spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") f() } benchmark.addCase(s"codegen = T hashmap = T") { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + spark.conf.set("spark.sql.codegen.wholeStage", "true") + spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") + spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") f() } @@ -216,7 +191,7 @@ object AggregateBenchmark extends BenchmarkBase { val benchmark = new Benchmark("Aggregate w multiple keys", N, output = output) - def f(): Unit = sparkSession.range(N) + def f(): Unit = spark.range(N) .selectExpr( "id", "(id & 1023) as k1", @@ -230,21 +205,21 @@ object AggregateBenchmark extends BenchmarkBase { .collect() benchmark.addCase(s"codegen = F") { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "false") + spark.conf.set("spark.sql.codegen.wholeStage", "false") f() } benchmark.addCase(s"codegen = T hashmap = F") { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") + spark.conf.set("spark.sql.codegen.wholeStage", "true") + spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") + spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") f() } benchmark.addCase(s"codegen = T hashmap = T") { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + spark.conf.set("spark.sql.codegen.wholeStage", "true") + spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") + spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") f() } @@ -256,7 +231,7 @@ object AggregateBenchmark extends BenchmarkBase { val benchmark = new Benchmark("max function bytecode size", N, output = output) - def f(): Unit = sparkSession.range(N) + def f(): Unit = spark.range(N) .selectExpr( "id", "(id & 1023) as k1", @@ -285,19 +260,19 @@ object AggregateBenchmark extends BenchmarkBase { .collect() benchmark.addCase("codegen = F") { iter => - sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "false") + spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "false") f() } benchmark.addCase("codegen = T hugeMethodLimit = 10000") { iter => - sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") - sparkSession.conf.set(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key, "10000") + spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + spark.conf.set(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key, "10000") f() } benchmark.addCase("codegen = T hugeMethodLimit = 1500") { iter => - sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") - sparkSession.conf.set(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key, "1500") + spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") + spark.conf.set(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key, "1500") f() } @@ -309,7 +284,7 @@ object AggregateBenchmark extends BenchmarkBase { val N = 5 << 20 runBenchmark("cube", N) { - sparkSession.range(N).selectExpr("id", "id % 1000 as k1", "id & 256 as k2") + spark.range(N).selectExpr("id", "id % 1000 as k1", "id & 256 as k2") .cube("k1", "k2").sum("id").collect() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RunBenchmarkWithCodegen.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RunBenchmarkWithCodegen.scala new file mode 100644 index 000000000000..e2d93fef4a7c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RunBenchmarkWithCodegen.scala @@ -0,0 +1,58 @@ +/* + * 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.benchmark + +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.internal.SQLConf + +/** + * Common base trait for micro benchmarks that are supposed to run standalone (i.e. not together + * with other test suites). + */ +trait RunBenchmarkWithCodegen extends BenchmarkBase { + + val spark: SparkSession = getSparkSession + + /** Subclass can override this function to build their own SparkSession */ + def getSparkSession: SparkSession = { + SparkSession.builder() + .master("local[1]") + .appName(this.getClass.getCanonicalName) + .config(SQLConf.SHUFFLE_PARTITIONS.key, 1) + .config(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, 1) + .getOrCreate() + } + + /** Runs function `f` with whole stage codegen on and off. */ + def runBenchmark(name: String, cardinality: Long)(f: => Unit): Unit = { + val benchmark = new Benchmark(name, cardinality, output = output) + + benchmark.addCase(s"$name wholestage off", numIters = 2) { iter => + spark.sqlContext.conf.setConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED, value = false) + f + } + + benchmark.addCase(s"$name wholestage on", numIters = 5) { iter => + spark.sqlContext.conf.setConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED, value = true) + f + } + + benchmark.run() + } +} From 2d778a4c8fb5d3b373856837496882c05ff1d42d Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 24 Sep 2018 13:49:07 +0800 Subject: [PATCH 03/10] Rename RunBenchmarkWithCodegen to SqlBasedBenchmark --- .../sql/execution/benchmark/AggregateBenchmark.scala | 10 +++++----- ...chmarkWithCodegen.scala => SqlBasedBenchmark.scala} | 7 +++---- 2 files changed, 8 insertions(+), 9 deletions(-) rename sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/{RunBenchmarkWithCodegen.scala => SqlBasedBenchmark.scala} (88%) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 8fef8998eaa7..0cfcf988e37d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -42,12 +42,12 @@ import org.apache.spark.unsafe.map.BytesToBytesMap * Results will be written to "benchmarks/AggregateBenchmark-results.txt". * }}} */ -object AggregateBenchmark extends RunBenchmarkWithCodegen { +object AggregateBenchmark extends SqlBasedBenchmark { override def benchmark(): Unit = { runBenchmark("aggregate without grouping") { val N = 500L << 22 - runBenchmark("agg w/o group", N) { + runBenchmarkWithCodegen("agg w/o group", N) { spark.range(N).selectExpr("sum(id)").collect() } } @@ -55,11 +55,11 @@ object AggregateBenchmark extends RunBenchmarkWithCodegen { runBenchmark("stat functions") { val N = 100L << 20 - runBenchmark("stddev", N) { + runBenchmarkWithCodegen("stddev", N) { spark.range(N).groupBy().agg("id" -> "stddev").collect() } - runBenchmark("kurtosis", N) { + runBenchmarkWithCodegen("kurtosis", N) { spark.range(N).groupBy().agg("id" -> "kurtosis").collect() } } @@ -283,7 +283,7 @@ object AggregateBenchmark extends RunBenchmarkWithCodegen { runBenchmark("cube") { val N = 5 << 20 - runBenchmark("cube", N) { + runBenchmarkWithCodegen("cube", N) { spark.range(N).selectExpr("id", "id % 1000 as k1", "id & 256 as k2") .cube("k1", "k2").sum("id").collect() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RunBenchmarkWithCodegen.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala similarity index 88% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RunBenchmarkWithCodegen.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala index e2d93fef4a7c..252911ee9811 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RunBenchmarkWithCodegen.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala @@ -22,10 +22,9 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.internal.SQLConf /** - * Common base trait for micro benchmarks that are supposed to run standalone (i.e. not together - * with other test suites). + * Common base trait to run benchmark with the Dataset and DataFrame API. */ -trait RunBenchmarkWithCodegen extends BenchmarkBase { +trait SqlBasedBenchmark extends BenchmarkBase { val spark: SparkSession = getSparkSession @@ -40,7 +39,7 @@ trait RunBenchmarkWithCodegen extends BenchmarkBase { } /** Runs function `f` with whole stage codegen on and off. */ - def runBenchmark(name: String, cardinality: Long)(f: => Unit): Unit = { + def runBenchmarkWithCodegen(name: String, cardinality: Long)(f: => Unit): Unit = { val benchmark = new Benchmark(name, cardinality, output = output) benchmark.addCase(s"$name wholestage off", numIters = 2) { iter => From 536d4e9bb0cc737ee2f7c887763a4c9ebef147f5 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 25 Sep 2018 07:49:50 +0800 Subject: [PATCH 04/10] Add withSQLConf to SqlBasedBenchmark --- .../benchmarks/AggregateBenchmark-results.txt | 76 +++---- .../benchmark/AggregateBenchmark.scala | 189 ++++++++++-------- .../benchmark/SqlBasedBenchmark.scala | 44 +++- 3 files changed, 179 insertions(+), 130 deletions(-) diff --git a/sql/core/benchmarks/AggregateBenchmark-results.txt b/sql/core/benchmarks/AggregateBenchmark-results.txt index b0ccef02cb47..ac378ffc58fb 100644 --- a/sql/core/benchmarks/AggregateBenchmark-results.txt +++ b/sql/core/benchmarks/AggregateBenchmark-results.txt @@ -7,8 +7,8 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz agg w/o group: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -agg w/o group wholestage off 40454 / 44424 51.8 19.3 1.0X -agg w/o group wholestage on 907 / 929 2312.8 0.4 44.6X +agg w/o group wholestage off 40468 / 44934 51.8 19.3 1.0X +agg w/o group wholestage on 932 / 949 2249.6 0.4 43.4X ================================================================================================ @@ -20,16 +20,16 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz stddev: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -stddev wholestage off 6281 / 6626 16.7 59.9 1.0X -stddev wholestage on 909 / 987 115.4 8.7 6.9X +stddev wholestage off 6664 / 6775 15.7 63.6 1.0X +stddev wholestage on 907 / 923 115.6 8.6 7.3X Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz kurtosis: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -kurtosis wholestage off 32819 / 33789 3.2 313.0 1.0X -kurtosis wholestage on 988 / 1046 106.1 9.4 33.2X +kurtosis wholestage off 31339 / 31357 3.3 298.9 1.0X +kurtosis wholestage on 982 / 997 106.8 9.4 31.9X ================================================================================================ @@ -41,9 +41,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 8490 / 8568 9.9 101.2 1.0X -codegen = T hashmap = F 5985 / 6916 14.0 71.4 1.4X -codegen = T hashmap = T 943 / 977 88.9 11.2 9.0X +codegen = F 9667 / 9936 8.7 115.2 1.0X +codegen = T hashmap = F 4537 / 5353 18.5 54.1 2.1X +codegen = T hashmap = T 924 / 946 90.8 11.0 10.5X ================================================================================================ @@ -55,9 +55,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 9509 / 9893 8.8 113.4 1.0X -codegen = T hashmap = F 5381 / 5715 15.6 64.1 1.8X -codegen = T hashmap = T 1792 / 1874 46.8 21.4 5.3X +codegen = F 9298 / 9300 9.0 110.8 1.0X +codegen = T hashmap = F 5357 / 5405 15.7 63.9 1.7X +codegen = T hashmap = T 1705 / 1860 49.2 20.3 5.5X ================================================================================================ @@ -69,9 +69,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz Aggregate w string key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 4372 / 4440 4.8 208.5 1.0X -codegen = T hashmap = F 2850 / 2940 7.4 135.9 1.5X -codegen = T hashmap = T 2033 / 2175 10.3 96.9 2.2X +codegen = F 4019 / 4083 5.2 191.7 1.0X +codegen = T hashmap = F 2743 / 2797 7.6 130.8 1.5X +codegen = T hashmap = T 1907 / 1955 11.0 91.0 2.1X ================================================================================================ @@ -83,9 +83,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz Aggregate w decimal key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 3819 / 4047 5.5 182.1 1.0X -codegen = T hashmap = F 2080 / 2122 10.1 99.2 1.8X -codegen = T hashmap = T 500 / 523 41.9 23.9 7.6X +codegen = F 3229 / 3313 6.5 154.0 1.0X +codegen = T hashmap = F 2732 / 2826 7.7 130.3 1.2X +codegen = T hashmap = T 435 / 447 48.2 20.7 7.4X ================================================================================================ @@ -97,9 +97,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz Aggregate w multiple keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 5815 / 6015 3.6 277.3 1.0X -codegen = T hashmap = F 3934 / 3955 5.3 187.6 1.5X -codegen = T hashmap = T 3196 / 3307 6.6 152.4 1.8X +codegen = F 6326 / 6800 3.3 301.6 1.0X +codegen = T hashmap = F 3915 / 4050 5.4 186.7 1.6X +codegen = T hashmap = T 2972 / 3054 7.1 141.7 2.1X ================================================================================================ @@ -111,9 +111,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz max function bytecode size: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 415 / 511 1.6 632.8 1.0X -codegen = T hugeMethodLimit = 10000 246 / 281 2.7 375.5 1.7X -codegen = T hugeMethodLimit = 1500 416 / 472 1.6 635.2 1.0X +codegen = F 436 / 518 1.5 664.6 1.0X +codegen = T hugeMethodLimit = 10000 259 / 406 2.5 394.8 1.7X +codegen = T hugeMethodLimit = 1500 458 / 733 1.4 698.4 1.0X ================================================================================================ @@ -125,8 +125,8 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz cube: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -cube wholestage off 2192 / 2270 2.4 418.1 1.0X -cube wholestage on 910 / 1001 5.8 173.5 2.4X +cube wholestage off 2957 / 2985 1.8 564.1 1.0X +cube wholestage on 1146 / 1675 4.6 218.5 2.6X ================================================================================================ @@ -138,17 +138,17 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz BytesToBytesMap: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -UnsafeRowhash 218 / 225 96.4 10.4 1.0X -murmur3 hash 114 / 121 183.3 5.5 1.9X -fast hash 59 / 64 356.3 2.8 3.7X -arrayEqual 143 / 151 146.6 6.8 1.5X -Java HashMap (Long) 99 / 127 211.0 4.7 2.2X -Java HashMap (two ints) 101 / 136 207.2 4.8 2.1X -Java HashMap (UnsafeRow) 781 / 938 26.9 37.2 0.3X -LongToUnsafeRowMap (opt=false) 398 / 412 52.7 19.0 0.5X -LongToUnsafeRowMap (opt=true) 88 / 95 238.1 4.2 2.5X -BytesToBytesMap (off Heap) 1091 / 1113 19.2 52.0 0.2X -BytesToBytesMap (on Heap) 792 / 835 26.5 37.8 0.3X -Aggregate HashMap 36 / 41 587.9 1.7 6.1X +UnsafeRowhash 224 / 244 93.8 10.7 1.0X +murmur3 hash 114 / 134 184.2 5.4 2.0X +fast hash 61 / 66 342.8 2.9 3.7X +arrayEqual 150 / 229 139.5 7.2 1.5X +Java HashMap (Long) 127 / 305 164.7 6.1 1.8X +Java HashMap (two ints) 99 / 156 212.6 4.7 2.3X +Java HashMap (UnsafeRow) 1048 / 1472 20.0 50.0 0.2X +LongToUnsafeRowMap (opt=false) 517 / 771 40.6 24.7 0.4X +LongToUnsafeRowMap (opt=true) 90 / 107 234.1 4.3 2.5X +BytesToBytesMap (off Heap) 1417 / 1451 14.8 67.6 0.2X +BytesToBytesMap (on Heap) 934 / 1014 22.4 44.6 0.2X +Aggregate HashMap 38 / 42 558.6 1.8 6.0X diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 0cfcf988e37d..15db18de986c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -73,23 +73,26 @@ object AggregateBenchmark extends SqlBasedBenchmark { spark.range(N).selectExpr("(id & 65535) as k").groupBy("k").sum().collect() } - benchmark.addCase(s"codegen = F", numIters = 2) { iter => - spark.conf.set("spark.sql.codegen.wholeStage", "false") - f() + benchmark.addCase(s"codegen = F", numIters = 2) { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> false.toString) { + f() + } } - benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => - spark.conf.set("spark.sql.codegen.wholeStage", "true") - spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") - spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") - f() + benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> false.toString, + "spark.sql.codegen.aggregate.map.vectorized.enable" -> false.toString) { + f() + } } - benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => - spark.conf.set("spark.sql.codegen.wholeStage", "true") - spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") - f() + benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> true.toString, + "spark.sql.codegen.aggregate.map.vectorized.enable" -> true.toString) { + f() + } } benchmark.run() @@ -104,23 +107,26 @@ object AggregateBenchmark extends SqlBasedBenchmark { def f(): Unit = spark.sql("select k, k, sum(id) from test group by k, k").collect() - benchmark.addCase(s"codegen = F", numIters = 2) { iter => - spark.conf.set("spark.sql.codegen.wholeStage", value = false) - f() + benchmark.addCase(s"codegen = F", numIters = 2) { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> false.toString) { + f() + } } - benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => - spark.conf.set("spark.sql.codegen.wholeStage", value = true) - spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") - spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") - f() + benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> false.toString, + "spark.sql.codegen.aggregate.map.vectorized.enable" -> false.toString) { + f() + } } - benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => - spark.conf.set("spark.sql.codegen.wholeStage", value = true) - spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") - f() + benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> true.toString, + "spark.sql.codegen.aggregate.map.vectorized.enable" -> true.toString) { + f() + } } benchmark.run() @@ -134,23 +140,27 @@ object AggregateBenchmark extends SqlBasedBenchmark { def f(): Unit = spark.range(N).selectExpr("id", "cast(id & 1023 as string) as k") .groupBy("k").count().collect() - benchmark.addCase(s"codegen = F", numIters = 2) { iter => + benchmark.addCase(s"codegen = F", numIters = 2) { _ => spark.conf.set("spark.sql.codegen.wholeStage", "false") - f() + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> false.toString) { + f() + } } - benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { iter => - spark.conf.set("spark.sql.codegen.wholeStage", "true") - spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") - spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") - f() + benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> false.toString, + "spark.sql.codegen.aggregate.map.vectorized.enable" -> false.toString) { + f() + } } - benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { iter => - spark.conf.set("spark.sql.codegen.wholeStage", "true") - spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") - f() + benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> true.toString, + "spark.sql.codegen.aggregate.map.vectorized.enable" -> true.toString) { + f() + } } benchmark.run() @@ -164,23 +174,26 @@ object AggregateBenchmark extends SqlBasedBenchmark { def f(): Unit = spark.range(N).selectExpr("id", "cast(id & 65535 as decimal) as k") .groupBy("k").count().collect() - benchmark.addCase(s"codegen = F") { iter => - spark.conf.set("spark.sql.codegen.wholeStage", "false") - f() + benchmark.addCase(s"codegen = F") { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> false.toString) { + f() + } } - benchmark.addCase(s"codegen = T hashmap = F") { iter => - spark.conf.set("spark.sql.codegen.wholeStage", "true") - spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") - spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") - f() + benchmark.addCase(s"codegen = T hashmap = F") { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> false.toString, + "spark.sql.codegen.aggregate.map.vectorized.enable" -> false.toString) { + f() + } } - benchmark.addCase(s"codegen = T hashmap = T") { iter => - spark.conf.set("spark.sql.codegen.wholeStage", "true") - spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") - f() + benchmark.addCase(s"codegen = T hashmap = T") { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> true.toString, + "spark.sql.codegen.aggregate.map.vectorized.enable" -> true.toString) { + f() + } } benchmark.run() @@ -204,23 +217,26 @@ object AggregateBenchmark extends SqlBasedBenchmark { .sum() .collect() - benchmark.addCase(s"codegen = F") { iter => - spark.conf.set("spark.sql.codegen.wholeStage", "false") - f() + benchmark.addCase(s"codegen = F") { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> false.toString) { + f() + } } - benchmark.addCase(s"codegen = T hashmap = F") { iter => - spark.conf.set("spark.sql.codegen.wholeStage", "true") - spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") - spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "false") - f() + benchmark.addCase(s"codegen = T hashmap = F") { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> false.toString, + "spark.sql.codegen.aggregate.map.vectorized.enable" -> false.toString) { + f() + } } - benchmark.addCase(s"codegen = T hashmap = T") { iter => - spark.conf.set("spark.sql.codegen.wholeStage", "true") - spark.conf.set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - spark.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") - f() + benchmark.addCase(s"codegen = T hashmap = T") { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> true.toString, + "spark.sql.codegen.aggregate.map.vectorized.enable" -> true.toString) { + f() + } } benchmark.run() @@ -259,21 +275,24 @@ object AggregateBenchmark extends SqlBasedBenchmark { .sum() .collect() - benchmark.addCase("codegen = F") { iter => - spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "false") - f() + benchmark.addCase("codegen = F") { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> false.toString) { + f() + } } - benchmark.addCase("codegen = T hugeMethodLimit = 10000") { iter => - spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") - spark.conf.set(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key, "10000") - f() + benchmark.addCase("codegen = T hugeMethodLimit = 10000") { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, + SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> "10000") { + f() + } } - benchmark.addCase("codegen = T hugeMethodLimit = 1500") { iter => - spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") - spark.conf.set(SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key, "1500") - f() + benchmark.addCase("codegen = T hugeMethodLimit = 1500") { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, + SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> "1500") { + f() + } } benchmark.run() @@ -294,7 +313,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { val benchmark = new Benchmark("BytesToBytesMap", N, output = output) - benchmark.addCase("UnsafeRowhash") { iter => + benchmark.addCase("UnsafeRowhash") { _ => var i = 0 val keyBytes = new Array[Byte](16) val key = new UnsafeRow(1) @@ -309,7 +328,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { } } - benchmark.addCase("murmur3 hash") { iter => + benchmark.addCase("murmur3 hash") { _ => var i = 0 val keyBytes = new Array[Byte](16) val key = new UnsafeRow(1) @@ -324,7 +343,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { } } - benchmark.addCase("fast hash") { iter => + benchmark.addCase("fast hash") { _ => var i = 0 val keyBytes = new Array[Byte](16) val key = new UnsafeRow(1) @@ -342,7 +361,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { } } - benchmark.addCase("arrayEqual") { iter => + benchmark.addCase("arrayEqual") { _ => var i = 0 val keyBytes = new Array[Byte](16) val valueBytes = new Array[Byte](16) @@ -361,7 +380,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { } } - benchmark.addCase("Java HashMap (Long)") { iter => + benchmark.addCase("Java HashMap (Long)") { _ => var i = 0 val keyBytes = new Array[Byte](16) val valueBytes = new Array[Byte](16) @@ -384,7 +403,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { } } - benchmark.addCase("Java HashMap (two ints) ") { iter => + benchmark.addCase("Java HashMap (two ints) ") { _ => var i = 0 val valueBytes = new Array[Byte](16) val value = new UnsafeRow(1) @@ -408,7 +427,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { } } - benchmark.addCase("Java HashMap (UnsafeRow)") { iter => + benchmark.addCase("Java HashMap (UnsafeRow)") { _ => var i = 0 val keyBytes = new Array[Byte](16) val valueBytes = new Array[Byte](16) @@ -436,7 +455,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { } Seq(false, true).foreach { optimized => - benchmark.addCase(s"LongToUnsafeRowMap (opt=$optimized)") { iter => + benchmark.addCase(s"LongToUnsafeRowMap (opt=$optimized)") { _ => var i = 0 val valueBytes = new Array[Byte](16) val value = new UnsafeRow(1) @@ -472,7 +491,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { } Seq("off", "on").foreach { heap => - benchmark.addCase(s"BytesToBytesMap ($heap Heap)") { iter => + benchmark.addCase(s"BytesToBytesMap ($heap Heap)") { _ => val taskMemoryManager = new TaskMemoryManager( new StaticMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, s"${heap == "off"}") @@ -514,7 +533,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { } } - benchmark.addCase("Aggregate HashMap") { iter => + benchmark.addCase("Aggregate HashMap") { _ => var i = 0 val numKeys = 65536 val schema = new StructType() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala index 252911ee9811..667871c2cd77 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.benchmark import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.internal.SQLConf /** @@ -38,18 +38,48 @@ trait SqlBasedBenchmark extends BenchmarkBase { .getOrCreate() } + /** + * Sets all SQL configurations specified in `pairs`, calls `f`, and then restores all SQL + * configurations. + */ + protected def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + val conf = SQLConf.get + val (keys, values) = pairs.unzip + val currentValues = keys.map { key => + if (conf.contains(key)) { + Some(conf.getConfString(key)) + } else { + None + } + } + (keys, values).zipped.foreach { (k, v) => + if (SQLConf.staticConfKeys.contains(k)) { + throw new AnalysisException(s"Cannot modify the value of a static config: $k") + } + conf.setConfString(k, v) + } + try f finally { + keys.zip(currentValues).foreach { + case (key, Some(value)) => conf.setConfString(key, value) + case (key, None) => conf.unsetConf(key) + } + } + } + /** Runs function `f` with whole stage codegen on and off. */ def runBenchmarkWithCodegen(name: String, cardinality: Long)(f: => Unit): Unit = { val benchmark = new Benchmark(name, cardinality, output = output) - benchmark.addCase(s"$name wholestage off", numIters = 2) { iter => - spark.sqlContext.conf.setConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED, value = false) - f + benchmark.addCase(s"$name wholestage off", numIters = 2) { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> false.toString) { + f + } } - benchmark.addCase(s"$name wholestage on", numIters = 5) { iter => - spark.sqlContext.conf.setConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED, value = true) - f + benchmark.addCase(s"$name wholestage on", numIters = 5) { _ => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString) { + f + } } benchmark.run() From f783099f818d3ed509ab95133ff46e2f183444a8 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 25 Sep 2018 08:11:20 +0800 Subject: [PATCH 05/10] true.toString -> "true" and false.toString -> "false" --- .../benchmark/AggregateBenchmark.scala | 76 +++++++++---------- .../benchmark/SqlBasedBenchmark.scala | 4 +- 2 files changed, 40 insertions(+), 40 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 15db18de986c..74eca6b3c5ec 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -74,23 +74,23 @@ object AggregateBenchmark extends SqlBasedBenchmark { } benchmark.addCase(s"codegen = F", numIters = 2) { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> false.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { f() } } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, - SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> false.toString, - "spark.sql.codegen.aggregate.map.vectorized.enable" -> false.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", + "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { f() } } benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, - SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> true.toString, - "spark.sql.codegen.aggregate.map.vectorized.enable" -> true.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", + "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { f() } } @@ -108,23 +108,23 @@ object AggregateBenchmark extends SqlBasedBenchmark { def f(): Unit = spark.sql("select k, k, sum(id) from test group by k, k").collect() benchmark.addCase(s"codegen = F", numIters = 2) { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> false.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { f() } } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, - SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> false.toString, - "spark.sql.codegen.aggregate.map.vectorized.enable" -> false.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", + "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { f() } } benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, - SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> true.toString, - "spark.sql.codegen.aggregate.map.vectorized.enable" -> true.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", + "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { f() } } @@ -142,23 +142,23 @@ object AggregateBenchmark extends SqlBasedBenchmark { benchmark.addCase(s"codegen = F", numIters = 2) { _ => spark.conf.set("spark.sql.codegen.wholeStage", "false") - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> false.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { f() } } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, - SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> false.toString, - "spark.sql.codegen.aggregate.map.vectorized.enable" -> false.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", + "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { f() } } benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, - SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> true.toString, - "spark.sql.codegen.aggregate.map.vectorized.enable" -> true.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", + "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { f() } } @@ -175,23 +175,23 @@ object AggregateBenchmark extends SqlBasedBenchmark { .groupBy("k").count().collect() benchmark.addCase(s"codegen = F") { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> false.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { f() } } benchmark.addCase(s"codegen = T hashmap = F") { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, - SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> false.toString, - "spark.sql.codegen.aggregate.map.vectorized.enable" -> false.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", + "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { f() } } benchmark.addCase(s"codegen = T hashmap = T") { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, - SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> true.toString, - "spark.sql.codegen.aggregate.map.vectorized.enable" -> true.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", + "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { f() } } @@ -218,23 +218,23 @@ object AggregateBenchmark extends SqlBasedBenchmark { .collect() benchmark.addCase(s"codegen = F") { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> false.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { f() } } benchmark.addCase(s"codegen = T hashmap = F") { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, - SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> false.toString, - "spark.sql.codegen.aggregate.map.vectorized.enable" -> false.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", + "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { f() } } benchmark.addCase(s"codegen = T hashmap = T") { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, - SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> true.toString, - "spark.sql.codegen.aggregate.map.vectorized.enable" -> true.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", + "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { f() } } @@ -276,20 +276,20 @@ object AggregateBenchmark extends SqlBasedBenchmark { .collect() benchmark.addCase("codegen = F") { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> false.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { f() } } benchmark.addCase("codegen = T hugeMethodLimit = 10000") { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> "10000") { f() } } benchmark.addCase("codegen = T hugeMethodLimit = 1500") { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString, + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> "1500") { f() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala index 667871c2cd77..1519a52a4f18 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala @@ -71,13 +71,13 @@ trait SqlBasedBenchmark extends BenchmarkBase { val benchmark = new Benchmark(name, cardinality, output = output) benchmark.addCase(s"$name wholestage off", numIters = 2) { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> false.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { f } } benchmark.addCase(s"$name wholestage on", numIters = 5) { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> true.toString) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { f } } From d5fecdcca267cb34c3d9aaa72d8ea4a3a7a3cc60 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 25 Sep 2018 09:13:25 +0800 Subject: [PATCH 06/10] Fix indentation --- .../benchmark/AggregateBenchmark.scala | 36 ++++++++++++------- 1 file changed, 24 insertions(+), 12 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 74eca6b3c5ec..d277aa8d9ab5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -80,7 +80,8 @@ object AggregateBenchmark extends SqlBasedBenchmark { } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { f() @@ -88,7 +89,8 @@ object AggregateBenchmark extends SqlBasedBenchmark { } benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { f() @@ -114,7 +116,8 @@ object AggregateBenchmark extends SqlBasedBenchmark { } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { f() @@ -122,7 +125,8 @@ object AggregateBenchmark extends SqlBasedBenchmark { } benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { f() @@ -148,7 +152,8 @@ object AggregateBenchmark extends SqlBasedBenchmark { } benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { f() @@ -156,7 +161,8 @@ object AggregateBenchmark extends SqlBasedBenchmark { } benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { f() @@ -181,7 +187,8 @@ object AggregateBenchmark extends SqlBasedBenchmark { } benchmark.addCase(s"codegen = T hashmap = F") { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { f() @@ -189,7 +196,8 @@ object AggregateBenchmark extends SqlBasedBenchmark { } benchmark.addCase(s"codegen = T hashmap = T") { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { f() @@ -224,7 +232,8 @@ object AggregateBenchmark extends SqlBasedBenchmark { } benchmark.addCase(s"codegen = T hashmap = F") { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { f() @@ -232,7 +241,8 @@ object AggregateBenchmark extends SqlBasedBenchmark { } benchmark.addCase(s"codegen = T hashmap = T") { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { f() @@ -282,14 +292,16 @@ object AggregateBenchmark extends SqlBasedBenchmark { } benchmark.addCase("codegen = T hugeMethodLimit = 10000") { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> "10000") { f() } } benchmark.addCase("codegen = T hugeMethodLimit = 1500") { _ => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", + withSQLConf( + SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> "1500") { f() } From 0fae54d9e1a40c021d0ebb5af3d9d969431807f7 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Wed, 26 Sep 2018 15:13:03 +0800 Subject: [PATCH 07/10] use SQLHelper --- .../benchmarks/AggregateBenchmark-results.txt | 76 +++++++++---------- .../benchmark/SqlBasedBenchmark.scala | 33 +------- 2 files changed, 41 insertions(+), 68 deletions(-) diff --git a/sql/core/benchmarks/AggregateBenchmark-results.txt b/sql/core/benchmarks/AggregateBenchmark-results.txt index ac378ffc58fb..ec655c641ab6 100644 --- a/sql/core/benchmarks/AggregateBenchmark-results.txt +++ b/sql/core/benchmarks/AggregateBenchmark-results.txt @@ -7,8 +7,8 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz agg w/o group: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -agg w/o group wholestage off 40468 / 44934 51.8 19.3 1.0X -agg w/o group wholestage on 932 / 949 2249.6 0.4 43.4X +agg w/o group wholestage off 40943 / 47670 51.2 19.5 1.0X +agg w/o group wholestage on 979 / 1452 2142.8 0.5 41.8X ================================================================================================ @@ -20,16 +20,16 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz stddev: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -stddev wholestage off 6664 / 6775 15.7 63.6 1.0X -stddev wholestage on 907 / 923 115.6 8.6 7.3X +stddev wholestage off 6752 / 6756 15.5 64.4 1.0X +stddev wholestage on 921 / 950 113.9 8.8 7.3X Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz kurtosis: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -kurtosis wholestage off 31339 / 31357 3.3 298.9 1.0X -kurtosis wholestage on 982 / 997 106.8 9.4 31.9X +kurtosis wholestage off 31176 / 34086 3.4 297.3 1.0X +kurtosis wholestage on 1036 / 1076 101.2 9.9 30.1X ================================================================================================ @@ -41,9 +41,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 9667 / 9936 8.7 115.2 1.0X -codegen = T hashmap = F 4537 / 5353 18.5 54.1 2.1X -codegen = T hashmap = T 924 / 946 90.8 11.0 10.5X +codegen = F 10796 / 11265 7.8 128.7 1.0X +codegen = T hashmap = F 7349 / 8640 11.4 87.6 1.5X +codegen = T hashmap = T 944 / 995 88.8 11.3 11.4X ================================================================================================ @@ -55,9 +55,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 9298 / 9300 9.0 110.8 1.0X -codegen = T hashmap = F 5357 / 5405 15.7 63.9 1.7X -codegen = T hashmap = T 1705 / 1860 49.2 20.3 5.5X +codegen = F 13825 / 14425 6.1 164.8 1.0X +codegen = T hashmap = F 5962 / 6289 14.1 71.1 2.3X +codegen = T hashmap = T 1856 / 2074 45.2 22.1 7.4X ================================================================================================ @@ -69,9 +69,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz Aggregate w string key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 4019 / 4083 5.2 191.7 1.0X -codegen = T hashmap = F 2743 / 2797 7.6 130.8 1.5X -codegen = T hashmap = T 1907 / 1955 11.0 91.0 2.1X +codegen = F 4094 / 4161 5.1 195.2 1.0X +codegen = T hashmap = F 2844 / 3192 7.4 135.6 1.4X +codegen = T hashmap = T 1917 / 2011 10.9 91.4 2.1X ================================================================================================ @@ -83,9 +83,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz Aggregate w decimal key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 3229 / 3313 6.5 154.0 1.0X -codegen = T hashmap = F 2732 / 2826 7.7 130.3 1.2X -codegen = T hashmap = T 435 / 447 48.2 20.7 7.4X +codegen = F 3831 / 3910 5.5 182.7 1.0X +codegen = T hashmap = F 2097 / 2338 10.0 100.0 1.8X +codegen = T hashmap = T 427 / 478 49.1 20.4 9.0X ================================================================================================ @@ -97,9 +97,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz Aggregate w multiple keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 6326 / 6800 3.3 301.6 1.0X -codegen = T hashmap = F 3915 / 4050 5.4 186.7 1.6X -codegen = T hashmap = T 2972 / 3054 7.1 141.7 2.1X +codegen = F 5916 / 6192 3.5 282.1 1.0X +codegen = T hashmap = F 3981 / 4011 5.3 189.8 1.5X +codegen = T hashmap = T 3288 / 3337 6.4 156.8 1.8X ================================================================================================ @@ -111,9 +111,9 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz max function bytecode size: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 436 / 518 1.5 664.6 1.0X -codegen = T hugeMethodLimit = 10000 259 / 406 2.5 394.8 1.7X -codegen = T hugeMethodLimit = 1500 458 / 733 1.4 698.4 1.0X +codegen = F 475 / 578 1.4 725.1 1.0X +codegen = T hugeMethodLimit = 10000 276 / 331 2.4 420.9 1.7X +codegen = T hugeMethodLimit = 1500 449 / 510 1.5 685.4 1.1X ================================================================================================ @@ -125,8 +125,8 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz cube: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -cube wholestage off 2957 / 2985 1.8 564.1 1.0X -cube wholestage on 1146 / 1675 4.6 218.5 2.6X +cube wholestage off 2536 / 2537 2.1 483.8 1.0X +cube wholestage on 1055 / 1237 5.0 201.1 2.4X ================================================================================================ @@ -138,17 +138,17 @@ Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz BytesToBytesMap: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -UnsafeRowhash 224 / 244 93.8 10.7 1.0X -murmur3 hash 114 / 134 184.2 5.4 2.0X -fast hash 61 / 66 342.8 2.9 3.7X -arrayEqual 150 / 229 139.5 7.2 1.5X -Java HashMap (Long) 127 / 305 164.7 6.1 1.8X -Java HashMap (two ints) 99 / 156 212.6 4.7 2.3X -Java HashMap (UnsafeRow) 1048 / 1472 20.0 50.0 0.2X -LongToUnsafeRowMap (opt=false) 517 / 771 40.6 24.7 0.4X -LongToUnsafeRowMap (opt=true) 90 / 107 234.1 4.3 2.5X -BytesToBytesMap (off Heap) 1417 / 1451 14.8 67.6 0.2X -BytesToBytesMap (on Heap) 934 / 1014 22.4 44.6 0.2X -Aggregate HashMap 38 / 42 558.6 1.8 6.0X +UnsafeRowhash 231 / 242 90.8 11.0 1.0X +murmur3 hash 116 / 122 180.3 5.5 2.0X +fast hash 61 / 65 346.5 2.9 3.8X +arrayEqual 147 / 151 142.9 7.0 1.6X +Java HashMap (Long) 101 / 136 206.8 4.8 2.3X +Java HashMap (two ints) 102 / 123 205.1 4.9 2.3X +Java HashMap (UnsafeRow) 968 / 980 21.7 46.1 0.2X +LongToUnsafeRowMap (opt=false) 480 / 503 43.7 22.9 0.5X +LongToUnsafeRowMap (opt=true) 89 / 98 234.7 4.3 2.6X +BytesToBytesMap (off Heap) 1125 / 1151 18.6 53.6 0.2X +BytesToBytesMap (on Heap) 1168 / 1192 18.0 55.7 0.2X +Aggregate HashMap 41 / 57 516.4 1.9 5.7X diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala index 1519a52a4f18..430f58d346a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala @@ -18,13 +18,14 @@ package org.apache.spark.sql.execution.benchmark import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.internal.SQLConf /** * Common base trait to run benchmark with the Dataset and DataFrame API. */ -trait SqlBasedBenchmark extends BenchmarkBase { +trait SqlBasedBenchmark extends BenchmarkBase with SQLHelper { val spark: SparkSession = getSparkSession @@ -38,34 +39,6 @@ trait SqlBasedBenchmark extends BenchmarkBase { .getOrCreate() } - /** - * Sets all SQL configurations specified in `pairs`, calls `f`, and then restores all SQL - * configurations. - */ - protected def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { - val conf = SQLConf.get - val (keys, values) = pairs.unzip - val currentValues = keys.map { key => - if (conf.contains(key)) { - Some(conf.getConfString(key)) - } else { - None - } - } - (keys, values).zipped.foreach { (k, v) => - if (SQLConf.staticConfKeys.contains(k)) { - throw new AnalysisException(s"Cannot modify the value of a static config: $k") - } - conf.setConfString(k, v) - } - try f finally { - keys.zip(currentValues).foreach { - case (key, Some(value)) => conf.setConfString(key, value) - case (key, None) => conf.unsetConf(key) - } - } - } - /** Runs function `f` with whole stage codegen on and off. */ def runBenchmarkWithCodegen(name: String, cardinality: Long)(f: => Unit): Unit = { val benchmark = new Benchmark(name, cardinality, output = output) From 45add92f824c8d679086df13380f9fd27d6533d3 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 26 Sep 2018 22:55:10 -0700 Subject: [PATCH 08/10] Update result (#12) --- .../benchmarks/AggregateBenchmark-results.txt | 131 ++++++++---------- 1 file changed, 60 insertions(+), 71 deletions(-) diff --git a/sql/core/benchmarks/AggregateBenchmark-results.txt b/sql/core/benchmarks/AggregateBenchmark-results.txt index ec655c641ab6..19e524777692 100644 --- a/sql/core/benchmarks/AggregateBenchmark-results.txt +++ b/sql/core/benchmarks/AggregateBenchmark-results.txt @@ -2,153 +2,142 @@ aggregate without grouping ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz agg w/o group: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -agg w/o group wholestage off 40943 / 47670 51.2 19.5 1.0X -agg w/o group wholestage on 979 / 1452 2142.8 0.5 41.8X +agg w/o group wholestage off 65374 / 70665 32.1 31.2 1.0X +agg w/o group wholestage on 1178 / 1209 1779.8 0.6 55.5X ================================================================================================ stat functions ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz stddev: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -stddev wholestage off 6752 / 6756 15.5 64.4 1.0X -stddev wholestage on 921 / 950 113.9 8.8 7.3X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +stddev wholestage off 8667 / 8851 12.1 82.7 1.0X +stddev wholestage on 1266 / 1273 82.8 12.1 6.8X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz kurtosis: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -kurtosis wholestage off 31176 / 34086 3.4 297.3 1.0X -kurtosis wholestage on 1036 / 1076 101.2 9.9 30.1X +kurtosis wholestage off 41218 / 41231 2.5 393.1 1.0X +kurtosis wholestage on 1347 / 1357 77.8 12.8 30.6X ================================================================================================ aggregate with linear keys ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 10796 / 11265 7.8 128.7 1.0X -codegen = T hashmap = F 7349 / 8640 11.4 87.6 1.5X -codegen = T hashmap = T 944 / 995 88.8 11.3 11.4X +codegen = F 9309 / 9389 9.0 111.0 1.0X +codegen = T hashmap = F 4417 / 4435 19.0 52.7 2.1X +codegen = T hashmap = T 1289 / 1298 65.1 15.4 7.2X ================================================================================================ aggregate with randomized keys ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Aggregate w keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 13825 / 14425 6.1 164.8 1.0X -codegen = T hashmap = F 5962 / 6289 14.1 71.1 2.3X -codegen = T hashmap = T 1856 / 2074 45.2 22.1 7.4X +codegen = F 11424 / 11426 7.3 136.2 1.0X +codegen = T hashmap = F 6441 / 6496 13.0 76.8 1.8X +codegen = T hashmap = T 2333 / 2344 36.0 27.8 4.9X ================================================================================================ aggregate with string key ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Aggregate w string key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 4094 / 4161 5.1 195.2 1.0X -codegen = T hashmap = F 2844 / 3192 7.4 135.6 1.4X -codegen = T hashmap = T 1917 / 2011 10.9 91.4 2.1X +codegen = F 4751 / 4890 4.4 226.5 1.0X +codegen = T hashmap = F 3146 / 3182 6.7 150.0 1.5X +codegen = T hashmap = T 2211 / 2261 9.5 105.4 2.1X ================================================================================================ aggregate with decimal key ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Aggregate w decimal key: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 3831 / 3910 5.5 182.7 1.0X -codegen = T hashmap = F 2097 / 2338 10.0 100.0 1.8X -codegen = T hashmap = T 427 / 478 49.1 20.4 9.0X +codegen = F 3029 / 3062 6.9 144.4 1.0X +codegen = T hashmap = F 1534 / 1569 13.7 73.2 2.0X +codegen = T hashmap = T 575 / 578 36.5 27.4 5.3X ================================================================================================ aggregate with multiple key types ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Aggregate w multiple keys: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 5916 / 6192 3.5 282.1 1.0X -codegen = T hashmap = F 3981 / 4011 5.3 189.8 1.5X -codegen = T hashmap = T 3288 / 3337 6.4 156.8 1.8X +codegen = F 7506 / 7521 2.8 357.9 1.0X +codegen = T hashmap = F 4791 / 4808 4.4 228.5 1.6X +codegen = T hashmap = T 3553 / 3585 5.9 169.4 2.1X ================================================================================================ max function bytecode size of wholestagecodegen ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz max function bytecode size: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -codegen = F 475 / 578 1.4 725.1 1.0X -codegen = T hugeMethodLimit = 10000 276 / 331 2.4 420.9 1.7X -codegen = T hugeMethodLimit = 1500 449 / 510 1.5 685.4 1.1X +codegen = F 608 / 656 1.1 927.1 1.0X +codegen = T hugeMethodLimit = 10000 402 / 419 1.6 613.5 1.5X +codegen = T hugeMethodLimit = 1500 616 / 619 1.1 939.9 1.0X ================================================================================================ cube ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cube: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -cube wholestage off 2536 / 2537 2.1 483.8 1.0X -cube wholestage on 1055 / 1237 5.0 201.1 2.4X +cube wholestage off 3229 / 3237 1.6 615.9 1.0X +cube wholestage on 1285 / 1306 4.1 245.2 2.5X ================================================================================================ hash and BytesToBytesMap ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz BytesToBytesMap: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -UnsafeRowhash 231 / 242 90.8 11.0 1.0X -murmur3 hash 116 / 122 180.3 5.5 2.0X -fast hash 61 / 65 346.5 2.9 3.8X -arrayEqual 147 / 151 142.9 7.0 1.6X -Java HashMap (Long) 101 / 136 206.8 4.8 2.3X -Java HashMap (two ints) 102 / 123 205.1 4.9 2.3X -Java HashMap (UnsafeRow) 968 / 980 21.7 46.1 0.2X -LongToUnsafeRowMap (opt=false) 480 / 503 43.7 22.9 0.5X -LongToUnsafeRowMap (opt=true) 89 / 98 234.7 4.3 2.6X -BytesToBytesMap (off Heap) 1125 / 1151 18.6 53.6 0.2X -BytesToBytesMap (on Heap) 1168 / 1192 18.0 55.7 0.2X -Aggregate HashMap 41 / 57 516.4 1.9 5.7X +UnsafeRowhash 328 / 330 64.0 15.6 1.0X +murmur3 hash 167 / 167 125.4 8.0 2.0X +fast hash 84 / 85 249.0 4.0 3.9X +arrayEqual 192 / 192 109.3 9.1 1.7X +Java HashMap (Long) 144 / 147 145.9 6.9 2.3X +Java HashMap (two ints) 147 / 153 142.3 7.0 2.2X +Java HashMap (UnsafeRow) 785 / 788 26.7 37.4 0.4X +LongToUnsafeRowMap (opt=false) 456 / 457 46.0 21.8 0.7X +LongToUnsafeRowMap (opt=true) 125 / 125 168.3 5.9 2.6X +BytesToBytesMap (off Heap) 885 / 885 23.7 42.2 0.4X +BytesToBytesMap (on Heap) 860 / 864 24.4 41.0 0.4X +Aggregate HashMap 56 / 56 373.9 2.7 5.8X From 3439992a4bf1ef2ec6f3a083bf0339694216b916 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 27 Sep 2018 14:13:33 +0800 Subject: [PATCH 09/10] Fix string variable --- .../benchmark/AggregateBenchmark.scala | 31 +++++++++---------- 1 file changed, 15 insertions(+), 16 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index d277aa8d9ab5..3ca2e6255041 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -73,13 +73,13 @@ object AggregateBenchmark extends SqlBasedBenchmark { spark.range(N).selectExpr("(id & 65535) as k").groupBy("k").sum().collect() } - benchmark.addCase(s"codegen = F", numIters = 2) { _ => + benchmark.addCase("codegen = F", numIters = 2) { _ => withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { f() } } - benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { _ => + benchmark.addCase("codegen = T hashmap = F", numIters = 3) { _ => withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", @@ -88,7 +88,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { } } - benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { _ => + benchmark.addCase("codegen = T hashmap = T", numIters = 5) { _ => withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", @@ -109,13 +109,13 @@ object AggregateBenchmark extends SqlBasedBenchmark { def f(): Unit = spark.sql("select k, k, sum(id) from test group by k, k").collect() - benchmark.addCase(s"codegen = F", numIters = 2) { _ => + benchmark.addCase("codegen = F", numIters = 2) { _ => withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { f() } } - benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { _ => + benchmark.addCase("codegen = T hashmap = F", numIters = 3) { _ => withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", @@ -124,7 +124,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { } } - benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { _ => + benchmark.addCase("codegen = T hashmap = T", numIters = 5) { _ => withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", @@ -144,14 +144,13 @@ object AggregateBenchmark extends SqlBasedBenchmark { def f(): Unit = spark.range(N).selectExpr("id", "cast(id & 1023 as string) as k") .groupBy("k").count().collect() - benchmark.addCase(s"codegen = F", numIters = 2) { _ => - spark.conf.set("spark.sql.codegen.wholeStage", "false") + benchmark.addCase("codegen = F", numIters = 2) { _ => withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { f() } } - benchmark.addCase(s"codegen = T hashmap = F", numIters = 3) { _ => + benchmark.addCase("codegen = T hashmap = F", numIters = 3) { _ => withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", @@ -160,7 +159,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { } } - benchmark.addCase(s"codegen = T hashmap = T", numIters = 5) { _ => + benchmark.addCase("codegen = T hashmap = T", numIters = 5) { _ => withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", @@ -180,13 +179,13 @@ object AggregateBenchmark extends SqlBasedBenchmark { def f(): Unit = spark.range(N).selectExpr("id", "cast(id & 65535 as decimal) as k") .groupBy("k").count().collect() - benchmark.addCase(s"codegen = F") { _ => + benchmark.addCase("codegen = F") { _ => withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { f() } } - benchmark.addCase(s"codegen = T hashmap = F") { _ => + benchmark.addCase("codegen = T hashmap = F") { _ => withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", @@ -195,7 +194,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { } } - benchmark.addCase(s"codegen = T hashmap = T") { _ => + benchmark.addCase("codegen = T hashmap = T") { _ => withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", @@ -225,13 +224,13 @@ object AggregateBenchmark extends SqlBasedBenchmark { .sum() .collect() - benchmark.addCase(s"codegen = F") { _ => + benchmark.addCase("codegen = F") { _ => withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { f() } } - benchmark.addCase(s"codegen = T hashmap = F") { _ => + benchmark.addCase("codegen = T hashmap = F") { _ => withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", @@ -240,7 +239,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { } } - benchmark.addCase(s"codegen = T hashmap = T") { _ => + benchmark.addCase("codegen = T hashmap = T") { _ => withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", From 6c46ad59c063fa6283fb23046300404767a82248 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 1 Oct 2018 14:37:46 +0800 Subject: [PATCH 10/10] Address comments --- .../sql/execution/benchmark/AggregateBenchmark.scala | 8 ++++---- .../spark/sql/execution/benchmark/SqlBasedBenchmark.scala | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 3ca2e6255041..296ae104a94a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -47,7 +47,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { override def benchmark(): Unit = { runBenchmark("aggregate without grouping") { val N = 500L << 22 - runBenchmarkWithCodegen("agg w/o group", N) { + codegenBenchmark("agg w/o group", N) { spark.range(N).selectExpr("sum(id)").collect() } } @@ -55,11 +55,11 @@ object AggregateBenchmark extends SqlBasedBenchmark { runBenchmark("stat functions") { val N = 100L << 20 - runBenchmarkWithCodegen("stddev", N) { + codegenBenchmark("stddev", N) { spark.range(N).groupBy().agg("id" -> "stddev").collect() } - runBenchmarkWithCodegen("kurtosis", N) { + codegenBenchmark("kurtosis", N) { spark.range(N).groupBy().agg("id" -> "kurtosis").collect() } } @@ -313,7 +313,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { runBenchmark("cube") { val N = 5 << 20 - runBenchmarkWithCodegen("cube", N) { + codegenBenchmark("cube", N) { spark.range(N).selectExpr("id", "id % 1000 as k1", "id & 256 as k2") .cube("k1", "k2").sum("id").collect() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala index 430f58d346a0..e95e5a960246 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.internal.SQLConf */ trait SqlBasedBenchmark extends BenchmarkBase with SQLHelper { - val spark: SparkSession = getSparkSession + protected val spark: SparkSession = getSparkSession /** Subclass can override this function to build their own SparkSession */ def getSparkSession: SparkSession = { @@ -40,7 +40,7 @@ trait SqlBasedBenchmark extends BenchmarkBase with SQLHelper { } /** Runs function `f` with whole stage codegen on and off. */ - def runBenchmarkWithCodegen(name: String, cardinality: Long)(f: => Unit): Unit = { + final def codegenBenchmark(name: String, cardinality: Long)(f: => Unit): Unit = { val benchmark = new Benchmark(name, cardinality, output = output) benchmark.addCase(s"$name wholestage off", numIters = 2) { _ =>