aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/test/scala
diff options
context:
space:
mode:
authorQifan Pu <qifan.pu@gmail.com>2016-09-01 16:56:35 -0700
committerDavies Liu <davies.liu@gmail.com>2016-09-01 16:56:35 -0700
commit03d77af9ec4ce9a42affd6ab4381ae5bd3c79a5a (patch)
tree995655dd8f46c969cf701206521ee9273b3a2430 /sql/core/src/test/scala
parent15539e54c2650a164f09c072f8fae934bb0468c9 (diff)
downloadspark-03d77af9ec4ce9a42affd6ab4381ae5bd3c79a5a.tar.gz
spark-03d77af9ec4ce9a42affd6ab4381ae5bd3c79a5a.tar.bz2
spark-03d77af9ec4ce9a42affd6ab4381ae5bd3c79a5a.zip
[SPARK-16525] [SQL] Enable Row Based HashMap in HashAggregateExec
## What changes were proposed in this pull request? This PR is the second step for the following feature: For hash aggregation in Spark SQL, we use a fast aggregation hashmap to act as a "cache" in order to boost aggregation performance. Previously, the hashmap is backed by a `ColumnarBatch`. This has performance issues when we have wide schema for the aggregation table (large number of key fields or value fields). In this JIRA, we support another implementation of fast hashmap, which is backed by a `RowBatch`. We then automatically pick between the two implementations based on certain knobs. In this second-step PR, we enable `RowBasedHashMapGenerator` in `HashAggregateExec`. ## How was this patch tested? Added tests: `RowBasedAggregateHashMapSuite` and ` VectorizedAggregateHashMapSuite` Additional micro-benchmarks tests and TPCDS results will be added in a separate PR in the series. Author: Qifan Pu <qifan.pu@gmail.com> Author: ooq <qifan.pu@gmail.com> Closes #14176 from ooq/rowbasedfastaggmap-pr2.
Diffstat (limited to 'sql/core/src/test/scala')
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala79
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala8
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala25
3 files changed, 102 insertions, 10 deletions
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala
new file mode 100644
index 0000000000..3e85d95523
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala
@@ -0,0 +1,79 @@
+/*
+ * 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
+
+import org.scalatest.BeforeAndAfter
+
+class SingleLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter {
+
+ protected override def beforeAll(): Unit = {
+ sparkConf.set("spark.sql.codegen.fallback", "false")
+ sparkConf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "false")
+ super.beforeAll()
+ }
+
+ // adding some checking after each test is run, assuring that the configs are not changed
+ // in test code
+ after {
+ assert(sparkConf.get("spark.sql.codegen.fallback") == "false",
+ "configuration parameter changed in test body")
+ assert(sparkConf.get("spark.sql.codegen.aggregate.map.twolevel.enable") == "false",
+ "configuration parameter changed in test body")
+ }
+}
+
+class TwoLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter {
+
+ protected override def beforeAll(): Unit = {
+ sparkConf.set("spark.sql.codegen.fallback", "false")
+ sparkConf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "true")
+ super.beforeAll()
+ }
+
+ // adding some checking after each test is run, assuring that the configs are not changed
+ // in test code
+ after {
+ assert(sparkConf.get("spark.sql.codegen.fallback") == "false",
+ "configuration parameter changed in test body")
+ assert(sparkConf.get("spark.sql.codegen.aggregate.map.twolevel.enable") == "true",
+ "configuration parameter changed in test body")
+ }
+}
+
+class TwoLevelAggregateHashMapWithVectorizedMapSuite extends DataFrameAggregateSuite with
+BeforeAndAfter {
+
+ protected override def beforeAll(): Unit = {
+ sparkConf.set("spark.sql.codegen.fallback", "false")
+ sparkConf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "true")
+ sparkConf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true")
+ super.beforeAll()
+ }
+
+ // adding some checking after each test is run, assuring that the configs are not changed
+ // in test code
+ after {
+ assert(sparkConf.get("spark.sql.codegen.fallback") == "false",
+ "configuration parameter changed in test body")
+ assert(sparkConf.get("spark.sql.codegen.aggregate.map.twolevel.enable") == "true",
+ "configuration parameter changed in test body")
+ assert(sparkConf.get("spark.sql.codegen.aggregate.map.vectorized.enable") == "true",
+ "configuration parameter changed in test body")
+ }
+}
+
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
index 69a3b5f278..427390a90f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
@@ -485,4 +485,12 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext {
spark.sql("select avg(a) over () from values 1.0, 2.0, 3.0 T(a)"),
Row(2.0) :: Row(2.0) :: Row(2.0) :: Nil)
}
+
+ test("SQL decimal test (used for catching certain demical handling bugs in aggregates)") {
+ checkAnswer(
+ decimalData.groupBy('a cast DecimalType(10, 2)).agg(avg('b cast DecimalType(10, 2))),
+ Seq(Row(new java.math.BigDecimal(1.0), new java.math.BigDecimal(1.5)),
+ Row(new java.math.BigDecimal(2.0), new java.math.BigDecimal(1.5)),
+ Row(new java.math.BigDecimal(3.0), new java.math.BigDecimal(1.5))))
+ }
}
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 bf3a39c84b..8a2993bdf4 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
@@ -106,13 +106,14 @@ class AggregateBenchmark extends BenchmarkBase {
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.columns.max", "0")
+ sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.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.columns.max", "3")
+ sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "true")
+ sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true")
f()
}
@@ -146,13 +147,14 @@ class AggregateBenchmark extends BenchmarkBase {
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.columns.max", 0)
+ sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.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.columns.max", 3)
+ sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "true")
+ sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true")
f()
}
@@ -184,13 +186,14 @@ class AggregateBenchmark extends BenchmarkBase {
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.columns.max", "0")
+ sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.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.columns.max", "3")
+ sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "true")
+ sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true")
f()
}
@@ -221,13 +224,14 @@ class AggregateBenchmark extends BenchmarkBase {
benchmark.addCase(s"codegen = T hashmap = F") { iter =>
sparkSession.conf.set("spark.sql.codegen.wholeStage", "true")
- sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "0")
+ sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.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.columns.max", "3")
+ sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "true")
+ sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true")
f()
}
@@ -268,13 +272,14 @@ class AggregateBenchmark extends BenchmarkBase {
benchmark.addCase(s"codegen = T hashmap = F") { iter =>
sparkSession.conf.set("spark.sql.codegen.wholeStage", "true")
- sparkSession.conf.set("spark.sql.codegen.aggregate.map.columns.max", "0")
+ sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.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.columns.max", "10")
+ sparkSession.conf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "true")
+ sparkSession.conf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true")
f()
}