Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id E8A74200C83 for ; Sat, 13 May 2017 19:39:04 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id E72F2160BBB; Sat, 13 May 2017 17:39:04 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 6E28A160BC9 for ; Sat, 13 May 2017 19:39:02 +0200 (CEST) Received: (qmail 85858 invoked by uid 500); 13 May 2017 17:39:01 -0000 Mailing-List: contact commits-help@drill.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: commits@drill.apache.org Delivered-To: mailing list commits@drill.apache.org Received: (qmail 85751 invoked by uid 99); 13 May 2017 17:38:59 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 13 May 2017 17:38:59 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id B8DF1E01C3; Sat, 13 May 2017 17:38:59 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: parthc@apache.org To: commits@drill.apache.org Date: Sat, 13 May 2017 17:39:02 -0000 Message-Id: <2d76f18b098b46cb902e383610cf5b5f@git.apache.org> In-Reply-To: <1a9c05ec8b9b4196ade2b44de95d26bf@git.apache.org> References: <1a9c05ec8b9b4196ade2b44de95d26bf@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [4/7] drill git commit: DRILL-5419: Calculate return string length for literals & some string functions archived-at: Sat, 13 May 2017 17:39:05 -0000 DRILL-5419: Calculate return string length for literals & some string functions 1. Revisited calculation logic for string literals and some string functions (cast, upper, lower, initcap, reverse, concat, concat operator, rpad, lpad, case statement, coalesce, first_value, last_value, lag, lead). Synchronized return type length calculation logic between limit 0 and regular queries. 2. Deprecated width and changed it to precision for string types in MajorType. 3. Revisited FunctionScope and splitted it into FunctionScope and ReturnType. FunctionScope will indicate only function usage in term of number of in / out rows, (n -> 1, 1 -> 1, 1->n). New annotation in UDFs ReturnType will indicate which return type strategy should be used. 4. Changed MAX_VARCHAR_LENGTH from 65536 to 65535. 5. Updated calculation of precision and display size for INTERVALYEAR & INTERVALDAY. 6. Refactored part of function code-gen logic (ValueReference, WorkspaceReference, FunctionAttributes, DrillFuncHolder). This closes #819 Project: http://git-wip-us.apache.org/repos/asf/drill/repo Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/6741e68a Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/6741e68a Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/6741e68a Branch: refs/heads/master Commit: 6741e68a4724365aadf50d5fd5d6990c95f386e7 Parents: 41ffed5 Author: Arina Ielchiieva Authored: Thu Apr 6 13:44:26 2017 +0300 Committer: Parth Chandra Committed: Fri May 12 17:06:49 2017 -0700 ---------------------------------------------------------------------- .../org/apache/drill/common/types/Types.java | 206 +++++++---- .../native/client/src/clientlib/fieldmeta.cpp | 10 +- .../drill/exec/store/hive/HiveUtilities.java | 20 +- .../apache/drill/exec/hive/TestHiveStorage.java | 41 ++- .../exec/hive/TestInfoSchemaOnHiveStorage.java | 5 +- .../main/codegen/templates/CastDateVarChar.java | 9 +- .../CastFunctionsSrcVarLenTargetVarLen.java | 7 +- .../templates/CastFunctionsTargetVarLen.java | 11 +- .../codegen/templates/CastIntervalVarChar.java | 19 +- .../templates/ConvertToNullableHolder.java | 16 +- .../Decimal/CastDecimalDenseDecimalSparse.java | 7 +- .../templates/Decimal/CastDecimalSimilar.java | 7 +- .../Decimal/CastDecimalSparseDecimalDense.java | 7 +- .../templates/Decimal/CastDecimalVarchar.java | 12 +- .../templates/Decimal/CastDownwardDecimal.java | 17 +- .../templates/Decimal/CastFloatDecimal.java | 7 +- .../templates/Decimal/CastIntDecimal.java | 7 +- .../templates/Decimal/CastSrcDecimalSimple.java | 17 +- .../templates/Decimal/CastVarCharDecimal.java | 22 +- .../Decimal/DecimalAggrTypeFunctions1.java | 7 +- .../Decimal/DecimalAggrTypeFunctions2.java | 6 +- .../templates/Decimal/DecimalFunctions.java | 253 +++++++++---- .../apache/drill/exec/expr/ClassGenerator.java | 2 +- .../drill/exec/expr/EvaluationVisitor.java | 2 +- .../exec/expr/ExpressionTreeMaterializer.java | 18 +- .../exec/expr/annotations/FunctionTemplate.java | 99 +++-- .../drill/exec/expr/fn/DrillAggFuncHolder.java | 73 ++-- .../exec/expr/fn/DrillBooleanOPHolder.java | 27 -- .../expr/fn/DrillComplexWriterFuncHolder.java | 10 +- .../exec/expr/fn/DrillDecimalAddFuncHolder.java | 76 ---- .../exec/expr/fn/DrillDecimalAggFuncHolder.java | 45 --- .../expr/fn/DrillDecimalCastFuncHolder.java | 62 ---- .../expr/fn/DrillDecimalDivScaleFuncHolder.java | 72 ---- .../expr/fn/DrillDecimalMaxScaleFuncHolder.java | 59 --- .../expr/fn/DrillDecimalModScaleFuncHolder.java | 76 ---- .../expr/fn/DrillDecimalSetScaleFuncHolder.java | 63 ---- .../expr/fn/DrillDecimalSumAggFuncHolder.java | 45 --- .../expr/fn/DrillDecimalSumScaleFuncHolder.java | 68 ---- .../fn/DrillDecimalZeroScaleFuncHolder.java | 58 --- .../drill/exec/expr/fn/DrillFuncHolder.java | 222 +++-------- .../exec/expr/fn/DrillSimpleFuncHolder.java | 18 +- .../drill/exec/expr/fn/FunctionAttributes.java | 53 ++- .../drill/exec/expr/fn/FunctionConverter.java | 58 +-- .../drill/exec/expr/fn/FunctionUtils.java | 48 +++ .../drill/exec/expr/fn/ValueReference.java | 81 ++++ .../drill/exec/expr/fn/WorkspaceReference.java | 64 ++++ .../drill/exec/expr/fn/impl/BitFunctions.java | 4 +- .../exec/expr/fn/impl/SimpleCastFunctions.java | 7 +- .../exec/expr/fn/impl/StringFunctions.java | 66 +++- .../fn/output/ConcatReturnTypeInference.java | 63 ++++ .../fn/output/DecimalReturnTypeInference.java | 369 +++++++++++++++++++ .../fn/output/DefaultReturnTypeInference.java | 65 ++++ .../expr/fn/output/PadReturnTypeInference.java | 57 +++ .../expr/fn/output/ReturnTypeInference.java | 33 ++ .../SameInOutLengthReturnTypeInference.java | 53 +++ .../output/StringCastReturnTypeInference.java | 57 +++ .../impl/project/ProjectRecordBatch.java | 9 +- .../impl/union/UnionAllRecordBatch.java | 42 +-- .../drill/exec/planner/logical/DrillOptiq.java | 47 ++- .../drill/exec/planner/sql/SqlConverter.java | 3 +- .../exec/planner/sql/TypeInferenceUtils.java | 169 ++++++--- .../planner/sql/handlers/FindLimit0Visitor.java | 37 +- .../apache/drill/exec/store/mock/ColumnDef.java | 5 +- .../work/prepare/PreparedStatementProvider.java | 5 +- .../impl/limit/TestEarlyLimit0Optimization.java | 31 +- .../work/prepare/PreparedStatementTestBase.java | 131 +++++++ .../TestLimit0VsRegularQueriesMetadata.java | 315 ++++++++++++++++ .../prepare/TestPreparedStatementProvider.java | 127 +------ .../jdbc/DatabaseMetaDataGetColumnsTest.java | 6 +- .../drill/jdbc/PreparedStatementTest.java | 6 +- .../jdbc/test/TestInformationSchemaColumns.java | 6 +- .../main/codegen/templates/BasicTypeHelper.java | 14 +- .../drill/exec/record/MaterializedField.java | 42 ++- .../drill/common/expression/parser/ExprParser.g | 6 +- .../expression/ExpressionStringBuilder.java | 4 +- .../drill/common/expression/IfExpression.java | 28 +- .../common/expression/ValueExpressions.java | 15 +- .../apache/drill/common/types/TypeProtos.java | 48 +-- protocol/src/main/protobuf/Types.proto | 4 +- 79 files changed, 2408 insertions(+), 1578 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/common/src/main/java/org/apache/drill/common/types/Types.java ---------------------------------------------------------------------- diff --git a/common/src/main/java/org/apache/drill/common/types/Types.java b/common/src/main/java/org/apache/drill/common/types/Types.java index 942b27c..692d8f5 100644 --- a/common/src/main/java/org/apache/drill/common/types/Types.java +++ b/common/src/main/java/org/apache/drill/common/types/Types.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -27,10 +27,14 @@ import org.apache.drill.common.types.TypeProtos.MajorType; import org.apache.drill.common.types.TypeProtos.MinorType; import com.google.protobuf.TextFormat; +import org.apache.drill.common.util.CoreDecimalUtility; public class Types { static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Types.class); + public static final int MAX_VARCHAR_LENGTH = 65535; + public static final int UNDEFINED = 0; + public static final MajorType NULL = required(MinorType.NULL); public static final MajorType LATE_BIND_TYPE = optional(MinorType.LATE); public static final MajorType REQUIRED_BIT = required(MinorType.BIT); @@ -41,8 +45,8 @@ public class Types { return toType.getMinorType() == MinorType.UNION; } - public static enum Comparability { - UNKNOWN, NONE, EQUAL, ORDERED; + public enum Comparability { + UNKNOWN, NONE, EQUAL, ORDERED } public static boolean isComplex(final MajorType type) { @@ -282,80 +286,79 @@ public class Types { public static int getJdbcDisplaySize(MajorType type) { if (type.getMode() == DataMode.REPEATED || type.getMinorType() == MinorType.LIST) { - return 0; + return UNDEFINED; } final int precision = getPrecision(type); - switch(type.getMinorType()) { - case BIT: return 1; // 1 digit - - case TINYINT: return 4; // sign + 3 digit - case SMALLINT: return 6; // sign + 5 digits - case INT: return 11; // sign + 10 digits - case BIGINT: return 20; // sign + 19 digits - - case UINT1: return 3; // 3 digits - case UINT2: return 5; // 5 digits - case UINT4: return 10; // 10 digits - case UINT8: return 19; // 19 digits + case BIT: return 1; // 1 digit - case FLOAT4: return 14; // sign + 7 digits + decimal point + E + 2 digits - case FLOAT8: return 24; // sign + 15 digits + decimal point + E + 3 digits - - case DECIMAL9: - case DECIMAL18: - case DECIMAL28DENSE: - case DECIMAL28SPARSE: - case DECIMAL38DENSE: - case DECIMAL38SPARSE: - case MONEY: return 2 + precision; // precision of the column plus a sign and a decimal point + case TINYINT: return 4; // sign + 3 digit + case SMALLINT: return 6; // sign + 5 digits + case INT: return 11; // sign + 10 digits + case BIGINT: return 20; // sign + 19 digits - case VARCHAR: - case FIXEDCHAR: - case VAR16CHAR: - case FIXED16CHAR: return precision; // number of characters + case UINT1: return 3; // 3 digits + case UINT2: return 5; // 5 digits + case UINT4: return 10; // 10 digits + case UINT8: return 19; // 19 digits - case VARBINARY: - case FIXEDBINARY: return 2 * precision; // each binary byte is represented as a 2digit hex number + case FLOAT4: return 14; // sign + 7 digits + decimal point + E + 2 digits + case FLOAT8: return 24; // sign + 15 digits + decimal point + E + 3 digits - case DATE: return 10; // yyyy-mm-dd - case TIME: - return precision > 0 - ? 9 + precision // hh-mm-ss.SSS - : 8; // hh-mm-ss - case TIMETZ: - return precision > 0 - ? 15 + precision // hh-mm-ss.SSS-zz:zz - : 14; // hh-mm-ss-zz:zz - case TIMESTAMP: - return precision > 0 - ? 20 + precision // yyyy-mm-ddThh:mm:ss.SSS - : 19; // yyyy-mm-ddThh:mm:ss - case TIMESTAMPTZ: - return precision > 0 - ? 26 + precision // yyyy-mm-ddThh:mm:ss.SSS:ZZ-ZZ - : 25; // yyyy-mm-ddThh:mm:ss-ZZ:ZZ + case DECIMAL9: + case DECIMAL18: + case DECIMAL28DENSE: + case DECIMAL28SPARSE: + case DECIMAL38DENSE: + case DECIMAL38SPARSE: + case MONEY: return 2 + precision; // precision of the column plus a sign and a decimal point - case INTERVALYEAR: - return precision > 0 - ? 5 + precision // P..Y12M - : 0; // if precision is not set, return 0 because there's not enough info + case VARCHAR: + case FIXEDCHAR: + case VAR16CHAR: + case FIXED16CHAR: return precision; // number of characters - case INTERVALDAY: - return precision > 0 - ? 12 + precision // P..DT12H60M60S assuming fractional seconds precision is not supported - : 0; // if precision is not set, return 0 because there's not enough info + case VARBINARY: + case FIXEDBINARY: return 2 * precision; // each binary byte is represented as a 2digit hex number - case INTERVAL: - case MAP: - case LATE: - case NULL: - case UNION: return 0; + case DATE: return 10; // yyyy-mm-dd + case TIME: + return precision > 0 + ? 9 + precision // hh-mm-ss.SSS + : 8; // hh-mm-ss + case TIMETZ: + return precision > 0 + ? 15 + precision // hh-mm-ss.SSS-zz:zz + : 14; // hh-mm-ss-zz:zz + case TIMESTAMP: + return precision > 0 + ? 20 + precision // yyyy-mm-ddThh:mm:ss.SSS + : 19; // yyyy-mm-ddThh:mm:ss + case TIMESTAMPTZ: + return precision > 0 + ? 26 + precision // yyyy-mm-ddThh:mm:ss.SSS:ZZ-ZZ + : 25; // yyyy-mm-ddThh:mm:ss-ZZ:ZZ + + case INTERVALYEAR: + return precision > 0 + ? 5 + precision // P..Y12M + : 9; // we assume max is P9999Y12M + + case INTERVALDAY: + return precision > 0 + ? 12 + precision // P..DT12H60M60S assuming fractional seconds precision is not supported + : 22; // the first 4 bytes give the number of days, so we assume max is P2147483648DT12H60M60S + + case INTERVAL: + case MAP: + case LATE: + case NULL: + case UNION: + return UNDEFINED; - default: - throw new UnsupportedOperationException( - "Unexpected/unhandled MinorType value " + type.getMinorType() ); + default: + throw new UnsupportedOperationException("Unexpected/unhandled MinorType value " + type.getMinorType()); } } public static boolean usesHolderForGet(final MajorType type) { @@ -399,7 +402,13 @@ public class Types { } - public static boolean isStringScalarType(final MajorType type) { + /** + * Checks if given major type is string scalar type. + * + * @param type major type + * @return true if given major type is scalar string, false otherwise + */ + public static boolean isScalarStringType(final MajorType type) { if (type.getMode() == REPEATED) { return false; } @@ -475,6 +484,18 @@ public class Types { return MajorType.newBuilder().setMode(mode).setMinorType(type).build(); } + /** + * Builds major type using given minor type, data mode and precision. + * + * @param type minor type + * @param mode data mode + * @param precision precision value + * @return major type + */ + public static MajorType withPrecision(final MinorType type, final DataMode mode, final int precision) { + return MajorType.newBuilder().setMinorType(type).setMode(mode).setPrecision(precision).build(); + } + public static MajorType withScaleAndPrecision(final MinorType type, final DataMode mode, final int scale, final int precision) { return MajorType.newBuilder().setMinorType(type).setMode(mode).setScale(scale).setPrecision(precision).build(); } @@ -636,43 +657,66 @@ public class Types { /** * Get the precision of given type. - * @param majorType - * @return + * + * @param majorType major type + * @return precision value */ public static int getPrecision(MajorType majorType) { - MinorType type = majorType.getMinorType(); - - if (type == MinorType.VARBINARY || type == MinorType.VARCHAR) { - return 65536; - } - if (majorType.hasPrecision()) { return majorType.getPrecision(); } - return 0; + return isScalarStringType(majorType) ? MAX_VARCHAR_LENGTH : UNDEFINED; } /** * Get the scale of given type. - * @param majorType - * @return + * + * @param majorType major type + * @return scale value */ public static int getScale(MajorType majorType) { if (majorType.hasScale()) { return majorType.getScale(); } - return 0; + return UNDEFINED; } /** - * Is the given type column be used in ORDER BY clause? - * @param type - * @return + * Checks if the given type column can be used in ORDER BY clause. + * + * @param type minor type + * @return true if type can be used in ORDER BY clause */ public static boolean isSortable(MinorType type) { // Currently only map and list columns are not sortable. return type != MinorType.MAP && type != MinorType.LIST; } + + /** + * Sets max precision from both types if these types are string scalar types. + * Sets max precision and scale from both types if these types are decimal types. + * Both types should be of the same minor type. + * + * @param leftType type from left side + * @param rightType type from right side + * @param typeBuilder type builder + * @return type builder + */ + public static MajorType.Builder calculateTypePrecisionAndScale(MajorType leftType, MajorType rightType, MajorType.Builder typeBuilder) { + if (leftType.getMinorType().equals(rightType.getMinorType())) { + boolean isScalarString = Types.isScalarStringType(leftType) && Types.isScalarStringType(rightType); + boolean isDecimal = CoreDecimalUtility.isDecimalType(leftType); + + if ((isScalarString || isDecimal) && leftType.hasPrecision() && rightType.hasPrecision()) { + typeBuilder.setPrecision(Math.max(leftType.getPrecision(), rightType.getPrecision())); + } + + if (isDecimal && leftType.hasScale() && rightType.hasScale()) { + typeBuilder.setScale(Math.max(leftType.getScale(), rightType.getScale())); + } + } + return typeBuilder; + } } http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/contrib/native/client/src/clientlib/fieldmeta.cpp ---------------------------------------------------------------------- diff --git a/contrib/native/client/src/clientlib/fieldmeta.cpp b/contrib/native/client/src/clientlib/fieldmeta.cpp index d9d6bd1..13e1134 100644 --- a/contrib/native/client/src/clientlib/fieldmeta.cpp +++ b/contrib/native/client/src/clientlib/fieldmeta.cpp @@ -247,12 +247,12 @@ static uint32_t getColumnSize(const std::string& type, uint32_t precision) { else if (type == SQLIntervalYearMonth) { return (precision > 0) ? 5 + precision // P..M31 - : 0; // if precision is not set, return 0 because there's not enough info + : 9; // we assume max is P9999Y12M } else if (type == SQLIntervalDaySecond) { return (precision > 0) ? 12 + precision // P..DT12H60M60....S - : 0; // if precision is not set, return 0 because there's not enough info + : 22; // the first 4 bytes give the number of days, so we assume max is P2147483648DT12H60M60S } else { return 0; @@ -267,7 +267,7 @@ static uint32_t getPrecision(const ::common::MajorType& type) { } if (minor_type == ::common::VARBINARY || minor_type == ::common::VARCHAR) { - return 65536; + return 65535; } return 0; @@ -336,12 +336,12 @@ static uint32_t getDisplaySize(const ::common::MajorType& type) { case ::common::INTERVALYEAR: return precision > 0 ? 5 + precision // P..Y12M - : 0; // if precision is not set, return 0 because there's not enough info + : 9; // we assume max is P9999Y12M case ::common::INTERVALDAY: return precision > 0 ? 12 + precision // P..DT12H60M60S assuming fractional seconds precision is not supported - : 0; // if precision is not set, return 0 because there's not enough info + : 22; // the first 4 bytes give the number of days, so we assume max is P2147483648DT12H60M60S default: // We don't know how to compute a display size, let's return 0 (unknown) http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveUtilities.java ---------------------------------------------------------------------- diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveUtilities.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveUtilities.java index 1d5e6bf..1e5ea6c 100644 --- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveUtilities.java +++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveUtilities.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -51,6 +51,7 @@ import org.apache.drill.exec.vector.ValueVector; import org.apache.drill.exec.work.ExecErrorConstants; import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.common.type.HiveVarchar; import org.apache.hadoop.hive.metastore.MetaStoreUtils; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; @@ -59,6 +60,7 @@ import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; import org.apache.hadoop.hive.ql.metadata.HiveUtils; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory; +import org.apache.hadoop.hive.serde2.typeinfo.BaseCharTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; @@ -294,10 +296,18 @@ public class HiveUtilities { MajorType.Builder typeBuilder = MajorType.newBuilder().setMinorType(minorType) .setMode(DataMode.OPTIONAL); // Hive columns (both regular and partition) could have null values - if (primitiveTypeInfo.getPrimitiveCategory() == PrimitiveCategory.DECIMAL) { - DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) primitiveTypeInfo; - typeBuilder.setPrecision(decimalTypeInfo.precision()) - .setScale(decimalTypeInfo.scale()).build(); + switch (primitiveTypeInfo.getPrimitiveCategory()) { + case CHAR: + case VARCHAR: + BaseCharTypeInfo baseCharTypeInfo = (BaseCharTypeInfo) primitiveTypeInfo; + typeBuilder.setPrecision(baseCharTypeInfo.getLength()); + break; + case DECIMAL: + DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) primitiveTypeInfo; + typeBuilder.setPrecision(decimalTypeInfo.getPrecision()).setScale(decimalTypeInfo.getScale()); + break; + default: + // do nothing, other primitive categories do not have precision or scale } return typeBuilder.build(); http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java ---------------------------------------------------------------------- diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java index 46691fb..e00314d 100644 --- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java +++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -18,10 +18,13 @@ package org.apache.drill.exec.hive; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; import org.apache.drill.common.exceptions.UserRemoteException; import org.apache.drill.exec.ExecConstants; import org.apache.drill.exec.planner.physical.PlannerSettings; +import org.apache.drill.exec.proto.UserProtos; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hive.common.type.HiveVarchar; import org.joda.time.DateTime; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -30,11 +33,15 @@ import org.junit.Test; import java.math.BigDecimal; import java.sql.Date; import java.sql.Timestamp; +import java.util.List; import java.util.Map; import static org.hamcrest.CoreMatchers.containsString; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; public class TestHiveStorage extends HiveTestBase { @BeforeClass @@ -510,6 +517,38 @@ public class TestHiveStorage extends HiveTestBase { .go(); } + @Test + public void testStringColumnsMetadata() throws Exception { + String query = "select varchar_field, char_field, string_field from hive.readtest"; + + Map expectedResult = Maps.newHashMap(); + expectedResult.put("varchar_field", 50); + expectedResult.put("char_field", 10); + expectedResult.put("string_field", HiveVarchar.MAX_VARCHAR_LENGTH); + + verifyColumnsMetadata(client.createPreparedStatement(query).get() + .getPreparedStatement().getColumnsList(), expectedResult); + + try { + test("alter session set `%s` = true", ExecConstants.EARLY_LIMIT0_OPT_KEY); + verifyColumnsMetadata(client.createPreparedStatement(String.format("select * from (%s) t limit 0", query)).get() + .getPreparedStatement().getColumnsList(), expectedResult); + } finally { + test("alter session reset `%s`", ExecConstants.EARLY_LIMIT0_OPT_KEY); + } + } + + private void verifyColumnsMetadata(List columnsList, Map expectedResult) { + for (UserProtos.ResultColumnMetadata columnMetadata : columnsList) { + assertTrue("Column should be present in result set", expectedResult.containsKey(columnMetadata.getColumnName())); + Integer expectedSize = expectedResult.get(columnMetadata.getColumnName()); + assertNotNull("Expected size should not be null", expectedSize); + assertEquals("Display size should match", expectedSize.intValue(), columnMetadata.getDisplaySize()); + assertEquals("Precision should match", expectedSize.intValue(), columnMetadata.getPrecision()); + assertTrue("Column should be nullable", columnMetadata.getIsNullable()); + } + } + @AfterClass public static void shutdownOptions() throws Exception { test(String.format("alter session set `%s` = false", PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY)); http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java ---------------------------------------------------------------------- diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java index fb4bb17..72bb810 100644 --- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java +++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -19,6 +19,7 @@ package org.apache.drill.exec.hive; import com.google.common.base.Strings; import org.apache.drill.TestBuilder; +import org.apache.hadoop.hive.common.type.HiveVarchar; import org.junit.Test; public class TestInfoSchemaOnHiveStorage extends HiveTestBase { @@ -195,7 +196,7 @@ public class TestInfoSchemaOnHiveStorage extends HiveTestBase { "NUMERIC_SCALE") .baselineValues("inttype", "INTEGER", null, 2, 32, 0) .baselineValues("decimaltype", "DECIMAL", null, 10, 38, 2) - .baselineValues("stringtype", "CHARACTER VARYING", 65535, null, null, null) + .baselineValues("stringtype", "CHARACTER VARYING", HiveVarchar.MAX_VARCHAR_LENGTH, null, null, null) .baselineValues("varchartype", "CHARACTER VARYING", 20, null, null, null) .baselineValues("chartype", "CHARACTER", 10, null, null, null) .go(); http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/CastDateVarChar.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/codegen/templates/CastDateVarChar.java b/exec/java-exec/src/main/codegen/templates/CastDateVarChar.java index 7005977..891666a 100644 --- a/exec/java-exec/src/main/codegen/templates/CastDateVarChar.java +++ b/exec/java-exec/src/main/codegen/templates/CastDateVarChar.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -52,8 +52,11 @@ import org.apache.drill.exec.expr.fn.impl.DateUtility; */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL, - costCategory = FunctionCostCategory.COMPLEX) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.STRING_CAST, + nulls = NullHandling.NULL_IF_NULL, + costCategory = FunctionCostCategory.COMPLEX) public class Cast${type.from}To${type.to} implements DrillSimpleFunc { @Param ${type.from}Holder in; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/CastFunctionsSrcVarLenTargetVarLen.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/codegen/templates/CastFunctionsSrcVarLenTargetVarLen.java b/exec/java-exec/src/main/codegen/templates/CastFunctionsSrcVarLenTargetVarLen.java index 4524f5b..a017614 100644 --- a/exec/java-exec/src/main/codegen/templates/CastFunctionsSrcVarLenTargetVarLen.java +++ b/exec/java-exec/src/main/codegen/templates/CastFunctionsSrcVarLenTargetVarLen.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -46,7 +46,10 @@ import io.netty.buffer.DrillBuf; */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + <#if type.to == 'VarChar'>returnType = FunctionTemplate.ReturnType.STRING_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc{ @Param ${type.from}Holder in; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/CastFunctionsTargetVarLen.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/codegen/templates/CastFunctionsTargetVarLen.java b/exec/java-exec/src/main/codegen/templates/CastFunctionsTargetVarLen.java index 2a318a6..9228f34 100644 --- a/exec/java-exec/src/main/codegen/templates/CastFunctionsTargetVarLen.java +++ b/exec/java-exec/src/main/codegen/templates/CastFunctionsTargetVarLen.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -46,7 +46,10 @@ import io.netty.buffer.DrillBuf; * This class is generated using freemarker and the ${.template_name} template. */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + <#if type.to == 'VarChar'>returnType = FunctionTemplate.ReturnType.STRING_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc{ @Param ${type.from}Holder in; @@ -58,11 +61,11 @@ public class Cast${type.from}${type.to} implements DrillSimpleFunc{ } public void eval() { - buffer = buffer.reallocIfNeeded((int) len.value); + buffer = buffer.reallocIfNeeded((int)len.value); String istr = (new ${type.javaType}(in.value)).toString(); out.buffer = buffer; out.start = 0; - out.end = Math.min((int)len.value, istr.length()); // truncate if target type has length smaller than that of input's string + out.end = Math.min((int)len.value, istr.length()); // truncate if target type has length smaller than that of input's string out.buffer.setBytes(0, istr.substring(0,out.end).getBytes()); } } http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/CastIntervalVarChar.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/codegen/templates/CastIntervalVarChar.java b/exec/java-exec/src/main/codegen/templates/CastIntervalVarChar.java index 6dedca7..43f9303 100644 --- a/exec/java-exec/src/main/codegen/templates/CastIntervalVarChar.java +++ b/exec/java-exec/src/main/codegen/templates/CastIntervalVarChar.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -50,7 +50,10 @@ import org.apache.drill.exec.expr.fn.impl.DateUtility; * This class is generated using freemarker and the ${.template_name} template. */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.STRING_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}To${type.to} implements DrillSimpleFunc { @Param ${type.from}Holder in; @@ -124,7 +127,10 @@ import org.joda.time.DateMidnight; import org.apache.drill.exec.expr.fn.impl.DateUtility; @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.STRING_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}To${type.to} implements DrillSimpleFunc { @Param ${type.from}Holder in; @@ -183,8 +189,11 @@ import javax.inject.Inject; import io.netty.buffer.DrillBuf; @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL, - costCategory = FunctionCostCategory.COMPLEX) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.STRING_CAST, + nulls = NullHandling.NULL_IF_NULL, + costCategory = FunctionCostCategory.COMPLEX) public class Cast${type.from}To${type.to} implements DrillSimpleFunc { @Param ${type.from}Holder in; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/ConvertToNullableHolder.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/codegen/templates/ConvertToNullableHolder.java b/exec/java-exec/src/main/codegen/templates/ConvertToNullableHolder.java index cb67251..7615bc5 100644 --- a/exec/java-exec/src/main/codegen/templates/ConvertToNullableHolder.java +++ b/exec/java-exec/src/main/codegen/templates/ConvertToNullableHolder.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -35,12 +35,14 @@ import org.apache.drill.exec.record.RecordBatch; * This class is generated using freemarker and the ${.template_name} template. */ - -<#if minor.class.startsWith("Decimal")> -@FunctionTemplate(name = "convertToNullable${minor.class?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, nulls = FunctionTemplate.NullHandling.INTERNAL) -<#else> -@FunctionTemplate(name = "convertToNullable${minor.class?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL) - +@FunctionTemplate(name = "convertToNullable${minor.class?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + <#if minor.class.startsWith("Decimal")> + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + <#elseif minor.class.startsWith("Var")> + returnType = FunctionTemplate.ReturnType.SAME_IN_OUT_LENGTH, + + nulls = FunctionTemplate.NullHandling.INTERNAL) public class ${className} implements DrillSimpleFunc { @Param ${minor.class}Holder input; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalDenseDecimalSparse.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalDenseDecimalSparse.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalDenseDecimalSparse.java index dc26889..d637dde 100644 --- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalDenseDecimalSparse.java +++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalDenseDecimalSparse.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -46,7 +46,10 @@ import java.nio.ByteBuffer; */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc{ @Param ${type.from}Holder in; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSimilar.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSimilar.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSimilar.java index e0fba0a..8b97b1a 100644 --- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSimilar.java +++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSimilar.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -49,7 +49,10 @@ import java.nio.ByteBuffer; */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc{ @Param ${type.from}Holder in; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSparseDecimalDense.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSparseDecimalDense.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSparseDecimalDense.java index 5ea8a8d..69010ef 100644 --- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSparseDecimalDense.java +++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSparseDecimalDense.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -47,7 +47,10 @@ import java.nio.ByteBuffer; */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc{ @Param ${type.from}Holder in; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java index c5ca782..239ea28 100644 --- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java +++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -51,7 +51,10 @@ import java.nio.ByteBuffer; */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.STRING_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc { @Param ${type.from}Holder in; @@ -136,7 +139,10 @@ import java.nio.ByteBuffer; */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.STRING_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc { @Param ${type.from}Holder in; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/Decimal/CastDownwardDecimal.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDownwardDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDownwardDecimal.java index 467405e..a1ed7c8 100644 --- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDownwardDecimal.java +++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDownwardDecimal.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -48,7 +48,10 @@ import java.nio.ByteBuffer; */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc { @Param ${type.from}Holder in; @@ -125,7 +128,10 @@ import java.nio.ByteBuffer; */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc { @Param ${type.from}Holder in; @@ -172,7 +178,10 @@ import java.nio.ByteBuffer; */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc { @Param ${type.from}Holder in; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/Decimal/CastFloatDecimal.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastFloatDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastFloatDecimal.java index 7d7ec35..95508a1 100644 --- a/exec/java-exec/src/main/codegen/templates/Decimal/CastFloatDecimal.java +++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastFloatDecimal.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -51,7 +51,10 @@ import java.nio.ByteBuffer; */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc { @Param ${type.from}Holder in; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java index eb01566..abacf49 100644 --- a/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java +++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -47,7 +47,10 @@ import java.nio.ByteBuffer; */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc { @Param ${type.from}Holder in; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java index fca6e99..541b40a 100644 --- a/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java +++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -49,7 +49,10 @@ import java.nio.ByteBuffer; */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc { @Param ${type.from}Holder in; @@ -157,7 +160,10 @@ import java.nio.ByteBuffer; */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc{ @Param ${type.from}Holder in; @@ -261,7 +267,10 @@ import java.nio.ByteBuffer; */ @SuppressWarnings("unused") -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc { @Param ${type.from}Holder in; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java index bb011cb..a4341e9 100644 --- a/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java +++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -54,10 +54,16 @@ import java.nio.ByteBuffer; @SuppressWarnings("unused") <#if type.major == "VarCharDecimalSimple"> -@FunctionTemplate(name ="cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc { <#elseif type.major == "EmptyStringVarCharDecimalSimple"> -@FunctionTemplate(name ="castEmptyString${type.from}To${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_CAST, nulls=NullHandling.INTERNAL) +@FunctionTemplate(name ="castEmptyString${type.from}To${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_CAST, + nulls = NullHandling.INTERNAL) public class CastEmptyString${type.from}To${type.to} implements DrillSimpleFunc { @Param ${type.from}Holder in; @@ -231,10 +237,16 @@ import java.nio.ByteBuffer; @SuppressWarnings("unused") <#if type.major == "VarCharDecimalComplex"> -@FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL) +@FunctionTemplate(name = "cast${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_CAST, + nulls = NullHandling.NULL_IF_NULL) public class Cast${type.from}${type.to} implements DrillSimpleFunc { <#elseif type.major == "EmptyStringVarCharDecimalComplex"> -@FunctionTemplate(name = "castEmptyString${type.from}To${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_CAST, nulls=NullHandling.INTERNAL) +@FunctionTemplate(name = "castEmptyString${type.from}To${type.to?upper_case}", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_CAST, + nulls = NullHandling.INTERNAL) public class CastEmptyString${type.from}To${type.to} implements DrillSimpleFunc { @Param ${type.from}Holder in; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java ---------------------------------------------------------------------- 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 7737356..8e85777 100644 --- a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java +++ b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -60,7 +60,10 @@ public class Decimal${aggrtype.className}Functions { <#list aggrtype.types as type> -@FunctionTemplate(name = "${aggrtype.funcName}", <#if aggrtype.funcName == "sum"> scope = FunctionTemplate.FunctionScope.DECIMAL_SUM_AGGREGATE <#else>scope = FunctionTemplate.FunctionScope.DECIMAL_AGGREGATE) +@FunctionTemplate(name = "${aggrtype.funcName}", + scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE, + <#if aggrtype.funcName == "sum"> returnType = FunctionTemplate.ReturnType.DECIMAL_SUM_AGGREGATE + <#else>returnType = FunctionTemplate.ReturnType.DECIMAL_AGGREGATE) public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc{ @Param ${type.inputType}Holder in; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions2.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions2.java b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions2.java index f885e08..8729a31 100644 --- a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions2.java +++ b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions2.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -59,7 +59,9 @@ import io.netty.buffer.ByteBuf; public class Decimal${aggrtype.className}Functions { <#list aggrtype.types as type> -@FunctionTemplate(name = "${aggrtype.funcName}", scope = FunctionTemplate.FunctionScope.DECIMAL_SUM_AGGREGATE) +@FunctionTemplate(name = "${aggrtype.funcName}", + scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE, + returnType = FunctionTemplate.ReturnType.DECIMAL_SUM_AGGREGATE) public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc{ @Param ${type.inputType}Holder in; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java index a4bf06a..6197b06 100644 --- a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java +++ b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -133,7 +133,11 @@ import java.nio.ByteBuffer; @SuppressWarnings("unused") public class ${type.name}Functions { - @FunctionTemplate(name = "subtract", scope = FunctionTemplate.FunctionScope.DECIMAL_ADD_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(name = "subtract", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_ADD_SCALE, + nulls = NullHandling.NULL_IF_NULL, + checkPrecisionRange = true) public static class ${type.name}SubtractFunction implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -171,7 +175,11 @@ public class ${type.name}Functions { } } - @FunctionTemplate(name = "add", scope = FunctionTemplate.FunctionScope.DECIMAL_ADD_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(name = "add", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_ADD_SCALE, + nulls = NullHandling.NULL_IF_NULL, + checkPrecisionRange = true) public static class ${type.name}AddFunction implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -209,7 +217,11 @@ public class ${type.name}Functions { } } - @FunctionTemplate(name = "multiply", scope = FunctionTemplate.FunctionScope.DECIMAL_MUL_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(name = "multiply", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_SUM_SCALE, + nulls = NullHandling.NULL_IF_NULL, + checkPrecisionRange = true) public static class ${type.name}MultiplyFunction implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -379,7 +391,11 @@ public class ${type.name}Functions { } } - @FunctionTemplate(name = "exact_divide", scope = FunctionTemplate.FunctionScope.DECIMAL_DIV_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(name = "exact_divide", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_DIV_SCALE, + nulls = NullHandling.NULL_IF_NULL, + checkPrecisionRange = true) public static class ${type.name}DivideFunction implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -416,7 +432,11 @@ public class ${type.name}Functions { } } - @FunctionTemplate(name = "mod", scope = FunctionTemplate.FunctionScope.DECIMAL_MOD_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(name = "mod", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MOD_SCALE, + nulls = NullHandling.NULL_IF_NULL, + checkPrecisionRange = true) public static class ${type.name}ModFunction implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -454,7 +474,10 @@ public class ${type.name}Functions { } } - @FunctionTemplate(name = "abs", scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(name = "abs", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}AbsFunction implements DrillSimpleFunc { @Param ${type.name}Holder in; @@ -500,7 +523,10 @@ public class ${type.name}Functions { } } - @FunctionTemplate(names = {"ceil", "ceiling"}, scope = FunctionTemplate.FunctionScope.DECIMAL_ZERO_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(names = {"ceil", "ceiling"}, + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_ZERO_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}CeilFunction implements DrillSimpleFunc { @Param ${type.name}Holder in; @@ -566,7 +592,10 @@ public class ${type.name}Functions { } } - @FunctionTemplate(name = "floor", scope = FunctionTemplate.FunctionScope.DECIMAL_ZERO_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(name = "floor", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_ZERO_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}FloorFunction implements DrillSimpleFunc { @Param ${type.name}Holder in; @@ -631,7 +660,10 @@ public class ${type.name}Functions { } } - @FunctionTemplate(names = {"trunc", "truncate"}, scope = FunctionTemplate.FunctionScope.DECIMAL_ZERO_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(names = {"trunc", "truncate"}, + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_ZERO_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}TruncateFunction implements DrillSimpleFunc { @Param ${type.name}Holder in; @@ -665,7 +697,10 @@ public class ${type.name}Functions { } } - @FunctionTemplate(names = {"trunc", "truncate"}, scope = FunctionTemplate.FunctionScope.DECIMAL_SET_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(names = {"trunc", "truncate"}, + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_SET_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}TruncateScaleFunction implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -742,7 +777,10 @@ public class ${type.name}Functions { } } - @FunctionTemplate(name = "round", scope = FunctionTemplate.FunctionScope.DECIMAL_ZERO_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(name = "round", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_ZERO_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}RoundFunction implements DrillSimpleFunc { @Param ${type.name}Holder in; @@ -804,7 +842,10 @@ public class ${type.name}Functions { } } - @FunctionTemplate(name = "round", scope = FunctionTemplate.FunctionScope.DECIMAL_SET_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(name = "round", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_SET_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}RoundScaleFunction implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -834,8 +875,9 @@ public class ${type.name}Functions { <#-- Comparison function for sorting and grouping relational operators (not for comparison expression operators (=, <, etc.)). --> @FunctionTemplate(name = FunctionGenerationHelper.COMPARE_TO_NULLS_HIGH, - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.INTERNAL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.INTERNAL) public static class GCompare${leftType}Vs${rightType}NullHigh implements DrillSimpleFunc { @Param ${leftType}Holder left; @@ -854,8 +896,9 @@ public class ${type.name}Functions { <#-- Comparison function for sorting and grouping relational operators (not for comparison expression operators (=, <, etc.)). --> @FunctionTemplate(name = FunctionGenerationHelper.COMPARE_TO_NULLS_LOW, - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.INTERNAL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.INTERNAL) public static class GCompare${leftType}Vs${rightType}NullLow implements DrillSimpleFunc { @Param ${leftType}Holder left; @@ -876,8 +919,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "less than", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}LessThan implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -898,8 +942,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "less than or equal to", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}LessThanEq implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -918,8 +963,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "greater than", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}GreaterThan implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -938,8 +984,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "greater than or equal to", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}GreaterThanEq implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -958,8 +1005,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "Equal", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}Equal implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -978,8 +1026,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "not equal", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}NotEqual implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1033,8 +1082,9 @@ public class ${type.name}Functions { <#-- Comparison function for sorting and grouping relational operators (not for comparison expression operators (=, <, etc.)). --> @FunctionTemplate(name = FunctionGenerationHelper.COMPARE_TO_NULLS_HIGH, - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.INTERNAL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.INTERNAL) public static class GCompare${leftType}Vs${rightType}NullHigh implements DrillSimpleFunc { @Param ${leftType}Holder left; @@ -1056,8 +1106,9 @@ public class ${type.name}Functions { <#-- Comparison function for sorting and grouping relational operators (not for comparison expression operators (=, <, etc.)). --> @FunctionTemplate(name = FunctionGenerationHelper.COMPARE_TO_NULLS_LOW, - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.INTERNAL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.INTERNAL) public static class GCompare${leftType}Vs${rightType}NullLow implements DrillSimpleFunc { @Param ${leftType}Holder left; @@ -1083,8 +1134,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "less than", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}LessThan implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1101,8 +1153,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "less than or equal to", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}LessThanEq implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1119,8 +1172,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "greater than", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}GreaterThan implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1137,8 +1191,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "greater than or equal to", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}GreaterThanEq implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1155,8 +1210,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "Equal", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}Equal implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1174,8 +1230,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "not equal", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}NotEqual implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1218,7 +1275,11 @@ import java.nio.ByteBuffer; @SuppressWarnings("unused") public class ${type.name}Functions { - @FunctionTemplate(name = "add", scope = FunctionTemplate.FunctionScope.DECIMAL_ADD_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(name = "add", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_ADD_SCALE, + nulls = NullHandling.NULL_IF_NULL, + checkPrecisionRange = true) public static class ${type.name}AddFunction implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1247,7 +1308,11 @@ public class ${type.name}Functions { } } - @FunctionTemplate(name = "subtract", scope = FunctionTemplate.FunctionScope.DECIMAL_ADD_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(name = "subtract", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_ADD_SCALE, + nulls = NullHandling.NULL_IF_NULL, + checkPrecisionRange = true) public static class ${type.name}SubtractFunction implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1274,7 +1339,12 @@ public class ${type.name}Functions { result.scale = outputScale; } } - @FunctionTemplate(name = "multiply", scope = FunctionTemplate.FunctionScope.DECIMAL_MUL_SCALE, nulls = NullHandling.NULL_IF_NULL) + + @FunctionTemplate(name = "multiply", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_SUM_SCALE, + nulls = NullHandling.NULL_IF_NULL, + checkPrecisionRange = true) public static class ${type.name}MultiplyFunction implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1300,7 +1370,10 @@ public class ${type.name}Functions { } } - @FunctionTemplate(name = "abs", scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(name = "abs", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}AbsFunction implements DrillSimpleFunc { @Param ${type.name}Holder in; @@ -1320,7 +1393,11 @@ public class ${type.name}Functions { } } - @FunctionTemplate(name = "exact_divide", scope = FunctionTemplate.FunctionScope.DECIMAL_DIV_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(name = "exact_divide", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_DIV_SCALE, + nulls = NullHandling.NULL_IF_NULL, + checkPrecisionRange = true) public static class ${type.name}DivideFunction implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1353,7 +1430,11 @@ public class ${type.name}Functions { } } - @FunctionTemplate(name = "mod", scope = FunctionTemplate.FunctionScope.DECIMAL_MOD_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(name = "mod", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MOD_SCALE, + nulls = NullHandling.NULL_IF_NULL, + checkPrecisionRange = true) public static class ${type.name}ModFunction implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1399,7 +1480,10 @@ public class ${type.name}Functions { } } - @FunctionTemplate(names = {"trunc", "truncate"}, scope = FunctionTemplate.FunctionScope.DECIMAL_ZERO_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(names = {"trunc", "truncate"}, + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_ZERO_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}TruncFunction implements DrillSimpleFunc { @Param ${type.name}Holder in; @@ -1415,7 +1499,10 @@ public class ${type.name}Functions { } } - @FunctionTemplate(names = {"trunc", "truncate"}, scope = FunctionTemplate.FunctionScope.DECIMAL_SET_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(names = {"trunc", "truncate"}, + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_SET_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}TruncateScaleFunction implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1432,7 +1519,10 @@ public class ${type.name}Functions { } } - @FunctionTemplate(names = {"ceil", "ceiling"}, scope = FunctionTemplate.FunctionScope.DECIMAL_ZERO_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(names = {"ceil", "ceiling"}, + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_ZERO_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}CeilFunction implements DrillSimpleFunc { @Param ${type.name}Holder in; @@ -1459,7 +1549,10 @@ public class ${type.name}Functions { } } - @FunctionTemplate(name = "floor", scope = FunctionTemplate.FunctionScope.DECIMAL_ZERO_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(name = "floor", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_ZERO_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}FloorFunction implements DrillSimpleFunc { @Param ${type.name}Holder in; @@ -1484,7 +1577,10 @@ public class ${type.name}Functions { } } - @FunctionTemplate(name = "round", scope = FunctionTemplate.FunctionScope.DECIMAL_ZERO_SCALE, nulls = NullHandling.NULL_IF_NULL) + @FunctionTemplate(name = "round", + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_ZERO_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}RoundFunction implements DrillSimpleFunc { @Param ${type.name}Holder in; @@ -1518,8 +1614,9 @@ public class ${type.name}Functions { } @FunctionTemplate(name = "round", - scope = FunctionTemplate.FunctionScope.DECIMAL_SET_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_SET_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}RoundScaleFunction implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1577,8 +1674,9 @@ public class ${type.name}Functions { <#-- Comparison function for sorting and grouping relational operators (not for comparison expression operators (=, <, etc.)). --> @FunctionTemplate(name = FunctionGenerationHelper.COMPARE_TO_NULLS_HIGH, - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.INTERNAL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.INTERNAL) public static class GCompare${leftType}Vs${rightType}NullHigh implements DrillSimpleFunc { @Param ${leftType}Holder left; @@ -1601,8 +1699,9 @@ public class ${type.name}Functions { <#-- Comparison function for sorting and grouping relational operators (not for comparison expression operators (=, <, etc.)). --> @FunctionTemplate(name = FunctionGenerationHelper.COMPARE_TO_NULLS_LOW, - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.INTERNAL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.INTERNAL) public static class GCompare${leftType}Vs${rightType}NullLow implements DrillSimpleFunc { @Param ${leftType}Holder left; @@ -1627,8 +1726,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "less than", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}LessThan implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1645,8 +1745,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "less than or equal to", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}LessThanEq implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1663,8 +1764,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "greater than", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}GreaterThan implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1681,8 +1783,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "greater than or equal to", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}GreaterThanEq implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1699,8 +1802,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "Equal", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}Equal implements DrillSimpleFunc { @Param ${type.name}Holder left; @@ -1718,8 +1822,9 @@ public class ${type.name}Functions { <#-- Comparison function for comparison expression operator (=, <, etc.), not for sorting and grouping relational operators.) --> @FunctionTemplate(name = "not equal", - scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, - nulls = NullHandling.NULL_IF_NULL) + scope = FunctionTemplate.FunctionScope.SIMPLE, + returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE, + nulls = NullHandling.NULL_IF_NULL) public static class ${type.name}NotEqual implements DrillSimpleFunc { @Param ${type.name}Holder left; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java index c2ca492..c94bed5 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java @@ -36,7 +36,7 @@ import org.apache.drill.exec.compile.sig.GeneratorMapping; import org.apache.drill.exec.compile.sig.MappingSet; import org.apache.drill.exec.compile.sig.SignatureHolder; import org.apache.drill.exec.exception.SchemaChangeException; -import org.apache.drill.exec.expr.fn.DrillFuncHolder.WorkspaceReference; +import org.apache.drill.exec.expr.fn.WorkspaceReference; import org.apache.drill.exec.record.TypedFieldId; import com.google.common.base.Preconditions; http://git-wip-us.apache.org/repos/asf/drill/blob/6741e68a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java ---------------------------------------------------------------------- diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java index 73a0363..f299df2 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java @@ -572,7 +572,7 @@ public class EvaluationVisitor { @Override public HoldingContainer visitQuotedStringConstant(QuotedString e, ClassGenerator generator) throws RuntimeException { - MajorType majorType = Types.required(MinorType.VARCHAR); + MajorType majorType = e.getMajorType(); JBlock setup = generator.getBlock(BlockType.SETUP); JType holderType = generator.getHolderType(majorType); JVar var = generator.declareClassField("string", holderType);