Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -165,10 +165,10 @@ private void freeCurrentPage() {
protected FixedLengthRowBasedKeyValueBatch(StructType keySchema, StructType valueSchema,
int maxRows, TaskMemoryManager manager) {
super(keySchema, valueSchema, maxRows, manager);
klen = keySchema.defaultSize()
+ UnsafeRow.calculateBitSetWidthInBytes(keySchema.length());
vlen = valueSchema.defaultSize()
+ UnsafeRow.calculateBitSetWidthInBytes(valueSchema.length());
int keySize = keySchema.size() * 8; // each fixed-length field is stored in a 8-byte word
int valueSize = valueSchema.size() * 8;
klen = keySize + UnsafeRow.calculateBitSetWidthInBytes(keySchema.length());
vlen = valueSize + UnsafeRow.calculateBitSetWidthInBytes(valueSchema.length());
recordLength = klen + vlen + 8;
}
}

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -141,8 +141,16 @@ class RowBasedHashMapGenerator(
}

val createUnsafeRowForKey = groupingKeys.zipWithIndex.map { case (key: Buffer, ordinal: Int) =>
s"agg_rowWriter.write(${ordinal}, ${key.name})"}
.mkString(";\n")
key.dataType match {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add a regression test for this in [sql]?

case t: DecimalType =>
s"agg_rowWriter.write(${ordinal}, ${key.name}, ${t.precision}, ${t.scale})"
case t: DataType =>
if (!t.isInstanceOf[StringType] && !ctx.isPrimitiveType(t)) {
throw new IllegalArgumentException(s"cannot generate code for unsupported type: $t")
}
s"agg_rowWriter.write(${ordinal}, ${key.name})"
}
}.mkString(";\n")

s"""
|public org.apache.spark.sql.catalyst.expressions.UnsafeRow findOrInsert(${
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -499,14 +499,15 @@ object SQLConf {
.intConf
.createWithDefault(40)

val VECTORIZED_AGG_MAP_MAX_COLUMNS =
SQLConfigBuilder("spark.sql.codegen.aggregate.map.columns.max")
val ENABLE_TWOLEVEL_AGG_MAP =
SQLConfigBuilder("spark.sql.codegen.aggregate.map.twolevel.enable")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's also make sure that all references to the old config are also appropriately modified.

.internal()
.doc("Sets the maximum width of schema (aggregate keys + values) for which aggregate with" +
"keys uses an in-memory columnar map to speed up execution. Setting this to 0 effectively" +
"disables the columnar map")
.intConf
.createWithDefault(3)
.doc("Enable two-level aggregate hash map. When enabled, records will first be " +
"inserted/looked-up at a 1st-level, small, fast map, and then fallback to a " +
"2nd-level, larger, slower map when 1st level is full or keys cannot be found. " +
"When disabled, records go directly to the 2nd level. Defaults to true.")
.booleanConf
.createWithDefault(true)

val FILE_SINK_LOG_DELETION = SQLConfigBuilder("spark.sql.streaming.fileSink.log.deletion")
.internal()
Expand Down Expand Up @@ -673,7 +674,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging {

override def runSQLonFile: Boolean = getConf(RUN_SQL_ON_FILES)

def vectorizedAggregateMapMaxColumns: Int = getConf(VECTORIZED_AGG_MAP_MAX_COLUMNS)
def enableTwoLevelAggMap: Boolean = getConf(ENABLE_TWOLEVEL_AGG_MAP)

def variableSubstituteEnabled: Boolean = getConf(VARIABLE_SUBSTITUTE_ENABLED)

Expand Down
Original file line number Diff line number Diff line change
@@ -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")
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -475,4 +475,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))))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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()
}

Expand Down Expand Up @@ -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()
}

Expand Down Expand Up @@ -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()
}

Expand Down Expand Up @@ -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()
}

Expand Down Expand Up @@ -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()
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -998,9 +998,9 @@ class HashAggregationQuerySuite extends AggregationQuerySuite
class HashAggregationQueryWithControlledFallbackSuite extends AggregationQuerySuite {

override protected def checkAnswer(actual: => DataFrame, expectedAnswer: Seq[Row]): Unit = {
Seq(0, 10).foreach { maxColumnarHashMapColumns =>
withSQLConf("spark.sql.codegen.aggregate.map.columns.max" ->
maxColumnarHashMapColumns.toString) {
Seq("true", "false").foreach { enableTwoLevelMaps =>
withSQLConf("spark.sql.codegen.aggregate.map.twolevel.enable" ->
enableTwoLevelMaps) {
(1 to 3).foreach { fallbackStartsAt =>
withSQLConf("spark.sql.TungstenAggregate.testFallbackStartsAt" ->
s"${(fallbackStartsAt - 1).toString}, ${fallbackStartsAt.toString}") {
Expand Down