You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by bo...@apache.org on 2019/05/10 22:45:53 UTC
[drill] branch master updated: DRILL-7237: Fix single_value
aggregate function for variable length types
This is an automated email from the ASF dual-hosted git repository.
boaz pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git
The following commit(s) were added to refs/heads/master by this push:
new 0195d1f DRILL-7237: Fix single_value aggregate function for variable length types
0195d1f is described below
commit 0195d1f34be7fd385ba76d2fd3e14a9fa13bd375
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Mon May 6 17:35:12 2019 +0300
DRILL-7237: Fix single_value aggregate function for variable length types
- Add implementations of single_value for complex data types
closes #1782
---
exec/java-exec/src/main/codegen/config.fmpp | 1 -
.../java-exec/src/main/codegen/data/AggrTypes1.tdd | 148 ++++++++-----
.../src/main/codegen/data/DecimalAggrTypes1.tdd | 6 +-
.../src/main/codegen/data/SingleValue.tdd | 62 ------
.../main/codegen/templates/AggrTypeFunctions1.java | 237 +++++++++++----------
.../codegen/templates/ComplexAggrFunctions1.java | 124 ++++++-----
.../templates/DateIntervalAggrFunctions1.java | 13 +-
.../Decimal/DecimalAggrTypeFunctions1.java | 38 +---
.../src/main/codegen/templates/SingleValueAgg.java | 144 -------------
.../codegen/templates/VarCharAggrFunctions1.java | 9 +-
.../test/java/org/apache/drill/PlanTestBase.java | 5 +-
.../java/org/apache/drill/TestExampleQueries.java | 15 ++
.../drill/exec/fn/impl/TestAggregateFunctions.java | 115 +++++++++-
13 files changed, 442 insertions(+), 475 deletions(-)
diff --git a/exec/java-exec/src/main/codegen/config.fmpp b/exec/java-exec/src/main/codegen/config.fmpp
index e233974..50f110d 100644
--- a/exec/java-exec/src/main/codegen/config.fmpp
+++ b/exec/java-exec/src/main/codegen/config.fmpp
@@ -43,7 +43,6 @@ data: {
intervalNumericTypes: tdd(../data/IntervalNumericTypes.tdd),
extract: tdd(../data/ExtractTypes.tdd),
sumzero: tdd(../data/SumZero.tdd),
- singleValue: tdd(../data/SingleValue.tdd),
numericTypes: tdd(../data/NumericTypes.tdd),
casthigh: tdd(../data/CastHigh.tdd),
countAggrTypes: tdd(../data/CountAggrTypes.tdd)
diff --git a/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd b/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
index 3fb2601..7512129 100644
--- a/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
+++ b/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
@@ -44,8 +44,8 @@
{inputType: "VarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "VarBytes"},
{inputType: "NullableVarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "VarBytes"}
]
- },
- {className: "Max", funcName: "max", types: [
+ },
+ {className: "Max", funcName: "max", types: [
{inputType: "Int", outputType: "NullableInt", runningType: "Int", major: "Numeric"},
{inputType: "BigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
{inputType: "NullableInt", outputType: "NullableInt", runningType: "Int", major: "Numeric"},
@@ -71,8 +71,8 @@
{inputType: "VarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "VarBytes"},
{inputType: "NullableVarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "VarBytes"}
]
- },
- {className: "Sum", funcName: "sum", types: [
+ },
+ {className: "Sum", funcName: "sum", types: [
{inputType: "Int", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
{inputType: "BigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
{inputType: "NullableInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
@@ -88,52 +88,100 @@
{inputType: "Interval", outputType: "NullableInterval", runningType: "Interval", major: "Date", initialValue: "0"},
{inputType: "NullableInterval", outputType: "NullableInterval", runningType: "Interval", major: "Date", initialValue: "0"}
]
- },
- {className: "AnyValue", funcName: "any_value", types: [
- {inputType: "Bit", outputType: "NullableBit", runningType: "Bit", major: "Numeric"},
- {inputType: "Int", outputType: "NullableInt", runningType: "Int", major: "Numeric"},
- {inputType: "BigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
- {inputType: "NullableBit", outputType: "NullableBit", runningType: "Bit", major: "Numeric"},
- {inputType: "NullableInt", outputType: "NullableInt", runningType: "Int", major: "Numeric"},
- {inputType: "NullableBigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
- {inputType: "Float4", outputType: "NullableFloat4", runningType: "Float4", major: "Numeric"},
- {inputType: "Float8", outputType: "NullableFloat8", runningType: "Float8", major: "Numeric"},
- {inputType: "NullableFloat4", outputType: "NullableFloat4", runningType: "Float4", major: "Numeric"},
- {inputType: "NullableFloat8", outputType: "NullableFloat8", runningType: "Float8", major: "Numeric"},
- {inputType: "Date", outputType: "NullableDate", runningType: "Date", major: "Date", initialValue: "0"},
- {inputType: "NullableDate", outputType: "NullableDate", runningType: "Date", major: "Date", initialValue: "0"},
- {inputType: "TimeStamp", outputType: "NullableTimeStamp", runningType: "TimeStamp", major: "Date", initialValue: "0"},
- {inputType: "NullableTimeStamp", outputType: "NullableTimeStamp", runningType: "TimeStamp", major: "Date", initialValue: "0"},
- {inputType: "Time", outputType: "NullableTime", runningType: "Time", major: "Date", initialValue: "0"},
- {inputType: "NullableTime", outputType: "NullableTime", runningType: "Time", major: "Date", initialValue: "0"},
- {inputType: "IntervalDay", outputType: "NullableIntervalDay", runningType: "IntervalDay", major: "Date", initialValue: "0"},
- {inputType: "NullableIntervalDay", outputType: "NullableIntervalDay", runningType: "IntervalDay", major: "Date", initialValue: "0"},
- {inputType: "IntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "0"},
- {inputType: "NullableIntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "0"},
- {inputType: "Interval", outputType: "NullableInterval", runningType: "Interval", major: "Date", initialValue: "0"},
- {inputType: "NullableInterval", outputType: "NullableInterval", runningType: "Interval", major: "Date", initialValue: "0"},
- {inputType: "VarChar", outputType: "NullableVarChar", runningType: "VarChar", major: "VarBytes", initialValue: ""},
- {inputType: "NullableVarChar", outputType: "NullableVarChar", runningType: "VarChar", major: "VarBytes", initialValue: ""},
- {inputType: "VarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "VarBytes"},
- {inputType: "NullableVarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "VarBytes"}
- {inputType: "List", outputType: "List", runningType: "List", major: "Complex"}
- {inputType: "Map", outputType: "Map", runningType: "Map", major: "Complex"}
- {inputType: "RepeatedBit", outputType: "RepeatedBit", runningType: "RepeatedBit", major: "Complex"},
- {inputType: "RepeatedInt", outputType: "RepeatedInt", runningType: "RepeatedInt", major: "Complex"},
- {inputType: "RepeatedBigInt", outputType: "RepeatedBigInt", runningType: "RepeatedBigInt", major: "Complex"},
- {inputType: "RepeatedFloat4", outputType: "RepeatedFloat4", runningType: "RepeatedFloat4", major: "Complex"},
- {inputType: "RepeatedFloat8", outputType: "RepeatedFloat8", runningType: "RepeatedFloat8", major: "Complex"},
- {inputType: "RepeatedDate", outputType: "RepeatedDate", runningType: "RepeatedDate", major: "Complex"},
- {inputType: "RepeatedTimeStamp", outputType: "RepeatedTimeStamp", runningType: "RepeatedTimeStamp", major: "Complex"},
- {inputType: "RepeatedTime", outputType: "RepeatedTime", runningType: "RepeatedTime", major: "Complex"},
- {inputType: "RepeatedIntervalDay", outputType: "RepeatedIntervalDay", runningType: "RepeatedIntervalDay", major: "Complex"},
- {inputType: "RepeatedIntervalYear", outputType: "RepeatedIntervalYear", runningType: "RepeatedIntervalYear", major: "Complex"},
- {inputType: "RepeatedInterval", outputType: "RepeatedInterval", runningType: "RepeatedInterval", major: "Complex"},
- {inputType: "RepeatedVarChar", outputType: "RepeatedVarChar", runningType: "RepeatedVarChar", major: "Complex"},
- {inputType: "RepeatedVarBinary", outputType: "RepeatedVarBinary", runningType: "RepeatedVarBinary", major: "Complex"},
- {inputType: "RepeatedList", outputType: "RepeatedList", runningType: "RepeatedList", major: "Complex"},
- {inputType: "RepeatedMap", outputType: "RepeatedMap", runningType: "RepeatedMap", major: "Complex"}
+ },
+ {className: "AnyValue", funcName: "any_value", types: [
+ {inputType: "Bit", outputType: "NullableBit", runningType: "Bit", major: "Numeric"},
+ {inputType: "Int", outputType: "NullableInt", runningType: "Int", major: "Numeric"},
+ {inputType: "BigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
+ {inputType: "NullableBit", outputType: "NullableBit", runningType: "Bit", major: "Numeric"},
+ {inputType: "NullableInt", outputType: "NullableInt", runningType: "Int", major: "Numeric"},
+ {inputType: "NullableBigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
+ {inputType: "Float4", outputType: "NullableFloat4", runningType: "Float4", major: "Numeric"},
+ {inputType: "Float8", outputType: "NullableFloat8", runningType: "Float8", major: "Numeric"},
+ {inputType: "NullableFloat4", outputType: "NullableFloat4", runningType: "Float4", major: "Numeric"},
+ {inputType: "NullableFloat8", outputType: "NullableFloat8", runningType: "Float8", major: "Numeric"},
+ {inputType: "Date", outputType: "NullableDate", runningType: "Date", major: "Date", initialValue: "0"},
+ {inputType: "NullableDate", outputType: "NullableDate", runningType: "Date", major: "Date", initialValue: "0"},
+ {inputType: "TimeStamp", outputType: "NullableTimeStamp", runningType: "TimeStamp", major: "Date", initialValue: "0"},
+ {inputType: "NullableTimeStamp", outputType: "NullableTimeStamp", runningType: "TimeStamp", major: "Date", initialValue: "0"},
+ {inputType: "Time", outputType: "NullableTime", runningType: "Time", major: "Date", initialValue: "0"},
+ {inputType: "NullableTime", outputType: "NullableTime", runningType: "Time", major: "Date", initialValue: "0"},
+ {inputType: "IntervalDay", outputType: "NullableIntervalDay", runningType: "IntervalDay", major: "Date", initialValue: "0"},
+ {inputType: "NullableIntervalDay", outputType: "NullableIntervalDay", runningType: "IntervalDay", major: "Date", initialValue: "0"},
+ {inputType: "IntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "0"},
+ {inputType: "NullableIntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "0"},
+ {inputType: "Interval", outputType: "NullableInterval", runningType: "Interval", major: "Date", initialValue: "0"},
+ {inputType: "NullableInterval", outputType: "NullableInterval", runningType: "Interval", major: "Date", initialValue: "0"},
+ {inputType: "VarChar", outputType: "NullableVarChar", runningType: "VarChar", major: "VarBytes", initialValue: ""},
+ {inputType: "NullableVarChar", outputType: "NullableVarChar", runningType: "VarChar", major: "VarBytes", initialValue: ""},
+ {inputType: "VarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "VarBytes"},
+ {inputType: "NullableVarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "VarBytes"}
+ {inputType: "List", outputType: "List", runningType: "List", major: "Complex"}
+ {inputType: "Map", outputType: "Map", runningType: "Map", major: "Complex"}
+ {inputType: "RepeatedBit", outputType: "RepeatedBit", runningType: "RepeatedBit", major: "Complex"},
+ {inputType: "RepeatedInt", outputType: "RepeatedInt", runningType: "RepeatedInt", major: "Complex"},
+ {inputType: "RepeatedBigInt", outputType: "RepeatedBigInt", runningType: "RepeatedBigInt", major: "Complex"},
+ {inputType: "RepeatedFloat4", outputType: "RepeatedFloat4", runningType: "RepeatedFloat4", major: "Complex"},
+ {inputType: "RepeatedFloat8", outputType: "RepeatedFloat8", runningType: "RepeatedFloat8", major: "Complex"},
+ {inputType: "RepeatedDate", outputType: "RepeatedDate", runningType: "RepeatedDate", major: "Complex"},
+ {inputType: "RepeatedTimeStamp", outputType: "RepeatedTimeStamp", runningType: "RepeatedTimeStamp", major: "Complex"},
+ {inputType: "RepeatedTime", outputType: "RepeatedTime", runningType: "RepeatedTime", major: "Complex"},
+ {inputType: "RepeatedIntervalDay", outputType: "RepeatedIntervalDay", runningType: "RepeatedIntervalDay", major: "Complex"},
+ {inputType: "RepeatedIntervalYear", outputType: "RepeatedIntervalYear", runningType: "RepeatedIntervalYear", major: "Complex"},
+ {inputType: "RepeatedInterval", outputType: "RepeatedInterval", runningType: "RepeatedInterval", major: "Complex"},
+ {inputType: "RepeatedVarChar", outputType: "RepeatedVarChar", runningType: "RepeatedVarChar", major: "Complex"},
+ {inputType: "RepeatedVarBinary", outputType: "RepeatedVarBinary", runningType: "RepeatedVarBinary", major: "Complex"},
+ {inputType: "RepeatedVarDecimal", outputType: "RepeatedVarDecimal", runningType: "RepeatedVarDecimal", major: "Complex"},
+ {inputType: "RepeatedList", outputType: "RepeatedList", runningType: "RepeatedList", major: "Complex"},
+ {inputType: "RepeatedMap", outputType: "RepeatedMap", runningType: "RepeatedMap", major: "Complex"}
+ ]
+ },
+ {className: "SingleValue", funcName: "single_value", types: [
+ {inputType: "Bit", outputType: "NullableBit", runningType: "Bit", major: "Numeric"},
+ {inputType: "Int", outputType: "NullableInt", runningType: "Int", major: "Numeric"},
+ {inputType: "BigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
+ {inputType: "NullableBit", outputType: "NullableBit", runningType: "Bit", major: "Numeric"},
+ {inputType: "NullableInt", outputType: "NullableInt", runningType: "Int", major: "Numeric"},
+ {inputType: "NullableBigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "Numeric"},
+ {inputType: "Float4", outputType: "NullableFloat4", runningType: "Float4", major: "Numeric"},
+ {inputType: "Float8", outputType: "NullableFloat8", runningType: "Float8", major: "Numeric"},
+ {inputType: "NullableFloat4", outputType: "NullableFloat4", runningType: "Float4", major: "Numeric"},
+ {inputType: "NullableFloat8", outputType: "NullableFloat8", runningType: "Float8", major: "Numeric"},
+ {inputType: "Date", outputType: "NullableDate", runningType: "Date", major: "Date", initialValue: "0"},
+ {inputType: "NullableDate", outputType: "NullableDate", runningType: "Date", major: "Date", initialValue: "0"},
+ {inputType: "TimeStamp", outputType: "NullableTimeStamp", runningType: "TimeStamp", major: "Date", initialValue: "0"},
+ {inputType: "NullableTimeStamp", outputType: "NullableTimeStamp", runningType: "TimeStamp", major: "Date", initialValue: "0"},
+ {inputType: "Time", outputType: "NullableTime", runningType: "Time", major: "Date", initialValue: "0"},
+ {inputType: "NullableTime", outputType: "NullableTime", runningType: "Time", major: "Date", initialValue: "0"},
+ {inputType: "IntervalDay", outputType: "NullableIntervalDay", runningType: "IntervalDay", major: "Date", initialValue: "0"},
+ {inputType: "NullableIntervalDay", outputType: "NullableIntervalDay", runningType: "IntervalDay", major: "Date", initialValue: "0"},
+ {inputType: "IntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "0"},
+ {inputType: "NullableIntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "Date", initialValue: "0"},
+ {inputType: "Interval", outputType: "NullableInterval", runningType: "Interval", major: "Date", initialValue: "0"},
+ {inputType: "NullableInterval", outputType: "NullableInterval", runningType: "Interval", major: "Date", initialValue: "0"},
+ {inputType: "VarChar", outputType: "NullableVarChar", runningType: "VarChar", major: "VarBytes", initialValue: ""},
+ {inputType: "NullableVarChar", outputType: "NullableVarChar", runningType: "VarChar", major: "VarBytes", initialValue: ""},
+ {inputType: "VarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "VarBytes"},
+ {inputType: "NullableVarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "VarBytes"}
+ {inputType: "List", outputType: "List", runningType: "List", major: "Complex"}
+ {inputType: "Map", outputType: "Map", runningType: "Map", major: "Complex"}
+ {inputType: "RepeatedBit", outputType: "RepeatedBit", runningType: "RepeatedBit", major: "Complex"},
+ {inputType: "RepeatedInt", outputType: "RepeatedInt", runningType: "RepeatedInt", major: "Complex"},
+ {inputType: "RepeatedBigInt", outputType: "RepeatedBigInt", runningType: "RepeatedBigInt", major: "Complex"},
+ {inputType: "RepeatedFloat4", outputType: "RepeatedFloat4", runningType: "RepeatedFloat4", major: "Complex"},
+ {inputType: "RepeatedFloat8", outputType: "RepeatedFloat8", runningType: "RepeatedFloat8", major: "Complex"},
+ {inputType: "RepeatedDate", outputType: "RepeatedDate", runningType: "RepeatedDate", major: "Complex"},
+ {inputType: "RepeatedTimeStamp", outputType: "RepeatedTimeStamp", runningType: "RepeatedTimeStamp", major: "Complex"},
+ {inputType: "RepeatedTime", outputType: "RepeatedTime", runningType: "RepeatedTime", major: "Complex"},
+ {inputType: "RepeatedIntervalDay", outputType: "RepeatedIntervalDay", runningType: "RepeatedIntervalDay", major: "Complex"},
+ {inputType: "RepeatedIntervalYear", outputType: "RepeatedIntervalYear", runningType: "RepeatedIntervalYear", major: "Complex"},
+ {inputType: "RepeatedInterval", outputType: "RepeatedInterval", runningType: "RepeatedInterval", major: "Complex"},
+ {inputType: "RepeatedVarChar", outputType: "RepeatedVarChar", runningType: "RepeatedVarChar", major: "Complex"},
+ {inputType: "RepeatedVarBinary", outputType: "RepeatedVarBinary", runningType: "RepeatedVarBinary", major: "Complex"},
+ {inputType: "RepeatedVarDecimal", outputType: "RepeatedVarDecimal", runningType: "RepeatedVarDecimal", major: "Complex"},
+ {inputType: "RepeatedList", outputType: "RepeatedList", runningType: "RepeatedList", major: "Complex"},
+ {inputType: "RepeatedMap", outputType: "RepeatedMap", runningType: "RepeatedMap", major: "Complex"}
]
- }
+ }
]
}
diff --git a/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd b/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd
index 003bbfa..4e06d04 100644
--- a/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd
+++ b/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd
@@ -39,7 +39,11 @@
{className: "AnyValue", funcName: "any_value", types: [
{inputType: "VarDecimal", outputType: "NullableVarDecimal"},
{inputType: "NullableVarDecimal", outputType: "NullableVarDecimal"}
- {inputType: "RepeatedVarDecimal", outputType: "RepeatedVarDecimal"}
+ ]
+ },
+ {className: "SingleValue", funcName: "single_value", types: [
+ {inputType: "VarDecimal", outputType: "NullableVarDecimal"},
+ {inputType: "NullableVarDecimal", outputType: "NullableVarDecimal"}
]
}
]
diff --git a/exec/java-exec/src/main/codegen/data/SingleValue.tdd b/exec/java-exec/src/main/codegen/data/SingleValue.tdd
deleted file mode 100644
index a42fe3b..0000000
--- a/exec/java-exec/src/main/codegen/data/SingleValue.tdd
+++ /dev/null
@@ -1,62 +0,0 @@
-# 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.
-
-{
-types: [
- {inputType: "Bit", outputType: "NullableBit", runningType: "Bit", major: "primitive"},
- {inputType: "TinyInt", outputType: "NullableTinyInt", runningType: "TinyInt", major: "primitive"},
- {inputType: "NullableTinyInt", outputType: "NullableTinyInt", runningType: "TinyInt", major: "primitive"},
- {inputType: "UInt1", outputType: "NullableUInt1", runningType: "UInt1", major: "primitive"},
- {inputType: "NullableUInt1", outputType: "NullableUInt1", runningType: "UInt1", major: "primitive"},
- {inputType: "UInt2", outputType: "NullableUInt2", runningType: "UInt2", major: "primitive"},
- {inputType: "NullableUInt2", outputType: "NullableUInt2", runningType: "UInt2", major: "primitive"},
- {inputType: "SmallInt", outputType: "NullableSmallInt", runningType: "SmallInt", major: "primitive"},
- {inputType: "NullableSmallInt", outputType: "NullableSmallInt", runningType: "SmallInt", major: "primitive"},
- {inputType: "UInt4", outputType: "NullableUInt4", runningType: "UInt4", major: "primitive"},
- {inputType: "NullableUInt4", outputType: "NullableUInt4", runningType: "UInt4", major: "primitive"},
- {inputType: "UInt8", outputType: "NullableUInt8", runningType: "UInt8", major: "primitive"},
- {inputType: "NullableUInt8", outputType: "NullableUInt8", runningType: "UInt8", major: "primitive"},
- {inputType: "Int", outputType: "NullableInt", runningType: "Int", major: "primitive"},
- {inputType: "BigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "primitive"},
- {inputType: "NullableBit", outputType: "NullableBit", runningType: "Bit", major: "primitive"},
- {inputType: "NullableInt", outputType: "NullableInt", runningType: "Int", major: "primitive"},
- {inputType: "NullableBigInt", outputType: "NullableBigInt", runningType: "BigInt", major: "primitive"},
- {inputType: "Float4", outputType: "NullableFloat4", runningType: "Float4", major: "primitive"},
- {inputType: "Float8", outputType: "NullableFloat8", runningType: "Float8", major: "primitive"},
- {inputType: "NullableFloat4", outputType: "NullableFloat4", runningType: "Float4", major: "primitive"},
- {inputType: "NullableFloat8", outputType: "NullableFloat8", runningType: "Float8", major: "primitive"},
- {inputType: "Date", outputType: "NullableDate", runningType: "Date", major: "primitive"},
- {inputType: "NullableDate", outputType: "NullableDate", runningType: "Date", major: "primitive"},
- {inputType: "TimeStamp", outputType: "NullableTimeStamp", runningType: "TimeStamp", major: "primitive"},
- {inputType: "NullableTimeStamp", outputType: "NullableTimeStamp", runningType: "TimeStamp", major: "primitive"},
- {inputType: "Time", outputType: "NullableTime", runningType: "Time", major: "primitive"},
- {inputType: "NullableTime", outputType: "NullableTime", runningType: "Time", major: "primitive"},
- {inputType: "IntervalDay", outputType: "NullableIntervalDay", runningType: "IntervalDay", major: "IntervalDay"},
- {inputType: "NullableIntervalDay", outputType: "NullableIntervalDay", runningType: "IntervalDay", major: "IntervalDay"},
- {inputType: "IntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "primitive"},
- {inputType: "NullableIntervalYear", outputType: "NullableIntervalYear", runningType: "IntervalYear", major: "primitive"},
- {inputType: "Interval", outputType: "NullableInterval", runningType: "Interval", major: "Interval"},
- {inputType: "NullableInterval", outputType: "NullableInterval", runningType: "Interval", major: "Interval"},
- {inputType: "VarDecimal", outputType: "NullableVarDecimal", runningType: "VarDecimal", major: "VarDecimal"},
- {inputType: "NullableVarDecimal", outputType: "NullableVarDecimal", runningType: "VarDecimal", major: "VarDecimal"},
- {inputType: "VarChar", outputType: "NullableVarChar", runningType: "VarChar", major: "bytes"},
- {inputType: "NullableVarChar", outputType: "NullableVarChar", runningType: "VarChar", major: "bytes"},
- {inputType: "Var16Char", outputType: "NullableVar16Char", runningType: "Var16Char", major: "bytes"},
- {inputType: "NullableVar16Char", outputType: "NullableVar16Char", runningType: "Var16Char", major: "bytes"},
- {inputType: "VarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "bytes"},
- {inputType: "NullableVarBinary", outputType: "NullableVarBinary", runningType: "VarBinary", major: "bytes"}
- ]
-}
diff --git a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
index 59d3715..1b5068a 100644
--- a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
@@ -44,134 +44,137 @@ import org.apache.drill.exec.expr.holders.*;
@SuppressWarnings("unused")
public class ${aggrtype.className}Functions {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${aggrtype.className}Functions.class);
<#list aggrtype.types as type>
<#if type.major == "Numeric">
-@FunctionTemplate(name = "${aggrtype.funcName}", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
-public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc{
-
- @Param ${type.inputType}Holder in;
- @Workspace ${type.runningType}Holder value;
- @Workspace BigIntHolder nonNullCount;
- @Output ${type.outputType}Holder out;
-
- public void setup() {
- value = new ${type.runningType}Holder();
- nonNullCount = new BigIntHolder();
- nonNullCount.value = 0;
- <#if aggrtype.funcName == "sum" || aggrtype.funcName == "any_value">
- value.value = 0;
- <#elseif aggrtype.funcName == "min">
- <#if type.runningType?starts_with("Bit")>
+ @FunctionTemplate(name = "${aggrtype.funcName}",
+ scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc {
+ @Param ${type.inputType}Holder in;
+ @Workspace ${type.runningType}Holder value;
+ @Workspace BigIntHolder nonNullCount;
+ @Output ${type.outputType}Holder out;
+
+ public void setup() {
+ value = new ${type.runningType}Holder();
+ nonNullCount = new BigIntHolder();
+ nonNullCount.value = 0;
+ <#if aggrtype.funcName == "sum" || aggrtype.funcName == "any_value" || aggrtype.funcName == "single_value">
+ value.value = 0;
+ <#elseif aggrtype.funcName == "min">
+ <#if type.runningType?starts_with("Bit")>
value.value = 1;
- <#elseif type.runningType?starts_with("Int")>
- value.value = Integer.MAX_VALUE;
- <#elseif type.runningType?starts_with("BigInt")>
- value.value = Long.MAX_VALUE;
- <#elseif type.runningType?starts_with("Float4")>
- value.value = Float.NaN;
- <#elseif type.runningType?starts_with("Float8")>
- value.value = Double.NaN;
- </#if>
- <#elseif aggrtype.funcName == "max">
- <#if type.runningType?starts_with("Bit")>
+ <#elseif type.runningType?starts_with("Int")>
+ value.value = Integer.MAX_VALUE;
+ <#elseif type.runningType?starts_with("BigInt")>
+ value.value = Long.MAX_VALUE;
+ <#elseif type.runningType?starts_with("Float4")>
+ value.value = Float.NaN;
+ <#elseif type.runningType?starts_with("Float8")>
+ value.value = Double.NaN;
+ </#if>
+ <#elseif aggrtype.funcName == "max">
+ <#if type.runningType?starts_with("Bit")>
value.value = 0;
- <#elseif type.runningType?starts_with("Int")>
- value.value = Integer.MIN_VALUE;
- <#elseif type.runningType?starts_with("BigInt")>
- value.value = Long.MIN_VALUE;
- <#elseif type.runningType?starts_with("Float4")>
- value.value = -Float.MAX_VALUE;
- <#elseif type.runningType?starts_with("Float8")>
- value.value = -Double.MAX_VALUE;
- </#if>
- </#if>
-
- }
-
- @Override
- public void add() {
- <#if type.inputType?starts_with("Nullable")>
- sout: {
- if (in.isSet == 0) {
- // processing nullable input and the value is null, so don't do anything...
- break sout;
- }
- </#if>
- nonNullCount.value = 1;
- // For min/max functions: NaN is the biggest value,
- // Infinity is the second biggest value
- // -Infinity is the smallest value
- <#if aggrtype.funcName == "min">
- <#if type.inputType?contains("Float4")>
- if(!Float.isNaN(in.value)) {
- value.value = Float.isNaN(value.value) ? in.value : Math.min(value.value, in.value);
- }
- <#elseif type.inputType?contains("Float8")>
- if(!Double.isNaN(in.value)) {
- value.value = Double.isNaN(value.value) ? in.value : Math.min(value.value, in.value);
- }
+ <#elseif type.runningType?starts_with("Int")>
+ value.value = Integer.MIN_VALUE;
+ <#elseif type.runningType?starts_with("BigInt")>
+ value.value = Long.MIN_VALUE;
+ <#elseif type.runningType?starts_with("Float4")>
+ value.value = -Float.MAX_VALUE;
+ <#elseif type.runningType?starts_with("Float8")>
+ value.value = -Double.MAX_VALUE;
+ </#if>
+ </#if>
+ }
+
+ @Override
+ public void add() {
+ <#if type.inputType?starts_with("Nullable")>
+ sout: {
+ if (in.isSet == 0) {
+ // processing nullable input and the value is null, so don't do anything...
+ break sout;
+ }
+ </#if>
+ <#if aggrtype.funcName == "single_value">
+ if (nonNullCount.value > 0) {
+ throw org.apache.drill.common.exceptions.UserException.functionError()
+ .message("Input for single_value function has more than one row")
+ .build();
+ }
+ </#if>
+ nonNullCount.value = 1;
+ // For min/max functions: NaN is the biggest value,
+ // Infinity is the second biggest value
+ // -Infinity is the smallest value
+ <#if aggrtype.funcName == "min">
+ <#if type.inputType?contains("Float4")>
+ if(!Float.isNaN(in.value)) {
+ value.value = Float.isNaN(value.value) ? in.value : Math.min(value.value, in.value);
+ }
+ <#elseif type.inputType?contains("Float8")>
+ if(!Double.isNaN(in.value)) {
+ value.value = Double.isNaN(value.value) ? in.value : Math.min(value.value, in.value);
+ }
+ <#else>
+ value.value = Math.min(value.value, in.value);
+ </#if>
+ <#elseif aggrtype.funcName == "max">
+ value.value = Math.max(value.value, in.value);
+ <#elseif aggrtype.funcName == "sum">
+ value.value += in.value;
+ <#elseif aggrtype.funcName == "count">
+ value.value++;
+ <#elseif aggrtype.funcName == "any_value" || aggrtype.funcName == "single_value">
+ value.value = in.value;
<#else>
- value.value = Math.min(value.value, in.value);
- </#if>
- <#elseif aggrtype.funcName == "max">
- value.value = Math.max(value.value, in.value);
- <#elseif aggrtype.funcName == "sum">
- value.value += in.value;
- <#elseif aggrtype.funcName == "count">
- value.value++;
- <#elseif aggrtype.funcName == "any_value">
- value.value = in.value;
- <#else>
- // TODO: throw an error ?
- </#if>
- <#if type.inputType?starts_with("Nullable")>
- } // end of sout block
- </#if>
- }
+ // TODO: throw an error ?
+ </#if>
+ <#if type.inputType?starts_with("Nullable")>
+ } // end of sout block
+ </#if>
+ }
- @Override
- public void output() {
- if (nonNullCount.value > 0) {
- out.value = value.value;
- out.isSet = 1;
- } else {
- out.isSet = 0;
+ @Override
+ public void output() {
+ if (nonNullCount.value > 0) {
+ out.value = value.value;
+ out.isSet = 1;
+ } else {
+ out.isSet = 0;
+ }
}
- }
- @Override
- public void reset() {
- nonNullCount.value = 0;
- <#if aggrtype.funcName == "sum" || aggrtype.funcName == "count" || aggrtype.funcName == "any_value">
- value.value = 0;
- <#elseif aggrtype.funcName == "min">
- <#if type.runningType?starts_with("Int")>
- value.value = Integer.MAX_VALUE;
- <#elseif type.runningType?starts_with("BigInt")>
- value.value = Long.MAX_VALUE;
- <#elseif type.runningType?starts_with("Float4")>
- value.value = Float.NaN;
- <#elseif type.runningType?starts_with("Float8")>
- value.value = Double.NaN;
- </#if>
- <#elseif aggrtype.funcName == "max">
- <#if type.runningType?starts_with("Int")>
- value.value = Integer.MIN_VALUE;
- <#elseif type.runningType?starts_with("BigInt")>
- value.value = Long.MIN_VALUE;
- <#elseif type.runningType?starts_with("Float4")>
- value.value = -Float.MAX_VALUE;
- <#elseif type.runningType?starts_with("Float8")>
- value.value = -Double.MAX_VALUE;
- </#if>
- </#if>
-
+ @Override
+ public void reset() {
+ nonNullCount.value = 0;
+ <#if aggrtype.funcName == "sum" || aggrtype.funcName == "count" || aggrtype.funcName == "any_value" || aggrtype.funcName == "single_value">
+ value.value = 0;
+ <#elseif aggrtype.funcName == "min">
+ <#if type.runningType?starts_with("Int")>
+ value.value = Integer.MAX_VALUE;
+ <#elseif type.runningType?starts_with("BigInt")>
+ value.value = Long.MAX_VALUE;
+ <#elseif type.runningType?starts_with("Float4")>
+ value.value = Float.NaN;
+ <#elseif type.runningType?starts_with("Float8")>
+ value.value = Double.NaN;
+ </#if>
+ <#elseif aggrtype.funcName == "max">
+ <#if type.runningType?starts_with("Int")>
+ value.value = Integer.MIN_VALUE;
+ <#elseif type.runningType?starts_with("BigInt")>
+ value.value = Long.MIN_VALUE;
+ <#elseif type.runningType?starts_with("Float4")>
+ value.value = -Float.MAX_VALUE;
+ <#elseif type.runningType?starts_with("Float8")>
+ value.value = -Double.MAX_VALUE;
+ </#if>
+ </#if>
+ }
}
-
- }
</#if>
</#list>
diff --git a/exec/java-exec/src/main/codegen/templates/ComplexAggrFunctions1.java b/exec/java-exec/src/main/codegen/templates/ComplexAggrFunctions1.java
index 6aa92e3..562551d 100644
--- a/exec/java-exec/src/main/codegen/templates/ComplexAggrFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/ComplexAggrFunctions1.java
@@ -48,72 +48,82 @@ import org.apache.drill.exec.vector.complex.writer.BaseWriter.*;
@SuppressWarnings("unused")
public class ${aggrtype.className}ComplexFunctions {
-static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${aggrtype.className}ComplexFunctions.class);
<#list aggrtype.types as type>
<#if type.major == "Complex">
-@FunctionTemplate(name = "${aggrtype.funcName}", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
-public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc{
- @Param ${type.inputType}Holder inHolder;
- @Workspace BigIntHolder nonNullCount;
- @Output ComplexWriter writer;
-
- public void setup() {
- nonNullCount = new BigIntHolder();
- nonNullCount.value = 0;
- }
-
- @Override
- public void add() {
- <#if type.inputType?starts_with("Nullable")>
- sout: {
- if (inHolder.isSet == 0) {
- // processing nullable input and the value is null, so don't do anything...
- break sout;
- }
- </#if>
- <#if aggrtype.funcName == "any_value">
- <#if type.runningType?starts_with("Map")>
- if (nonNullCount.value == 0) {
- org.apache.drill.exec.expr.fn.impl.MappifyUtility.createMap(inHolder.reader, writer, "any_value");
- }
- <#elseif type.runningType?starts_with("RepeatedMap")>
- if (nonNullCount.value == 0) {
- org.apache.drill.exec.expr.fn.impl.MappifyUtility.createRepeatedMapOrList(inHolder.reader, writer, "any_value");
- }
- <#elseif type.runningType?starts_with("List")>
- if (nonNullCount.value == 0) {
- org.apache.drill.exec.expr.fn.impl.MappifyUtility.createList(inHolder.reader, writer, "any_value");
- }
- <#elseif type.runningType?starts_with("RepeatedList")>
- if (nonNullCount.value == 0) {
- org.apache.drill.exec.expr.fn.impl.MappifyUtility.createRepeatedMapOrList(inHolder.reader, writer, "any_value");
- }
- <#elseif type.runningType?starts_with("Repeated")>
- if (nonNullCount.value == 0) {
- org.apache.drill.exec.expr.fn.impl.MappifyUtility.createList(inHolder.reader, writer, "any_value");
+ @FunctionTemplate(name = "${aggrtype.funcName}",
+ <#if type.major == "VarDecimal">
+ returnType = FunctionTemplate.ReturnType.DECIMAL_AVG_AGGREGATE,
+ </#if>
+ scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+ public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc {
+ @Param ${type.inputType}Holder inHolder;
+ @Workspace BigIntHolder nonNullCount;
+ @Output ComplexWriter writer;
+
+ public void setup() {
+ nonNullCount = new BigIntHolder();
+ nonNullCount.value = 0;
}
+
+ @Override
+ public void add() {
+ <#if type.inputType?starts_with("Nullable")>
+ sout: {
+ if (inHolder.isSet == 0) {
+ // processing nullable input and the value is null, so don't do anything...
+ break sout;
+ }
</#if>
- </#if>
- nonNullCount.value = 1;
- <#if type.inputType?starts_with("Nullable")>
- } // end of sout block
- </#if>
- }
+ <#if aggrtype.funcName == "single_value">
+ if (nonNullCount.value > 0) {
+ throw org.apache.drill.common.exceptions.UserException.functionError()
+ .message("Input for single_value function has more than one row")
+ .build();
+ }
+ </#if>
+ <#if aggrtype.funcName == "any_value" || aggrtype.funcName == "single_value">
+ <#if type.runningType?starts_with("Map")>
+ if (nonNullCount.value == 0) {
+ org.apache.drill.exec.expr.fn.impl.MappifyUtility.createMap(inHolder.reader, writer, "${aggrtype.funcName}");
+ }
+ <#elseif type.runningType?starts_with("RepeatedMap")>
+ if (nonNullCount.value == 0) {
+ org.apache.drill.exec.expr.fn.impl.MappifyUtility.createRepeatedMapOrList(inHolder.reader, writer, "${aggrtype.funcName}");
+ }
+ <#elseif type.runningType?starts_with("List")>
+ if (nonNullCount.value == 0) {
+ org.apache.drill.exec.expr.fn.impl.MappifyUtility.createList(inHolder.reader, writer, "${aggrtype.funcName}");
+ }
+ <#elseif type.runningType?starts_with("RepeatedList")>
+ if (nonNullCount.value == 0) {
+ org.apache.drill.exec.expr.fn.impl.MappifyUtility.createRepeatedMapOrList(inHolder.reader, writer, "${aggrtype.funcName}");
+ }
+ <#elseif type.runningType?starts_with("Repeated")>
+ if (nonNullCount.value == 0) {
+ org.apache.drill.exec.expr.fn.impl.MappifyUtility.createList(inHolder.reader, writer, "${aggrtype.funcName}");
+ }
+ </#if>
+ </#if>
+ nonNullCount.value = 1;
+ <#if type.inputType?starts_with("Nullable")>
+ } // end of sout block
+ </#if>
+ }
- @Override
- public void output() {
- //Do nothing since the complex writer takes care of everything!
- }
+ @Override
+ public void output() {
+ //Do nothing since the complex writer takes care of everything!
+ }
- @Override
- public void reset() {
- <#if aggrtype.funcName == "any_value">
- nonNullCount.value = 0;
- </#if>
+ @Override
+ public void reset() {
+ <#if aggrtype.funcName == "any_value" || aggrtype.funcName == "single_value">
+ nonNullCount.value = 0;
+ </#if>
+ }
}
-}
</#if>
</#list>
}
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java b/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
index 8080ea7..f60a06a 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
@@ -49,7 +49,7 @@ public class ${aggrtype.className}DateTypeFunctions {
<#list aggrtype.types as type>
<#if type.major == "Date">
@FunctionTemplate(name = "${aggrtype.funcName}", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
-public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc{
+public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc {
@Param ${type.inputType}Holder in;
@Workspace ${type.runningType}Holder value;
@@ -81,6 +81,13 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
break sout;
}
</#if>
+ <#if aggrtype.funcName == "single_value">
+ if (nonNullCount.value > 0) {
+ throw org.apache.drill.common.exceptions.UserException.functionError()
+ .message("Input for single_value function has more than one row")
+ .build();
+ }
+ </#if>
nonNullCount.value = 1;
<#if aggrtype.funcName == "min">
@@ -131,7 +138,7 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
</#if>
<#elseif aggrtype.funcName == "count">
value.value++;
- <#elseif aggrtype.funcName == "any_value">
+ <#elseif aggrtype.funcName == "any_value" || aggrtype.funcName == "single_value">
<#if type.outputType?ends_with("Interval")>
value.days = in.days;
value.months = in.months;
@@ -139,6 +146,8 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
<#elseif type.outputType?ends_with("IntervalDay")>
value.days = in.days;
value.milliseconds = in.milliseconds;
+ <#else>
+ value.value = in.value;
</#if>
<#else>
// TODO: throw an error ?
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java
index 083a3cd..cd14bbb 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java
@@ -127,37 +127,7 @@ public class Decimal${aggrtype.className}Functions {
nonNullCount.value = 0;
}
}
- <#elseif aggrtype.funcName.contains("any_value") && type.inputType?starts_with("Repeated")>
- @FunctionTemplate(name = "${aggrtype.funcName}",
- scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE,
- returnType = FunctionTemplate.ReturnType.DECIMAL_AGGREGATE)
- public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc {
- @Param ${type.inputType}Holder in;
- @Output ComplexWriter writer;
- @Workspace BigIntHolder nonNullCount;
-
- public void setup() {
- nonNullCount = new BigIntHolder();
- }
-
- @Override
- public void add() {
- if (nonNullCount.value == 0) {
- org.apache.drill.exec.expr.fn.impl.MappifyUtility.createList(in.reader, writer, "any_value");
- }
- nonNullCount.value = 1;
- }
-
- @Override
- public void output() {
- }
-
- @Override
- public void reset() {
- nonNullCount.value = 0;
- }
- }
- <#elseif aggrtype.funcName.contains("any_value")>
+ <#elseif aggrtype.funcName.contains("any_value") || aggrtype.funcName.contains("single_value")>
@FunctionTemplate(name = "${aggrtype.funcName}",
scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE,
returnType = FunctionTemplate.ReturnType.DECIMAL_AGGREGATE)
@@ -190,6 +160,12 @@ public class Decimal${aggrtype.className}Functions {
.getBigDecimalFromDrillBuf(in.buffer,in.start,in.end-in.start,in.scale);
scale.value = in.scale;
precision.value = in.precision;
+ <#if aggrtype.funcName.contains("single_value")>
+ } else {
+ throw org.apache.drill.common.exceptions.UserException.functionError()
+ .message("Input for single_value function has more than one row")
+ .build();
+ </#if>
}
nonNullCount.value = 1;
<#if type.inputType?starts_with("Nullable")>
diff --git a/exec/java-exec/src/main/codegen/templates/SingleValueAgg.java b/exec/java-exec/src/main/codegen/templates/SingleValueAgg.java
deleted file mode 100644
index c0ff6cf..0000000
--- a/exec/java-exec/src/main/codegen/templates/SingleValueAgg.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * 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.
- */
-<@pp.dropOutputFile />
-
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gaggr/SingleValueFunctions.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl.gaggr;
-
-import org.apache.drill.exec.expr.DrillAggFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.annotations.Workspace;
-import org.apache.drill.exec.expr.holders.*;
-
-import javax.inject.Inject;
-import io.netty.buffer.DrillBuf;
-
-/*
- * This class is generated using freemarker and the ${.template_name} template.
- */
-@SuppressWarnings("unused")
-public class SingleValueFunctions {
-<#list singleValue.types as type>
-
- @FunctionTemplate(name = "single_value",
- <#if type.major == "VarDecimal">
- returnType = FunctionTemplate.ReturnType.DECIMAL_AVG_AGGREGATE,
- </#if>
- scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
- public static class ${type.inputType}SingleValue implements DrillAggFunc {
- @Param ${type.inputType}Holder in;
- @Workspace ${type.runningType}Holder value;
- @Output ${type.outputType}Holder out;
- @Workspace BigIntHolder nonNullCount;
- <#if type.major == "VarDecimal" || type.major == "bytes">
- @Inject DrillBuf buffer;
- </#if>
-
- public void setup() {
- nonNullCount = new BigIntHolder();
- nonNullCount.value = 0;
- value = new ${type.runningType}Holder();
- }
-
- @Override
- public void add() {
- <#if type.inputType?starts_with("Nullable")>
- sout: {
- if (in.isSet == 0) {
- // processing nullable input and the value is null, so don't do anything...
- break sout;
- }
- </#if>
- if (nonNullCount.value == 0) {
- nonNullCount.value = 1;
- } else {
- throw org.apache.drill.common.exceptions.UserException.functionError()
- .message("Input for single_value function has more than one row")
- .build();
- }
- <#if type.major == "primitive">
- value.value = in.value;
- <#elseif type.major == "IntervalDay">
- value.days = in.days;
- value.milliseconds = in.milliseconds;
- <#elseif type.major == "Interval">
- value.days = in.days;
- value.milliseconds = in.milliseconds;
- value.months = in.months;
- <#elseif type.major == "VarDecimal">
- value.start = in.start;
- value.end = in.end;
- value.buffer = in.buffer;
- value.scale = in.scale;
- value.precision = in.precision;
- <#elseif type.major == "bytes">
- value.start = in.start;
- value.end = in.end;
- value.buffer = in.buffer;
- </#if>
- <#if type.inputType?starts_with("Nullable")>
- } // end of sout block
- </#if>
- }
-
- @Override
- public void output() {
- if (nonNullCount.value > 0) {
- out.isSet = 1;
- <#if type.major == "primitive">
- out.value = value.value;
- <#elseif type.major == "IntervalDay">
- out.days = value.days;
- out.milliseconds = value.milliseconds;
- <#elseif type.major == "Interval">
- out.days = value.days;
- out.milliseconds = value.milliseconds;
- out.months = value.months;
- <#elseif type.major == "VarDecimal">
- out.start = value.start;
- out.end = value.end;
- out.buffer = buffer.reallocIfNeeded(value.end - value.start);
- out.buffer.writeBytes(value.buffer, value.start, value.end - value.start);
- out.scale = value.scale;
- out.precision = value.precision;
- <#elseif type.major == "bytes">
- out.start = value.start;
- out.end = value.end;
- out.buffer = buffer.reallocIfNeeded(value.end - value.start);
- out.buffer.writeBytes(value.buffer, value.start, value.end - value.start);
- </#if>
- } else {
- out.isSet = 0;
- }
- }
-
- @Override
- public void reset() {
- value = new ${type.runningType}Holder();
- nonNullCount.value = 0;
- }
- }
-</#list>
-}
-
diff --git a/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java b/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
index de5d705..f661851 100644
--- a/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
@@ -52,7 +52,6 @@ import io.netty.buffer.ByteBuf;
@SuppressWarnings("unused")
public class ${aggrtype.className}VarBytesFunctions {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${aggrtype.className}Functions.class);
<#list aggrtype.types as type>
<#if type.major == "VarBytes">
@@ -90,7 +89,7 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
break sout;
}
</#if>
- <#if aggrtype.className == "AnyValue">
+ <#if aggrtype.className == "AnyValue" || aggrtype.className == "SingleValue">
if (nonNullCount.value == 0) {
nonNullCount.value = 1;
int inputLength = in.end - in.start;
@@ -98,6 +97,12 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
byte[] tempArray = new byte[inputLength];
in.buffer.getBytes(in.start, tempArray, 0, inputLength);
tmp.setBytes(tempArray);
+ <#if aggrtype.className == "SingleValue">
+ } else {
+ throw org.apache.drill.common.exceptions.UserException.functionError()
+ .message("Input for single_value function has more than one row")
+ .build();
+ </#if>
}
<#else>
nonNullCount.value = 1;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
index a777276..cba86ed 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
@@ -130,12 +130,11 @@ public class PlanTestBase extends BaseTestQuery {
/**
* The same as above, but without excludedPatterns
*/
- public static void testPlanMatchingPatterns(String query, String[] expectedPatterns) throws Exception {
+ public static void testPlanMatchingPatterns(String query, String... expectedPatterns) throws Exception {
testPlanMatchingPatterns(query, expectedPatterns, null);
}
- private static Pattern[] stringsToPatterns(String[] strings)
- {
+ private static Pattern[] stringsToPatterns(String[] strings) {
if (strings == null) {
return null;
}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java b/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
index 00b5e61..3a60e1d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
@@ -1198,4 +1198,19 @@ public class TestExampleQueries extends BaseTestQuery {
.baselineValues("Nowmer", "ARGENTINA")
.go();
}
+
+ @Test
+ public void testSubQueryInProjectWithVarChar() throws Exception {
+ String query = "select n_name," +
+ "(select r.r_name from cp.`tpch/region.parquet` r where r.r_regionkey = n.n_regionkey) as r_name\n" +
+ "from cp.`tpch/nation.parquet` n order by n.n_name limit 1";
+ PlanTestBase.testPlanMatchingPatterns(query, "agg.*SINGLE_VALUE");
+
+ testBuilder()
+ .sqlQuery(query)
+ .unOrdered()
+ .baselineColumns("n_name", "r_name")
+ .baselineValues("ALGERIA", "AFRICA")
+ .go();
+ }
}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
index f1f74a6..5c3a433 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
@@ -18,6 +18,7 @@
package org.apache.drill.exec.fn.impl;
import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
import org.apache.commons.lang3.tuple.Pair;
@@ -38,6 +39,7 @@ import org.apache.drill.common.expression.SchemaPath;
import org.apache.drill.common.types.TypeProtos;
import org.apache.drill.exec.proto.UserBitShared;
import org.apache.drill.exec.rpc.user.QueryDataBatch;
+import org.apache.drill.test.TestBuilder;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Rule;
@@ -51,10 +53,13 @@ import java.io.FileWriter;
import java.math.BigDecimal;
import java.nio.file.Paths;
import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
import static org.hamcrest.CoreMatchers.containsString;
import static org.junit.Assert.assertTrue;
@@ -582,7 +587,7 @@ public class TestAggregateFunctions extends BaseTestQuery {
@Test
public void testSingleValueFunction() throws Exception {
- List<String> tableNames = ImmutableList.of(
+ List<String> tableNames = Arrays.asList(
"cp.`parquet/alltypes_required.parquet`",
"cp.`parquet/alltypes_optional.parquet`");
for (String tableName : tableNames) {
@@ -620,8 +625,7 @@ public class TestAggregateFunctions extends BaseTestQuery {
loader.clear();
result.release();
- String columnsList = columns.stream()
- .collect(Collectors.joining(", "));
+ String columnsList = String.join(", ", columns);
final List<Map<String, Object>> baselineRecords = new ArrayList<>();
baselineRecords.add(resultingValues);
@@ -640,11 +644,112 @@ public class TestAggregateFunctions extends BaseTestQuery {
}
@Test
- public void testSingleValueWithMultipleValuesInput() throws Exception {
+ public void testHashAggSingleValueFunction() throws Exception {
+ List<String> tableNames = Arrays.asList(
+ "cp.`parquet/alltypes_required.parquet`",
+ "cp.`parquet/alltypes_optional.parquet`");
+ for (String tableName : tableNames) {
+ Map<String, Object> resultingValues = getBaselineRecords(tableName);
+
+ List<Boolean> optionValues = Arrays.asList(true, false);
+
+ try {
+ for (Boolean optionValue : optionValues) {
+ for (Map.Entry<String, Object> entry : resultingValues.entrySet()) {
+ String columnName = String.format("`%s`", entry.getKey());
+
+ // disable interval types when stream agg is disabled due to DRILL-7241
+ if (optionValue || !columnName.startsWith("`col_intrvl")) {
+ setSessionOption(PlannerSettings.STREAMAGG.getOptionName(), optionValue);
+ testBuilder()
+ .sqlQuery("select single_value(t.%1$s) as %1$s\n" +
+ "from (select %1$s from %2$s limit 1) t group by t.%1$s", columnName, tableName)
+ .ordered()
+ .baselineRecords(Collections.singletonList(ImmutableMap.of(columnName, entry.getValue())))
+ .go();
+ }
+ }
+ }
+ } finally {
+ resetSessionOption(PlannerSettings.STREAMAGG.getOptionName());
+ }
+ }
+ }
+
+ private static Map<String, Object> getBaselineRecords(String tableName) throws Exception {
+ QueryDataBatch result =
+ testSqlWithResults(String.format("select * from %s limit 1", tableName)).get(0);
+
+ Map<String, Object> resultingValues = new HashMap<>();
+
+ RecordBatchLoader loader = new RecordBatchLoader(getAllocator());
+ loader.load(result.getHeader().getDef(), result.getData());
+
+ for (VectorWrapper<?> vectorWrapper : loader.getContainer()) {
+ String fieldName = vectorWrapper.getField().getName();
+ Object object = vectorWrapper.getValueVector().getAccessor().getObject(0);
+ // VarCharVector returns Text instance, but baseline values should contain String value
+ if (object instanceof Text) {
+ object = object.toString();
+ }
+ resultingValues.put(fieldName, object);
+ }
+ loader.clear();
+ result.release();
+ return resultingValues;
+ }
+
+ @Test
+ public void testSingleValueWithComplexInput() throws Exception {
+ String query = "select single_value(a) as any_a, single_value(f) as any_f, single_value(m) as any_m," +
+ "single_value(p) as any_p from (select * from cp.`store/json/test_anyvalue.json` limit 1)";
+ testBuilder()
+ .sqlQuery(query)
+ .unOrdered()
+ .baselineColumns("any_a", "any_f", "any_m", "any_p")
+ .baselineValues(TestBuilder.listOf(TestBuilder.mapOf("b", 10L, "c", 15L),
+ TestBuilder.mapOf("b", 20L, "c", 45L)),
+ TestBuilder.listOf(TestBuilder.mapOf("g", TestBuilder.mapOf("h",
+ TestBuilder.listOf(TestBuilder.mapOf("k", 10L), TestBuilder.mapOf("k", 20L))))),
+ TestBuilder.listOf(TestBuilder.mapOf("n", TestBuilder.listOf(1L, 2L, 3L))),
+ TestBuilder.mapOf("q", TestBuilder.listOf(27L, 28L, 29L)))
+ .go();
+ }
+
+ @Test
+ public void testSingleValueWithMultipleValuesInputsAllTypes() throws Exception {
+ List<String> tableNames = Arrays.asList(
+ "cp.`parquet/alltypes_required.parquet`",
+ "cp.`parquet/alltypes_optional.parquet`");
+ for (String tableName : tableNames) {
+ QueryDataBatch result =
+ testSqlWithResults(String.format("select * from %s limit 1", tableName)).get(0);
+
+ RecordBatchLoader loader = new RecordBatchLoader(getAllocator());
+ loader.load(result.getHeader().getDef(), result.getData());
+
+ List<String> columns = StreamSupport.stream(loader.getContainer().spliterator(), false)
+ .map(vectorWrapper -> vectorWrapper.getField().getName())
+ .collect(Collectors.toList());
+ loader.clear();
+ result.release();
+ for (String columnName : columns) {
+ try {
+ test("select single_value(t.%1$s) as %1$s from %2$s t", columnName, tableName);
+ } catch (UserRemoteException e) {
+ assertTrue("No expected current \"FUNCTION ERROR\" and/or \"Input for single_value function has more than one row\"",
+ e.getMessage().matches("^FUNCTION ERROR(.|\\n)*Input for single_value function has more than one row(.|\\n)*"));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testSingleValueWithMultipleComplexInputs() throws Exception {
thrown.expect(UserRemoteException.class);
thrown.expectMessage(containsString("FUNCTION ERROR"));
thrown.expectMessage(containsString("Input for single_value function has more than one row"));
- test("select single_value(n_name) from cp.`tpch/nation.parquet`");
+ test("select single_value(t1.a) from cp.`store/json/test_anyvalue.json` t1");
}
/*