You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by vo...@apache.org on 2018/05/04 17:30:55 UTC

[drill] 02/02: DRILL-6094: Decimal data type enhancements

This is an automated email from the ASF dual-hosted git repository.

volodymyr pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 4c4953bcab4886be14fc9b7f95a77caa86a7629f
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Thu Apr 5 15:35:42 2018 +0300

    DRILL-6094: Decimal data type enhancements
    
    Add ExprVisitors for VARDECIMAL
    Modify writers/readers to support VARDECIMAL
    - Added usage of VarDecimal for parquet, hive, maprdb, jdbc;
    - Added options to store decimals as int32 and int64 or fixed_len_byte_array or binary;
    Add UDFs for VARDECIMAL data type
    - modify type inference rules
    - remove UDFs for obsolete DECIMAL types
    Enable DECIMAL data type by default
    Add unit tests for DECIMAL data type
    Fix mapping for NLJ when literal with non-primitive type is used in join conditions
    Refresh protobuf C++ source files
    Changes in C++ files
    Add support for decimal logical type in Avro.
    Add support for date, time and timestamp logical types.
    Update Avro version to 1.8.2.
---
 .../java/org/apache/drill/common/types/Types.java  |    7 +-
 .../drill/common/util/CoreDecimalUtility.java      |   47 +-
 .../util/DecimalScalePrecisionDivideFunction.java  |   62 -
 .../mapr/db/json/CompareFunctionsProcessor.java    |   11 +-
 .../store/mapr/db/json/MaprDBJsonRecordReader.java |   11 +-
 .../native/client/src/clientlib/decimalUtils.cpp   |   15 +
 contrib/native/client/src/clientlib/fieldmeta.cpp  |    3 +
 contrib/native/client/src/clientlib/metadata.cpp   |   51 +
 .../native/client/src/clientlib/recordBatch.cpp    |    4 +
 .../client/src/include/drill/decimalUtils.hpp      |    1 +
 .../client/src/include/drill/protobuf/Types.pb.h   |    5 +-
 .../client/src/include/drill/recordBatch.hpp       |   55 +-
 contrib/native/client/src/protobuf/Types.pb.cc     |   11 +-
 .../core/src/main/codegen/data/HiveTypes.tdd       |    2 +-
 .../main/codegen/templates/ObjectInspectors.java   |   44 +-
 .../drill/exec/expr/fn/HiveFunctionRegistry.java   |    5 +-
 .../drill/exec/store/hive/HiveFieldConverter.java  |  112 +-
 .../drill/exec/store/hive/HiveUtilities.java       |   14 +-
 .../store/hive/readers/HiveAbstractReader.java     |    2 +-
 contrib/storage-hive/hive-exec-shade/pom.xml       |    4 +
 .../drill/exec/store/jdbc/JdbcRecordReader.java    |   84 +-
 .../store/mongo/MongoCompareFunctionProcessor.java |   13 +-
 exec/java-exec/pom.xml                             |   49 +-
 exec/java-exec/src/main/codegen/config.fmpp        |    1 +
 exec/java-exec/src/main/codegen/data/Casts.tdd     |  138 +-
 .../main/codegen/data/ComparisonTypesDecimal.tdd   |    4 +-
 .../src/main/codegen/data/CountAggrTypes.tdd       |    2 +
 .../src/main/codegen/data/DecimalAggrTypes1.tdd    |   49 +-
 .../src/main/codegen/data/DecimalAggrTypes2.tdd    |   12 +-
 .../src/main/codegen/data/DecimalAggrTypes3.tdd    |   43 +
 exec/java-exec/src/main/codegen/data/MathFunc.tdd  |   24 +-
 .../src/main/codegen/data/NumericTypes.tdd         |    3 +-
 .../src/main/codegen/templates/CastHigh.java       |   18 +-
 .../ToTimeStampFunction.java                       |   48 +-
 .../Decimal/CastDecimalDenseDecimalSparse.java     |  170 --
 .../templates/Decimal/CastDecimalFloat.java        |   69 +-
 .../codegen/templates/Decimal/CastDecimalInt.java  |   88 +-
 .../Decimal/CastDecimalSparseDecimalDense.java     |  185 --
 ...eVarDecimal.java => CastDecimalVarDecimal.java} |   59 +-
 .../templates/Decimal/CastDecimalVarchar.java      |  203 +-
 .../templates/Decimal/CastDownwardDecimal.java     |  209 --
 .../templates/Decimal/CastFloatDecimal.java        |   87 +-
 .../codegen/templates/Decimal/CastIntDecimal.java  |  100 +-
 .../templates/Decimal/CastSrcDecimalSimple.java    |  294 ---
 .../templates/Decimal/CastVarCharDecimal.java      |  513 +---
 ...imalSimilar.java => CastVarDecimalDecimal.java} |   89 +-
 .../Decimal/DecimalAggrTypeFunctions1.java         |  356 ++-
 .../Decimal/DecimalAggrTypeFunctions2.java         |   42 +-
 .../Decimal/DecimalAggrTypeFunctions3.java         |  184 ++
 .../templates/Decimal/DecimalFunctions.java        | 2534 +++-----------------
 .../src/main/codegen/templates/MathFunctions.java  |   10 -
 .../codegen/templates/NumericToCharFunctions.java  |   65 +-
 .../templates/ParquetOutputRecordWriter.java       |   44 +-
 .../main/codegen/templates/ParquetTypeHelper.java  |   29 +-
 .../src/main/codegen/templates/SqlAccessors.java   |   31 +-
 .../codegen/templates/VarCharAggrFunctions1.java   |    7 +-
 .../java/org/apache/drill/exec/ExecConstants.java  |   10 +
 .../compile/bytecode/ScalarReplacementTypes.java   |    4 +
 .../compile/sig/ConstantExpressionIdentifier.java  |    6 +
 .../org/apache/drill/exec/expr/CloneVisitor.java   |    6 +
 .../apache/drill/exec/expr/EqualityVisitor.java    |   17 +-
 .../apache/drill/exec/expr/EvaluationVisitor.java  |   48 +-
 .../exec/expr/ExpressionTreeMaterializer.java      |   62 +-
 .../apache/drill/exec/expr/GetSetVectorHelper.java |   32 +-
 .../org/apache/drill/exec/expr/HashVisitor.java    |   11 +-
 .../exec/expr/annotations/FunctionTemplate.java    |   10 +-
 .../drill/exec/expr/fn/impl/Hash32AsDouble.java    |  161 +-
 .../drill/exec/expr/fn/impl/Hash32Functions.java   |  175 +-
 .../exec/expr/fn/impl/Hash32FunctionsWithSeed.java |  153 +-
 .../exec/expr/fn/impl/Hash32WithSeedAsDouble.java  |  178 +-
 .../drill/exec/expr/fn/impl/Hash64AsDouble.java    |  189 +-
 .../drill/exec/expr/fn/impl/Hash64Functions.java   |  163 +-
 .../exec/expr/fn/impl/Hash64FunctionsWithSeed.java |  139 +-
 .../exec/expr/fn/impl/Hash64WithSeedAsDouble.java  |  159 +-
 .../expr/fn/interpreter/InterpreterEvaluator.java  |    7 +
 .../expr/fn/output/DecimalReturnTypeInference.java |  113 +-
 .../apache/drill/exec/physical/impl/ScanBatch.java |    4 +-
 .../drill/exec/physical/impl/join/JoinUtils.java   |   13 +-
 .../physical/impl/join/NestedLoopJoinBatch.java    |    7 +-
 .../exec/planner/ParquetPartitionDescriptor.java   |   26 +-
 .../exec/planner/common/DrillValuesRelBase.java    |    4 +-
 .../exec/planner/logical/DrillAggregateRel.java    |   20 +-
 .../exec/planner/logical/DrillConstExecutor.java   |   32 +-
 .../drill/exec/planner/logical/DrillOptiq.java     |   52 +-
 .../planner/logical/DrillReduceAggregatesRule.java |   18 +-
 .../drill/exec/planner/sql/SqlConverter.java       |   14 +
 .../drill/exec/planner/sql/TypeInferenceUtils.java |  208 +-
 .../planner/sql/handlers/FindLimit0Visitor.java    |   11 +-
 .../exec/planner/types/DrillRelDataTypeSystem.java |    1 -
 .../decimal}/DecimalScalePrecisionAddFunction.java |    8 +-
 .../DecimalScalePrecisionDivideFunction.java       |   23 +-
 .../decimal}/DecimalScalePrecisionModFunction.java |    5 +-
 .../decimal}/DecimalScalePrecisionMulFunction.java |   11 +-
 .../decimal}/DrillBaseComputeScalePrecision.java   |   17 +-
 .../apache/drill/exec/record/VectorContainer.java  |    3 +-
 .../drill/exec/resolver/ExactFunctionResolver.java |   17 +-
 .../exec/resolver/ResolverTypePrecedence.java      |   38 +-
 .../apache/drill/exec/resolver/TypeCastRules.java  |  187 +-
 .../exec/server/options/SystemOptionManager.java   |    2 +
 .../drill/exec/store/avro/AvroDrillTable.java      |   48 +-
 .../drill/exec/store/avro/AvroRecordReader.java    |   82 +-
 .../apache/drill/exec/store/mock/ColumnDef.java    |    2 +
 .../exec/store/parquet/ParquetFilterBuilder.java   |   13 +
 .../exec/store/parquet/ParquetFormatPlugin.java    |    6 +
 .../exec/store/parquet/ParquetReaderUtility.java   |    2 +-
 .../exec/store/parquet/ParquetRecordWriter.java    |   39 +-
 .../parquet/columnreaders/ColumnReaderFactory.java |   80 +-
 .../columnreaders/FixedByteAlignedReader.java      |   50 +-
 .../NullableFixedByteAlignedReaders.java           |  130 +-
 .../ParquetFixedWidthDictionaryReaders.java        |   97 +-
 .../columnreaders/ParquetToDrillTypeConverter.java |   21 +-
 .../columnreaders/VarLengthColumnReaders.java      |  141 --
 .../decimal/BinaryDecimalParquetValueWriter.java}  |   26 +-
 .../store/parquet/decimal/DecimalValueWriter.java  |   61 +
 .../decimal/FixedLenDecimalParquetValueWriter.java |   50 +
 .../decimal/Int32DecimalParquetValueWriter.java    |   51 +
 .../decimal/Int64DecimalParquetValueWriter.java    |   50 +
 .../store/parquet2/DrillParquetGroupConverter.java |  186 +-
 .../apache/drill/exec/store/pojo/PojoDataType.java |    5 +-
 .../apache/drill/exec/store/pojo/PojoWriters.java  |   22 +
 .../drill/exec/vector/complex/MapUtility.java      |    7 +-
 .../apache/drill/exec/work/ExecErrorConstants.java |    3 +-
 .../work/prepare/PreparedStatementProvider.java    |    1 +
 .../java-exec/src/main/resources/drill-module.conf |    4 +-
 .../test/java/org/apache/drill/PlanningBase.java   |   19 +
 .../apache/drill/TestDisabledFunctionality.java    |   19 +-
 .../java/org/apache/drill/TestFrameworkTest.java   |    2 +-
 .../java/org/apache/drill/TestFunctionsQuery.java  |   87 +-
 .../org/apache/drill/exec/TestEmptyInputSql.java   |   58 +-
 .../org/apache/drill/exec/TestWindowFunctions.java |   18 +-
 .../drill/exec/cache/TestBatchSerialization.java   |    1 +
 .../drill/exec/fn/impl/TestAggregateFunctions.java |  245 +-
 .../drill/exec/fn/impl/TestCastFunctions.java      |  158 +-
 .../exec/fn/impl/TestVarDecimalFunctions.java      |  743 ++++++
 .../drill/exec/physical/impl/TestDecimal.java      |   26 +-
 .../physical/impl/join/TestNestedLoopJoin.java     |   26 +
 .../physical/impl/writer/TestParquetWriter.java    |  326 ++-
 .../physical/impl/xsort/managed/TestCopier.java    |    5 +-
 .../physical/impl/xsort/managed/TestSorter.java    |    9 +-
 .../planner/logical/TestCaseNullableTypes.java     |   18 +
 .../drill/exec/store/avro/AvroFormatTest.java      |   72 +
 .../apache/drill/exec/store/avro/AvroTestUtil.java |   26 +
 .../exec/store/parquet/TestFixedlenDecimal.java    |   25 +-
 .../exec/store/parquet/TestParquetComplex.java     |   41 +-
 .../exec/store/parquet/TestVarlenDecimal.java      |  105 +-
 .../columnreaders/TestColumnReaderFactory.java     |   10 +-
 .../drill/test/rowSet/test/TestFillEmpties.java    |    1 +
 .../test/resources/decimal/cast_decimal_float.json |    4 +-
 .../test/resources/decimal/cast_decimal_int.json   |    4 +-
 .../resources/decimal/cast_decimal_vardecimal.json |   56 +
 .../test/resources/decimal/cast_float_decimal.json |    4 +-
 .../test/resources/decimal/cast_int_decimal.json   |    8 +-
 .../resources/decimal/cast_simple_decimal.json     |    4 +-
 ...cimal_int.json => cast_vardecimal_decimal.json} |   18 +-
 .../decimal/simple_decimal_arithmetic.json         |    4 +-
 .../resources/decimal/simple_decimal_math.json     |    2 +-
 .../resources/decimal/test_decimal_complex.json    |    4 +-
 .../decimal/test_decimal_sort_complex.json         |    2 +-
 .../repeatedIntLondFixedLenBinaryDecimal.parquet   |  Bin 0 -> 863 bytes
 .../org/apache/drill/jdbc/impl/DrillMetaImpl.java  |    2 +
 .../drill/jdbc/impl/TypeConvertingSqlAccessor.java |   46 +-
 .../jdbc/ResultSetGetMethodConversionsTest.java    |    9 +-
 .../src/main/codegen/data/ValueVectorTypes.tdd     |    4 +-
 .../codegen/templates/AbstractFieldReader.java     |   46 +-
 .../codegen/templates/AbstractFieldWriter.java     |   15 +-
 .../templates/AbstractPromotableFieldWriter.java   |   24 +-
 .../src/main/codegen/templates/BaseWriter.java     |    6 +-
 .../main/codegen/templates/BasicTypeHelper.java    |   39 +-
 .../main/codegen/templates/ColumnAccessors.java    |   38 +-
 .../src/main/codegen/templates/ComplexCopier.java  |   12 +-
 .../src/main/codegen/templates/ComplexReaders.java |   66 +-
 .../src/main/codegen/templates/ComplexWriters.java |   42 +-
 .../main/codegen/templates/HolderReaderImpl.java   |   13 +-
 .../src/main/codegen/templates/ListWriters.java    |   18 +-
 .../src/main/codegen/templates/MapWriters.java     |    4 +-
 .../src/main/codegen/templates/NullReader.java     |   48 +-
 .../codegen/templates/NullableValueVectors.java    |   16 +-
 .../codegen/templates/RepeatedValueVectors.java    |   18 +-
 .../main/codegen/templates/UnionListWriter.java    |   12 +
 .../src/main/codegen/templates/UnionWriter.java    |   16 +-
 .../src/main/codegen/templates/ValueHolders.java   |  141 +-
 .../codegen/templates/VariableLengthVectors.java   |   22 +-
 .../org/apache/drill/exec/util/DecimalUtility.java |  204 +-
 .../drill/exec/vector/ValueHolderHelper.java       |   19 +
 .../vector/complex/impl/MapOrListWriterImpl.java   |    5 +
 .../drill/common/expression/parser/ExprLexer.g     |    2 +-
 .../common/expression/ExpressionStringBuilder.java |    9 +-
 .../drill/common/expression/ValueExpressions.java  |   43 +-
 .../expression/visitors/AbstractExprVisitor.java   |    6 +
 .../expression/visitors/AggregateChecker.java      |    6 +
 .../expression/visitors/ConstantChecker.java       |   10 +-
 .../common/expression/visitors/ExprVisitor.java    |    2 +
 .../expression/visitors/ExpressionValidator.java   |    6 +
 .../expression/visitors/SimpleExprVisitor.java     |    7 +
 pom.xml                                            |   39 +
 .../org/apache/drill/common/types/MinorType.java   |    4 +-
 196 files changed, 5364 insertions(+), 7668 deletions(-)

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 ae26925..1346f47 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
@@ -318,6 +318,7 @@ public class Types {
       case DECIMAL28SPARSE:
       case DECIMAL38DENSE:
       case DECIMAL38SPARSE:
+      case VARDECIMAL:
       case MONEY:           return 2 + precision; // precision of the column plus a sign and a decimal point
 
       case VARCHAR:
@@ -575,7 +576,7 @@ public class Types {
     case "double":
       return MinorType.FLOAT8;
     case "decimal":
-      return MinorType.DECIMAL38SPARSE;
+      return MinorType.VARDECIMAL;
     case "symbol":
     case "char":
     case "utf8":
@@ -638,13 +639,9 @@ public class Types {
       case FLOAT8:
         return "double";
       case VARDECIMAL:
-        return "decimal";
       case DECIMAL9:
-        return "decimal";
       case DECIMAL18:
-        return "decimal";
       case DECIMAL28SPARSE:
-        return "decimal";
       case DECIMAL38SPARSE:
         return "decimal";
       case VARCHAR:
diff --git a/common/src/main/java/org/apache/drill/common/util/CoreDecimalUtility.java b/common/src/main/java/org/apache/drill/common/util/CoreDecimalUtility.java
index 4b4b950..0e8a1c7 100644
--- a/common/src/main/java/org/apache/drill/common/util/CoreDecimalUtility.java
+++ b/common/src/main/java/org/apache/drill/common/util/CoreDecimalUtility.java
@@ -31,42 +31,6 @@ public class CoreDecimalUtility {
     return (input.unscaledValue().longValue());
   }
 
-  public static int getMaxPrecision(TypeProtos.MinorType decimalType) {
-    if (decimalType == TypeProtos.MinorType.DECIMAL9) {
-      return 9;
-    } else if (decimalType == TypeProtos.MinorType.DECIMAL18) {
-      return 18;
-    } else if (decimalType == TypeProtos.MinorType.DECIMAL28SPARSE) {
-      return 28;
-    } else if (decimalType == TypeProtos.MinorType.DECIMAL38SPARSE) {
-      return 38;
-    }
-    return 0;
-  }
-
-  /*
-   * Function returns the Minor decimal type given the precision
-   */
-  public static TypeProtos.MinorType getDecimalDataType(int precision) {
-    if (precision <= 9) {
-      return TypeProtos.MinorType.DECIMAL9;
-    } else if (precision <= 18) {
-      return TypeProtos.MinorType.DECIMAL18;
-    } else if (precision <= 28) {
-      return TypeProtos.MinorType.DECIMAL28SPARSE;
-    } else {
-      return TypeProtos.MinorType.DECIMAL38SPARSE;
-    }
-  }
-
-  /*
-   * Given a precision it provides the max precision of that decimal data type;
-   * For eg: given the precision 12, we would use DECIMAL18 to store the data
-   * which has a max precision range of 18 digits
-   */
-  public static int getPrecisionRange(int precision) {
-    return getMaxPrecision(getDecimalDataType(precision));
-  }
   public static int getDecimal9FromBigDecimal(BigDecimal input, int scale, int precision) {
     // Truncate/ or pad to set the input to the correct scale
     input = input.setScale(scale, BigDecimal.ROUND_HALF_UP);
@@ -82,11 +46,10 @@ public class CoreDecimalUtility {
   }
 
   public static boolean isDecimalType(TypeProtos.MinorType minorType) {
-    if (minorType == TypeProtos.MinorType.VARDECIMAL || minorType == TypeProtos.MinorType.DECIMAL9 ||
-        minorType == TypeProtos.MinorType.DECIMAL18 || minorType == TypeProtos.MinorType.DECIMAL28SPARSE ||
-        minorType == TypeProtos.MinorType.DECIMAL38SPARSE) {
-      return true;
-    }
-    return false;
+    return minorType == TypeProtos.MinorType.VARDECIMAL ||
+           minorType == TypeProtos.MinorType.DECIMAL9 ||
+           minorType == TypeProtos.MinorType.DECIMAL18 ||
+           minorType == TypeProtos.MinorType.DECIMAL28SPARSE ||
+           minorType == TypeProtos.MinorType.DECIMAL38SPARSE;
   }
 }
diff --git a/common/src/main/java/org/apache/drill/common/util/DecimalScalePrecisionDivideFunction.java b/common/src/main/java/org/apache/drill/common/util/DecimalScalePrecisionDivideFunction.java
deleted file mode 100644
index 74e6dbf..0000000
--- a/common/src/main/java/org/apache/drill/common/util/DecimalScalePrecisionDivideFunction.java
+++ /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.
- */
-package org.apache.drill.common.util;
-
-
-/*
- * Here we compute the scale and precision of the output decimal data type
- * based on the input scale and precision. Since division operation can be
- * a multiplication operation we compute the scale to be the sum of the inputs.
- * Eg: Input1 : precision = 5, scale = 3 ==> max integer digits = 2
- *     Input2 : precision = 7, scale = 4 ==> max integer digits = 3
- *
- *     Output: max integer digits ==> 2 (left integer digits) + 4 (right scale, when divide results in multiplication)
- *             max scale          ==> 3 + 4 = 7
- *
- *             Minimum precision required ==> 6 + 7 = 13
- *
- * Since our minimum precision required is 13, we will use DECIMAL18 as the output type
- * but since this is divide we will grant the remaining digits in DECIMAL18 to scale
- * so we have the following
- *    output scale      ==> 7 + (18 - 13) = 12
- *    output precision  ==> 18
- */
-public class DecimalScalePrecisionDivideFunction extends DrillBaseComputeScalePrecision {
-
-  public DecimalScalePrecisionDivideFunction(int leftPrecision, int leftScale, int rightPrecision, int rightScale) {
-    super(leftPrecision, leftScale, rightPrecision, rightScale);
-  }
-
-  @Override
-  public void computeScalePrecision(int leftPrecision, int leftScale, int rightPrecision, int rightScale) {
-
-    // compute the output scale and precision here
-    outputScale = leftScale + rightScale;
-    int leftIntegerDigits = leftPrecision - leftScale;
-    int maxResultIntegerDigits = leftIntegerDigits + rightScale;
-
-
-    outputPrecision = CoreDecimalUtility.getPrecisionRange(outputScale + maxResultIntegerDigits);
-
-    // Output precision should be greater or equal to the input precision
-    outputPrecision = Math.max(outputPrecision, Math.max(leftPrecision, rightPrecision));
-
-    // Try and increase the scale if we have any room
-    outputScale = (outputPrecision - maxResultIntegerDigits >= 0) ? (outputPrecision - maxResultIntegerDigits) : 0;
-  }
-}
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/CompareFunctionsProcessor.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/CompareFunctionsProcessor.java
index 36e7309..3410939 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/CompareFunctionsProcessor.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/CompareFunctionsProcessor.java
@@ -31,12 +31,12 @@ import org.apache.drill.common.expression.ValueExpressions.LongExpression;
 import org.apache.drill.common.expression.ValueExpressions.QuotedString;
 import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
 import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
+import org.apache.drill.common.expression.ValueExpressions.VarDecimalExpression;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
 import org.joda.time.LocalTime;
 import org.ojai.Value;
 import org.ojai.types.ODate;
 import org.ojai.types.OTime;
-import org.ojai.types.OTimestamp;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
@@ -173,6 +173,14 @@ class CompareFunctionsProcessor extends AbstractExprVisitor<Boolean, LogicalExpr
       return true;
     }
 
+    // MaprDB does not support decimals completely, therefore double value is used.
+    // See com.mapr.db.impl.ConditionImpl.is(FieldPath path, QueryCondition.Op op, BigDecimal value) method
+    if (valueArg instanceof VarDecimalExpression) {
+      this.value = KeyValueBuilder.initFrom(((VarDecimalExpression) valueArg).getBigDecimal().doubleValue());
+      this.path = path;
+      return true;
+    }
+
     if (valueArg instanceof TimeStampExpression) {
       // disable pushdown of TimeStampExpression type until bug 22824 is fixed.
       //
@@ -196,6 +204,7 @@ class CompareFunctionsProcessor extends AbstractExprVisitor<Boolean, LogicalExpr
         .add(LongExpression.class)
         .add(QuotedString.class)
         .add(TimeExpression.class)
+        .add(VarDecimalExpression.class)
         .build();
   }
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
index 9f93e18..fde4d28 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
@@ -304,7 +304,7 @@ public class MaprDBJsonRecordReader extends AbstractRecordReader {
           writeDouble(writer, fieldName, reader);
           break;
         case DECIMAL:
-          throw unsupportedError("Decimal type is currently not supported.");
+          writeDecimal(writer, fieldName, reader);
         case DATE:
           writeDate(writer, fieldName, reader);
           break;
@@ -365,6 +365,15 @@ public class MaprDBJsonRecordReader extends AbstractRecordReader {
     }
   }
 
+  private void writeDecimal(MapOrListWriterImpl writer, String fieldName, DBDocumentReaderBase reader) {
+    if (allTextMode) {
+      writeString(writer, fieldName, String.valueOf(reader.getDecimal()));
+    } else {
+      writer.varDecimal(fieldName, reader.getDecimalScale(), reader.getDecimalPrecision())
+          .writeVarDecimal(reader.getDecimal());
+    }
+  }
+
   private void writeFloat(MapOrListWriterImpl writer, String fieldName, DBDocumentReaderBase reader) {
     if (allTextMode) {
       writeString(writer, fieldName, String.valueOf(reader.getFloat()));
diff --git a/contrib/native/client/src/clientlib/decimalUtils.cpp b/contrib/native/client/src/clientlib/decimalUtils.cpp
index 6e26c55..465eefe 100644
--- a/contrib/native/client/src/clientlib/decimalUtils.cpp
+++ b/contrib/native/client/src/clientlib/decimalUtils.cpp
@@ -95,6 +95,21 @@ DecimalValue getDecimalValueFromByteBuf(SlicedByteBuf& data, size_t startIndex,
     return val;
 }
 
+DecimalValue getDecimalValueFromByteBuf(SlicedByteBuf& data, size_t length, int scale) {
+
+    cpp_int decimalDigits;
+    // casts the first unsigned byte to signed to determine the sign of the value
+    decimalDigits = decimalDigits | cpp_int(static_cast<int8_t>(data.getByte(0))) << (length - 1) * 8;
+    for (int pos = length - 1; pos > 0; pos--) {
+        decimalDigits = decimalDigits | cpp_int(data.getByte(pos)) << (length - pos - 1) * 8;
+    }
+
+    DecimalValue val;
+    val.m_unscaledValue = decimalDigits;
+    val.m_scale = scale;
+    return val;
+}
+
 DecimalValue getDecimalValueFromDense(SlicedByteBuf& data, size_t startIndex, int nDecimalDigits, int scale, int maxPrecision, int width)
 {
     /* This method converts the dense representation to
diff --git a/contrib/native/client/src/clientlib/fieldmeta.cpp b/contrib/native/client/src/clientlib/fieldmeta.cpp
index 797b038..b48f3bc 100644
--- a/contrib/native/client/src/clientlib/fieldmeta.cpp
+++ b/contrib/native/client/src/clientlib/fieldmeta.cpp
@@ -71,6 +71,7 @@ static const std::string& getSQLType(common::MinorType type, common::DataMode mo
     case common::DECIMAL28DENSE:
     case common::DECIMAL28SPARSE:
     case common::DECIMAL38DENSE:
+    case common::VARDECIMAL:
     case common::DECIMAL38SPARSE: return SQLDecimal;
 
     case common::VARCHAR:         return SQLVarchar;
@@ -133,6 +134,7 @@ static bool isSigned(common::MinorType type, common::DataMode mode) {
     case common::DECIMAL28DENSE:
     case common::DECIMAL38DENSE:
     case common::DECIMAL38SPARSE:
+    case common::VARDECIMAL:
 
     case common::INTERVALYEAR:
     case common::INTERVALDAY:
@@ -304,6 +306,7 @@ static uint32_t getDisplaySize(const ::common::MajorType& type) {
     case ::common::DECIMAL28SPARSE:
     case ::common::DECIMAL38DENSE:
     case ::common::DECIMAL38SPARSE:
+    case ::common::VARDECIMAL:
     case ::common::MONEY:           return 2 + precision; // precision of the column plus a sign and a decimal point
 
     case ::common::VARCHAR:
diff --git a/contrib/native/client/src/clientlib/metadata.cpp b/contrib/native/client/src/clientlib/metadata.cpp
index 637c83b..0f1cf2e 100644
--- a/contrib/native/client/src/clientlib/metadata.cpp
+++ b/contrib/native/client/src/clientlib/metadata.cpp
@@ -110,6 +110,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::TINYINT, common::DECIMAL18))
 		(ConvertSupport(common::TINYINT, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::TINYINT, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::TINYINT, common::VARDECIMAL))
 		(ConvertSupport(common::TINYINT, common::DATE))
 		(ConvertSupport(common::TINYINT, common::TIME))
 		(ConvertSupport(common::TINYINT, common::TIMESTAMP))
@@ -128,6 +129,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::SMALLINT, common::DECIMAL18))
 		(ConvertSupport(common::SMALLINT, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::SMALLINT, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::SMALLINT, common::VARDECIMAL))
 		(ConvertSupport(common::SMALLINT, common::DATE))
 		(ConvertSupport(common::SMALLINT, common::TIME))
 		(ConvertSupport(common::SMALLINT, common::TIMESTAMP))
@@ -146,6 +148,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::INT, common::DECIMAL18))
 		(ConvertSupport(common::INT, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::INT, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::INT, common::VARDECIMAL))
 		(ConvertSupport(common::INT, common::DATE))
 		(ConvertSupport(common::INT, common::TIME))
 		(ConvertSupport(common::INT, common::TIMESTAMP))
@@ -164,6 +167,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::BIGINT, common::DECIMAL18))
 		(ConvertSupport(common::BIGINT, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::BIGINT, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::BIGINT, common::VARDECIMAL))
 		(ConvertSupport(common::BIGINT, common::DATE))
 		(ConvertSupport(common::BIGINT, common::TIME))
 		(ConvertSupport(common::BIGINT, common::TIMESTAMP))
@@ -182,6 +186,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::DECIMAL9, common::DECIMAL18))
 		(ConvertSupport(common::DECIMAL9, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::DECIMAL9, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::DECIMAL9, common::VARDECIMAL))
 		(ConvertSupport(common::DECIMAL9, common::DATE))
 		(ConvertSupport(common::DECIMAL9, common::TIME))
 		(ConvertSupport(common::DECIMAL9, common::TIMESTAMP))
@@ -200,6 +205,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::DECIMAL18, common::DECIMAL18))
 		(ConvertSupport(common::DECIMAL18, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::DECIMAL18, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::DECIMAL18, common::VARDECIMAL))
 		(ConvertSupport(common::DECIMAL18, common::DATE))
 		(ConvertSupport(common::DECIMAL18, common::TIME))
 		(ConvertSupport(common::DECIMAL18, common::TIMESTAMP))
@@ -218,6 +224,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::DECIMAL28SPARSE, common::DECIMAL18))
 		(ConvertSupport(common::DECIMAL28SPARSE, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::DECIMAL28SPARSE, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::DECIMAL28SPARSE, common::VARDECIMAL))
 		(ConvertSupport(common::DECIMAL28SPARSE, common::DATE))
 		(ConvertSupport(common::DECIMAL28SPARSE, common::TIME))
 		(ConvertSupport(common::DECIMAL28SPARSE, common::TIMESTAMP))
@@ -236,6 +243,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::DECIMAL38SPARSE, common::DECIMAL18))
 		(ConvertSupport(common::DECIMAL38SPARSE, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::DECIMAL38SPARSE, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::DECIMAL38SPARSE, common::VARDECIMAL))
 		(ConvertSupport(common::DECIMAL38SPARSE, common::DATE))
 		(ConvertSupport(common::DECIMAL38SPARSE, common::TIME))
 		(ConvertSupport(common::DECIMAL38SPARSE, common::TIMESTAMP))
@@ -248,12 +256,32 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::DECIMAL38SPARSE, common::VARBINARY))
 		(ConvertSupport(common::DECIMAL38SPARSE, common::INTERVALYEAR))
 		(ConvertSupport(common::DECIMAL38SPARSE, common::INTERVALDAY))
+		(ConvertSupport(common::VARDECIMAL, common::INT))
+		(ConvertSupport(common::VARDECIMAL, common::BIGINT))
+		(ConvertSupport(common::VARDECIMAL, common::DECIMAL9))
+		(ConvertSupport(common::VARDECIMAL, common::DECIMAL18))
+		(ConvertSupport(common::VARDECIMAL, common::DECIMAL28SPARSE))
+		(ConvertSupport(common::VARDECIMAL, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::VARDECIMAL, common::VARDECIMAL))
+		(ConvertSupport(common::VARDECIMAL, common::DATE))
+		(ConvertSupport(common::VARDECIMAL, common::TIME))
+		(ConvertSupport(common::VARDECIMAL, common::TIMESTAMP))
+		(ConvertSupport(common::VARDECIMAL, common::INTERVAL))
+		(ConvertSupport(common::VARDECIMAL, common::FLOAT4))
+		(ConvertSupport(common::VARDECIMAL, common::FLOAT8))
+		(ConvertSupport(common::VARDECIMAL, common::BIT))
+		(ConvertSupport(common::VARDECIMAL, common::VARCHAR))
+		(ConvertSupport(common::VARDECIMAL, common::VAR16CHAR))
+		(ConvertSupport(common::VARDECIMAL, common::VARBINARY))
+		(ConvertSupport(common::VARDECIMAL, common::INTERVALYEAR))
+		(ConvertSupport(common::VARDECIMAL, common::INTERVALDAY))
 		(ConvertSupport(common::MONEY, common::INT))
 		(ConvertSupport(common::MONEY, common::BIGINT))
 		(ConvertSupport(common::MONEY, common::DECIMAL9))
 		(ConvertSupport(common::MONEY, common::DECIMAL18))
 		(ConvertSupport(common::MONEY, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::MONEY, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::MONEY, common::VARDECIMAL))
 		(ConvertSupport(common::MONEY, common::DATE))
 		(ConvertSupport(common::MONEY, common::TIME))
 		(ConvertSupport(common::MONEY, common::TIMESTAMP))
@@ -272,6 +300,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::DATE, common::DECIMAL18))
 		(ConvertSupport(common::DATE, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::DATE, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::DATE, common::VARDECIMAL))
 		(ConvertSupport(common::DATE, common::DATE))
 		(ConvertSupport(common::DATE, common::TIME))
 		(ConvertSupport(common::DATE, common::TIMESTAMP))
@@ -290,6 +319,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::TIME, common::DECIMAL18))
 		(ConvertSupport(common::TIME, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::TIME, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::TIME, common::VARDECIMAL))
 		(ConvertSupport(common::TIME, common::DATE))
 		(ConvertSupport(common::TIME, common::TIME))
 		(ConvertSupport(common::TIME, common::TIMESTAMP))
@@ -308,6 +338,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::TIMESTAMPTZ, common::DECIMAL18))
 		(ConvertSupport(common::TIMESTAMPTZ, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::TIMESTAMPTZ, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::TIMESTAMPTZ, common::VARDECIMAL))
 		(ConvertSupport(common::TIMESTAMPTZ, common::DATE))
 		(ConvertSupport(common::TIMESTAMPTZ, common::TIME))
 		(ConvertSupport(common::TIMESTAMPTZ, common::TIMESTAMP))
@@ -326,6 +357,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::TIMESTAMP, common::DECIMAL18))
 		(ConvertSupport(common::TIMESTAMP, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::TIMESTAMP, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::TIMESTAMP, common::VARDECIMAL))
 		(ConvertSupport(common::TIMESTAMP, common::DATE))
 		(ConvertSupport(common::TIMESTAMP, common::TIME))
 		(ConvertSupport(common::TIMESTAMP, common::TIMESTAMP))
@@ -344,6 +376,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::INTERVAL, common::DECIMAL18))
 		(ConvertSupport(common::INTERVAL, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::INTERVAL, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::INTERVAL, common::VARDECIMAL))
 		(ConvertSupport(common::INTERVAL, common::DATE))
 		(ConvertSupport(common::INTERVAL, common::TIME))
 		(ConvertSupport(common::INTERVAL, common::TIMESTAMP))
@@ -362,6 +395,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::FLOAT4, common::DECIMAL18))
 		(ConvertSupport(common::FLOAT4, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::FLOAT4, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::FLOAT4, common::VARDECIMAL))
 		(ConvertSupport(common::FLOAT4, common::DATE))
 		(ConvertSupport(common::FLOAT4, common::TIME))
 		(ConvertSupport(common::FLOAT4, common::TIMESTAMP))
@@ -380,6 +414,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::FLOAT8, common::DECIMAL18))
 		(ConvertSupport(common::FLOAT8, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::FLOAT8, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::FLOAT8, common::VARDECIMAL))
 		(ConvertSupport(common::FLOAT8, common::DATE))
 		(ConvertSupport(common::FLOAT8, common::TIME))
 		(ConvertSupport(common::FLOAT8, common::TIMESTAMP))
@@ -399,6 +434,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::BIT, common::DECIMAL18))
 		(ConvertSupport(common::BIT, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::BIT, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::BIT, common::VARDECIMAL))
 		(ConvertSupport(common::BIT, common::DATE))
 		(ConvertSupport(common::BIT, common::TIME))
 		(ConvertSupport(common::BIT, common::TIMESTAMP))
@@ -418,6 +454,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::FIXEDCHAR, common::DECIMAL18))
 		(ConvertSupport(common::FIXEDCHAR, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::FIXEDCHAR, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::FIXEDCHAR, common::VARDECIMAL))
 		(ConvertSupport(common::FIXEDCHAR, common::DATE))
 		(ConvertSupport(common::FIXEDCHAR, common::TIME))
 		(ConvertSupport(common::FIXEDCHAR, common::TIMESTAMP))
@@ -437,6 +474,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::FIXED16CHAR, common::DECIMAL18))
 		(ConvertSupport(common::FIXED16CHAR, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::FIXED16CHAR, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::FIXED16CHAR, common::VARDECIMAL))
 		(ConvertSupport(common::FIXED16CHAR, common::DATE))
 		(ConvertSupport(common::FIXED16CHAR, common::TIME))
 		(ConvertSupport(common::FIXED16CHAR, common::TIMESTAMP))
@@ -455,6 +493,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::FIXEDBINARY, common::DECIMAL18))
 		(ConvertSupport(common::FIXEDBINARY, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::FIXEDBINARY, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::FIXEDBINARY, common::VARDECIMAL))
 		(ConvertSupport(common::FIXEDBINARY, common::DATE))
 		(ConvertSupport(common::FIXEDBINARY, common::TIME))
 		(ConvertSupport(common::FIXEDBINARY, common::TIMESTAMP))
@@ -474,6 +513,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::VARCHAR, common::DECIMAL18))
 		(ConvertSupport(common::VARCHAR, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::VARCHAR, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::VARCHAR, common::VARDECIMAL))
 		(ConvertSupport(common::VARCHAR, common::DATE))
 		(ConvertSupport(common::VARCHAR, common::TIME))
 		(ConvertSupport(common::VARCHAR, common::TIMESTAMP))
@@ -493,6 +533,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::VAR16CHAR, common::DECIMAL18))
 		(ConvertSupport(common::VAR16CHAR, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::VAR16CHAR, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::VAR16CHAR, common::VARDECIMAL))
 		(ConvertSupport(common::VAR16CHAR, common::DATE))
 		(ConvertSupport(common::VAR16CHAR, common::TIME))
 		(ConvertSupport(common::VAR16CHAR, common::TIMESTAMP))
@@ -511,6 +552,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::VARBINARY, common::DECIMAL18))
 		(ConvertSupport(common::VARBINARY, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::VARBINARY, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::VARBINARY, common::VARDECIMAL))
 		(ConvertSupport(common::VARBINARY, common::DATE))
 		(ConvertSupport(common::VARBINARY, common::TIME))
 		(ConvertSupport(common::VARBINARY, common::TIMESTAMP))
@@ -529,6 +571,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::UINT1, common::DECIMAL18))
 		(ConvertSupport(common::UINT1, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::UINT1, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::UINT1, common::VARDECIMAL))
 		(ConvertSupport(common::UINT1, common::DATE))
 		(ConvertSupport(common::UINT1, common::TIME))
 		(ConvertSupport(common::UINT1, common::TIMESTAMP))
@@ -547,6 +590,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::UINT2, common::DECIMAL18))
 		(ConvertSupport(common::UINT2, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::UINT2, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::UINT2, common::VARDECIMAL))
 		(ConvertSupport(common::UINT2, common::DATE))
 		(ConvertSupport(common::UINT2, common::TIME))
 		(ConvertSupport(common::UINT2, common::TIMESTAMP))
@@ -565,6 +609,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::UINT4, common::DECIMAL18))
 		(ConvertSupport(common::UINT4, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::UINT4, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::UINT4, common::VARDECIMAL))
 		(ConvertSupport(common::UINT4, common::DATE))
 		(ConvertSupport(common::UINT4, common::TIME))
 		(ConvertSupport(common::UINT4, common::TIMESTAMP))
@@ -583,6 +628,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::UINT8, common::DECIMAL18))
 		(ConvertSupport(common::UINT8, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::UINT8, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::UINT8, common::VARDECIMAL))
 		(ConvertSupport(common::UINT8, common::DATE))
 		(ConvertSupport(common::UINT8, common::TIME))
 		(ConvertSupport(common::UINT8, common::TIMESTAMP))
@@ -601,6 +647,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::DECIMAL28DENSE, common::DECIMAL18))
 		(ConvertSupport(common::DECIMAL28DENSE, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::DECIMAL28DENSE, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::DECIMAL28DENSE, common::VARDECIMAL))
 		(ConvertSupport(common::DECIMAL28DENSE, common::DATE))
 		(ConvertSupport(common::DECIMAL28DENSE, common::TIME))
 		(ConvertSupport(common::DECIMAL28DENSE, common::TIMESTAMP))
@@ -619,6 +666,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::DECIMAL38DENSE, common::DECIMAL18))
 		(ConvertSupport(common::DECIMAL38DENSE, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::DECIMAL38DENSE, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::DECIMAL38DENSE, common::VARDECIMAL))
 		(ConvertSupport(common::DECIMAL38DENSE, common::DATE))
 		(ConvertSupport(common::DECIMAL38DENSE, common::TIME))
 		(ConvertSupport(common::DECIMAL38DENSE, common::TIMESTAMP))
@@ -638,6 +686,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::DM_UNKNOWN, common::DECIMAL18))
 		(ConvertSupport(common::DM_UNKNOWN, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::DM_UNKNOWN, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::DM_UNKNOWN, common::VARDECIMAL))
 		(ConvertSupport(common::DM_UNKNOWN, common::DATE))
 		(ConvertSupport(common::DM_UNKNOWN, common::TIME))
 		(ConvertSupport(common::DM_UNKNOWN, common::TIMESTAMP))
@@ -656,6 +705,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::INTERVALYEAR, common::DECIMAL18))
 		(ConvertSupport(common::INTERVALYEAR, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::INTERVALYEAR, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::INTERVALYEAR, common::VARDECIMAL))
 		(ConvertSupport(common::INTERVALYEAR, common::DATE))
 		(ConvertSupport(common::INTERVALYEAR, common::TIME))
 		(ConvertSupport(common::INTERVALYEAR, common::TIMESTAMP))
@@ -674,6 +724,7 @@ static const convert_support_set s_convertMap = boost::assign::list_of
 		(ConvertSupport(common::INTERVALDAY, common::DECIMAL18))
 		(ConvertSupport(common::INTERVALDAY, common::DECIMAL28SPARSE))
 		(ConvertSupport(common::INTERVALDAY, common::DECIMAL38SPARSE))
+		(ConvertSupport(common::INTERVALDAY, common::VARDECIMAL))
 		(ConvertSupport(common::INTERVALDAY, common::DATE))
 		(ConvertSupport(common::INTERVALDAY, common::TIME))
 		(ConvertSupport(common::INTERVALDAY, common::TIMESTAMP))
diff --git a/contrib/native/client/src/clientlib/recordBatch.cpp b/contrib/native/client/src/clientlib/recordBatch.cpp
index d7c196d..1c897d6 100644
--- a/contrib/native/client/src/clientlib/recordBatch.cpp
+++ b/contrib/native/client/src/clientlib/recordBatch.cpp
@@ -201,6 +201,8 @@ ValueVectorBase* ValueVectorFactory::allocateValueVector(const Drill::FieldMetad
                     return new ValueVectorDecimal28Sparse(b,f.getValueCount(), f.getScale());
                 case common::DECIMAL38SPARSE:
                     return new ValueVectorDecimal38Sparse(b,f.getValueCount(), f.getScale());
+                case common::VARDECIMAL:
+                    return new ValueVectorVarDecimal(b, f.getValueCount(), f.getScale());
                 case common::DATE:
                     return new ValueVectorTyped<DateHolder, int64_t>(b,f.getValueCount());
                 case common::TIMESTAMP:
@@ -251,6 +253,8 @@ ValueVectorBase* ValueVectorFactory::allocateValueVector(const Drill::FieldMetad
                     return new NullableValueVectorDecimal28Sparse(b,f.getValueCount(), f.getScale());
                 case common::DECIMAL38SPARSE:
                     return new NullableValueVectorDecimal38Sparse(b,f.getValueCount(), f.getScale());
+                case common::VARDECIMAL:
+                    return new NullableValueVectorVarDecimal(b, f.getValueCount(), f.getScale());
                 case common::DATE:
                     return new NullableValueVectorTyped<DateHolder,
                            ValueVectorTyped<DateHolder, int64_t> >(b,f.getValueCount());
diff --git a/contrib/native/client/src/include/drill/decimalUtils.hpp b/contrib/native/client/src/include/drill/decimalUtils.hpp
index 2ace857..5f9d37a 100644
--- a/contrib/native/client/src/include/drill/decimalUtils.hpp
+++ b/contrib/native/client/src/include/drill/decimalUtils.hpp
@@ -41,6 +41,7 @@ struct DecimalValue
 
 // These functions need not be exported. They are used by the templates that return the DecimalValue class.
 DecimalValue getDecimalValueFromByteBuf(SlicedByteBuf& data, size_t startIndex, int nDecimalDigits, int scale, bool truncateScale);
+DecimalValue getDecimalValueFromByteBuf(SlicedByteBuf& data, size_t length, int scale);
 DecimalValue getDecimalValueFromDense(SlicedByteBuf& data, size_t startIndex, int nDecimalDigits, int scale, int maxPrecision, int width);
 
 inline DecimalValue getDecimalValueFromIntermediate(SlicedByteBuf& data, size_t startIndex, int nDecimalDigits, int scale)
diff --git a/contrib/native/client/src/include/drill/protobuf/Types.pb.h b/contrib/native/client/src/include/drill/protobuf/Types.pb.h
index f9200ec..b1dec7b 100644
--- a/contrib/native/client/src/include/drill/protobuf/Types.pb.h
+++ b/contrib/native/client/src/include/drill/protobuf/Types.pb.h
@@ -74,11 +74,12 @@ enum MinorType {
   INTERVALDAY = 39,
   LIST = 40,
   GENERIC_OBJECT = 41,
-  UNION = 42
+  UNION = 42,
+  VARDECIMAL = 43
 };
 bool MinorType_IsValid(int value);
 const MinorType MinorType_MIN = LATE;
-const MinorType MinorType_MAX = UNION;
+const MinorType MinorType_MAX = VARDECIMAL;
 const int MinorType_ARRAYSIZE = MinorType_MAX + 1;
 
 const ::google::protobuf::EnumDescriptor* MinorType_descriptor();
diff --git a/contrib/native/client/src/include/drill/recordBatch.hpp b/contrib/native/client/src/include/drill/recordBatch.hpp
index 30287b6..435eb77 100644
--- a/contrib/native/client/src/include/drill/recordBatch.hpp
+++ b/contrib/native/client/src/include/drill/recordBatch.hpp
@@ -346,7 +346,8 @@ template <int DECIMAL_DIGITS, int WIDTH_IN_BYTES, bool IS_SPARSE, int MAX_PRECIS
                     strncpy(buf, str.c_str(), nChars);
                 } else {
                     size_t idxDecimalMark = str.length() - m_scale;
-                    const std::string& decStr= str.substr(0, idxDecimalMark) + "." + str.substr(idxDecimalMark, m_scale);
+                    const std::string& decStr =
+                            (idxDecimalMark == 0 ? "0" : str.substr(0, idxDecimalMark)) + "." + str.substr(idxDecimalMark, m_scale);
                     strncpy(buf, decStr.c_str(), nChars);
                 }
                 return;
@@ -734,6 +735,49 @@ class DECLSPEC_DRILL_CLIENT ValueVectorVarChar:public ValueVectorVarWidth{
         }
 };
 
+class DECLSPEC_DRILL_CLIENT ValueVectorVarDecimal:public ValueVectorVarWidth{
+    public:
+        ValueVectorVarDecimal(SlicedByteBuf *b, size_t rowCount, size_t scale):
+            ValueVectorVarWidth(b, rowCount),
+            m_scale(scale)
+        {
+        }
+        DecimalValue get(size_t index) const {
+            size_t length = getSize(index);
+            ByteBuf_t buff = getRaw(index);
+            SlicedByteBuf intermediateData(&buff[0], 0, length);
+            return getDecimalValueFromByteBuf(intermediateData, length, this->m_scale);
+        }
+
+        void getValueAt(size_t index, char* buf, size_t nChars) const {
+            const DecimalValue& val = this->get(index);
+            std::string str = boost::lexical_cast<std::string>(val.m_unscaledValue);
+            if (str[0] == '-') {
+                str = str.substr(1);
+                while (str.length() < m_scale) {
+                    str = "0" + str;
+                }
+                str = "-" + str;
+            } else {
+                while (str.length() < m_scale) {
+                    str = "0" + str;
+                }
+            }
+            if (m_scale == 0) {
+                strncpy(buf, str.c_str(), nChars);
+            } else {
+                size_t idxDecimalMark = str.length() - m_scale;
+                const std::string& decStr =
+                        (idxDecimalMark == 0 ? "0" : str.substr(0, idxDecimalMark)) + "." + str.substr(idxDecimalMark, m_scale);
+                strncpy(buf, decStr.c_str(), nChars);
+            }
+            return;
+        }
+
+    private:
+        int32_t m_scale;
+};
+
 class DECLSPEC_DRILL_CLIENT ValueVectorVarBinary:public ValueVectorVarWidth{
     public:
         ValueVectorVarBinary(SlicedByteBuf *b, size_t rowCount):ValueVectorVarWidth(b, rowCount){
@@ -764,10 +808,11 @@ typedef ValueVectorDecimal<6, 24, true, 38>  ValueVectorDecimal38Sparse;
 
 typedef NullableValueVectorTyped<int32_t, ValueVectorDecimal9> NullableValueVectorDecimal9;
 typedef NullableValueVectorTyped<int64_t, ValueVectorDecimal18> NullableValueVectorDecimal18;
-typedef NullableValueVectorTyped<DecimalValue , ValueVectorDecimal28Dense> NullableValueVectorDecimal28Dense;
-typedef NullableValueVectorTyped<DecimalValue , ValueVectorDecimal38Dense> NullableValueVectorDecimal38Dense;
-typedef NullableValueVectorTyped<DecimalValue , ValueVectorDecimal28Sparse> NullableValueVectorDecimal28Sparse;
-typedef NullableValueVectorTyped<DecimalValue , ValueVectorDecimal38Sparse> NullableValueVectorDecimal38Sparse;
+typedef NullableValueVectorTyped<DecimalValue, ValueVectorDecimal28Dense> NullableValueVectorDecimal28Dense;
+typedef NullableValueVectorTyped<DecimalValue, ValueVectorDecimal38Dense> NullableValueVectorDecimal38Dense;
+typedef NullableValueVectorTyped<DecimalValue, ValueVectorDecimal28Sparse> NullableValueVectorDecimal28Sparse;
+typedef NullableValueVectorTyped<DecimalValue, ValueVectorDecimal38Sparse> NullableValueVectorDecimal38Sparse;
+typedef NullableValueVectorTyped<DecimalValue, ValueVectorVarDecimal> NullableValueVectorVarDecimal;
 
 typedef ValueVectorTyped<DateHolder, int64_t> ValueVectorDate;
 typedef ValueVectorTyped<DateTimeHolder, int64_t> ValueVectorTimestamp;
diff --git a/contrib/native/client/src/protobuf/Types.pb.cc b/contrib/native/client/src/protobuf/Types.pb.cc
index ec8a1c8..675bba0 100644
--- a/contrib/native/client/src/protobuf/Types.pb.cc
+++ b/contrib/native/client/src/protobuf/Types.pb.cc
@@ -93,7 +93,7 @@ void protobuf_AddDesc_Types_2eproto() {
     "de\030\002 \001(\0162\020.common.DataMode\022\r\n\005width\030\003 \001("
     "\005\022\021\n\tprecision\030\004 \001(\005\022\r\n\005scale\030\005 \001(\005\022\020\n\010t"
     "imeZone\030\006 \001(\005\022#\n\010sub_type\030\007 \003(\0162\021.common"
-    ".MinorType*\233\004\n\tMinorType\022\010\n\004LATE\020\000\022\007\n\003MA"
+    ".MinorType*\253\004\n\tMinorType\022\010\n\004LATE\020\000\022\007\n\003MA"
     "P\020\001\022\013\n\007TINYINT\020\003\022\014\n\010SMALLINT\020\004\022\007\n\003INT\020\005\022"
     "\n\n\006BIGINT\020\006\022\014\n\010DECIMAL9\020\007\022\r\n\tDECIMAL18\020\010"
     "\022\023\n\017DECIMAL28SPARSE\020\t\022\023\n\017DECIMAL38SPARSE"
@@ -106,10 +106,10 @@ void protobuf_AddDesc_Types_2eproto() {
     "\n\005UINT4\020\037\022\t\n\005UINT8\020 \022\022\n\016DECIMAL28DENSE\020!"
     "\022\022\n\016DECIMAL38DENSE\020\"\022\016\n\nDM_UNKNOWN\020%\022\020\n\014"
     "INTERVALYEAR\020&\022\017\n\013INTERVALDAY\020\'\022\010\n\004LIST\020"
-    "(\022\022\n\016GENERIC_OBJECT\020)\022\t\n\005UNION\020**=\n\010Data"
-    "Mode\022\017\n\013DM_OPTIONAL\020\000\022\017\n\013DM_REQUIRED\020\001\022\017"
-    "\n\013DM_REPEATED\020\002B-\n\035org.apache.drill.comm"
-    "on.typesB\nTypeProtosH\001", 862);
+    "(\022\022\n\016GENERIC_OBJECT\020)\022\t\n\005UNION\020*\022\016\n\nVARD"
+    "ECIMAL\020+*=\n\010DataMode\022\017\n\013DM_OPTIONAL\020\000\022\017\n"
+    "\013DM_REQUIRED\020\001\022\017\n\013DM_REPEATED\020\002B-\n\035org.a"
+    "pache.drill.common.typesB\nTypeProtosH\001", 878);
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile(
     "Types.proto", &protobuf_RegisterTypes);
   MajorType::default_instance_ = new MajorType();
@@ -167,6 +167,7 @@ bool MinorType_IsValid(int value) {
     case 40:
     case 41:
     case 42:
+    case 43:
       return true;
     default:
       return false;
diff --git a/contrib/storage-hive/core/src/main/codegen/data/HiveTypes.tdd b/contrib/storage-hive/core/src/main/codegen/data/HiveTypes.tdd
index e13dc36..864610c 100644
--- a/contrib/storage-hive/core/src/main/codegen/data/HiveTypes.tdd
+++ b/contrib/storage-hive/core/src/main/codegen/data/HiveTypes.tdd
@@ -104,7 +104,7 @@
       hiveType: "DECIMAL",
       hiveOI: "HiveDecimalObjectInspector",
       javaType: "org.apache.hadoop.hive.common.type.HiveDecimal",
-      drillType: "Decimal38Sparse",
+      drillType: "VarDecimal",
       needOIForDrillType: true
     },
     {
diff --git a/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectors.java b/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectors.java
index 9c45314..a539b7f 100644
--- a/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectors.java
+++ b/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectors.java
@@ -100,12 +100,12 @@ public class Drill${entry.drillType}${entry.hiveOI} {
     }
 
     @Override
-    public String getPrimitiveJavaObject(Object o){
+    public String getPrimitiveJavaObject(Object o) {
     <#if mode == "Optional">
       if (o == null) {
         return null;
       }
-      final NullableVarCharHolder h = (NullableVarCharHolder)o;
+      final NullableVarCharHolder h = (NullableVarCharHolder) o;
     <#else>
       final VarCharHolder h = (VarCharHolder)o;
     </#if>
@@ -118,9 +118,9 @@ public class Drill${entry.drillType}${entry.hiveOI} {
       if (o == null) {
         return null;
       }
-      final NullableVarBinaryHolder h = (NullableVarBinaryHolder)o;
+      final NullableVarBinaryHolder h = (NullableVarBinaryHolder) o;
     <#else>
-      final VarBinaryHolder h = (VarBinaryHolder)o;
+      final VarBinaryHolder h = (VarBinaryHolder) o;
     </#if>
       final byte[] buf = new byte[h.end-h.start];
       h.buffer.getBytes(h.start, buf, 0, h.end-h.start);
@@ -133,9 +133,9 @@ public class Drill${entry.drillType}${entry.hiveOI} {
       if (o == null) {
         return null;
       }
-      final NullableVarBinaryHolder h = (NullableVarBinaryHolder)o;
+      final NullableVarBinaryHolder h = (NullableVarBinaryHolder) o;
     <#else>
-      final VarBinaryHolder h = (VarBinaryHolder)o;
+      final VarBinaryHolder h = (VarBinaryHolder) o;
     </#if>
       final byte[] buf = new byte[h.end-h.start];
       h.buffer.getBytes(h.start, buf, 0, h.end-h.start);
@@ -174,18 +174,18 @@ public class Drill${entry.drillType}${entry.hiveOI} {
       return Boolean.valueOf(((BitHolder)o).value != 0);
     </#if>
     }
-<#elseif entry.drillType == "Decimal38Sparse">
+<#elseif entry.drillType == "VarDecimal">
     @Override
-    public HiveDecimal getPrimitiveJavaObject(Object o){
+    public HiveDecimal getPrimitiveJavaObject(Object o) {
     <#if mode == "Optional">
       if (o == null) {
         return null;
       }
-      final NullableDecimal38SparseHolder h = (NullableDecimal38SparseHolder) o;
+      final NullableVarDecimalHolder h = (NullableVarDecimalHolder) o;
     <#else>
-      final Decimal38SparseHolder h = (Decimal38SparseHolder) o;
+      final VarDecimalHolder h = (VarDecimalHolder) o;
     </#if>
-      return HiveDecimal.create(DecimalUtility.getBigDecimalFromSparse(h.buffer, h.start, h.nDecimalDigits, h.scale));
+      return HiveDecimal.create(DecimalUtility.getBigDecimalFromDrillBuf(h.buffer, h.start, h.end - h.start, h.scale));
     }
 
     @Override
@@ -194,17 +194,17 @@ public class Drill${entry.drillType}${entry.hiveOI} {
       if (o == null) {
         return null;
       }
-      final NullableDecimal38SparseHolder h = (NullableDecimal38SparseHolder) o;
+      final NullableVarDecimalHolder h = (NullableVarDecimalHolder) o;
     <#else>
-      final Decimal38SparseHolder h = (Decimal38SparseHolder) o;
+      final VarDecimalHolder h = (VarDecimalHolder) o;
     </#if>
       return new HiveDecimalWritable(
-          HiveDecimal.create(DecimalUtility.getBigDecimalFromSparse(h.buffer, h.start, h.nDecimalDigits, h.scale)));
+          HiveDecimal.create(DecimalUtility.getBigDecimalFromDrillBuf(h.buffer, h.start, h.end - h.start, h.scale)));
     }
 
 <#elseif entry.drillType == "TimeStamp">
     @Override
-    public java.sql.Timestamp getPrimitiveJavaObject(Object o){
+    public java.sql.Timestamp getPrimitiveJavaObject(Object o) {
     <#if mode == "Optional">
       if (o == null) {
         return null;
@@ -237,7 +237,7 @@ public class Drill${entry.drillType}${entry.hiveOI} {
 
 <#elseif entry.drillType == "Date">
     @Override
-    public java.sql.Date getPrimitiveJavaObject(Object o){
+    public java.sql.Date getPrimitiveJavaObject(Object o) {
     <#if mode == "Optional">
       if (o == null) {
         return null;
@@ -270,11 +270,11 @@ public class Drill${entry.drillType}${entry.hiveOI} {
 
 <#else>
     @Override
-    public ${entry.javaType} get(Object o){
+    public ${entry.javaType} get(Object o) {
     <#if mode == "Optional">
-      return ((Nullable${entry.drillType}Holder)o).value;
+      return ((Nullable${entry.drillType}Holder) o).value;
     <#else>
-      return ((${entry.drillType}Holder)o).value;
+      return ((${entry.drillType}Holder) o).value;
     </#if>
     }
 
@@ -295,9 +295,9 @@ public class Drill${entry.drillType}${entry.hiveOI} {
       if (o == null) {
         return null;
       }
-      return new ${entry.javaType?cap_first}(((Nullable${entry.drillType}Holder)o).value);
+      return new ${entry.javaType?cap_first}(((Nullable${entry.drillType}Holder) o).value);
     <#else>
-      return new ${entry.javaType?cap_first}(((${entry.drillType}Holder)o).value);
+      return new ${entry.javaType?cap_first}(((${entry.drillType}Holder) o).value);
     </#if>
     }
 </#if>
@@ -310,7 +310,7 @@ public class Drill${entry.drillType}${entry.hiveOI} {
       }
       final Nullable${entry.drillType}Holder h = (Nullable${entry.drillType}Holder) o;
     <#else>
-      final ${entry.drillType}Holder h = (${entry.drillType}Holder)o;
+      final ${entry.drillType}Holder h = (${entry.drillType}Holder) o;
     </#if>
       return new ${entry.javaType?cap_first}Writable(h.value);
     }
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFunctionRegistry.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFunctionRegistry.java
index f6232f6..92e7f83 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFunctionRegistry.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFunctionRegistry.java
@@ -32,7 +32,6 @@ import org.apache.drill.common.scanner.ClassPathScanner;
 import org.apache.drill.common.scanner.persistence.ScanResult;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.expr.fn.impl.hive.ObjectInspectorHelper;
 import org.apache.drill.exec.planner.sql.DrillOperatorTable;
@@ -267,9 +266,9 @@ public class HiveFunctionRegistry implements PluggableFunctionRegistry{
             .build(logger);
       }
 
-      return TypeInferenceUtils.createCalciteTypeWithNullability(
+      return TypeInferenceUtils.convertToCalciteType(
           opBinding.getTypeFactory(),
-          TypeInferenceUtils.getCalciteTypeFromDrillType(hiveFuncHolder.getReturnType().getMinorType()),
+          hiveFuncHolder.getReturnType(),
           hiveFuncHolder.getReturnType().getMode() != TypeProtos.DataMode.REQUIRED);
     }
   }
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveFieldConverter.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveFieldConverter.java
index 2fab5a4..6caecbb 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveFieldConverter.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveFieldConverter.java
@@ -17,27 +17,19 @@
  */
 package org.apache.drill.exec.store.hive;
 
+import java.math.RoundingMode;
 import java.util.Map;
 
-import org.apache.drill.exec.expr.holders.Decimal18Holder;
-import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
-import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.Decimal9Holder;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.util.DecimalUtility;
 import org.apache.drill.exec.vector.NullableBigIntVector;
 import org.apache.drill.exec.vector.NullableBitVector;
 import org.apache.drill.exec.vector.NullableDateVector;
-import org.apache.drill.exec.vector.NullableDecimal18Vector;
-import org.apache.drill.exec.vector.NullableDecimal28SparseVector;
-import org.apache.drill.exec.vector.NullableDecimal38SparseVector;
-import org.apache.drill.exec.vector.NullableDecimal9Vector;
 import org.apache.drill.exec.vector.NullableFloat4Vector;
 import org.apache.drill.exec.vector.NullableFloat8Vector;
 import org.apache.drill.exec.vector.NullableIntVector;
 import org.apache.drill.exec.vector.NullableTimeStampVector;
 import org.apache.drill.exec.vector.NullableVarBinaryVector;
 import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.NullableVarDecimalVector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
@@ -55,7 +47,6 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspect
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.io.Text;
@@ -92,33 +83,18 @@ public abstract class HiveFieldConverter {
     primMap.put(PrimitiveCategory.TIMESTAMP, Timestamp.class);
     primMap.put(PrimitiveCategory.DATE, Date.class);
     primMap.put(PrimitiveCategory.CHAR, Char.class);
+    primMap.put(PrimitiveCategory.DECIMAL, VarDecimal.class);
   }
 
 
-  public static HiveFieldConverter create(TypeInfo typeInfo, FragmentContext fragmentContext)
+  public static HiveFieldConverter create(TypeInfo typeInfo)
       throws IllegalAccessException, InstantiationException {
     switch (typeInfo.getCategory()) {
       case PRIMITIVE:
         final PrimitiveCategory pCat = ((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory();
-        if (pCat != PrimitiveCategory.DECIMAL) {
-          Class<? extends HiveFieldConverter> clazz = primMap.get(pCat);
-          if (clazz != null) {
-            return clazz.newInstance();
-          }
-        } else {
-          // For decimal, based on precision return appropriate converter.
-          DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfo;
-          int precision = decimalTypeInfo.precision();
-          int scale = decimalTypeInfo.scale();
-          if (precision <= 9) {
-            return new Decimal9(precision, scale);
-          } else if (precision <= 18) {
-            return new Decimal18(precision, scale);
-          } else if (precision <= 28) {
-            return new Decimal28(precision, scale, fragmentContext);
-          } else {
-            return new Decimal38(precision, scale, fragmentContext);
-          }
+        Class<? extends HiveFieldConverter> clazz = primMap.get(pCat);
+        if (clazz != null) {
+          return clazz.newInstance();
         }
 
         throwUnsupportedHiveDataTypeError(pCat.toString());
@@ -151,75 +127,15 @@ public abstract class HiveFieldConverter {
     }
   }
 
-  public static class Decimal9 extends HiveFieldConverter {
-    private final Decimal9Holder holder = new Decimal9Holder();
-
-    public Decimal9(int precision, int scale) {
-      holder.scale = scale;
-      holder.precision = precision;
-    }
-
-    @Override
-    public void setSafeValue(ObjectInspector oi, Object hiveFieldValue, ValueVector outputVV, int outputIndex) {
-      holder.value = DecimalUtility.getDecimal9FromBigDecimal(
-          ((HiveDecimalObjectInspector)oi).getPrimitiveJavaObject(hiveFieldValue).bigDecimalValue(),
-          holder.scale, holder.precision);
-      ((NullableDecimal9Vector) outputVV).getMutator().setSafe(outputIndex, holder);
-    }
-  }
-
-  public static class Decimal18 extends HiveFieldConverter {
-    private final Decimal18Holder holder = new Decimal18Holder();
-
-    public Decimal18(int precision, int scale) {
-      holder.scale = scale;
-      holder.precision = precision;
-    }
-
-    @Override
-    public void setSafeValue(ObjectInspector oi, Object hiveFieldValue, ValueVector outputVV, int outputIndex) {
-      holder.value = DecimalUtility.getDecimal18FromBigDecimal(
-          ((HiveDecimalObjectInspector)oi).getPrimitiveJavaObject(hiveFieldValue).bigDecimalValue(),
-          holder.scale, holder.precision);
-      ((NullableDecimal18Vector) outputVV).getMutator().setSafe(outputIndex, holder);
-    }
-  }
-
-  public static class Decimal28 extends HiveFieldConverter {
-    private final Decimal28SparseHolder holder = new Decimal28SparseHolder();
-
-    public Decimal28(int precision, int scale, FragmentContext context) {
-      holder.scale = scale;
-      holder.precision = precision;
-      holder.buffer = context.getManagedBuffer(Decimal28SparseHolder.nDecimalDigits * DecimalUtility.INTEGER_SIZE);
-      holder.start = 0;
-    }
-
-    @Override
-    public void setSafeValue(ObjectInspector oi, Object hiveFieldValue, ValueVector outputVV, int outputIndex) {
-      DecimalUtility.getSparseFromBigDecimal(
-          ((HiveDecimalObjectInspector)oi).getPrimitiveJavaObject(hiveFieldValue).bigDecimalValue(),
-          holder.buffer, holder.start, holder.scale, holder.precision, Decimal28SparseHolder.nDecimalDigits);
-      ((NullableDecimal28SparseVector) outputVV).getMutator().setSafe(outputIndex, holder);
-    }
-  }
-
-  public static class Decimal38 extends HiveFieldConverter {
-    private final Decimal38SparseHolder holder = new Decimal38SparseHolder();
-
-    public Decimal38(int precision, int scale, FragmentContext context) {
-      holder.scale = scale;
-      holder.precision = precision;
-      holder.buffer = context.getManagedBuffer(Decimal38SparseHolder.nDecimalDigits * DecimalUtility.INTEGER_SIZE);
-      holder.start = 0;
-    }
-
+  public static class VarDecimal extends HiveFieldConverter {
     @Override
     public void setSafeValue(ObjectInspector oi, Object hiveFieldValue, ValueVector outputVV, int outputIndex) {
-      DecimalUtility.getSparseFromBigDecimal(
-          ((HiveDecimalObjectInspector)oi).getPrimitiveJavaObject(hiveFieldValue).bigDecimalValue(),
-          holder.buffer, holder.start, holder.scale, holder.precision, Decimal38SparseHolder.nDecimalDigits);
-      ((NullableDecimal38SparseVector) outputVV).getMutator().setSafe(outputIndex, holder);
+      ((NullableVarDecimalVector) outputVV).getMutator()
+          .setSafe(
+              outputIndex,
+              ((HiveDecimalObjectInspector) oi)
+                  .getPrimitiveJavaObject(hiveFieldValue).bigDecimalValue()
+                  .setScale(outputVV.getField().getScale(), RoundingMode.HALF_UP));
     }
   }
 
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 5ff5506..91d0567 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
@@ -50,6 +50,7 @@ import org.apache.drill.exec.vector.NullableIntVector;
 import org.apache.drill.exec.vector.NullableTimeStampVector;
 import org.apache.drill.exec.vector.NullableVarBinaryVector;
 import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.NullableVarDecimalVector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.work.ExecErrorConstants;
 
@@ -80,6 +81,7 @@ import org.joda.time.DateTimeZone;
 
 import javax.annotation.Nullable;
 import java.math.BigDecimal;
+import java.math.RoundingMode;
 import java.sql.Date;
 import java.sql.Timestamp;
 import java.util.List;
@@ -293,6 +295,15 @@ public class HiveUtilities {
         }
         break;
       }
+      case VARDECIMAL: {
+        final BigDecimal value = ((HiveDecimal) val).bigDecimalValue()
+            .setScale(vector.getField().getScale(), RoundingMode.HALF_UP);
+        final NullableVarDecimalVector v = ((NullableVarDecimalVector) vector);
+        for (int i = start; i < end; i++) {
+          v.getMutator().setSafe(i, value);
+        }
+        break;
+      }
     }
   }
 
@@ -346,8 +357,7 @@ public class HiveUtilities {
               .message(ExecErrorConstants.DECIMAL_DISABLE_ERR_MSG)
               .build(logger);
         }
-        DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) primitiveTypeInfo;
-        return DecimalUtility.getDecimalDataType(decimalTypeInfo.precision());
+        return MinorType.VARDECIMAL;
       }
       case DOUBLE:
         return TypeProtos.MinorType.FLOAT8;
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/readers/HiveAbstractReader.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/readers/HiveAbstractReader.java
index a922b4c..d0d9ed0 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/readers/HiveAbstractReader.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/readers/HiveAbstractReader.java
@@ -227,7 +227,7 @@ public abstract class HiveAbstractReader extends AbstractRecordReader {
 
         selectedColumnObjInspectors.add(fieldOI);
         selectedColumnTypes.add(typeInfo);
-        selectedColumnFieldConverters.add(HiveFieldConverter.create(typeInfo, fragmentContext));
+        selectedColumnFieldConverters.add(HiveFieldConverter.create(typeInfo));
       }
 
       for(int i=0; i<selectedColumnNames.size(); ++i){
diff --git a/contrib/storage-hive/hive-exec-shade/pom.xml b/contrib/storage-hive/hive-exec-shade/pom.xml
index 3d4dceb..ad572ef 100644
--- a/contrib/storage-hive/hive-exec-shade/pom.xml
+++ b/contrib/storage-hive/hive-exec-shade/pom.xml
@@ -117,6 +117,10 @@
               <pattern>org.apache.parquet.</pattern>
               <shadedPattern>hive.org.apache.parquet.</shadedPattern>
             </relocation>
+            <relocation>
+              <pattern>org.apache.avro.</pattern>
+              <shadedPattern>hive.org.apache.avro.</shadedPattern>
+            </relocation>
           </relocations>
           <filters>
             <filter>
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java
index bea99c7..5f3f713 100755
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java
@@ -35,9 +35,9 @@ import javax.sql.DataSource;
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.ops.OperatorContext;
@@ -54,6 +54,7 @@ import org.apache.drill.exec.vector.NullableTimeStampVector;
 import org.apache.drill.exec.vector.NullableTimeVector;
 import org.apache.drill.exec.vector.NullableVarBinaryVector;
 import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.NullableVarDecimalVector;
 import org.apache.drill.exec.vector.ValueVector;
 
 import com.google.common.base.Charsets;
@@ -104,7 +105,7 @@ class JdbcRecordReader extends AbstractRecordReader {
         .put(java.sql.Types.BLOB, MinorType.VARBINARY)
 
         .put(java.sql.Types.NUMERIC, MinorType.FLOAT8)
-        .put(java.sql.Types.DECIMAL, MinorType.FLOAT8)
+        .put(java.sql.Types.DECIMAL, MinorType.VARDECIMAL)
         .put(java.sql.Types.REAL, MinorType.FLOAT8)
 
         .put(java.sql.Types.DATE, MinorType.DATE)
@@ -137,29 +138,45 @@ class JdbcRecordReader extends AbstractRecordReader {
 
   private Copier<?> getCopier(int jdbcType, int offset, ResultSet result, ValueVector v) {
 
-    if (v instanceof NullableBigIntVector) {
-      return new BigIntCopier(offset, result, (NullableBigIntVector.Mutator) v.getMutator());
-    } else if (v instanceof NullableFloat4Vector) {
-      return new Float4Copier(offset, result, (NullableFloat4Vector.Mutator) v.getMutator());
-    } else if (v instanceof NullableFloat8Vector) {
-      return new Float8Copier(offset, result, (NullableFloat8Vector.Mutator) v.getMutator());
-    } else if (v instanceof NullableIntVector) {
-      return new IntCopier(offset, result, (NullableIntVector.Mutator) v.getMutator());
-    } else if (v instanceof NullableVarCharVector) {
-      return new VarCharCopier(offset, result, (NullableVarCharVector.Mutator) v.getMutator());
-    } else if (v instanceof NullableVarBinaryVector) {
-      return new VarBinaryCopier(offset, result, (NullableVarBinaryVector.Mutator) v.getMutator());
-    } else if (v instanceof NullableDateVector) {
-      return new DateCopier(offset, result, (NullableDateVector.Mutator) v.getMutator());
-    } else if (v instanceof NullableTimeVector) {
-      return new TimeCopier(offset, result, (NullableTimeVector.Mutator) v.getMutator());
-    } else if (v instanceof NullableTimeStampVector) {
-      return new TimeStampCopier(offset, result, (NullableTimeStampVector.Mutator) v.getMutator());
-    } else if (v instanceof NullableBitVector) {
-      return new BitCopier(offset, result, (NullableBitVector.Mutator) v.getMutator());
+    switch (jdbcType) {
+      case java.sql.Types.BIGINT:
+        return new BigIntCopier(offset, result, (NullableBigIntVector.Mutator) v.getMutator());
+      case java.sql.Types.FLOAT:
+        return new Float4Copier(offset, result, (NullableFloat4Vector.Mutator) v.getMutator());
+      case java.sql.Types.DOUBLE:
+      case java.sql.Types.NUMERIC:
+      case java.sql.Types.REAL:
+        return new Float8Copier(offset, result, (NullableFloat8Vector.Mutator) v.getMutator());
+      case java.sql.Types.TINYINT:
+      case java.sql.Types.SMALLINT:
+      case java.sql.Types.INTEGER:
+        return new IntCopier(offset, result, (NullableIntVector.Mutator) v.getMutator());
+      case java.sql.Types.CHAR:
+      case java.sql.Types.VARCHAR:
+      case java.sql.Types.LONGVARCHAR:
+      case java.sql.Types.CLOB:
+      case java.sql.Types.NCHAR:
+      case java.sql.Types.NVARCHAR:
+      case java.sql.Types.LONGNVARCHAR:
+        return new VarCharCopier(offset, result, (NullableVarCharVector.Mutator) v.getMutator());
+      case java.sql.Types.VARBINARY:
+      case java.sql.Types.LONGVARBINARY:
+      case java.sql.Types.BLOB:
+        return new VarBinaryCopier(offset, result, (NullableVarBinaryVector.Mutator) v.getMutator());
+      case java.sql.Types.DATE:
+        return new DateCopier(offset, result, (NullableDateVector.Mutator) v.getMutator());
+      case java.sql.Types.TIME:
+        return new TimeCopier(offset, result, (NullableTimeVector.Mutator) v.getMutator());
+      case java.sql.Types.TIMESTAMP:
+        return new TimeStampCopier(offset, result, (NullableTimeStampVector.Mutator) v.getMutator());
+      case java.sql.Types.BOOLEAN:
+      case java.sql.Types.BIT:
+        return new BitCopier(offset, result, (NullableBitVector.Mutator) v.getMutator());
+      case java.sql.Types.DECIMAL:
+        return new DecimalCopier(offset, result, (NullableVarDecimalVector.Mutator) v.getMutator());
+      default:
+        throw new IllegalArgumentException("Unknown how to handle vector.");
     }
-
-    throw new IllegalArgumentException("Unknown how to handle vector.");
   }
 
   @Override
@@ -197,9 +214,14 @@ class JdbcRecordReader extends AbstractRecordReader {
           continue;
         }
 
-        final MajorType type = Types.optional(minorType);
+        final MajorType type = MajorType.newBuilder()
+            .setMode(TypeProtos.DataMode.OPTIONAL)
+            .setMinorType(minorType)
+            .setScale(scale)
+            .setPrecision(width)
+            .build();
         final MaterializedField field = MaterializedField.create(name, type);
-        final Class<? extends ValueVector> clazz = (Class<? extends ValueVector>) TypeHelper.getValueVectorClass(
+        final Class<? extends ValueVector> clazz = TypeHelper.getValueVectorClass(
             minorType, type.getMode());
         ValueVector vector = output.addField(field, clazz);
         vectorBuilder.add(vector);
@@ -225,10 +247,10 @@ class JdbcRecordReader extends AbstractRecordReader {
     int counter = 0;
     Boolean b = true;
     try {
-      while (counter < 4095 && b == true) { // loop at 4095 since nullables use one more than record count and we
+      while (counter < 4095 && b) { // loop at 4095 since nullables use one more than record count and we
                                             // allocate on powers of two.
         b = resultSet.next();
-        if(b == false) {
+        if (!b) {
             break;
         }
         for (Copier<?> c : copiers) {
@@ -335,9 +357,9 @@ class JdbcRecordReader extends AbstractRecordReader {
 
   }
 
-  private class DecimalCopier extends Copier<NullableFloat8Vector.Mutator> {
+  private class DecimalCopier extends Copier<NullableVarDecimalVector.Mutator> {
 
-    public DecimalCopier(int columnIndex, ResultSet result, NullableFloat8Vector.Mutator mutator) {
+    public DecimalCopier(int columnIndex, ResultSet result, NullableVarDecimalVector.Mutator mutator) {
       super(columnIndex, result, mutator);
     }
 
@@ -345,7 +367,7 @@ class JdbcRecordReader extends AbstractRecordReader {
     void copy(int index) throws SQLException {
       BigDecimal decimal = result.getBigDecimal(columnIndex);
       if (decimal != null) {
-        mutator.setSafe(index, decimal.doubleValue());
+        mutator.setSafe(index, decimal);
       }
     }
 
diff --git a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoCompareFunctionProcessor.java b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoCompareFunctionProcessor.java
index 130ea0f..b12887d 100644
--- a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoCompareFunctionProcessor.java
+++ b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoCompareFunctionProcessor.java
@@ -30,6 +30,7 @@ import org.apache.drill.common.expression.ValueExpressions.IntExpression;
 import org.apache.drill.common.expression.ValueExpressions.LongExpression;
 import org.apache.drill.common.expression.ValueExpressions.QuotedString;
 import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
+import org.apache.drill.common.expression.ValueExpressions.VarDecimalExpression;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
 
 import com.google.common.collect.ImmutableMap;
@@ -110,7 +111,7 @@ public class MongoCompareFunctionProcessor extends
   @Override
   public Boolean visitConvertExpression(ConvertExpression e,
       LogicalExpression valueArg) throws RuntimeException {
-    if (e.getConvertFunction() == ConvertExpression.CONVERT_FROM
+    if (ConvertExpression.CONVERT_FROM.equals(e.getConvertFunction())
         && e.getInput() instanceof SchemaPath) {
       String encodingType = e.getEncodingType();
       switch (encodingType) {
@@ -219,6 +220,14 @@ public class MongoCompareFunctionProcessor extends
       return true;
     }
 
+    // Mongo does not support decimals, therefore double value is used.
+    // See list of supported types in BsonValueCodecProvider.
+    if (valueArg instanceof VarDecimalExpression) {
+      this.value = ((VarDecimalExpression) valueArg).getBigDecimal().doubleValue();
+      this.path = path;
+      return true;
+    }
+
     return false;
   }
 
@@ -230,7 +239,7 @@ public class MongoCompareFunctionProcessor extends
         .add(DateExpression.class).add(DoubleExpression.class)
         .add(FloatExpression.class).add(IntExpression.class)
         .add(LongExpression.class).add(QuotedString.class)
-        .add(TimeExpression.class).build();
+        .add(TimeExpression.class).add(VarDecimalExpression.class).build();
   }
 
   private static final ImmutableMap<String, String> COMPARE_FUNCTIONS_TRANSPOSE_MAP;
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 8e3e4cf..cbc3a02 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -511,48 +511,43 @@
     <dependency>
       <groupId>org.apache.avro</groupId>
       <artifactId>avro</artifactId>
-      <version>1.7.7</version>
-      <exclusions>
-    	<exclusion>
-    	  <groupId>org.xerial.snappy</groupId>
-    	  <artifactId>snappy-java</artifactId>
-    	</exclusion>
-        <exclusion>
-            <groupId>io.netty</groupId>
-            <artifactId>netty</artifactId>
-        </exclusion>
-        <exclusion>
-            <groupId>io.netty</groupId>
-            <artifactId>netty-all</artifactId>
-        </exclusion>
-      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.avro</groupId>
       <artifactId>avro-mapred</artifactId>
-      <version>1.7.7</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <scope>test</scope>
+      <classifier>tests</classifier>
       <exclusions>
         <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>servlet-api</artifactId>
+          <groupId>io.netty</groupId>
+          <artifactId>netty</artifactId>
         </exclusion>
         <exclusion>
-            <groupId>io.netty</groupId>
-            <artifactId>netty</artifactId>
+          <groupId>io.netty</groupId>
+          <artifactId>netty-all</artifactId>
         </exclusion>
         <exclusion>
-            <groupId>io.netty</groupId>
-            <artifactId>netty-all</artifactId>
+          <groupId>commons-codec</groupId>
+          <artifactId>commons-codec</artifactId>
         </exclusion>
       </exclusions>
     </dependency>
     <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-      <scope>test</scope>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro</artifactId>
+      <version>${avro.version}</version>
       <classifier>tests</classifier>
+      <scope>test</scope>
       <exclusions>
         <exclusion>
+          <groupId>org.xerial.snappy</groupId>
+          <artifactId>snappy-java</artifactId>
+        </exclusion>
+        <exclusion>
           <groupId>io.netty</groupId>
           <artifactId>netty</artifactId>
         </exclusion>
@@ -560,10 +555,6 @@
           <groupId>io.netty</groupId>
           <artifactId>netty-all</artifactId>
         </exclusion>
-        <exclusion>
-          <groupId>commons-codec</groupId>
-          <artifactId>commons-codec</artifactId>
-        </exclusion>
       </exclusions>
     </dependency>
     <dependency>
diff --git a/exec/java-exec/src/main/codegen/config.fmpp b/exec/java-exec/src/main/codegen/config.fmpp
index c70f6da..50f110d 100644
--- a/exec/java-exec/src/main/codegen/config.fmpp
+++ b/exec/java-exec/src/main/codegen/config.fmpp
@@ -31,6 +31,7 @@ data: {
     aggrtypes1:               tdd(../data/AggrTypes1.tdd),
     decimalaggrtypes1:        tdd(../data/DecimalAggrTypes1.tdd),
     decimalaggrtypes2:        tdd(../data/DecimalAggrTypes2.tdd),
+    decimalaggrtypes3:        tdd(../data/DecimalAggrTypes3.tdd),
     aggrtypes2:               tdd(../data/AggrTypes2.tdd),
     aggrtypes3:               tdd(../data/AggrTypes3.tdd),
     covarTypes:               tdd(../data/CovarTypes.tdd),
diff --git a/exec/java-exec/src/main/codegen/data/Casts.tdd b/exec/java-exec/src/main/codegen/data/Casts.tdd
index 63fa0e4..e43572a 100644
--- a/exec/java-exec/src/main/codegen/data/Casts.tdd
+++ b/exec/java-exec/src/main/codegen/data/Casts.tdd
@@ -80,149 +80,65 @@
     {from: "IntervalYear", to: "Interval", major: "IntervalSimpleToComplex", months: "in.value", days: "0", millis: "0"},
     {from: "IntervalDay", to: "Interval", major: "IntervalSimpleToComplex", months: "0", days: "in.days", millis: "in.milliseconds"},
     {from: "Interval", to: "IntervalYear", major: "IntervalComplexToSimple"},
-    {from: "Interval", to: "IntervalDay", major: "IntervalComplexToSimple"}
-
-    {from: "Decimal9", to: "Decimal18", major: "DecimalSimpleDecimalSimple", javatype: "long"},
-
-    {from: "Decimal9", to: "Decimal28Dense", major: "DecimalSimpleDecimalDense", arraySize: "3"},
-    {from: "Decimal9", to: "Decimal38Dense", major: "DecimalSimpleDecimalDense", arraySize: "4"},
-    {from: "Decimal18", to: "Decimal28Dense", major: "DecimalSimpleDecimalDense", arraySize: "3"},
-    {from: "Decimal18", to: "Decimal38Dense", major: "DecimalSimpleDecimalDense", arraySize: "4"},
-
-    {from: "Decimal9", to: "Decimal28Sparse", major: "DecimalSimpleDecimalSparse", arraySize: "5"},
-    {from: "Decimal9", to: "Decimal38Sparse", major: "DecimalSimpleDecimalSparse", arraySize: "6"},
-    {from: "Decimal18", to: "Decimal28Sparse", major: "DecimalSimpleDecimalSparse", arraySize: "5"},
-    {from: "Decimal18", to: "Decimal38Sparse", major: "DecimalSimpleDecimalSparse", arraySize: "6"},
-
-    {from: "Decimal28Dense", to: "Decimal28Sparse", major: "DecimalDenseDecimalSparse", arraySize: "5"},
-    {from: "Decimal28Dense", to: "Decimal38Sparse", major: "DecimalDenseDecimalSparse",arraySize: "6"},
-    {from: "Decimal38Dense", to: "Decimal38Sparse", major: "DecimalDenseDecimalSparse", arraySize: "6"},
-
-    {from: "Decimal28Sparse", to: "Decimal28Dense", major: "DecimalSparseDecimalDense",  arraySize: "3"},
-    {from: "Decimal28Sparse", to: "Decimal38Dense", major: "DecimalSparseDecimalDense", arraySize: "4"},
-    {from: "Decimal38Sparse", to: "Decimal38Dense", major: "DecimalSparseDecimalDense", arraySize: "4"},
-
-    {from: "Decimal28Sparse", to: "VarDecimal", major: "DecimalSparseVarDecimal", arraySize: "6"},
-    {from: "Decimal38Sparse", to: "VarDecimal", major: "DecimalSparseVarDecimal", arraySize: "6"},
-
-    {from: "Decimal28Dense", to: "Decimal38Dense", major: "DecimalSimilar", arraySize: "4"},
-    {from: "Decimal28Sparse", to: "Decimal38Sparse", major: "DecimalSimilar", arraySize: "6"},
-
-    {from: "Int", to: "Decimal9", major: "IntDecimal", javatype: "int"},
-    {from: "Int", to: "Decimal18", major: "IntDecimal", javatype: "long"},
-    {from: "Int", to: "Decimal28Sparse", major: "IntDecimal", arraySize: "5"},
-    {from: "Int", to: "Decimal38Sparse", major: "IntDecimal", arraySize: "6"},
-    {from: "Int", to: "VarDecimal", major: "IntDecimal", arraySize: "6"},
-
-    {from: "BigInt", to: "Decimal9", major: "BigIntDecimal", javatype: "int"},
-    {from: "BigInt", to: "Decimal18", major: "BigIntDecimal", javatype: "long"},
-    {from: "BigInt", to: "Decimal28Sparse", major: "BigIntDecimal", arraySize: "5"},
-    {from: "BigInt", to: "Decimal38Sparse", major: "BigIntDecimal", arraySize: "6"},
-    {from: "BigInt", to: "VarDecimal", major: "BigIntDecimal", arraySize: "6"},
-
-    {from: "Decimal9", to: "Int", major: "DecimalSimpleInt", javatype: "int"},
-    {from: "Decimal18", to: "Int", major: "DecimalSimpleInt", javatype: "int"},
-    {from: "Decimal28Sparse", to: "Int", major: "DecimalComplexInt", javatype: "int"},
-    {from: "Decimal38Sparse", to: "Int", major: "DecimalComplexInt", javatype: "int"},
+    {from: "Interval", to: "IntervalDay", major: "IntervalComplexToSimple"},
+
+    {from: "Decimal9", to: "VarDecimal", major: "DecimalToVarDecimal"},
+    {from: "Decimal18", to: "VarDecimal", major: "DecimalToVarDecimal"},
+    {from: "Decimal28Sparse", to: "VarDecimal", major: "DecimalToVarDecimal"},
+    {from: "Decimal38Sparse", to: "VarDecimal", major: "DecimalToVarDecimal"},
+    {from: "VarDecimal", to: "VarDecimal", major: "DecimalToVarDecimal"},
+
+    {from: "VarDecimal", to: "Decimal9", major: "VarDecimalToDecimal"},
+    {from: "VarDecimal", to: "Decimal18", major: "VarDecimalToDecimal"},
+    {from: "VarDecimal", to: "Decimal28Sparse", major: "VarDecimalToDecimal", arraySize: "5"},
+    {from: "VarDecimal", to: "Decimal38Sparse", major: "VarDecimalToDecimal", arraySize: "6"},
+
+    {from: "Int", to: "VarDecimal", major: "IntDecimal"},
+
+    {from: "BigInt", to: "VarDecimal", major: "BigIntDecimal"},
+
     {from: "VarDecimal", to: "Int", major: "DecimalComplexInt", javatype: "int"},
 
-    {from: "Decimal9", to: "BigInt", major: "DecimalSimpleBigInt", javatype: "long"},
-    {from: "Decimal18", to: "BigInt", major: "DecimalSimpleBigInt", javatype: "long"},
-    {from: "Decimal28Sparse", to: "BigInt", major: "DecimalComplexBigInt", javatype: "long"},
-    {from: "Decimal38Sparse", to: "BigInt", major: "DecimalComplexBigInt", javatype: "long"},
     {from: "VarDecimal", to: "BigInt", major: "DecimalComplexBigInt", javatype: "long"},
 
-    {from: "Decimal9", to: "Float4", major: "DecimalSimpleFloat", javatype: "float"},
-    {from: "Decimal18", to: "Float4", major: "DecimalSimpleFloat", javatype: "float"},
-    {from: "Decimal28Sparse", to: "Float4", major: "DecimalComplexFloat", javatype: "float"},
-    {from: "Decimal28Dense", to: "Float4", major: "DecimalComplexFloat", javatype: "float"},
-    {from: "Decimal38Sparse", to: "Float4", major: "DecimalComplexFloat", javatype: "float"},
-    {from: "Decimal38Dense", to: "Float4", major: "DecimalComplexFloat", javatype: "float"},
     {from: "VarDecimal", to: "Float4", major: "DecimalComplexFloat", javatype: "float"},
 
-    {from: "Float4", to: "Decimal9", major: "FloatDecimalSimple", javatype: "int"},
-    {from: "Float4", to: "Decimal18", major: "FloatDecimalSimple", javatype: "long"},
-    {from: "Float4", to: "Decimal28Sparse", major: "FloatDecimalComplex", arraySize: "5"},
-    {from: "Float4", to: "Decimal38Sparse", major: "FloatDecimalComplex", arraySize: "6"},
-    {from: "Float4", to: "VarDecimal", major: "FloatDecimalComplex", arraySize: "6"},
-
-    {from: "Float8", to: "Decimal9", major: "DoubleDecimalSimple", javatype: "int"},
-    {from: "Float8", to: "Decimal18", major: "DoubleDecimalSimple", javatype: "long"},
-    {from: "Float8", to: "Decimal28Sparse", major: "DoubleDecimalComplex", arraySize: "5"},
-    {from: "Float8", to: "Decimal38Sparse", major: "DoubleDecimalComplex", arraySize: "6"}
-    {from: "Float8", to: "VarDecimal", major: "DoubleDecimalComplex", arraySize: "6"}
-
-    {from: "Decimal9", to: "Float8", major: "DecimalSimpleDouble", javatype: "double"},
-    {from: "Decimal18", to: "Float8", major: "DecimalSimpleDouble", javatype: "double"},
-    {from: "Decimal28Sparse", to: "Float8", major: "DecimalComplexDouble", javatype: "double"},
-    {from: "Decimal28Dense", to: "Float8", major: "DecimalComplexDouble", javatype: "double"},
-    {from: "Decimal38Sparse", to: "Float8", major: "DecimalComplexDouble", javatype: "double"},
-    {from: "Decimal38Dense", to: "Float8", major: "DecimalComplexDouble", javatype: "double"},
-    {from: "VarDecimal", to: "Float8", major: "DecimalComplexDouble", javatype: "double"},
-
-    {from: "VarChar", to: "Decimal9", major: "VarCharDecimalSimple", javatype: "int"},
-    {from: "VarChar", to: "Decimal18", major: "VarCharDecimalSimple", javatype: "long"},
-    {from: "VarChar", to: "Decimal28Sparse", major: "VarCharDecimalComplex", arraySize: "5"},
-    {from: "VarChar", to: "Decimal38Sparse", major: "VarCharDecimalComplex", arraySize: "6"},
-    {from: "VarChar", to: "VarDecimal", major: "VarCharDecimalComplex", arraySize: "6"},
+    {from: "Float4", to: "VarDecimal", major: "FloatDecimalComplex"},
 
-    {from: "Decimal9", to: "VarChar", major: "DecimalSimpleVarChar", bufferSize: "11", javatype: "int"},
-    {from: "Decimal18", to: "VarChar", major: "DecimalSimpleVarChar", bufferSize: "20", javatype: "long"},
-    {from: "Decimal28Sparse", to: "VarChar", major: "DecimalComplexVarChar", bufferSize: "30", arraySize: "5"},
-    {from: "Decimal38Sparse", to: "VarChar", major: "DecimalComplexVarChar", bufferSize: "40", arraySize: "6"},
-    {from: "VarDecimal", to: "VarChar", major: "DecimalComplexVarChar", bufferSize: "40", arraySize: "6"},
+    {from: "Float8", to: "VarDecimal", major: "DoubleDecimalComplex"},
 
-    {from: "Decimal18", to: "Decimal9", major: "DownwardDecimalSimpleDecimalSimple", javatype: "int"},
+    {from: "VarDecimal", to: "Float8", major: "DecimalComplexDouble", javatype: "double"},
 
-    {from: "Decimal28Sparse", to: "Decimal18", major: "DownwardDecimalSimpleDecimalComplex", javatype: "long"},
-    {from: "Decimal28Sparse", to: "Decimal9", major: "DownwardDecimalSimpleDecimalComplex", javatype: "int"},
+    {from: "VarChar", to: "VarDecimal", major: "VarCharDecimalComplex"},
 
-    {from: "Decimal38Sparse", to: "Decimal28Sparse", major: "DownwardDecimalComplexDecimalComplex", arraySize: "5"},
-    {from: "Decimal38Sparse", to: "Decimal18", major: "DownwardDecimalComplexDecimalSimple", javatype: "long"},
-    {from: "Decimal38Sparse", to: "Decimal9", major: "DownwardDecimalComplexDecimalSimple", javatype: "int"},
+    {from: "VarDecimal", to: "VarChar", major: "DecimalComplexVarChar"},
 
     {from: "VarChar", to: "NullableInt", major: "EmptyString", javaType:"Integer", primeType:"int"},
     {from: "VarChar", to: "NullableBigInt", major: "EmptyString", javaType: "Long", primeType: "long"},
     {from: "VarChar", to: "NullableFloat4", major: "EmptyString", javaType:"Float", parse:"Float"},
     {from: "VarChar", to: "NullableFloat8", major: "EmptyString", javaType:"Double", parse:"Double"},
 
-    {from: "VarChar", to: "NullableDecimal9", major: "EmptyStringVarCharDecimalSimple", javatype: "int"},
-    {from: "VarChar", to: "NullableDecimal18", major: "EmptyStringVarCharDecimalSimple", javatype: "long"},
-    {from: "VarChar", to: "NullableDecimal28Sparse", major: "EmptyStringVarCharDecimalComplex", arraySize: "5"},
-    {from: "VarChar", to: "NullableDecimal38Sparse", major: "EmptyStringVarCharDecimalComplex", arraySize: "6"},
-    {from: "VarChar", to: "NullableVarDecimal", major: "EmptyStringVarCharDecimalComplex", arraySize: "6"},
+    {from: "VarChar", to: "NullableVarDecimal", major: "EmptyStringVarCharDecimalComplex"},
 
     {from: "NullableVarChar", to: "NullableInt", major: "EmptyString", javaType:"Integer", primeType:"int"},
     {from: "NullableVarChar", to: "NullableBigInt", major: "EmptyString", javaType: "Long", primeType: "long"},
     {from: "NullableVarChar", to: "NullableFloat4", major: "EmptyString", javaType:"Float", parse:"Float"},
     {from: "NullableVarChar", to: "NullableFloat8", major: "EmptyString", javaType:"Double", parse:"Double"},
 
-    {from: "NullableVarChar", to: "NullableDecimal9", major: "EmptyStringVarCharDecimalSimple", javatype: "int"},
-    {from: "NullableVarChar", to: "NullableDecimal18", major: "EmptyStringVarCharDecimalSimple", javatype: "long"},
-    {from: "NullableVarChar", to: "NullableDecimal28Sparse", major: "EmptyStringVarCharDecimalComplex", arraySize: "5"},
-    {from: "NullableVarChar", to: "NullableDecimal38Sparse", major: "EmptyStringVarCharDecimalComplex", arraySize: "6"},
-    {from: "NullableVarChar", to: "NullableVarDecimal", major: "EmptyStringVarCharDecimalComplex", arraySize: "6"},
+    {from: "NullableVarChar", to: "NullableVarDecimal", major: "EmptyStringVarCharDecimalComplex"},
 
     {from: "NullableVar16Char", to: "NullableInt", major: "EmptyString", javaType:"Integer", primeType:"int"},
     {from: "NullableVar16Char", to: "NullableBigInt", major: "EmptyString", javaType: "Long", primeType: "long"},
     {from: "NullableVar16Char", to: "NullableFloat4", major: "EmptyString", javaType:"Float", parse:"Float"},
     {from: "NullableVar16Char", to: "NullableFloat8", major: "EmptyString", javaType:"Double", parse:"Double"},
 
-    {from: "NullableVar16Char", to: "NullableDecimal9", major: "EmptyStringVarCharDecimalSimple", javatype: "int"},
-    {from: "NullableVar16Char", to: "NullableDecimal18", major: "EmptyStringVarCharDecimalSimple", javatype: "long"},
-    {from: "NullableVar16Char", to: "NullableDecimal28Sparse", major: "EmptyStringVarCharDecimalComplex", arraySize: "5"},
-    {from: "NullableVar16Char", to: "NullableDecimal38Sparse", major: "EmptyStringVarCharDecimalComplex", arraySize: "6"},
-    {from: "NullableVar16Char", to: "NullableVarDecimal", major: "EmptyStringVarCharDecimalComplex", arraySize: "6"},
+    {from: "NullableVar16Char", to: "NullableVarDecimal", major: "EmptyStringVarCharDecimalComplex"},
 
     {from: "NullableVarBinary", to: "NullableInt", major: "EmptyString", javaType:"Integer", primeType:"int"},
     {from: "NullableVarBinary", to: "NullableBigInt", major: "EmptyString", javaType: "Long", primeType: "long"},
     {from: "NullableVarBinary", to: "NullableFloat4", major: "EmptyString", javaType:"Float", parse:"Float"},
     {from: "NullableVarBinary", to: "NullableFloat8", major: "EmptyString", javaType:"Double", parse:"Double"},
 
-    {from: "NullableVarBinary", to: "NullableDecimal9", major: "EmptyStringVarCharDecimalSimple", javatype: "int"},
-    {from: "NullableVarBinary", to: "NullableDecimal18", major: "EmptyStringVarCharDecimalSimple", javatype: "long"},
-    {from: "NullableVarBinary", to: "NullableDecimal28Sparse", major: "EmptyStringVarCharDecimalComplex", arraySize: "5"},
-    {from: "NullableVarBinary", to: "NullableDecimal38Sparse", major: "EmptyStringVarCharDecimalComplex", arraySize: "6"},
-    {from: "NullableVarBinary", to: "NullableVarDecimal", major: "EmptyStringVarCharDecimalComplex", arraySize: "6"},
+    {from: "NullableVarBinary", to: "NullableVarDecimal", major: "EmptyStringVarCharDecimalComplex"},
   ]
-} 
+}
diff --git a/exec/java-exec/src/main/codegen/data/ComparisonTypesDecimal.tdd b/exec/java-exec/src/main/codegen/data/ComparisonTypesDecimal.tdd
index db73825..f5b912e 100644
--- a/exec/java-exec/src/main/codegen/data/ComparisonTypesDecimal.tdd
+++ b/exec/java-exec/src/main/codegen/data/ComparisonTypesDecimal.tdd
@@ -13,11 +13,9 @@
 # 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.
-#
-# TODO: the assumption that VarDecimal takes 6 bytes might not always apply
 {
   { decimalTypes: [
-      {name: "VarDecimal", storage: "6"},
+      {name: "VarDecimal"},
       {name: "Decimal28Sparse", storage: "5"},
       {name: "Decimal38Sparse", storage: "6"},
       {name: "Decimal28Dense", storage: "4"},
diff --git a/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd b/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd
index 188517a..21fcf20 100644
--- a/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd
+++ b/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd
@@ -40,6 +40,8 @@
     "NullableVarBinary",
     "RepeatedVarBinary"
     "VarDecimal",
+    "NullableVarDecimal",
+    "RepeatedVarDecimal",
     "Decimal9",
     "NullableDecimal9",
     "RepeatedDecimal9",
diff --git a/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd b/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd
index f246291..7da2d07 100644
--- a/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd
+++ b/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd
@@ -17,43 +17,24 @@
 {
   aggrtypes: [
    {className: "Max", funcName: "max", types: [
-      {inputType: "Decimal9", outputType: "Decimal9", runningType: "Decimal9", initValue: "Integer.MIN_VALUE"},
-      {inputType: "NullableDecimal9", outputType: "Decimal9", runningType: "Decimal9", initValue: "Integer.MIN_VALUE"},
-      {inputType: "Decimal18", outputType: "Decimal18", runningType: "Decimal18", initValue: "Long.MIN_VALUE"},
-      {inputType: "NullableDecimal18", outputType: "Decimal18", runningType: "Decimal18", initValue: "Long.MIN_VALUE"},
-      {inputType: "Decimal28Sparse", outputType: "Decimal28Sparse", runningType: "Decimal28Sparse"},
-      {inputType: "NullableDecimal28Sparse", outputType: "Decimal28Sparse", runningType: "Decimal28Sparse"},
-      {inputType: "Decimal38Sparse", outputType: "Decimal38Sparse", runningType: "Decimal38Sparse"},
-      {inputType: "NullableDecimal38Sparse", outputType: "Decimal38Sparse", runningType: "Decimal38Sparse"},
-      {inputType: "VarDecimal", outputType: "VarDecimal", runningType: "VarDecimal"},
-      {inputType: "NullableVarDecimal", outputType: "VarDecimal", runningType: "VarDecimal"}
+      {inputType: "VarDecimal", outputType: "NullableVarDecimal"},
+      {inputType: "NullableVarDecimal", outputType: "NullableVarDecimal"}
      ]
    },
    {className: "Min", funcName: "min", types: [
-      {inputType: "Decimal9", outputType: "Decimal9", runningType: "Decimal9", initValue: "Integer.MAX_VALUE"},
-      {inputType: "NullableDecimal9", outputType: "Decimal9", runningType: "Decimal9", initValue: "Integer.MAX_VALUE"},
-      {inputType: "Decimal18", outputType: "Decimal18", runningType: "Decimal18", initValue: "Integer.MAX_VALUE"},
-      {inputType: "NullableDecimal18", outputType: "Decimal18", runningType: "Decimal18", initValue: "Integer.MAX_VALUE"},
-      {inputType: "Decimal28Sparse", outputType: "Decimal28Sparse", runningType: "Decimal28Sparse"},
-      {inputType: "NullableDecimal28Sparse", outputType: "Decimal28Sparse", runningType: "Decimal28Sparse"},
-      {inputType: "Decimal38Sparse", outputType: "Decimal38Sparse", runningType: "Decimal38Sparse"},
-      {inputType: "NullableDecimal38Sparse", outputType: "Decimal38Sparse", runningType: "Decimal38Sparse"},
-      {inputType: "VarDecimal", outputType: "VarDecimal", runningType: "VarDecimal"},
-      {inputType: "NullableVarDecimal", outputType: "VarDecimal", runningType: "VarDecimal"}
+      {inputType: "VarDecimal", outputType: "NullableVarDecimal"},
+      {inputType: "NullableVarDecimal", outputType: "NullableVarDecimal"}
      ]
+   },
+   {className: "Sum", funcName: "sum", types: [
+       {inputType: "VarDecimal", outputType: "NullableVarDecimal"},
+       {inputType: "NullableVarDecimal", outputType: "NullableVarDecimal"}
+      ]
+   },
+   {className: "SumZero", funcName: "$sum0", types: [
+       {inputType: "VarDecimal", outputType: "NullableVarDecimal"},
+       {inputType: "NullableVarDecimal", outputType: "NullableVarDecimal"}
+      ]
    }
-      {className: "Sum", funcName: "sum", types: [
-        {inputType: "Decimal9", outputType: "Decimal38Sparse", major: "Numeric"},
-        {inputType: "NullableDecimal9", outputType: "Decimal38Sparse", major: "Numeric"},
-        {inputType: "Decimal18", outputType: "Decimal38Sparse", major: "Numeric"},
-        {inputType: "NullableDecimal18", outputType: "Decimal38Sparse", major: "Numeric"},
-        {inputType: "Decimal28Sparse", outputType: "Decimal38Sparse", major: "Numeric"},
-        {inputType: "NullableDecimal28Sparse", outputType: "Decimal38Sparse", major: "Numeric"},
-        {inputType: "Decimal38Sparse", outputType: "Decimal38Sparse", major: "Numeric"},
-        {inputType: "NullableDecimal38Sparse", outputType: "Decimal38Sparse", major: "Numeric"},
-        {inputType: "VarDecimal", outputType: "VarDecimal", major: "Numeric"},
-        {inputType: "NullableVarDecimal", outputType: "VarDecimal", major: "Numeric"}
-        ]
-      }
-   ]
+  ]
 }
diff --git a/exec/java-exec/src/main/codegen/data/DecimalAggrTypes2.tdd b/exec/java-exec/src/main/codegen/data/DecimalAggrTypes2.tdd
index faa0afd..36a9233 100644
--- a/exec/java-exec/src/main/codegen/data/DecimalAggrTypes2.tdd
+++ b/exec/java-exec/src/main/codegen/data/DecimalAggrTypes2.tdd
@@ -17,16 +17,8 @@
 {
   aggrtypes: [
     {className: "Avg", funcName: "avg", types: [
-        {inputType: "Decimal9", outputType: "Decimal38Sparse", countRunningType: "BigInt",  major: "Numeric"},
-        {inputType: "NullableDecimal9", outputType: "Decimal38Sparse", countRunningType: "BigInt", major: "Numeric"},
-        {inputType: "Decimal18", outputType: "Decimal38Sparse", countRunningType: "BigInt", major: "Numeric"},
-        {inputType: "NullableDecimal18", outputType: "Decimal38Sparse", countRunningType: "BigInt", major: "Numeric"},
-        {inputType: "Decimal28Sparse", outputType: "Decimal38Sparse", countRunningType: "BigInt", major: "Numeric"},
-        {inputType: "NullableDecimal28Sparse", outputType: "Decimal38Sparse", countRunningType: "BigInt", major: "Numeric"},
-        {inputType: "Decimal38Sparse", outputType: "Decimal38Sparse", countRunningType: "BigInt", major: "Numeric"},
-        {inputType: "NullableDecimal38Sparse", outputType: "Decimal38Sparse", countRunningType: "BigInt", major: "Numeric"},
-        {inputType: "VarDecimal", outputType: "VarDecimal", countRunningType: "BigInt", major: "Numeric"},
-        {inputType: "NullableVarDecimal", outputType: "VarDecimal", countRunningType: "BigInt", major: "Numeric"}
+        {inputType: "VarDecimal", outputType: "NullableVarDecimal", countRunningType: "BigInt"},
+        {inputType: "NullableVarDecimal", outputType: "NullableVarDecimal", countRunningType: "BigInt"}
       ]
     }
   ]
diff --git a/exec/java-exec/src/main/codegen/data/DecimalAggrTypes3.tdd b/exec/java-exec/src/main/codegen/data/DecimalAggrTypes3.tdd
new file mode 100644
index 0000000..670c53d
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/data/DecimalAggrTypes3.tdd
@@ -0,0 +1,43 @@
+# 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.
+
+{
+  aggrtypes: [
+    {className: "StdDevPop", funcName: "stddev_pop", aliasName: "", types: [
+        {inputType: "VarDecimal", outputType: "NullableVarDecimal"},
+        {inputType: "NullableVarDecimal", outputType: "NullableVarDecimal"},
+      ]
+    },
+
+    {className: "VariancePop", funcName: "var_pop", aliasName: "", types: [
+        {inputType: "VarDecimal", outputType: "NullableVarDecimal"},
+        {inputType: "NullableVarDecimal", outputType: "NullableVarDecimal"},
+      ]
+    },
+
+    {className: "StdDevSample", funcName: "stddev_samp", aliasName: "stddev", types: [
+        {inputType: "VarDecimal", outputType: "NullableVarDecimal"},
+        {inputType: "NullableVarDecimal", outputType: "NullableVarDecimal"},
+      ]
+    },
+
+    {className: "VarianceSample", funcName: "var_samp", aliasName: "variance", types: [
+        {inputType: "VarDecimal", outputType: "NullableVarDecimal"},
+        {inputType: "NullableVarDecimal", outputType: "NullableVarDecimal"},
+      ]
+    }
+  ]
+}
diff --git a/exec/java-exec/src/main/codegen/data/MathFunc.tdd b/exec/java-exec/src/main/codegen/data/MathFunc.tdd
index cd44916..51a1b28 100644
--- a/exec/java-exec/src/main/codegen/data/MathFunc.tdd
+++ b/exec/java-exec/src/main/codegen/data/MathFunc.tdd
@@ -199,9 +199,7 @@ logBaseMathFunction : [
       {input: "UInt1"},
       {input: "UInt2"},
       {input: "UInt4"},
-      {input: "UInt8"},
-      {input: "Decimal9"},
-      {input: "Decimal18"}
+      {input: "UInt8"}
     ]
    }
 ],
@@ -216,9 +214,7 @@ extendedUnaryMathFunctions : [
       {input: "UInt1"},
       {input: "UInt2"},
       {input: "UInt4"},
-      {input: "UInt8"},
-      {input: "Decimal9"},
-      {input: "Decimal18"}
+      {input: "UInt8"}
     ]
    },
 
@@ -232,9 +228,7 @@ extendedUnaryMathFunctions : [
       {input: "UInt1"},
       {input: "UInt2"},
       {input: "UInt4"},
-      {input: "UInt8"},
-      {input: "Decimal9"},
-      {input: "Decimal18"}
+      {input: "UInt8"}
     ]
    },
 
@@ -248,9 +242,7 @@ extendedUnaryMathFunctions : [
       {input: "UInt1"},
       {input: "UInt2"},
       {input: "UInt4"},
-      {input: "UInt8"},
-      {input: "Decimal9"},
-      {input: "Decimal18"}
+      {input: "UInt8"}
     ]
    },
 
@@ -278,9 +270,7 @@ extendedUnaryMathFunctions : [
       {input: "UInt1"},
       {input: "UInt2"},
       {input: "UInt4"},
-      {input: "UInt8"},
-      {input: "Decimal9"},
-      {input: "Decimal18"}
+      {input: "UInt8"}
     ]
    },
 
@@ -294,9 +284,7 @@ extendedUnaryMathFunctions : [
       {input: "UInt1"},
       {input: "UInt2"},
       {input: "UInt4"},
-      {input: "UInt8"},
-      {input: "Decimal9"},
-      {input: "Decimal18"}
+      {input: "UInt8"}
     ]
    }
 ],
diff --git a/exec/java-exec/src/main/codegen/data/NumericTypes.tdd b/exec/java-exec/src/main/codegen/data/NumericTypes.tdd
index afc0ee8..9f33353 100644
--- a/exec/java-exec/src/main/codegen/data/NumericTypes.tdd
+++ b/exec/java-exec/src/main/codegen/data/NumericTypes.tdd
@@ -15,8 +15,7 @@
 # limitations under the License.
 
  {
-   numeric: ["Int", "BigInt", "TinyInt", "SmallInt", "UInt1", "UInt2", "UInt4", "UInt8", "Float4", "Float8", "VarDecimal",
-             "Decimal9", "Decimal18", "Decimal28Dense", "Decimal28Sparse", "Decimal38Dense", "Decimal38Sparse"],
+   numeric: ["Int", "BigInt", "TinyInt", "SmallInt", "UInt1", "UInt2", "UInt4", "UInt8", "Float4", "Float8", "VarDecimal"],
 
   numericFunctions: [
   {className: "IsNumeric", funcName: "isnumeric", aliasName: "", outputType: "Int", types: [
diff --git a/exec/java-exec/src/main/codegen/templates/CastHigh.java b/exec/java-exec/src/main/codegen/templates/CastHigh.java
index 26d6be8..1a876c8 100644
--- a/exec/java-exec/src/main/codegen/templates/CastHigh.java
+++ b/exec/java-exec/src/main/codegen/templates/CastHigh.java
@@ -42,26 +42,30 @@ public class CastHighFunctions {
   <#list casthigh.types as type>
 
   @SuppressWarnings("unused")
-  @FunctionTemplate(name = "casthigh", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "casthigh",
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    <#if type.from.contains("Decimal")>
+                    returnType = FunctionTemplate.ReturnType.DECIMAL_AGGREGATE,
+                    </#if>
+                    nulls = NullHandling.NULL_IF_NULL)
   public static class CastHigh${type.from} implements DrillSimpleFunc {
 
     @Param ${type.from}Holder in;
     <#if type.from.contains("Decimal")>
-      @Output ${type.from}Holder out;
+    @Output ${type.from}Holder out;
     <#else>
-      @Output ${type.to}Holder out;
+    @Output ${type.to}Holder out;
     </#if>
 
     public void setup() {}
 
     public void eval() {
-      <#if type.value >
-        out.value = (double) in.value;
+      <#if type.value>
+      out.value = (double) in.value;
       <#else>
-        out = in;
+      out = in;
       </#if>
     }
   }
 </#list>
 }
-
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToTimeStampFunction.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToTimeStampFunction.java
index 63c7614..b6cbfeb 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToTimeStampFunction.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToTimeStampFunction.java
@@ -40,41 +40,27 @@ import org.apache.drill.exec.record.RecordBatch;
  * This class is generated using freemarker and the ${.template_name} template.
  */
 
-@FunctionTemplate(name = "to_timestamp" , scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "to_timestamp",
+                  scope = FunctionTemplate.FunctionScope.SIMPLE,
+                  nulls = NullHandling.NULL_IF_NULL)
 public class G${numerics}ToTimeStamp implements DrillSimpleFunc {
 
+  @Param  ${numerics}Holder left;
+  @Output TimeStampHolder out;
 
-    @Param  ${numerics}Holder left;
-    <#if numerics.startsWith("Decimal")>
-    @Workspace java.math.BigInteger millisConstant;
-    </#if>
-    @Output TimeStampHolder out;
-
-    public void setup() {
-      <#if numerics.startsWith("Decimal")>
-      millisConstant = java.math.BigInteger.valueOf(1000);
-      </#if>
-    }
+  public void setup() {
+  }
 
-    public void eval() {
-        long inputMillis = 0;
+  public void eval() {
+    long inputMillis = 0;
 
-        <#if (numerics.contains("Decimal"))>
-        <#if (numerics == "VarDecimal")>
-        java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(left.buffer, left.start, left.end - left.start, left.scale);
-        inputMillis = input.multiply(new java.math.BigDecimal(1000)).longValue();
-        <#elseif (numerics == "Decimal9") || (numerics == "Decimal18")>
-        java.math.BigInteger value = java.math.BigInteger.valueOf(left.value);
-        value = value.multiply(millisConstant);
-        inputMillis = (new java.math.BigDecimal(value, left.scale)).longValue();
-        <#elseif (numerics == "Decimal28Sparse") || (numerics == "Decimal38Sparse")>
-        java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(left.buffer, left.start, left.nDecimalDigits, left.scale);
-        inputMillis = input.multiply(new java.math.BigDecimal(1000)).longValue();
-        </#if>
-        <#else>
-        inputMillis = (long) (left.value * 1000l);
-        </#if>
-        out.value = new org.joda.time.DateTime(inputMillis).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis();
-    }
+    <#if (numerics == "VarDecimal")>
+    java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(left.buffer, left.start, left.end - left.start, left.scale);
+    inputMillis = input.multiply(new java.math.BigDecimal(1000)).longValue();
+    <#else>
+    inputMillis = (long) (left.value * 1000L);
+    </#if>
+    out.value = new org.joda.time.DateTime(inputMillis).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis();
+  }
 }
 </#list>
\ No newline at end of file
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalDenseDecimalSparse.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalDenseDecimalSparse.java
deleted file mode 100644
index d637dde..0000000
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalDenseDecimalSparse.java
+++ /dev/null
@@ -1,170 +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 />
-
-<#list cast.types as type>
-<#if type.major == "DecimalDenseDecimalSparse">
-
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl.gcast;
-
-<#include "/@includes/vv_imports.ftl" />
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.expr.annotations.Workspace;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.DrillBuf;
-
-import java.nio.ByteBuffer;
-/*
- * This class is generated using freemarker and the ${.template_name} template.
- */
-
-@SuppressWarnings("unused")
-@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;
-    @Inject DrillBuf buffer;
-    @Param BigIntHolder precision;
-    @Param BigIntHolder scale;
-    @Output ${type.to}Holder out;
-
-    public void setup() {
-        int size = (${type.arraySize} * (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE));
-        buffer = buffer.reallocIfNeeded(size);
-    }
-
-    public void eval() {
-        out.buffer = buffer;
-        out.start = 0;
-
-        // Re initialize the buffer everytime
-        for (int i = 0; i < ${type.arraySize}; i++) {
-            out.setInteger(i, 0, out.start, out.buffer);
-        }
-        out.scale = (int) scale.value;
-        out.precision = (int) precision.value;
-        out.setSign(in.getSign(in.start, in.buffer), out.start, out.buffer);
-
-        /* We store base 1 Billion integers in our representation, which requires
-         * 30 bits, but a typical integer requires 32 bits. In our dense representation
-         * we shift bits around to utilize the two available bits, to get back to our sparse
-         * representation rearrange the bits so that we use 32 bits represent the digits.
-         */
-
-        byte[] intermediateBytes = new byte[(in.nDecimalDigits * org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE) + 1];
-
-        int[] mask = {0x03, 0x0F, 0x3F, 0xFF};
-        int[] reverseMask = {0xFC, 0xF0, 0xC0, 0x00};
-
-        <#if (type.from == "Decimal38Dense")>
-        int maskIndex = 0;
-        int shiftOrder = 6;
-        byte shiftBits = 0x00;
-        intermediateBytes[0] = (byte) (in.buffer.getByte(0) & 0x7F);
-        <#elseif (type.from == "Decimal28Dense")>
-        int maskIndex = 1;
-        int shiftOrder = 4;
-        byte shiftBits = (byte) ((in.buffer.getByte(0) & 0x03) << shiftOrder);
-        intermediateBytes[0] = (byte) (((in.buffer.getByte(0) & 0x3C) & 0xFF) >>> 2);
-        </#if>
-
-        int intermediateIndex = 1;
-        int inputIndex = in.start + 1;
-
-        while (intermediateIndex < in.WIDTH) {
-
-            intermediateBytes[intermediateIndex] = (byte) ((shiftBits) | (((in.buffer.getByte(inputIndex) & reverseMask[maskIndex]) & 0xFF) >>> (8 - shiftOrder)));
-
-            shiftBits = (byte) ((in.buffer.getByte(inputIndex) & mask[maskIndex]) << shiftOrder);
-
-            inputIndex++;
-            intermediateIndex++;
-
-            if (((intermediateIndex - 1) % org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE) == 0) {
-                shiftBits = (byte) ((shiftBits & 0xFF) >>> 2);
-                maskIndex++;
-                shiftOrder -= 2;
-            }
-
-        }
-
-        /* copy the last byte */
-        intermediateBytes[intermediateIndex] = shiftBits;
-
-        /* We have shifted the bits around and now each digit is represented by 32 digits
-         * Now we transfer the bytes into a integer array and separate out the scale and
-         * integer part of the decimal. Also pad the scale part with zeroes if needed
-         */
-        int[] intermediate = new int[(intermediateBytes.length/org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE) + 1];
-
-        java.nio.ByteBuffer wrapper = java.nio.ByteBuffer.wrap(intermediateBytes);
-        intermediate[0] = wrapper.get(0);
-
-        int intermediateIdx = 1;
-
-        for (int i = 1; i < intermediate.length; i++) {
-            intermediate[i] = wrapper.getInt(intermediateIdx);
-            intermediateIdx += 4;
-        }
-
-        int actualDigits;
-        int srcIndex = intermediate.length - 1;
-        int dstIndex = out.nDecimalDigits - 1;
-
-        // break the scale and integer part and pad zeroes
-        if (in.scale > 0 && (actualDigits = (in.scale % org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS)) > 0) {
-
-            int paddedDigits = org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS - actualDigits;
-            int padding = (int) (Math.pow(10, paddedDigits));
-            int transferDigitMask = (int) (Math.pow(10, actualDigits));
-
-            /* copy the remaining scale over to the last deciml digit */
-            out.setInteger(dstIndex, ((intermediate[srcIndex] % transferDigitMask) * (padding)), out.start, out.buffer);
-            dstIndex--;
-
-            while (srcIndex > 0) {
-                out.setInteger(dstIndex, ((intermediate[srcIndex]/transferDigitMask) + ((intermediate[srcIndex - 1] % transferDigitMask) * padding)), out.start, out.buffer);
-
-                dstIndex--;
-                srcIndex--;
-            }
-
-            out.setInteger(dstIndex, (intermediate[0]/transferDigitMask), out.start, out.buffer);
-        } else {
-            for (; srcIndex >= 0; srcIndex--, dstIndex--)
-                out.setInteger(dstIndex, intermediate[srcIndex], out.start, out.buffer);
-        }
-    }
-}
-</#if>
-</#list>
\ No newline at end of file
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalFloat.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalFloat.java
index fe79409..15686ba 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalFloat.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalFloat.java
@@ -19,47 +19,7 @@
 
 
 <#list cast.types as type>
-<#if type.major == "DecimalSimpleFloat" || type.major == "DecimalSimpleDouble"> <#-- Cast function template for conversion from Decimal9, Decimal18 to Float4 and Float8-->
-
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl.gcast;
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.util.DecimalUtility;
-import org.apache.drill.exec.expr.annotations.Workspace;
-import io.netty.buffer.ByteBuf;
-import java.nio.ByteBuffer;
-
-/*
- * 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)
-public class Cast${type.from}${type.to} implements DrillSimpleFunc {
-
-@Param ${type.from}Holder in;
-@Output ${type.to}Holder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-
-        // Divide the decimal with the scale to get the floating point value
-        out.value = ((${type.javatype}) (in.value)) / (org.apache.drill.exec.util.DecimalUtility.getPowerOfTen((int) in.scale));
-    }
-}
-<#elseif type.major == "DecimalComplexFloat" || type.major == "DecimalComplexDouble"> <#-- Cast function template for conversion from Decimal9, Decimal18 to Float4 -->
+<#if type.major == "DecimalComplexFloat" || type.major == "DecimalComplexDouble"> <#-- Cast function template for conversion from VarDecimal to Float4 -->
 
 <@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
 
@@ -86,26 +46,21 @@ 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,
+                  nulls = NullHandling.NULL_IF_NULL)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc {
 
-@Param ${type.from}Holder in;
-@Output ${type.to}Holder out;
-
-    public void setup() {
-    }
+  @Param ${type.from}Holder in;
+  @Output ${type.to}Holder out;
 
-    public void eval() {
+  public void setup() {
+  }
 
-        <#if type.from == "VarDecimal">
-        java.math.BigDecimal bigDecimal = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
-        <#elseif type.from == "Decimal28Dense" || type.from == "Decimal38Dense">
-        java.math.BigDecimal bigDecimal = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDense(in.buffer, in.start, in.nDecimalDigits, in.scale, in.maxPrecision, in.WIDTH);
-        <#else>
-        java.math.BigDecimal bigDecimal = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer, in.start, in.nDecimalDigits, in.scale, true);
-        </#if>
-        out.value = bigDecimal.${type.javatype}Value();
-    }
+  public void eval() {
+    java.math.BigDecimal bigDecimal = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
+    out.value = bigDecimal.${type.javatype}Value();
+  }
 }
 </#if>
 </#list>
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalInt.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalInt.java
index 3641818..e5123b2 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalInt.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalInt.java
@@ -19,17 +19,13 @@
 
 
 <#list cast.types as type>
-
-<#if type.major == "DecimalSimpleInt" || type.major == "DecimalSimpleBigInt"> <#-- Cast function template for conversion from Decimal9, Decimal18 to Int and BigInt -->
-
+<#if type.major == "DecimalComplexInt" || type.major == "DecimalComplexBigInt"> <#-- Cast function template for conversion from VarDecimal to Int and BigInt -->
 <@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
 
 <#include "/@includes/license.ftl" />
 
 package org.apache.drill.exec.expr.fn.impl.gcast;
 
-<#include "/@includes/vv_imports.ftl" />
-
 import org.apache.drill.exec.expr.DrillSimpleFunc;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
@@ -45,86 +41,24 @@ import java.nio.ByteBuffer;
 /*
  * 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)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc {
 
-@Param ${type.from}Holder in;
-@Output ${type.to}Holder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-
-        int carry = (org.apache.drill.exec.util.DecimalUtility.getFirstFractionalDigit(in.value, in.scale) > 4)
-                    ? (int) java.lang.Math.signum(in.value) : 0;
-        // Assign the integer part of the decimal to the output holder
-        out.value = (${type.javatype}) (org.apache.drill.exec.util.DecimalUtility.adjustScaleDivide(in.value, (int) in.scale) + carry);
-    }
-}
-
-<#elseif type.major == "DecimalComplexInt" || type.major == "DecimalComplexBigInt"> <#-- Cast function template for conversion from Decimal28Sparse, Decimal38Sparse to Int and BigInt -->
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl.gcast;
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.util.DecimalUtility;
-import org.apache.drill.exec.expr.annotations.Workspace;
-import io.netty.buffer.ByteBuf;
-import java.nio.ByteBuffer;
-
-/*
- * 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)
-public class Cast${type.from}${type.to} implements DrillSimpleFunc {
-
-@Param ${type.from}Holder in;
-@Output ${type.to}Holder out;
-
-    public void setup() {
-    }
+  @Param ${type.from}Holder in;
+  @Output ${type.to}Holder out;
 
-    public void eval() {
+  public void setup() {
+  }
 
-<#if type.from.contains("VarDecimal")>
-        java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
-        bd.setScale(0, java.math.BigDecimal.ROUND_HALF_UP);
-        long lval = bd.longValue();   // round off to nearest integer
-        out.value = (${type.javatype}) lval;
-<#else>
-        int carry = (org.apache.drill.exec.util.DecimalUtility.getFirstFractionalDigit(in.buffer, in.scale, in.start, in.nDecimalDigits) > 4) ? 1 : 0;
-
-        // Get the index, where the integer part of the decimal ends
-        int integerEndIndex = in.nDecimalDigits - org.apache.drill.exec.util.DecimalUtility.roundUp(in.scale);
-
-        for (int i = 0 ; i < integerEndIndex; i++) {
-            // We store values as base 1 billion integers, use this to compute the output (we don't care about overflows)
-            out.value = (${type.javatype}) ((out.value * org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE) + in.getInteger(i, in.start, in.buffer));
-        }
-
-        out.value += carry;
-
-        if (in.getSign(in.start, in.buffer) == true) {
-            out.value *= -1;
-        }
-</#if>
-    }
+  public void eval() {
+    java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
+    long lval = bd.setScale(0, java.math.BigDecimal.ROUND_HALF_UP).longValue(); // round off to nearest integer
+    out.value = (${type.javatype}) lval;
+  }
 }
-
 </#if> <#-- type.major -->
 </#list>
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSparseDecimalDense.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSparseDecimalDense.java
deleted file mode 100644
index 69010ef..0000000
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSparseDecimalDense.java
+++ /dev/null
@@ -1,185 +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 />
-
-<#list cast.types as type>
-<#if type.major == "DecimalSparseDecimalDense">
-
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl.gcast;
-
-<#include "/@includes/vv_imports.ftl" />
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.expr.annotations.Workspace;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.DrillBuf;
-
-import java.nio.ByteBuffer;
-
-/*
- * This class is generated using freemarker and the ${.template_name} template.
- */
-
-@SuppressWarnings("unused")
-@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;
-    @Inject DrillBuf buffer;
-    @Param BigIntHolder precision;
-    @Param BigIntHolder scale;
-    @Output ${type.to}Holder out;
-
-    public void setup() {
-        int size = (${type.arraySize} * (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE));
-        buffer = buffer.reallocIfNeeded(size);
-    }
-
-    public void eval() {
-
-        out.buffer = buffer;
-        out.start = 0;
-
-        // Re initialize the buffer everytime
-        for (int i = 0; i < ${type.arraySize}; i++) {
-            out.setInteger(i, 0, out.start, out.buffer);
-        }
-
-        out.scale = (int) scale.value;
-        out.precision = (int) precision.value;
-
-        /* Before converting from a sparse representation to a dense representation
-         * we need to convert it to an intermediate representation. In the sparse
-         * representation we separate out the scale and the integer part of the decimal
-         * and pad the scale part with additional zeroes for ease of performing arithmetic
-         * operations. In the intermediate representation we strip out the extra zeroes and
-         * combine the scale and integer part.
-         */
-        int[] intermediate = new int[in.nDecimalDigits - 1];
-
-        int index = in.nDecimalDigits - 1;
-        int actualDigits;
-
-        if (in.scale > 0 && (actualDigits = (in.scale % org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS)) > 0) {
-
-            int paddedDigits = org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS - actualDigits;
-
-            int paddedMask = (int) Math.pow(10, paddedDigits);
-
-            /* We have a scale that does not completely occupy a decimal
-             * digit, so we have padded zeroes to it for ease of arithmetic
-             * Truncate the extra zeroes added and move the digits to the right
-             */
-            int temp = (in.getInteger(index, in.start, in.buffer)/paddedMask);
-            index--;
-
-            while(index >= 0) {
-
-                int transferDigits = (in.getInteger(index, in.start, in.buffer) % (paddedMask));
-
-                intermediate[index] = (int) (temp + (Math.pow(10, actualDigits) * transferDigits));
-
-                temp = (in.getInteger(index, in.start, in.buffer)/(paddedMask));
-
-                index--;
-            }
-        } else {
-
-            /* If the scale does not exist or it perfectly fits within a decimal digit
-             * then we have padded no zeroes, which means there can atmost be only 38 digits, which
-             * need only 5 decimal digit to be stored, simply copy over the integers
-             */
-            for (int i = 1; i < in.nDecimalDigits; i++)
-                intermediate[i - 1] = in.getInteger(i, in.start, in.buffer);
-
-        }
-
-        /* Now we have an intermediate representation in the array intermediate[]
-         * Every number in the intermediate representation is base 1 billion number
-         * To represent it we require only 30 bits, but every integer has 32 bits.
-         * By shifting the bits around we can utilize the extra two bits on every
-         * number and create a dense representation
-         */
-
-          /* Allocate a byte array */
-          int size = (((intermediate.length - 1) * org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE) + 1);
-          byte[] intermediateBytes = new byte[size];
-          java.nio.ByteBuffer wrapper = java.nio.ByteBuffer.wrap(intermediateBytes);
-
-          wrapper.put((byte) intermediate[0]);
-
-          for (int i = 1; i < intermediate.length; i++) {
-            wrapper.put(java.nio.ByteBuffer.allocate(org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE).putInt(intermediate[i]).array());
-          }
-
-          final int[] mask = {0x03, 0x0F, 0x3F, 0xFF};
-          int maskIndex = 0;
-          int shiftOrder = 2;
-
-          // Start just after the last integer and shift bits to the right
-          index = size - (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE+ 1);
-
-          while (index >= 0) {
-
-              /* get the last bits that need to shifted to the next byte */
-              byte shiftBits = (byte) ((intermediateBytes[index] & mask[maskIndex]) << (8 - shiftOrder));
-
-              int shiftOrder1 = ((index % org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE) == 0) ? shiftOrder - 2 : shiftOrder;
-
-              /* transfer the bits from the left to the right */
-              intermediateBytes[index + 1] = (byte) (((intermediateBytes[index + 1] & 0xFF) >>> (shiftOrder1)) | shiftBits);
-
-              index--;
-
-              if ((index % org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE) == 0) {
-                  /* We are on a border */
-                  shiftOrder += 2;
-                  maskIndex++;
-              }
-          }
-
-          <#if (type.from == "Decimal28Sparse") && (type.to == "Decimal38Dense")>
-          /* Decimal38Dense representation has four bytes more than that needed to
-           * represent Decimal28Dense. So our first four bytes are empty in that scenario
-           */
-          int dstIndex = 4;
-          <#else>
-          int dstIndex = 0;
-          </#if>
-
-          // Set the bytes in the buffer
-          out.buffer.setBytes(dstIndex, intermediateBytes, 1, (size - 1));
-          out.setSign(in.getSign(in.start, in.buffer), out.start, out.buffer);
-    }
-}
-</#if>
-</#list>
\ No newline at end of file
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSparseVarDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarDecimal.java
similarity index 57%
rename from exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSparseVarDecimal.java
rename to exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarDecimal.java
index ae068c3..aeffe5a 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSparseVarDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarDecimal.java
@@ -18,7 +18,7 @@
 <@pp.dropOutputFile />
 
 <#list cast.types as type>
-<#if type.major == "DecimalSparseVarDecimal">
+<#if type.major == "DecimalToVarDecimal">
 
 <@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
 
@@ -45,34 +45,45 @@ import java.nio.ByteBuffer;
 /*
  * This class is generated using freemarker and the ${.template_name} template.
  */
-
 @SuppressWarnings("unused")
 @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{
+                  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;
-    @Inject DrillBuf buffer;
-    @Param BigIntHolder precision;
-    @Param BigIntHolder scale;
-    @Output ${type.to}Holder out;
+  @Param ${type.from}Holder in;
+  @Inject DrillBuf buffer;
+  @Param IntHolder precision;
+  @Param IntHolder scale;
+  @Output ${type.to}Holder out;
 
-    public void setup() {
-        // VarDecimal size in bytes is determined once the input BigDecimal is known
-    }
+  public void setup() {
+  }
 
-    public void eval() {
-        java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer, in.start, in.nDecimalDigits, in.scale, true);
-        out.scale = (int) scale.value;
-        int len = bd.unscaledValue().toByteArray().length;
-        buffer = buffer.reallocIfNeeded(len);
-        out.start = 0;
-        out.buffer = buffer;
-        org.apache.drill.exec.util.DecimalUtility.getVarDecimalFromBigDecimal(bd, out.buffer, out.start);
-        out.end = out.start + len;
-    }
+  public void eval() {
+    java.math.BigDecimal bd =
+        <#if type.from == "Decimal9" || type.from == "Decimal18">
+        java.math.BigDecimal.valueOf(in.value)
+        <#else>
+        org.apache.drill.exec.util.DecimalUtility
+          <#if type.from.contains("Sparse")>
+            .getBigDecimalFromDrillBuf(in.buffer, in.start, in.nDecimalDigits, in.scale, true)
+          <#elseif type.from == "VarDecimal">
+            .getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale)
+          </#if>
+        </#if>
+                .setScale(scale.value, java.math.RoundingMode.HALF_UP)
+                .round(new java.math.MathContext(precision.value, java.math.RoundingMode.HALF_UP));
+    out.scale = scale.value;
+    out.precision = precision.value;
+    out.start = 0;
+    byte[] bytes = bd.unscaledValue().toByteArray();
+    int len = bytes.length;
+    out.buffer = buffer.reallocIfNeeded(len);
+    out.buffer.setBytes(out.start, bytes);
+    out.end = out.start + len;
+  }
 }
 </#if>
 </#list>
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 c4ac6d7..369ba02 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java
@@ -20,7 +20,7 @@
 
 <#list cast.types as type>
 
-<#if type.major == "DecimalSimpleVarChar"> <#-- Cast function template for conversion from Decimal9, Decimal18 to VarChar -->
+<#if type.major == "DecimalComplexVarChar"> <#-- Cast function template for conversion from VarDecimal to VarChar -->
 
 <@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
 
@@ -41,8 +41,6 @@ import org.apache.drill.exec.util.DecimalUtility;
 import org.apache.drill.exec.expr.annotations.Workspace;
 
 import io.netty.buffer.ByteBuf;
-import io.netty.buffer.DrillBuf;
-import io.netty.buffer.SwappedByteBuf;
 
 import java.nio.ByteBuffer;
 
@@ -56,188 +54,23 @@ import java.nio.ByteBuffer;
     returnType = FunctionTemplate.ReturnType.STRING_CAST,
     nulls = NullHandling.NULL_IF_NULL)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc {
-
-    @Param ${type.from}Holder in;
-    @Param BigIntHolder len;
-    @Inject DrillBuf buffer;
-    @Output ${type.to}Holder out;
-
-    public void setup() {
-        buffer = buffer.reallocIfNeeded(${type.bufferSize});
-    }
-
-    public void eval() {
-
-        StringBuilder str = new StringBuilder();
-
-        ${type.javatype} value = in.value;
-        if (value < 0) {
-            // Negative value, add '-' to the string
-            str.append("-");
-
-            // Negate the number
-            value *= -1;
-        }
-
-        ${type.javatype} separator = (${type.javatype}) org.apache.drill.exec.util.DecimalUtility.getPowerOfTen((int) in.scale);
-
-        str.append(value / separator);
-
-        if (in.scale > 0) {
-            str.append(".");
-
-            String fractionalPart = String.valueOf(value % separator);
-
-            /* Since we are taking modulus to find fractional part,
-             * we will miss printing the leading zeroes in the fractional part
-             * Account for those zeroes
-             *
-             * Eg: 1.0002
-             * Scale: 3
-             *
-             * Stored as: 10002
-             *
-             * We print integer part by 10002/1000 = 1
-             * We print fractional part by 10002 % 1000 = 2
-             *
-             * We missed the initial zeroes in the fractional part. Below logic accounts for this case
-             */
-            str.append(org.apache.drill.exec.util.DecimalUtility.toStringWithZeroes((value % separator), in.scale));
-        }
-
-        out.buffer = buffer;
-        out.start = 0;
-        out.end = Math.min((int)len.value, str.length()); // truncate if target type has length smaller than that of input's string
-        out.buffer.setBytes(0, String.valueOf(str.substring(0,out.end)).getBytes());
-    }
-}
-<#elseif type.major == "DecimalComplexVarChar"> <#-- Cast function template for conversion from Decimal28Sparse, Decimal38Sparse to VarChar -->
-
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl.gcast;
-
-<#include "/@includes/vv_imports.ftl" />
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.util.DecimalUtility;
-import org.apache.drill.exec.expr.annotations.Workspace;
-
-import io.netty.buffer.ByteBuf;
-
-import java.nio.ByteBuffer;
-
-/*
- * This class is generated using freemarker and the ${.template_name} template.
- */
-
-@SuppressWarnings("unused")
-@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;
-    @Param BigIntHolder len;
-    @Inject DrillBuf buffer;
-    @Output ${type.to}Holder out;
-
-    public void setup() {
-        buffer = buffer.reallocIfNeeded((int) len.value);
-    }
-
-    public void eval() {
-
-<#if type.from.contains("VarDecimal")>
-        java.math.BigDecimal bigDecimal = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
-        String str = bigDecimal.toString();
-        out.buffer = buffer;
-        out.start = 0;
-        out.end = Math.min((int)len.value, str.length());
-        out.buffer.setBytes(0, String.valueOf(str.substring(0, out.end)).getBytes());
-<#else>
-        StringBuilder str = new StringBuilder();
-        int index = 0;
-        int fractionalStartIndex = ${type.arraySize} - org.apache.drill.exec.util.DecimalUtility.roundUp(in.scale);
-
-        // Find the first non-zero value in the integer part of the decimal
-        while (index < fractionalStartIndex && in.getInteger(index, in.start, in.buffer) == 0)  {
-            index++;
-        }
-
-
-        // If we have valid digits print '-' sign
-        if ((in.getSign(in.start, in.buffer) == true) && index < ${type.arraySize}) {
-            str.append("-");
-        }
-
-        // If all the integer digits are zero, print a single zero
-        if (index == fractionalStartIndex) {
-            str.append("0");
-        }
-
-        boolean fillZeroes = false;
-
-        // convert the integer part
-        while (index < fractionalStartIndex) {
-            int value =  in.getInteger(index++, in.start, in.buffer);
-
-            if (fillZeroes == true) {
-                str.append(org.apache.drill.exec.util.DecimalUtility.toStringWithZeroes(value, org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS));
-            } else {
-                str.append(value);
-                fillZeroes = true;
-            }
-            //str.append(value);
-        }
-
-        if (fractionalStartIndex < ${type.arraySize}) {
-            // We have fractional part, print '.'
-            str.append(".");
-
-            /* convert the fractional part (except the last decimal digit,
-             * as it might have padding that needs to be stripped
-             */
-            while (fractionalStartIndex < ${type.arraySize} - 1) {
-                int value = in.getInteger(fractionalStartIndex++, in.start, in.buffer);
-
-                // Fill zeroes at the beginning of the decimal digit
-                str.append(org.apache.drill.exec.util.DecimalUtility.toStringWithZeroes(value, org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS));
-            }
-
-            // Last decimal digit, strip the extra zeroes we may have padded
-            int actualDigits = in.scale % org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS;
-
-            int lastFractionalDigit = in.getInteger(${type.arraySize} - 1, in.start, in.buffer);
-
-            if (actualDigits != 0) {
-
-                // Strip padded zeroes at the end that is not part of the scale
-                lastFractionalDigit /= (int) (org.apache.drill.exec.util.DecimalUtility.getPowerOfTen((int) (org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS - actualDigits)));
-                str.append(org.apache.drill.exec.util.DecimalUtility.toStringWithZeroes(lastFractionalDigit, actualDigits));
-            } else {
-                // Last digit does not have any padding print as is
-                str.append(org.apache.drill.exec.util.DecimalUtility.toStringWithZeroes(lastFractionalDigit, org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS));
-            }
-
-
-        }
-
-        out.buffer = buffer;
-        out.start = 0;
-        out.end = Math.min((int)len.value, str.length()); // truncate if target type has length smaller than that of input's string
-        out.buffer.setBytes(0, String.valueOf(str.substring(0,out.end)).getBytes());
-</#if>
-    }
+  @Param ${type.from}Holder in;
+  @Param BigIntHolder len;
+  @Inject DrillBuf buffer;
+  @Output ${type.to}Holder out;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    java.math.BigDecimal bigDecimal = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
+    String str = bigDecimal.toString();
+    out.buffer = buffer;
+    out.start = 0;
+    out.end = Math.min((int) len.value, str.length());
+    buffer = buffer.reallocIfNeeded((int) out.end);
+    out.buffer.setBytes(0, str.substring(0, out.end).getBytes());
+  }
 }
 </#if> <#-- type.major -->
 </#list>
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDownwardDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDownwardDecimal.java
deleted file mode 100644
index a1ed7c8..0000000
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDownwardDecimal.java
+++ /dev/null
@@ -1,209 +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 />
-
-<#list cast.types as type>
-
-<#if type.major == "DownwardDecimalSimpleDecimalSimple">  <#-- Cast function template for conversion from Decimal18, Decimal9 -->
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl.gcast;
-
-<#include "/@includes/vv_imports.ftl" />
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.util.DecimalUtility;
-import org.apache.drill.exec.expr.annotations.Workspace;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.DrillBuf;
-
-import java.nio.ByteBuffer;
-
-/*
- * This class is generated using freemarker and the ${.template_name} template.
- */
-
-@SuppressWarnings("unused")
-@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;
-    @Param BigIntHolder precision;
-    @Param BigIntHolder scale;
-    @Output ${type.to}Holder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      out.scale = (int) scale.value;
-      out.precision = (int) precision.value;
-      out.value = (int) in.value;
-      // Check if we need to truncate or round up
-      if (out.scale > in.scale) {
-        out.value *= (int) org.apache.drill.exec.util.DecimalUtility.getPowerOfTen(out.scale - in.scale);
-      } else if (out.scale < in.scale) {
-        // need to round up since we are truncating fractional part
-        int scaleFactor = (int) (org.apache.drill.exec.util.DecimalUtility.getPowerOfTen((int) in.scale));
-        int newScaleFactor = (int) (org.apache.drill.exec.util.DecimalUtility.getPowerOfTen((int) scale.value));
-        int truncScaleFactor = (int) (org.apache.drill.exec.util.DecimalUtility.getPowerOfTen( (int) (Math.abs(in.scale - scale.value))));
-        int truncFactor = (int) (in.scale - scale.value);
-
-        // Assign the integer part
-        out.value = (int) (in.value / scaleFactor);
-
-        // Get the fractional part
-        int fractionalPart = (int) (in.value % scaleFactor);
-
-        // From the entire fractional part extract the digits upto which rounding is needed
-        int newFractionalPart = (int) (org.apache.drill.exec.util.DecimalUtility.adjustScaleDivide(fractionalPart, truncFactor));
-        int truncatedFraction = fractionalPart % truncScaleFactor;
-
-        // Get the truncated fractional part and extract the first digit to see if we need to add 1
-        int digit = Math.abs((int) org.apache.drill.exec.util.DecimalUtility.adjustScaleDivide(truncatedFraction, truncFactor - 1));
-
-        if (digit > 4) {
-          if (in.value > 0) {
-            newFractionalPart++;
-          } else if (in.value < 0) {
-            newFractionalPart--;
-          }
-        }
-        out.value = (int) ((out.value * newScaleFactor) + newFractionalPart);
-      }
-    }
-}
-<#elseif type.major == "DownwardDecimalComplexDecimalSimple">  <#-- Cast function template for conversion from Decimal28/Decimal9 to Decimal18/Decimal9 -->
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl.gcast;
-
-<#include "/@includes/vv_imports.ftl" />
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.util.DecimalUtility;
-import org.apache.drill.exec.expr.annotations.Workspace;
-
-import io.netty.buffer.ByteBuf;
-
-import java.nio.ByteBuffer;
-
-/*
- * This class is generated using freemarker and the ${.template_name} template.
- */
-
-@SuppressWarnings("unused")
-@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;
-    @Param BigIntHolder precision;
-    @Param BigIntHolder scale;
-    @Output ${type.to}Holder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      java.math.BigDecimal temp = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer, in.start, in.nDecimalDigits, in.scale);
-      temp = temp.setScale((int) scale.value, java.math.BigDecimal.ROUND_HALF_UP);
-      out.value = temp.unscaledValue().${type.javatype}Value();
-      out.precision = (int) precision.value;
-      out.scale = (int) scale.value;
-    }
-}
-<#elseif type.major == "DownwardDecimalComplexDecimalComplex">  <#-- Cast function template for conversion from Decimal28/Decimal9 to Decimal18/Decimal9 -->
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl.gcast;
-
-<#include "/@includes/vv_imports.ftl" />
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.util.DecimalUtility;
-import org.apache.drill.exec.expr.annotations.Workspace;
-
-import io.netty.buffer.ByteBuf;
-
-import java.nio.ByteBuffer;
-
-/*
- * This class is generated using freemarker and the ${.template_name} template.
- */
-
-@SuppressWarnings("unused")
-@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;
-    @Param BigIntHolder precision;
-    @Param BigIntHolder scale;
-    @Inject DrillBuf buffer;
-    @Output ${type.to}Holder out;
-
-    public void setup() {
-      int size = (${type.arraySize} * (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE));
-      buffer = buffer.reallocIfNeeded(size);
-    }
-
-    public void eval() {
-      java.math.BigDecimal temp = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer, in.start, in.nDecimalDigits, in.scale);
-      temp = temp.setScale((int) scale.value, java.math.BigDecimal.ROUND_HALF_UP);
-      out.precision = (int) precision.value;
-      out.scale = (int) scale.value;
-      out.buffer = buffer;
-      out.start = 0;
-      org.apache.drill.exec.util.DecimalUtility.getSparseFromBigDecimal(temp, out.buffer, out.start, out.scale, out.precision, out.nDecimalDigits);
-    }
-}
-</#if> <#-- type.major -->
-</#list>
\ No newline at end of file
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 ba8f651..f051310 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastFloatDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastFloatDecimal.java
@@ -20,8 +20,8 @@
 
 <#list cast.types as type>
 
-<#-- Cast function template for conversion from Float to Decimal9, Decimal18, Decimal28, Decimal38 -->
-<#if type.major == "FloatDecimalComplex" || type.major == "DoubleDecimalComplex" || type.major == "FloatDecimalSimple" || type.major == "DoubleDecimalSimple">
+<#-- Cast function template for conversion from Float to VarDecimal -->
+<#if type.major == "FloatDecimalComplex" || type.major == "DoubleDecimalComplex">
 
 <@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
 
@@ -52,59 +52,38 @@ import java.nio.ByteBuffer;
 
 @SuppressWarnings("unused")
 @FunctionTemplate(name = "cast${type.to?upper_case}",
-    scope = FunctionTemplate.FunctionScope.SIMPLE,
-    returnType = FunctionTemplate.ReturnType.DECIMAL_CAST,
-    nulls = NullHandling.NULL_IF_NULL)
+                  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;
-<#if type.major == "FloatDecimalComplex" || type.major == "DoubleDecimalComplex">
-@Inject DrillBuf buffer;
-</#if>
-@Param BigIntHolder precision;
-@Param BigIntHolder scale;
-@Output ${type.to}Holder out;
-
-    public void setup() {
-        <#if type.major == "FloatDecimalComplex" || type.major == "DoubleDecimalComplex">
-        int size = ${type.arraySize} * (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE);
-        buffer = buffer.reallocIfNeeded(size);
-        </#if>
-    }
-
-    public void eval() {
-
-        out.scale = (int) scale.value;
-
-        <#if !type.to.endsWith("VarDecimal")>
-        out.precision = (int) precision.value;
-        </#if>
-
-        <#if type.to.endsWith("VarDecimal")>
-        // TODO: the entire set of logic, above, appears to be using BigDecimal constructors completely wrong!!!!
-        // for the time being, this new logic is added for VarDecimal only.
-        double d = in.value;
-        for (int i = 0; i < out.scale; ++i) {  // loop to compute unscaled value
-            d *= 10.0;
-        }
-        long lval = (long)(d >= 0.0 ? d + 0.5 : d - 0.5);  // round off unscaled integer, from float8 to nearest integer
-        java.math.BigInteger bi = new java.math.BigInteger(Long.toString(lval));
-        java.math.BigDecimal bd = new java.math.BigDecimal(bi, out.scale);
-        int len = org.apache.drill.exec.util.DecimalUtility.getVarDecimalFromBigDecimal(bd, out.buffer, out.start);
-        out.end = out.start + len;
-        <#elseif type.major == "FloatDecimalComplex" || type.major == "DoubleDecimalComplex">
-        out.start = 0;
-        out.buffer = buffer;
-
-       // Assign the integer part of the decimal to the output holder
-        org.apache.drill.exec.util.DecimalUtility.getSparseFromBigDecimal(new java.math.BigDecimal(String.valueOf(in.value)), out.buffer, out.start, out.scale, out.precision, out.nDecimalDigits);
-
-        <#elseif type.to.endsWith("Decimal9")>
-        out.value = org.apache.drill.exec.util.DecimalUtility.getDecimal9FromBigDecimal(new java.math.BigDecimal(String.valueOf(in.value)), out.scale, out.precision);
-        <#elseif type.to.endsWith("Decimal18")>
-        out.value = org.apache.drill.exec.util.DecimalUtility.getDecimal18FromBigDecimal(new java.math.BigDecimal(String.valueOf(in.value)), out.scale, out.precision);
-        </#if>
-    }
+  @Param ${type.from}Holder in;
+  @Inject DrillBuf buffer;
+  @Param IntHolder precision;
+  @Param IntHolder scale;
+  @Output ${type.to}Holder out;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    out.scale = scale.value;
+    out.precision = precision.value;
+
+    out.start = 0;
+    java.math.BigDecimal bd =
+        new java.math.BigDecimal(
+            String.valueOf(in.value),
+            new java.math.MathContext(
+                precision.value,
+                java.math.RoundingMode.HALF_UP))
+        .setScale(scale.value, java.math.RoundingMode.HALF_UP);
+
+    byte[] bytes = bd.unscaledValue().toByteArray();
+    int len = bytes.length;
+    out.buffer = buffer.reallocIfNeeded(len);
+    out.buffer.setBytes(out.start, bytes);
+    out.end = out.start + len;
+  }
 }
 </#if>
 </#list>
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 15304da..162c562 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
@@ -48,79 +48,35 @@ import java.nio.ByteBuffer;
 
 @SuppressWarnings("unused")
 @FunctionTemplate(name = "cast${type.to?upper_case}",
-    scope = FunctionTemplate.FunctionScope.SIMPLE,
-    returnType = FunctionTemplate.ReturnType.DECIMAL_CAST,
-    nulls = NullHandling.NULL_IF_NULL)
+                  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;
-    <#if type.to.startsWith("Decimal28") || type.to.startsWith("Decimal38") || type.to.endsWith("VarDecimal")>
-    @Inject DrillBuf buffer;
-    </#if>
-    @Param BigIntHolder precision;
-    @Param BigIntHolder scale;
-    @Output ${type.to}Holder out;
-
-    public void setup() {
-        <#if type.to.startsWith("Decimal28") || type.to.startsWith("Decimal38")>
-        int size = ${type.arraySize} * (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE);
-        buffer = buffer.reallocIfNeeded(size);
-        </#if>
-
-    }
-
-    public void eval() {
-        out.scale = (int) scale.value;
-
-        <#if !type.to.endsWith("VarDecimal")>
-        out.precision = (int) precision.value;
-        </#if>
-
-        <#if type.to.endsWith("VarDecimal")>
-        out.start = 0;
-        out.buffer = buffer;
-        StringBuilder sb = new StringBuilder();
-        sb.append(Long.toString((long)in.value));
-        for (int i = 0; i < out.scale; ++i) {  // add 0's to get unscaled integer
-            sb.append("0");
-        }
-        java.math.BigInteger bi = new java.math.BigInteger(sb.toString());
-        java.math.BigDecimal bd = new java.math.BigDecimal(bi, out.scale);
-        int len = org.apache.drill.exec.util.DecimalUtility.getVarDecimalFromBigDecimal(bd, out.buffer, out.start);
-        out.end = out.start + len;
-        <#elseif type.to == "Decimal9" || type.to == "Decimal18">
-        out.value = (${type.javatype}) in.value;
-
-        // converting from integer to decimal, pad zeroes if scale is non zero
-        out.value = (${type.javatype}) org.apache.drill.exec.util.DecimalUtility.adjustScaleMultiply(out.value, (int) scale.value);
-
-        <#else>
-
-        out.start = 0;
-        out.buffer = buffer;
-
-        // Initialize the buffer
-        for (int i = 0; i < ${type.arraySize}; i++) {
-            out.setInteger(i, 0, out.start, out.buffer);
-        }
-
-        // check if input is a negative number and store the sign
-        if (in.value < 0) {
-            out.setSign(true, out.start, out.buffer);
-        }
-
-        // Figure out how many array positions to be left for the scale part
-        int scaleSize = org.apache.drill.exec.util.DecimalUtility.roundUp((int) scale.value);
-        int integerIndex = (${type.arraySize} - scaleSize - 1);
-
-        long inValue = in.value;
-        while (inValue != 0 && integerIndex >= 0) {
-            out.setInteger(integerIndex--, (int) Math.abs((inValue % org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE)), out.start, out.buffer);
-            inValue = inValue / org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE;
-        }
-
-        </#if>
-    }
+  @Param ${type.from}Holder in;
+  @Inject DrillBuf buffer;
+  @Param IntHolder precision;
+  @Param IntHolder scale;
+  @Output ${type.to}Holder out;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    out.scale = scale.value;
+    out.precision = precision.value;
+
+    out.start = 0;
+    out.buffer = buffer;
+    java.math.BigDecimal bd = new java.math.BigDecimal(in.value,
+        new java.math.MathContext(precision.value, java.math.RoundingMode.HALF_UP))
+        .setScale(out.scale, java.math.BigDecimal.ROUND_DOWN);
+
+    byte[] bytes = bd.unscaledValue().toByteArray();
+    int len = bytes.length;
+    out.buffer = out.buffer.reallocIfNeeded(len);
+    out.buffer.setBytes(out.start, bytes);
+    out.end = out.start + len;
+  }
 }
 </#if> <#-- type.major -->
 </#list>
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java
deleted file mode 100644
index 541b40a..0000000
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java
+++ /dev/null
@@ -1,294 +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 />
-
-<#list cast.types as type>
-
-<#-- Template code for converting from Decimal9, Decimal18 to Decimal28Dense and Decimal38Dense -->
-<#if type.major == "DecimalSimpleDecimalDense">
-
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl.gcast;
-
-<#include "/@includes/vv_imports.ftl" />
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.expr.annotations.Workspace;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.DrillBuf;
-
-import java.nio.ByteBuffer;
-
-/*
- * This class is generated using freemarker and the ${.template_name} template.
- */
-
-@SuppressWarnings("unused")
-@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;
-    <#if type.to.startsWith("Decimal28") || type.to.startsWith("Decimal38")>
-    @Inject DrillBuf buffer;
-    </#if>
-    @Param BigIntHolder precision;
-    @Param BigIntHolder scale;
-    @Output ${type.to}Holder out;
-
-    public void setup() {
-        int size = (${type.arraySize} * (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE));
-        buffer = buffer.reallocIfNeeded(size);
-    }
-
-    public void eval() {
-
-        out.buffer = buffer;
-        out.start = 0;
-
-        // Re initialize the buffer everytime
-        for (int i = 0; i < ${type.arraySize}; i++) {
-            out.setInteger(i, 0, out.start, out.buffer);
-        }
-
-        out.scale = (int) scale.value;
-        out.precision = (int) precision.value;
-
-        out.buffer = buffer;
-        out.start = 0;
-        out.setSign((in.value < 0), out.start, out.buffer);
-
-        /* Since we will be dividing the decimal value with base 1 billion
-         * we don't want negative results if the decimal is negative.
-         */
-        long value = (in.value < 0) ? (in.value * -1) : in.value;
-
-        int index = out.nDecimalDigits - 1;
-
-        // store the decimal value as sequence of integers of base 1 billion.
-        while (value > 0) {
-
-            out.setInteger(index, (int) (value % org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE), out.start, out.buffer);
-            value = value/org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE;
-            index--;
-        }
-
-        /* We have stored the decimal value in the intermediate format, which is basically that the
-         * scale and integer part of the decimal together, with no additional zeroes padded to the
-         * scale. Now we simply need to shift the bits around to get a more compact representation
-         */
-        int[] mask = {0x03, 0x0F, 0x3F, 0xFF};
-        int maskIndex = 0;
-        int shiftOrder = 2;
-
-        // Start shifting bits just after the first integer
-        int byteIndex = in.WIDTH - (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE + 1);
-
-        while (byteIndex >= 0) {
-
-            /* get the last bits that need to shifted to the next byte */
-            byte shiftBits = (byte) ((out.buffer.getByte(byteIndex) & mask[maskIndex]) << (8 - shiftOrder));
-
-            int shiftOrder1 = ((byteIndex % 4) == 0) ? shiftOrder - 2 : shiftOrder;
-
-            /* transfer the bits from the left to the right */
-            out.buffer.setByte(byteIndex + 1,  (byte) (((out.buffer.getByte(byteIndex + 1) & 0xFF) >>> (shiftOrder1)) | shiftBits));
-
-            byteIndex--;
-
-            if (byteIndex % 4 == 0) {
-                /* We are on a border */
-                shiftOrder += 2;
-                maskIndex++;
-            }
-        }
-    }
-}
-
-<#-- Template code for converting from Decimal9, Decimal18 to Decimal28Sparse and Decimal38Sparse -->
-<#elseif type.major == "DecimalSimpleDecimalSparse">
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl.gcast;
-
-<#include "/@includes/vv_imports.ftl" />
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.expr.annotations.Workspace;
-
-import io.netty.buffer.ByteBuf;
-
-import java.nio.ByteBuffer;
-
-/*
- * This class is generated using freemarker and the ${.template_name} template.
- */
-
-@SuppressWarnings("unused")
-@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;
-    @Inject DrillBuf buffer;
-    @Param BigIntHolder precision;
-    @Param BigIntHolder scale;
-    @Output ${type.to}Holder out;
-
-    public void setup() {
-        int size = (${type.arraySize} * (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE));
-        buffer = buffer.reallocIfNeeded(size);
-    }
-
-    public void eval() {
-        out.buffer = buffer;
-        out.start = 0;
-
-        // Re initialize the buffer everytime
-        for (int i = 0; i < ${type.arraySize}; i++) {
-            out.setInteger(i, 0, out.start, out.buffer);
-        }
-        out.scale = (int) scale.value;
-        out.precision = (int) precision.value;
-
-        out.buffer = buffer;
-        out.start = 0;
-
-        /* Since we will be dividing the decimal value with base 1 billion
-         * we don't want negative results if the decimal is negative.
-         */
-        long value = (in.value < 0) ? (in.value * -1) : in.value;
-
-        int index = out.nDecimalDigits - 1;
-
-        // Separate out the scale part and store it
-        int remainingScale = in.scale;
-
-        while(remainingScale > 0) {
-
-            int power = (remainingScale % org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS);
-            int padding = 1;
-
-            if (power == 0) {
-                power = 9;
-            } else {
-                padding = (int) (org.apache.drill.exec.util.DecimalUtility.getPowerOfTen((int) (org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS - power)));
-            }
-
-            int mask = (int) org.apache.drill.exec.util.DecimalUtility.getPowerOfTen(power);
-
-            out.setInteger(index, (int) ((value % mask) * padding), out.start, out.buffer);
-
-            value = value/mask;
-
-            remainingScale -= power;
-
-            index--;
-        }
-
-        while (value > 0) {
-            out.setInteger(index, (int) (value % org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE), out.start, out.buffer);
-            value = value/org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE;
-            index--;
-        }
-
-        // Round up or down the scale
-        if (in.scale != out.scale) {
-          org.apache.drill.exec.util.DecimalUtility.roundDecimal(out.buffer, out.start, out.nDecimalDigits, out.scale, in.scale);
-        }
-        // Set the sign
-        out.setSign((in.value < 0), out.start, out.buffer);
-    }
-}
-
-<#-- Template code for converting from Decimal9 to Decimal18 -->
-<#elseif type.major == "DecimalSimpleDecimalSimple">
-
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl.gcast;
-
-<#include "/@includes/vv_imports.ftl" />
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.expr.annotations.Workspace;
-
-import io.netty.buffer.ByteBuf;
-
-import java.nio.ByteBuffer;
-
-/*
- * This class is generated using freemarker and the ${.template_name} template.
- */
-
-@SuppressWarnings("unused")
-@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;
-    @Param BigIntHolder precision;
-    @Param BigIntHolder scale;
-    @Output ${type.to}Holder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-        out.scale = (int) scale.value;
-        out.precision = (int) precision.value;
-        out.value = in.value;
-
-        // Truncate or pad additional zeroes if the output scale is different from input scale
-        out.value = (${type.javatype}) (org.apache.drill.exec.util.DecimalUtility.adjustScaleMultiply(out.value, (int) (out.scale - in.scale)));
-    }
-}
-</#if>
-</#list>
\ No newline at end of file
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 851d424..a6e209d 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
@@ -19,211 +19,7 @@
 
 <#list cast.types as type>
 
-<#if type.major == "VarCharDecimalSimple" || type.major == "EmptyStringVarCharDecimalSimple">  <#-- Cast function template for conversion from VarChar to Decimal9, Decimal18 -->
-
-<#if type.major == "VarCharDecimalSimple">
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java"/>
-<#elseif type.major == "EmptyStringVarCharDecimalSimple">
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/CastEmptyString${type.from}To${type.to}.java"/>
-</#if>
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl.gcast;
-
-<#include "/@includes/vv_imports.ftl" />
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.util.DecimalUtility;
-import org.apache.drill.exec.expr.annotations.Workspace;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.DrillBuf;
-
-import java.nio.ByteBuffer;
-
-/*
- * This class is generated using freemarker and the ${.template_name} template.
- */
-
-@SuppressWarnings("unused")
-<#if type.major == "VarCharDecimalSimple">
-@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.SIMPLE,
-    returnType = FunctionTemplate.ReturnType.DECIMAL_CAST,
-    nulls = NullHandling.INTERNAL)
-public class CastEmptyString${type.from}To${type.to} implements DrillSimpleFunc {
-</#if>
-    @Param ${type.from}Holder in;
-    @Param BigIntHolder precision;
-    @Param BigIntHolder scale;
-
-    <#if type.major == "VarCharDecimalSimple">
-    @Output ${type.to}Holder out;
-    <#elseif type.major == "EmptyStringVarCharDecimalSimple">
-    @Output ${type.to}Holder out;
-    </#if>
-
-    public void setup() {
-    }
-
-    public void eval() {
-        <#if type.major == "EmptyStringVarCharDecimalSimple">
-        // Check if the input is null or empty string
-        if(<#if type.from == "NullableVarChar"> in.isSet == 0 || </#if> in.end == in.start) {
-            out.isSet = 0;
-            return;
-        }
-        out.isSet = 1;
-        </#if>
-
-        // Assign the scale and precision
-        out.scale = (int) scale.value;
-
-        <#if type.to.endsWith("VarDecimal")>
-
-        // VarDecimal gets its own cast logic
-        byte[] buf = new byte[in.end - in.start];
-        buffer.getBytes(in.start, buf, 0, in.end - in.start);
-        String s = new String(buf, Charsets.UTF_8);
-        Double d = Double.valueOf(s);
-        for (int i = 0; i < out.scale; ++i) {  // loop to compute unscaled value
-            d *= 10.0;
-        }
-        long lval = (long)(d >= 0.0 ? d + 0.5 : d - 0.5);  // round off unscaled integer, and hope it's close enough
-        java.math.BigInteger bi = new java.math.BigInteger(Long.toString(lval));
-        java.math.BigDecimal bd = new java.math.BigDecimal(bi, out.scale);
-        int len = org.apache.drill.exec.util.DecimalUtility.getVarDecimalFromBigDecimal(bd, out.buffer, out.start);
-        out.end = out.start + len;
-
-        <#else>   // !VarDecimal
-
-        out.precision = (int) precision.value;
-
-        int readIndex = in.start;
-        int endIndex  = in.end;
-
-        <#if type.major == "VarCharDecimalSimple">
-        // Check if its an empty string
-        if (endIndex - readIndex == 0) {
-            throw new org.apache.drill.common.exceptions.DrillRuntimeException("Empty String, cannot cast to Decimal");
-        }
-        </#if>
-
-        // Starting position of fractional part
-        int scaleIndex = -1;
-        // true if we have a negative sign at the beginning
-        boolean negative = false;
-
-        // Check the first byte for '-'
-        byte next = (in.buffer.getByte(readIndex));
-
-        // If its a negative number
-        if (next == '-') {
-            negative = true;
-            readIndex++;
-        }
-
-
-        /* Below two fields are used to compute if the precision is sufficient to store
-         * the scale along with the integer digits in the string
-         */
-        int integerStartIndex = readIndex;
-        int integerEndIndex = endIndex;
-        boolean leadingDigitFound = false;
-        boolean round = false;
-
-        int radix = 10;
-
-        // Start parsing the digits
-        while (readIndex < endIndex) {
-            next = in.buffer.getByte(readIndex++);
-
-            if (next == '.') {
-                scaleIndex = readIndex;
-                // Integer end index is just before the scale part begins
-                integerEndIndex = scaleIndex - 1;
-                // If the number of fractional digits is > scale specified we might have to truncate
-                if ((scaleIndex + out.scale) < endIndex ) {
-                    endIndex = scaleIndex + out.scale;
-                    round    = true;
-                }
-                continue;
-            } else {
-                // If its not a '.' we expect only numbers
-                next = (byte) Character.digit(next, radix);
-            }
-
-            if (next == -1) {
-                // not a valid digit
-                byte[] buf = new byte[in.end - in.start];
-                in.buffer.getBytes(in.start, buf, 0, in.end - in.start);
-                throw new org.apache.drill.common.exceptions.DrillRuntimeException(new String(buf, com.google.common.base.Charsets.UTF_8));
-            } else if (leadingDigitFound == false) {
-                if (next == 0) {
-                    // Ignore the leading zeroes while validating if input digits will fit within the given precision
-                    integerStartIndex++;
-                } else {
-                    leadingDigitFound = true;
-                }
-            }
-            out.value *= radix;
-            out.value += next;
-        }
-
-        // Check if the provided precision is enough to store the given input
-        if (((integerEndIndex - integerStartIndex) + out.scale) > out.precision) {
-            byte[] buf = new byte[in.end - in.start];
-            in.buffer.getBytes(in.start, buf, 0, in.end - in.start);
-            throw new org.apache.drill.common.exceptions.DrillRuntimeException("Precision is insufficient for the provided input: " + new String(buf, com.google.common.base.Charsets.UTF_8) + " Precision: " + out.precision +
-                                                                               " Total Digits: " + (out.scale + (integerEndIndex - integerStartIndex)));
-            // TODO:  Use JDK's java.nio.charset.StandardCharsets.UTF_8.
-        }
-
-        // Check if we need to round up
-        if (round == true) {
-            next = in.buffer.getByte(endIndex);
-            next = (byte) Character.digit(next, radix);
-            if (next == -1) {
-                // not a valid digit
-                byte[] buf = new byte[in.end - in.start];
-                in.buffer.getBytes(in.start, buf, 0, in.end - in.start);
-                throw new org.apache.drill.common.exceptions.DrillRuntimeException(new String(buf, com.google.common.base.Charsets.UTF_8));
-            }
-            if (next > 4) {
-                out.value++;
-            }
-        }
-
-        // Number of fractional digits in the input
-        int fractionalDigits = (scaleIndex == -1) ? 0 : ((endIndex - scaleIndex));
-
-        // Pad the number with zeroes if number of fractional digits is less than scale
-        if (fractionalDigits < scale.value) {
-            out.value = (${type.javatype}) (org.apache.drill.exec.util.DecimalUtility.adjustScaleMultiply(out.value, (int) (scale.value - fractionalDigits)));
-        }
-
-        // Negate the number if we saw a -ve sign
-        if (negative == true) {
-            out.value *= -1;
-        }
-        </#if>
-    }
-}
-
-<#elseif type.major == "VarCharDecimalComplex" || type.major == "EmptyStringVarCharDecimalComplex">  <#-- Cast function template for conversion from VarChar to Decimal28, Decimal38 -->
+<#if type.major == "VarCharDecimalComplex" || type.major == "EmptyStringVarCharDecimalComplex">  <#-- Cast function template for conversion from VarChar to VarDecimal -->
 
 <#if type.major == "VarCharDecimalComplex">
 <@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java"/>
@@ -258,276 +54,57 @@ import java.nio.ByteBuffer;
 @SuppressWarnings("unused")
 <#if type.major == "VarCharDecimalComplex">
 @FunctionTemplate(name = "cast${type.to?upper_case}",
-    scope = FunctionTemplate.FunctionScope.SIMPLE,
-    returnType = FunctionTemplate.ReturnType.DECIMAL_CAST,
-    nulls = NullHandling.NULL_IF_NULL)
+                  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.SIMPLE,
-    returnType = FunctionTemplate.ReturnType.DECIMAL_CAST,
-    nulls = NullHandling.INTERNAL)
+                  scope = FunctionTemplate.FunctionScope.SIMPLE,
+                  returnType = FunctionTemplate.ReturnType.DECIMAL_CAST,
+                  nulls = NullHandling.INTERNAL)
 public class CastEmptyString${type.from}To${type.to} implements DrillSimpleFunc {
 </#if>
-    @Param ${type.from}Holder in;
-    @Inject DrillBuf buffer;
-    @Param BigIntHolder precision;
-    @Param BigIntHolder scale;
-
-    <#if type.major == "VarCharDecimalComplex">
-    @Output ${type.to}Holder out;
-    <#elseif type.major == "EmptyStringVarCharDecimalComplex">
-    @Output ${type.to}Holder out;
-    </#if>
-
-    public void setup() {
-        int size = ${type.arraySize} * (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE);
-        buffer = buffer.reallocIfNeeded(size);
+  @Param ${type.from}Holder in;
+  @Inject DrillBuf buffer;
+  @Param IntHolder precision;
+  @Param IntHolder scale;
+
+  @Output ${type.to}Holder out;
+
+  public void setup() {
+  }
+
+  public void eval() {
+    <#if type.major == "EmptyStringVarCharDecimalComplex">
+    // Check if the input is null or empty string
+    if (<#if type.from == "NullableVarChar"> in.isSet == 0 || </#if> in.end == in.start) {
+      out.isSet = 0;
+      return;
     }
+    out.isSet = 1;
+    </#if>
 
-    public void eval() {
-        <#if type.major == "EmptyStringVarCharDecimalComplex">
-        // Check if the input is null or empty string
-        if(<#if type.from == "NullableVarChar"> in.isSet == 0 || </#if> in.end == in.start) {
-            out.isSet = 0;
-            return;
-        }
-        out.isSet = 1;
-        </#if>
-
-        out.buffer = buffer;
-        out.start  = 0;
-
-        out.scale = (int) scale.value;
-
-        <#if type.to.endsWith("VarDecimal")>
-
-        // VarDecimal gets its own cast logic
-        byte[] buf = new byte[in.end - in.start];
-        buffer.getBytes(in.start, buf, 0, in.end - in.start);
-        String s = new String(buf, Charsets.UTF_8);
-        Double d = Double.valueOf(s);
-        for (int i = 0; i < out.scale; ++i) {  // loop to compute unscaled value
-            d *= 10.0;
-        }
-        long lval = (long)(d >= 0.0 ? d + 0.5 : d - 0.5);  // round off unscaled integer, and hope it's close enough
-        java.math.BigInteger bi = new java.math.BigInteger(Long.toString(lval));
-        java.math.BigDecimal bd = new java.math.BigDecimal(bi, out.scale);
-        int len = org.apache.drill.exec.util.DecimalUtility.getVarDecimalFromBigDecimal(bd, out.buffer, out.start);
-        out.end = out.start + len;
-
-        <#else>
-
-        out.precision = (int) precision.value;
-        boolean sign = false;
-
-        // Initialize the output buffer
-        for (int i = 0; i < ${type.arraySize}; i++) {
-            out.setInteger(i, 0, out.start, out.buffer);
-        }
-
-        int startIndex;
-        int readIndex = in.start;
-        int integerDigits = 0;
-        int fractionalDigits = 0;
-        int scaleIndex = -1;
-        int scaleEndIndex = in.end;
-
-        byte[] buf1 = new byte[in.end - in.start];
-        in.buffer.getBytes(in.start, buf1, 0, in.end - in.start);
-
-        Byte next = in.buffer.getByte(readIndex);
-
-
-        if (next == '-') {
-            readIndex++;
-            sign = true;
-        }
-
-        if (next == '.') {
-            readIndex++;
-            scaleIndex = readIndex; // Fractional part starts at the first position
-        }
-
-        <#if type.major == "VarCharDecimalComplex">
-        // Check if its an empty string
-        if (in.end - readIndex == 0) {
-            throw new org.apache.drill.common.exceptions.DrillRuntimeException("Empty String, cannot cast to Decimal");
-        }
-        </#if>
-
-        // Store start index for the second pass
-        startIndex = readIndex;
-
-        int radix = 10;
-        boolean leadingDigitFound = false;
-        boolean round = false;
-
-        /* This is the first pass, we get the number of integer digits and based on the provided scale
-         * we compute which index into the ByteBuf we start storing the integer part of the Decimal
-         */
-        if (scaleIndex == -1) {
-
-            while (readIndex < in.end) {
-                next = in.buffer.getByte(readIndex++);
-
-                if (next == '.') {
-
-                    // We have found the decimal point. we can compute the starting index into the Decimal's bytebuf
-                    scaleIndex = readIndex;
-                    // We may have to truncate fractional part if > scale
-                    if ((in.end - scaleIndex) > out.scale) {
-                      scaleEndIndex =  scaleIndex + out.scale;
-                      round = true;
-                    }
-                    break;
-                }
-
-                // If its not a '.' we expect only numbers
-                next = (byte) Character.digit(next, radix);
-
-                if (next == -1) {
-                    // not a valid digit
-                    byte[] buf = new byte[in.end - in.start];
-                    in.buffer.getBytes(in.start, buf, 0, in.end - in.start);
-                    throw new NumberFormatException(new String(buf, com.google.common.base.Charsets.UTF_8));
-                }
-
-                if (leadingDigitFound == false && next != 0) {
-                    leadingDigitFound = true;
-                }
-
-                if (leadingDigitFound == true) {
-                    integerDigits++;
-                }
-            }
-        }
-
-        <#-- TODO:  Pull out much of this code into something parallel to
-             ByteFunctionHelpers but for DECIMAL type implementations. -->
-
-        /* Based on the number of integer digits computed and the scale throw an
-         * exception if the provided precision is not sufficient to store the value
-         */
-        if (integerDigits + out.scale > out.precision) {
-            byte[] buf = new byte[in.end - in.start];
-            in.buffer.getBytes(in.start, buf, 0, in.end - in.start);
-            throw new org.apache.drill.common.exceptions.DrillRuntimeException("Precision is insufficient for the provided input: " + new String(buf, com.google.common.base.Charsets.UTF_8) + " Precision: " + out.precision + " Total Digits: " + (out.scale + integerDigits));
-            <#-- TODO:  Revisit message.  (Message would be clearer and shorter
-                 as something like "Precision of X digits is insufficient for
-                 the provided input of "XXXXX.XXXXX" (X total digits)."  (An
-                 occurrence of "Precision is insufficient for the provided input:
-                 123456789.987654321 Precision: 5 Total Digits: 9" seemed to
-                 mean that 5 post-decimal digits and 9 total digits were allowed.)
-                 -->
-        }
-
-
-        // Compute the number of slots needed in the ByteBuf to store the integer and fractional part
-        int scaleRoundedUp   = org.apache.drill.exec.util.DecimalUtility.roundUp(out.scale);
-        int integerRoundedUp = org.apache.drill.exec.util.DecimalUtility.roundUp(integerDigits);
-
-        int ndigits = 0;
-
-        int decimalBufferIndex = ${type.arraySize} - scaleRoundedUp - 1;
-
-        /* Compute the end index of the integer part.
-         * If we haven't seen a '.' then entire string is integer.
-         * If we have seen a '.' it ends before the '.'
-         */
-        int integerEndIndex = (scaleIndex == -1) ? (in.end - 1) : (scaleIndex - 2);
-
-        // Traverse and extract the integer part
-        while (integerEndIndex >= startIndex) {
-            next = in.buffer.getByte(integerEndIndex--);
-
-            next = (byte) Character.digit(next, radix);
-
-            int value = (((int) org.apache.drill.exec.util.DecimalUtility.getPowerOfTen(ndigits)) * next) + (out.getInteger(decimalBufferIndex, out.start, out.buffer));
-            out.setInteger(decimalBufferIndex, value, out.start, out.buffer);
-
-            ndigits++;
-
-            /* We store the entire decimal as base 1 billion values, which has maximum of 9 digits (MAX_DIGITS)
-             * Once we have stored MAX_DIGITS in a given slot move to the next slot.
-             */
-            if (ndigits >= org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS) {
-                ndigits = 0;
-                decimalBufferIndex--;
-            }
-        }
-
-        // Traverse and extract the fractional part
-        decimalBufferIndex = (scaleRoundedUp > 0) ? (${type.arraySize} - scaleRoundedUp) : (${type.arraySize} - 1);
-        ndigits = 0;
-
-        if (scaleIndex != -1) {
-            while (scaleIndex < scaleEndIndex) {
-
-                // check if we have scanned MAX_DIGITS and we need to move to the next index
-                if (ndigits >= org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS) {
-                    ndigits = 0;
-                    decimalBufferIndex++;
-                }
-
-                next = in.buffer.getByte(scaleIndex++);
-
-                // We expect only numbers beyond this
-                next = (byte) Character.digit(next, radix);
-
-                if (next == -1) {
-                    // not a valid digit
-                    byte[] buf = new byte[in.end - in.start];
-                    in.buffer.getBytes(in.start, buf, 0, in.end - in.start);
-                    throw new NumberFormatException(new String(buf, com.google.common.base.Charsets.UTF_8));
-                }
-                int value = (out.getInteger(decimalBufferIndex, out.start, out.buffer) * radix) + next;
-                out.setInteger(decimalBufferIndex, value, out.start, out.buffer);
-
-                // added another digit to the current index
-                ndigits++;
-            }
-
-            // round up the decimal if we had to chop off a part of it
-            if (round == true) {
-               next = in.buffer.getByte(scaleEndIndex);
-
-                // We expect only numbers beyond this
-                next = (byte) Character.digit(next, radix);
-
-                if (next == -1) {
-                    // not a valid digit
-                    byte[] buf = new byte[in.end - in.start];
-                    in.buffer.getBytes(in.start, buf, 0, in.end - in.start);
-                    throw new NumberFormatException(new String(buf, com.google.common.base.Charsets.UTF_8));
-                }
-                if (next > 4) {
-                    // Need to round up
-                    out.setInteger(decimalBufferIndex, out.getInteger(decimalBufferIndex, out.start, out.buffer)+1, out.start, out.buffer);
-                }
-            }
-            // Pad zeroes in the fractional part so that number of digits = MAX_DIGITS
-            if (out.scale > 0) {
-              int padding = (int) org.apache.drill.exec.util.DecimalUtility.getPowerOfTen((int) (org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS - ndigits));
-              out.setInteger(decimalBufferIndex, out.getInteger(decimalBufferIndex, out.start, out.buffer) * padding, out.start, out.buffer);
-            }
-
-            int carry = 0;
-            do {
-                // propagate the carry
-                int tempValue = out.getInteger(decimalBufferIndex, out.start, out.buffer) + carry;
-                if (tempValue >= org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE) {
-                    carry = tempValue / org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE;
-                    tempValue = (tempValue % org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE);
-                } else {
-                    carry = 0;
-                }
-                out.setInteger(decimalBufferIndex--, tempValue, out.start, out.buffer);
-            } while (carry > 0 && decimalBufferIndex >= 0);
-        }
-        out.setSign(sign, out.start, out.buffer);
-        </#if>
-    }
+    out.start  = 0;
+
+    out.scale = scale.value;
+    out.precision = precision.value;
+
+    byte[] buf = new byte[in.end - in.start];
+    in.buffer.getBytes(in.start, buf, 0, in.end - in.start);
+    String s = new String(buf, com.google.common.base.Charsets.UTF_8);
+    java.math.BigDecimal bd =
+        new java.math.BigDecimal(s,
+            new java.math.MathContext(
+                precision.value,
+                java.math.RoundingMode.HALF_UP))
+        .setScale(scale.value, java.math.RoundingMode.HALF_UP);
+    byte[] bytes = bd.unscaledValue().toByteArray();
+    int len = bytes.length;
+    out.buffer = buffer.reallocIfNeeded(len);
+    out.buffer.setBytes(out.start, bytes);
+    out.end = out.start + len;
+  }
 }
 </#if> <#-- type.major -->
 </#list>
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSimilar.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastVarDecimalDecimal.java
similarity index 52%
rename from exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSimilar.java
rename to exec/java-exec/src/main/codegen/templates/Decimal/CastVarDecimalDecimal.java
index 8b97b1a..8ef2bf9 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSimilar.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastVarDecimalDecimal.java
@@ -17,10 +17,8 @@
  */
 <@pp.dropOutputFile />
 
-<#-- Template for converting between similar types of decimal. Decimal28Dense -> Decimal38Dense & Decimal28Sparse -> Decimal38Sparse -->
-
 <#list cast.types as type>
-<#if type.major == "DecimalSimilar">
+<#if type.major == "VarDecimalToDecimal">
 
 <@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
 
@@ -47,51 +45,48 @@ import java.nio.ByteBuffer;
 /*
  * This class is generated using freemarker and the ${.template_name} template.
  */
-
 @SuppressWarnings("unused")
 @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;
-    @Inject DrillBuf buffer;
-    @Param BigIntHolder precision;
-    @Param BigIntHolder scale;
-    @Output ${type.to}Holder out;
-
-    public void setup() {
-        int size = (${type.arraySize} * (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE));
-        buffer = buffer.reallocIfNeeded(size);
-    }
-
-    public void eval() {
-
-        out.buffer = buffer;
-        out.start = 0;
-        out.scale = (int) scale.value;
-        out.precision = (int) precision.value;
-        boolean sign = (in.getSign(in.start, in.buffer));
-
-        // Re initialize the buffer everytime
-        for (int i = 0; i < ${type.arraySize}; i++) {
-            out.setInteger(i, 0, out.start, out.buffer);
-        }
-
-        int inputIdx = in.nDecimalDigits - 1;
-        int outputIdx = out.nDecimalDigits - 1;
-
-        for (; inputIdx >= 0; inputIdx--, outputIdx--) {
-            out.setInteger(outputIdx, in.getInteger(inputIdx, in.start, in.buffer), out.start, out.buffer);
-        }
-
-        // round up or down the scale
-        if (in.scale != out.scale) {
-          org.apache.drill.exec.util.DecimalUtility.roundDecimal(out.buffer, out.start, out.nDecimalDigits, out.scale, in.scale);
-        }
-        out.setSign(sign, out.start, out.buffer);
-    }
+                  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;
+  <#if type.to.endsWith("Sparse") || type.to.endsWith("Dense")>
+  @Inject DrillBuf buffer;
+  </#if>
+  @Param IntHolder precision;
+  @Param IntHolder scale;
+  @Output ${type.to}Holder out;
+
+  public void setup() {
+  <#if type.to.endsWith("Sparse") || type.to.endsWith("Dense")>
+    int size = ${type.arraySize} * (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE);
+    buffer = buffer.reallocIfNeeded(size);
+  </#if>
+  }
+
+  public void eval() {
+    out.scale = scale.value;
+    out.precision = precision.value;
+    java.math.BigDecimal bd =
+        org.apache.drill.exec.util.DecimalUtility
+            .getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale)
+                .setScale(scale.value, java.math.RoundingMode.HALF_UP)
+                .round(new java.math.MathContext(precision.value, java.math.RoundingMode.HALF_UP));
+
+  <#if type.to.endsWith("Decimal9")>
+    out.value = org.apache.drill.exec.util.DecimalUtility.getDecimal9FromBigDecimal(bd, out.scale, out.precision);
+  <#elseif type.to.endsWith("Decimal18")>
+    out.value = org.apache.drill.exec.util.DecimalUtility.getDecimal18FromBigDecimal(bd, out.scale, out.precision);
+  <#elseif type.to.endsWith("Sparse")>
+    out.start = 0;
+    out.buffer = buffer;
+    org.apache.drill.exec.util.DecimalUtility
+        .getSparseFromBigDecimal(bd, out.buffer, out.start, out.scale, out.precision, out.nDecimalDigits);
+  </#if>
+  }
 }
-</#if> <#-- type.major -->
+</#if>
 </#list>
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 893438f..6b23f92 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java
@@ -17,15 +17,13 @@
  */
 <@pp.dropOutputFile />
 
-
-
 <#list decimalaggrtypes1.aggrtypes as aggrtype>
 <@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gaggr/Decimal${aggrtype.className}Functions.java" />
 
 <#include "/@includes/license.ftl" />
 
 <#-- A utility class that is used to generate java code for aggr functions for decimal data type that maintain a single -->
-<#-- running counter to hold the result.  This includes: MIN, MAX, COUNT. -->
+<#-- running counter to hold the result. This includes: MIN, MAX, SUM, $SUM0. -->
 
 /*
  * This class is automatically generated from AggrTypeFunctions1.tdd using FreeMarker.
@@ -54,204 +52,192 @@ import io.netty.buffer.ByteBuf;
 @SuppressWarnings("unused")
 
 public class Decimal${aggrtype.className}Functions {
-	static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(${aggrtype.className}Functions.class);
 
-<#list aggrtype.types as type>
+  <#list aggrtype.types as type>
+  <#if aggrtype.funcName.contains("sum")>
+  @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;
+    @Inject DrillBuf buffer;
+    @Workspace ObjectHolder value;
+    @Workspace IntHolder outputScale;
+    @Output ${type.outputType}Holder out;
+    @Workspace BigIntHolder nonNullCount;
+
+    public void setup() {
+      value = new ObjectHolder();
+      value.obj = java.math.BigDecimal.ZERO;
+      outputScale = new IntHolder();
+      outputScale.value = Integer.MIN_VALUE;
+      nonNullCount = new BigIntHolder();
+    }
 
-@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</#if>)
-public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc{
+    @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;
+      java.math.BigDecimal currentValue = org.apache.drill.exec.util.DecimalUtility
+          .getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
+      value.obj = ((java.math.BigDecimal) value.obj).add(currentValue);
+      if (outputScale.value == Integer.MIN_VALUE) {
+        outputScale.value = in.scale;
+      }
+      <#if type.inputType?starts_with("Nullable")>
+      } // end of sout block
+      </#if>
+    }
 
-  @Param ${type.inputType}Holder in;
-  <#if aggrtype.funcName == "sum">
-  @Inject DrillBuf buffer;
-  @Workspace ObjectHolder value;
-  @Workspace IntHolder outputScale;
-  <#elseif type.outputType.endsWith("Sparse")>
-  @Inject DrillBuf buffer;
-  @Workspace IntHolder scale;
-  @Workspace IntHolder precision;
-  @Workspace ObjectHolder value;
-  <#else>
-  @Workspace ${type.runningType}Holder value;
-  </#if>
-  @Output ${type.outputType}Holder out;
+    @Override
+    public void output() {
+      if (nonNullCount.value > 0) {
+        out.isSet = 1;
+        out.start  = 0;
+        out.scale = Math.min(outputScale.value,
+            org.apache.drill.exec.planner.types.DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM.getMaxNumericScale());
+        out.precision =
+            org.apache.drill.exec.planner.types.DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM.getMaxNumericPrecision();
+        value.obj = ((java.math.BigDecimal) value.obj).setScale(out.scale, java.math.BigDecimal.ROUND_HALF_UP);
+        byte[] bytes = ((java.math.BigDecimal) value.obj).unscaledValue().toByteArray();
+        int len = bytes.length;
+        out.buffer = buffer.reallocIfNeeded(len);
+        out.buffer.setBytes(0, bytes);
+        out.end = len;
+      } else {
+        out.isSet = 0;
+      }
+    }
 
-  public void setup() {
-	<#if aggrtype.funcName == "count">
-  	value = new ${type.runningType}Holder();
-    value.value = 0;
-	<#elseif aggrtype.funcName == "max" || aggrtype.funcName == "min">
-    <#if type.outputType.endsWith("Sparse")>
-    scale.value = 0;
-    precision.value = 0;
-    value = new ObjectHolder();
-    //${type.runningType}Holder tmp = new ${type.runningType}Holder();
-    byte[] byteArray = new byte[${type.runningType}Holder.WIDTH];
-    org.apache.drill.exec.util.Text tmp = new org.apache.drill.exec.util.Text(byteArray);
-    value.obj = tmp;
-    <#if aggrtype.funcName == "max">
-    for (int i = 0; i < ${type.runningType}Holder.nDecimalDigits; i++) {
-      org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.setInteger(tmp.getBytes(), i, 0xFFFFFFFF);
+    @Override
+    public void reset() {
+      value = new ObjectHolder();
+      value.obj = java.math.BigDecimal.ZERO;
+      outputScale = new IntHolder();
+      outputScale.value = Integer.MIN_VALUE;
+      nonNullCount.value = 0;
     }
-    org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.setSign(tmp.getBytes(), true);
-    <#elseif aggrtype.funcName == "min">
-    for (int i = 0; i < ${type.runningType}Holder.nDecimalDigits; i++) {
-      org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.setInteger(tmp.getBytes(), i, 0x7FFFFFFF);
+  }
+  <#elseif aggrtype.funcName == "max" || aggrtype.funcName == "min">
+
+  @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;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder scale;
+    @Workspace IntHolder precision;
+    @Workspace ObjectHolder tempResult;
+    @Output ${type.outputType}Holder out;
+    @Workspace BigIntHolder nonNullCount;
+    @Inject DrillBuf buf;
+
+    public void setup() {
+      tempResult = new ObjectHolder();
+      nonNullCount = new BigIntHolder();
     }
-    // Set sign to be positive so initial value is maximum
-    org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.setSign(tmp.getBytes(), false);
-    </#if>
-    <#elseif type.outputType == "Decimal9" || type.outputType == "Decimal18">
-    value = new ${type.runningType}Holder();
-    value.value = ${type.initValue};
-    </#if>
-  <#elseif aggrtype.funcName == "sum">
-    buffer = buffer.reallocIfNeeded(${type.outputType}Holder.WIDTH);
-    value = new ObjectHolder();
-    value.obj = java.math.BigDecimal.ZERO;
-    outputScale = new IntHolder();
-    outputScale.value = Integer.MIN_VALUE;
-	</#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;
+      org.apache.drill.exec.expr.fn.impl.DrillByteArray tmp = (org.apache.drill.exec.expr.fn.impl.DrillByteArray) tempResult.obj;
+      <#if aggrtype.funcName == "max">
+      int cmp = 0;
+      if (tmp != null) {
+        cmp = org.apache.drill.exec.util.DecimalUtility
+            .compareVarLenBytes(in.buffer, in.start, in.end, in.scale,
+                tmp.getBytes(), scale.value, false);
+      } else {
+        cmp = 1;
+        tmp = new org.apache.drill.exec.expr.fn.impl.DrillByteArray();
+        tempResult.obj = tmp;
+      }
 
-  @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 == "count">
-    value.value++;
-    <#elseif aggrtype.funcName == "max">
-    <#if type.outputType.endsWith("Sparse")>
-      //${type.runningType}Holder tmp = (${type.runningType}Holder) value.obj;
-      org.apache.drill.exec.util.Text tmp = (org.apache.drill.exec.util.Text) value.obj;
-      int cmp = org.apache.drill.exec.util.DecimalUtility.compareSparseSamePrecScale(in.buffer, in.start, tmp.getBytes(), tmp.getLength());
-    if (cmp == 1) {
-      //in.buffer.getBytes(in.start, tmp.getBytes(), 0, ${type.runningType}Holder.WIDTH);
-      for (int i = 0; i < ${type.runningType}Holder.nDecimalDigits; i++) {
-        org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.setInteger(tmp.getBytes(), i, ${type.runningType}Holder.getInteger(i, in.start, in.buffer));
+      if (cmp > 0) {
+        int inputLength = in.end - in.start;
+        if (tmp.getLength() >= inputLength) {
+          in.buffer.getBytes(in.start, tmp.getBytes(), 0, inputLength);
+          tmp.setLength(inputLength);
+        } else {
+          byte[] tempArray = new byte[in.end - in.start];
+          in.buffer.getBytes(in.start, tempArray, 0, in.end - in.start);
+          tmp.setBytes(tempArray);
+        }
+        scale.value = in.scale;
+        precision.value = in.precision;
       }
-      org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.setSign(tmp.getBytes(), in.getSign(in.start, in.buffer));
-      scale.value = in.scale;
-      precision.value = in.precision;
-    }
-    <#elseif type.outputType == "Decimal9" || type.outputType == "Decimal18">
-    value.value = Math.max(value.value, in.value);
-    </#if>
-    <#elseif aggrtype.funcName == "min">
-    <#if type.outputType.endsWith("Sparse")>
-    //${type.runningType}Holder tmp = (${type.runningType}Holder) value.obj;
-    org.apache.drill.exec.util.Text tmp = (org.apache.drill.exec.util.Text) value.obj;
-    int cmp = org.apache.drill.exec.util.DecimalUtility.compareSparseSamePrecScale(in.buffer, in.start, tmp.getBytes(), tmp.getLength());
-    if (cmp == -1) {
-      //in.buffer.getBytes(in.start, tmp.getBytes(), 0, ${type.runningType}Holder.WIDTH);
-      for (int i = 0; i < ${type.runningType}Holder.nDecimalDigits; i++) {
-        org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.setInteger(tmp.getBytes(), i, ${type.runningType}Holder.getInteger(i, in.start, in.buffer));
+      <#elseif aggrtype.funcName == "min">
+      int cmp = 0;
+      if (tmp != null) {
+        cmp = org.apache.drill.exec.util.DecimalUtility
+            .compareVarLenBytes(in.buffer, in.start, in.end, in.scale,
+                tmp.getBytes(), scale.value, false);
+      } else {
+        cmp = -1;
+        tmp = new org.apache.drill.exec.expr.fn.impl.DrillByteArray();
+        tempResult.obj = tmp;
       }
-      org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.setSign(tmp.getBytes(), in.getSign(in.start, in.buffer));
-      scale.value = in.scale;
-      precision.value = in.precision;
-    }
-    <#elseif type.outputType == "Decimal9" || type.outputType == "Decimal18">
-    value.value = Math.min(value.value, in.value);
-    </#if>
-    <#elseif aggrtype.funcName == "sum">
-   <#if type.inputType.endsWith("Decimal9") || type.inputType.endsWith("Decimal18")>
-    java.math.BigDecimal currentValue = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromPrimitiveTypes(in.value, in.scale, in.precision);
-    <#else>
-    java.math.BigDecimal currentValue = in.getBigDecimal();
-    </#if>
-    value.obj = ((java.math.BigDecimal)(value.obj)).add(currentValue);
-    if (outputScale.value == Integer.MIN_VALUE) {
-      outputScale.value = in.scale;
-    }
-    </#if>
-	<#if type.inputType?starts_with("Nullable")>
-    } // end of sout block
-	</#if>
-  }
 
-  @Override
-  public void output() {
-    <#if aggrtype.funcName == "count">
-    out.value = value.value;
-    <#elseif aggrtype.funcName == "sum">
-    out.buffer = buffer;
-    out.start  = 0;
-    out.scale = outputScale.value;
-    value.obj = ((java.math.BigDecimal) (value.obj)).setScale(out.scale, java.math.BigDecimal.ROUND_HALF_UP);
-<#if type.inputType.contains("VarDecimal")>
-    int len = org.apache.drill.exec.util.DecimalUtility.getVarDecimalFromBigDecimal((java.math.BigDecimal) value.obj, out.buffer, out.start);
-    out.end = out.start + len;
-<#else>
-    out.precision = 38;
-    org.apache.drill.exec.util.DecimalUtility.getSparseFromBigDecimal((java.math.BigDecimal) value.obj, out.buffer, out.start, out.scale, out.precision, out.nDecimalDigits);
-</#if>
-   <#else>
-    <#if type.outputType.endsWith("Sparse")>
-    org.apache.drill.exec.util.Text tmp = (org.apache.drill.exec.util.Text) value.obj;
-    buffer = buffer.reallocIfNeeded(tmp.getLength());
-    //buffer.setBytes(0, tmp.getBytes(), 0, tmp.getLength());
-    for (int i = 0; i < ${type.runningType}Holder.nDecimalDigits; i++) {
-      ${type.runningType}Holder.setInteger(i, org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.getInteger(tmp.getBytes(), i), 0, buffer);
+      if (cmp < 0) {
+        int inputLength = in.end - in.start;
+        if (tmp.getLength() >= inputLength) {
+          in.buffer.getBytes(in.start, tmp.getBytes(), 0, inputLength);
+          tmp.setLength(inputLength);
+        } else {
+        byte[] tempArray = new byte[in.end - in.start];
+          in.buffer.getBytes(in.start, tempArray, 0, in.end - in.start);
+          tmp.setBytes(tempArray);
+        }
+        scale.value = in.scale;
+        precision.value = in.precision;
+      }
+      </#if>
+      <#if type.inputType?starts_with("Nullable")>
+      } // end of sout block
+      </#if>
     }
-    out.buffer = buffer;
-    out.start = 0;
-    out.setSign(org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.getSign(tmp.getBytes()), out.start, out.buffer);
-    out.scale = scale.value;
-    out.precision = precision.value;
-    <#elseif type.outputType == "Decimal9" || type.outputType == "Decimal18">
-    out.value = value.value;
-    out.scale = value.scale;
-    out.precision = value.precision;
-    </#if>
-    </#if>
-  }
-
-  @Override
-  public void reset() {
 
-	<#if aggrtype.funcName == "count">
-	  value.value = 0;
-	<#elseif aggrtype.funcName == "max" || aggrtype.funcName == "min">
-    <#if type.outputType.endsWith("Sparse")>
-    org.apache.drill.exec.util.Text tmp = (org.apache.drill.exec.util.Text) value.obj;
-    for (int i = 0; i < ${type.runningType}Holder.nDecimalDigits; i++) {
-      org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.setInteger(tmp.getBytes(), i, 0xFFFFFFFF);
+    @Override
+    public void output() {
+      if (nonNullCount.value > 0) {
+        out.isSet = 1;
+        org.apache.drill.exec.expr.fn.impl.DrillByteArray tmp = (org.apache.drill.exec.expr.fn.impl.DrillByteArray) tempResult.obj;
+        buf = buf.reallocIfNeeded(tmp.getLength());
+        buf.setBytes(0, tmp.getBytes(), 0, tmp.getLength());
+        out.start = 0;
+        out.end = tmp.getLength();
+        out.buffer = buf;
+
+        out.scale = scale.value;
+        out.precision = precision.value;
+      } else {
+        out.isSet = 0;
+      }
     }
-    scale.value = 0;
-    precision.value = 0;
-    <#if aggrtype.funcName == "min">
-    // Set sign to be positive so initial value is maximum
-    org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.setSign(tmp.getBytes(), false);
-    <#elseif aggrtype.funcName == "max">
-    org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.setSign(tmp.getBytes(), true);
-    </#if>
-    <#elseif type.outputType == "Decimal9" || type.outputType == "Decimal18">
-    value = new ${type.runningType}Holder();
-    value.value = ${type.initValue};
-    </#if>
-  <#elseif aggrtype.funcName == "sum">
-    value = new ObjectHolder();
-    value.obj = java.math.BigDecimal.ZERO;
-    outputScale = new IntHolder();
-    outputScale.value = Integer.MIN_VALUE;
-	</#if>
 
+    @Override
+    public void reset() {
+      scale.value = 0;
+      precision.value = 0;
+      tempResult.obj = null;
+      nonNullCount.value = 0;
+    }
   }
-
- }
-
-
+  </#if>
 </#list>
 }
 </#list>
-
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 df1eb7e..c633b67 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions2.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions2.java
@@ -27,7 +27,7 @@ import org.apache.drill.exec.expr.annotations.Workspace;
 <#include "/@includes/license.ftl" />
 
 <#-- A utility class that is used to generate java code for aggr functions for decimal data type that maintain a single -->
-<#-- running counter to hold the result.  This includes: MIN, MAX, COUNT. -->
+<#-- running counter to hold the result. This includes: AVG. -->
 
 /*
  * This class is automatically generated from AggrTypeFunctions1.tdd using FreeMarker.
@@ -59,9 +59,9 @@ public class Decimal${aggrtype.className}Functions {
 <#list aggrtype.types as type>
 
 @FunctionTemplate(name = "${aggrtype.funcName}",
-    scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE,
-    returnType = FunctionTemplate.ReturnType.DECIMAL_SUM_AGGREGATE)
-public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc{
+                  scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE,
+                  returnType = FunctionTemplate.ReturnType.DECIMAL_AVG_AGGREGATE)
+public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc {
 
   @Param ${type.inputType}Holder in;
   @Inject DrillBuf buffer;
@@ -71,7 +71,6 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
   @Output ${type.outputType}Holder out;
 
   public void setup() {
-    buffer.reallocIfNeeded(${type.outputType}Holder.WIDTH);
     value = new ObjectHolder();
     value.obj = java.math.BigDecimal.ZERO;
     count = new ${type.countRunningType}Holder();
@@ -83,18 +82,15 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
   @Override
   public void add() {
 	  <#if type.inputType?starts_with("Nullable")>
-	    sout: {
+	  sout: {
 	    if (in.isSet == 0) {
 		    // processing nullable input and the value is null, so don't do anything...
 		    break sout;
 	    }
 	  </#if>
     count.value++;
-   <#if type.inputType.endsWith("Decimal9") || type.inputType.endsWith("Decimal18")>
-    java.math.BigDecimal currentValue = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromPrimitiveTypes(in.value, in.scale, in.precision);
-    <#else>
-    java.math.BigDecimal currentValue = in.getBigDecimal();
-    </#if>
+    java.math.BigDecimal currentValue = org.apache.drill.exec.util.DecimalUtility
+        .getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
     value.obj = ((java.math.BigDecimal)(value.obj)).add(currentValue);
     if (outputScale.value == Integer.MIN_VALUE) {
       outputScale.value = in.scale;
@@ -106,14 +102,21 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 
   @Override
   public void output() {
-    out.buffer = buffer;
-    out.start  = 0;
-    out.scale = outputScale.value;
-    java.math.BigDecimal average = ((java.math.BigDecimal)(value.obj)).divide(java.math.BigDecimal.valueOf(count.value, 0), out.scale, java.math.BigDecimal.ROUND_HALF_UP);
-<#if !type.inputType.contains("VarDecimal")>
-    out.precision = 38;
-    org.apache.drill.exec.util.DecimalUtility.getSparseFromBigDecimal(average, out.buffer, out.start, out.scale, out.precision, out.nDecimalDigits);
-</#if>
+    if (count.value > 0) {
+      out.isSet = 1;
+      out.start  = 0;
+      out.scale = Math.max(outputScale.value, 6);
+      java.math.BigDecimal average = ((java.math.BigDecimal) value.obj)
+            .divide(java.math.BigDecimal.valueOf(count.value), out.scale, java.math.BigDecimal.ROUND_HALF_UP);
+      out.precision = org.apache.drill.exec.planner.types.DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM.getMaxNumericPrecision();
+      byte[] bytes = average.unscaledValue().toByteArray();
+      int len = bytes.length;
+      out.buffer = buffer.reallocIfNeeded(len);
+      out.buffer.setBytes(0, bytes);
+      out.end = len;
+    } else {
+      out.isSet = 0;
+    }
   }
 
   @Override
@@ -127,7 +130,6 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
   }
 }
 
-
 </#list>
 }
 </#list>
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions3.java b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions3.java
new file mode 100644
index 0000000..3058381
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions3.java
@@ -0,0 +1,184 @@
+/*
+ * 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 />
+
+<#list decimalaggrtypes3.aggrtypes as aggrtype>
+<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gaggr/Decimal${aggrtype.className}Functions.java" />
+
+<#include "/@includes/license.ftl" />
+
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
+<#-- A utility class that is used to generate java code for aggr functions such as stddev, variance -->
+
+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.NullHandling;
+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;
+
+@SuppressWarnings("unused")
+
+public class Decimal${aggrtype.className}Functions {
+<#list aggrtype.types as type>
+
+  <#if aggrtype.aliasName == "">
+  @FunctionTemplate(name = "${aggrtype.funcName}",
+  <#else>
+  @FunctionTemplate(names = {"${aggrtype.funcName}", "${aggrtype.aliasName}"},
+  </#if>
+                    scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE,
+                    returnType = FunctionTemplate.ReturnType.DECIMAL_AVG_AGGREGATE)
+  public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc {
+    @Param ${type.inputType}Holder in;
+    @Inject DrillBuf buffer;
+    @Workspace ObjectHolder avg;
+    @Workspace ObjectHolder dev;
+    @Workspace BigIntHolder count;
+    @Workspace IntHolder scale;
+    @Output ${type.outputType}Holder out;
+    @Workspace BigIntHolder nonNullCount;
+
+    public void setup() {
+      avg = new ObjectHolder();
+      dev = new ObjectHolder();
+      count = new BigIntHolder();
+      scale = new IntHolder();
+      scale.value = Integer.MIN_VALUE;
+      // Initialize the workspace variables
+      avg.obj = java.math.BigDecimal.ZERO;
+      dev.obj = java.math.BigDecimal.ZERO;
+      count.value = 1;
+      nonNullCount = new BigIntHolder();
+    }
+
+    @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;
+
+      if (scale.value == Integer.MIN_VALUE) {
+        scale.value = Math.max(in.scale, 6);
+      }
+
+      // Welford's approach to compute standard deviation
+      // avg.value += ((in.value - temp) / count.value);
+      // dev.value += (in.value - temp) * (in.value - avg.value);
+      java.math.BigDecimal temp = (java.math.BigDecimal) avg.obj;
+      java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility
+          .getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
+      avg.obj = ((java.math.BigDecimal) avg.obj)
+          .add(input.subtract(temp)
+                  .divide(java.math.BigDecimal.valueOf(count.value),
+                      new java.math.MathContext(
+                          org.apache.drill.exec.planner.types.DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM.getMaxNumericPrecision(),
+                          java.math.RoundingMode.HALF_UP)));
+      dev.obj = ((java.math.BigDecimal) dev.obj)
+          .add(input.subtract(temp).multiply(input.subtract(((java.math.BigDecimal) avg.obj))));
+      count.value++;
+    <#if type.inputType?starts_with("Nullable")>
+      } // end of sout block
+    </#if>
+    }
+
+    @Override
+    public void output() {
+      if (nonNullCount.value > 0) {
+        out.isSet = 1;
+      <#if aggrtype.funcName == "stddev_pop">
+        if (count.value > 1) {
+          // out.value = Math.sqrt((dev.value / (count.value - 1)));
+          java.math.BigDecimal result =
+              org.apache.drill.exec.util.DecimalUtility.sqrt(((java.math.BigDecimal) dev.obj)
+                  .setScale(scale.value, java.math.RoundingMode.HALF_UP)
+                  .divide(java.math.BigDecimal.valueOf(count.value - 1),
+                      java.math.RoundingMode.HALF_UP),
+                  scale.value);
+      <#elseif aggrtype.funcName == "var_pop">
+        if (count.value  > 1) {
+          // out.value = (dev.value / (count.value - 1));
+          java.math.BigDecimal result =
+              ((java.math.BigDecimal) dev.obj)
+                  .setScale(scale.value, java.math.RoundingMode.HALF_UP)
+                  .divide(java.math.BigDecimal.valueOf(count.value - 1),
+                      java.math.RoundingMode.HALF_UP);
+      <#elseif aggrtype.funcName == "stddev_samp">
+        if (count.value  > 2) {
+          // out.value = Math.sqrt((dev.value / (count.value - 2)));
+          java.math.BigDecimal result =
+              org.apache.drill.exec.util.DecimalUtility.sqrt(((java.math.BigDecimal) dev.obj)
+                  .setScale(scale.value, java.math.RoundingMode.HALF_UP)
+                  .divide(java.math.BigDecimal.valueOf(count.value - 2),
+                      java.math.RoundingMode.HALF_UP),
+                  scale.value);
+      <#elseif aggrtype.funcName == "var_samp">
+        if (count.value > 2) {
+          // out.value = (dev.value / (count.value - 2));
+          java.math.BigDecimal result =
+              ((java.math.BigDecimal) dev.obj)
+                  .setScale(scale.value, java.math.RoundingMode.HALF_UP)
+                  .divide(java.math.BigDecimal.valueOf(count.value - 2),
+                      java.math.RoundingMode.HALF_UP);
+      </#if>
+          out.scale = scale.value;
+          result = result.setScale(out.scale, java.math.RoundingMode.HALF_UP);
+          out.start  = 0;
+          out.precision = org.apache.drill.exec.planner.types.DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM.getMaxNumericPrecision();
+          byte[] bytes = result.unscaledValue().toByteArray();
+          int len = bytes.length;
+          out.buffer = buffer.reallocIfNeeded(len);
+          out.buffer.setBytes(0, bytes);
+          out.end = len;
+        } else {
+          out.start = 0;
+          out.end = 0;
+          out.buffer = buffer;
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      avg.obj = java.math.BigDecimal.ZERO;
+      dev.obj = java.math.BigDecimal.ZERO;
+      count.value = 1;
+      nonNullCount.value = 0;
+    }
+  }
+
+</#list>
+}
+</#list>
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 53af71f..059080e 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java
@@ -86,25 +86,12 @@ import org.apache.drill.exec.expr.annotations.Workspace;
 </#macro>
 
 <#macro varCompareBlock leftType rightType absCompare output nullCompare nullComparesHigh>
-        outside:
-        {
-          <@compareNullsSubblock leftType=leftType rightType=rightType output=output breakTarget="outside" nullCompare=nullCompare nullComparesHigh=nullComparesHigh />
-
-          ${output} = org.apache.drill.exec.util.DecimalUtility.compareVarLenBytes(left.buffer, left.start, left.end, left.scale, right.buffer, right.start, right.end, right.scale, ${absCompare});
-        } // outside
-</#macro>
-
-<#macro adjustScale javaType leftType rightType>
-
-            // Adjust the scale of the two inputs to be the same
+      outside:
+      {
+        <@compareNullsSubblock leftType = leftType rightType=rightType output = output breakTarget = "outside" nullCompare = nullCompare nullComparesHigh=nullComparesHigh />
 
-            if (left.scale < right.scale) {
-                left.value = (${javaType}) (org.apache.drill.exec.util.DecimalUtility.adjustScaleMultiply(left.value, (int) (right.scale - left.scale)));
-                left.scale = right.scale;
-            } else if (right.scale < left.scale) {
-                right.value = (${javaType}) (org.apache.drill.exec.util.DecimalUtility.adjustScaleMultiply(right.value, (int) (left.scale - right.scale)));
-                right.scale = left.scale;
-            }
+        ${output} = org.apache.drill.exec.util.DecimalUtility.compareVarLenBytes(left.buffer, left.start, left.end, left.scale, right.buffer, right.start, right.end, right.scale, ${absCompare});
+      } // outside
 </#macro>
 
 <#-- For each DECIMAL... type (in DecimalTypes.tdd) ... -->
@@ -137,432 +124,215 @@ import java.nio.ByteBuffer;
 
 @SuppressWarnings("unused")
 public class ${type.name}Functions {
-    private static void initBuffer(DrillBuf buffer) {
-        // for VarDecimal, this method of setting initial size is actually only a very rough heuristic.
-        int size = (${type.storage} * (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE));
-        buffer = buffer.reallocIfNeeded(size);
-     }
-
-    @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;
-        @Param ${type.name}Holder right;
-        @Inject DrillBuf buffer;
-        @Workspace int outputScale;
-        @Output ${type.name}Holder result;
-
-        public void setup() {
-            initBuffer(buffer);
-        }
-
-        public void eval(){
-            outputScale=Math.max(left.scale,right.scale);   // output scale is the maximum of the two scales, to represent result exactly
-            result.scale=outputScale;
-            result.buffer=buffer;
-            result.start = 0;
-
-            java.math.BigDecimal leftInput=org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(left.buffer,left.start,left.end-left.start,left.scale);
-            java.math.BigDecimal rightInput=org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(right.buffer,right.start,right.end-right.start,right.scale);
-            java.math.BigDecimal addResult=leftInput.subtract(rightInput);
-
-            // Set the scale
-            addResult.setScale(result.scale,java.math.BigDecimal.ROUND_HALF_UP);
-
-            byte[]bytes=addResult.unscaledValue().toByteArray();
-            int len=bytes.length;
-            buffer=buffer.reallocIfNeeded(len);
-            buffer.setBytes(0,bytes);
-            result.end=len;
-        }
-    }
-
-    @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;
-        @Param ${type.name}Holder right;
-        @Workspace int outputScale;
-        @Inject DrillBuf buffer;
-        @Output ${type.name}Holder result;
-
-        public void setup() {
-            initBuffer(buffer);
-       }
-
-        public void eval() {
-            outputScale=Math.max(left.scale,right.scale);   // output scale is the maximum of the two scales, to represent result exactly
-            result.scale=outputScale;
-            result.buffer=buffer;
-            result.start = 0;
-
-            java.math.BigDecimal leftInput=org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(left.buffer,left.start,left.end-left.start,left.scale);
-            java.math.BigDecimal rightInput=org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(right.buffer,right.start,right.end-right.start,right.scale);
-            java.math.BigDecimal addResult=leftInput.add(rightInput);
-
-            // Set the scale
-            addResult.setScale(result.scale,java.math.BigDecimal.ROUND_HALF_UP);
-
-            byte[]bytes=addResult.unscaledValue().toByteArray();
-            int len=bytes.length;
-            buffer=buffer.reallocIfNeeded(len);
-            buffer.setBytes(0,bytes);
-            result.end=len;
-        }
-    }
-
-    @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;
-        @Param ${type.name}Holder right;
-        @Inject DrillBuf buffer;
-        @Workspace int outputScale;
-        @Output ${type.name}Holder result;
-
-        public void setup() {
-            initBuffer(buffer);
-        }
-
-        public void eval() {
-            outputScale=left.scale + right.scale;   // sum of scales is guaranteed to represent multiplication result exactly
-            result.scale=outputScale;
-            result.buffer=buffer;
-            result.start = 0;
-
-            java.math.BigDecimal leftInput=org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(left.buffer,left.start,left.end-left.start,left.scale);
-            java.math.BigDecimal rightInput=org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(right.buffer,right.start,right.end-right.start,right.scale);
-            java.math.BigDecimal output=leftInput.multiply(rightInput);
-
-            // Set the scale
-            output.setScale(result.scale,java.math.BigDecimal.ROUND_HALF_UP);
-
-            byte[]bytes=output.unscaledValue().toByteArray();
-            int len=bytes.length;
-            buffer=buffer.reallocIfNeeded(len);
-            buffer.setBytes(0,bytes);
-            result.end=len;
-        }
-    }
-
-    @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;
-        @Param ${type.name}Holder right;
-        @Output ${type.name}Holder result;
-        @Inject DrillBuf buffer;
-        @Workspace int outputScale;
-
-        public void setup() {
-            initBuffer(buffer);
-        }
-
-        public void eval() {
-            outputScale = left.scale + right.scale;   // preferred scale is heuristically set to sum of scales, even though rounding may be required
-            result.scale = outputScale;
-            result.buffer = buffer;
-            result.start = 0;
-
-            java.math.BigDecimal numerator = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(left.buffer, left.start, left.end - left.start, left.scale);
-            java.math.BigDecimal denominator = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(right.buffer, right.start, right.end - right.start, right.scale);
-
-            java.math.BigDecimal output = numerator.divide(denominator, (int) result.scale, java.math.BigDecimal.ROUND_HALF_UP);
-
-            byte[]bytes=output.unscaledValue().toByteArray();
-            int len=bytes.length;
-            buffer=buffer.reallocIfNeeded(len);
-            buffer.setBytes(0,bytes);
-            result.end=len;
-        }
-    }
 
-    @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;
-        @Param ${type.name}Holder right;
-        @Output ${type.name}Holder result;
-        @Inject DrillBuf buffer;
-        @Workspace int outputScale;
-
-        public void setup() {
-            initBuffer(buffer);
-        }
-
-        public void eval() {
-            outputScale = Math.max(left.scale,right.scale);   // for mod, as for add & subtract, use larger of the two input scales
-            result.scale = outputScale;
-            result.buffer = buffer;
-            result.start = 0;
-
-            java.math.BigDecimal numerator = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(left.buffer, left.start, left.end - left.start, left.scale);
-            java.math.BigDecimal denominator = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(right.buffer, right.start, right.end - right.start, right.scale);
-
-            java.math.BigDecimal output = numerator.remainder(denominator);
-            output.setScale(result.scale, java.math.BigDecimal.ROUND_HALF_UP);
-
-            byte[]bytes=output.unscaledValue().toByteArray();
-            int len=bytes.length;
-            buffer=buffer.reallocIfNeeded(len);
-            buffer.setBytes(0,bytes);
-            result.end=len;
-        }
+<#list ["Subtract", "Add", "Multiply", "Divide", "Mod"] as functionName>
+  @FunctionTemplate(name = "${functionName?lower_case}",
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                  <#if functionName == "Subtract" || functionName == "Add">
+                    returnType = FunctionTemplate.ReturnType.DECIMAL_ADD_SCALE,
+                  <#elseif functionName == "Multiply">
+                    returnType = FunctionTemplate.ReturnType.DECIMAL_SUM_SCALE,
+                  <#elseif functionName == "Divide">
+                    returnType = FunctionTemplate.ReturnType.DECIMAL_DIV_SCALE,
+                  <#elseif functionName == "Mod">
+                    returnType = FunctionTemplate.ReturnType.DECIMAL_MOD_SCALE,
+                  </#if>
+                    nulls = NullHandling.NULL_IF_NULL,
+                    checkPrecisionRange = true)
+  public static class ${type.name}${functionName}Function implements DrillSimpleFunc {
+    @Param ${type.name}Holder left;
+    @Param ${type.name}Holder right;
+    @Inject DrillBuf buffer;
+    @Output ${type.name}Holder result;
+
+    public void setup() {
     }
 
-    @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;
-        @Output ${type.name}Holder result;
-        @Inject DrillBuf buffer;
-
-        public void setup() {
-            initBuffer(buffer);
-        }
-
-        public void eval() {
-            result.scale = in.scale;
-            result.buffer = buffer;
-            result.start = 0;
-
-            java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
-            java.math.BigDecimal output = bd.abs();
-            output.setScale(result.scale, java.math.BigDecimal.ROUND_HALF_UP);
+    public void eval() {
+      result.start = 0;
+
+      java.math.BigDecimal leftInput =
+          org.apache.drill.exec.util.DecimalUtility
+              .getBigDecimalFromDrillBuf(left.buffer, left.start, left.end - left.start, left.scale);
+      java.math.BigDecimal rightInput =
+          org.apache.drill.exec.util.DecimalUtility
+              .getBigDecimalFromDrillBuf(right.buffer, right.start, right.end - right.start, right.scale);
+      org.apache.drill.exec.planner.types.decimal.DrillBaseComputeScalePrecision typeInference =
+      <#if functionName == "Subtract" || functionName == "Add">
+          new org.apache.drill.exec.planner.types.decimal.DecimalScalePrecisionAddFunction(
+      <#elseif functionName == "Multiply">
+          new org.apache.drill.exec.planner.types.decimal.DecimalScalePrecisionMulFunction(
+      <#elseif functionName == "Divide">
+          new org.apache.drill.exec.planner.types.decimal.DecimalScalePrecisionDivideFunction(
+      <#elseif functionName == "Mod">
+          new org.apache.drill.exec.planner.types.decimal.DecimalScalePrecisionModFunction(
+      </#if>
+              left.precision, left.scale,
+              right.precision, right.scale);
+
+      result.scale = typeInference.getOutputScale();
+      result.precision = typeInference.getOutputPrecision();
+
+      java.math.BigDecimal opResult =
+      <#if functionName == "Subtract" || functionName == "Add"
+          || functionName == "Multiply"|| functionName == "Divide">
+          leftInput.${functionName?lower_case}(rightInput,
+      <#elseif functionName == "Mod">
+        leftInput.remainder(rightInput,
+      </#if>
+              new java.math.MathContext(result.precision, java.math.RoundingMode.HALF_UP))
+            .setScale(result.scale, java.math.BigDecimal.ROUND_HALF_UP);
 
-            byte[]bytes=output.unscaledValue().toByteArray();
-            int len=bytes.length;
-            buffer=buffer.reallocIfNeeded(len);
-            buffer.setBytes(0,bytes);
-            result.end=len;
-        }
+      byte[] bytes = opResult.unscaledValue().toByteArray();
+      int len = bytes.length;
+      result.buffer = buffer.reallocIfNeeded(len);
+      result.buffer.setBytes(0, bytes);
+      result.end = len;
     }
+  }
 
-    @FunctionTemplate(name = "sign", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class ${type.name}SignFunction implements DrillSimpleFunc {
-
-        @Param ${type.name}Holder in;
-        @Output IntHolder out;
-
-        public void setup() {}
+</#list>
 
-        public void eval() {
-            java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
-            out.value = bd.signum();
-        }
+<#list ["Abs", "Ceil", "Floor", "Trunc", "Round"] as functionName>
+  <#if functionName == "Ceil">
+  @FunctionTemplate(names = {"ceil", "ceiling"},
+  <#elseif functionName == "Trunc">
+  @FunctionTemplate(names = {"trunc", "truncate"},
+  <#else>
+  @FunctionTemplate(name = "${functionName?lower_case}",
+  </#if>
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                  <#if functionName == "Abs">
+                    returnType = FunctionTemplate.ReturnType.DECIMAL_MAX_SCALE,
+                  <#elseif functionName == "Ceil" || functionName == "Floor"
+                      || functionName == "Trunc" || functionName == "Round">
+                    returnType = FunctionTemplate.ReturnType.DECIMAL_ZERO_SCALE,
+                  </#if>
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class ${type.name}${functionName}Function implements DrillSimpleFunc {
+    @Param  ${type.name}Holder in;
+    @Output ${type.name}Holder result;
+    @Inject DrillBuf buffer;
+
+    public void setup() {
     }
 
-    @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;
-        @Output ${type.name}Holder out;
-        @Inject DrillBuf buffer;
-
-        public void setup() {
-            initBuffer(buffer);
-        }
-
-        public void eval() {
-          out.scale = 0;
-          out.buffer = buffer;
-          out.start = 0;
-          java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
-          bd.setScale(0, java.math.BigDecimal.ROUND_CEILING);
-          byte[] bytes=bd.unscaledValue().toByteArray();
-          int len=bytes.length;
-          buffer=buffer.reallocIfNeeded(len);
-          buffer.setBytes(0,bytes);
-          out.end=len;
-        }
+    public void eval() {
+      result.start = 0;
+      result.precision = in.precision;
+
+      java.math.BigDecimal opResult =
+          org.apache.drill.exec.util.DecimalUtility
+              .getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale)
+          <#if functionName == "Abs">
+                  .abs();
+      result.scale = in.scale;
+          <#elseif functionName == "Ceil">
+                  .setScale(0, java.math.BigDecimal.ROUND_CEILING);
+          <#elseif functionName == "Floor">
+                  .setScale(0, java.math.BigDecimal.ROUND_FLOOR);
+          <#elseif functionName == "Trunc">
+                  .setScale(0, java.math.BigDecimal.ROUND_DOWN);
+          <#elseif functionName == "Round">
+                  .setScale(0, java.math.BigDecimal.ROUND_HALF_UP);
+          </#if>
+
+      byte[] bytes = opResult.unscaledValue().toByteArray();
+      int len = bytes.length;
+      result.buffer = buffer.reallocIfNeeded(len);
+      result.buffer.setBytes(0, bytes);
+      result.end = len;
     }
+  }
 
-    @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;
-        @Output ${type.name}Holder out;
-        @Inject DrillBuf buffer;
+</#list>
+  @FunctionTemplate(name = "sign",
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class ${type.name}SignFunction implements DrillSimpleFunc {
+    @Param ${type.name}Holder in;
+    @Output IntHolder out;
 
-        public void setup() {
-            initBuffer(buffer);
-        }
+    public void setup() {}
 
-        public void eval() {
-            out.scale = 0;
-            out.buffer = buffer;
-            out.start = 0;
-            java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
-            bd.setScale(0, java.math.BigDecimal.ROUND_FLOOR);
-            byte[] bytes=bd.unscaledValue().toByteArray();
-            int len=bytes.length;
-            buffer=buffer.reallocIfNeeded(len);
-            buffer.setBytes(0,bytes);
-            out.end=len;
-        }
+    public void eval() {
+      // TODO: optimize to get only bytes that corresponds to sign.
+      // Should be taken into account case when leading zero bytes are stored in buff.
+      java.math.BigDecimal bd =
+          org.apache.drill.exec.util.DecimalUtility
+              .getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
+      out.value = bd.signum();
     }
+  }
 
-    @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;
-        @Output ${type.name}Holder out;
-        @Inject DrillBuf buffer;
-
-        public void setup() {
-            initBuffer(buffer);
-        }
+  @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;
+    @Param  IntHolder right;
+    @Output ${type.name}Holder result;
+    @Inject DrillBuf buffer;
 
-        public void eval() {
-            out.scale = 0;
-            out.buffer = buffer;
-            out.start = 0;
-            java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
-            bd.setScale(0, java.math.BigDecimal.ROUND_DOWN);
-            byte[] bytes=bd.unscaledValue().toByteArray();
-            int len=bytes.length;
-            buffer=buffer.reallocIfNeeded(len);
-            buffer.setBytes(0,bytes);
-            out.end=len;
-        }
+    public void setup() {
     }
 
-    @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;
-        @Param  IntHolder right;
-        @Output ${type.name}Holder result;
-        @Inject DrillBuf buffer;
-
-        public void setup() {
-            initBuffer(buffer);
-        }
-
-        public void eval() {
-            result.scale = left.scale;
-            result.buffer = left.buffer;
-            result.start = left.start;
-            result.end = left.end;
-
-            // compute truncated value iff the indicated scale is less than the input's scale.
-            // otherwise, we just copy the input (left) to the result, with the same scale.
-            if (right.value < left.scale) {
-                result.buffer = buffer;
-                result.start = 0;
-                result.scale = right.value;
-                java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(left.buffer, left.start, left.end - left.start, left.scale);
-                bd.setScale(result.scale, java.math.BigDecimal.ROUND_DOWN);
-                byte[] bytes=bd.unscaledValue().toByteArray();
-                int len=bytes.length;
-                buffer=buffer.reallocIfNeeded(len);
-                buffer.setBytes(0,bytes);
-                result.end=len;
-            }
-        }
+    public void eval() {
+      result.start = 0;
+      result.scale = right.value;
+      result.precision = left.precision;
+      java.math.BigDecimal opResult =
+          org.apache.drill.exec.util.DecimalUtility
+              .getBigDecimalFromDrillBuf(left.buffer, left.start, left.end - left.start, left.scale)
+                  .setScale(result.scale, java.math.BigDecimal.ROUND_DOWN);
+      byte[] bytes = opResult.unscaledValue().toByteArray();
+      int len = bytes.length;
+      result.buffer = buffer.reallocIfNeeded(len);
+      result.buffer.setBytes(0, bytes);
+      result.end = len;
     }
+  }
 
-    @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;
-        @Output ${type.name}Holder out;
-        @Inject DrillBuf buffer;
-
-        public void setup() {
-            initBuffer(buffer);
-        }
+  @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;
+    @Param  IntHolder right;
+    @Output ${type.name}Holder result;
+    @Inject DrillBuf buffer;
 
-        public void eval() {
-            out.scale = 0;
-            out.buffer = buffer;
-            out.start = 0;
-            java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer, in.start, in.end - in.start, in.scale);
-            bd.setScale(0, java.math.BigDecimal.ROUND_HALF_UP);
-            byte[] bytes=bd.unscaledValue().toByteArray();
-            int len=bytes.length;
-            buffer=buffer.reallocIfNeeded(len);
-            buffer.setBytes(0,bytes);
-            out.end=len;
-        }
+    public void setup() {
     }
 
-    @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;
-        @Param  IntHolder right;
-        @Output ${type.name}Holder result;
-        @Inject DrillBuf buffer;
-
-        public void setup() {
-            initBuffer(buffer);
-        }
-
-        public void eval() {
-            result.scale = right.value;
-            result.buffer = buffer;
-            result.start = 0;
-            java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(left.buffer, left.start, left.end - left.start, left.scale);
-            bd.setScale(result.scale, java.math.BigDecimal.ROUND_HALF_UP);
-            byte[] bytes=bd.unscaledValue().toByteArray();
-            int len=bytes.length;
-            buffer=buffer.reallocIfNeeded(len);
-            buffer.setBytes(0,bytes);
-            result.end=len;
-        }
+    public void eval() {
+      result.scale = right.value;
+      result.precision = left.precision;
+      result.start = 0;
+      java.math.BigDecimal bd =
+          org.apache.drill.exec.util.DecimalUtility
+              .getBigDecimalFromDrillBuf(left.buffer, left.start, left.end - left.start, left.scale)
+                  .setScale(result.scale, java.math.BigDecimal.ROUND_HALF_UP);
+      byte[] bytes = bd.unscaledValue().toByteArray();
+      int len = bytes.length;
+      result.buffer = buffer.reallocIfNeeded(len);
+      result.buffer.setBytes(0, bytes);
+      result.end = len;
     }
+  }
 
- <#-- Handle 2 x 2 combinations of nullable and non-nullable arguments. -->
- <#list ["Nullable${type.name}", "${type.name}"] as leftType >
- <#list ["Nullable${type.name}", "${type.name}"] as rightType >
+  <#-- Handle 2 x 2 combinations of nullable and non-nullable arguments. -->
+  <#list ["Nullable${type.name}", "${type.name}"] as leftType >
+  <#list ["Nullable${type.name}", "${type.name}"] as rightType >
 
   <#-- Comparison function for sorting and grouping relational operators
        (not for comparison expression operators (=, <, etc.)). -->
   @FunctionTemplate(name = FunctionGenerationHelper.COMPARE_TO_NULLS_HIGH,
-      scope = FunctionTemplate.FunctionScope.SIMPLE,
-      returnType = FunctionTemplate.ReturnType.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;
@@ -572,7 +342,7 @@ public class ${type.name}Functions {
     public void setup() {}
 
     public void eval() {
-      <@varCompareBlock leftType=leftType rightType=rightType absCompare="false" output="out.value" nullCompare=true nullComparesHigh=true />
+      <@varCompareBlock leftType = leftType rightType = rightType absCompare="false" output = "out.value" nullCompare = true nullComparesHigh = true />
     }
   }
 
@@ -581,9 +351,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.SIMPLE,
-      returnType = FunctionTemplate.ReturnType.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;
@@ -593,1878 +363,132 @@ public class ${type.name}Functions {
     public void setup() {}
 
     public void eval() {
-      <@varCompareBlock leftType=leftType rightType=rightType absCompare="false" output="out.value" nullCompare=true nullComparesHigh=false />
+      <@varCompareBlock leftType = leftType rightType = rightType absCompare = "false" output = "out.value" nullCompare = true nullComparesHigh = false />
     }
   }
 
- </#list>
- </#list>
+  </#list>
+  </#list>
 
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
+  <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
          not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "less than",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp;
-            <@varCompareBlock leftType="leftType" rightType="rightType" absCompare="false" output="cmp" nullCompare=false nullComparesHigh=false />
-            out.value = cmp == -1 ? 1 : 0;
-        }
-    }
-
+  @FunctionTemplate(name = FunctionGenerationHelper.LT,
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class ${type.name}LessThan implements DrillSimpleFunc {
+
+    @Param ${type.name}Holder left;
+    @Param ${type.name}Holder right;
+    @Output BitHolder out;
+    public void setup() {}
 
-    // TODO:  RESOLVE:  Here there are spaces in function template names, but
-    // elsewhere there are underlines.  Are things being looked up correctly?
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-         not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "less than or equal to",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp;
-            <@varCompareBlock leftType="leftType" rightType="rightType" absCompare="false" output="cmp" nullCompare=false nullComparesHigh=false />
-            out.value = cmp < 1 ? 1 : 0;
-        }
+    public void eval() {
+      int cmp;
+      <@varCompareBlock leftType = "leftType" rightType = "rightType" absCompare = "false" output = "cmp" nullCompare = false nullComparesHigh = false />
+      out.value = cmp == -1 ? 1 : 0;
     }
+  }
 
-
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
+  <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
          not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "greater than",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp;
-            <@varCompareBlock leftType="leftType" rightType="rightType" absCompare="false" output="cmp" nullCompare=false nullComparesHigh=false />
-            out.value = cmp == 1 ? 1 : 0;
-        }
-    }
-
+  @FunctionTemplate(name = FunctionGenerationHelper.LE,
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class ${type.name}LessThanEq implements DrillSimpleFunc {
 
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-         not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "greater than or equal to",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp;
-            <@varCompareBlock leftType="leftType" rightType="rightType" absCompare="false" output="cmp" nullCompare=false nullComparesHigh=false />
-            out.value = cmp > -1 ? 1 : 0;
-        }
-    }
+    @Param ${type.name}Holder left;
+    @Param ${type.name}Holder right;
+    @Output BitHolder out;
 
+    public void setup() {}
 
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-         not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "Equal",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp;
-            <@varCompareBlock leftType="leftType" rightType="rightType" absCompare="false" output="cmp" nullCompare=false nullComparesHigh=false />
-            out.value = cmp == 0 ? 1 : 0;
-        }
+    public void eval() {
+      int cmp;
+      <@varCompareBlock leftType = "leftType" rightType = "rightType" absCompare = "false" output = "cmp" nullCompare = false nullComparesHigh = false />
+      out.value = cmp < 1 ? 1 : 0;
     }
+  }
 
-
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
+  <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
          not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "not equal",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp;
-            <@varCompareBlock leftType="leftType" rightType="rightType" absCompare="false" output="cmp" nullCompare=false nullComparesHigh=false />
-            out.value = cmp != 0 ? 1 : 0;
-        }
-    }
-}
-
-<#elseif type.name.endsWith("Sparse")>
-
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/${type.name}Functions.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl;
-
-<#include "/@includes/vv_imports.ftl" />
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
-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.fn.FunctionGenerationHelper;
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.DrillBuf;
-
-import java.nio.ByteBuffer;
-/*
- * This class is generated using freemarker and the ${.template_name} template.
- */
-
-@SuppressWarnings("unused")
-public class ${type.name}Functions {
-
-    @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;
-        @Param ${type.name}Holder right;
-        @Inject DrillBuf buffer;
-        @Workspace int outputScale;
-        @Workspace int outputPrecision;
-        @Output ${type.name}Holder result;
-
-        public void setup() {
-            int size = (${type.storage} * (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE));
-            buffer = buffer.reallocIfNeeded(size);
-            outputPrecision = Integer.MIN_VALUE;
-        }
-
-        public void eval() {
-            if (outputPrecision == Integer.MIN_VALUE) {
-                org.apache.drill.common.util.DecimalScalePrecisionAddFunction resultScalePrec =
-                new org.apache.drill.common.util.DecimalScalePrecisionAddFunction((int) left.precision, (int) left.scale, (int) right.precision, (int) right.scale);
-                outputScale = resultScalePrec.getOutputScale();
-                outputPrecision = resultScalePrec.getOutputPrecision();
-            }
-            result.precision = outputPrecision;
-            result.scale = outputScale;
-            result.buffer = buffer;
-            result.start = 0;
-
-            java.math.BigDecimal leftInput = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(left.buffer, left.start, left.nDecimalDigits, left.scale);
-            java.math.BigDecimal rightInput = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(right.buffer, right.start, right.nDecimalDigits, right.scale);
-            java.math.BigDecimal addResult = leftInput.subtract(rightInput);
-
-            // Set the scale
-            addResult.setScale(result.scale, java.math.BigDecimal.ROUND_HALF_UP);
-            org.apache.drill.exec.util.DecimalUtility.getSparseFromBigDecimal(addResult, result.buffer, result.start, result.scale, result.precision, result.nDecimalDigits);
-        }
-    }
-
-    @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;
-        @Param ${type.name}Holder right;
-        @Workspace int outputScale;
-        @Workspace int outputPrecision;
-        @Inject DrillBuf buffer;
-        @Output ${type.name}Holder result;
-
-        public void setup() {
-            int size = (${type.storage} * (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE));
-            buffer = buffer.reallocIfNeeded(size);
-            outputPrecision = Integer.MIN_VALUE;
-        }
-
-        public void eval() {
-            if (outputPrecision == Integer.MIN_VALUE) {
-                org.apache.drill.common.util.DecimalScalePrecisionAddFunction resultScalePrec =
-                new org.apache.drill.common.util.DecimalScalePrecisionAddFunction((int) left.precision, (int) left.scale, (int) right.precision, (int) right.scale);
-                outputScale = resultScalePrec.getOutputScale();
-                outputPrecision = resultScalePrec.getOutputPrecision();
-            }
-            result.precision = outputPrecision;
-            result.scale = outputScale;
-            result.buffer = buffer;
-            result.start = 0;
-
-            java.math.BigDecimal leftInput = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(left.buffer, left.start, left.nDecimalDigits, left.scale);
-            java.math.BigDecimal rightInput = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(right.buffer, right.start, right.nDecimalDigits, right.scale);
-            java.math.BigDecimal addResult = leftInput.add(rightInput);
-
-            // Set the scale
-            addResult.setScale(result.scale, java.math.BigDecimal.ROUND_HALF_UP);
-            org.apache.drill.exec.util.DecimalUtility.getSparseFromBigDecimal(addResult, result.buffer, result.start, result.scale, result.precision, result.nDecimalDigits);
-        }
-    }
-
-    @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;
-        @Param ${type.name}Holder right;
-        @Inject DrillBuf buffer;
-        @Workspace int[] tempResult;
-        @Workspace int outputScale;
-        @Workspace int outputPrecision;
-        @Output ${type.name}Holder result;
-
-        public void setup() {
-            int size = (${type.storage} * (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE));
-            buffer = buffer.reallocIfNeeded(size);
-            tempResult = new int[${type.storage} * ${type.storage}];
-            outputPrecision = Integer.MIN_VALUE;
-        }
-
-        public void eval() {
-            if (outputPrecision == Integer.MIN_VALUE) {
-                org.apache.drill.common.util.DecimalScalePrecisionMulFunction resultScalePrec =
-                new org.apache.drill.common.util.DecimalScalePrecisionMulFunction((int) left.precision, (int) left.scale, (int) right.precision, (int) right.scale);
-                outputScale = resultScalePrec.getOutputScale();
-                outputPrecision = resultScalePrec.getOutputPrecision();
-            }
-            // Set the scale and precision
-            result.scale = outputScale;
-            result.precision = outputPrecision;
-            result.buffer = buffer;
-            result.start = 0;
-
-            // Re initialize the temporary array
-            for (int i = 0; i < ${type.storage} * ${type.storage}; i++) {
-                tempResult[i] = 0;
-            }
-
-            // Remove the leading zeroes from the integer part of the input
-            int leftIndex = 0;
-            int leftStopIndex = left.nDecimalDigits - org.apache.drill.exec.util.DecimalUtility.roundUp(left.scale);
-
-            while (leftIndex < leftStopIndex) {
-                if (left.getInteger(leftIndex, left.start, left.buffer) > 0)
-                    break;
-                leftIndex++;
-            }
-
-            int leftIntegerSize = leftStopIndex - leftIndex;
-
-            /* Remove the leading zeroes from the integer part of the input */
-            int rightIndex = 0;
-            int rightStopIndex = right.nDecimalDigits - org.apache.drill.exec.util.DecimalUtility.roundUp(right.scale);
-
-            while(rightIndex < rightStopIndex) {
-                if (right.getInteger(rightIndex, right.start, right.buffer) > 0)
-                    break;
-                rightIndex++;
-            }
-
-            int rightIntegerSize = rightStopIndex - rightIndex;
-
-            int resultIntegerSize = leftIntegerSize + rightIntegerSize;
-            int resultScaleSize = org.apache.drill.exec.util.DecimalUtility.roundUp(left.scale + right.scale);
-
-            int leftSize  = left.nDecimalDigits - 1;
-            int rightSize = right.nDecimalDigits - 1;
-
-            int resultIndex = tempResult.length - 1;
-            int currentIndex = 0;
-
-            for (int i = leftSize; i >= leftIndex; i--) {
-
-                currentIndex = resultIndex;
-                int carry = 0;
-
-                for (int j = rightSize; j >= rightIndex; j--) {
-
-                    long mulResult = (long) right.getInteger(j, right.start, right.buffer) * (long) left.getInteger(i, left.start, left.buffer);
-
-                    long tempSum = tempResult[currentIndex] + mulResult + carry;
-
-                    if (tempSum >= org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE) {
-                        tempResult[currentIndex] = (int) (tempSum % org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE);
-                        carry = (int) (tempSum / org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE);
-                    } else {
-                        tempResult[currentIndex] = (int) tempSum;
-                        carry = 0;
-                    }
-
-                    currentIndex--;
-                }
-                /* Propagate the carry */
-                if (carry > 0)
-                    tempResult[currentIndex] += carry;
-
-                resultIndex--;
-            }
-
-            /* We have computed the result of the multiplication; check if we need to
-             * round a portion of the fractional part
-             */
-            resultScaleSize = org.apache.drill.exec.util.DecimalUtility.roundUp(result.scale);
-
-            if (result.scale < (left.scale + right.scale)) {
-              /* The scale of the output data type is less than the scale
-               * we obtained as a result of multiplication, we need to round
-               * a chunk of the fractional part
-               */
-              int lastScaleIndex = currentIndex + resultIntegerSize + resultScaleSize - 1;
-
-              // Compute the power of 10 necessary to find if we need to round up
-              int roundFactor = (int) (org.apache.drill.exec.util.DecimalUtility.getPowerOfTen(
-                                        org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS - ((result.scale + 1) % org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS)));
-
-              // Index of rounding digit
-              int roundIndex = currentIndex + resultIntegerSize + org.apache.drill.exec.util.DecimalUtility.roundUp(result.scale + 1) - 1;
-
-              // Check the first chopped digit to see if we need to round up
-              int carry = ((tempResult[roundIndex] / roundFactor) % 10) > 4 ? 1 : 0;
-
-              if (result.scale > 0) {
-
-                // Compute the power of 10 necessary to chop of the fractional part
-                int scaleFactor = (int) (org.apache.drill.exec.util.DecimalUtility.getPowerOfTen(
-                                         org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS - (result.scale % org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS)));
-                // Chop the unwanted fractional part
-                tempResult[lastScaleIndex] /=  scaleFactor;
-                tempResult[lastScaleIndex] *= scaleFactor;
-
-                // Adjust the carry so that it gets added to the correct digit
-                carry *= scaleFactor;
-              }
-
-              // Propagate the carry
-              while (carry > 0 && lastScaleIndex >= 0) {
-                int tempSum = tempResult[lastScaleIndex] + carry;
-                if (tempSum >= org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE) {
-                  tempResult[lastScaleIndex] = (tempSum % org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE);
-                  carry = (int) (tempSum / org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE);
-                } else {
-                  tempResult[lastScaleIndex] = tempSum;
-                  carry = 0;
-                }
-                lastScaleIndex--;
-              }
-
-              // Check if carry has increased integer digit
-              if ((lastScaleIndex + 1) < currentIndex) {
-                resultIntegerSize++;
-                currentIndex = lastScaleIndex + 1;
-              }
-            }
-
-            if (resultIntegerSize > result.nDecimalDigits) {
-              throw new org.apache.drill.common.exceptions.DrillRuntimeException("Cannot fit multiplication result in the given decimal type");
-            }
-
-            int outputIndex = result.nDecimalDigits - 1;
-
-            for (int i = (currentIndex + resultIntegerSize + resultScaleSize - 1); i >= currentIndex; i--) {
-                result.setInteger(outputIndex--, tempResult[i], result.start, result.buffer);
-            }
-
-            // Set the remaining digits to be zero
-            while(outputIndex >= 0) {
-              result.setInteger(outputIndex--, 0, result.start, result.buffer);
-            }
-            result.setSign(left.getSign(left.start, left.buffer) != right.getSign(right.start, right.buffer), result.start, result.buffer);
-        }
-    }
-
-    @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;
-        @Param ${type.name}Holder right;
-        @Output ${type.name}Holder result;
-        @Inject DrillBuf buffer;
-        @Workspace int outputScale;
-        @Workspace int outputPrecision;
-
-        public void setup() {
-            int size = (${type.storage} * (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE));
-            buffer = buffer.reallocIfNeeded(size);
-            outputPrecision = Integer.MIN_VALUE;
-        }
-
-        public void eval() {
-            if (outputPrecision == Integer.MIN_VALUE) {
-                org.apache.drill.common.util.DecimalScalePrecisionDivideFunction resultScalePrec =
-                new org.apache.drill.common.util.DecimalScalePrecisionDivideFunction((int) left.precision, (int) left.scale, (int) right.precision, (int) right.scale);
-                outputScale = resultScalePrec.getOutputScale();
-                outputPrecision = resultScalePrec.getOutputPrecision();
-            }
-            result.scale = outputScale;
-            result.precision = outputPrecision;
-            result.buffer = buffer;
-            result.start = 0;
-
-            java.math.BigDecimal numerator = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(left.buffer, left.start, left.nDecimalDigits, left.scale, true);
-            java.math.BigDecimal denominator = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(right.buffer, right.start, right.nDecimalDigits, right.scale, true);
-
-            java.math.BigDecimal output = numerator.divide(denominator, (int) result.scale, java.math.BigDecimal.ROUND_HALF_UP);
-
-            org.apache.drill.exec.util.DecimalUtility.getSparseFromBigDecimal(output, result.buffer, result.start, result.scale, result.precision, result.nDecimalDigits);
-        }
-    }
-
-    @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;
-        @Param ${type.name}Holder right;
-        @Output ${type.name}Holder result;
-        @Inject DrillBuf buffer;
-        @Workspace int outputScale;
-        @Workspace int outputPrecision;
-
-        public void setup() {
-            int size = (${type.storage} * (org.apache.drill.exec.util.DecimalUtility.INTEGER_SIZE));
-            buffer = buffer.reallocIfNeeded(size);
-            outputPrecision = Integer.MIN_VALUE;
-        }
-
-        public void eval() {
-            if (outputPrecision == Integer.MIN_VALUE) {
-                org.apache.drill.common.util.DecimalScalePrecisionModFunction resultScalePrec =
-                new org.apache.drill.common.util.DecimalScalePrecisionModFunction((int) left.precision, (int) left.scale, (int) right.precision, (int) right.scale);
-                outputScale = resultScalePrec.getOutputScale();
-                outputPrecision = resultScalePrec.getOutputPrecision();
-            }
-            result.scale = outputScale;
-            result.precision = outputPrecision;
-            result.buffer = buffer;
-            result.start = 0;
-
-            java.math.BigDecimal numerator = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(left.buffer, left.start, left.nDecimalDigits, left.scale, true);
-            java.math.BigDecimal denominator = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(right.buffer, right.start, right.nDecimalDigits, right.scale, true);
-
-            java.math.BigDecimal output = numerator.remainder(denominator);
-            output.setScale(result.scale, java.math.BigDecimal.ROUND_HALF_UP);
-
-            org.apache.drill.exec.util.DecimalUtility.getSparseFromBigDecimal(output, result.buffer, result.start, result.scale, result.precision, result.nDecimalDigits);
-        }
-    }
-
-    @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;
-        @Output ${type.name}Holder out;
-        public void setup() {
-
-        }
-
-        public void eval() {
-          out.scale = in.scale;
-          out.precision = in.precision;
-          out.buffer = in.buffer;
-          out.start = in.start;
-
-          // Set the output buffer with the positive sign
-          out.buffer.setInt(out.start, (out.buffer.getInt(out.start) & 0x7fffffff));
-        }
-    }
-
-    @FunctionTemplate(name = "sign", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class ${type.name}SignFunction implements DrillSimpleFunc {
-
-        @Param ${type.name}Holder in;
-        @Output IntHolder out;
-
-        public void setup() {}
-
-        public void eval() {
-
-          boolean zeroValue = true;
-
-          if (in.getSign(in.start, in.buffer) == true) {
-            out.value = -1;
-          } else {
-            for (int i = 0; i < ${type.storage}; i++) {
-              if (in.getInteger(i, in.start, in.buffer) != 0) {
-                zeroValue = false;
-                break;
-              }
-            }
-            out.value = (zeroValue == true) ? 0 : 1;
-          }
-        }
-    }
-
-    @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;
-        @Output ${type.name}Holder out;
-        public void setup() {
-
-        }
-
-        public void eval() {
-          out.scale = 0;
-          out.precision = in.precision;
-          out.buffer = in.buffer;
-          out.start = in.start;
-          boolean sign = in.getSign(in.start, in.buffer);
-
-          // Indicates whether we need to add 1 to the integer part, while performing ceil
-          int carry = 0;
-
-          int scaleStartIndex = ${type.storage} - org.apache.drill.exec.util.DecimalUtility.roundUp(in.scale);
-          int srcIntIndex = scaleStartIndex - 1;
-
-          if (sign == false) {
-            // For negative values ceil we don't need to increment the integer part
-            while (scaleStartIndex < ${type.storage}) {
-              if (out.getInteger(scaleStartIndex, out.start, out.buffer) != 0) {
-                carry = 1;
-                break;
-              }
-              scaleStartIndex++;
-            }
-          }
-
-          // Truncate the fractional part, move the integer part
-          int destIndex = ${type.storage} - 1;
-          while (srcIntIndex >= 0) {
-            out.setInteger(destIndex--, out.getInteger(srcIntIndex--, out.start, out.buffer), out.start, out.buffer);
-          }
-
-          // Set the remaining portion of the decimal to be zeroes
-          while (destIndex >= 0) {
-            out.setInteger(destIndex--, 0, out.start, out.buffer);
-          }
-
-          // Add the carry
-          if (carry != 0) {
-            destIndex = ${type.storage} - 1;
-
-            while (destIndex >= 0) {
-              int intValue = out.getInteger(destIndex, out.start, out.buffer);
-              intValue += carry;
-
-              if (intValue >= org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE) {
-                out.setInteger(destIndex--, intValue % org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE, out.start, out.buffer);
-                carry = intValue / org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE;
-              } else {
-                out.setInteger(destIndex--, intValue, out.start, out.buffer);
-                break;
-              }
-            }
-          }
-          // Set the sign
-          out.setSign(sign, out.start, out.buffer);
-        }
-    }
-
-    @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 {
+  @FunctionTemplate(name = FunctionGenerationHelper.GT,
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class ${type.name}GreaterThan implements DrillSimpleFunc {
 
-        @Param  ${type.name}Holder in;
-        @Output ${type.name}Holder out;
-        public void setup() {
-
-        }
-
-        public void eval() {
-          out.scale = 0;
-          out.precision = in.precision;
-          out.buffer = in.buffer;
-          out.start = in.start;
-          boolean sign = in.getSign(in.start, in.buffer);
-
-          // Indicates whether we need to decrement 1 from the integer part, while performing floor, done for -ve values
-          int carry = 0;
-
-          int scaleStartIndex = ${type.storage} - org.apache.drill.exec.util.DecimalUtility.roundUp(in.scale);
-          int srcIntIndex = scaleStartIndex - 1;
-
-          if (sign == true) {
-            // For negative values ceil we don't need to increment the integer part
-            while (scaleStartIndex < ${type.storage}) {
-              if (out.getInteger(scaleStartIndex, out.start, out.buffer) != 0) {
-                carry = 1;
-                break;
-              }
-              scaleStartIndex++;
-            }
-          }
+    @Param ${type.name}Holder left;
+    @Param ${type.name}Holder right;
+    @Output BitHolder out;
 
-          // Truncate the fractional part, move the integer part
-          int destIndex = ${type.storage} - 1;
-          while (srcIntIndex >= 0) {
-            out.setInteger(destIndex--, out.getInteger(srcIntIndex--, out.start, out.buffer), out.start, out.buffer);
-          }
+    public void setup() {}
 
-          // Set the remaining portion of the decimal to be zeroes
-          while (destIndex >= 0) {
-            out.setInteger(destIndex--, 0, out.start, out.buffer);
-          }
-          // Add the carry
-          if (carry != 0) {
-            destIndex = ${type.storage} - 1;
-
-            while (destIndex >= 0) {
-              int intValue = out.getInteger(destIndex, out.start, out.buffer);
-              intValue += carry;
-
-              if (intValue >= org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE) {
-                out.setInteger(destIndex--, intValue % org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE, out.start, out.buffer);
-                carry = intValue / org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE;
-              } else {
-                out.setInteger(destIndex--, intValue, out.start, out.buffer);
-                break;
-              }
-            }
-          }
-          // Set the sign
-          out.setSign(sign, out.start, out.buffer);
-        }
+    public void eval() {
+      int cmp;
+      <@varCompareBlock leftType = "leftType" rightType = "rightType" absCompare = "false" output = "cmp" nullCompare = false nullComparesHigh = false />
+      out.value = cmp == 1 ? 1 : 0;
     }
+  }
 
-    @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;
-        @Output ${type.name}Holder out;
-        public void setup() {
-
-        }
-
-        public void eval() {
-          out.scale = 0;
-          out.precision = in.precision;
-          out.buffer = in.buffer;
-          out.start = in.start;
-          boolean sign = in.getSign(in.start, in.buffer);
+  <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
+         not for sorting and grouping relational operators.) -->
+  @FunctionTemplate(name = FunctionGenerationHelper.GE,
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class ${type.name}GreaterThanEq implements DrillSimpleFunc {
 
-          // Integer part's src index
-          int srcIntIndex = ${type.storage} - org.apache.drill.exec.util.DecimalUtility.roundUp(in.scale) - 1;
+    @Param ${type.name}Holder left;
+    @Param ${type.name}Holder right;
+    @Output BitHolder out;
 
-          // Truncate the fractional part, move the integer part
-          int destIndex = ${type.storage} - 1;
-          while (srcIntIndex >= 0) {
-            out.setInteger(destIndex--, out.getInteger(srcIntIndex--, out.start, out.buffer), out.start, out.buffer);
-          }
+    public void setup() {}
 
-          // Set the remaining portion of the decimal to be zeroes
-          while (destIndex >= 0) {
-            out.setInteger(destIndex--, 0, out.start, out.buffer);
-          }
-          // Set the sign
-          out.setSign(sign, out.start, out.buffer);
-        }
+    public void eval() {
+      int cmp;
+      <@varCompareBlock leftType = "leftType" rightType = "rightType" absCompare = "false" output = "cmp" nullCompare = false nullComparesHigh = false />
+      out.value = cmp > -1 ? 1 : 0;
     }
+  }
 
-    @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 {
+  <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
+         not for sorting and grouping relational operators.) -->
+  @FunctionTemplate(name = FunctionGenerationHelper.EQ,
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class ${type.name}Equal implements DrillSimpleFunc {
 
-        @Param  ${type.name}Holder left;
-        @Param  IntHolder right;
-        @Output ${type.name}Holder result;
-        public void setup() {
+    @Param ${type.name}Holder left;
+    @Param ${type.name}Holder right;
+    @Output BitHolder out;
 
-        }
+    public void setup() {}
 
-        public void eval() {
-          result.scale = right.value;
-          result.precision = left.precision;
-          result.buffer = left.buffer;
-          result.start = left.start;
-          boolean sign = left.getSign(left.start, left.buffer);
-
-          int newScaleRoundedUp  = org.apache.drill.exec.util.DecimalUtility.roundUp(right.value);
-          int origScaleRoundedUp = org.apache.drill.exec.util.DecimalUtility.roundUp(left.scale);
-
-          if (right.value < left.scale) {
-            // Get the source index beyond which we will truncate
-            int srcIntIndex = ${type.storage} - origScaleRoundedUp - 1;
-            int srcIndex = srcIntIndex + newScaleRoundedUp;
-
-            // Truncate the remaining fractional part, move the integer part
-            int destIndex = ${type.storage} - 1;
-            if (srcIndex != destIndex) {
-              while (srcIndex >= 0) {
-                result.setInteger(destIndex--, result.getInteger(srcIndex--, result.start, result.buffer), result.start, result.buffer);
-              }
-
-              // Set the remaining portion of the decimal to be zeroes
-              while (destIndex >= 0) {
-                result.setInteger(destIndex--, 0, result.start, result.buffer);
-              }
-            }
-
-            // We truncated the decimal digit. Now we need to truncate within the base 1 billion fractional digit
-            int truncateFactor = org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS - (right.value % org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS);
-            if (truncateFactor != org.apache.drill.exec.util.DecimalUtility.MAX_DIGITS) {
-              truncateFactor = (int) org.apache.drill.exec.util.DecimalUtility.getPowerOfTen(truncateFactor);
-              int fractionalDigits = result.getInteger(${type.storage} - 1, result.start, result.buffer);
-              fractionalDigits /= truncateFactor;
-              result.setInteger(${type.storage} - 1, fractionalDigits * truncateFactor, result.start, result.buffer);
-            }
-          } else if (right.value > left.scale) {
-            // Add fractional digits to the decimal
-
-            // Check if we need to shift the decimal digits to the left
-            if (newScaleRoundedUp > origScaleRoundedUp) {
-              int srcIndex  = 0;
-              int destIndex = newScaleRoundedUp - origScaleRoundedUp;
-
-              // Check while extending scale, we are not overwriting integer part
-              while (srcIndex < destIndex) {
-                if (result.getInteger(srcIndex++, result.start, result.buffer) != 0) {
-                  throw new org.apache.drill.common.exceptions.DrillRuntimeException("Truncate resulting in loss of integer part, reduce scale specified");
-                }
-              }
-
-              srcIndex = 0;
-              while (destIndex < ${type.storage}) {
-                result.setInteger(srcIndex++, result.getInteger(destIndex++, result.start, result.buffer), result.start, result.buffer);
-              }
-
-              // Clear the remaining part
-              while (srcIndex < ${type.storage}) {
-                result.setInteger(srcIndex++, 0, result.start, result.buffer);
-              }
-            }
-          }
-          // Set the sign
-          result.setSign(sign, result.start, result.buffer);
-        }
+    public void eval() {
+      int cmp;
+      <@varCompareBlock leftType = "leftType" rightType = "rightType" absCompare = "false" output = "cmp" nullCompare = false nullComparesHigh = false />
+      out.value = cmp == 0 ? 1 : 0;
     }
+  }
 
-    @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;
-        @Output ${type.name}Holder out;
-        public void setup() {
-
-        }
-
-        public void eval() {
-          out.scale = 0;
-          out.precision = in.precision;
-          out.buffer = in.buffer;
-          out.start = in.start;
-          boolean sign = in.getSign(in.start, in.buffer);
-
-          boolean roundUp = false;
-
-          // Get the first fractional digit to see if want to round up or not
-          int scaleIndex = ${type.storage} - org.apache.drill.exec.util.DecimalUtility.roundUp(in.scale);
-          if (scaleIndex < ${type.storage}) {
-            int fractionalPart = out.getInteger(scaleIndex, out.start, out.buffer);
-            int digit = fractionalPart / (org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE / 10);
-
-            if (digit > 4) {
-              roundUp = true;
-            }
-          }
-
-          // Integer part's src index
-          int srcIntIndex = scaleIndex - 1;
+  <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
+         not for sorting and grouping relational operators.) -->
+  @FunctionTemplate(name = FunctionGenerationHelper.NE,
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class ${type.name}NotEqual implements DrillSimpleFunc {
 
-          // Truncate the fractional part, move the integer part
-          int destIndex = ${type.storage} - 1;
-          while (srcIntIndex >= 0) {
-            out.setInteger(destIndex--, out.getInteger(srcIntIndex--, out.start, out.buffer), out.start, out.buffer);
-          }
+    @Param ${type.name}Holder left;
+    @Param ${type.name}Holder right;
+    @Output BitHolder out;
 
-          // Set the remaining portion of the decimal to be zeroes
-          while (destIndex >= 0) {
-            out.setInteger(destIndex--, 0, out.start, out.buffer);
-          }
+    public void setup() {}
 
-          // Perform the roundup
-          srcIntIndex = ${type.storage} - 1;
-          if (roundUp == true) {
-            while (srcIntIndex >= 0) {
-              int value = out.getInteger(srcIntIndex, out.start, out.buffer) + 1;
-              if (value >= org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE) {
-                out.setInteger(srcIntIndex--, value % org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE, out.start, out.buffer);
-                value = value / org.apache.drill.exec.util.DecimalUtility.DIGITS_BASE;
-              } else {
-                out.setInteger(srcIntIndex--, value, out.start, out.buffer);
-                break;
-              }
-            }
-          }
-          // Set the sign
-          out.setSign(sign, out.start, out.buffer);
-        }
-    }
-
-    @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;
-        @Param  IntHolder right;
-        @Output ${type.name}Holder result;
-        public void setup() {
-
-        }
-
-        public void eval() {
-          result.scale = right.value;
-          result.precision = left.precision;
-          result.buffer = left.buffer;
-          result.start = left.start;
-          boolean sign = left.getSign(left.start, left.buffer);
-
-          org.apache.drill.exec.util.DecimalUtility.roundDecimal(result.buffer, result.start, result.nDecimalDigits, result.scale, left.scale);
-          // Set the sign
-          result.setSign(sign, result.start, result.buffer);
-        }
-    }
-
- <#-- Handle 2 x 2 combinations of nullable and non-nullable arguments. -->
- <#list ["Nullable${type.name}", "${type.name}"] as leftType >
- <#list ["Nullable${type.name}", "${type.name}"] as rightType >
-
-  <#-- Comparison function for sorting and grouping relational operators
-       (not for comparison expression operators (=, <, etc.)). -->
-  @FunctionTemplate(name = FunctionGenerationHelper.COMPARE_TO_NULLS_HIGH,
-      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;
-    @Param ${rightType}Holder right;
-    @Output IntHolder out;
-
-    public void setup() {}
-
-    public void eval() {
-      <@compareBlock leftType=leftType rightType=rightType absCompare="false" output="out.value" nullCompare=true nullComparesHigh=true />
-    }
-  }
-
-
-
-  <#-- Comparison function for sorting and grouping relational operators
-        (not for comparison expression operators (=, <, etc.)). -->
-  @FunctionTemplate(name = FunctionGenerationHelper.COMPARE_TO_NULLS_LOW,
-      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;
-    @Param ${rightType}Holder right;
-    @Output IntHolder out;
-
-    public void setup() {}
-
-    public void eval() {
-      <@compareBlock leftType=leftType rightType=rightType absCompare="false" output="out.value" nullCompare=true nullComparesHigh=false />
-    }
-  }
-
- </#list>
- </#list>
-
-
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-         not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "less than",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp;
-            <@compareBlock leftType="leftType" rightType="rightType" absCompare="false" output="cmp" nullCompare=false nullComparesHigh=false />
-            out.value = cmp == -1 ? 1 : 0;
-        }
-    }
-
-
-    // TODO:  RESOLVE:  Here there are spaces in function template names, but
-    // elsewhere there are underlines.  Are things being looked up correctly?
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-         not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "less than or equal to",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp;
-            <@compareBlock leftType="leftType" rightType="rightType" absCompare="false" output="cmp" nullCompare=false nullComparesHigh=false />
-            out.value = cmp < 1 ? 1 : 0;
-        }
-    }
-
-
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-         not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "greater than",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp;
-            <@compareBlock leftType="leftType" rightType="rightType" absCompare="false" output="cmp" nullCompare=false nullComparesHigh=false />
-            out.value = cmp == 1 ? 1 : 0;
-        }
-    }
-
-
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-         not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "greater than or equal to",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp;
-            <@compareBlock leftType="leftType" rightType="rightType" absCompare="false" output="cmp" nullCompare=false nullComparesHigh=false />
-            out.value = cmp > -1 ? 1 : 0;
-        }
-    }
-
-
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-         not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "Equal",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp;
-            <@compareBlock leftType="leftType" rightType="rightType" absCompare="false" output="cmp" nullCompare=false nullComparesHigh=false />
-            out.value = cmp == 0 ? 1 : 0;
-        }
-    }
-
-
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-         not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "not equal",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp;
-            <@compareBlock leftType="leftType" rightType="rightType" absCompare="false" output="cmp" nullCompare=false nullComparesHigh=false />
-            out.value = cmp != 0 ? 1 : 0;
-        }
-    }
-}
-
-<#elseif type.name.endsWith("Dense")>
-
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/${type.name}Functions.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl;
-
-<#include "/@includes/vv_imports.ftl" />
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.fn.FunctionGenerationHelper;
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
-
-import io.netty.buffer.ByteBuf;
-
-import java.nio.ByteBuffer;
-
-/*
- * This class is generated using freemarker and the ${.template_name} template.
- */
-
-@SuppressWarnings("unused")
-public class ${type.name}Functions {
-
-
- <#-- Handle 2 x 2 combinations of nullable and non-nullable arguments. -->
- <#list ["Nullable${type.name}", "${type.name}"] as leftType >
- <#list ["Nullable${type.name}", "${type.name}"] as rightType >
-
-  <#-- Comparison function for sorting and grouping relational operators
-       (not for comparison expression operators (=, <, etc.)). -->
-  @FunctionTemplate(name = FunctionGenerationHelper.COMPARE_TO_NULLS_HIGH,
-      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;
-    @Param ${rightType}Holder right;
-    @Output IntHolder out;
-
-    public void setup() {}
-
-    public void eval() {
-     outside:
-      {
-        <@compareNullsSubblock leftType=leftType rightType=rightType output="out.value" breakTarget="outside" nullCompare=true nullComparesHigh=true />
-
-        out.value = org.apache.drill.exec.util.DecimalUtility.compareDenseBytes(left.buffer, left.start, left.getSign(left.start, left.buffer), right.buffer, right.start, right.getSign(right.start, right.buffer), left.WIDTH);
-      } // outside
-    }
-  }
-
-  <#-- Comparison function for sorting and grouping relational operators
-       (not for comparison expression operators (=, <, etc.)). -->
-  @FunctionTemplate(name = FunctionGenerationHelper.COMPARE_TO_NULLS_LOW,
-      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;
-    @Param ${rightType}Holder right;
-    @Output IntHolder out;
-
-    public void setup() {}
-
-    public void eval() {
-     outside:
-      {
-        <@compareNullsSubblock leftType=leftType rightType=rightType output="out.value" breakTarget="outside" nullCompare=true nullComparesHigh=false />
-
-        out.value = org.apache.drill.exec.util.DecimalUtility.compareDenseBytes(left.buffer, left.start, left.getSign(left.start, left.buffer), right.buffer, right.start, right.getSign(right.start, right.buffer), left.WIDTH);
-      } // outside
-    }
-  }
-
- </#list>
- </#list>
-
-
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-         not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "less than",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp  = org.apache.drill.exec.util.DecimalUtility.compareDenseBytes(left.buffer, left.start, left.getSign(left.start, left.buffer), right.buffer, right.start, right.getSign(right.start, right.buffer), left.WIDTH);
-            out.value = cmp == -1 ? 1 : 0;
-        }
-    }
-
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-         not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "less than or equal to",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp = org.apache.drill.exec.util.DecimalUtility.compareDenseBytes(left.buffer, left.start, left.getSign(left.start, left.buffer), right.buffer, right.start, right.getSign(right.start, right.buffer), left.WIDTH);
-            out.value = cmp < 1 ? 1 : 0;
-        }
-    }
-
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-         not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "greater than",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp = org.apache.drill.exec.util.DecimalUtility.compareDenseBytes(left.buffer, left.start, left.getSign(left.start, left.buffer), right.buffer, right.start, right.getSign(right.start, right.buffer), left.WIDTH);
-            out.value = cmp == 1 ? 1 : 0;
-        }
-    }
-
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-         not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "greater than or equal to",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp = org.apache.drill.exec.util.DecimalUtility.compareDenseBytes(left.buffer, left.start, left.getSign(left.start, left.buffer), right.buffer, right.start, right.getSign(right.start, right.buffer), left.WIDTH);
-            out.value = cmp > -1 ? 1 : 0;
-        }
-    }
-
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-         not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "Equal",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            int cmp = org.apache.drill.exec.util.DecimalUtility.compareDenseBytes(left.buffer, left.start, left.getSign(left.start, left.buffer), right.buffer, right.start, right.getSign(right.start, right.buffer), left.WIDTH);
-            out.value = cmp == 0 ? 1 : 0;
-        }
-    }
-
-
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-         not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "not equal",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-
-            int cmp = org.apache.drill.exec.util.DecimalUtility.compareDenseBytes(left.buffer, left.start, left.getSign(left.start, left.buffer), right.buffer, right.start, right.getSign(right.start, right.buffer), left.WIDTH);
-            out.value = cmp != 0 ? 1 : 0;
-        }
-    }
-}
-
-<#elseif type.name.endsWith("Decimal9") || type.name.endsWith("Decimal18")>
-
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/${type.name}Functions.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl;
-
-<#include "/@includes/vv_imports.ftl" />
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
-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.fn.FunctionGenerationHelper;
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.RecordBatch;
-
-import io.netty.buffer.ByteBuf;
-
-import java.nio.ByteBuffer;
-
-@SuppressWarnings("unused")
-public class ${type.name}Functions {
-
-    @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;
-        @Param ${type.name}Holder right;
-        @Workspace int outputScale;
-        @Workspace int outputPrecision;
-        @Output ${type.name}Holder result;
-
-        public void setup() {
-            outputPrecision = Integer.MIN_VALUE;
-        }
-
-
-        public void eval() {
-            if (outputPrecision == Integer.MIN_VALUE) {
-                org.apache.drill.common.util.DecimalScalePrecisionAddFunction resultScalePrec =
-                new org.apache.drill.common.util.DecimalScalePrecisionAddFunction((int) left.precision, (int) left.scale, (int) right.precision, (int) right.scale);
-                outputScale = resultScalePrec.getOutputScale();
-                outputPrecision = resultScalePrec.getOutputPrecision();
-            }
-            <@adjustScale javaType=type.storage leftType="leftType" rightType="rightType"/>
-
-            result.value = left.value + right.value;
-            result.precision = outputPrecision;
-            result.scale = outputScale;
-        }
-    }
-
-    @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;
-        @Param ${type.name}Holder right;
-        @Workspace int outputScale;
-        @Workspace int outputPrecision;
-        @Output ${type.name}Holder result;
-
-        public void setup() {
-            outputPrecision = Integer.MIN_VALUE;
-        }
-
-        public void eval() {
-            if (outputPrecision == Integer.MIN_VALUE) {
-                org.apache.drill.common.util.DecimalScalePrecisionAddFunction resultScalePrec =
-                new org.apache.drill.common.util.DecimalScalePrecisionAddFunction((int) left.precision, (int) left.scale, (int) right.precision, (int) right.scale);
-                outputScale = resultScalePrec.getOutputScale();
-                outputPrecision = resultScalePrec.getOutputPrecision();
-            }
-            <@adjustScale javaType=type.storage leftType="leftType" rightType="rightType"/>
-
-            result.value = left.value - right.value;
-            result.precision = outputPrecision;
-            result.scale = outputScale;
-        }
-    }
-
-    @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;
-        @Param ${type.name}Holder right;
-        @Workspace int outputScale;
-        @Workspace int outputPrecision;
-        @Output ${type.name}Holder result;
-
-        public void setup() {
-            outputPrecision = Integer.MIN_VALUE;
-        }
-
-        public void eval() {
-            if (outputPrecision == Integer.MIN_VALUE) {
-                org.apache.drill.common.util.DecimalScalePrecisionMulFunction resultScalePrec =
-                new org.apache.drill.common.util.DecimalScalePrecisionMulFunction((int) left.precision, (int) left.scale, (int) right.precision, (int) right.scale);
-                outputScale = resultScalePrec.getOutputScale();
-                outputPrecision = resultScalePrec.getOutputPrecision();
-            }
-            result.value = left.value * right.value;
-            result.precision = outputPrecision;
-            result.scale = outputScale;
-        }
-    }
-
-    @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;
-        @Output ${type.name}Holder out;
-
-        public void setup() {}
-
-        public void eval() {
-            out.precision = out.maxPrecision;
-            out.scale = in.scale;
-
-            out.value = in.value;
-
-            if (out.value < 0){
-                out.value *= -1;
-            }
-        }
-    }
-
-    @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;
-        @Param ${type.name}Holder right;
-        @Output ${type.name}Holder result;
-        @Workspace int outputScale;
-        @Workspace int outputPrecision;
-
-        public void setup() {
-            outputPrecision = Integer.MIN_VALUE;
-        }
-
-        public void eval() {
-
-            if (outputPrecision == Integer.MIN_VALUE) {
-                org.apache.drill.common.util.DecimalScalePrecisionDivideFunction resultScalePrec =
-                new org.apache.drill.common.util.DecimalScalePrecisionDivideFunction((int) left.precision, (int) left.scale, (int) right.precision, (int) right.scale);
-                outputScale = resultScalePrec.getOutputScale();
-                outputPrecision = resultScalePrec.getOutputPrecision();
-            }
-            result.scale = outputScale;
-            result.precision = outputPrecision;
-
-            java.math.BigDecimal numerator = new java.math.BigDecimal(java.math.BigInteger.valueOf(left.value), left.scale);
-            java.math.BigDecimal denominator = new java.math.BigDecimal(java.math.BigInteger.valueOf(right.value), right.scale);
-
-            java.math.BigDecimal output = numerator.divide(denominator, (int) result.scale, java.math.BigDecimal.ROUND_HALF_UP);
-
-            result.value = output.unscaledValue().${type.storage}Value();
-        }
-    }
-
-    @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;
-        @Param ${type.name}Holder right;
-        @Workspace int outputScale;
-        @Workspace int outputPrecision;
-        @Output ${type.name}Holder result;
-
-        public void setup() {
-            outputPrecision = Integer.MIN_VALUE;
-        }
-
-        public void eval() {
-            if (outputPrecision == Integer.MIN_VALUE) {
-                org.apache.drill.common.util.DecimalScalePrecisionModFunction resultScalePrec =
-                new org.apache.drill.common.util.DecimalScalePrecisionModFunction((int) left.precision, (int) left.scale, (int) right.precision, (int) right.scale);
-                outputScale = resultScalePrec.getOutputScale();
-                outputPrecision = resultScalePrec.getOutputPrecision();
-            }
-            result.precision = outputPrecision;
-            result.scale = outputScale;
-            java.math.BigDecimal numerator = new java.math.BigDecimal(java.math.BigInteger.valueOf(left.value), left.scale);
-            java.math.BigDecimal denominator = new java.math.BigDecimal(java.math.BigInteger.valueOf(right.value), right.scale);
-
-            java.math.BigDecimal output = numerator.remainder(denominator);
-            output.setScale(result.scale, java.math.BigDecimal.ROUND_HALF_UP);
-
-            result.value = output.unscaledValue().${type.storage}Value();
-        }
-    }
-
-    @FunctionTemplate(name = "sign", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class ${type.name}SignFunction implements DrillSimpleFunc {
-
-        @Param ${type.name}Holder in;
-        @Output IntHolder out;
-
-        public void setup() {}
-
-        public void eval() {
-
-            out.value = (in.value < 0) ? -1 : ((in.value > 0) ? 1 : 0);
-        }
-    }
-
-    @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;
-        @Output ${type.name}Holder out;
-
-        public void setup() {}
-
-        public void eval() {
-
-            out.value =(${type.storage}) (org.apache.drill.exec.util.DecimalUtility.adjustScaleDivide(in.value, (int) in.scale));
-            out.precision = out.maxPrecision;
-            out.scale = 0;
-        }
-    }
-
-    @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;
-        @Param IntHolder right;
-        @Output ${type.name}Holder out;
-
-        public void setup() {}
-
-        public void eval() {
-
-            out.value = (${type.storage}) (org.apache.drill.exec.util.DecimalUtility.adjustScaleDivide(left.value, (int) (left.scale - right.value)));
-            out.precision = out.maxPrecision;
-            out.scale = right.value;
-        }
-    }
-
-    @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;
-        @Output ${type.name}Holder out;
-
-        public void setup() {
-
-        }
-
-        public void eval() {
-          ${type.storage} scaleFactor = (${type.storage}) (org.apache.drill.exec.util.DecimalUtility.getPowerOfTen((int) in.scale));
-
-          // Get the integer part
-          ${type.storage} integerPart = in.value / scaleFactor;
-
-          // Get the fractional part, if its non-zero increment the integer part
-          ${type.storage} fractionalPart = (${type.storage}) (in.value % scaleFactor);
-          if (fractionalPart != 0 && in.value >= 0) {
-            integerPart++;
-          }
-
-          out.scale = 0;
-          out.value = integerPart;
-        }
-    }
-
-    @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;
-        @Output ${type.name}Holder out;
-
-        public void setup() {
-        }
-
-        public void eval() {
-
-          ${type.storage} scaleFactor = (${type.storage}) (org.apache.drill.exec.util.DecimalUtility.getPowerOfTen((int) in.scale));
-          out.scale = 0;
-          out.value = (in.value / scaleFactor);
-
-          // For negative values we have to decrement by 1
-          if (in.value < 0) {
-            ${type.storage} fractionalPart = (${type.storage}) (in.value % scaleFactor);
-            if (fractionalPart != 0) {
-              out.value--;
-            }
-          }
-        }
-    }
-
-    @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;
-        @Output ${type.name}Holder out;
-
-        public void setup() {
-        }
-
-        public void eval() {
-
-          ${type.storage} scaleFactor = (${type.storage}) (org.apache.drill.exec.util.DecimalUtility.getPowerOfTen((int) in.scale));
-          ${type.storage} extractDigit = scaleFactor / 10;
-
-          out.scale = 0;
-          // Assign the integer part to the output
-          out.value = in.value / scaleFactor;
-
-          // Get the fractional part
-          ${type.storage} fractionalPart = in.value % scaleFactor;
-          // Get the first digit to check for rounding
-          int digit = Math.abs((int) (fractionalPart / extractDigit));
-
-          if (digit > 4) {
-            if (in.value > 0) {
-              out.value++;
-            } else if (in.value < 0) {
-              out.value--;
-            }
-          }
-        }
-    }
-
-    @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;
-        @Param IntHolder right;
-        @Output ${type.name}Holder out;
-
-
-        public void setup() {
-        }
-
-        public void eval() {
-
-          ${type.storage} scaleFactor = (${type.storage}) (org.apache.drill.exec.util.DecimalUtility.getPowerOfTen((int) left.scale));
-          ${type.storage} newScaleFactor = (${type.storage}) (org.apache.drill.exec.util.DecimalUtility.getPowerOfTen((int) right.value));
-          ${type.storage} truncScaleFactor = (${type.storage}) (org.apache.drill.exec.util.DecimalUtility.getPowerOfTen( Math.abs(left.scale - right.value)));
-          int truncFactor = (int) (left.scale - right.value);
-
-          // If rounding scale is >= current scale
-          if (right.value >= left.scale) {
-            out.value = (${type.storage}) (org.apache.drill.exec.util.DecimalUtility.adjustScaleMultiply(left.value, (int) (right.value - left.scale)));
-          }
-          else {
-            out.scale = right.value;
-            // Assign the integer part to the output
-            out.value = left.value / scaleFactor;
-
-            // Get the fractional part
-            ${type.storage} fractionalPart = left.value % scaleFactor;
-
-            // From the entire fractional part extract the digits upto which rounding is needed
-            ${type.storage} newFractionalPart = (${type.storage}) (org.apache.drill.exec.util.DecimalUtility.adjustScaleDivide(fractionalPart, truncFactor));
-            ${type.storage} truncatedFraction = fractionalPart % truncScaleFactor;
-
-
-            // Get the truncated fractional part and extract the first digit to see if we need to add 1
-            int digit = Math.abs((int) org.apache.drill.exec.util.DecimalUtility.adjustScaleDivide(truncatedFraction, truncFactor - 1));
-
-            if (digit > 4) {
-              if (left.value > 0) {
-                newFractionalPart++;
-              } else if (left.value < 0) {
-                newFractionalPart--;
-              }
-            }
-
-            out.value = (out.value * newScaleFactor) + newFractionalPart;
-          }
-        }
-    }
-
- <#-- Handle 2 x 2 combinations of nullable and non-nullable arguments. -->
- <#list ["Nullable${type.name}", "${type.name}"] as leftType >
- <#list ["Nullable${type.name}", "${type.name}"] as rightType >
-
-  <#-- Comparison function for sorting and grouping relational operators
-       (not for comparison expression operators (=, <, etc.)). -->
-  @FunctionTemplate(name = FunctionGenerationHelper.COMPARE_TO_NULLS_HIGH,
-      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;
-    @Param ${rightType}Holder right;
-    @Output IntHolder out;
-
-    public void setup() {}
-
-    public void eval() {
-     outside:
-      {
-        <@compareNullsSubblock leftType=leftType rightType=rightType output="out.value" breakTarget="outside" nullCompare=true nullComparesHigh=true />
-
-        <@adjustScale javaType=type.storage leftType="leftType" rightType="rightType"/>
-        out.value = (left.value < right.value) ? -1 : (left.value > right.value) ? 1 : 0;
-      } // outside
+    public void eval() {
+      int cmp;
+      <@varCompareBlock leftType = "leftType" rightType = "rightType" absCompare = "false" output = "cmp" nullCompare = false nullComparesHigh = false />
+      out.value = cmp != 0 ? 1 : 0;
     }
   }
-
-  <#-- Comparison function for sorting and grouping relational operators
-       (not for comparison expression operators (=, <, etc.)). -->
-  @FunctionTemplate(name = FunctionGenerationHelper.COMPARE_TO_NULLS_LOW,
-    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;
-    @Param ${rightType}Holder right;
-    @Output IntHolder out;
-
-    public void setup() {}
-
-    public void eval() {
-     outside:
-      {
-        <@compareNullsSubblock leftType=leftType rightType=rightType output="out.value" breakTarget="outside" nullCompare=true nullComparesHigh=false />
-        <@adjustScale javaType=type.storage leftType="leftType" rightType="rightType"/>
-        out.value = (left.value < right.value) ? -1 : (left.value > right.value) ? 1 : 0;
-      } // outside
-    }
-  }
-
- </#list>
- </#list>
-
-    <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-         not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "less than",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            <@adjustScale javaType=type.storage leftType="leftType" rightType="rightType"/>
-            out.value = (left.value < right.value) ? 1 : 0;
-        }
-    }
-
-     <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-          not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "less than or equal to",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            <@adjustScale javaType=type.storage leftType="leftType" rightType="rightType"/>
-            out.value = (left.value <= right.value) ? 1 : 0;
-        }
-    }
-
-     <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-          not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "greater than",
-        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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            <@adjustScale javaType=type.storage leftType="leftType" rightType="rightType"/>
-            out.value = (left.value > right.value) ? 1 : 0;
-        }
-    }
-
-     <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-          not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "greater than or equal to",
-            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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            <@adjustScale javaType=type.storage leftType="leftType" rightType="rightType"/>
-            out.value = (left.value >= right.value) ? 1 : 0;
-        }
-    }
-
-     <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-          not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "Equal",
-            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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            <@adjustScale javaType=type.storage leftType="leftType" rightType="rightType"/>
-            out.value = (left.value == right.value) ? 1 : 0;
-        }
-    }
-
-
-     <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
-          not for sorting and grouping relational operators.) -->
-    @FunctionTemplate(name = "not equal",
-            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;
-        @Param ${type.name}Holder right;
-        @Output BitHolder out;
-        public void setup() {}
-
-        public void eval() {
-            <@adjustScale javaType=type.storage leftType="leftType" rightType="rightType"/>
-            out.value = (left.value != right.value) ? 1 : 0;
-        }
-    }
 }
-
 </#if>
 
 </#list>
diff --git a/exec/java-exec/src/main/codegen/templates/MathFunctions.java b/exec/java-exec/src/main/codegen/templates/MathFunctions.java
index d1366cb..a17865c 100644
--- a/exec/java-exec/src/main/codegen/templates/MathFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/MathFunctions.java
@@ -227,12 +227,7 @@ public static class ${func.className}${type.input} implements DrillSimpleFunc {
   }
 
   public void eval() {
-	  <#if type.input?matches("^Decimal[1-9]*")>
-	  double dblval = new java.math.BigDecimal(in.value).setScale(in.scale).doubleValue();
-	  out.value = ${func.javaFunc}(dblval);
-	  <#else>
 	  out.value = ${func.javaFunc}(in.value);
-	  </#if>
   }
 }
 
@@ -257,12 +252,7 @@ public static class ${func.className}${type.input} implements DrillSimpleFunc {
   }
 
   public void eval() {
-	  <#if type.input?matches("^Decimal[1-9]*")>
-	  double dblval = new java.math.BigDecimal(val.value).setScale(val.scale).doubleValue();
-	  out.value = ${func.javaFunc}(dblval)/ ${func.javaFunc}(base.value);
-	  <#else>
 	  out.value = ${func.javaFunc}(val.value)/ ${func.javaFunc}(base.value);
-	  </#if>
   }
 }
 </#list>
diff --git a/exec/java-exec/src/main/codegen/templates/NumericToCharFunctions.java b/exec/java-exec/src/main/codegen/templates/NumericToCharFunctions.java
index 3fc45df..b192be3 100644
--- a/exec/java-exec/src/main/codegen/templates/NumericToCharFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/NumericToCharFunctions.java
@@ -48,45 +48,36 @@ import java.text.DecimalFormat;
  * This class is generated using freemarker and the ${.template_name} template.
  */
 @SuppressWarnings("unused")
-@FunctionTemplate(name = "to_char", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+@FunctionTemplate(name = "to_char",
+                  scope = FunctionTemplate.FunctionScope.SIMPLE,
+                  nulls = NullHandling.NULL_IF_NULL)
 public class G${type}ToChar implements DrillSimpleFunc {
 
-    @Param  ${type}Holder left;
-    @Param  VarCharHolder right;
-    @Inject DrillBuf buffer;
-    @Workspace java.text.NumberFormat outputFormat;
-    @Output VarCharHolder out;
+  @Param  ${type}Holder left;
+  @Param  VarCharHolder right;
+  @Inject DrillBuf buffer;
+  @Workspace java.text.NumberFormat outputFormat;
+  @Output VarCharHolder out;
 
-    public void setup() {
-        buffer = buffer.reallocIfNeeded(100);
-        byte[] buf = new byte[right.end - right.start];
-        right.buffer.getBytes(right.start, buf, 0, right.end - right.start);
-        String inputFormat = new String(buf);
-        outputFormat = new java.text.DecimalFormat(inputFormat);
-    }
+  public void setup() {
+    buffer = buffer.reallocIfNeeded(100);
+    byte[] buf = new byte[right.end - right.start];
+    right.buffer.getBytes(right.start, buf, 0, right.end - right.start);
+    String inputFormat = new String(buf);
+    outputFormat = new java.text.DecimalFormat(inputFormat);
+  }
 
-    public void eval() {
-
-        <#if type == "VarDecimal">
-        java.math.BigDecimal bigDecimal = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(left.buffer, left.start, left.end - left.start, left.scale);
-        String str = outputFormat.format(bigDecimal);
-        <#elseif type == "Decimal9" || type == "Decimal18">
-        java.math.BigDecimal bigDecimal = new java.math.BigDecimal(java.math.BigInteger.valueOf(left.value), left.scale);
-        String str = outputFormat.format(bigDecimal);
-        <#elseif type == "Decimal28Sparse" || type == "Decimal38Sparse">
-        java.math.BigDecimal bigDecimal = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(left.buffer, left.start, left.nDecimalDigits, left.scale, true);
-        String str = outputFormat.format(bigDecimal);
-        <#elseif type == "Decimal28Dense" || type == "Decimal38Dense">
-        java.math.BigDecimal bigDecimal = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDense(left.buffer, left.start, left.nDecimalDigits, left.scale, left.maxPrecision, left.WIDTH);
-        String str = outputFormat.format(bigDecimal);
-        <#else>
-        String str =  outputFormat.format(left.value);
-        </#if>
-        out.buffer = buffer;
-        out.start = 0;
-        out.end = Math.min(100, str.length()); // truncate if target type has length smaller than that of input's string
-        out.buffer.setBytes(0, str.substring(0,out.end).getBytes());
-
-    }
+  public void eval() {
+    <#if type == "VarDecimal">
+    java.math.BigDecimal bigDecimal = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(left.buffer, left.start, left.end - left.start, left.scale);
+    String str = outputFormat.format(bigDecimal);
+    <#else>
+    String str =  outputFormat.format(left.value);
+    </#if>
+    out.buffer = buffer;
+    out.start = 0;
+    out.end = Math.min(100, str.length()); // truncate if target type has length smaller than that of input's string
+    out.buffer.setBytes(0, str.substring(0, out.end).getBytes());
+  }
 }
-</#list>
\ No newline at end of file
+</#list>
diff --git a/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java b/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
index 26a33b9..20f4d60 100644
--- a/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/ParquetOutputRecordWriter.java
@@ -28,17 +28,21 @@ import java.util.Arrays;
 package org.apache.drill.exec.store;
 
 import com.google.common.collect.Lists;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.expr.holders.*;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
 import org.apache.drill.exec.store.parquet.ParquetTypeHelper;
+import org.apache.drill.exec.store.parquet.decimal.DecimalValueWriter;
 import org.apache.drill.exec.vector.*;
 import org.apache.drill.exec.util.DecimalUtility;
 import org.apache.drill.exec.vector.complex.reader.FieldReader;
 import org.apache.parquet.io.api.RecordConsumer;
 import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.io.api.Binary;
 import io.netty.buffer.DrillBuf;
 import org.apache.drill.exec.record.BatchSchema;
@@ -75,6 +79,8 @@ public abstract class ParquetOutputRecordWriter extends AbstractRecordWriter imp
     this.consumer = consumer;
   }
 
+  protected abstract PrimitiveType getPrimitiveType(MaterializedField field);
+
 <#list vv.types as type>
   <#list type.minor as minor>
     <#list vv.modes as mode>
@@ -86,19 +92,25 @@ public abstract class ParquetOutputRecordWriter extends AbstractRecordWriter imp
   public class ${mode.prefix}${minor.class}ParquetConverter extends FieldConverter {
     private Nullable${minor.class}Holder holder = new Nullable${minor.class}Holder();
     <#if minor.class?contains("Interval")>
-      private final byte[] output = new byte[12];
+    private final byte[] output = new byte[12];
+    <#elseif minor.class == "VarDecimal">
+    private final DecimalValueWriter decimalValueWriter;
     </#if>
 
     public ${mode.prefix}${minor.class}ParquetConverter(int fieldId, String fieldName, FieldReader reader) {
       super(fieldId, fieldName, reader);
+      <#if minor.class == "VarDecimal">
+      decimalValueWriter = DecimalValueWriter.
+          getDecimalValueWriterForType(getPrimitiveType(reader.getField()).getPrimitiveTypeName());
+      </#if>
     }
 
     @Override
     public void writeField() throws IOException {
   <#if mode.prefix == "Nullable" >
-    if (!reader.isSet()) {
-      return;
-    }
+      if (!reader.isSet()) {
+        return;
+      }
   <#elseif mode.prefix == "Repeated" >
     // empty lists are represented by simply not starting a field, rather than starting one and putting in 0 elements
     if (reader.size() == 0) {
@@ -229,18 +241,26 @@ public abstract class ParquetOutputRecordWriter extends AbstractRecordWriter imp
       <#else>
 
       </#if>
-  <#elseif minor.class == "VarChar" || minor.class == "Var16Char" || minor.class == "VarBinary">
+  <#elseif minor.class == "VarChar" || minor.class == "Var16Char"
+        || minor.class == "VarBinary" || minor.class == "VarDecimal">
     <#if mode.prefix == "Repeated">
       reader.read(i, holder);
-      //consumer.startField(fieldName, fieldId);
+      <#if minor.class == "VarDecimal">
+      decimalValueWriter.writeValue(consumer, holder.buffer,
+          holder.start, holder.end, reader.getField().getPrecision());
+      <#else>
       consumer.addBinary(Binary.fromByteBuffer(holder.buffer.nioBuffer(holder.start, holder.end - holder.start)));
-      //consumer.endField(fieldName, fieldId);
+      </#if>
     <#else>
-    reader.read(holder);
-    DrillBuf buf = holder.buffer;
-    consumer.startField(fieldName, fieldId);
-    consumer.addBinary(Binary.fromByteBuffer(holder.buffer.nioBuffer(holder.start, holder.end - holder.start)));
-    consumer.endField(fieldName, fieldId);
+      reader.read(holder);
+      consumer.startField(fieldName, fieldId);
+      <#if minor.class == "VarDecimal">
+      decimalValueWriter.writeValue(consumer, holder.buffer,
+          holder.start, holder.end, reader.getField().getPrecision());
+      <#else>
+      consumer.addBinary(Binary.fromByteBuffer(holder.buffer.nioBuffer(holder.start, holder.end - holder.start)));
+      </#if>
+      consumer.endField(fieldName, fieldId);
     </#if>
   </#if>
   <#if mode.prefix == "Repeated">
diff --git a/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java b/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
index c4166f9..d2f27cc 100644
--- a/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
+++ b/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
@@ -15,11 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-import org.apache.drill.common.types.MinorType;
-import org.apache.parquet.format.ConvertedType;
-import org.apache.parquet.schema.DecimalMetadata;
-import org.apache.parquet.schema.OriginalType;
-import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
 
 <@pp.dropOutputFile />
 <@pp.changeOutputFile name="org/apache/drill/exec/store/parquet/ParquetTypeHelper.java" />
@@ -29,6 +24,7 @@ package org.apache.drill.exec.store.parquet;
 
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.planner.types.DrillRelDataTypeSystem;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.parquet.schema.OriginalType;
 import org.apache.parquet.schema.DecimalMetadata;
@@ -85,7 +81,8 @@ public class ParquetTypeHelper {
             minor.class == "Decimal28Dense" ||
             minor.class == "Decimal38Dense" ||
             minor.class == "Decimal28Sparse" ||
-            minor.class == "Decimal38Sparse">
+            minor.class == "Decimal38Sparse" ||
+            minor.class == "VarDecimal">
                     typeMap.put(MinorType.${minor.class?upper_case}, PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY);
     <#elseif
             minor.class == "VarChar" ||
@@ -106,8 +103,8 @@ public class ParquetTypeHelper {
 
     <#list vv.types as type>
     <#list type.minor as minor>
-            <#if minor.class.startsWith("Decimal")>
-            originalTypeMap.put(MinorType.${minor.class?upper_case},OriginalType.DECIMAL);
+            <#if minor.class.contains("Decimal")>
+            originalTypeMap.put(MinorType.${minor.class?upper_case}, OriginalType.DECIMAL);
             </#if>
     </#list>
     </#list>
@@ -157,10 +154,26 @@ public class ParquetTypeHelper {
       case DECIMAL28SPARSE:
         return 12;
       case DECIMAL38SPARSE:
+      case VARDECIMAL:
         return 16;
       default:
         return 0;
     }
   }
 
+  public static int getMaxPrecisionForPrimitiveType(PrimitiveTypeName type) {
+    switch(type) {
+      case INT32:
+        return 9;
+      case INT64:
+        return 18;
+      case FIXED_LEN_BYTE_ARRAY:
+        return DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM.getMaxNumericPrecision();
+      default:
+        throw new UnsupportedOperationException(String.format(
+          "Specified PrimitiveTypeName %s cannot be used to determine max precision",
+          type));
+    }
+  }
+
 }
diff --git a/exec/java-exec/src/main/codegen/templates/SqlAccessors.java b/exec/java-exec/src/main/codegen/templates/SqlAccessors.java
index fe650ac..7e5e99b 100644
--- a/exec/java-exec/src/main/codegen/templates/SqlAccessors.java
+++ b/exec/java-exec/src/main/codegen/templates/SqlAccessors.java
@@ -129,19 +129,24 @@ public class ${name}Accessor extends AbstractSqlAccessor {
     <#case "VarDecimal">
 
     @Override
-    public String getString(int index){
-<#if mode=="Nullable">
-        if(ac.isNull(index)){
-          return null;
-        }
-</#if>
-        try {
-          BigDecimal bd=getBigDecimal(index);
-          return bd.toString();
-        } catch (Throwable t) {
-          // TODO: we cannot throw the exception, so return its string representation, for now.  This should be handled better.
-          return t.toString();
-        }
+    public String getString(int index) {
+      <#if mode == "Nullable">
+      if (ac.isNull(index)) {
+        return null;
+      }
+      </#if>
+      BigDecimal bd = getBigDecimal(index);
+      return bd.toString();
+    }
+
+    @Override
+    public BigDecimal getBigDecimal(int index) {
+    <#if mode == "Nullable">
+      if (ac.isNull(index)) {
+        return null;
+      }
+    </#if>
+      return ac.getObject(index);
     }
       <#break>
 
diff --git a/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java b/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
index c0930f3..a5afce9 100644
--- a/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
@@ -35,6 +35,7 @@ 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.FunctionTemplate.ReturnType;
 import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.annotations.Workspace;
@@ -56,7 +57,11 @@ public class ${aggrtype.className}VarBytesFunctions {
 <#list aggrtype.types as type>
 <#if type.major == "VarBytes">
 
-@FunctionTemplate(name = "${aggrtype.funcName}", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+@FunctionTemplate(name = "${aggrtype.funcName}",
+                  <#if type.inputType.contains("VarChar")>
+                  returnType = ReturnType.SAME_IN_OUT_LENGTH,
+                  </#if>
+                  scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
 public static class ${type.inputType}${aggrtype.className} implements DrillAggFunc{
 
   @Param ${type.inputType}Holder in;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 42fd7e0..671ce4b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -243,6 +243,16 @@ public final class ExecConstants {
   public static final OptionValidator PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING_VALIDATOR = new BooleanValidator(
       PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING);
 
+  public static final String PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS
+      = "store.parquet.writer.use_primitive_types_for_decimals";
+  public static final OptionValidator PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS_VALIDATOR = new BooleanValidator(
+    PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS);
+
+  public static final String PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS
+      = "store.parquet.writer.logical_type_for_decimals";
+  public static final OptionValidator PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS_VALIDATOR
+      = new EnumeratedStringValidator(PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS, "fixed_len_byte_array", "binary");
+
   public static final String PARQUET_VECTOR_FILL_THRESHOLD = "store.parquet.vector_fill_threshold";
   public static final OptionValidator PARQUET_VECTOR_FILL_THRESHOLD_VALIDATOR = new PositiveLongValidator(PARQUET_VECTOR_FILL_THRESHOLD, 99l);
   public static final String PARQUET_VECTOR_FILL_CHECK_THRESHOLD = "store.parquet.vector_fill_check_threshold";
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ScalarReplacementTypes.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ScalarReplacementTypes.java
index 001762f..e1cc745 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ScalarReplacementTypes.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ScalarReplacementTypes.java
@@ -51,12 +51,14 @@ import org.apache.drill.exec.expr.holders.NullableTimeHolder;
 import org.apache.drill.exec.expr.holders.NullableTimeStampHolder;
 import org.apache.drill.exec.expr.holders.NullableVarBinaryHolder;
 import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
+import org.apache.drill.exec.expr.holders.NullableVarDecimalHolder;
 import org.apache.drill.exec.expr.holders.TimeHolder;
 import org.apache.drill.exec.expr.holders.TimeStampHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 
 import com.google.common.collect.ImmutableSet;
+import org.apache.drill.exec.expr.holders.VarDecimalHolder;
 
 /**
  * Reference list of classes we will perform scalar replacement on.
@@ -87,6 +89,7 @@ public class ScalarReplacementTypes {
         TimeStampHolder.class,
         VarCharHolder.class,
         VarBinaryHolder.class,
+        VarDecimalHolder.class,
         NullableBitHolder.class,
         NullableIntHolder.class,
         NullableBigIntHolder.class,
@@ -94,6 +97,7 @@ public class ScalarReplacementTypes {
         NullableFloat8Holder.class,
         NullableVarCharHolder.class,
         NullableVarBinaryHolder.class,
+        NullableVarDecimalHolder.class,
         NullableDecimal9Holder.class,
         NullableDecimal18Holder.class,
         NullableDecimal28SparseHolder.class,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
index 1a0b7d5..d764663 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
@@ -47,6 +47,7 @@ import org.apache.drill.common.expression.ValueExpressions.LongExpression;
 import org.apache.drill.common.expression.ValueExpressions.QuotedString;
 import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
 import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
+import org.apache.drill.common.expression.ValueExpressions.VarDecimalExpression;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
 
 import com.google.common.collect.Lists;
@@ -187,6 +188,11 @@ public class ConstantExpressionIdentifier implements ExprVisitor<Boolean, Identi
   }
 
   @Override
+  public Boolean visitVarDecimalConstant(VarDecimalExpression decExpr, IdentityHashMap<LogicalExpression, Object> value) {
+    return true;
+  }
+
+  @Override
   public Boolean visitDoubleConstant(DoubleExpression dExpr, IdentityHashMap<LogicalExpression, Object> value){
     return true;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CloneVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CloneVisitor.java
index 84d1203..9c2e394 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CloneVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CloneVisitor.java
@@ -44,6 +44,7 @@ import org.apache.drill.common.expression.ValueExpressions.LongExpression;
 import org.apache.drill.common.expression.ValueExpressions.QuotedString;
 import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
 import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
+import org.apache.drill.common.expression.ValueExpressions.VarDecimalExpression;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
 import org.apache.drill.exec.expr.fn.DrillFuncHolder;
 
@@ -131,6 +132,11 @@ public class CloneVisitor extends AbstractExprVisitor<LogicalExpression,Void,Run
   }
 
   @Override
+  public LogicalExpression visitVarDecimalConstant(VarDecimalExpression decExpr, Void value) throws RuntimeException {
+    return visitUnknown(decExpr, value);
+  }
+
+  @Override
   public LogicalExpression visitDateConstant(DateExpression intExpr, Void value) throws RuntimeException {
     return visitUnknown(intExpr, value);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EqualityVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EqualityVisitor.java
index 3f322b8..94f8b79 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EqualityVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EqualityVisitor.java
@@ -43,6 +43,7 @@ import org.apache.drill.common.expression.ValueExpressions.LongExpression;
 import org.apache.drill.common.expression.ValueExpressions.QuotedString;
 import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
 import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
+import org.apache.drill.common.expression.ValueExpressions.VarDecimalExpression;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
 
 import java.util.List;
@@ -228,7 +229,21 @@ class EqualityVisitor extends AbstractExprVisitor<Boolean,LogicalExpression,Runt
     if (!decExpr.getMajorType().equals(((Decimal38Expression) value).getMajorType())) {
       return false;
     }
-    return false;
+    return true;
+  }
+
+  @Override
+  public Boolean visitVarDecimalConstant(VarDecimalExpression decExpr, LogicalExpression value) throws RuntimeException {
+    if (!(value instanceof VarDecimalExpression)) {
+      return false;
+    }
+    if (!decExpr.getMajorType().equals(value.getMajorType())) {
+      return false;
+    }
+    if (!decExpr.getBigDecimal().equals(((VarDecimalExpression) value).getBigDecimal())) {
+      return false;
+    }
+    return true;
   }
 
   @Override
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 dcc2668..64cfe66 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
@@ -53,6 +53,7 @@ import org.apache.drill.common.expression.ValueExpressions.LongExpression;
 import org.apache.drill.common.expression.ValueExpressions.QuotedString;
 import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
 import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
+import org.apache.drill.common.expression.ValueExpressions.VarDecimalExpression;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
@@ -657,8 +658,10 @@ public class EvaluationVisitor {
       JType holderType = generator.getHolderType(majorType);
       JVar var = generator.declareClassField("dec28", holderType);
       JExpression stringLiteral = JExpr.lit(e.getBigDecimal().toString());
+      JExpression buffer = generator.getMappingSet().getIncoming().invoke("getContext").invoke("getManagedBuffer");
       setup.assign(var,
-          generator.getModel().ref(ValueHolderHelper.class).staticInvoke("getDecimal28Holder").arg(stringLiteral));
+          generator.getModel().ref(ValueHolderHelper.class).staticInvoke("getDecimal28Holder")
+              .arg(buffer).arg(stringLiteral));
       return new HoldingContainer(majorType, var, null, null);
     }
 
@@ -670,8 +673,25 @@ public class EvaluationVisitor {
       JType holderType = generator.getHolderType(majorType);
       JVar var = generator.declareClassField("dec38", holderType);
       JExpression stringLiteral = JExpr.lit(e.getBigDecimal().toString());
+      JExpression buffer = generator.getMappingSet().getIncoming().invoke("getContext").invoke("getManagedBuffer");
+      setup.assign(var,
+          generator.getModel().ref(ValueHolderHelper.class).staticInvoke("getDecimal38Holder")
+              .arg(buffer).arg(stringLiteral));
+      return new HoldingContainer(majorType, var, null, null);
+    }
+
+    @Override
+    public HoldingContainer visitVarDecimalConstant(VarDecimalExpression e, ClassGenerator<?> generator)
+        throws RuntimeException {
+      MajorType majorType = e.getMajorType();
+      JBlock setup = generator.getBlock(BlockType.SETUP);
+      JType holderType = generator.getHolderType(majorType);
+      JVar var = generator.declareClassField("varDec", holderType);
+      JExpression stringLiteral = JExpr.lit(e.getBigDecimal().toString());
+      JExpression buffer = generator.getMappingSet().getIncoming().invoke("getContext").invoke("getManagedBuffer");
       setup.assign(var,
-          generator.getModel().ref(ValueHolderHelper.class).staticInvoke("getVarCharHolder").arg(stringLiteral));
+          generator.getModel().ref(ValueHolderHelper.class).staticInvoke("getVarDecimalHolder")
+              .arg(buffer).arg(stringLiteral));
       return new HoldingContainer(majorType, var, null, null);
     }
 
@@ -997,6 +1017,16 @@ public class EvaluationVisitor {
     }
 
     @Override
+    public HoldingContainer visitVarDecimalConstant(VarDecimalExpression decExpr, ClassGenerator<?> generator) throws RuntimeException {
+      HoldingContainer hc = getPrevious(decExpr, generator.getMappingSet());
+      if (hc == null) {
+        hc = super.visitVarDecimalConstant(decExpr, generator);
+        put(decExpr, hc, generator.getMappingSet());
+      }
+      return hc;
+    }
+
+    @Override
     public HoldingContainer visitDoubleConstant(DoubleExpression dExpr, ClassGenerator<?> generator) throws RuntimeException {
       HoldingContainer hc = getPrevious(dExpr, generator.getMappingSet());
       if (hc == null) {
@@ -1221,6 +1251,20 @@ public class EvaluationVisitor {
     }
 
     @Override
+    public HoldingContainer visitVarDecimalConstant(VarDecimalExpression e, ClassGenerator<?> generator)
+        throws RuntimeException {
+      if (constantBoundaries.contains(e)) {
+        generator.getMappingSet().enterConstant();
+        HoldingContainer c = super.visitVarDecimalConstant(e, generator);
+        return renderConstantExpression(generator, c);
+      } else if (generator.getMappingSet().isWithinConstant()) {
+        return super.visitVarDecimalConstant(e, generator).setConstant(true);
+      } else {
+        return super.visitVarDecimalConstant(e, generator);
+      }
+    }
+
+    @Override
     public HoldingContainer visitIntConstant(IntExpression e, ClassGenerator<?> generator) throws RuntimeException {
       if (constantBoundaries.contains(e)) {
         generator.getMappingSet().enterConstant();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
index 6e8e315..ff38e0c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
@@ -62,6 +62,7 @@ import org.apache.drill.common.expression.ValueExpressions.LongExpression;
 import org.apache.drill.common.expression.ValueExpressions.QuotedString;
 import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
 import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
+import org.apache.drill.common.expression.ValueExpressions.VarDecimalExpression;
 import org.apache.drill.common.expression.fn.CastFunctions;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
 import org.apache.drill.common.expression.visitors.ConditionalExprOptimizer;
@@ -210,7 +211,11 @@ public class ExpressionTreeMaterializer {
       return fromExpr;
     }
 
-    if (!Types.isFixedWidthType(toType) && !Types.isUnion(toType)) {
+    if (CoreDecimalUtility.isDecimalType(toType)) {
+      // Add the scale and precision to the arguments of the implicit cast
+      castArgs.add(new ValueExpressions.IntExpression(toType.getPrecision(), null));
+      castArgs.add(new ValueExpressions.IntExpression(toType.getScale(), null));
+    } else if (!Types.isFixedWidthType(toType) && !Types.isUnion(toType)) {
 
       /* We are implicitly casting to VARCHAR so we don't have a max length,
        * using an arbitrary value. We trim down the size of the stored bytes
@@ -218,11 +223,7 @@ public class ExpressionTreeMaterializer {
        */
       castArgs.add(new ValueExpressions.LongExpression(Types.MAX_VARCHAR_LENGTH, null));
     }
-    else if (CoreDecimalUtility.isDecimalType(toType)) {
-      // Add the scale and precision to the arguments of the implicit cast
-      castArgs.add(new ValueExpressions.LongExpression(toType.getPrecision(), null));
-      castArgs.add(new ValueExpressions.LongExpression(toType.getScale(), null));
-    }
+
     FunctionCall castCall = new FunctionCall(castFuncName, castArgs, ExpressionPosition.UNKNOWN);
     FunctionResolver resolver;
     if (exactResolver) {
@@ -381,14 +382,8 @@ public class ExpressionTreeMaterializer {
     }
 
     private int computePrecision(LogicalExpression currentArg) {
-        int precision = currentArg.getMajorType().getPrecision();
-        if (currentArg.getMajorType().getMinorType() == MinorType.INT) {
-            precision = DecimalUtility.MAX_DIGITS_INT;
-        }
-        else if (currentArg.getMajorType().getMinorType() == MinorType.BIGINT) {
-            precision = DecimalUtility.MAX_DIGITS_BIGINT;
-        }
-        return precision;
+      int precision = currentArg.getMajorType().getPrecision();
+      return DecimalUtility.getDefaultPrecision(currentArg.getMajorType().getMinorType(), precision);
     }
 
     @Override
@@ -444,7 +439,19 @@ public class ExpressionTreeMaterializer {
           }
         }
 
-        return matchedFuncHolder.getExpr(call.getName(), argsWithCast, call.getPosition());
+        FunctionHolderExpression funcExpr = matchedFuncHolder.getExpr(call.getName(), argsWithCast, call.getPosition());
+        MajorType funcExprMajorType = funcExpr.getMajorType();
+        if (DecimalUtility.isObsoleteDecimalType(funcExprMajorType.getMinorType())) {
+          MajorType majorType =
+              MajorType.newBuilder()
+                  .setMinorType(MinorType.VARDECIMAL)
+                  .setMode(funcExprMajorType.getMode())
+                  .setScale(funcExprMajorType.getScale())
+                  .setPrecision(funcExprMajorType.getPrecision())
+                  .build();
+          return addCastExpression(funcExpr, majorType, functionLookupContext, errorCollector);
+        }
+        return funcExpr;
       }
 
       // as no drill func is found, search for a non-Drill function.
@@ -813,6 +820,11 @@ public class ExpressionTreeMaterializer {
     }
 
     @Override
+    public LogicalExpression visitVarDecimalConstant(VarDecimalExpression decExpr, FunctionLookupContext functionLookupContext) {
+      return decExpr;
+    }
+
+    @Override
     public LogicalExpression visitDoubleConstant(DoubleExpression dExpr, FunctionLookupContext functionLookupContext) {
       return dExpr;
     }
@@ -868,12 +880,11 @@ public class ExpressionTreeMaterializer {
         List<LogicalExpression> newArgs = Lists.newArrayList();
         newArgs.add(input);  //input_expr
 
-        //VarLen type
-        if (!Types.isFixedWidthType(type)) {
+        if (CoreDecimalUtility.isDecimalType(type)) {
+          newArgs.add(new ValueExpressions.IntExpression(type.getPrecision(), null));
+          newArgs.add(new ValueExpressions.IntExpression(type.getScale(), null));
+        } else if (!Types.isFixedWidthType(type)) { //VarLen type
           newArgs.add(new ValueExpressions.LongExpression(type.getPrecision(), null));
-        }  if (CoreDecimalUtility.isDecimalType(type)) {
-            newArgs.add(new ValueExpressions.LongExpression(type.getPrecision(), null));
-            newArgs.add(new ValueExpressions.LongExpression(type.getScale(), null));
         }
 
         FunctionCall fc = new FunctionCall(castFuncWithType, newArgs, e.getPosition());
@@ -912,17 +923,8 @@ public class ExpressionTreeMaterializer {
       case DECIMAL28SPARSE:
       case DECIMAL38DENSE:
       case DECIMAL38SPARSE:
-        if (to.getScale() == from.getScale() && to.getPrecision() == from.getPrecision()) {
-          return true;
-        }
-        return false;
-
       case VARDECIMAL:
-          // for VARDECIMAL, precision does not matter (it's variable precision, to fit the number), but scale matters
-          if (to.getScale() == from.getScale()) {
-              return true;
-          }
-          return false;
+        return to.getScale() == from.getScale() && to.getPrecision() == from.getPrecision();
 
       case FIXED16CHAR:
       case FIXEDBINARY:
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/GetSetVectorHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/GetSetVectorHelper.java
index 8ceb3d9..3b7e2b4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/GetSetVectorHelper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/GetSetVectorHelper.java
@@ -87,22 +87,23 @@ public class GetSetVectorHelper {
         eval.assign(out.getHolder().ref("start"), JExpr.cast(model._ref(int.class), se));
         eval.assign(out.getHolder().ref("end"), JExpr.cast(model._ref(int.class), se.shr(JExpr.lit(32))));
         eval.assign(out.getHolder().ref("scale"), vector.invoke("getField").invoke("getScale"));
+        eval.assign(out.getHolder().ref("precision"), vector.invoke("getField").invoke("getPrecision"));
         return;
       }
-      case INTERVAL:{
+      case INTERVAL: {
         JVar start = eval.decl(model.INT, "start", JExpr.lit(TypeHelper.getSize(type)).mul(indexVariable));
         JVar data = eval.decl(model.ref(DrillBuf.class), "data", vector.invoke("getBuffer"));
         eval.assign(out.getHolder().ref("months"), data.invoke("getInt").arg(start));
         eval.assign(out.getHolder().ref("days"), data.invoke("getInt").arg(start.plus(JExpr.lit(4))));
         eval.assign(out.getHolder().ref("milliseconds"), data.invoke("getInt").arg(start.plus(JExpr.lit(8))));
         return;
-        }
+      }
       case INTERVALDAY: {
         JVar start = eval.decl(model.INT, "start", JExpr.lit(TypeHelper.getSize(type)).mul(indexVariable));
         eval.assign(out.getHolder().ref("days"), vector.invoke("getBuffer").invoke("getInt").arg(start));
         eval.assign(out.getHolder().ref("milliseconds"), vector.invoke("getBuffer").invoke("getInt").arg(start.plus(JExpr.lit(4))));
         return;
-        }
+      }
       case VAR16CHAR:
       case VARBINARY:
       case VARCHAR: {
@@ -111,7 +112,7 @@ public class GetSetVectorHelper {
          eval.assign(out.getHolder().ref("start"), JExpr.cast(model._ref(int.class), se));
          eval.assign(out.getHolder().ref("end"), JExpr.cast(model._ref(int.class), se.shr(JExpr.lit(32))));
         return;
-        }
+      }
       }
     }
 
@@ -149,38 +150,37 @@ public class GetSetVectorHelper {
       case BIT:
       case DECIMAL9:
       case DECIMAL18:
-        return setMethod //
+        return setMethod
             .arg(in.getValue());
       case DECIMAL28DENSE:
       case DECIMAL28SPARSE:
       case DECIMAL38DENSE:
       case DECIMAL38SPARSE:
-        return setMethod //
-            .arg(in.f("start")) //
+        return setMethod
+            .arg(in.f("start"))
             .arg(in.f("buffer"));
       case INTERVAL:{
-        return setMethod //
-            .arg(in.f("months")) //
-            .arg(in.f("days")) //
+        return setMethod
+            .arg(in.f("months"))
+            .arg(in.f("days"))
             .arg(in.f("milliseconds"));
       }
       case INTERVALDAY: {
-        return setMethod //
-            .arg(in.f("days")) //
+        return setMethod
+            .arg(in.f("days"))
             .arg(in.f("milliseconds"));
       }
       case VAR16CHAR:
       case VARBINARY:
       case VARCHAR:
       case VARDECIMAL:
-        return setMethod //
-            .arg(in.f("start")) //
-            .arg(in.f("end")) //
+        return setMethod
+            .arg(in.f("start"))
+            .arg(in.f("end"))
             .arg(in.f("buffer"));
       }
     }
 
-
     return setMethod.arg(in.getHolder());
 
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/HashVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/HashVisitor.java
index 53fbaa3..87d6688 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/HashVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/HashVisitor.java
@@ -17,14 +17,12 @@
  */
 package org.apache.drill.exec.expr;
 
-import com.google.common.collect.Lists;
 import org.apache.drill.common.expression.BooleanOperator;
 import org.apache.drill.common.expression.CastExpression;
 import org.apache.drill.common.expression.ConvertExpression;
 import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.FunctionHolderExpression;
 import org.apache.drill.common.expression.IfExpression;
-import org.apache.drill.common.expression.IfExpression.IfCondition;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.NullExpression;
 import org.apache.drill.common.expression.SchemaPath;
@@ -44,10 +42,8 @@ import org.apache.drill.common.expression.ValueExpressions.LongExpression;
 import org.apache.drill.common.expression.ValueExpressions.QuotedString;
 import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
 import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
+import org.apache.drill.common.expression.ValueExpressions.VarDecimalExpression;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
-import org.apache.drill.exec.expr.fn.DrillFuncHolder;
-
-import java.util.List;
 
 public class HashVisitor extends AbstractExprVisitor<Integer,Void,RuntimeException> {
   @Override
@@ -176,6 +172,11 @@ public class HashVisitor extends AbstractExprVisitor<Integer,Void,RuntimeExcepti
     return compute(e, 25);
   }
 
+  @Override
+  public Integer visitVarDecimalConstant(VarDecimalExpression decExpr, Void value) throws RuntimeException {
+    return compute(decExpr, 26);
+  }
+
   private int compute(LogicalExpression e, int seed) {
     int hash = seed;
     for (LogicalExpression child : e) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
index ac4ba07..1a5470b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java
@@ -20,7 +20,6 @@ package org.apache.drill.exec.expr.annotations;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.expr.fn.FunctionAttributes;
-import org.apache.drill.exec.expr.fn.FunctionInitializer;
 import org.apache.drill.exec.expr.fn.output.ConcatReturnTypeInference;
 import org.apache.drill.exec.expr.fn.output.DecimalReturnTypeInference;
 import org.apache.drill.exec.expr.fn.output.DefaultReturnTypeInference;
@@ -89,7 +88,7 @@ public @interface FunctionTemplate {
   boolean isNiladic() default false;
   boolean checkPrecisionRange() default false;
 
-  public enum NullHandling {
+  enum NullHandling {
     /**
      * Method handles nulls.
      */
@@ -108,7 +107,7 @@ public @interface FunctionTemplate {
    * Function scope is used to indicate function output rows relation:
    * simple / scalar (1 -> 1) or aggregate (n -> 1).
    */
-  public enum FunctionScope {
+  enum FunctionScope {
     SIMPLE,
     POINT_AGGREGATE,
     HOLISTIC_AGGREGATE,
@@ -119,7 +118,7 @@ public @interface FunctionTemplate {
    * Return type enum is used to indicate which return type calculation logic
    * should be used for functions.
    */
-  public enum ReturnType {
+  enum ReturnType {
     DEFAULT(DefaultReturnTypeInference.INSTANCE),
 
     STRING_CAST(StringCastReturnTypeInference.INSTANCE),
@@ -129,6 +128,7 @@ public @interface FunctionTemplate {
 
     DECIMAL_AGGREGATE(DecimalReturnTypeInference.DecimalAggReturnTypeInference.INSTANCE),
     DECIMAL_SUM_AGGREGATE(DecimalReturnTypeInference.DecimalSumAggReturnTypeInference.INSTANCE),
+    DECIMAL_AVG_AGGREGATE(DecimalReturnTypeInference.DecimalAvgAggReturnTypeInference.INSTANCE),
     DECIMAL_MAX_SCALE(DecimalReturnTypeInference.DecimalMaxScaleReturnTypeInference.INSTANCE),
     DECIMAL_SUM_SCALE(DecimalReturnTypeInference.DecimalSumScaleReturnTypeInference.INSTANCE),
     DECIMAL_CAST(DecimalReturnTypeInference.DecimalCastReturnTypeInference.INSTANCE),
@@ -150,7 +150,7 @@ public @interface FunctionTemplate {
 
   }
 
-  public enum FunctionCostCategory {
+  enum FunctionCostCategory {
     SIMPLE(1), MEDIUM(20), COMPLEX(50);
 
     private final int value;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32AsDouble.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32AsDouble.java
index e134c9d..c9502fb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32AsDouble.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32AsDouble.java
@@ -23,20 +23,12 @@ import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
 import org.apache.drill.exec.expr.holders.BigIntHolder;
-import org.apache.drill.exec.expr.holders.Decimal18Holder;
-import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
 import org.apache.drill.exec.expr.holders.VarDecimalHolder;
 import org.apache.drill.exec.expr.holders.NullableVarDecimalHolder;
-import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.Decimal9Holder;
 import org.apache.drill.exec.expr.holders.Float4Holder;
 import org.apache.drill.exec.expr.holders.Float8Holder;
 import org.apache.drill.exec.expr.holders.IntHolder;
 import org.apache.drill.exec.expr.holders.NullableBigIntHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal18Holder;
-import org.apache.drill.exec.expr.holders.NullableDecimal28SparseHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal9Holder;
 import org.apache.drill.exec.expr.holders.NullableFloat4Holder;
 import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
 import org.apache.drill.exec.expr.holders.NullableIntHolder;
@@ -45,6 +37,7 @@ import org.apache.drill.exec.expr.holders.NullableIntHolder;
  * hash32 function definitions for numeric data types. These functions cast the input numeric value to a
  * double before doing the hashing. See comments in {@link Hash64AsDouble} for the reason for doing this.
  */
+@SuppressWarnings("unused")
 public class Hash32AsDouble {
   @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
 
@@ -134,7 +127,7 @@ public class Hash32AsDouble {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32((long) in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
       }
     }
   }
@@ -170,7 +163,7 @@ public class Hash32AsDouble {
     }
 
     public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32((long)in.value, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
     }
   }
 
@@ -190,78 +183,6 @@ public class Hash32AsDouble {
   }
 
   @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal9Hash implements DrillSimpleFunc {
-    @Param
-    Decimal9Holder in;
-    @Output
-    IntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), 0);
-    }
-  }
-
-  @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal9Hash implements DrillSimpleFunc {
-    @Param
-    NullableDecimal9Holder in;
-    @Output
-    IntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = 0;
-      } else {
-        java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), 0);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal18Hash implements DrillSimpleFunc {
-    @Param
-    Decimal18Holder in;
-    @Output
-    IntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), 0);
-    }
-  }
-
-  @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal18Hash implements DrillSimpleFunc {
-    @Param
-    NullableDecimal18Holder in;
-    @Output
-    IntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = 0;
-      } else {
-        java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), 0);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
   public static class VarDecimalHash implements DrillSimpleFunc {
     @Param
     VarDecimalHolder in;
@@ -298,80 +219,4 @@ public class Hash32AsDouble {
       }
     }
   }
-
-  @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal28Hash implements DrillSimpleFunc {
-    @Param
-    Decimal28SparseHolder in;
-    @Output
-    IntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
-          in.start, in.nDecimalDigits, in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), 0);
-    }
-  }
-
-  @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal28Hash implements DrillSimpleFunc {
-    @Param
-    NullableDecimal28SparseHolder in;
-    @Output
-    IntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = 0;
-      } else {
-        java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
-            in.start, in.nDecimalDigits, in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), 0);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal38Hash implements DrillSimpleFunc {
-    @Param
-    Decimal38SparseHolder in;
-    @Output
-    IntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
-          in.start, in.nDecimalDigits, in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), 0);
-    }
-  }
-
-  @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal38Hash implements DrillSimpleFunc {
-    @Param
-    NullableDecimal38SparseHolder in;
-    @Output
-    IntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = 0;
-      } else {
-        java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
-            in.start, in.nDecimalDigits, in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), 0);
-      }
-    }
-  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32Functions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32Functions.java
index a6d24f5..412674f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32Functions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32Functions.java
@@ -25,22 +25,14 @@ import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.BigIntHolder;
 import org.apache.drill.exec.expr.holders.BitHolder;
 import org.apache.drill.exec.expr.holders.DateHolder;
-import org.apache.drill.exec.expr.holders.Decimal18Holder;
-import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
 import org.apache.drill.exec.expr.holders.VarDecimalHolder;
 import org.apache.drill.exec.expr.holders.NullableVarDecimalHolder;
-import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.Decimal9Holder;
 import org.apache.drill.exec.expr.holders.Float4Holder;
 import org.apache.drill.exec.expr.holders.Float8Holder;
 import org.apache.drill.exec.expr.holders.IntHolder;
 import org.apache.drill.exec.expr.holders.NullableBigIntHolder;
 import org.apache.drill.exec.expr.holders.NullableBitHolder;
 import org.apache.drill.exec.expr.holders.NullableDateHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal18Holder;
-import org.apache.drill.exec.expr.holders.NullableDecimal28SparseHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal9Holder;
 import org.apache.drill.exec.expr.holders.NullableFloat4Holder;
 import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
 import org.apache.drill.exec.expr.holders.NullableIntHolder;
@@ -55,8 +47,7 @@ import org.apache.drill.exec.expr.holders.Var16CharHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 
-import java.math.BigDecimal;
-
+@SuppressWarnings("unused")
 public class Hash32Functions {
 
   @FunctionTemplate(names = {"hash", "hash32"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL )
@@ -373,66 +364,6 @@ public class Hash32Functions {
   }
 
   @FunctionTemplate(names = {"hash", "hash32"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal9Hash implements DrillSimpleFunc {
-    @Param  Decimal9Holder in;
-    @Output IntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
-    }
-  }
-
-  @FunctionTemplate(names = {"hash", "hash32"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal9Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal9Holder in;
-    @Output IntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = 0;
-      } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
-      }
-    }
-  }
-
-  @FunctionTemplate(names = {"hash", "hash32"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal18Hash implements DrillSimpleFunc {
-    @Param  Decimal18Holder in;
-    @Output IntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
-    }
-  }
-
-  @FunctionTemplate(names = {"hash", "hash32"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal18Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal18Holder in;
-    @Output IntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = 0;
-      } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, 0);
-      }
-    }
-  }
-
-  @FunctionTemplate(names = {"hash", "hash32"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
   public static class VarDecimalHash implements DrillSimpleFunc {
     @Param  VarDecimalHolder in;
     @Output IntHolder out;
@@ -441,28 +372,8 @@ public class Hash32Functions {
     }
 
     public void eval() {
-      java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer,
-              in.start, in.end - in.start, in.scale);
-      out.value = hashBigDecimal(bd, 0);
-    }
-  }
-
-  public static int hashBigDecimal(BigDecimal bd, int seed) {
-    java.math.BigInteger bi = bd.unscaledValue();
-    byte[] barr = bi.toByteArray();
-    int shiftCount = 0;
-    int xor = 0;
-    for (int i = 0; i < barr.length; ++i) {
-      byte b = barr[i];
-      int ival = (int) b;
-      ival <<= shiftCount;
-      ++shiftCount;
-      if (shiftCount > 3) {
-        shiftCount = 0;
-      }
-      xor = xor ^ ival;
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, 0);
     }
-    return org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(xor, seed);
   }
 
   @FunctionTemplate(names = {"hash", "hash32"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
@@ -477,87 +388,7 @@ public class Hash32Functions {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer,
-                in.start, in.end - in.start, in.scale);
-        out.value = hashBigDecimal(bd, 0);
-      }
-    }
-  }
-
-  @FunctionTemplate(names = {"hash", "hash32"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal28Hash implements DrillSimpleFunc {
-    @Param  Decimal28SparseHolder in;
-    @Output IntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-
-      int xor = 0;
-      for (int i = 0; i < in.nDecimalDigits; i++) {
-        xor = xor ^ Decimal28SparseHolder.getInteger(i, in.start, in.buffer);
-      }
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(xor, 0);
-    }
-  }
-
-  @FunctionTemplate(names = {"hash", "hash32"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal28Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal28SparseHolder in;
-    @Output IntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = 0;
-      } else {
-        int xor = 0;
-        for (int i = 0; i < in.nDecimalDigits; i++) {
-          xor = xor ^ NullableDecimal28SparseHolder.getInteger(i, in.start, in.buffer);
-        }
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(xor, 0);
-      }
-    }
-  }
-
-  @FunctionTemplate(names = {"hash", "hash32"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal38Hash implements DrillSimpleFunc {
-    @Param  Decimal38SparseHolder in;
-    @Output IntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-
-      int xor = 0;
-      for (int i = 0; i < in.nDecimalDigits; i++) {
-        xor = xor ^ Decimal38SparseHolder.getInteger(i, in.start, in.buffer);
-      }
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(xor, 0);
-    }
-  }
-
-  @FunctionTemplate(names = {"hash", "hash32"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal38Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal38SparseHolder in;
-    @Output IntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = 0;
-      } else {
-        int xor = 0;
-        for (int i = 0; i < in.nDecimalDigits; i++) {
-          xor = xor ^ NullableDecimal38SparseHolder.getInteger(i, in.start, in.buffer);
-        }
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(xor, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, 0);
       }
     }
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32FunctionsWithSeed.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32FunctionsWithSeed.java
index cda47ff..1741eb8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32FunctionsWithSeed.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32FunctionsWithSeed.java
@@ -25,22 +25,14 @@ import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.BigIntHolder;
 import org.apache.drill.exec.expr.holders.BitHolder;
 import org.apache.drill.exec.expr.holders.DateHolder;
-import org.apache.drill.exec.expr.holders.Decimal18Holder;
-import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
 import org.apache.drill.exec.expr.holders.VarDecimalHolder;
 import org.apache.drill.exec.expr.holders.NullableVarDecimalHolder;
-import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.Decimal9Holder;
 import org.apache.drill.exec.expr.holders.Float4Holder;
 import org.apache.drill.exec.expr.holders.Float8Holder;
 import org.apache.drill.exec.expr.holders.IntHolder;
 import org.apache.drill.exec.expr.holders.NullableBigIntHolder;
 import org.apache.drill.exec.expr.holders.NullableBitHolder;
 import org.apache.drill.exec.expr.holders.NullableDateHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal18Holder;
-import org.apache.drill.exec.expr.holders.NullableDecimal28SparseHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal9Holder;
 import org.apache.drill.exec.expr.holders.NullableFloat4Holder;
 import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
 import org.apache.drill.exec.expr.holders.NullableIntHolder;
@@ -58,6 +50,7 @@ import org.apache.drill.exec.expr.holders.VarCharHolder;
 /*
  * Class contains hash32 function definitions for different data types.
  */
+@SuppressWarnings("unused")
 public class Hash32FunctionsWithSeed {
   @FunctionTemplate(name = "hash32", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL )
   public static class NullableFloatHash implements DrillSimpleFunc {
@@ -414,74 +407,6 @@ public class Hash32FunctionsWithSeed {
   }
 
   @FunctionTemplate(name = "hash32", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal9Hash implements DrillSimpleFunc {
-    @Param  Decimal9Holder in;
-    @Param IntHolder seed;
-    @Output IntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
-    }
-  }
-
-  @FunctionTemplate(name = "hash32", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal9Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal9Holder in;
-    @Param IntHolder seed;
-    @Output IntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = seed.value;
-      } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash32", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal18Hash implements DrillSimpleFunc {
-    @Param  Decimal18Holder in;
-    @Param IntHolder seed;
-    @Output IntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
-    }
-  }
-
-  @FunctionTemplate(name = "hash32", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal18Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal18Holder in;
-    @Param IntHolder seed;
-    @Output IntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = seed.value;
-      } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.value, seed.value);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash32", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
   public static class VarDecimalHash implements DrillSimpleFunc {
     @Param  VarDecimalHolder in;
     @Param IntHolder seed;
@@ -491,9 +416,7 @@ public class Hash32FunctionsWithSeed {
     }
 
     public void eval() {
-      java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer,
-              in.start, in.end - in.start, in.scale);
-      out.value = Hash32Functions.hashBigDecimal(bd, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, seed.value);
     }
   }
 
@@ -510,77 +433,7 @@ public class Hash32FunctionsWithSeed {
       if (in.isSet == 0) {
         out.value = seed.value;
       } else {
-        java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer,
-                in.start, in.end - in.start, in.scale);
-        out.value = Hash32Functions.hashBigDecimal(bd, seed.value);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash32", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal28Hash implements DrillSimpleFunc {
-    @Param  Decimal28SparseHolder in;
-    @Param IntHolder seed;
-    @Output IntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.start + Decimal28SparseHolder.WIDTH, in.buffer, seed.value);
-    }
-  }
-
-  @FunctionTemplate(name = "hash32", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal28Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal28SparseHolder in;
-    @Param IntHolder seed;
-    @Output IntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = seed.value;
-      } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.start + NullableDecimal28SparseHolder.WIDTH, in.buffer, seed.value);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash32", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal38Hash implements DrillSimpleFunc {
-    @Param  Decimal38SparseHolder in;
-    @Param IntHolder seed;
-    @Output IntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.start + Decimal38SparseHolder.WIDTH, in.buffer, seed.value);
-    }
-  }
-
-  @FunctionTemplate(name = "hash32", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal38Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal38SparseHolder in;
-    @Param IntHolder seed;
-    @Output IntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = seed.value;
-      } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.start + NullableDecimal38SparseHolder.WIDTH, in.buffer, seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, seed.value);
       }
     }
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32WithSeedAsDouble.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32WithSeedAsDouble.java
index 1402935..67bbfa7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32WithSeedAsDouble.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash32WithSeedAsDouble.java
@@ -23,42 +23,21 @@ 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.holders.BigIntHolder;
-import org.apache.drill.exec.expr.holders.BitHolder;
-import org.apache.drill.exec.expr.holders.DateHolder;
-import org.apache.drill.exec.expr.holders.Decimal18Holder;
-import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
 import org.apache.drill.exec.expr.holders.VarDecimalHolder;
 import org.apache.drill.exec.expr.holders.NullableVarDecimalHolder;
-import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.Decimal9Holder;
 import org.apache.drill.exec.expr.holders.Float4Holder;
 import org.apache.drill.exec.expr.holders.Float8Holder;
 import org.apache.drill.exec.expr.holders.IntHolder;
 import org.apache.drill.exec.expr.holders.NullableBigIntHolder;
-import org.apache.drill.exec.expr.holders.NullableBitHolder;
-import org.apache.drill.exec.expr.holders.NullableDateHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal18Holder;
-import org.apache.drill.exec.expr.holders.NullableDecimal28SparseHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal9Holder;
 import org.apache.drill.exec.expr.holders.NullableFloat4Holder;
 import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
 import org.apache.drill.exec.expr.holders.NullableIntHolder;
-import org.apache.drill.exec.expr.holders.NullableTimeHolder;
-import org.apache.drill.exec.expr.holders.NullableTimeStampHolder;
-import org.apache.drill.exec.expr.holders.NullableVar16CharHolder;
-import org.apache.drill.exec.expr.holders.NullableVarBinaryHolder;
-import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
-import org.apache.drill.exec.expr.holders.TimeHolder;
-import org.apache.drill.exec.expr.holders.TimeStampHolder;
-import org.apache.drill.exec.expr.holders.Var16CharHolder;
-import org.apache.drill.exec.expr.holders.VarBinaryHolder;
-import org.apache.drill.exec.expr.holders.VarCharHolder;
 
 /**
  * hash32 with seed function definitions for numeric data types. These functions cast the input numeric value to a
  * double before doing the hashing. See comments in {@link Hash64AsDouble} for the reason for doing this.
  */
+@SuppressWarnings("unused")
 public class Hash32WithSeedAsDouble {
   @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL )
   public static class NullableFloatHash implements DrillSimpleFunc {
@@ -206,81 +185,8 @@ public class Hash32WithSeedAsDouble {
   }
 
   @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal9Hash implements DrillSimpleFunc {
-    @Param  Decimal9Holder in;
-    @Param IntHolder seed;
-    @Output IntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), seed.value);
-    }
-  }
-
-  @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal9Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal9Holder in;
-    @Param IntHolder seed;
-    @Output IntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = seed.value;
-      } else {
-        java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), seed.value);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal18Hash implements DrillSimpleFunc {
-    @Param  Decimal18Holder in;
-    @Param IntHolder seed;
-    @Output IntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), seed.value);
-    }
-  }
-
-  @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal18Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal18Holder in;
-    @Param IntHolder seed;
-    @Output IntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = seed.value;
-      } else {
-        java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), seed.value);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
   public static class VarDecimalHash implements DrillSimpleFunc {
-    @Param
-    VarDecimalHolder in;
+    @Param VarDecimalHolder in;
     @Param IntHolder seed;
     @Output IntHolder out;
 
@@ -290,7 +196,7 @@ public class Hash32WithSeedAsDouble {
     public void eval() {
       java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer,
               in.start, in.end - in.start, in.scale);
-      out.value = Hash32Functions.hashBigDecimal(bd, seed.value);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(bd.doubleValue(), seed.value);
     }
   }
 
@@ -309,83 +215,7 @@ public class Hash32WithSeedAsDouble {
       } else {
         java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer,
                 in.start, in.end - in.start, in.scale);
-        out.value = Hash32Functions.hashBigDecimal(bd, seed.value);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal28Hash implements DrillSimpleFunc {
-    @Param  Decimal28SparseHolder in;
-    @Param IntHolder seed;
-    @Output IntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
-          in.start, in.nDecimalDigits, in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), seed.value);
-    }
-  }
-
-  @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal28Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal28SparseHolder in;
-    @Param IntHolder seed;
-    @Output IntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = seed.value;
-      } else {
-        java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
-            in.start, in.nDecimalDigits, in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), seed.value);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal38Hash implements DrillSimpleFunc {
-    @Param  Decimal38SparseHolder in;
-    @Param IntHolder seed;
-    @Output IntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
-          in.start, in.nDecimalDigits, in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), seed.value);
-    }
-  }
-
-  @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal38Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal38SparseHolder in;
-    @Param IntHolder seed;
-    @Output IntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = seed.value;
-      } else {
-        java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
-            in.start, in.nDecimalDigits, in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(input.doubleValue(), seed.value);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(bd.doubleValue(), seed.value);
       }
     }
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64AsDouble.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64AsDouble.java
index cfda6c8..9161b5e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64AsDouble.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64AsDouble.java
@@ -23,37 +23,15 @@ import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
 import org.apache.drill.exec.expr.holders.BigIntHolder;
-import org.apache.drill.exec.expr.holders.BitHolder;
-import org.apache.drill.exec.expr.holders.DateHolder;
-import org.apache.drill.exec.expr.holders.Decimal18Holder;
-import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
 import org.apache.drill.exec.expr.holders.VarDecimalHolder;
 import org.apache.drill.exec.expr.holders.NullableVarDecimalHolder;
-import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.Decimal9Holder;
 import org.apache.drill.exec.expr.holders.Float4Holder;
 import org.apache.drill.exec.expr.holders.Float8Holder;
 import org.apache.drill.exec.expr.holders.IntHolder;
 import org.apache.drill.exec.expr.holders.NullableBigIntHolder;
-import org.apache.drill.exec.expr.holders.NullableBitHolder;
-import org.apache.drill.exec.expr.holders.NullableDateHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal18Holder;
-import org.apache.drill.exec.expr.holders.NullableDecimal28SparseHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal9Holder;
 import org.apache.drill.exec.expr.holders.NullableFloat4Holder;
 import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
 import org.apache.drill.exec.expr.holders.NullableIntHolder;
-import org.apache.drill.exec.expr.holders.NullableTimeHolder;
-import org.apache.drill.exec.expr.holders.NullableTimeStampHolder;
-import org.apache.drill.exec.expr.holders.NullableVar16CharHolder;
-import org.apache.drill.exec.expr.holders.NullableVarBinaryHolder;
-import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
-import org.apache.drill.exec.expr.holders.TimeHolder;
-import org.apache.drill.exec.expr.holders.TimeStampHolder;
-import org.apache.drill.exec.expr.holders.Var16CharHolder;
-import org.apache.drill.exec.expr.holders.VarBinaryHolder;
-import org.apache.drill.exec.expr.holders.VarCharHolder;
 
 /*
  * Class contains hash64 function definitions for different data types.
@@ -63,6 +41,7 @@ import org.apache.drill.exec.expr.holders.VarCharHolder;
  * as expected we would need to hash the same value represented in different data types (int, bigint, float etc)
  * to hash to the same node, this is why we cast all numeric values to double before performing the actual hash.
  */
+@SuppressWarnings("unused")
 public class Hash64AsDouble {
   @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
   public static class NullableFloatHash implements DrillSimpleFunc {
@@ -115,7 +94,7 @@ public class Hash64AsDouble {
       if (in.isSet == 0) {
         out.value = 0;
       } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64((double) in.value, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
       }
     }
   }
@@ -208,83 +187,9 @@ public class Hash64AsDouble {
   }
 
   @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal9Hash implements DrillSimpleFunc {
-    @Param
-    Decimal9Holder in;
-    @Output
-    BigIntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), 0);
-    }
-  }
-
-  @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal9Hash implements DrillSimpleFunc {
-    @Param
-    NullableDecimal9Holder in;
-    @Output
-    BigIntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = 0;
-      } else {
-        java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), 0);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal18Hash implements DrillSimpleFunc {
-    @Param
-    Decimal18Holder in;
-    @Output
-    BigIntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), 0);
-    }
-  }
-
-  @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal18Hash implements DrillSimpleFunc {
-    @Param
-    NullableDecimal18Holder in;
-    @Output
-    BigIntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = 0;
-      } else {
-        java.math.BigDecimal input = new java.math.BigDecimal(java.math.BigInteger.valueOf(in.value), in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), 0);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
   public static class VarDecimalHash implements DrillSimpleFunc {
-    @Param
-    VarDecimalHolder in;
-    @Output
-    BigIntHolder out;
+    @Param VarDecimalHolder in;
+    @Output BigIntHolder out;
 
     public void setup() {
     }
@@ -292,16 +197,14 @@ public class Hash64AsDouble {
     public void eval() {
       java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer,
               in.start, in.end - in.start, in.scale);
-      out.value = Hash32Functions.hashBigDecimal(bd, 0);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(bd.doubleValue(), 0);
     }
   }
 
   @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
   public static class NullableVarDecimalHash implements DrillSimpleFunc {
-    @Param
-    NullableVarDecimalHolder in;
-    @Output
-    BigIntHolder out;
+    @Param NullableVarDecimalHolder in;
+    @Output BigIntHolder out;
 
     public void setup() {
     }
@@ -312,83 +215,7 @@ public class Hash64AsDouble {
       } else {
         java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer,
                 in.start, in.end - in.start, in.scale);
-        out.value = Hash32Functions.hashBigDecimal(bd, 0);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal28Hash implements DrillSimpleFunc {
-    @Param
-    Decimal28SparseHolder in;
-    @Output
-    BigIntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
-          in.start, in.nDecimalDigits, in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), 0);
-    }
-  }
-
-  @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal28Hash implements DrillSimpleFunc {
-    @Param
-    NullableDecimal28SparseHolder in;
-    @Output
-    BigIntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = 0;
-      } else {
-        java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
-            in.start, in.nDecimalDigits, in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), 0);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal38Hash implements DrillSimpleFunc {
-    @Param
-    Decimal38SparseHolder in;
-    @Output
-    BigIntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
-          in.start, in.nDecimalDigits, in.scale);
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), 0);
-    }
-  }
-
-  @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal38Hash implements DrillSimpleFunc {
-    @Param
-    NullableDecimal38SparseHolder in;
-    @Output
-    BigIntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = 0;
-      } else {
-        java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer,
-            in.start, in.nDecimalDigits, in.scale);
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(bd.doubleValue(), 0);
       }
     }
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64Functions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64Functions.java
index e94b133..582fbb5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64Functions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64Functions.java
@@ -25,22 +25,14 @@ import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
 import org.apache.drill.exec.expr.holders.BigIntHolder;
 import org.apache.drill.exec.expr.holders.BitHolder;
 import org.apache.drill.exec.expr.holders.DateHolder;
-import org.apache.drill.exec.expr.holders.Decimal18Holder;
-import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
 import org.apache.drill.exec.expr.holders.VarDecimalHolder;
 import org.apache.drill.exec.expr.holders.NullableVarDecimalHolder;
-import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.Decimal9Holder;
 import org.apache.drill.exec.expr.holders.Float4Holder;
 import org.apache.drill.exec.expr.holders.Float8Holder;
 import org.apache.drill.exec.expr.holders.IntHolder;
 import org.apache.drill.exec.expr.holders.NullableBigIntHolder;
 import org.apache.drill.exec.expr.holders.NullableBitHolder;
 import org.apache.drill.exec.expr.holders.NullableDateHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal18Holder;
-import org.apache.drill.exec.expr.holders.NullableDecimal28SparseHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal9Holder;
 import org.apache.drill.exec.expr.holders.NullableFloat4Holder;
 import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
 import org.apache.drill.exec.expr.holders.NullableIntHolder;
@@ -55,11 +47,10 @@ import org.apache.drill.exec.expr.holders.Var16CharHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 
-import java.math.BigDecimal;
-
 /*
  * Class contains hash64 function definitions for different data types.
  */
+@SuppressWarnings("unused")
 public class Hash64Functions {
   @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
   public static class NullableFloatHash implements DrillSimpleFunc {
@@ -415,79 +406,9 @@ public class Hash64Functions {
   }
 
   @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal9Hash implements DrillSimpleFunc {
-    @Param
-    Decimal9Holder in;
-    @Output
-    BigIntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
-    }
-  }
-
-  @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal9Hash implements DrillSimpleFunc {
-    @Param
-    NullableDecimal9Holder in;
-    @Output
-    BigIntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = 0;
-      } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal18Hash implements DrillSimpleFunc {
-    @Param
-    Decimal18Holder in;
-    @Output
-    BigIntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
-    }
-  }
-
-  @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal18Hash implements DrillSimpleFunc {
-    @Param
-    NullableDecimal18Holder in;
-    @Output
-    BigIntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = 0;
-      } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, 0);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
   public static class VarDecimalHash implements DrillSimpleFunc {
-    @Param
-    VarDecimalHolder in;
-    @Output
-    BigIntHolder out;
+    @Param VarDecimalHolder in;
+    @Output BigIntHolder out;
 
     public void setup() {
     }
@@ -499,10 +420,8 @@ public class Hash64Functions {
 
   @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
   public static class NullableVarDecimalHash implements DrillSimpleFunc {
-    @Param
-    NullableVarDecimalHolder in;
-    @Output
-    BigIntHolder out;
+    @Param NullableVarDecimalHolder in;
+    @Output BigIntHolder out;
 
     public void setup() {
     }
@@ -516,78 +435,6 @@ public class Hash64Functions {
     }
   }
 
-  @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal28Hash implements DrillSimpleFunc {
-    @Param
-    Decimal28SparseHolder in;
-    @Output
-    BigIntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.start + Decimal28SparseHolder.WIDTH,
-          in.buffer, 0);
-    }
-  }
-
-  @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal28Hash implements DrillSimpleFunc {
-    @Param
-    NullableDecimal28SparseHolder in;
-    @Output
-    BigIntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
-        out.value = 0;
-      } else {
-        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.start
-            + NullableDecimal28SparseHolder.WIDTH, in.buffer, 0);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal38Hash implements DrillSimpleFunc {
-    @Param
-    Decimal38SparseHolder in;
-    @Output
-    BigIntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.start + Decimal38SparseHolder.WIDTH,
-          in.buffer, 0);
-    }
-  }
-
-  @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal38Hash implements DrillSimpleFunc {
-    @Param
-    NullableDecimal38SparseHolder in;
-    @Output
-    BigIntHolder out;
-
-    public void setup() {
-    }
-
-    public void eval() {
-      if (in.isSet == 0) {
... 9389 lines suppressed ...

-- 
To stop receiving notification emails like this one, please contact
volodymyr@apache.org.