You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@drill.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/05/04 17:31:17 UTC

[jira] [Commented] (DRILL-6094) Decimal data type enhancements

    [ https://issues.apache.org/jira/browse/DRILL-6094?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16464172#comment-16464172 ] 

ASF GitHub Bot commented on DRILL-6094:
---------------------------------------

vvysotskyi closed pull request #1232: DRILL-6094: Decimal data type enhancements
URL: https://github.com/apache/drill/pull/1232
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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 8f5d1f920f..1346f478d4 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
@@ -73,6 +73,7 @@ public static boolean isNumericType(final MajorType type) {
 
     switch(type.getMinorType()) {
     case BIGINT:
+    case VARDECIMAL:
     case DECIMAL38SPARSE:
     case DECIMAL38DENSE:
     case DECIMAL28SPARSE:
@@ -120,6 +121,7 @@ public static String getSqlTypeName(final MajorType type) {
       case FLOAT4:          return "FLOAT";
       case FLOAT8:          return "DOUBLE";
 
+      case VARDECIMAL:
       case DECIMAL9:
       case DECIMAL18:
       case DECIMAL28DENSE:
@@ -238,6 +240,7 @@ public static boolean isJdbcSignedType( final MajorType type ) {
           case INTERVALYEAR:    // SQL INTERVAL w/YEAR and/or MONTH
           case INTERVALDAY:     // SQL INTERVAL w/DAY, HOUR, MINUTE and/or SECOND
           // Not-yet seen/verified signed types:
+          case VARDECIMAL:      // SQL DECIMAL (if used)
           case DECIMAL9:        // SQL DECIMAL (if used)
           case DECIMAL18:       // SQL DECIMAL (if used)
           case DECIMAL28SPARSE: // SQL DECIMAL (if used)
@@ -315,6 +318,7 @@ public static int getJdbcDisplaySize(MajorType type) {
       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:
@@ -398,6 +402,7 @@ public static boolean isFixedWidthType(final MajorType type) {
     case VAR16CHAR:
     case VARCHAR:
     case UNION:
+    case VARDECIMAL:
       return false;
     default:
       return true;
@@ -571,7 +576,7 @@ public static MinorType getMinorTypeFromName(String typeName) {
     case "double":
       return MinorType.FLOAT8;
     case "decimal":
-      return MinorType.DECIMAL38SPARSE;
+      return MinorType.VARDECIMAL;
     case "symbol":
     case "char":
     case "utf8":
@@ -633,12 +638,10 @@ public static String getNameOfMinorType(final MinorType type) {
         return "float";
       case FLOAT8:
         return "double";
+      case VARDECIMAL:
       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 33c1f48deb..0e8a1c721f 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 static long getDecimal18FromBigDecimal(BigDecimal input, int scale, int p
     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,7 +46,8 @@ public static boolean isDecimalType(TypeProtos.MajorType type) {
   }
 
   public static boolean isDecimalType(TypeProtos.MinorType minorType) {
-    return minorType == TypeProtos.MinorType.DECIMAL9 ||
+    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 74e6dbf222..0000000000
--- 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 36e7309fe0..341093947e 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.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 @@ public Boolean visitSchemaPath(SchemaPath path, LogicalExpression valueArg) thro
       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 @@ public Boolean visitSchemaPath(SchemaPath path, LogicalExpression valueArg) thro
         .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 9f93e18b3b..fde4d28504 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 @@ private void writeToListOrMap(MapOrListWriterImpl writer, DBDocumentReaderBase r
           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 @@ private void writeDouble(MapOrListWriterImpl writer, String fieldName, DBDocumen
     }
   }
 
+  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 6e26c55287..465eefeb5b 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 797b038ebc..b48f3bc062 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 637c83b33e..0f1cf2eab8 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 d7c196d000..1c897d694d 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 2ace857725..5f9d37a998 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 f9200ec093..b1dec7bd62 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 30287b6adc..435eb77158 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 ec8a1c84ff..675bba02d3 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 e13dc36443..864610c7b7 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 9c45314163..a539b7f1d9 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 Text getPrimitiveWritableObject(Object o) {
     }
 
     @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 BytesWritable getPrimitiveWritableObject(Object o) {
       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 BytesWritable getPrimitiveWritableObject(Object o) {
       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 Boolean getPrimitiveJavaObject(Object o) {
       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 HiveDecimalWritable getPrimitiveWritableObject(Object o) {
       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 TimestampWritable getPrimitiveWritableObject(Object o) {
 
 <#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 DateWritable getPrimitiveWritableObject(Object o) {
 
 <#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 Integer getPrimitiveJavaObject(Object o) {
       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 Integer getPrimitiveJavaObject(Object o) {
       }
       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 f6232f680d..92e7f83ccc 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.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 RelDataType inferReturnType(SqlOperatorBinding opBinding) {
             .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 2fab5a4d4a..6caecbb824 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.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 @@
     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 void setSafeValue(ObjectInspector oi, Object hiveFieldValue, ValueVector
     }
   }
 
-  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 5ff55060a8..91d0567af1 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.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 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 static void populateVector(final ValueVector vector, final DrillBuf manag
         }
         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 static MajorType getMajorTypeFromHiveTypeInfo(final TypeInfo typeInfo, fi
               .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 a922b4ca06..d0d9ed067b 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 @@ private void init() throws ExecutionSetupException {
 
         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 3d4dcebc34..ad572efaff 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 bea99c71a6..5f3f7131be 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 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.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 @@ public JdbcRecordReader(DataSource source, String sql, String storagePluginName)
         .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 @@ private static String nameFromType(int javaSqlType) {
 
   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 @@ public void setup(OperatorContext operatorContext, OutputMutator output) throws
           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 @@ public int next() {
     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 @@ void copy(int index) throws SQLException {
 
   }
 
-  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 @@ public DecimalCopier(int columnIndex, ResultSet result, NullableFloat8Vector.Mut
     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 130ea0f06f..b12887d7cf 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.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 Boolean visitCastExpression(CastExpression e,
   @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 Boolean visitSchemaPath(SchemaPath path, LogicalExpression valueArg)
       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 Boolean visitSchemaPath(SchemaPath path, LogicalExpression valueArg)
         .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 8e3e4cfdbc..cbc3a02aa6 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -511,47 +511,42 @@
     <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>
@@ -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 c70f6da6e0..50f110dd18 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/CastHigh.tdd b/exec/java-exec/src/main/codegen/data/CastHigh.tdd
index 54c337ddfe..dc4830b1ba 100644
--- a/exec/java-exec/src/main/codegen/data/CastHigh.tdd
+++ b/exec/java-exec/src/main/codegen/data/CastHigh.tdd
@@ -24,5 +24,6 @@
     {value: false, from: "Decimal18"},
     {value: false, from: "Decimal28Sparse"},
     {value: false, from: "Decimal38Sparse"},
+    {value: false, from: "VarDecimal"}
   ]
 } 
diff --git a/exec/java-exec/src/main/codegen/data/Casts.tdd b/exec/java-exec/src/main/codegen/data/Casts.tdd
index 307c0f2630..e43572a878 100644
--- a/exec/java-exec/src/main/codegen/data/Casts.tdd
+++ b/exec/java-exec/src/main/codegen/data/Casts.tdd
@@ -80,132 +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: "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: "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: "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: "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: "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: "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: "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: "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: "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: "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: "Decimal18", to: "Decimal9", major: "DownwardDecimalSimpleDecimalSimple", javatype: "int"},
-
-    {from: "Decimal28Sparse", to: "Decimal18", major: "DownwardDecimalSimpleDecimalComplex", javatype: "long"},
-    {from: "Decimal28Sparse", to: "Decimal9", major: "DownwardDecimalSimpleDecimalComplex", javatype: "int"},
-
-    {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: "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: "VarDecimal", to: "BigInt", major: "DecimalComplexBigInt", javatype: "long"},
+
+    {from: "VarDecimal", to: "Float4", major: "DecimalComplexFloat", javatype: "float"},
+
+    {from: "Float4", to: "VarDecimal", major: "FloatDecimalComplex"},
+
+    {from: "Float8", to: "VarDecimal", major: "DoubleDecimalComplex"},
+
+    {from: "VarDecimal", to: "Float8", major: "DecimalComplexDouble", javatype: "double"},
+
+    {from: "VarChar", to: "VarDecimal", major: "VarCharDecimalComplex"},
+
+    {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"},
 
     {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"},
 
     {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"},
 
     {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"},
   ]
-} 
+}
diff --git a/exec/java-exec/src/main/codegen/data/ComparisonTypesDecimal.tdd b/exec/java-exec/src/main/codegen/data/ComparisonTypesDecimal.tdd
index 423fe895d1..f5b912e837 100644
--- a/exec/java-exec/src/main/codegen/data/ComparisonTypesDecimal.tdd
+++ b/exec/java-exec/src/main/codegen/data/ComparisonTypesDecimal.tdd
@@ -15,6 +15,7 @@
 # limitations under the License.
 {
   { decimalTypes: [
+      {name: "VarDecimal"},
       {name: "Decimal28Sparse", storage: "5"},
       {name: "Decimal38Sparse", storage: "6"},
       {name: "Decimal28Dense", storage: "4"},
@@ -23,4 +24,4 @@
       {name: "Decimal18", storage: "long"}
     ]
   }
-}
\ No newline at end of file
+}
diff --git a/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd b/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd
index aec73d1a09..21fcf20433 100644
--- a/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd
+++ b/exec/java-exec/src/main/codegen/data/CountAggrTypes.tdd
@@ -39,6 +39,9 @@
     "VarBinary"
     "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 92e93bb839..7da2d071f1 100644
--- a/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd
+++ b/exec/java-exec/src/main/codegen/data/DecimalAggrTypes1.tdd
@@ -17,37 +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: "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: "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"}
-        ]
-      }
-   ]
+  ]
 }
diff --git a/exec/java-exec/src/main/codegen/data/DecimalAggrTypes2.tdd b/exec/java-exec/src/main/codegen/data/DecimalAggrTypes2.tdd
index 5aa8b7fba3..36a92336a8 100644
--- a/exec/java-exec/src/main/codegen/data/DecimalAggrTypes2.tdd
+++ b/exec/java-exec/src/main/codegen/data/DecimalAggrTypes2.tdd
@@ -17,14 +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: "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 0000000000..670c53d30f
--- /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 cd449166d9..51a1b289e5 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 90b432e23d..9f33353a8c 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",
-             "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 26d6be80dc..1a876c8a2d 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 @@
   <#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 8dacca8659..b6cbfeb2c8 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToTimeStampFunction.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToTimeStampFunction.java
@@ -40,38 +40,27 @@
  * 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.startsWith("Decimal"))>
-        <#if (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 d637dde651..0000000000
--- 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 1acbe5ebd0..15686bacb7 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,24 +46,21 @@ public void eval() {
  */
 
 @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 == "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 37f35f3815..e5123b2377 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,79 +41,24 @@
 /*
  * 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() {
+  }
 
-        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;
-        }
-    }
+  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>
\ No newline at end of file
+</#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 69010efcc3..0000000000
--- 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/CastDecimalSimilar.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarDecimal.java
similarity index 53%
rename from exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSimilar.java
rename to exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarDecimal.java
index 8b97b1a814..aeffe5a876 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSimilar.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarDecimal.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 == "DecimalToVarDecimal">
 
 <@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/gcast/Cast${type.from}${type.to}.java" />
 
@@ -47,51 +45,45 @@
 /*
  * 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;
+  @Inject DrillBuf buffer;
+  @Param IntHolder precision;
+  @Param IntHolder scale;
+  @Output ${type.to}Holder out;
+
+  public void setup() {
+  }
+
+  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> <#-- type.major -->
+</#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 e4c221ee8e..369ba02044 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.expr.annotations.Workspace;
 
 import io.netty.buffer.ByteBuf;
-import io.netty.buffer.DrillBuf;
-import io.netty.buffer.SwappedByteBuf;
 
 import java.nio.ByteBuffer;
 
@@ -56,179 +54,23 @@
     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() {
-
-        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());
-    }
+  @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>
\ No newline at end of file
+</#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 a1ed7c8d7f..0000000000
--- 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 95508a1ed7..f051310d2a 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,44 +52,38 @@
 
 @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;
-        out.precision = (int) precision.value;
-
-        <#if 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 abacf49005..162c5629e1 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
@@ -48,63 +48,35 @@
 
 @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")>
-    @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;
-        out.precision = (int) precision.value;
-
-        <#if 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>
\ No newline at end of file
+</#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 541b40a3e4..0000000000
--- 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 a4341e98ca..a6e209d1d1 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
@@ -19,191 +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;
-        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;
-        }
-    }
-}
-
-<#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"/>
@@ -238,256 +54,57 @@ public void eval() {
 @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;
-        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);
-    }
+    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>
\ No newline at end of file
+</#list>
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastVarDecimalDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastVarDecimalDecimal.java
new file mode 100644
index 0000000000..8ef2bf9a0e
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastVarDecimalDecimal.java
@@ -0,0 +1,92 @@
+/*
+ * 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 == "VarDecimalToDecimal">
+
+<@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.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>
+</#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 55ce0772ea..6b23f92edc 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,199 +52,192 @@
 @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>
-
-@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{
-
-  @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;
-
-  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);
-    }
-    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);
-    }
-    // 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>
 
-  }
+  <#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();
+    }
 
-  @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));
+    @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;
       }
-      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;
+      <#if type.inputType?starts_with("Nullable")>
+      } // end of sout block
+      </#if>
     }
-    <#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));
+
+    @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;
       }
-      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 = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer, in.start, in.nDecimalDigits, in.scale);
-    </#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;
-    out.precision = 38;
-    value.obj = ((java.math.BigDecimal) (value.obj)).setScale(out.scale, java.math.BigDecimal.ROUND_HALF_UP);
-    org.apache.drill.exec.util.DecimalUtility.getSparseFromBigDecimal((java.math.BigDecimal) value.obj, out.buffer, out.start, out.scale, out.precision, out.nDecimalDigits);
-   <#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);
+    @Override
+    public void reset() {
+      value = new ObjectHolder();
+      value.obj = java.math.BigDecimal.ZERO;
+      outputScale = new IntHolder();
+      outputScale.value = Integer.MIN_VALUE;
+      nonNullCount.value = 0;
     }
-    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);
+  <#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();
     }
-    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 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;
+      }
 
- }
+      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;
+      }
+      <#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;
+      }
 
+      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>
+    }
 
+    @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;
+      }
+    }
+
+    @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 e2da4282b6..c633b678c9 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 @@
 <#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 @@
 <#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 @@
   @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 void setup() {
   @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 = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromSparse(in.buffer, in.start, in.nDecimalDigits, in.scale);
-    </#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,12 +102,21 @@ public void add() {
 
   @Override
   public void output() {
-    out.buffer = buffer;
-    out.start  = 0;
-    out.scale = outputScale.value;
-    out.precision = 38;
-    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);
-    org.apache.drill.exec.util.DecimalUtility.getSparseFromBigDecimal(average, out.buffer, out.start, out.scale, out.precision, out.nDecimalDigits);
+    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
@@ -125,7 +130,6 @@ public void reset() {
   }
 }
 
-
 </#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 0000000000..3058381d9f
--- /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 721a9d49c7..059080e15f 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java
@@ -85,23 +85,19 @@
           } // outside
 </#macro>
 
-<#macro adjustScale javaType leftType rightType>
-
-            // Adjust the scale of the two inputs to be the same
+<#macro varCompareBlock leftType rightType absCompare output nullCompare nullComparesHigh>
+      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) ... -->
 <#list comparisonTypesDecimal.decimalTypes as type>
 
-<#if type.name.endsWith("Sparse")>
+<#if type.name.endsWith("VarDecimal")>
 
 <@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/${type.name}Functions.java" />
 
@@ -125,758 +121,218 @@
 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);
-        }
+<#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 = "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;
+    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);
 
-          // Set the output buffer with the positive sign
-          out.buffer.setInt(out.start, (out.buffer.getInt(out.start) & 0x7fffffff));
-        }
+      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() {}
-
-        public void eval() {
-
-          boolean zeroValue = true;
+</#list>
 
-          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;
-          }
-        }
+<#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;
-        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);
-        }
+    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;
-        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++;
-            }
-          }
+</#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;
 
-          // 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() {
+      // 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;
-        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);
-
-          // Integer part's src index
-          int srcIntIndex = ${type.storage} - org.apache.drill.exec.util.DecimalUtility.roundUp(in.scale) - 1;
-
-          // 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);
-          }
+  @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;
 
-          // 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 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;
-        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() {
+      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;
-        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;
-
-          // 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);
-          }
+  @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;
 
-          // Set the remaining portion of the decimal to be zeroes
-          while (destIndex >= 0) {
-            out.setInteger(destIndex--, 0, out.start, out.buffer);
-          }
-
-          // 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);
-        }
+    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;
-        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);
-        }
+    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;
@@ -886,7 +342,7 @@ public void eval() {
     public void setup() {}
 
     public void eval() {
-      <@compareBlock 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 />
     }
   }
 
@@ -895,9 +351,9 @@ public void eval() {
   <#-- 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;
@@ -907,937 +363,132 @@ public void eval() {
     public void setup() {}
 
     public void eval() {
-      <@compareBlock 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>
-
-
-    <#-- 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;
-        }
-    }
-
+  </#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 = "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;
-
+  @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() {}
 
     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
+      int cmp;
+      <@varCompareBlock leftType = "leftType" rightType = "rightType" absCompare = "false" output = "cmp" nullCompare = false nullComparesHigh = false />
+      out.value = cmp == -1 ? 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 {
+  <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
+         not for sorting and grouping relational operators.) -->
+  @FunctionTemplate(name = FunctionGenerationHelper.LE,
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class ${type.name}LessThanEq implements DrillSimpleFunc {
 
-    @Param ${leftType}Holder left;
-    @Param ${rightType}Holder right;
-    @Output IntHolder out;
+    @Param ${type.name}Holder left;
+    @Param ${type.name}Holder right;
+    @Output BitHolder 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
+      int cmp;
+      <@varCompareBlock leftType = "leftType" rightType = "rightType" absCompare = "false" output = "cmp" nullCompare = false nullComparesHigh = false />
+      out.value = cmp < 1 ? 1 : 0;
     }
   }
 
- </#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  = 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;
-        }
-    }
+  @FunctionTemplate(name = FunctionGenerationHelper.GT,
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class ${type.name}GreaterThan implements DrillSimpleFunc {
 
-    <#-- 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;
-        }
-    }
+    @Param ${type.name}Holder left;
+    @Param ${type.name}Holder right;
+    @Output BitHolder out;
 
-    <#-- 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;
-        }
-    }
+    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 = 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;
-        }
+    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 = 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 {
+  @FunctionTemplate(name = FunctionGenerationHelper.GE,
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class ${type.name}GreaterThanEq 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;
+    @Param ${type.name}Holder left;
+    @Param ${type.name}Holder right;
+    @Output BitHolder 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
+      int cmp;
+      <@varCompareBlock leftType = "leftType" rightType = "rightType" absCompare = "false" output = "cmp" nullCompare = false nullComparesHigh = false />
+      out.value = cmp > -1 ? 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 {
+  <#-- 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 ${leftType}Holder left;
-    @Param ${rightType}Holder right;
-    @Output IntHolder out;
+    @Param ${type.name}Holder left;
+    @Param ${type.name}Holder right;
+    @Output BitHolder 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
+      int cmp;
+      <@varCompareBlock leftType = "leftType" rightType = "rightType" absCompare = "false" output = "cmp" nullCompare = false nullComparesHigh = false />
+      out.value = cmp == 0 ? 1 : 0;
     }
   }
 
- </#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() {
-            <@adjustScale javaType=type.storage leftType="leftType" rightType="rightType"/>
-            out.value = (left.value < right.value) ? 1 : 0;
-        }
-    }
+  @FunctionTemplate(name = FunctionGenerationHelper.NE,
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class ${type.name}NotEqual implements DrillSimpleFunc {
 
-     <#-- 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;
-        }
-    }
+    @Param ${type.name}Holder left;
+    @Param ${type.name}Holder right;
+    @Output BitHolder out;
 
-     <#-- 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 setup() {}
 
-        public void eval() {
-            <@adjustScale javaType=type.storage leftType="leftType" rightType="rightType"/>
-            out.value = (left.value != right.value) ? 1 : 0;
-        }
+    public void eval() {
+      int cmp;
+      <@varCompareBlock leftType = "leftType" rightType = "rightType" absCompare = "false" output = "cmp" nullCompare = false nullComparesHigh = false />
+      out.value = cmp != 0 ? 1 : 0;
     }
+  }
 }
-
 </#if>
 
 </#list>
diff --git a/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java b/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java
index 2bd3d0c50f..a3bbc9b9a0 100644
--- a/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/JsonOutputRecordWriter.java
@@ -93,6 +93,7 @@ public void writeField() throws IOException {
   <#case "Decimal28Dense">
   <#case "Decimal38Dense">
   <#case "Decimal38Sparse">
+  <#case "VarDecimal">
     <#assign typeName = "Decimal">
     <#break>
   <#case "Float4">
diff --git a/exec/java-exec/src/main/codegen/templates/MathFunctions.java b/exec/java-exec/src/main/codegen/templates/MathFunctions.java
index d1366cb090..a17865cbf9 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 void setup() {
   }
 
   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 void setup() {
   }
 
   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 638095015d..b192be36e1 100644
--- a/exec/java-exec/src/main/codegen/templates/NumericToCharFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/NumericToCharFunctions.java
@@ -48,42 +48,36 @@
  * 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 == "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 26a33b9dcf..20f4d6023e 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 @@
 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 void setUp(MessageType schema, RecordConsumer consumer) {
     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 void setUp(MessageType schema, RecordConsumer consumer) {
   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 void writeField() throws IOException {
       <#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 fc7100b8de..d2f27cca94 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 @@
 
 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 @@
             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 @@
 
     <#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>
@@ -141,6 +138,7 @@ public static DecimalMetadata getDecimalMetadataForField(MaterializedField field
       case DECIMAL28DENSE:
       case DECIMAL38SPARSE:
       case DECIMAL38DENSE:
+      case VARDECIMAL:
         return new DecimalMetadata(field.getPrecision(), field.getScale());
       default:
         return null;
@@ -156,10 +154,26 @@ public static int getLengthForMinorType(MinorType minorType) {
       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 29f1ac5d03..7e5e99ba99 100644
--- a/exec/java-exec/src/main/codegen/templates/SqlAccessors.java
+++ b/exec/java-exec/src/main/codegen/templates/SqlAccessors.java
@@ -126,6 +126,30 @@ public String getString(int index) {
     }
       <#break>
 
+    <#case "VarDecimal">
+
+    @Override
+    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>
+
     <#case "VarChar">
 
     @Override
@@ -228,7 +252,7 @@ public String getString(int index) {
     return String.valueOf(ac.getAsStringBuilder(index));
   }
 
-  <#elseif minor.class.startsWith("Decimal")>
+  <#elseif minor.class.contains("Decimal")>
 
   @Override
   public BigDecimal getBigDecimal(int index) {
@@ -345,4 +369,4 @@ public boolean getBoolean(int index) {
 }
 </#list>
 </#list>
-</#list>
\ No newline at end of file
+</#list>
diff --git a/exec/java-exec/src/main/codegen/templates/StringOutputRecordWriter.java b/exec/java-exec/src/main/codegen/templates/StringOutputRecordWriter.java
index cee157bba0..5fe90ceede 100644
--- a/exec/java-exec/src/main/codegen/templates/StringOutputRecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/StringOutputRecordWriter.java
@@ -140,13 +140,13 @@ public void writeField() throws IOException {
         minor.class == "Decimal28Dense" ||
         minor.class == "Decimal38Dense" ||
         minor.class == "Decimal28Sparse" ||
-        minor.class == "Decimal38Sparse">
-
+        minor.class == "Decimal38Sparse" ||
+        minor.class == "VarChar" ||
+        minor.class == "Var16Char" ||
+        minor.class == "VarBinary" ||
+        minor.class == "VarDecimal">
     // TODO: error check
     addField(fieldId, reader.readObject().toString());
-
-  <#elseif minor.class == "VarChar" || minor.class == "Var16Char" || minor.class == "VarBinary">
-    addField(fieldId, reader.readObject().toString());
   <#else>
     throw new UnsupportedOperationException(String.format("Unsupported field type: %s"),
       holder.getCanonicalClass());
diff --git a/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java b/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
index c0930f3ea4..a5afce98cf 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 @@
 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 @@
 <#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 42fd7e0742..671ce4bdcc 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 @@ private 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 001762f7e2..e1cc745f3d 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.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 @@ private ScalarReplacementTypes() {
         TimeStampHolder.class,
         VarCharHolder.class,
         VarBinaryHolder.class,
+        VarDecimalHolder.class,
         NullableBitHolder.class,
         NullableIntHolder.class,
         NullableBigIntHolder.class,
@@ -94,6 +97,7 @@ private 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 1a0b7d539a..d7646633d6 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.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;
@@ -186,6 +187,11 @@ public Boolean visitDecimal38Constant(Decimal38Expression decExpr, IdentityHashM
     return true;
   }
 
+  @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 84d120357c..9c2e394aa2 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.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;
 
@@ -130,6 +131,11 @@ public LogicalExpression visitDecimal38Constant(Decimal38Expression decExpr, Voi
     return visitUnknown(decExpr, value);
   }
 
+  @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 3f322b8da8..94f8b79489 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.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 @@ public Boolean visitDecimal38Constant(Decimal38Expression decExpr, LogicalExpres
     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 dcc2668a52..64cfe66953 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.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 HoldingContainer visitDecimal28Constant(Decimal28Expression e, ClassGener
       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 HoldingContainer visitDecimal38Constant(Decimal38Expression e, ClassGener
       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);
     }
 
@@ -996,6 +1016,16 @@ public HoldingContainer visitDecimal38Constant(Decimal38Expression decExpr, Clas
       return hc;
     }
 
+    @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());
@@ -1220,6 +1250,20 @@ public HoldingContainer visitDecimal38Constant(Decimal38Expression e, ClassGener
       }
     }
 
+    @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)) {
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 f1b50c95d7..ff38e0c884 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.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 static LogicalExpression addCastExpression(LogicalExpression fromExpr, Ma
       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 static LogicalExpression addCastExpression(LogicalExpression fromExpr, Ma
        */
       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 LogicalExpression visitBooleanOperator(BooleanOperator op, FunctionLookup
     }
 
     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 LogicalExpression visitFunctionCall(FunctionCall call, FunctionLookupCont
           }
         }
 
-        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.
@@ -812,6 +819,11 @@ public LogicalExpression visitDecimal38Constant(Decimal38Expression decExpr, Fun
       return decExpr;
     }
 
+    @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 LogicalExpression visitCastExpression(CastExpression e, FunctionLookupCon
         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,10 +923,8 @@ private boolean castEqual(ExpressionPosition pos, MajorType from, MajorType to)
       case DECIMAL28SPARSE:
       case DECIMAL38DENSE:
       case DECIMAL38SPARSE:
-        if (to.getScale() == from.getScale() && to.getPrecision() == from.getPrecision()) {
-          return true;
-        }
-        return false;
+      case VARDECIMAL:
+        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 fc93983859..3b7e2b4dea 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
@@ -81,7 +81,16 @@ public static void read(MajorType type, JExpression vector, JBlock eval, Holding
         eval.assign(out.getHolder().ref("start"), JExpr.lit(TypeHelper.getSize(type)).mul(indexVariable));
         eval.assign(out.getHolder().ref("buffer"), vector.invoke("getBuffer"));
         return;
-      case INTERVAL:{
+      case VARDECIMAL: {
+        eval.assign(out.getHolder().ref("buffer"), vector.invoke("getBuffer"));
+        JVar se = eval.decl(model.LONG, "startEnd", getValueAccessor.invoke("getStartEnd").arg(indexVariable));
+        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: {
         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));
@@ -97,13 +106,13 @@ public static void read(MajorType type, JExpression vector, JBlock eval, Holding
       }
       case VAR16CHAR:
       case VARBINARY:
-      case VARCHAR:
+      case VARCHAR: {
          eval.assign(out.getHolder().ref("buffer"), vector.invoke("getBuffer"));
          JVar se = eval.decl(model.LONG, "startEnd", getValueAccessor.invoke("getStartEnd").arg(indexVariable));
          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;
-
+      }
       }
     }
 
@@ -141,37 +150,37 @@ public static JInvocation write(MajorType type, JVar vector, HoldingContainer in
       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:
-        return setMethod //
-            .arg(in.f("start")) //
-            .arg(in.f("end")) //
+      case VARDECIMAL:
+        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 53fbaa3a66..87d66883a5 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.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 Integer visitUnknown(LogicalExpression e, Void value) throws RuntimeExcep
     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 ac4ba07bb1..1a5470ba20 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 @@
 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 @@
   boolean isNiladic() default false;
   boolean checkPrecisionRange() default false;
 
-  public enum NullHandling {
+  enum NullHandling {
     /**
      * Method handles nulls.
      */
@@ -108,7 +107,7 @@
    * 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 @@
    * 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 @@
 
     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 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 e0db999be0..c9502fb195 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,18 +23,12 @@
 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.Decimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.Decimal9Holder;
+import org.apache.drill.exec.expr.holders.VarDecimalHolder;
+import org.apache.drill.exec.expr.holders.NullableVarDecimalHolder;
 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;
@@ -43,6 +37,7 @@
  * 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)
 
@@ -132,7 +127,7 @@ public void eval() {
       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);
       }
     }
   }
@@ -168,7 +163,7 @@ public void setup() {
     }
 
     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);
     }
   }
 
@@ -188,119 +183,9 @@ public void eval() {
   }
 
   @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 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 {
+  public static class VarDecimalHash implements DrillSimpleFunc {
     @Param
-    Decimal38SparseHolder in;
+    VarDecimalHolder in;
     @Output
     IntHolder out;
 
@@ -308,16 +193,16 @@ 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);
+      java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer,
+              in.start, in.end - in.start, 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 {
+  public static class NullableVarDecimalHash implements DrillSimpleFunc {
     @Param
-    NullableDecimal38SparseHolder in;
+    NullableVarDecimalHolder in;
     @Output
     IntHolder out;
 
@@ -328,8 +213,8 @@ 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);
+        java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer,
+                in.start, in.end - in.start, 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 c04dd958d0..412674f138 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,20 +25,14 @@
 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.Decimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.Decimal9Holder;
+import org.apache.drill.exec.expr.holders.VarDecimalHolder;
+import org.apache.drill.exec.expr.holders.NullableVarDecimalHolder;
 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;
@@ -53,6 +47,7 @@
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 
+@SuppressWarnings("unused")
 public class Hash32Functions {
 
   @FunctionTemplate(names = {"hash", "hash32"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL )
@@ -369,125 +364,21 @@ public void eval() {
   }
 
   @FunctionTemplate(names = {"hash", "hash32"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal9Hash implements DrillSimpleFunc {
-    @Param  Decimal9Holder in;
+  public static class VarDecimalHash implements DrillSimpleFunc {
+    @Param  VarDecimalHolder 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 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);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, 0);
     }
   }
 
   @FunctionTemplate(names = {"hash", "hash32"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal38Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal38SparseHolder in;
+  public static class NullableVarDecimalHash implements DrillSimpleFunc {
+    @Param  NullableVarDecimalHolder in;
     @Output IntHolder out;
 
     public void setup() {
@@ -497,11 +388,7 @@ 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 65fb0eec96..1741eb8e0b 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,20 +25,14 @@
 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.Decimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.Decimal9Holder;
+import org.apache.drill.exec.expr.holders.VarDecimalHolder;
+import org.apache.drill.exec.expr.holders.NullableVarDecimalHolder;
 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;
@@ -56,6 +50,7 @@
 /*
  * 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 {
@@ -412,129 +407,25 @@ public void eval() {
   }
 
   @FunctionTemplate(name = "hash32", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal9Hash implements DrillSimpleFunc {
-    @Param  Decimal9Holder in;
+  public static class VarDecimalHash implements DrillSimpleFunc {
+    @Param  VarDecimalHolder 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 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);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(in.start, in.end, in.buffer, seed.value);
     }
   }
 
   @FunctionTemplate(name = "hash32", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal38Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal38SparseHolder in;
+  public static class NullableVarDecimalHash implements DrillSimpleFunc {
+    @Param  NullableVarDecimalHolder in;
     @Param IntHolder seed;
     @Output IntHolder out;
 
-
     public void setup() {
     }
 
@@ -542,7 +433,7 @@ 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 038d917d98..67bbfa7ade 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,18 +23,12 @@
 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.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.expr.holders.VarDecimalHolder;
+import org.apache.drill.exec.expr.holders.NullableVarDecimalHolder;
 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;
@@ -43,6 +37,7 @@
  * 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 {
@@ -190,139 +185,27 @@ public void eval() {
   }
 
   @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal9Hash implements DrillSimpleFunc {
-    @Param  Decimal9Holder in;
+  public static class VarDecimalHash implements DrillSimpleFunc {
+    @Param VarDecimalHolder 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 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);
+      java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer,
+              in.start, in.end - in.start, in.scale);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash32(bd.doubleValue(), seed.value);
     }
   }
 
   @FunctionTemplate(name = "hash32AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal28Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal28SparseHolder in;
+  public static class NullableVarDecimalHash implements DrillSimpleFunc {
+    @Param  NullableVarDecimalHolder 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() {
     }
 
@@ -330,9 +213,9 @@ 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);
+        java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer,
+                in.start, in.end - in.start, in.scale);
+        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 3329664b53..9161b5ec7e 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,18 +23,12 @@
 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.Decimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.Decimal9Holder;
+import org.apache.drill.exec.expr.holders.VarDecimalHolder;
+import org.apache.drill.exec.expr.holders.NullableVarDecimalHolder;
 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;
@@ -47,6 +41,7 @@
  * 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 {
@@ -99,7 +94,7 @@ public void eval() {
       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);
       }
     }
   }
@@ -192,138 +187,24 @@ public void eval() {
   }
 
   @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 static class VarDecimalHash implements DrillSimpleFunc {
+    @Param VarDecimalHolder 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);
+      java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer,
+              in.start, in.end - in.start, in.scale);
+      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 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 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 static class NullableVarDecimalHash implements DrillSimpleFunc {
+    @Param NullableVarDecimalHolder in;
+    @Output BigIntHolder out;
 
     public void setup() {
     }
@@ -332,9 +213,9 @@ 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);
+        java.math.BigDecimal bd = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer,
+                in.start, in.end - in.start, in.scale);
+        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 dc508cb97e..582fbb58e1 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,20 +25,14 @@
 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.Decimal38SparseHolder;
-import org.apache.drill.exec.expr.holders.Decimal9Holder;
+import org.apache.drill.exec.expr.holders.VarDecimalHolder;
+import org.apache.drill.exec.expr.holders.NullableVarDecimalHolder;
 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;
@@ -56,6 +50,7 @@
 /*
  * 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 {
@@ -411,131 +406,22 @@ public void eval() {
   }
 
   @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 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 static class VarDecimalHash implements DrillSimpleFunc {
+    @Param VarDecimalHolder 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);
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.end, 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 static class NullableVarDecimalHash implements DrillSimpleFunc {
+    @Param NullableVarDecimalHolder in;
+    @Output BigIntHolder out;
 
     public void setup() {
     }
@@ -544,8 +430,7 @@ 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
-            + NullableDecimal38SparseHolder.WIDTH, in.buffer, 0);
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.start, in.end, in.buffer, 0);
       }
     }
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64FunctionsWithSeed.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64FunctionsWithSeed.java
index 47f903d2b9..d77bf0d5d1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64FunctionsWithSeed.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64FunctionsWithSeed.java
@@ -27,6 +27,9 @@
 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;
@@ -56,6 +59,7 @@
 /*
  * Class contains hash64 function definitions for different data types.
  */
+@SuppressWarnings("unused")
 public class Hash64FunctionsWithSeed {
   @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL )
   public static class NullableFloatHash implements DrillSimpleFunc {
@@ -412,80 +416,11 @@ public void eval() {
   }
 
   @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal9Hash implements DrillSimpleFunc {
-    @Param  Decimal9Holder in;
-    @Param BigIntHolder seed;
-    @Output BigIntHolder out;
-
-
-    public void setup() {
-    }
-
-    public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
-    }
-  }
-
-  @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal9Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal9Holder in;
-    @Param BigIntHolder seed;
-    @Output BigIntHolder 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.hash64(in.value, seed.value);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal18Hash implements DrillSimpleFunc {
-    @Param  Decimal18Holder in;
+  public static class VarDecimalHash implements DrillSimpleFunc {
+    @Param  VarDecimalHolder in;
     @Param BigIntHolder seed;
     @Output BigIntHolder out;
 
-
-    public void setup() {
-    }
-
-    public void eval() {
-      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(in.value, seed.value);
-    }
-  }
-
-  @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal18Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal18Holder in;
-    @Param BigIntHolder seed;
-    @Output BigIntHolder 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.hash64(in.value, seed.value);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal28Hash implements DrillSimpleFunc {
-    @Param  Decimal28SparseHolder in;
-    @Param BigIntHolder seed;
-    @Output BigIntHolder out;
-
-
     public void setup() {
     }
 
@@ -495,12 +430,11 @@ public void eval() {
   }
 
   @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal28Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal28SparseHolder in;
+  public static class NullableVarDecimalHash implements DrillSimpleFunc {
+    @Param  NullableVarDecimalHolder in;
     @Param BigIntHolder seed;
     @Output BigIntHolder out;
 
-
     public void setup() {
     }
 
@@ -513,40 +447,6 @@ public void eval() {
     }
   }
 
-  @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal38Hash implements DrillSimpleFunc {
-    @Param  Decimal38SparseHolder in;
-    @Param BigIntHolder seed;
-    @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, seed.value);
-    }
-  }
-
-  @FunctionTemplate(name = "hash64", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal38Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal38SparseHolder in;
-    @Param BigIntHolder seed;
-    @Output BigIntHolder 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.hash64(in.start, in.start + NullableDecimal38SparseHolder.WIDTH, in.buffer, seed.value);
-      }
-    }
-  }
-
   @FunctionTemplate(names = {"hash64", "hash64AsDouble"}, scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL )
   public static class NullableBitHash implements DrillSimpleFunc {
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64WithSeedAsDouble.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64WithSeedAsDouble.java
index cce6ea018a..2497e11991 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64WithSeedAsDouble.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Hash64WithSeedAsDouble.java
@@ -23,18 +23,12 @@
 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.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.expr.holders.VarDecimalHolder;
+import org.apache.drill.exec.expr.holders.NullableVarDecimalHolder;
 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;
@@ -47,6 +41,7 @@
  * 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 Hash64WithSeedAsDouble {
   @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL )
   public static class NullableFloatHash implements DrillSimpleFunc {
@@ -194,139 +189,27 @@ public void eval() {
   }
 
   @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal9Hash implements DrillSimpleFunc {
-    @Param  Decimal9Holder in;
+  public static class VarDecimalHash implements DrillSimpleFunc {
+    @Param  VarDecimalHolder in;
     @Param BigIntHolder seed;
     @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(), seed.value);
-    }
-  }
-
-  @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal9Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal9Holder in;
-    @Param BigIntHolder seed;
-    @Output BigIntHolder 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.hash64(input.doubleValue(), seed.value);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal18Hash implements DrillSimpleFunc {
-    @Param  Decimal18Holder in;
-    @Param BigIntHolder seed;
-    @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(), seed.value);
-    }
-  }
-
-  @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal18Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal18Holder in;
-    @Param BigIntHolder seed;
-    @Output BigIntHolder 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.hash64(input.doubleValue(), seed.value);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal28Hash implements DrillSimpleFunc {
-    @Param  Decimal28SparseHolder in;
-    @Param BigIntHolder seed;
-    @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(), seed.value);
-    }
-  }
-
-  @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal28Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal28SparseHolder in;
-    @Param BigIntHolder seed;
-    @Output BigIntHolder 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.hash64(input.doubleValue(), seed.value);
-      }
-    }
-  }
-
-  @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class Decimal38Hash implements DrillSimpleFunc {
-    @Param  Decimal38SparseHolder in;
-    @Param BigIntHolder seed;
-    @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);
+      java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer,
+              in.start, in.end - in.start, in.scale);
       out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), seed.value);
     }
   }
 
   @FunctionTemplate(name = "hash64AsDouble", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-  public static class NullableDecimal38Hash implements DrillSimpleFunc {
-    @Param  NullableDecimal38SparseHolder in;
+  public static class NullableVarDecimalHash implements DrillSimpleFunc {
+    @Param  NullableVarDecimalHolder in;
     @Param BigIntHolder seed;
     @Output BigIntHolder out;
 
-
     public void setup() {
     }
 
@@ -334,8 +217,8 @@ 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);
+        java.math.BigDecimal input = org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(in.buffer,
+                in.start, in.end - in.start, in.scale);
         out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash64(input.doubleValue(), seed.value);
       }
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/UnionFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/UnionFunctions.java
index 0aa9d48541..6b2b7aabb9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/UnionFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/UnionFunctions.java
@@ -115,6 +115,7 @@ private static int getTypeValue(MinorType type) {
     case DECIMAL18:
     case DECIMAL28SPARSE:
     case DECIMAL38SPARSE:
+    case VARDECIMAL:
     case FLOAT4:
     case FLOAT8:
       return 0;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/interpreter/InterpreterEvaluator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/interpreter/InterpreterEvaluator.java
index 4d837d838f..9e8af1b5ab 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/interpreter/InterpreterEvaluator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/interpreter/InterpreterEvaluator.java
@@ -19,6 +19,7 @@
 
 import java.lang.reflect.Field;
 import java.lang.reflect.Method;
+import java.util.Objects;
 
 import javax.annotation.Nullable;
 import javax.inject.Inject;
@@ -242,6 +243,12 @@ public ValueHolder apply(DrillBuf buffer) {
       });
     }
 
+    @Override
+    public ValueHolder visitVarDecimalConstant(final ValueExpressions.VarDecimalExpression decExpr, Integer value) throws RuntimeException {
+      return getConstantValueHolder(decExpr.getBigDecimal().toString(), decExpr.getMajorType().getMinorType(),
+          buffer -> ValueHolderHelper.getVarDecimalHolder(Objects.requireNonNull(buffer), decExpr.getBigDecimal().toString()));
+    }
+
     @Override
     public ValueHolder visitDateConstant(ValueExpressions.DateExpression dateExpr,Integer value) throws RuntimeException {
       return ValueHolderHelper.getDateHolder(dateExpr.getDate());
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/output/DecimalReturnTypeInference.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/output/DecimalReturnTypeInference.java
index ab9c3185cc..442514f792 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/output/DecimalReturnTypeInference.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/output/DecimalReturnTypeInference.java
@@ -21,10 +21,10 @@
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.ValueExpressions;
 import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.common.util.DecimalScalePrecisionAddFunction;
-import org.apache.drill.common.util.DecimalScalePrecisionDivideFunction;
-import org.apache.drill.common.util.DecimalScalePrecisionModFunction;
-import org.apache.drill.common.util.DecimalScalePrecisionMulFunction;
+import org.apache.drill.exec.planner.types.decimal.DecimalScalePrecisionAddFunction;
+import org.apache.drill.exec.planner.types.decimal.DecimalScalePrecisionDivideFunction;
+import org.apache.drill.exec.planner.types.decimal.DecimalScalePrecisionModFunction;
+import org.apache.drill.exec.planner.types.decimal.DecimalScalePrecisionMulFunction;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate;
 import org.apache.drill.exec.expr.fn.FunctionAttributes;
 import org.apache.drill.exec.expr.fn.FunctionUtils;
@@ -32,6 +32,8 @@
 
 import java.util.List;
 
+import static org.apache.drill.exec.planner.types.DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM;
+
 public class DecimalReturnTypeInference {
 
   /**
@@ -56,13 +58,17 @@
 
       assert logicalExpressions.size() == 2;
 
+      TypeProtos.MajorType leftMajorType = logicalExpressions.get(0).getMajorType();
+      TypeProtos.MajorType rightMajorType = logicalExpressions.get(1).getMajorType();
+
       DecimalScalePrecisionAddFunction outputScalePrec =
-          new DecimalScalePrecisionAddFunction(logicalExpressions.get(0).getMajorType().getPrecision(),
-              logicalExpressions.get(0).getMajorType().getScale(),
-              logicalExpressions.get(1).getMajorType().getPrecision(),
-              logicalExpressions.get(1).getMajorType().getScale());
+          new DecimalScalePrecisionAddFunction(
+              DecimalUtility.getDefaultPrecision(leftMajorType.getMinorType(), leftMajorType.getPrecision()),
+              leftMajorType.getScale(),
+              DecimalUtility.getDefaultPrecision(rightMajorType.getMinorType(), rightMajorType.getPrecision()),
+              rightMajorType.getScale());
       return TypeProtos.MajorType.newBuilder()
-          .setMinorType(DecimalUtility.getDecimalDataType(outputScalePrec.getOutputPrecision()))
+          .setMinorType(TypeProtos.MinorType.VARDECIMAL)
           .setScale(outputScalePrec.getOutputScale())
           .setPrecision(outputScalePrec.getOutputPrecision())
           .setMode(mode)
@@ -93,7 +99,7 @@
           .setMinorType(attributes.getReturnValue().getType().getMinorType())
           .setScale(scale)
           .setPrecision(precision)
-          .setMode(TypeProtos.DataMode.REQUIRED)
+          .setMode(TypeProtos.DataMode.OPTIONAL)
           .build();
     }
   }
@@ -118,8 +124,8 @@
         throw new DrillRuntimeException("Decimal cast function invoked with incorrect arguments" + err);
       }
 
-      int scale = (int) ((ValueExpressions.LongExpression)(logicalExpressions.get(logicalExpressions.size() - 1))).getLong();
-      int precision = (int) ((ValueExpressions.LongExpression)(logicalExpressions.get(logicalExpressions.size() - 2))).getLong();
+      int scale = ((ValueExpressions.IntExpression) logicalExpressions.get(logicalExpressions.size() - 1)).getInt();
+      int precision = ((ValueExpressions.IntExpression) logicalExpressions.get(logicalExpressions.size() - 2)).getInt();
       return TypeProtos.MajorType.newBuilder()
           .setMinorType(attributes.getReturnValue().getType().getMinorType())
           .setScale(scale)
@@ -151,13 +157,17 @@
 
       assert logicalExpressions.size() == 2;
 
+      TypeProtos.MajorType leftMajorType = logicalExpressions.get(0).getMajorType();
+      TypeProtos.MajorType rightMajorType = logicalExpressions.get(1).getMajorType();
+
       DecimalScalePrecisionDivideFunction outputScalePrec =
-          new DecimalScalePrecisionDivideFunction(logicalExpressions.get(0).getMajorType().getPrecision(),
-              logicalExpressions.get(0).getMajorType().getScale(),
-              logicalExpressions.get(1).getMajorType().getPrecision(),
-              logicalExpressions.get(1).getMajorType().getScale());
+          new DecimalScalePrecisionDivideFunction(
+              DecimalUtility.getDefaultPrecision(leftMajorType.getMinorType(), leftMajorType.getPrecision()),
+              leftMajorType.getScale(),
+              DecimalUtility.getDefaultPrecision(rightMajorType.getMinorType(), rightMajorType.getPrecision()),
+              rightMajorType.getScale());
       return TypeProtos.MajorType.newBuilder()
-          .setMinorType(DecimalUtility.getDecimalDataType(outputScalePrec.getOutputPrecision()))
+          .setMinorType(TypeProtos.MinorType.VARDECIMAL)
           .setScale(outputScalePrec.getOutputScale())
           .setPrecision(outputScalePrec.getOutputPrecision())
           .setMode(mode)
@@ -216,13 +226,17 @@
 
       assert logicalExpressions.size() == 2;
 
+      TypeProtos.MajorType leftMajorType = logicalExpressions.get(0).getMajorType();
+      TypeProtos.MajorType rightMajorType = logicalExpressions.get(1).getMajorType();
+
       DecimalScalePrecisionModFunction outputScalePrec =
-          new DecimalScalePrecisionModFunction(logicalExpressions.get(0).getMajorType().getPrecision(),
-              logicalExpressions.get(0).getMajorType().getScale(),
-              logicalExpressions.get(1).getMajorType().getPrecision(),
-              logicalExpressions.get(1).getMajorType().getScale());
+          new DecimalScalePrecisionModFunction(
+              DecimalUtility.getDefaultPrecision(leftMajorType.getMinorType(), leftMajorType.getPrecision()),
+              leftMajorType.getScale(),
+              DecimalUtility.getDefaultPrecision(rightMajorType.getMinorType(), rightMajorType.getPrecision()),
+              rightMajorType.getScale());
       return TypeProtos.MajorType.newBuilder()
-          .setMinorType(DecimalUtility.getDecimalDataType(outputScalePrec.getOutputPrecision()))
+          .setMinorType(TypeProtos.MinorType.VARDECIMAL)
           .setScale(outputScalePrec.getOutputScale())
           .setPrecision(outputScalePrec.getOutputPrecision())
           .setMode(mode)
@@ -281,20 +295,47 @@
     @Override
     public TypeProtos.MajorType getType(List<LogicalExpression> logicalExpressions, FunctionAttributes attributes) {
       int scale = 0;
-      int precision = 0;
 
       // Get the max scale and precision from the inputs
       for (LogicalExpression e : logicalExpressions) {
         scale = Math.max(scale, e.getMajorType().getScale());
-        precision = Math.max(precision, e.getMajorType().getPrecision());
       }
 
-      return (TypeProtos.MajorType.newBuilder()
-          .setMinorType(attributes.getReturnValue().getType().getMinorType())
+      return TypeProtos.MajorType.newBuilder()
+          .setMinorType(TypeProtos.MinorType.VARDECIMAL)
           .setScale(scale)
-          .setPrecision(38)
-          .setMode(TypeProtos.DataMode.REQUIRED)
-          .build());
+          .setPrecision(DRILL_REL_DATATYPE_SYSTEM.getMaxNumericPrecision())
+          .setMode(TypeProtos.DataMode.OPTIONAL)
+          .build();
+    }
+  }
+
+  /**
+   * Return type calculation implementation for functions with return type set as
+   * {@link org.apache.drill.exec.expr.annotations.FunctionTemplate.ReturnType#DECIMAL_AVG_AGGREGATE}.
+   * Resulting scale is calculated as the max of 6 and the scale of input.
+   * Resulting precision is max allowed numeric precision.
+   */
+  public static class DecimalAvgAggReturnTypeInference implements ReturnTypeInference {
+
+    public static final DecimalAvgAggReturnTypeInference INSTANCE = new DecimalAvgAggReturnTypeInference();
+
+    @Override
+    public TypeProtos.MajorType getType(List<LogicalExpression> logicalExpressions, FunctionAttributes attributes) {
+      int scale = 0;
+
+      // Get the max scale and precision from the inputs
+      for (LogicalExpression e : logicalExpressions) {
+        scale = Math.max(scale, e.getMajorType().getScale());
+      }
+
+      return TypeProtos.MajorType.newBuilder()
+          .setMinorType(TypeProtos.MinorType.VARDECIMAL)
+          .setScale(Math.min(Math.max(6, scale),
+              DRILL_REL_DATATYPE_SYSTEM.getMaxNumericScale()))
+          .setPrecision(DRILL_REL_DATATYPE_SYSTEM.getMaxNumericPrecision())
+          .setMode(TypeProtos.DataMode.OPTIONAL)
+          .build();
     }
   }
 
@@ -313,13 +354,17 @@
 
       assert logicalExpressions.size() == 2;
 
+      TypeProtos.MajorType leftMajorType = logicalExpressions.get(0).getMajorType();
+      TypeProtos.MajorType rightMajorType = logicalExpressions.get(1).getMajorType();
+
       DecimalScalePrecisionMulFunction outputScalePrec =
-          new DecimalScalePrecisionMulFunction(logicalExpressions.get(0).getMajorType().getPrecision(),
-              logicalExpressions.get(0).getMajorType().getScale(),
-              logicalExpressions.get(1).getMajorType().getPrecision(),
-              logicalExpressions.get(1).getMajorType().getScale());
+          new DecimalScalePrecisionMulFunction(
+              DecimalUtility.getDefaultPrecision(leftMajorType.getMinorType(), leftMajorType.getPrecision()),
+              leftMajorType.getScale(),
+              DecimalUtility.getDefaultPrecision(rightMajorType.getMinorType(), rightMajorType.getPrecision()),
+              rightMajorType.getScale());
       return TypeProtos.MajorType.newBuilder()
-          .setMinorType(DecimalUtility.getDecimalDataType(outputScalePrec.getOutputPrecision()))
+          .setMinorType(TypeProtos.MinorType.VARDECIMAL)
           .setScale(outputScalePrec.getOutputScale())
           .setPrecision(outputScalePrec.getOutputPrecision())
           .setMode(mode)
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index a937c7a700..ea943b2ec2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -409,7 +409,9 @@ public void clear() {
 
       // Check if the field exists.
       ValueVector v = fieldVectorMap.get(field.getName());
-      if (v == null || v.getClass() != clazz) {
+      // for the cases when fields have a different scale or precision,
+      // the new vector should be used to handle the value correctly
+      if (v == null || !v.getField().getType().equals(field.getType())) {
         // Field does not exist--add it to the map and the output container.
         v = TypeHelper.getNewVector(field, allocator, callBack);
         if (!clazz.isAssignableFrom(v.getClass())) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinUtils.java
index 036f3cdcdf..b6604d2725 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinUtils.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinUtils.java
@@ -36,6 +36,7 @@
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.resolver.TypeCastRules;
+import org.apache.drill.exec.util.DecimalUtility;
 
 import java.util.LinkedList;
 import java.util.List;
@@ -119,8 +120,11 @@ public static boolean checkCartesianJoin(RelNode relNode, List<Integer> leftKeys
    * @return true if implicit cast is allowed false otherwise
    */
   private static boolean allowImplicitCast(TypeProtos.MinorType input1, TypeProtos.MinorType input2) {
-    // allow implicit cast if both the input types are numeric
-    if (TypeCastRules.isNumericType(input1) && TypeCastRules.isNumericType(input2)) {
+    // allow implicit cast if both the input types are numeric and any of them is non-decimal
+    // or both of them are decimal
+    if (TypeCastRules.isNumericType(input1) && TypeCastRules.isNumericType(input2)
+        && ((!DecimalUtility.isDecimalType(input1) && !DecimalUtility.isDecimalType(input2))
+          || DecimalUtility.isDecimalType(input1) && DecimalUtility.isDecimalType(input2))) {
       return true;
     }
 
@@ -166,8 +170,9 @@ public static void addLeastRestrictiveCasts(LogicalExpression[] leftExpressions,
 
         // currently we only support implicit casts if the input types are numeric or varchar/varbinary
         if (!allowImplicitCast(rightType, leftType)) {
-          throw new DrillRuntimeException(String.format("Join only supports implicit casts between " +
-              "1. Numeric data\n 2. Varchar, Varbinary data 3. Date, Timestamp data " +
+          throw new DrillRuntimeException(String.format("Join only supports implicit casts between\n" +
+              "1. Numeric data (none of types is decimal or both of them are decimal)\n" +
+              "2. Varchar, Varbinary data\n3. Date, Timestamp data\n" +
               "Left type: %s, Right type: %s. Add explicit casts to avoid this error", leftType, rightType));
         }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinBatch.java
index c8b676e489..03bf58b36b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/NestedLoopJoinBatch.java
@@ -121,6 +121,11 @@
       "outgoing" /* write container */,
       EMIT_LEFT_CONSTANT, EMIT_LEFT);
 
+  private static final MappingSet SETUP_LEFT_MAPPING = new MappingSet("leftIndex" /* read index */, "outIndex" /* write index */,
+      "leftBatch" /* read container */,
+      "outgoing" /* write container */,
+      ClassGenerator.DEFAULT_CONSTANT_MAP, ClassGenerator.DEFAULT_SCALAR_MAP);
+
   protected NestedLoopJoinBatch(NestedLoopJoinPOP popConfig, FragmentContext context, RecordBatch left, RecordBatch right) throws OutOfMemoryException {
     super(popConfig, context, left, right);
     Preconditions.checkNotNull(left);
@@ -220,7 +225,7 @@ private boolean hasMore(IterOutcome outcome) {
    */
   private NestedLoopJoin setupWorker() throws IOException, ClassTransformationException, SchemaChangeException {
     final CodeGenerator<NestedLoopJoin> nLJCodeGenerator = CodeGenerator.get(
-        NestedLoopJoin.TEMPLATE_DEFINITION, context.getOptions());
+        SETUP_LEFT_MAPPING, NestedLoopJoin.TEMPLATE_DEFINITION, context.getOptions());
     nLJCodeGenerator.plainJavaCapable(true);
     // Uncomment out this line to debug the generated code.
     // nLJCodeGenerator.saveCodeForDebugging(true);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java
index cbc530b40b..aaf41f7c39 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.exec.planner;
 
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.util.BitSets;
 import org.apache.drill.common.expression.SchemaPath;
@@ -30,11 +32,9 @@
 import org.apache.drill.exec.store.dfs.MetadataContext;
 import org.apache.drill.exec.store.parquet.AbstractParquetGroupScan;
 import org.apache.drill.exec.store.parquet.ParquetReaderUtility;
-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.NullableFloat4Vector;
 import org.apache.drill.exec.vector.NullableFloat8Vector;
 import org.apache.drill.exec.vector.NullableIntVector;
@@ -48,6 +48,7 @@
 import org.apache.drill.exec.vector.NullableUInt4Vector;
 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.collect.Lists;
@@ -311,32 +312,21 @@ private void populatePruningVector(ValueVector v, int index, SchemaPath column,
         varBinaryVector.getMutator().setSafe(index, bytes, 0, bytes.length);
         return;
       }
-      case DECIMAL18: {
-        NullableDecimal18Vector decimalVector = (NullableDecimal18Vector) v;
+      case VARDECIMAL: {
+        NullableVarDecimalVector decimalVector = (NullableVarDecimalVector) v;
         Object s = groupScan.getPartitionValue(path, column, Object.class);
         byte[] bytes;
         if (s == null) {
           decimalVector.getMutator().setNull(index);
           return;
         } else if (s instanceof Integer) {
-          long value = DecimalUtility.getBigDecimalFromPrimitiveTypes(
-              (Integer) s,
-              majorType.getScale(),
-              majorType.getPrecision()).longValue();
-          decimalVector.getMutator().setSafe(index, value);
-          return;
+          bytes = Ints.toByteArray((int) s);
         } else if (s instanceof Long) {
-          long value = DecimalUtility.getBigDecimalFromPrimitiveTypes(
-              (Long) s,
-              majorType.getScale(),
-              majorType.getPrecision()).longValue();
-          decimalVector.getMutator().setSafe(index, value);
-          return;
+          bytes = Longs.toByteArray((long) s);
         } else {
           bytes = getBytes(type, s);
         }
-        long value = DecimalUtility.getBigDecimalFromByteArray(bytes, 0, bytes.length, majorType.getScale()).longValue();
-        decimalVector.getMutator().setSafe(index, value);
+        decimalVector.getMutator().setSafe(index, bytes, 0, bytes.length);
         return;
       }
       case DATE: {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java
index 96702ebf20..7b82ff315e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java
@@ -47,7 +47,6 @@
  */
 public abstract class DrillValuesRelBase extends Values implements DrillRelNode {
 
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillValuesRelBase.class);
   private static final ObjectMapper MAPPER = new ObjectMapper();
 
   protected final JSONOptions content;
@@ -169,12 +168,13 @@ private static void writeLiteral(RexLiteral literal, JsonOutput out) throws IOEx
         return;
 
       case DECIMAL:
+        // Converting exact decimal into double since values in the list may have different scales
+        // so the resulting scale wouldn't be calculated correctly
         if (isLiteralNull(literal)) {
           out.writeDoubleNull();
         } else {
           out.writeDouble(((BigDecimal) literal.getValue()).doubleValue());
         }
-        logger.warn("Converting exact decimal into approximate decimal. Should be fixed once full decimal support is implemented.");
         return;
 
       case VARCHAR:
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRel.java
index 21a4c90b3a..55cd7bfc22 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRel.java
@@ -23,6 +23,8 @@
 import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.BitSets;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.drill.common.expression.ExpressionPosition;
@@ -91,8 +93,13 @@ public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
       // For avg, stddev_pop, stddev_samp, var_pop and var_samp, the ReduceAggregatesRule is supposed
       // to convert them to use sum and count. Here, we make the cost of the original functions high
       // enough such that the planner does not choose them and instead chooses the rewritten functions.
-      if (name.equals("AVG") || name.equals("STDDEV_POP") || name.equals("STDDEV_SAMP")
-          || name.equals("VAR_POP") || name.equals("VAR_SAMP")) {
+      // Except when AVG, STDDEV_POP, STDDEV_SAMP, VAR_POP and VAR_SAMP are used with DECIMAL type.
+      if ((name.equals(SqlKind.AVG.name())
+            || name.equals(SqlKind.STDDEV_POP.name())
+            || name.equals(SqlKind.STDDEV_SAMP.name())
+            || name.equals(SqlKind.VAR_POP.name())
+            || name.equals(SqlKind.VAR_SAMP.name()))
+          && aggCall.getType().getSqlTypeName() != SqlTypeName.DECIMAL) {
         return planner.getCostFactory().makeHugeCost();
       }
     }
@@ -108,14 +115,13 @@ public static LogicalExpression toDrill(AggregateCall call, List<String> fn, Dri
 
     // for count(1).
     if (args.isEmpty()) {
-      args.add(new ValueExpressions.LongExpression(1l));
+      args.add(new ValueExpressions.LongExpression(1L));
     }
-    LogicalExpression expr = FunctionCallFactory.createExpression(call.getAggregation().getName().toLowerCase(), ExpressionPosition.UNKNOWN, args);
-    return expr;
+    return FunctionCallFactory.createExpression(call.getAggregation().getName().toLowerCase(), ExpressionPosition.UNKNOWN, args);
   }
 
-  public static DrillAggregateRel convert(GroupingAggregate groupBy, ConversionContext value)
-      throws InvalidRelException {
+  @SuppressWarnings("unused")
+  public static DrillAggregateRel convert(GroupingAggregate groupBy, ConversionContext value) {
     throw new UnsupportedOperationException();
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java
index 68beea250f..ab27c7a832 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java
@@ -62,10 +62,12 @@
 import org.apache.drill.exec.expr.holders.NullableTimeHolder;
 import org.apache.drill.exec.expr.holders.NullableTimeStampHolder;
 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.ValueHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.expr.holders.VarDecimalHolder;
 import org.apache.drill.exec.ops.UdfUtilities;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rex.RexBuilder;
@@ -243,6 +245,32 @@ public RexNode apply(ValueHolder output) {
                 TypeInferenceUtils.createCalciteTypeWithNullability(typeFactory, SqlTypeName.DECIMAL, newCall.getType().isNullable()),
                 false);
             }
+            case VARDECIMAL: {
+              DrillBuf buffer;
+              int start;
+              int end;
+              int scale;
+              int precision;
+              if (materializedExpr.getMajorType().getMode() == TypeProtos.DataMode.OPTIONAL) {
+                NullableVarDecimalHolder varDecimalHolder = (NullableVarDecimalHolder) output;
+                buffer = varDecimalHolder.buffer;
+                start = varDecimalHolder.start;
+                end = varDecimalHolder.end;
+                scale = varDecimalHolder.scale;
+                precision = varDecimalHolder.precision;
+              } else {
+                VarDecimalHolder varDecimalHolder = (VarDecimalHolder) output;
+                buffer = varDecimalHolder.buffer;
+                start = varDecimalHolder.start;
+                end = varDecimalHolder.end;
+                scale = varDecimalHolder.scale;
+                precision = varDecimalHolder.precision;
+              }
+              return rexBuilder.makeLiteral(
+                  org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(buffer, start, end - start, scale),
+                  typeFactory.createSqlType(SqlTypeName.DECIMAL, precision, scale),
+                  false);
+            }
             case DECIMAL28SPARSE: {
               DrillBuf buffer;
               int start;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
index ab073becfe..fd791dd7f3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
@@ -61,6 +61,8 @@
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.work.ExecErrorConstants;
 
+import static org.apache.drill.exec.planner.physical.PlannerSettings.ENABLE_DECIMAL_DATA_TYPE;
+
 /**
  * Utilities for Drill's planner.
  */
@@ -314,18 +316,13 @@ private LogicalExpression getDrillCastFunctionFromOptiq(RexCall call){
         int precision = call.getType().getPrecision();
         int scale = call.getType().getScale();
 
-        if (precision <= 9) {
-          castType = TypeProtos.MajorType.newBuilder().setMinorType(MinorType.DECIMAL9).setPrecision(precision).setScale(scale).build();
-        } else if (precision <= 18) {
-          castType = TypeProtos.MajorType.newBuilder().setMinorType(MinorType.DECIMAL18).setPrecision(precision).setScale(scale).build();
-        } else if (precision <= 28) {
-          // Inject a cast to SPARSE before casting to the dense type.
-          castType = TypeProtos.MajorType.newBuilder().setMinorType(MinorType.DECIMAL28SPARSE).setPrecision(precision).setScale(scale).build();
-        } else if (precision <= 38) {
-          castType = TypeProtos.MajorType.newBuilder().setMinorType(MinorType.DECIMAL38SPARSE).setPrecision(precision).setScale(scale).build();
-        } else {
-          throw new UnsupportedOperationException("Only Decimal types with precision range 0 - 38 is supported");
-        }
+        castType =
+            TypeProtos.MajorType
+                .newBuilder()
+                .setMinorType(MinorType.VARDECIMAL)
+                .setPrecision(precision)
+                .setScale(scale)
+                .build();
         break;
 
         case "INTERVAL_YEAR":
@@ -543,24 +540,21 @@ public LogicalExpression visitLiteral(RexLiteral literal) {
         return ValueExpressions.getInt(a);
 
       case DECIMAL:
-        /* TODO: Enable using Decimal literals once we have more functions implemented for Decimal
-         * For now continue using Double instead of decimals
-
-        int precision = ((BigDecimal) literal.getValue()).precision();
-        if (precision <= 9) {
-            return ValueExpressions.getDecimal9((BigDecimal)literal.getValue());
-        } else if (precision <= 18) {
-            return ValueExpressions.getDecimal18((BigDecimal)literal.getValue());
-        } else if (precision <= 28) {
-            return ValueExpressions.getDecimal28((BigDecimal)literal.getValue());
-        } else if (precision <= 38) {
-            return ValueExpressions.getDecimal38((BigDecimal)literal.getValue());
-        } */
-        if (isLiteralNull(literal)) {
-          return createNullExpr(MinorType.FLOAT8);
+        if (context.getPlannerSettings().getOptions()
+            .getBoolean(ENABLE_DECIMAL_DATA_TYPE.getOptionName())) {
+          if (isLiteralNull(literal)) {
+            return new TypedNullConstant(
+                Types.withScaleAndPrecision(
+                    MinorType.VARDECIMAL,
+                    TypeProtos.DataMode.OPTIONAL,
+                    literal.getType().getScale(),
+                    literal.getType().getPrecision()));
+          }
+          return ValueExpressions.getVarDecimal((BigDecimal) literal.getValue());
         }
         double dbl = ((BigDecimal) literal.getValue()).doubleValue();
-        logger.warn("Converting exact decimal into approximate decimal.  Should be fixed once decimal is implemented.");
+        logger.warn("Converting exact decimal into approximate decimal.\n" +
+            "Please enable decimal data types using `planner.enable_decimal_data_type`.");
         return ValueExpressions.getFloat8(dbl);
       case VARCHAR:
         if (isLiteralNull(literal)) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceAggregatesRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceAggregatesRule.java
index 496ac1a97e..6b0576cb46 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceAggregatesRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceAggregatesRule.java
@@ -238,6 +238,20 @@ private RexNode reduceAgg(
       return reduceSum(oldAggRel, oldCall, newCalls, aggCallMapping);
     }
     if (sqlAggFunction instanceof SqlAvgAggFunction) {
+      // for DECIMAL data types does not produce rewriting of complex calls,
+      // since SUM returns value with 38 precision and further handling of the value
+      // causes the loss of the scale
+      if (oldCall.getType().getSqlTypeName() == SqlTypeName.DECIMAL) {
+        return oldAggRel.getCluster().getRexBuilder().addAggCall(
+            oldCall,
+            oldAggRel.getGroupCount(),
+            oldAggRel.indicator,
+            newCalls,
+            aggCallMapping,
+            ImmutableList.of(getFieldType(
+                oldAggRel.getInput(),
+                oldCall.getArgList().get(0))));
+      }
       final SqlKind subtype = sqlAggFunction.getKind();
       switch (subtype) {
       case AVG:
@@ -253,7 +267,7 @@ private RexNode reduceAgg(
             oldAggRel, oldCall, true, true, newCalls, aggCallMapping,
             inputExprs);
       case STDDEV_SAMP:
-        // replace original STDDEV_POP(x) with
+        // replace original STDDEV_SAMP(x) with
         //   SQRT(
         //     (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x))
         //     / CASE COUNT(x) WHEN 1 THEN NULL ELSE COUNT(x) - 1 END)
@@ -268,7 +282,7 @@ private RexNode reduceAgg(
             oldAggRel, oldCall, true, false, newCalls, aggCallMapping,
             inputExprs);
       case VAR_SAMP:
-        // replace original VAR_POP(x) with
+        // replace original VAR_SAMP(x) with
         //     (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x))
         //     / CASE COUNT(x) WHEN 1 THEN NULL ELSE COUNT(x) - 1 END
         return reduceStddev(
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
index 69362d9392..2366b8ad37 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.planner.sql;
 
+import java.math.BigDecimal;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
@@ -559,6 +560,19 @@ public RexNode makeCast(RelDataType type, RexNode exp, boolean matchNullability)
       if (matchNullability) {
         return makeAbstractCast(type, exp);
       }
+      // for the case when BigDecimal literal has a scale or precision
+      // that differs from the value from specified RelDataType, cast cannot be removed
+      // TODO: remove this code when CALCITE-1468 is fixed
+      if (type.getSqlTypeName() == SqlTypeName.DECIMAL && exp instanceof RexLiteral) {
+        RexLiteral literal = (RexLiteral) exp;
+        Comparable value = literal.getValueAs(Comparable.class);
+        if (value instanceof BigDecimal) {
+          BigDecimal bigDecimal = (BigDecimal) value;
+          if (bigDecimal.scale() != type.getScale() || bigDecimal.precision() != type.getPrecision()) {
+            return makeAbstractCast(type, exp);
+          }
+        }
+      }
       return super.makeCast(type, exp, false);
     }
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java
index 078094b5ab..529546f54b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.planner.sql;
 
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
 import org.apache.calcite.avatica.util.TimeUnit;
@@ -45,12 +46,16 @@
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
+import org.apache.drill.common.util.CoreDecimalUtility;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
 import org.apache.drill.exec.expr.fn.DrillFuncHolder;
+import org.apache.drill.exec.planner.types.DrillRelDataTypeSystem;
 import org.apache.drill.exec.resolver.FunctionResolver;
 import org.apache.drill.exec.resolver.FunctionResolverFactory;
 import org.apache.drill.exec.resolver.TypeCastRules;
 
 import java.util.List;
+import java.util.Set;
 
 public class TypeInferenceUtils {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
@@ -69,6 +74,7 @@
       .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
       .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
       .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
+      .put(TypeProtos.MinorType.VARDECIMAL, SqlTypeName.DECIMAL)
       .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
       .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
       .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
@@ -110,6 +116,7 @@
       .put(SqlTypeName.INTERVAL_MINUTE, TypeProtos.MinorType.INTERVALDAY)
       .put(SqlTypeName.INTERVAL_MINUTE_SECOND, TypeProtos.MinorType.INTERVALDAY)
       .put(SqlTypeName.INTERVAL_SECOND, TypeProtos.MinorType.INTERVALDAY)
+      .put(SqlTypeName.DECIMAL, TypeProtos.MinorType.VARDECIMAL)
 
       // SqlTypeName.CHAR is the type for Literals in Calcite, Drill treats Literals as VARCHAR also
       .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
@@ -153,10 +160,10 @@
       .put("CONVERT_FROM", DrillDeferToExecSqlReturnTypeInference.INSTANCE)
 
       // Functions that return the same type
-      .put("LOWER", DrillSameSqlReturnTypeInference.INSTANCE)
-      .put("UPPER", DrillSameSqlReturnTypeInference.INSTANCE)
-      .put("INITCAP", DrillSameSqlReturnTypeInference.INSTANCE)
-      .put("REVERSE", DrillSameSqlReturnTypeInference.INSTANCE)
+      .put("LOWER", DrillSameSqlReturnTypeInference.THE_SAME_RETURN_TYPE)
+      .put("UPPER", DrillSameSqlReturnTypeInference.THE_SAME_RETURN_TYPE)
+      .put("INITCAP", DrillSameSqlReturnTypeInference.THE_SAME_RETURN_TYPE)
+      .put("REVERSE", DrillSameSqlReturnTypeInference.THE_SAME_RETURN_TYPE)
 
       // Window Functions
       // RANKING
@@ -174,8 +181,8 @@
       .put(SqlKind.LAG.name(), DrillLeadLagSqlReturnTypeInference.INSTANCE)
 
       // FIRST_VALUE, LAST_VALUE
-      .put(SqlKind.FIRST_VALUE.name(), DrillSameSqlReturnTypeInference.INSTANCE)
-      .put(SqlKind.LAST_VALUE.name(), DrillSameSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.FIRST_VALUE.name(), DrillSameSqlReturnTypeInference.THE_SAME_RETURN_TYPE)
+      .put(SqlKind.LAST_VALUE.name(), DrillSameSqlReturnTypeInference.THE_SAME_RETURN_TYPE)
 
       // Functions rely on DrillReduceAggregatesRule for expression simplification as opposed to getting evaluated directly
       .put(SqlKind.AVG.name(), DrillAvgAggSqlReturnTypeInference.INSTANCE)
@@ -183,6 +190,18 @@
       .put(SqlKind.STDDEV_SAMP.name(), DrillAvgAggSqlReturnTypeInference.INSTANCE)
       .put(SqlKind.VAR_POP.name(), DrillAvgAggSqlReturnTypeInference.INSTANCE)
       .put(SqlKind.VAR_SAMP.name(), DrillAvgAggSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.MIN.name(), DrillSameSqlReturnTypeInference.ALL_NULLABLE)
+      .put(SqlKind.MAX.name(), DrillSameSqlReturnTypeInference.ALL_NULLABLE)
+      .build();
+
+  /**
+   * Set of the decimal functions which return type cannot be determined exactly for some reasons at the current stage.
+   * For example functions which takes as an parameter scale or precision of return type.
+   */
+  private static final Set<String> SET_SCALE_DECIMAL_FUNCTIONS = ImmutableSet.<String> builder()
+      .add("ROUND")
+      .add("TRUNC")
+      .add("TRUNCATE")
       .build();
 
   /**
@@ -280,17 +299,27 @@ public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
                 factory.createSqlType(SqlTypeName.ANY),
                 true);
           }
+        } else if (SET_SCALE_DECIMAL_FUNCTIONS.contains(opBinding.getOperator().getName())
+            && getDrillTypeFromCalciteType(type) == TypeProtos.MinorType.VARDECIMAL) {
+          return factory.createTypeWithNullability(
+              factory.createSqlType(SqlTypeName.ANY),
+              true);
         }
       }
 
-      final DrillFuncHolder func = resolveDrillFuncHolder(opBinding, functions);
-      final RelDataType returnType = getReturnType(opBinding, func);
+      final FunctionCall functionCall = convertSqlOperatorBindingToFunctionCall(opBinding);
+      final DrillFuncHolder func = resolveDrillFuncHolder(opBinding, functions, functionCall);
+
+      final RelDataType returnType = getReturnType(opBinding,
+          func.getReturnType(functionCall.args), func.getNullHandling());
+
       return returnType.getSqlTypeName() == SqlTypeName.VARBINARY
           ? createCalciteTypeWithNullability(factory, SqlTypeName.ANY, returnType.isNullable())
               : returnType;
     }
 
-    private static RelDataType getReturnType(final SqlOperatorBinding opBinding, final DrillFuncHolder func) {
+    private static RelDataType getReturnType(final SqlOperatorBinding opBinding,
+        final TypeProtos.MajorType returnType, FunctionTemplate.NullHandling nullHandling) {
       final RelDataTypeFactory factory = opBinding.getTypeFactory();
 
       // least restrictive type (nullable ANY type)
@@ -298,7 +327,6 @@ private static RelDataType getReturnType(final SqlOperatorBinding opBinding, fin
           factory.createSqlType(SqlTypeName.ANY),
           true);
 
-      final TypeProtos.MajorType returnType = func.getReturnType();
       if (UNKNOWN_TYPE.equals(returnType)) {
         return nullableAnyType;
       }
@@ -317,7 +345,7 @@ private static RelDataType getReturnType(final SqlOperatorBinding opBinding, fin
           break;
 
         case REQUIRED:
-          switch (func.getNullHandling()) {
+          switch (nullHandling) {
             case INTERNAL:
               isNullable = false;
               break;
@@ -342,10 +370,7 @@ private static RelDataType getReturnType(final SqlOperatorBinding opBinding, fin
           throw new UnsupportedOperationException();
       }
 
-      return createCalciteTypeWithNullability(
-          factory,
-          sqlTypeName,
-          isNullable);
+      return convertToCalciteType(factory, returnType, isNullable);
     }
   }
 
@@ -380,15 +405,36 @@ public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
             isNullable);
       }
 
+      // Determines SqlTypeName of the result.
+      // For the case when input may be implicitly casted to BIGINT, the type of result is BIGINT.
+      // Else for the case when input may be implicitly casted to FLOAT4, the type of result is DOUBLE.
+      // Else for the case when input may be implicitly casted to VARDECIMAL, the type of result is DECIMAL
+      // with the same scale as input and max allowed numeric precision.
+      // Else for the case when input may be implicitly casted to FLOAT8, the type of result is DOUBLE.
+      // When none of these conditions is satisfied, error is thrown.
+      // This order of checks is caused by the order of types in ResolverTypePrecedence.precedenceMap
       final RelDataType operandType = opBinding.getOperandType(0);
       final TypeProtos.MinorType inputMinorType = getDrillTypeFromCalciteType(operandType);
-      if(TypeCastRules.getLeastRestrictiveType(Lists.newArrayList(inputMinorType, TypeProtos.MinorType.BIGINT))
+      if (TypeCastRules.getLeastRestrictiveType(Lists.newArrayList(inputMinorType, TypeProtos.MinorType.BIGINT))
           == TypeProtos.MinorType.BIGINT) {
         return createCalciteTypeWithNullability(
             factory,
             SqlTypeName.BIGINT,
             isNullable);
-      } else if(TypeCastRules.getLeastRestrictiveType(Lists.newArrayList(inputMinorType, TypeProtos.MinorType.FLOAT8))
+      } else if (TypeCastRules.getLeastRestrictiveType(Lists.newArrayList(inputMinorType, TypeProtos.MinorType.FLOAT4))
+          == TypeProtos.MinorType.FLOAT4) {
+        return createCalciteTypeWithNullability(
+            factory,
+            SqlTypeName.DOUBLE,
+            isNullable);
+      } else if (TypeCastRules.getLeastRestrictiveType(Lists.newArrayList(inputMinorType, TypeProtos.MinorType.VARDECIMAL))
+          == TypeProtos.MinorType.VARDECIMAL) {
+        RelDataType sqlType = factory.createSqlType(SqlTypeName.DECIMAL,
+          DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM.getMaxNumericPrecision(),
+          Math.min(operandType.getScale(),
+            DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM.getMaxNumericScale()));
+        return factory.createTypeWithNullability(sqlType, isNullable);
+      } else if (TypeCastRules.getLeastRestrictiveType(Lists.newArrayList(inputMinorType, TypeProtos.MinorType.FLOAT8))
           == TypeProtos.MinorType.FLOAT8) {
         return createCalciteTypeWithNullability(
             factory,
@@ -668,10 +714,21 @@ public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
   }
 
   private static class DrillSameSqlReturnTypeInference implements SqlReturnTypeInference {
-    private static final DrillSameSqlReturnTypeInference INSTANCE = new DrillSameSqlReturnTypeInference();
+    private static final DrillSameSqlReturnTypeInference THE_SAME_RETURN_TYPE = new DrillSameSqlReturnTypeInference(true);
+    private static final DrillSameSqlReturnTypeInference ALL_NULLABLE = new DrillSameSqlReturnTypeInference(false);
+
+    private final boolean preserveNullability;
+
+    public DrillSameSqlReturnTypeInference(boolean preserveNullability) {
+      this.preserveNullability = preserveNullability;
+    }
+
     @Override
     public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
-      return opBinding.getOperandType(0);
+      if (preserveNullability) {
+        return opBinding.getOperandType(0);
+      }
+      return opBinding.getTypeFactory().createTypeWithNullability(opBinding.getOperandType(0), true);
     }
   }
 
@@ -679,27 +736,72 @@ public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
     private static final DrillAvgAggSqlReturnTypeInference INSTANCE = new DrillAvgAggSqlReturnTypeInference();
     @Override
     public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
-      final boolean isNullable = opBinding.getGroupCount() == 0 || opBinding.hasFilter() || opBinding.getOperandType(0).isNullable();
-      return createCalciteTypeWithNullability(
-          opBinding.getTypeFactory(),
-          SqlTypeName.DOUBLE,
-          isNullable);
+      final RelDataTypeFactory factory = opBinding.getTypeFactory();
+      // If there is group-by and the imput type is Non-nullable,
+      // the output is Non-nullable;
+      // Otherwise, the output is nullable.
+      final boolean isNullable = opBinding.getGroupCount() == 0
+          || opBinding.getOperandType(0).isNullable();
+
+      if (getDrillTypeFromCalciteType(opBinding.getOperandType(0)) == TypeProtos.MinorType.LATE) {
+        return createCalciteTypeWithNullability(
+            factory,
+            SqlTypeName.ANY,
+            isNullable);
+      }
+
+      // Determines SqlTypeName of the result.
+      // For the case when input may be implicitly casted to FLOAT4, the type of result is DOUBLE.
+      // Else for the case when input may be implicitly casted to VARDECIMAL, the type of result is DECIMAL
+      // with scale max(6, input) and max allowed numeric precision.
+      // Else for the case when input may be implicitly casted to FLOAT8, the type of result is DOUBLE.
+      // When none of these conditions is satisfied, error is thrown.
+      // This order of checks is caused by the order of types in ResolverTypePrecedence.precedenceMap
+      final RelDataType operandType = opBinding.getOperandType(0);
+      final TypeProtos.MinorType inputMinorType = getDrillTypeFromCalciteType(operandType);
+      if (TypeCastRules.getLeastRestrictiveType(Lists.newArrayList(inputMinorType, TypeProtos.MinorType.FLOAT4))
+          == TypeProtos.MinorType.FLOAT4) {
+        return createCalciteTypeWithNullability(
+            factory,
+            SqlTypeName.DOUBLE,
+            isNullable);
+      } else if (TypeCastRules.getLeastRestrictiveType(Lists.newArrayList(inputMinorType, TypeProtos.MinorType.VARDECIMAL))
+          == TypeProtos.MinorType.VARDECIMAL) {
+        RelDataType sqlType = factory.createSqlType(SqlTypeName.DECIMAL,
+            DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM.getMaxNumericPrecision(),
+            Math.min(Math.max(6, operandType.getScale()),
+                DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM.getMaxNumericScale()));
+        return factory.createTypeWithNullability(sqlType, isNullable);
+      } else if (TypeCastRules.getLeastRestrictiveType(Lists.newArrayList(inputMinorType, TypeProtos.MinorType.FLOAT8))
+          == TypeProtos.MinorType.FLOAT8) {
+        return createCalciteTypeWithNullability(
+            factory,
+            SqlTypeName.DOUBLE,
+            isNullable);
+      } else {
+        throw UserException
+            .functionError()
+            .message(String.format("%s does not support operand types (%s)",
+                opBinding.getOperator().getName(),
+                opBinding.getOperandType(0).getSqlTypeName()))
+            .build(logger);
+      }
     }
   }
 
-  private static DrillFuncHolder resolveDrillFuncHolder(final SqlOperatorBinding opBinding, final List<DrillFuncHolder> functions) {
-    final FunctionCall functionCall = convertSqlOperatorBindingToFunctionCall(opBinding);
+  private static DrillFuncHolder resolveDrillFuncHolder(final SqlOperatorBinding opBinding,
+      final List<DrillFuncHolder> functions, FunctionCall functionCall) {
     final FunctionResolver functionResolver = FunctionResolverFactory.getResolver(functionCall);
     final DrillFuncHolder func = functionResolver.getBestMatch(functions, functionCall);
 
     // Throw an exception
     // if no DrillFuncHolder matched for the given list of operand types
-    if(func == null) {
-      String operandTypes = "";
-      for(int i = 0; i < opBinding.getOperandCount(); ++i) {
-        operandTypes += opBinding.getOperandType(i).getSqlTypeName();
-        if(i < opBinding.getOperandCount() - 1) {
-          operandTypes += ",";
+    if (func == null) {
+      StringBuilder operandTypes = new StringBuilder();
+      for (int i = 0; i < opBinding.getOperandCount(); ++i) {
+        operandTypes.append(opBinding.getOperandType(i).getSqlTypeName());
+        if (i < opBinding.getOperandCount() - 1) {
+          operandTypes.append(",");
         }
       }
 
@@ -707,7 +809,7 @@ private static DrillFuncHolder resolveDrillFuncHolder(final SqlOperatorBinding o
           .functionError()
           .message(String.format("%s does not support operand types (%s)",
               opBinding.getOperator().getName(),
-              operandTypes))
+              operandTypes.toString()))
           .build(logger);
     }
     return func;
@@ -766,6 +868,25 @@ public static RelDataType createCalciteTypeWithNullability(RelDataTypeFactory ty
     return typeFactory.createTypeWithNullability(type, isNullable);
   }
 
+  /**
+   * Creates a RelDataType using specified RelDataTypeFactory which corresponds to specified TypeProtos.MajorType.
+   *
+   * @param typeFactory RelDataTypeFactory used to create the RelDataType
+   * @param drillType   the given TypeProtos.MajorType
+   * @param isNullable  nullability of the resulting type
+   * @return RelDataType which corresponds to specified TypeProtos.MajorType
+   */
+  public static RelDataType convertToCalciteType(RelDataTypeFactory typeFactory,
+                                                 TypeProtos.MajorType drillType, boolean isNullable) {
+    SqlTypeName sqlTypeName = getCalciteTypeFromDrillType(drillType.getMinorType());
+    if (sqlTypeName == SqlTypeName.DECIMAL) {
+      return typeFactory.createTypeWithNullability(
+          typeFactory.createSqlType(sqlTypeName, drillType.getPrecision(),
+              drillType.getScale()), isNullable);
+    }
+    return createCalciteTypeWithNullability(typeFactory, sqlTypeName, isNullable);
+  }
+
   /**
    * Given a SqlOperatorBinding, convert it to FunctionCall
    * @param  opBinding    the given SqlOperatorBinding
@@ -777,22 +898,28 @@ public static FunctionCall convertSqlOperatorBindingToFunctionCall(final SqlOper
     for (int i = 0; i < opBinding.getOperandCount(); ++i) {
       final RelDataType type = opBinding.getOperandType(i);
       final TypeProtos.MinorType minorType = getDrillTypeFromCalciteType(type);
-      final TypeProtos.MajorType majorType;
-      if (type.isNullable()) {
-        majorType = Types.optional(minorType);
-      } else {
-        majorType = Types.required(minorType);
+      TypeProtos.DataMode dataMode =
+          type.isNullable() ? TypeProtos.DataMode.OPTIONAL : TypeProtos.DataMode.REQUIRED;
+
+      TypeProtos.MajorType.Builder builder =
+          TypeProtos.MajorType.newBuilder()
+              .setMode(dataMode)
+              .setMinorType(minorType);
+
+      if (CoreDecimalUtility.isDecimalType(minorType)) {
+        builder
+            .setScale(type.getScale())
+            .setPrecision(type.getPrecision());
       }
 
-      args.add(new MajorTypeInLogicalExpression(majorType));
+      args.add(new MajorTypeInLogicalExpression(builder.build()));
     }
 
     final String drillFuncName = FunctionCallFactory.replaceOpWithFuncName(opBinding.getOperator().getName());
-    final FunctionCall functionCall = new FunctionCall(
+    return new FunctionCall(
         drillFuncName,
         args,
         ExpressionPosition.UNKNOWN);
-    return functionCall;
   }
 
   /**
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java
index 03d5f75f01..8031609b89 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java
@@ -30,9 +30,6 @@
 import org.apache.calcite.rel.logical.LogicalSort;
 import org.apache.calcite.rel.logical.LogicalUnion;
 import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.types.TypeProtos;
@@ -62,7 +59,6 @@
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FindLimit0Visitor.class);
 
   // Some types are excluded in this set:
-  // + DECIMAL type is not fully supported in general.
   // + VARBINARY is not fully tested.
   // + MAP, ARRAY are currently not exposed to the planner.
   // + TINYINT, SMALLINT are defined in the Drill type system but have been turned off for now.
@@ -77,7 +73,7 @@
               SqlTypeName.INTERVAL_MONTH, SqlTypeName.INTERVAL_DAY, SqlTypeName.INTERVAL_DAY_HOUR,
               SqlTypeName.INTERVAL_DAY_MINUTE, SqlTypeName.INTERVAL_DAY_SECOND, SqlTypeName.INTERVAL_HOUR,
               SqlTypeName.INTERVAL_HOUR_MINUTE, SqlTypeName.INTERVAL_HOUR_SECOND, SqlTypeName.INTERVAL_MINUTE,
-              SqlTypeName.INTERVAL_MINUTE_SECOND, SqlTypeName.INTERVAL_SECOND, SqlTypeName.CHAR)
+              SqlTypeName.INTERVAL_MINUTE_SECOND, SqlTypeName.INTERVAL_SECOND, SqlTypeName.CHAR, SqlTypeName.DECIMAL)
           .build();
 
   /**
@@ -101,7 +97,10 @@ public static DrillRel getDirectScanRelIfFullySchemaed(RelNode rel) {
             .setMode(field.getType().isNullable() ? TypeProtos.DataMode.OPTIONAL : TypeProtos.DataMode.REQUIRED)
             .setMinorType(TypeInferenceUtils.getDrillTypeFromCalciteType(sqlTypeName));
 
-        if (TypeInferenceUtils.isScalarStringType(sqlTypeName)) {
+        if (sqlTypeName == SqlTypeName.DECIMAL) {
+          builder.setScale(field.getType().getScale());
+          builder.setPrecision(field.getType().getPrecision());
+        } else if (TypeInferenceUtils.isScalarStringType(sqlTypeName)) {
           builder.setPrecision(field.getType().getPrecision());
         }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/DrillRelDataTypeSystem.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/DrillRelDataTypeSystem.java
index 7f3db66710..c32a28dba7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/DrillRelDataTypeSystem.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/DrillRelDataTypeSystem.java
@@ -19,7 +19,6 @@
 
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
-import org.apache.calcite.sql.type.SqlTypeName;
 
 public class DrillRelDataTypeSystem extends RelDataTypeSystemImpl {
 
diff --git a/common/src/main/java/org/apache/drill/common/util/DecimalScalePrecisionAddFunction.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/decimal/DecimalScalePrecisionAddFunction.java
similarity index 82%
rename from common/src/main/java/org/apache/drill/common/util/DecimalScalePrecisionAddFunction.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/decimal/DecimalScalePrecisionAddFunction.java
index 77a2e7d942..138f97f638 100644
--- a/common/src/main/java/org/apache/drill/common/util/DecimalScalePrecisionAddFunction.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/decimal/DecimalScalePrecisionAddFunction.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.common.util;
+package org.apache.drill.exec.planner.types.decimal;
 
 public class DecimalScalePrecisionAddFunction extends DrillBaseComputeScalePrecision {
 
@@ -31,10 +31,6 @@ public void computeScalePrecision(int leftPrecision, int leftScale, int rightPre
 
     outputPrecision = (outputScale + maxResultIntegerDigits);
 
-    // If we are beyond the maximum precision range, cut down the fractional part
-    if (outputPrecision > 38) {
-      outputPrecision = 38;
-      outputScale = (outputPrecision - maxResultIntegerDigits >= 0) ? (outputPrecision - maxResultIntegerDigits) : 0;
-    }
+    checkPrecisionRange();
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/decimal/DecimalScalePrecisionDivideFunction.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/decimal/DecimalScalePrecisionDivideFunction.java
new file mode 100644
index 0000000000..eb79d44ae0
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/decimal/DecimalScalePrecisionDivideFunction.java
@@ -0,0 +1,38 @@
+/*
+ * 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.exec.planner.types.decimal;
+
+import static org.apache.drill.exec.planner.types.DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM;
+
+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
+    int maxResultIntegerDigits = Math.min(leftPrecision - leftScale + rightScale, MAX_NUMERIC_PRECISION);
+    outputScale = Math.max(6, leftScale + rightPrecision + 1);
+    outputScale = Math.min(outputScale, MAX_NUMERIC_PRECISION - maxResultIntegerDigits);
+    outputScale = Math.min(outputScale, DRILL_REL_DATATYPE_SYSTEM.getMaxNumericScale());
+    outputPrecision = maxResultIntegerDigits + outputScale;
+  }
+}
diff --git a/common/src/main/java/org/apache/drill/common/util/DecimalScalePrecisionModFunction.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/decimal/DecimalScalePrecisionModFunction.java
similarity index 87%
rename from common/src/main/java/org/apache/drill/common/util/DecimalScalePrecisionModFunction.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/decimal/DecimalScalePrecisionModFunction.java
index 075a50bd26..6901580eb2 100644
--- a/common/src/main/java/org/apache/drill/common/util/DecimalScalePrecisionModFunction.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/decimal/DecimalScalePrecisionModFunction.java
@@ -15,8 +15,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.common.util;
+package org.apache.drill.exec.planner.types.decimal;
 
+import static org.apache.drill.exec.planner.types.DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM;
 
 public class DecimalScalePrecisionModFunction extends DrillBaseComputeScalePrecision {
 
@@ -31,7 +32,7 @@ public void computeScalePrecision(int leftPrecision, int leftScale, int rightPre
     outputScale = Math.max(leftScale, rightScale);
     int leftIntegerDigits = leftPrecision - leftScale;
 
-    outputPrecision = CoreDecimalUtility.getPrecisionRange(outputScale + leftIntegerDigits);
+    outputPrecision = DRILL_REL_DATATYPE_SYSTEM.getMaxNumericPrecision();
 
     if (outputScale + leftIntegerDigits > outputPrecision) {
       outputScale = outputPrecision - leftIntegerDigits;
diff --git a/common/src/main/java/org/apache/drill/common/util/DecimalScalePrecisionMulFunction.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/decimal/DecimalScalePrecisionMulFunction.java
similarity index 77%
rename from common/src/main/java/org/apache/drill/common/util/DecimalScalePrecisionMulFunction.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/decimal/DecimalScalePrecisionMulFunction.java
index 3caed6a454..2a8b22b9d3 100644
--- a/common/src/main/java/org/apache/drill/common/util/DecimalScalePrecisionMulFunction.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/decimal/DecimalScalePrecisionMulFunction.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.common.util;
+package org.apache.drill.exec.planner.types.decimal;
 
 /*
  * Here we compute the output scale and precision of the multiply function.
@@ -32,15 +32,10 @@ public DecimalScalePrecisionMulFunction(int leftPrecision, int leftScale, int ri
   public void computeScalePrecision(int leftPrecision, int leftScale, int rightPrecision, int rightScale) {
     // compute the output scale and precision here
     outputScale = leftScale + rightScale;
-    int integerDigits = (leftPrecision - leftScale) + (rightPrecision - rightScale);
 
-    outputPrecision = integerDigits + outputScale;
+    outputPrecision = leftPrecision + rightPrecision;
 
-    // If we are beyond the maximum precision range, cut down the fractional part
-    if (outputPrecision > 38) {
-      outputPrecision = 38;
-      outputScale = (outputPrecision - integerDigits >= 0) ? (outputPrecision - integerDigits) : 0;
-    }
+    checkPrecisionRange();
   }
 }
 
diff --git a/common/src/main/java/org/apache/drill/common/util/DrillBaseComputeScalePrecision.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/decimal/DrillBaseComputeScalePrecision.java
similarity index 69%
rename from common/src/main/java/org/apache/drill/common/util/DrillBaseComputeScalePrecision.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/decimal/DrillBaseComputeScalePrecision.java
index 4f11351787..1c3c186670 100644
--- a/common/src/main/java/org/apache/drill/common/util/DrillBaseComputeScalePrecision.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/decimal/DrillBaseComputeScalePrecision.java
@@ -15,9 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.common.util;
+package org.apache.drill.exec.planner.types.decimal;
+
+import static org.apache.drill.exec.planner.types.DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM;
 
 public abstract class DrillBaseComputeScalePrecision {
+  protected final static int MAX_NUMERIC_PRECISION = DRILL_REL_DATATYPE_SYSTEM.getMaxNumericPrecision();
+
   protected int outputScale = 0;
   protected int outputPrecision = 0;
 
@@ -34,4 +38,15 @@ public int getOutputScale() {
   public int getOutputPrecision() {
     return outputPrecision;
   }
+
+  /**
+   * Cuts down the fractional part if the current precision
+   * exceeds the maximum precision range.
+   */
+  protected void checkPrecisionRange() {
+    if (outputPrecision > MAX_NUMERIC_PRECISION) {
+      outputScale = outputScale - (outputPrecision - MAX_NUMERIC_PRECISION);
+      outputPrecision = MAX_NUMERIC_PRECISION;
+    }
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
index c46efaff27..03e9249ccf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
@@ -136,10 +136,9 @@ public void transferOut(VectorContainer containerOut) {
   public <T extends ValueVector> T addOrGet(final MaterializedField field, final SchemaChangeCallBack callBack) {
     final TypedFieldId id = getValueVectorId(SchemaPath.getSimplePath(field.getName()));
     final ValueVector vector;
-    final Class<?> clazz = TypeHelper.getValueVectorClass(field.getType().getMinorType(), field.getType().getMode());
     if (id != null) {
       vector = getValueAccessorById(id.getFieldIds()).getValueVector();
-      if (id.getFieldIds().length == 1 && clazz != null && !clazz.isAssignableFrom(vector.getClass())) {
+      if (id.getFieldIds().length == 1 && !vector.getField().getType().equals(field.getType())) {
         final ValueVector newVector = TypeHelper.getNewVector(field, this.getAllocator(), callBack);
         replace(vector, newVector);
         return (T) newVector;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ExactFunctionResolver.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ExactFunctionResolver.java
index 73776b1faa..ddbd8039c7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ExactFunctionResolver.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ExactFunctionResolver.java
@@ -27,7 +27,6 @@
 
 public class ExactFunctionResolver implements FunctionResolver {
 
-
   /*
    * This function resolves the input call to a func holder only if all
    * the input argument types match exactly with the func holder arguments. This is used when we
@@ -37,17 +36,17 @@
   @Override
   public DrillFuncHolder getBestMatch(List<DrillFuncHolder> methods, FunctionCall call) {
 
-    int currcost;
+    int currCost;
 
-    for (DrillFuncHolder h : methods) {
-      final List<TypeProtos.MajorType> argumentTypes = Lists.newArrayList();
-      for (LogicalExpression expression : call.args) {
-        argumentTypes.add(expression.getMajorType());
-      }
-      currcost = TypeCastRules.getCost(argumentTypes, h);
+    final List<TypeProtos.MajorType> argumentTypes = Lists.newArrayList();
+    for (LogicalExpression expression : call.args) {
+      argumentTypes.add(expression.getMajorType());
+    }
 
+    for (DrillFuncHolder h : methods) {
+      currCost = TypeCastRules.getCost(argumentTypes, h);
       // Return if we found a function that has an exact match with the input arguments
-      if (currcost  == 0){
+      if (currCost == 0) {
         return h;
       }
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java
index 5ec36c77c8..247594f68d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/ResolverTypePrecedence.java
@@ -42,7 +42,7 @@
      * tinyint could be promoted into int; but int could NOT be promoted into tinyint (due to possible precision loss).
      */
     int i = 0;
-    precedenceMap = new HashMap<MinorType, Integer>();
+    precedenceMap = new HashMap<>();
     precedenceMap.put(MinorType.NULL, i += 2);       // NULL is legal to implicitly be promoted to any other type
     precedenceMap.put(MinorType.FIXEDBINARY, i += 2); // Fixed-length is promoted to var length
     precedenceMap.put(MinorType.VARBINARY, i += 2);
@@ -67,6 +67,7 @@
     precedenceMap.put(MinorType.DECIMAL28SPARSE, i += 2);
     precedenceMap.put(MinorType.DECIMAL38DENSE, i += 2);
     precedenceMap.put(MinorType.DECIMAL38SPARSE, i += 2);
+    precedenceMap.put(MinorType.VARDECIMAL, i += 2);
     precedenceMap.put(MinorType.FLOAT8, i += 2);
     precedenceMap.put(MinorType.DATE, i += 2);
     precedenceMap.put(MinorType.TIMESTAMP, i += 2);
@@ -106,12 +107,7 @@
     rule.add(MinorType.UINT2);
     rule.add(MinorType.UINT4);
     rule.add(MinorType.UINT8);
-    rule.add(MinorType.DECIMAL9);
-    rule.add(MinorType.DECIMAL18);
-    rule.add(MinorType.DECIMAL28SPARSE);
-    rule.add(MinorType.DECIMAL28DENSE);
-    rule.add(MinorType.DECIMAL38SPARSE);
-    rule.add(MinorType.DECIMAL38DENSE);
+    rule.add(MinorType.VARDECIMAL);
     rule.add(MinorType.MONEY);
     rule.add(MinorType.FLOAT4);
     rule.add(MinorType.FLOAT8);
@@ -138,6 +134,34 @@
     rule.add(MinorType.VARCHAR);
     secondaryImplicitCastRules.put(MinorType.VARBINARY, rule);
 
-  }
+    rule = new HashSet<>();
+
+    // Be able to implicitly cast to VARDECIMAL
+    rule.add(MinorType.FLOAT8);
+    secondaryImplicitCastRules.put(MinorType.VARDECIMAL, rule);
+
+    rule = new HashSet<>();
+
+    // Be able to implicitly cast to DECIMAL9
+    rule.add(MinorType.VARDECIMAL);
+    secondaryImplicitCastRules.put(MinorType.DECIMAL9, rule);
+
+    rule = new HashSet<>();
+
+    // Be able to implicitly cast to DECIMAL18
+    rule.add(MinorType.VARDECIMAL);
+    secondaryImplicitCastRules.put(MinorType.DECIMAL18, rule);
+
+    rule = new HashSet<>();
 
+    // Be able to implicitly cast to DECIMAL28SPARSE
+    rule.add(MinorType.VARDECIMAL);
+    secondaryImplicitCastRules.put(MinorType.DECIMAL28SPARSE, rule);
+
+    rule = new HashSet<>();
+
+    // Be able to implicitly cast to DECIMAL38SPARSE
+    rule.add(MinorType.VARDECIMAL);
+    secondaryImplicitCastRules.put(MinorType.DECIMAL38SPARSE, rule);
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
index 0105ba0153..93bcf58dfe 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
@@ -30,11 +30,12 @@
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
 import org.apache.drill.exec.expr.fn.DrillFuncHolder;
-import org.apache.drill.exec.util.DecimalUtility;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
+import static org.apache.drill.exec.planner.types.DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM;
+
 public class TypeCastRules {
 
   private static Map<MinorType, Set<MinorType>> rules;
@@ -67,6 +68,7 @@ private static void initTypeRules() {
     rule.add(MinorType.DECIMAL28DENSE);
     rule.add(MinorType.DECIMAL38SPARSE);
     rule.add(MinorType.DECIMAL38DENSE);
+    rule.add(MinorType.VARDECIMAL);
     rule.add(MinorType.MONEY);
     rule.add(MinorType.FLOAT4);
     rule.add(MinorType.FLOAT8);
@@ -95,6 +97,7 @@ private static void initTypeRules() {
     rule.add(MinorType.DECIMAL28DENSE);
     rule.add(MinorType.DECIMAL38SPARSE);
     rule.add(MinorType.DECIMAL38DENSE);
+    rule.add(MinorType.VARDECIMAL);
     rule.add(MinorType.MONEY);
     rule.add(MinorType.FLOAT4);
     rule.add(MinorType.FLOAT8);
@@ -123,6 +126,7 @@ private static void initTypeRules() {
     rule.add(MinorType.DECIMAL28DENSE);
     rule.add(MinorType.DECIMAL38SPARSE);
     rule.add(MinorType.DECIMAL38DENSE);
+    rule.add(MinorType.VARDECIMAL);
     rule.add(MinorType.MONEY);
     rule.add(MinorType.FLOAT4);
     rule.add(MinorType.FLOAT8);
@@ -151,6 +155,7 @@ private static void initTypeRules() {
     rule.add(MinorType.DECIMAL28DENSE);
     rule.add(MinorType.DECIMAL38SPARSE);
     rule.add(MinorType.DECIMAL38DENSE);
+    rule.add(MinorType.VARDECIMAL);
     rule.add(MinorType.MONEY);
     rule.add(MinorType.FLOAT4);
     rule.add(MinorType.FLOAT8);
@@ -179,6 +184,7 @@ private static void initTypeRules() {
     rule.add(MinorType.DECIMAL28DENSE);
     rule.add(MinorType.DECIMAL38SPARSE);
     rule.add(MinorType.DECIMAL38DENSE);
+    rule.add(MinorType.VARDECIMAL);
     rule.add(MinorType.MONEY);
     rule.add(MinorType.FLOAT4);
     rule.add(MinorType.FLOAT8);
@@ -207,6 +213,7 @@ private static void initTypeRules() {
     rule.add(MinorType.DECIMAL28DENSE);
     rule.add(MinorType.DECIMAL38SPARSE);
     rule.add(MinorType.DECIMAL38DENSE);
+    rule.add(MinorType.VARDECIMAL);
     rule.add(MinorType.MONEY);
     rule.add(MinorType.FLOAT4);
     rule.add(MinorType.FLOAT8);
@@ -221,117 +228,30 @@ private static void initTypeRules() {
 
     /** DECIMAL9 cast able from **/
     rule = new HashSet<>();
-    rule.add(MinorType.TINYINT);
-    rule.add(MinorType.SMALLINT);
-    rule.add(MinorType.INT);
-    rule.add(MinorType.BIGINT);
-    rule.add(MinorType.UINT1);
-    rule.add(MinorType.UINT2);
-    rule.add(MinorType.UINT4);
-    rule.add(MinorType.UINT8);
     rule.add(MinorType.DECIMAL9);
-    rule.add(MinorType.DECIMAL18);
-    rule.add(MinorType.DECIMAL28SPARSE);
-    rule.add(MinorType.DECIMAL28DENSE);
-    rule.add(MinorType.DECIMAL38SPARSE);
-    rule.add(MinorType.DECIMAL38DENSE);
-    rule.add(MinorType.MONEY);
-    rule.add(MinorType.FLOAT4);
-    rule.add(MinorType.FLOAT8);
-    rule.add(MinorType.BIT);
-    rule.add(MinorType.FIXEDCHAR);
-    rule.add(MinorType.FIXED16CHAR);
-    rule.add(MinorType.FIXEDBINARY);
-    rule.add(MinorType.VARCHAR);
-    rule.add(MinorType.VAR16CHAR);
-    rule.add(MinorType.VARBINARY);
+    rule.add(MinorType.VARDECIMAL);
     rules.put(MinorType.DECIMAL9, rule);
 
     /** DECIMAL18 cast able from **/
     rule = new HashSet<>();
-    rule.add(MinorType.TINYINT);
-    rule.add(MinorType.SMALLINT);
-    rule.add(MinorType.INT);
-    rule.add(MinorType.BIGINT);
-    rule.add(MinorType.UINT1);
-    rule.add(MinorType.UINT2);
-    rule.add(MinorType.UINT4);
-    rule.add(MinorType.UINT8);
-    rule.add(MinorType.DECIMAL9);
     rule.add(MinorType.DECIMAL18);
-    rule.add(MinorType.DECIMAL28SPARSE);
-    rule.add(MinorType.DECIMAL28DENSE);
-    rule.add(MinorType.DECIMAL38SPARSE);
-    rule.add(MinorType.DECIMAL38DENSE);
-    rule.add(MinorType.MONEY);
-    rule.add(MinorType.FLOAT4);
-    rule.add(MinorType.FLOAT8);
-    rule.add(MinorType.BIT);
-    rule.add(MinorType.FIXEDCHAR);
-    rule.add(MinorType.FIXED16CHAR);
-    rule.add(MinorType.FIXEDBINARY);
-    rule.add(MinorType.VARCHAR);
-    rule.add(MinorType.VAR16CHAR);
-    rule.add(MinorType.VARBINARY);
+    rule.add(MinorType.VARDECIMAL);
     rules.put(MinorType.DECIMAL18, rule);
 
     /** DECIMAL28Dense cast able from **/
     rule = new HashSet<>();
-    rule.add(MinorType.TINYINT);
-    rule.add(MinorType.SMALLINT);
-    rule.add(MinorType.INT);
-    rule.add(MinorType.BIGINT);
-    rule.add(MinorType.UINT1);
-    rule.add(MinorType.UINT2);
-    rule.add(MinorType.UINT4);
-    rule.add(MinorType.UINT8);
-    rule.add(MinorType.DECIMAL9);
-    rule.add(MinorType.DECIMAL18);
-    rule.add(MinorType.DECIMAL28SPARSE);
     rule.add(MinorType.DECIMAL28DENSE);
-    rule.add(MinorType.DECIMAL38SPARSE);
-    rule.add(MinorType.DECIMAL38DENSE);
-    rule.add(MinorType.MONEY);
-    rule.add(MinorType.FLOAT4);
-    rule.add(MinorType.FLOAT8);
-    rule.add(MinorType.BIT);
-    rule.add(MinorType.FIXEDCHAR);
-    rule.add(MinorType.FIXED16CHAR);
-    rule.add(MinorType.FIXEDBINARY);
-    rule.add(MinorType.VARCHAR);
-    rule.add(MinorType.VAR16CHAR);
-    rule.add(MinorType.VARBINARY);
+    rule.add(MinorType.VARDECIMAL);
     rules.put(MinorType.DECIMAL28DENSE, rule);
 
     /** DECIMAL28Sparse cast able from **/
+
     rule = new HashSet<>();
-    rule.add(MinorType.TINYINT);
-    rule.add(MinorType.SMALLINT);
-    rule.add(MinorType.INT);
-    rule.add(MinorType.BIGINT);
-    rule.add(MinorType.UINT1);
-    rule.add(MinorType.UINT2);
-    rule.add(MinorType.UINT4);
-    rule.add(MinorType.UINT8);
-    rule.add(MinorType.DECIMAL9);
-    rule.add(MinorType.DECIMAL18);
     rule.add(MinorType.DECIMAL28SPARSE);
-    rule.add(MinorType.DECIMAL28DENSE);
-    rule.add(MinorType.DECIMAL38SPARSE);
-    rule.add(MinorType.DECIMAL38DENSE);
-    rule.add(MinorType.MONEY);
-    rule.add(MinorType.FLOAT4);
-    rule.add(MinorType.FLOAT8);
-    rule.add(MinorType.BIT);
-    rule.add(MinorType.FIXEDCHAR);
-    rule.add(MinorType.FIXED16CHAR);
-    rule.add(MinorType.FIXEDBINARY);
-    rule.add(MinorType.VARCHAR);
-    rule.add(MinorType.VAR16CHAR);
-    rule.add(MinorType.VARBINARY);
+    rule.add(MinorType.VARDECIMAL);
     rules.put(MinorType.DECIMAL28SPARSE, rule);
 
-    /** DECIMAL38Dense cast able from **/
+    /* VARDECIMAL cast able from **/
     rule = new HashSet<>();
     rule.add(MinorType.TINYINT);
     rule.add(MinorType.SMALLINT);
@@ -341,51 +261,20 @@ private static void initTypeRules() {
     rule.add(MinorType.UINT2);
     rule.add(MinorType.UINT4);
     rule.add(MinorType.UINT8);
-    rule.add(MinorType.DECIMAL9);
-    rule.add(MinorType.DECIMAL18);
-    rule.add(MinorType.DECIMAL28SPARSE);
-    rule.add(MinorType.DECIMAL28DENSE);
-    rule.add(MinorType.DECIMAL38SPARSE);
+    rule.add(MinorType.VARDECIMAL);
+    rules.put(MinorType.VARDECIMAL, rule);
+
+    /** DECIMAL38Dense cast able from **/
+    rule = new HashSet<>();
     rule.add(MinorType.DECIMAL38DENSE);
-    rule.add(MinorType.MONEY);
-    rule.add(MinorType.FLOAT4);
-    rule.add(MinorType.FLOAT8);
-    rule.add(MinorType.BIT);
-    rule.add(MinorType.FIXEDCHAR);
-    rule.add(MinorType.FIXED16CHAR);
-    rule.add(MinorType.FIXEDBINARY);
-    rule.add(MinorType.VARCHAR);
-    rule.add(MinorType.VAR16CHAR);
-    rule.add(MinorType.VARBINARY);
+    rule.add(MinorType.VARDECIMAL);
     rules.put(MinorType.DECIMAL38DENSE, rule);
 
 
     /** DECIMAL38Sparse cast able from **/
     rule = new HashSet<>();
-    rule.add(MinorType.TINYINT);
-    rule.add(MinorType.SMALLINT);
-    rule.add(MinorType.INT);
-    rule.add(MinorType.BIGINT);
-    rule.add(MinorType.UINT1);
-    rule.add(MinorType.UINT2);
-    rule.add(MinorType.UINT4);
-    rule.add(MinorType.UINT8);
-    rule.add(MinorType.DECIMAL9);
-    rule.add(MinorType.DECIMAL18);
-    rule.add(MinorType.DECIMAL28SPARSE);
-    rule.add(MinorType.DECIMAL28DENSE);
     rule.add(MinorType.DECIMAL38SPARSE);
-    rule.add(MinorType.DECIMAL38DENSE);
-    rule.add(MinorType.MONEY);
-    rule.add(MinorType.FLOAT4);
-    rule.add(MinorType.FLOAT8);
-    rule.add(MinorType.BIT);
-    rule.add(MinorType.FIXEDCHAR);
-    rule.add(MinorType.FIXED16CHAR);
-    rule.add(MinorType.FIXEDBINARY);
-    rule.add(MinorType.VARCHAR);
-    rule.add(MinorType.VAR16CHAR);
-    rule.add(MinorType.VARBINARY);
+    rule.add(MinorType.VARDECIMAL);
     rules.put(MinorType.DECIMAL38SPARSE, rule);
 
     /** MONEY cast able from **/
@@ -404,6 +293,7 @@ private static void initTypeRules() {
     rule.add(MinorType.DECIMAL28DENSE);
     rule.add(MinorType.DECIMAL38SPARSE);
     rule.add(MinorType.DECIMAL38DENSE);
+    rule.add(MinorType.VARDECIMAL);
     rule.add(MinorType.MONEY);
     rule.add(MinorType.FLOAT4);
     rule.add(MinorType.FLOAT8);
@@ -539,6 +429,7 @@ private static void initTypeRules() {
     rule.add(MinorType.DECIMAL28DENSE);
     rule.add(MinorType.DECIMAL38SPARSE);
     rule.add(MinorType.DECIMAL38DENSE);
+    rule.add(MinorType.VARDECIMAL);
     rule.add(MinorType.MONEY);
     rule.add(MinorType.FLOAT4);
     rule.add(MinorType.BIT);
@@ -564,6 +455,7 @@ private static void initTypeRules() {
     rule.add(MinorType.DECIMAL28DENSE);
     rule.add(MinorType.DECIMAL38SPARSE);
     rule.add(MinorType.DECIMAL38DENSE);
+    rule.add(MinorType.VARDECIMAL);
     rule.add(MinorType.MONEY);
     rule.add(MinorType.FLOAT4);
     rule.add(MinorType.FLOAT8);
@@ -592,7 +484,6 @@ private static void initTypeRules() {
     rule.add(MinorType.INT);
     rule.add(MinorType.BIGINT);
     rule.add(MinorType.MONEY);
-    rule.add(MinorType.TIMESTAMPTZ);
     rule.add(MinorType.UINT1);
     rule.add(MinorType.UINT2);
     rule.add(MinorType.UINT4);
@@ -603,7 +494,7 @@ private static void initTypeRules() {
     rule.add(MinorType.DECIMAL28DENSE);
     rule.add(MinorType.DECIMAL38SPARSE);
     rule.add(MinorType.DECIMAL38DENSE);
-    rule.add(MinorType.TIMESTAMP);
+    rule.add(MinorType.VARDECIMAL);
     rule.add(MinorType.FLOAT4);
     rule.add(MinorType.FLOAT8);
     rule.add(MinorType.BIT);
@@ -638,8 +529,8 @@ private static void initTypeRules() {
     rule.add(MinorType.DECIMAL28DENSE);
     rule.add(MinorType.DECIMAL38SPARSE);
     rule.add(MinorType.DECIMAL38DENSE);
+    rule.add(MinorType.VARDECIMAL);
     rule.add(MinorType.MONEY);
-    rule.add(MinorType.TIMESTAMPTZ);
     rule.add(MinorType.FLOAT4);
     rule.add(MinorType.FLOAT8);
     rule.add(MinorType.BIT);
@@ -674,8 +565,8 @@ private static void initTypeRules() {
     rule.add(MinorType.DECIMAL28DENSE);
     rule.add(MinorType.DECIMAL38SPARSE);
     rule.add(MinorType.DECIMAL38DENSE);
+    rule.add(MinorType.VARDECIMAL);
     rule.add(MinorType.MONEY);
-    rule.add(MinorType.TIMESTAMPTZ);
     rule.add(MinorType.FLOAT4);
     rule.add(MinorType.FLOAT8);
     rule.add(MinorType.BIT);
@@ -701,6 +592,7 @@ private static void initTypeRules() {
     rule.add(MinorType.DECIMAL28DENSE);
     rule.add(MinorType.DECIMAL38SPARSE);
     rule.add(MinorType.DECIMAL38DENSE);
+    rule.add(MinorType.VARDECIMAL);
     rule.add(MinorType.MONEY);
     rule.add(MinorType.TIMESTAMPTZ);
     rule.add(MinorType.FLOAT4);
@@ -737,6 +629,7 @@ private static void initTypeRules() {
     rule.add(MinorType.DECIMAL28DENSE);
     rule.add(MinorType.DECIMAL38SPARSE);
     rule.add(MinorType.DECIMAL38DENSE);
+    rule.add(MinorType.VARDECIMAL);
     rule.add(MinorType.MONEY);
     rule.add(MinorType.FLOAT4);
     rule.add(MinorType.FLOAT8);
@@ -772,6 +665,7 @@ private static void initTypeRules() {
     rule.add(MinorType.DECIMAL28DENSE);
     rule.add(MinorType.DECIMAL38SPARSE);
     rule.add(MinorType.DECIMAL38DENSE);
+    rule.add(MinorType.VARDECIMAL);
     rule.add(MinorType.MONEY);
     rule.add(MinorType.TIMESTAMP);
     rule.add(MinorType.TIMESTAMPTZ);
@@ -800,7 +694,7 @@ public static boolean isCastableWithNullHandling(MajorType from, MajorType to, N
 
   public static boolean isCastable(MinorType from, MinorType to) {
     return from.equals(MinorType.NULL) ||      //null could be casted to any other type.
-        (rules.get(to) == null ? false : rules.get(to).contains(from));
+        (rules.get(to) != null && rules.get(to).contains(from));
   }
 
   public static DataMode getLeastRestrictiveDataMode(List<DataMode> dataModes) {
@@ -883,14 +777,14 @@ public static int getCost(List<MajorType> argumentTypes, DrillFuncHolder holder)
      * If we are determining function holder for decimal data type, we need to make sure the output type of
      * the function can fit the precision that we need based on the input types.
      */
-    if (holder.checkPrecisionRange() == true) {
+    if (holder.checkPrecisionRange()) {
       List<LogicalExpression> logicalExpressions = Lists.newArrayList();
       for(MajorType majorType : argumentTypes) {
         logicalExpressions.add(
             new MajorTypeInLogicalExpression(majorType));
       }
 
-      if (DecimalUtility.getMaxPrecision(holder.getReturnType().getMinorType()) <
+      if (DRILL_REL_DATATYPE_SYSTEM.getMaxNumericPrecision() <
           holder.getReturnType(logicalExpressions).getPrecision()) {
         return -1;
       }
@@ -939,8 +833,8 @@ public static int getCost(List<MajorType> argumentTypes, DrillFuncHolder holder)
          * if the seconday rules allow us to cast
          */
         Set<MinorType> rules;
-        if ((rules = (ResolverTypePrecedence.secondaryImplicitCastRules.get(parmType.getMinorType()))) != null &&
-            rules.contains(argType.getMinorType()) != false) {
+        if ((rules = (ResolverTypePrecedence.secondaryImplicitCastRules.get(parmType.getMinorType()))) != null
+            && rules.contains(argType.getMinorType())) {
           secondaryCast = true;
         } else {
           return -1;
@@ -1012,6 +906,7 @@ public static boolean isNumericType(MinorType inputType) {
       case DECIMAL18:
       case DECIMAL28SPARSE:
       case DECIMAL38SPARSE:
+      case VARDECIMAL:
       case FLOAT4:
       case FLOAT8:
         return true;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index 63f1dc2e94..a9ef7376a1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -130,6 +130,8 @@
       new OptionDefinition(ExecConstants.PARQUET_DICT_PAGE_SIZE_VALIDATOR),
       new OptionDefinition(ExecConstants.PARQUET_WRITER_COMPRESSION_TYPE_VALIDATOR),
       new OptionDefinition(ExecConstants.PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING_VALIDATOR),
+      new OptionDefinition(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS_VALIDATOR),
+      new OptionDefinition(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS_VALIDATOR),
       new OptionDefinition(ExecConstants.PARQUET_VECTOR_FILL_THRESHOLD_VALIDATOR),
       new OptionDefinition(ExecConstants.PARQUET_VECTOR_FILL_CHECK_THRESHOLD_VALIDATOR),
       new OptionDefinition(ExecConstants.PARQUET_RECORD_READER_IMPLEMENTATION_VALIDATOR),
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroDrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroDrillTable.java
index d89079b856..d3f90ee1c2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroDrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroDrillTable.java
@@ -20,6 +20,7 @@
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.avro.LogicalType;
 import org.apache.avro.Schema;
 import org.apache.avro.Schema.Field;
 import org.apache.avro.file.DataFileReader;
@@ -95,6 +96,8 @@ public RelDataType getRowType(RelDataTypeFactory typeFactory) {
 
   private RelDataType getNullableRelDataTypeFromAvroType(
       RelDataTypeFactory typeFactory, Schema fieldSchema) {
+    LogicalType logicalType = fieldSchema.getLogicalType();
+    String logicalTypeName = logicalType != null ? logicalType.getName() : "";
     RelDataType relDataType = null;
     switch (fieldSchema.getType()) {
     case ARRAY:
@@ -105,22 +108,57 @@ private RelDataType getNullableRelDataTypeFromAvroType(
       relDataType = typeFactory.createSqlType(SqlTypeName.BOOLEAN);
       break;
     case BYTES:
-      relDataType = typeFactory.createSqlType(SqlTypeName.BINARY);
+      switch (logicalTypeName) {
+        case "decimal":
+          relDataType = typeFactory.createSqlType(SqlTypeName.DECIMAL);
+          break;
+        default:
+          relDataType = typeFactory.createSqlType(SqlTypeName.BINARY);
+      }
       break;
     case DOUBLE:
       relDataType = typeFactory.createSqlType(SqlTypeName.DOUBLE);
       break;
     case FIXED:
-      logger.error("{} type not supported", fieldSchema.getType());
-      throw UserException.unsupportedError().message("FIXED type not supported yet").build(logger);
+      switch (logicalTypeName) {
+        case "decimal":
+          relDataType = typeFactory.createSqlType(SqlTypeName.DECIMAL);
+          break;
+        default:
+          logger.error("{} type not supported", fieldSchema.getType());
+          throw UserException.unsupportedError().message("FIXED type not supported yet").build(logger);
+      }
+      break;
     case FLOAT:
       relDataType = typeFactory.createSqlType(SqlTypeName.FLOAT);
       break;
     case INT:
-      relDataType = typeFactory.createSqlType(SqlTypeName.INTEGER);
+      switch (logicalTypeName) {
+        case "date":
+          relDataType = typeFactory.createSqlType(SqlTypeName.DATE);
+          break;
+        case "time-millis":
+          relDataType = typeFactory.createSqlType(SqlTypeName.TIME);
+          break;
+        default:
+          relDataType = typeFactory.createSqlType(SqlTypeName.INTEGER);
+      }
       break;
     case LONG:
-      relDataType = typeFactory.createSqlType(SqlTypeName.BIGINT);
+      switch (logicalTypeName) {
+        case "date":
+          relDataType = typeFactory.createSqlType(SqlTypeName.DATE);
+          break;
+        case "time-micros":
+          relDataType = typeFactory.createSqlType(SqlTypeName.TIME);
+          break;
+        case "timestamp-millis":
+        case "timestamp-micros":
+          relDataType = typeFactory.createSqlType(SqlTypeName.TIMESTAMP);
+          break;
+        default:
+          relDataType = typeFactory.createSqlType(SqlTypeName.BIGINT);
+      }
       break;
     case MAP:
       RelDataType valueType = getNullableRelDataTypeFromAvroType(typeFactory, fieldSchema.getValueType());
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java
index c7d2aed27b..09ee5af6cf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java
@@ -19,6 +19,8 @@
 
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
 import java.util.HashMap;
@@ -26,24 +28,28 @@
 import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.avro.LogicalType;
+import org.apache.avro.LogicalTypes;
 import org.apache.avro.Schema;
 import org.apache.avro.Schema.Type;
 import org.apache.avro.file.DataFileReader;
 import org.apache.avro.generic.GenericArray;
 import org.apache.avro.generic.GenericContainer;
 import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericFixed;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.mapred.FsInput;
 import org.apache.avro.util.Utf8;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.common.expression.PathSegment;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.store.AbstractRecordReader;
 import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.store.parquet.ParquetReaderUtility;
 import org.apache.drill.exec.util.ImpersonationUtil;
 import org.apache.drill.exec.vector.complex.fn.FieldSelection;
 import org.apache.drill.exec.vector.complex.impl.MapOrListWriterImpl;
@@ -56,6 +62,7 @@
 import com.google.common.base.Stopwatch;
 
 import io.netty.buffer.DrillBuf;
+import org.joda.time.DateTimeConstants;
 
 /**
  * A RecordReader implementation for Avro data files.
@@ -64,12 +71,13 @@
  */
 public class AvroRecordReader extends AbstractRecordReader {
 
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AvroRecordReader.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AvroRecordReader.class);
 
   private final Path hadoop;
   private final long start;
   private final long end;
   private final FieldSelection fieldSelection;
+  private final OptionManager optionManager;
   private DrillBuf buffer;
   private VectorContainerWriter writer;
 
@@ -98,6 +106,7 @@ public AvroRecordReader(final FragmentContext fragmentContext,
     this.queryUserName = fragmentContext.getQueryUserName();
     setColumns(projectedColumns);
     this.fieldSelection = FieldSelection.getFieldSelection(projectedColumns);
+    optionManager = fragmentContext.getOptions();
   }
 
   private DataFileReader<GenericContainer> getReader(final Path hadoop, final FileSystem fs) throws ExecutionSetupException {
@@ -233,7 +242,6 @@ private void process(final Object value, final Schema schema, final String field
         writer.end();
         break;
       case FIXED:
-        throw new UnsupportedOperationException("Unimplemented type: " + type.toString());
       case ENUM:  // Enum symbols are strings
       case NULL:  // Treat null type as a primitive
       default:
@@ -245,21 +253,24 @@ private void process(final Object value, final Schema schema, final String field
           }
         }
 
-        processPrimitive(value, schema.getType(), fieldName, writer);
+        processPrimitive(value, schema, fieldName, writer);
         break;
     }
 
   }
 
-  private void processPrimitive(final Object value, final Schema.Type type, final String fieldName,
+  private void processPrimitive(final Object value, final Schema schema, final String fieldName,
                                 final MapOrListWriterImpl writer) {
+    LogicalType logicalType = schema.getLogicalType();
+    String logicalTypeName = logicalType != null ? logicalType.getName() : "";
+
     if (value == null) {
       return;
     }
 
-    switch (type) {
+    switch (schema.getType()) {
       case STRING:
-        byte[] binary = null;
+        byte[] binary;
         final int length;
         if (value instanceof Utf8) {
           binary = ((Utf8) value).getBytes();
@@ -273,10 +284,34 @@ private void processPrimitive(final Object value, final Schema.Type type, final
         writer.varChar(fieldName).writeVarChar(0, length, buffer);
         break;
       case INT:
-        writer.integer(fieldName).writeInt((Integer) value);
+        switch (logicalTypeName) {
+          case "date":
+            writer.date(fieldName).writeDate((int) value * (long) DateTimeConstants.MILLIS_PER_DAY);
+            break;
+          case "time-millis":
+            writer.time(fieldName).writeTime((Integer) value);
+            break;
+          default:
+            writer.integer(fieldName).writeInt((Integer) value);
+        }
         break;
       case LONG:
-        writer.bigInt(fieldName).writeBigInt((Long) value);
+        switch (logicalTypeName) {
+          case "date":
+            writer.date(fieldName).writeDate((Long) value);
+            break;
+          case "time-micros":
+            writer.time(fieldName).writeTime((int) ((long) value / 1000));
+            break;
+          case "timestamp-millis":
+            writer.timeStamp(fieldName).writeTimeStamp((Long) value);
+            break;
+          case "timestamp-micros":
+            writer.timeStamp(fieldName).writeTimeStamp((long) value / 1000);
+            break;
+          default:
+            writer.bigInt(fieldName).writeBigInt((Long) value);
+        }
         break;
       case FLOAT:
         writer.float4(fieldName).writeFloat4((Float) value);
@@ -292,7 +327,16 @@ private void processPrimitive(final Object value, final Schema.Type type, final
         length = buf.remaining();
         ensure(length);
         buffer.setBytes(0, buf);
-        writer.binary(fieldName).writeVarBinary(0, length, buffer);
+        switch (logicalTypeName) {
+          case "decimal":
+            ParquetReaderUtility.checkDecimalTypeEnabled(optionManager);
+            LogicalTypes.Decimal decimalType = (LogicalTypes.Decimal) logicalType;
+            writer.varDecimal(fieldName, decimalType.getScale(), decimalType.getPrecision())
+                .writeVarDecimal(0, length, buffer, decimalType.getScale(), decimalType.getPrecision());
+            break;
+          default:
+            writer.binary(fieldName).writeVarBinary(0, length, buffer);
+        }
         break;
       case NULL:
         // Nothing to do for null type
@@ -301,17 +345,29 @@ private void processPrimitive(final Object value, final Schema.Type type, final
         final String symbol = value.toString();
         final byte[] b;
         try {
-          b = symbol.getBytes("UTF-8");
+          b = symbol.getBytes(Charsets.UTF_8.name());
         } catch (UnsupportedEncodingException e) {
           throw new DrillRuntimeException("Unable to read enum value for field: " + fieldName, e);
         }
         ensure(b.length);
         buffer.setBytes(0, b);
         writer.varChar(fieldName).writeVarChar(0, b.length, buffer);
-
+        break;
+      case FIXED:
+        GenericFixed genericFixed = (GenericFixed) value;
+        switch (logicalTypeName) {
+          case "decimal":
+            ParquetReaderUtility.checkDecimalTypeEnabled(optionManager);
+            LogicalTypes.Decimal decimalType = (LogicalTypes.Decimal) logicalType;
+            writer.varDecimal(fieldName, decimalType.getScale(), decimalType.getPrecision())
+                .writeVarDecimal(new BigDecimal(new BigInteger(genericFixed.bytes()), decimalType.getScale()));
+            break;
+          default:
+            throw new UnsupportedOperationException("Unimplemented type: " + schema.getType().toString());
+        }
         break;
       default:
-        throw new DrillRuntimeException("Unhandled Avro type: " + type.toString());
+        throw new DrillRuntimeException("Unhandled Avro type: " + schema.getType().toString());
     }
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java
index c1137bdfa6..1a7dc30316 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java
@@ -96,6 +96,8 @@ private void makeDefaultGenerator() {
       break;
     case DECIMAL38SPARSE:
       break;
+    case VARDECIMAL:
+      break;
     case DECIMAL9:
       break;
     case FIXED16CHAR:
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
index 9804e2496b..52c47230e0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
@@ -39,10 +39,12 @@
 import org.apache.drill.exec.expr.holders.TimeHolder;
 import org.apache.drill.exec.expr.holders.TimeStampHolder;
 import org.apache.drill.exec.expr.holders.ValueHolder;
+import org.apache.drill.exec.expr.holders.VarDecimalHolder;
 import org.apache.drill.exec.expr.stat.ParquetBooleanPredicates;
 import org.apache.drill.exec.expr.stat.ParquetComparisonPredicates;
 import org.apache.drill.exec.expr.stat.ParquetIsPredicates;
 import org.apache.drill.exec.ops.UdfUtilities;
+import org.apache.drill.exec.util.DecimalUtility;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -109,6 +111,12 @@ public LogicalExpression visitLongConstant(ValueExpressions.LongExpression intEx
     return intExpr;
   }
 
+  @Override
+  public LogicalExpression visitVarDecimalConstant(ValueExpressions.VarDecimalExpression decExpr, Set<LogicalExpression> value)
+      throws RuntimeException {
+    return decExpr;
+  }
+
   @Override
   public LogicalExpression visitDateConstant(ValueExpressions.DateExpression dateExpr, Set<LogicalExpression> value) throws RuntimeException {
     return dateExpr;
@@ -169,6 +177,11 @@ private LogicalExpression getValueExpressionFromConst(ValueHolder holder, TypePr
       return ValueExpressions.getFloat4(((Float4Holder) holder).value);
     case FLOAT8:
       return ValueExpressions.getFloat8(((Float8Holder) holder).value);
+    case VARDECIMAL:
+      VarDecimalHolder decimalHolder = (VarDecimalHolder) holder;
+      return ValueExpressions.getVarDecimal(
+          DecimalUtility.getBigDecimalFromDrillBuf(decimalHolder.buffer,
+              decimalHolder.start, decimalHolder.end - decimalHolder.start, decimalHolder.scale));
     case DATE:
       return ValueExpressions.getDate(((DateHolder) holder).value);
     case TIMESTAMP:
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
index f66cf303bc..94760da6f0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
@@ -153,6 +153,12 @@ public RecordWriter getRecordWriter(FragmentContext context, ParquetWriter write
     options.put(ExecConstants.PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING,
         context.getOptions().getOption(ExecConstants.PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING).bool_val.toString());
 
+    options.put(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS,
+        context.getOptions().getOption(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS).string_val);
+
+    options.put(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS,
+        context.getOptions().getOption(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS).bool_val.toString());
+
     RecordWriter recordWriter = new ParquetRecordWriter(context, writer);
     recordWriter.init(options);
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
index 9157590698..377dfbe0e6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
@@ -486,7 +486,7 @@ public static long getDateTimeValueFromBinary(Binary binaryTimeStampValue, boole
     if (originalType != null) {
       switch (originalType) {
         case DECIMAL:
-          return Types.withScaleAndPrecision(TypeProtos.MinorType.DECIMAL18, TypeProtos.DataMode.OPTIONAL, scale, precision);
+          return Types.withScaleAndPrecision(TypeProtos.MinorType.VARDECIMAL, TypeProtos.DataMode.OPTIONAL, scale, precision);
         case DATE:
           return Types.optional(TypeProtos.MinorType.DATE);
         case TIME_MILLIS:
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
index 9b84e810a9..ec7861974e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
@@ -45,6 +45,7 @@
 import org.apache.drill.exec.store.EventBasedRecordWriter;
 import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
 import org.apache.drill.exec.store.ParquetOutputRecordWriter;
+import org.apache.drill.exec.util.DecimalUtility;
 import org.apache.drill.exec.vector.BitVector;
 import org.apache.drill.exec.vector.complex.reader.FieldReader;
 import org.apache.hadoop.conf.Configuration;
@@ -114,9 +115,10 @@
   private OperatorContext oContext;
   private List<String> partitionColumns;
   private boolean hasPartitions;
+  private PrimitiveTypeName logicalTypeForDecimals;
+  private boolean usePrimitiveTypesForDecimals;
 
-  public ParquetRecordWriter(FragmentContext context, ParquetWriter writer) throws OutOfMemoryException{
-    super();
+  public ParquetRecordWriter(FragmentContext context, ParquetWriter writer) throws OutOfMemoryException {
     this.oContext = context.newOperatorContext(writer);
     this.codecFactory = CodecFactory.createDirectCodecFactory(writer.getFormatPlugin().getFsConf(),
         new ParquetDirectByteBufferAllocator(oContext.getAllocator()), pageSize);
@@ -158,8 +160,24 @@ public void init(Map<String, String> writerOptions) throws IOException {
       throw new UnsupportedOperationException(String.format("Unknown compression type: %s", codecName));
     }
 
+    String logicalTypeNameForDecimals = writerOptions.get(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS).toLowerCase();
+    switch (logicalTypeNameForDecimals) {
+      case "fixed_len_byte_array":
+        logicalTypeForDecimals = PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY;
+        break;
+      case "binary":
+        logicalTypeForDecimals = PrimitiveTypeName.BINARY;
+        break;
+      default:
+        throw new UnsupportedOperationException(
+            String.format(
+                "Unsupported logical type for decimals: %s\n" +
+                "Supported types: ['fixed_len_byte_array', 'binary']", codecName));
+    }
+
     enableDictionary = Boolean.parseBoolean(writerOptions.get(ExecConstants.PARQUET_WRITER_ENABLE_DICTIONARY_ENCODING));
     useSingleFSBlock = Boolean.parseBoolean(writerOptions.get(ExecConstants.PARQUET_WRITER_USE_SINGLE_FS_BLOCK));
+    usePrimitiveTypesForDecimals = Boolean.parseBoolean(writerOptions.get(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS));
 
     if (useSingleFSBlock) {
       // Round up blockSize to multiple of 64K.
@@ -228,14 +246,27 @@ private void newSchema() throws IOException {
     setUp(schema, consumer);
   }
 
-  private PrimitiveType getPrimitiveType(MaterializedField field) {
+  protected PrimitiveType getPrimitiveType(MaterializedField field) {
     MinorType minorType = field.getType().getMinorType();
     String name = field.getName();
+    int length = ParquetTypeHelper.getLengthForMinorType(minorType);
     PrimitiveTypeName primitiveTypeName = ParquetTypeHelper.getPrimitiveTypeNameForMinorType(minorType);
+    if (DecimalUtility.isDecimalType(minorType)) {
+      primitiveTypeName = logicalTypeForDecimals;
+      if (usePrimitiveTypesForDecimals) {
+        if (field.getPrecision() <= ParquetTypeHelper.getMaxPrecisionForPrimitiveType(PrimitiveTypeName.INT32)) {
+          primitiveTypeName = PrimitiveTypeName.INT32;
+        } else if (field.getPrecision() <= ParquetTypeHelper.getMaxPrecisionForPrimitiveType(PrimitiveTypeName.INT64)) {
+          primitiveTypeName = PrimitiveTypeName.INT64;
+        }
+      }
+
+      length = DecimalUtility.getMaxBytesSizeForPrecision(field.getPrecision());
+    }
+
     Repetition repetition = ParquetTypeHelper.getRepetitionForDataMode(field.getDataMode());
     OriginalType originalType = ParquetTypeHelper.getOriginalTypeForMinorType(minorType);
     DecimalMetadata decimalMetadata = ParquetTypeHelper.getDecimalMetadataForField(field);
-    int length = ParquetTypeHelper.getLengthForMinorType(minorType);
     return new PrimitiveType(repetition, primitiveTypeName, length, name, originalType, decimalMetadata, null);
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReaderFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReaderFactory.java
index 7edfa00e1a..cf297fa72e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReaderFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReaderFactory.java
@@ -20,13 +20,11 @@
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.vector.VarDecimalVector;
+import org.apache.drill.exec.vector.NullableVarDecimalVector;
 import org.apache.drill.exec.vector.BigIntVector;
 import org.apache.drill.exec.vector.BitVector;
 import org.apache.drill.exec.vector.DateVector;
-import org.apache.drill.exec.vector.Decimal18Vector;
-import org.apache.drill.exec.vector.Decimal28SparseVector;
-import org.apache.drill.exec.vector.Decimal38SparseVector;
-import org.apache.drill.exec.vector.Decimal9Vector;
 import org.apache.drill.exec.vector.Float4Vector;
 import org.apache.drill.exec.vector.Float8Vector;
 import org.apache.drill.exec.vector.IntVector;
@@ -34,10 +32,6 @@
 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;
@@ -78,25 +72,23 @@
     ConvertedType convertedType = schemaElement.getConverted_type();
     // if the column is required, or repeated (in which case we just want to use this to generate our appropriate
     // ColumnReader for actually transferring data into the data vector inside of our repeated vector
-    if (descriptor.getMaxDefinitionLevel() == 0 || descriptor.getMaxRepetitionLevel() > 0){
+    if (descriptor.getMaxDefinitionLevel() == 0 || descriptor.getMaxRepetitionLevel() > 0) {
       if (columnChunkMetaData.getType() == PrimitiveType.PrimitiveTypeName.BOOLEAN){
         return new BitReader(recordReader, allocateSize, descriptor, columnChunkMetaData,
             fixedLength, (BitVector) v, schemaElement);
       } else if (!columnChunkMetaData.getEncodings().contains(Encoding.PLAIN_DICTIONARY) && (
           columnChunkMetaData.getType() == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY
               || columnChunkMetaData.getType() == PrimitiveType.PrimitiveTypeName.INT96)) {
-        if (convertedType == ConvertedType.DECIMAL){
-          int length = schemaElement.type_length;
-          if (length <= 12) {
-            return new FixedByteAlignedReader.Decimal28Reader(recordReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (Decimal28SparseVector) v, schemaElement);
-          } else if (length <= 16) {
-            return new FixedByteAlignedReader.Decimal38Reader(recordReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (Decimal38SparseVector) v, schemaElement);
-          }
-        } else if (convertedType == ConvertedType.INTERVAL) {
-          return new FixedByteAlignedReader.IntervalReader(recordReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (IntervalVector) v, schemaElement);
-        }
-        else {
-          return new FixedByteAlignedReader.FixedBinaryReader(recordReader, allocateSize, descriptor, columnChunkMetaData, (VariableWidthVector) v, schemaElement);
+        switch (convertedType) {
+          case DECIMAL:
+            return new FixedByteAlignedReader.VarDecimalReader(recordReader, allocateSize, descriptor,
+                columnChunkMetaData, fixedLength, (VarDecimalVector) v, schemaElement);
+          case INTERVAL:
+            return new FixedByteAlignedReader.IntervalReader(recordReader, allocateSize, descriptor,
+                columnChunkMetaData, fixedLength, (IntervalVector) v, schemaElement);
+          default:
+            return new FixedByteAlignedReader.FixedBinaryReader(recordReader, allocateSize, descriptor,
+                columnChunkMetaData, (VariableWidthVector) v, schemaElement);
         }
       } else if (columnChunkMetaData.getType() == PrimitiveType.PrimitiveTypeName.INT32 && convertedType == ConvertedType.DATE){
         switch(recordReader.getDateCorruptionStatus()) {
@@ -112,7 +104,7 @@
                         "unrecognized date corruption status %s. See DRILL-4203 for more info.",
                 recordReader.getDateCorruptionStatus()));
         }
-      } else{
+      } else {
         if (columnChunkMetaData.getEncodings().contains(Encoding.PLAIN_DICTIONARY)) {
           switch (columnChunkMetaData.getType()) {
             case INT32:
@@ -121,7 +113,8 @@
               }
               switch (convertedType) {
                 case DECIMAL:
-                  return new ParquetFixedWidthDictionaryReaders.DictionaryDecimal9Reader(recordReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (Decimal9Vector) v, schemaElement);
+                  return new ParquetFixedWidthDictionaryReaders.DictionaryVarDecimalReader(recordReader, allocateSize,
+                      descriptor, columnChunkMetaData, fixedLength, (VarDecimalVector) v, schemaElement);
                 case TIME_MILLIS:
                   return new ParquetFixedWidthDictionaryReaders.DictionaryTimeReader(recordReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (TimeVector) v, schemaElement);
                 case INT_8:
@@ -145,7 +138,8 @@
                 case UINT_64:
                   return new ParquetFixedWidthDictionaryReaders.DictionaryUInt8Reader(recordReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (UInt8Vector) v, schemaElement);
                 case DECIMAL:
-                  return new ParquetFixedWidthDictionaryReaders.DictionaryDecimal18Reader(recordReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (Decimal18Vector) v, schemaElement);
+                  return new ParquetFixedWidthDictionaryReaders.DictionaryVarDecimalReader(recordReader, allocateSize,
+                      descriptor, columnChunkMetaData, fixedLength, (VarDecimalVector) v, schemaElement);
                 case TIMESTAMP_MILLIS:
                   return new ParquetFixedWidthDictionaryReaders.DictionaryTimeStampReader(recordReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (TimeStampVector) v, schemaElement);
                 default:
@@ -167,17 +161,19 @@
               throw new ExecutionSetupException("Unsupported dictionary column type " + descriptor.getType().name() );
           }
 
+        } else if (convertedType == ConvertedType.DECIMAL) {
+          return new FixedByteAlignedReader.VarDecimalReader(recordReader, allocateSize,
+            descriptor, columnChunkMetaData, fixedLength, (VarDecimalVector) v, schemaElement);
         } else {
           return new FixedByteAlignedReader<>(recordReader, allocateSize, descriptor, columnChunkMetaData,
               fixedLength, v, schemaElement);
         }
       }
-    }
-    else { // if the column is nullable
-      if (columnChunkMetaData.getType() == PrimitiveType.PrimitiveTypeName.BOOLEAN){
+    } else { // if the column is nullable
+      if (columnChunkMetaData.getType() == PrimitiveType.PrimitiveTypeName.BOOLEAN) {
         return new NullableBitReader(recordReader, allocateSize, descriptor, columnChunkMetaData,
             fixedLength, (NullableBitVector) v, schemaElement);
-      } else if (columnChunkMetaData.getType() == PrimitiveType.PrimitiveTypeName.INT32 && convertedType == ConvertedType.DATE){
+      } else if (columnChunkMetaData.getType() == PrimitiveType.PrimitiveTypeName.INT32 && convertedType == ConvertedType.DATE) {
         switch(recordReader.getDateCorruptionStatus()) {
           case META_SHOWS_CORRUPTION:
             return new NullableFixedByteAlignedReaders.NullableCorruptDateReader(recordReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (NullableDateVector)v, schemaElement);
@@ -193,12 +189,8 @@
         }
       } else if (columnChunkMetaData.getType() == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) {
         if (convertedType == ConvertedType.DECIMAL) {
-          int length = schemaElement.type_length;
-          if (length <= 12) {
-            return new NullableFixedByteAlignedReaders.NullableDecimal28Reader(recordReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (NullableDecimal28SparseVector) v, schemaElement);
-          } else if (length <= 16) {
-            return new NullableFixedByteAlignedReaders.NullableDecimal38Reader(recordReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (NullableDecimal38SparseVector) v, schemaElement);
-          }
+          return new NullableFixedByteAlignedReaders.NullableVarDecimalReader(recordReader, allocateSize,
+              descriptor, columnChunkMetaData, fixedLength, (NullableVarDecimalVector) v, schemaElement);
         } else if (convertedType == ConvertedType.INTERVAL) {
           return new NullableFixedByteAlignedReaders.NullableIntervalReader(recordReader, allocateSize, descriptor,
               columnChunkMetaData, fixedLength, (NullableIntervalVector) v, schemaElement);
@@ -226,10 +218,8 @@
           case ENUM:
             return new VarLengthColumnReaders.VarCharColumn(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (VarCharVector) v, schemaElement);
           case DECIMAL:
-            if (v instanceof Decimal28SparseVector) {
-              return new VarLengthColumnReaders.Decimal28Column(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (Decimal28SparseVector) v, schemaElement);
-            } else if (v instanceof Decimal38SparseVector) {
-              return new VarLengthColumnReaders.Decimal38Column(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (Decimal38SparseVector) v, schemaElement);
+            if (v instanceof VarDecimalVector) {
+              return new VarLengthColumnReaders.VarDecimalColumn(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (VarDecimalVector) v, schemaElement);
             }
           default:
             return new VarLengthColumnReaders.VarBinaryColumn(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (VarBinaryVector) v, schemaElement);
@@ -244,10 +234,8 @@
           case ENUM:
             return new VarLengthColumnReaders.NullableVarCharColumn(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (NullableVarCharVector) v, schemaElement);
           case DECIMAL:
-            if (v instanceof NullableDecimal28SparseVector) {
-              return new VarLengthColumnReaders.NullableDecimal28Column(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (NullableDecimal28SparseVector) v, schemaElement);
-            } else if (v instanceof NullableDecimal38SparseVector) {
-              return new VarLengthColumnReaders.NullableDecimal38Column(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (NullableDecimal38SparseVector) v, schemaElement);
+            if (v instanceof NullableVarDecimalVector) {
+              return new VarLengthColumnReaders.NullableVarDecimalColumn(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (NullableVarDecimalVector) v, schemaElement);
             }
           default:
             return new VarLengthColumnReaders.NullableVarBinaryColumn(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, (NullableVarBinaryVector) v, schemaElement);
@@ -271,7 +259,13 @@
         } else {
           return new NullableFixedByteAlignedReaders.NullableFixedBinaryReader(parentReader, allocateSize, columnDescriptor, columnChunkMetaData, true, (NullableVarBinaryVector) valueVec, schemaElement);
         }
-      }else{
+      } else if (convertedType == ConvertedType.DECIMAL) {
+        // NullableVarDecimalVector allows storing of values with different width,
+        // so every time when the value is added, offset vector should be updated.
+        // Therefore NullableVarDecimalReader is used here instead of NullableFixedByteAlignedReader.
+        return new NullableFixedByteAlignedReaders.NullableVarDecimalReader(parentReader, allocateSize,
+            columnDescriptor, columnChunkMetaData, fixedLength, (NullableVarDecimalVector) valueVec, schemaElement);
+      } else {
         return new NullableFixedByteAlignedReaders.NullableFixedByteAlignedReader<>(parentReader, allocateSize, columnDescriptor, columnChunkMetaData, fixedLength, valueVec, schemaElement);
       }
     } else {
@@ -282,7 +276,8 @@
           }
           switch (convertedType) {
             case DECIMAL:
-              return new NullableFixedByteAlignedReaders.NullableDictionaryDecimal9Reader(parentReader, allocateSize, columnDescriptor, columnChunkMetaData, fixedLength, (NullableDecimal9Vector) valueVec, schemaElement);
+              return new NullableFixedByteAlignedReaders.NullableDictionaryVarDecimalReader(parentReader, allocateSize,
+                  columnDescriptor, columnChunkMetaData, fixedLength, (NullableVarDecimalVector) valueVec, schemaElement);
             case TIME_MILLIS:
               return new NullableFixedByteAlignedReaders.NullableDictionaryTimeReader(parentReader, allocateSize, columnDescriptor, columnChunkMetaData, fixedLength, (NullableTimeVector)valueVec, schemaElement);
             default:
@@ -294,7 +289,8 @@
           }
           switch (convertedType) {
             case DECIMAL:
-              return new NullableFixedByteAlignedReaders.NullableDictionaryDecimal18Reader(parentReader, allocateSize, columnDescriptor, columnChunkMetaData, fixedLength, (NullableDecimal18Vector)valueVec, schemaElement);
+              return new NullableFixedByteAlignedReaders.NullableDictionaryVarDecimalReader(parentReader, allocateSize,
+                  columnDescriptor, columnChunkMetaData, fixedLength, (NullableVarDecimalVector) valueVec, schemaElement);
             case TIMESTAMP_MILLIS:
               return new NullableFixedByteAlignedReaders.NullableDictionaryTimeStampReader(parentReader, allocateSize, columnDescriptor, columnChunkMetaData, fixedLength, (NullableTimeStampVector)valueVec, schemaElement);
             default:
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedByteAlignedReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedByteAlignedReader.java
index e6708ec0b3..385b117c13 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedByteAlignedReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/FixedByteAlignedReader.java
@@ -17,18 +17,14 @@
  */
 package org.apache.drill.exec.store.parquet.columnreaders;
 
-import java.math.BigDecimal;
-
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
-import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
 import org.apache.drill.exec.store.parquet.ParquetReaderUtility;
-import org.apache.drill.exec.util.DecimalUtility;
 import org.apache.drill.exec.vector.DateVector;
-import org.apache.drill.exec.vector.Decimal28SparseVector;
-import org.apache.drill.exec.vector.Decimal38SparseVector;
 import org.apache.drill.exec.vector.IntervalVector;
 import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarDecimalVector;
 import org.apache.drill.exec.vector.VariableWidthVector;
 import org.apache.parquet.column.ColumnDescriptor;
 import org.apache.parquet.format.SchemaElement;
@@ -199,37 +195,27 @@ void addNext(int start, int index) {
 
   }
 
-  public static class Decimal28Reader extends ConvertedReader<Decimal28SparseVector> {
+  public static class VarDecimalReader extends ConvertedReader<VarDecimalVector> {
 
-    Decimal28Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
-                    boolean fixedLength, Decimal28SparseVector v, SchemaElement schemaElement) throws ExecutionSetupException {
+    VarDecimalReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
+        boolean fixedLength, VarDecimalVector v, SchemaElement schemaElement) throws ExecutionSetupException {
       super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
     }
 
     @Override
     void addNext(int start, int index) {
-      int width = Decimal28SparseHolder.WIDTH;
-      BigDecimal intermediate = DecimalUtility.getBigDecimalFromDrillBuf(bytebuf, start, dataTypeLengthInBytes,
-          schemaElement.getScale());
-      DecimalUtility.getSparseFromBigDecimal(intermediate, valueVec.getBuffer(), index * width, schemaElement.getScale(),
-              schemaElement.getPrecision(), Decimal28SparseHolder.nDecimalDigits);
-    }
-  }
-
-  public static class Decimal38Reader extends ConvertedReader<Decimal38SparseVector> {
-
-    Decimal38Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
-                    boolean fixedLength, Decimal38SparseVector v, SchemaElement schemaElement) throws ExecutionSetupException {
-      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
-    }
-
-    @Override
-    void addNext(int start, int index) {
-      int width = Decimal38SparseHolder.WIDTH;
-      BigDecimal intermediate = DecimalUtility.getBigDecimalFromDrillBuf(bytebuf, start, dataTypeLengthInBytes,
-          schemaElement.getScale());
-      DecimalUtility.getSparseFromBigDecimal(intermediate, valueVec.getBuffer(), index * width, schemaElement.getScale(),
-              schemaElement.getPrecision(), Decimal38SparseHolder.nDecimalDigits);
+      switch (columnChunkMetaData.getType()) {
+        case INT32:
+          valueVec.getMutator().setSafe(index, Ints.toByteArray(bytebuf.getInt(start)), 0, dataTypeLengthInBytes);
+          break;
+        case INT64:
+          valueVec.getMutator().setSafe(index, Longs.toByteArray(bytebuf.getLong(start)), 0, dataTypeLengthInBytes);
+          break;
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          valueVec.getMutator().setSafe(index, start, start + dataTypeLengthInBytes, bytebuf);
+          break;
+      }
     }
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
index 86cba6e664..9e66b1f1b0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
@@ -17,21 +17,15 @@
  */
 package org.apache.drill.exec.store.parquet.columnreaders;
 
-import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.exec.expr.holders.NullableDecimal28SparseHolder;
-import org.apache.drill.exec.expr.holders.NullableDecimal38SparseHolder;
 import org.apache.drill.exec.expr.holders.NullableTimeStampHolder;
 import org.apache.drill.exec.store.parquet.ParquetReaderUtility;
-import org.apache.drill.exec.util.DecimalUtility;
 import org.apache.drill.exec.vector.NullableBigIntVector;
 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;
@@ -39,6 +33,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.NullableVarDecimalVector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.parquet.column.ColumnDescriptor;
 import org.apache.parquet.format.SchemaElement;
@@ -164,29 +159,6 @@ protected void readField(long recordsToReadInThisPass) {
     }
   }
 
-  static class NullableDictionaryDecimal9Reader extends NullableColumnReader<NullableDecimal9Vector> {
-
-    NullableDictionaryDecimal9Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
-                                ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, NullableDecimal9Vector v,
-                                SchemaElement schemaElement) throws ExecutionSetupException {
-      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
-    }
-
-    // this method is called by its superclass during a read loop
-    @Override
-    protected void readField(long recordsToReadInThisPass) {
-      if (usingDictionary) {
-        for (int i = 0; i < recordsToReadInThisPass; i++){
-          valueVec.getMutator().setSafe(valuesReadInCurrentPass + i, pageReader.dictionaryValueReader.readInteger());
-        }
-      } else {
-        for (int i = 0; i < recordsToReadInThisPass; i++){
-          valueVec.getMutator().setSafe(valuesReadInCurrentPass + i, pageReader.valueReader.readInteger());
-        }
-      }
-    }
-  }
-
   static class NullableDictionaryTimeReader extends NullableColumnReader<NullableTimeVector> {
 
     NullableDictionaryTimeReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
@@ -255,28 +227,53 @@ protected void readField(long recordsToReadInThisPass) {
       }
     }
   }
-  static class NullableDictionaryDecimal18Reader extends NullableColumnReader<NullableDecimal18Vector> {
 
-    NullableDictionaryDecimal18Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
-                                   ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, NullableDecimal18Vector v,
-                                   SchemaElement schemaElement) throws ExecutionSetupException {
+  static class NullableDictionaryVarDecimalReader extends NullableColumnReader<NullableVarDecimalVector> {
+
+    NullableDictionaryVarDecimalReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
+        ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, NullableVarDecimalVector v,
+        SchemaElement schemaElement) throws ExecutionSetupException {
       super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
     }
 
     // this method is called by its superclass during a read loop
     @Override
     protected void readField(long recordsToReadInThisPass) {
-      if (usingDictionary) {
-        for (int i = 0; i < recordsToReadInThisPass; i++){
-          valueVec.getMutator().setSafe(valuesReadInCurrentPass + i, pageReader.dictionaryValueReader.readLong());
-        }
-      } else {
-        for (int i = 0; i < recordsToReadInThisPass; i++){
-          valueVec.getMutator().setSafe(valuesReadInCurrentPass + i, pageReader.valueReader.readLong());
-        }
+      switch (columnDescriptor.getType()) {
+        case INT32:
+          if (usingDictionary) {
+            for (int i = 0; i < recordsToReadInThisPass; i++) {
+              byte[] bytes = Ints.toByteArray(pageReader.dictionaryValueReader.readInteger());
+              setValueBytes(i, bytes);
+            }
+          } else {
+            for (int i = 0; i < recordsToReadInThisPass; i++) {
+              byte[] bytes = Ints.toByteArray(pageReader.valueReader.readInteger());
+              setValueBytes(i, bytes);
+            }
+          }
+          break;
+        case INT64:
+          if (usingDictionary) {
+            for (int i = 0; i < recordsToReadInThisPass; i++) {
+              byte[] bytes = Longs.toByteArray(pageReader.dictionaryValueReader.readLong());
+              setValueBytes(i, bytes);
+            }
+          } else {
+            for (int i = 0; i < recordsToReadInThisPass; i++) {
+              byte[] bytes = Longs.toByteArray(pageReader.valueReader.readLong());
+              setValueBytes(i, bytes);
+            }
+          }
+          break;
       }
     }
+
+    private void setValueBytes(int i, byte[] bytes) {
+      valueVec.getMutator().setSafe(valuesReadInCurrentPass + i, bytes, 0, bytes.length);
+    }
   }
+
   static class NullableDictionaryFloat4Reader extends NullableColumnReader<NullableFloat4Vector> {
 
     NullableDictionaryFloat4Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
@@ -398,14 +395,11 @@ void addNext(int start, int index) {
    */
   public static class CorruptionDetectingNullableDateReader extends NullableConvertedReader<NullableDateVector> {
 
-    NullableDateVector dateVector;
-
     CorruptionDetectingNullableDateReader(ParquetRecordReader parentReader, int allocateSize,
                                           ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
                                           boolean fixedLength, NullableDateVector v, SchemaElement schemaElement)
         throws ExecutionSetupException {
       super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
-      dateVector = (NullableDateVector) v;
     }
 
     @Override
@@ -418,43 +412,35 @@ void addNext(int start, int index) {
       }
 
       if (intValue > ParquetReaderUtility.DATE_CORRUPTION_THRESHOLD) {
-        dateVector.getMutator().set(index, (intValue - ParquetReaderUtility.CORRECT_CORRUPT_DATE_SHIFT) * DateTimeConstants.MILLIS_PER_DAY);
+        valueVec.getMutator().set(index, (intValue - ParquetReaderUtility.CORRECT_CORRUPT_DATE_SHIFT) * DateTimeConstants.MILLIS_PER_DAY);
       } else {
-        dateVector.getMutator().set(index, intValue * (long) DateTimeConstants.MILLIS_PER_DAY);
+        valueVec.getMutator().set(index, intValue * (long) DateTimeConstants.MILLIS_PER_DAY);
       }
     }
   }
 
-  public static class NullableDecimal28Reader extends NullableConvertedReader<NullableDecimal28SparseVector> {
-
-    NullableDecimal28Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
-                            boolean fixedLength, NullableDecimal28SparseVector v, SchemaElement schemaElement) throws ExecutionSetupException {
+  public static class NullableVarDecimalReader extends NullableConvertedReader<NullableVarDecimalVector> {
+    NullableVarDecimalReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
+        boolean fixedLength, NullableVarDecimalVector v, SchemaElement schemaElement) throws ExecutionSetupException {
       super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
     }
 
+    // TODO: allow reading page instead of reading every record separately
     @Override
     void addNext(int start, int index) {
-      int width = NullableDecimal28SparseHolder.WIDTH;
-      BigDecimal intermediate = DecimalUtility.getBigDecimalFromDrillBuf(bytebuf, start, dataTypeLengthInBytes,
-          schemaElement.getScale());
-      DecimalUtility.getSparseFromBigDecimal(intermediate, valueVec.getBuffer(), index * width, schemaElement.getScale(),
-          schemaElement.getPrecision(), NullableDecimal28SparseHolder.nDecimalDigits);
-    }
-  }
-
-  public static class NullableDecimal38Reader extends NullableConvertedReader<NullableDecimal38SparseVector> {
-    NullableDecimal38Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor, ColumnChunkMetaData columnChunkMetaData,
-                            boolean fixedLength, NullableDecimal38SparseVector v, SchemaElement schemaElement) throws ExecutionSetupException {
-      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
-    }
+      switch (columnChunkMetaData.getType()) {
+        case INT32:
+          valueVec.getMutator().setSafe(index, Ints.toByteArray(bytebuf.getInt(start)), 0, dataTypeLengthInBytes);
+          break;
+        case INT64:
+          valueVec.getMutator().setSafe(index, Longs.toByteArray(bytebuf.getLong(start)), 0, dataTypeLengthInBytes);
+          break;
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          valueVec.getMutator().setSafe(index, 1, start, start + dataTypeLengthInBytes, bytebuf);
+          break;
+      }
 
-    @Override
-    void addNext(int start, int index) {
-      int width = NullableDecimal38SparseHolder.WIDTH;
-      BigDecimal intermediate = DecimalUtility.getBigDecimalFromDrillBuf(bytebuf, start, dataTypeLengthInBytes,
-          schemaElement.getScale());
-      DecimalUtility.getSparseFromBigDecimal(intermediate, valueVec.getBuffer(), index * width, schemaElement.getScale(),
-          schemaElement.getPrecision(), NullableDecimal38SparseHolder.nDecimalDigits);
     }
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetFixedWidthDictionaryReaders.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetFixedWidthDictionaryReaders.java
index 5bc9ea11b2..38ab700375 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetFixedWidthDictionaryReaders.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetFixedWidthDictionaryReaders.java
@@ -17,10 +17,10 @@
  */
 package org.apache.drill.exec.store.parquet.columnreaders;
 
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.vector.BigIntVector;
-import org.apache.drill.exec.vector.Decimal18Vector;
-import org.apache.drill.exec.vector.Decimal9Vector;
 import org.apache.drill.exec.vector.Float4Vector;
 import org.apache.drill.exec.vector.Float8Vector;
 import org.apache.drill.exec.vector.IntVector;
@@ -29,6 +29,7 @@
 import org.apache.drill.exec.vector.UInt4Vector;
 import org.apache.drill.exec.vector.UInt8Vector;
 import org.apache.drill.exec.vector.VarBinaryVector;
+import org.apache.drill.exec.vector.VarDecimalVector;
 import org.apache.parquet.column.ColumnDescriptor;
 import org.apache.parquet.format.SchemaElement;
 import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
@@ -38,6 +39,8 @@
 
 public class ParquetFixedWidthDictionaryReaders {
 
+  private static final double BITS_COUNT_IN_BYTE_DOUBLE_VALUE = 8.0;
+
   static class DictionaryIntReader extends FixedByteAlignedReader<IntVector> {
     DictionaryIntReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
                                 ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, IntVector v,
@@ -86,7 +89,7 @@ protected void readField(long recordsToReadInThisPass) {
         // and we will go into the else condition below. The readField method of the parent class requires the
         // writer index to be set correctly.
         readLengthInBits = recordsReadInThisIteration * dataTypeLengthInBits;
-        readLength = (int) Math.ceil(readLengthInBits / 8.0);
+        readLength = (int) Math.ceil(readLengthInBits / BITS_COUNT_IN_BYTE_DOUBLE_VALUE);
         int writerIndex = valueVec.getBuffer().writerIndex();
         valueVec.getBuffer().setIndex(0, writerIndex + (int) readLength);
       } else {
@@ -109,7 +112,7 @@ protected void readField(long recordsToReadInThisPass) {
       recordsReadInThisIteration = Math.min(pageReader.currentPageCount
           - pageReader.valuesRead, recordsToReadInThisPass - valuesReadInCurrentPass);
       readLengthInBits = recordsReadInThisIteration * dataTypeLengthInBits;
-      readLength = (int) Math.ceil(readLengthInBits / 8.0);
+      readLength = (int) Math.ceil(readLengthInBits / BITS_COUNT_IN_BYTE_DOUBLE_VALUE);
 
       if (usingDictionary) {
         VarBinaryVector.Mutator mutator =  valueVec.getMutator();
@@ -137,28 +140,6 @@ protected void readField(long recordsToReadInThisPass) {
     }
   }
 
-  static class DictionaryDecimal9Reader extends FixedByteAlignedReader<Decimal9Vector> {
-    DictionaryDecimal9Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
-                        ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, Decimal9Vector v,
-                        SchemaElement schemaElement) throws ExecutionSetupException {
-      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
-    }
-
-    // this method is called by its superclass during a read loop
-    @Override
-    protected void readField(long recordsToReadInThisPass) {
-
-      recordsReadInThisIteration = Math.min(pageReader.currentPageCount
-        - pageReader.valuesRead, recordsToReadInThisPass - valuesReadInCurrentPass);
-
-      if (usingDictionary) {
-        for (int i = 0; i < recordsReadInThisIteration; i++){
-          valueVec.getMutator().setSafe(valuesReadInCurrentPass + i, pageReader.dictionaryValueReader.readInteger());
-        }
-      }
-    }
-  }
-
   static class DictionaryTimeReader extends FixedByteAlignedReader<TimeVector> {
     DictionaryTimeReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
                         ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, TimeVector v,
@@ -204,7 +185,7 @@ protected void readField(long recordsToReadInThisPass) {
         // and we will go into the else condition below. The readField method of the parent class requires the
         // writer index to be set correctly.
         readLengthInBits = recordsReadInThisIteration * dataTypeLengthInBits;
-        readLength = (int) Math.ceil(readLengthInBits / 8.0);
+        readLength = (int) Math.ceil(readLengthInBits / BITS_COUNT_IN_BYTE_DOUBLE_VALUE);
         int writerIndex = valueVec.getBuffer().writerIndex();
         valueVec.getBuffer().setIndex(0, writerIndex + (int)readLength);
       } else {
@@ -239,7 +220,7 @@ protected void readField(long recordsToReadInThisPass) {
         // and we will go into the else condition below. The readField method of the parent class requires the
         // writer index to be set correctly.
         readLengthInBits = recordsReadInThisIteration * dataTypeLengthInBits;
-        readLength = (int) Math.ceil(readLengthInBits / 8.0);
+        readLength = (int) Math.ceil(readLengthInBits / BITS_COUNT_IN_BYTE_DOUBLE_VALUE);
         int writerIndex = valueVec.getBuffer().writerIndex();
         valueVec.getBuffer().setIndex(0, writerIndex + (int) readLength);
       } else {
@@ -248,27 +229,61 @@ protected void readField(long recordsToReadInThisPass) {
     }
   }
 
-  static class DictionaryDecimal18Reader extends FixedByteAlignedReader<Decimal18Vector> {
-    DictionaryDecimal18Reader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
-                           ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, Decimal18Vector v,
-                           SchemaElement schemaElement) throws ExecutionSetupException {
+  static class DictionaryVarDecimalReader extends FixedByteAlignedReader<VarDecimalVector> {
+
+    DictionaryVarDecimalReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
+        ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, VarDecimalVector v,
+        SchemaElement schemaElement) throws ExecutionSetupException {
       super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
     }
 
     // this method is called by its superclass during a read loop
     @Override
     protected void readField(long recordsToReadInThisPass) {
+      recordsReadInThisIteration =
+          Math.min(pageReader.currentPageCount - pageReader.valuesRead,
+              recordsToReadInThisPass - valuesReadInCurrentPass);
+
+      switch (columnDescriptor.getType()) {
+        case INT32:
+          if (usingDictionary) {
+            for (int i = 0; i < recordsReadInThisIteration; i++) {
+              byte[] bytes = Ints.toByteArray(pageReader.dictionaryValueReader.readInteger());
+              setValueBytes(i, bytes);
+            }
+            setWriteIndex();
+          } else {
+            super.readField(recordsToReadInThisPass);
+          }
+          break;
+        case INT64:
+          if (usingDictionary) {
+            for (int i = 0; i < recordsReadInThisIteration; i++) {
+              byte[] bytes = Longs.toByteArray(pageReader.dictionaryValueReader.readLong());
+              setValueBytes(i, bytes);
+            }
+            setWriteIndex();
+          } else {
+            super.readField(recordsToReadInThisPass);
+          }
+          break;
+      }
+    }
 
-      recordsReadInThisIteration = Math.min(pageReader.currentPageCount
-        - pageReader.valuesRead, recordsToReadInThisPass - valuesReadInCurrentPass);
+    /**
+     * Set the write Index. The next page that gets read might be a page that does not use dictionary encoding
+     * and we will go into the else condition below. The readField method of the parent class requires the
+     * writer index to be set correctly.
+     */
+    private void setWriteIndex() {
+      readLengthInBits = recordsReadInThisIteration * dataTypeLengthInBits;
+      readLength = (int) Math.ceil(readLengthInBits / BITS_COUNT_IN_BYTE_DOUBLE_VALUE);
+      int writerIndex = valueVec.getBuffer().writerIndex();
+      valueVec.getBuffer().setIndex(0, writerIndex + (int) readLength);
+    }
 
-      for (int i = 0; i < recordsReadInThisIteration; i++){
-        try {
-          valueVec.getMutator().setSafe(valuesReadInCurrentPass + i, pageReader.dictionaryValueReader.readLong());
-        } catch ( Exception ex) {
-          throw ex;
-        }
-      }
+    private void setValueBytes(int i, byte[] bytes) {
+      valueVec.getMutator().setSafe(valuesReadInCurrentPass + i, bytes, 0, bytes.length);
     }
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java
index fc1c4af055..309ed89bfc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java
@@ -32,15 +32,8 @@
 
 public class ParquetToDrillTypeConverter {
 
-
-  private static TypeProtos.MinorType getDecimalType(int precision) {
-    return precision <= 28 ? TypeProtos.MinorType.DECIMAL28SPARSE : MinorType.DECIMAL38SPARSE;
-  }
-
   private static TypeProtos.MinorType getMinorType(PrimitiveType.PrimitiveTypeName primitiveTypeName, int length,
-                                                   ConvertedType convertedType, int precision, OptionManager options) {
-
-
+      ConvertedType convertedType, OptionManager options) {
     switch (primitiveTypeName) {
       case BINARY:
         if (convertedType == null) {
@@ -52,7 +45,7 @@
             return (TypeProtos.MinorType.VARCHAR);
           case DECIMAL:
             ParquetReaderUtility.checkDecimalTypeEnabled(options);
-            return (getDecimalType(precision));
+            return TypeProtos.MinorType.VARDECIMAL;
           default:
             return (TypeProtos.MinorType.VARBINARY);
         }
@@ -67,7 +60,7 @@
             return TypeProtos.MinorType.UINT8;
           case DECIMAL:
             ParquetReaderUtility.checkDecimalTypeEnabled(options);
-            return TypeProtos.MinorType.DECIMAL18;
+            return TypeProtos.MinorType.VARDECIMAL;
           // TODO - add this back if it is decided to be added upstream, was removed form our pull request July 2014
 //              case TIME_MICROS:
 //                throw new UnsupportedOperationException();
@@ -91,7 +84,7 @@
             return TypeProtos.MinorType.INT;
           case DECIMAL:
             ParquetReaderUtility.checkDecimalTypeEnabled(options);
-            return TypeProtos.MinorType.DECIMAL9;
+            return TypeProtos.MinorType.VARDECIMAL;
           case DATE:
             return TypeProtos.MinorType.DATE;
           case TIME_MILLIS:
@@ -119,7 +112,7 @@
           return TypeProtos.MinorType.VARBINARY;
         } else if (convertedType == ConvertedType.DECIMAL) {
           ParquetReaderUtility.checkDecimalTypeEnabled(options);
-          return getDecimalType(precision);
+          return TypeProtos.MinorType.VARDECIMAL;
         } else if (convertedType == ConvertedType.INTERVAL) {
           return TypeProtos.MinorType.INTERVAL;
         }
@@ -131,17 +124,14 @@
   public static TypeProtos.MajorType toMajorType(PrimitiveType.PrimitiveTypeName primitiveTypeName, int length,
                                           TypeProtos.DataMode mode, SchemaElement schemaElement,
                                           OptionManager options) {
-    return toMajorType(primitiveTypeName, length, mode, schemaElement.getConverted_type(),
-        schemaElement.getPrecision(), schemaElement.getScale(), options);
-  }
-
-  public static TypeProtos.MajorType toMajorType(PrimitiveType.PrimitiveTypeName primitiveTypeName, int length,
-      TypeProtos.DataMode mode, ConvertedType convertedType, int precision, int scale,
-      OptionManager options) {
-    MinorType minorType = getMinorType(primitiveTypeName, length, convertedType, precision, options);
+    ConvertedType convertedType = schemaElement.getConverted_type();
+    MinorType minorType = getMinorType(primitiveTypeName, length, convertedType, options);
     TypeProtos.MajorType.Builder typeBuilder = TypeProtos.MajorType.newBuilder().setMinorType(minorType).setMode(mode);
 
     if (CoreDecimalUtility.isDecimalType(minorType)) {
+      int precision = schemaElement.getPrecision();
+      int scale = schemaElement.getScale();
+
       typeBuilder.setPrecision(precision).setScale(scale);
     }
     return typeBuilder.build();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLengthColumnReaders.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLengthColumnReaders.java
index b3f511f009..dd61abda27 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLengthColumnReaders.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/VarLengthColumnReaders.java
@@ -19,22 +19,17 @@
 
 import io.netty.buffer.DrillBuf;
 
-import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
-import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
-import org.apache.drill.exec.util.DecimalUtility;
-import org.apache.drill.exec.vector.Decimal28SparseVector;
-import org.apache.drill.exec.vector.Decimal38SparseVector;
-import org.apache.drill.exec.vector.NullableDecimal28SparseVector;
-import org.apache.drill.exec.vector.NullableDecimal38SparseVector;
+import org.apache.drill.exec.vector.VarDecimalVector;
+import org.apache.drill.exec.vector.NullableVarDecimalVector;
 import org.apache.drill.exec.vector.NullableVarBinaryVector;
 import org.apache.drill.exec.vector.NullableVarCharVector;
 import org.apache.drill.exec.vector.VarBinaryVector;
 import org.apache.drill.exec.vector.VarCharVector;
 
+
 import org.apache.parquet.column.ColumnDescriptor;
 import org.apache.parquet.format.SchemaElement;
 import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
@@ -42,126 +37,70 @@
 public class VarLengthColumnReaders {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VarLengthColumnReaders.class);
 
-  public static class Decimal28Column extends VarLengthValuesColumn<Decimal28SparseVector> {
-
-    protected Decimal28SparseVector decimal28Vector;
+  public static class VarDecimalColumn extends VarLengthValuesColumn<VarDecimalVector> {
 
-    Decimal28Column(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
-                   ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, Decimal28SparseVector v,
-                   SchemaElement schemaElement) throws ExecutionSetupException {
-      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
-      this.decimal28Vector = v;
-    }
-
-    @Override
-    public boolean setSafe(int index, DrillBuf bytebuf, int start, int length) {
-      int width = Decimal28SparseHolder.WIDTH;
-      BigDecimal intermediate = DecimalUtility.getBigDecimalFromDrillBuf(bytebuf, start, length,
-          schemaElement.getScale());
-      if (index >= decimal28Vector.getValueCapacity()) {
-        return false;
-      }
-      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal28Vector.getBuffer(), index * width, schemaElement.getScale(),
-              schemaElement.getPrecision(), Decimal28SparseHolder.nDecimalDigits);
-      return true;
-    }
+    protected VarDecimalVector varDecimalVector;
+    protected VarDecimalVector.Mutator mutator;
 
-    @Override
-    public int capacity() {
-      return decimal28Vector.getBuffer().capacity();
-    }
-  }
-
-  public static class NullableDecimal28Column extends NullableVarLengthValuesColumn<NullableDecimal28SparseVector> {
-
-    protected NullableDecimal28SparseVector nullableDecimal28Vector;
-
-    NullableDecimal28Column(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
-                    ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, NullableDecimal28SparseVector v,
+    VarDecimalColumn(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
+                    ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, VarDecimalVector v,
                     SchemaElement schemaElement) throws ExecutionSetupException {
       super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
-      nullableDecimal28Vector = v;
+      this.varDecimalVector = v;
+      this.mutator = v.getMutator();
     }
 
     @Override
-    public boolean setSafe(int index, DrillBuf bytebuf, int start, int length) {
-      int width = Decimal28SparseHolder.WIDTH;
-      BigDecimal intermediate = DecimalUtility.getBigDecimalFromDrillBuf(bytebuf, start, length,
-          schemaElement.getScale());
-      if (index >= nullableDecimal28Vector.getValueCapacity()) {
+    public boolean setSafe(int index, DrillBuf value, int start, int length) {
+      if (index >= varDecimalVector.getValueCapacity()) {
         return false;
       }
-      DecimalUtility.getSparseFromBigDecimal(intermediate, nullableDecimal28Vector.getBuffer(), index * width, schemaElement.getScale(),
-              schemaElement.getPrecision(), Decimal28SparseHolder.nDecimalDigits);
-      nullableDecimal28Vector.getMutator().setIndexDefined(index);
-      return true;
-    }
-
-    @Override
-    public int capacity() {
-      return nullableDecimal28Vector.getBuffer().capacity();
-    }
-  }
-
-  public static class Decimal38Column extends VarLengthValuesColumn<Decimal38SparseVector> {
-
-    protected Decimal38SparseVector decimal28Vector;
-
-    Decimal38Column(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
-                    ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, Decimal38SparseVector v,
-                    SchemaElement schemaElement) throws ExecutionSetupException {
-      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
-      decimal28Vector = v;
-    }
-
-    @Override
-    public boolean setSafe(int index, DrillBuf bytebuf, int start, int length) {
-      int width = Decimal38SparseHolder.WIDTH;
-      BigDecimal intermediate = DecimalUtility.getBigDecimalFromDrillBuf(bytebuf, start, length,
-          schemaElement.getScale());
-      if (index >= decimal28Vector.getValueCapacity()) {
-        return false;
+      if (usingDictionary) {
+        currDictValToWrite = pageReader.dictionaryValueReader.readBytes();
+        ByteBuffer buf = currDictValToWrite.toByteBuffer();
+        mutator.setSafe(index, buf, buf.position(), currDictValToWrite.length());
+      } else {
+        mutator.setSafe(index, start, start + length, value);
       }
-      DecimalUtility.getSparseFromBigDecimal(intermediate, decimal28Vector.getBuffer(), index * width, schemaElement.getScale(),
-              schemaElement.getPrecision(), Decimal38SparseHolder.nDecimalDigits);
       return true;
     }
 
     @Override
     public int capacity() {
-      return decimal28Vector.getBuffer().capacity();
+      return varDecimalVector.getBuffer().capacity();
     }
   }
 
-  public static class NullableDecimal38Column extends NullableVarLengthValuesColumn<NullableDecimal38SparseVector> {
+  public static class NullableVarDecimalColumn extends NullableVarLengthValuesColumn<NullableVarDecimalVector> {
 
-    private final NullableDecimal38SparseVector nullableDecimal38Vector;
+    protected NullableVarDecimalVector nullableVarDecimalVector;
+    protected NullableVarDecimalVector.Mutator mutator;
 
-    NullableDecimal38Column(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
-                            ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, NullableDecimal38SparseVector v,
+    NullableVarDecimalColumn(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
+                            ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, NullableVarDecimalVector v,
                             SchemaElement schemaElement) throws ExecutionSetupException {
       super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
-      nullableDecimal38Vector = v;
+      nullableVarDecimalVector = v;
+      this.mutator = v.getMutator();
     }
 
     @Override
-    public boolean setSafe(int index, DrillBuf bytebuf, int start, int length) {
-      int width = Decimal38SparseHolder.WIDTH;
-      BigDecimal intermediate = DecimalUtility.getBigDecimalFromDrillBuf(bytebuf, start, length,
-          schemaElement.getScale());
-      if (index >= nullableDecimal38Vector.getValueCapacity()) {
+    public boolean setSafe(int index, DrillBuf value, int start, int length) {
+      if (index >= nullableVarDecimalVector.getValueCapacity()) {
         return false;
       }
-
-      DecimalUtility.getSparseFromBigDecimal(intermediate, nullableDecimal38Vector.getBuffer(), index * width, schemaElement.getScale(),
-              schemaElement.getPrecision(), Decimal38SparseHolder.nDecimalDigits);
-      nullableDecimal38Vector.getMutator().setIndexDefined(index);
+      if (usingDictionary) {
+        ByteBuffer buf = currDictValToWrite.toByteBuffer();
+        mutator.setSafe(index, buf, buf.position(), currDictValToWrite.length());
+      } else {
+        mutator.setSafe(index, 1, start, start + length, value);
+      }
       return true;
     }
 
     @Override
     public int capacity() {
-      return nullableDecimal38Vector.getBuffer().capacity();
+      return nullableVarDecimalVector.getBuffer().capacity();
     }
   }
 
@@ -203,8 +142,6 @@ public int capacity() {
 
   public static class NullableVarCharColumn extends NullableVarLengthValuesColumn<NullableVarCharVector> {
 
-    int nullsRead;
-    boolean currentValNull = false;
     // store a hard reference to the vector (which is also stored in the superclass) to prevent repetitive casting
     protected final NullableVarCharVector.Mutator mutator;
     private final NullableVarCharVector vector;
@@ -276,8 +213,6 @@ public int capacity() {
 
   public static class NullableVarBinaryColumn extends NullableVarLengthValuesColumn<NullableVarBinaryVector> {
 
-    int nullsRead;
-    boolean currentValNull = false;
     // store a hard reference to the vector (which is also stored in the superclass) to prevent repetitive casting
     private final NullableVarBinaryVector nullableVarBinaryVector;
     private final NullableVarBinaryVector.Mutator mutator;
@@ -290,7 +225,6 @@ public int capacity() {
       mutator = v.getMutator();
     }
 
-
     @Override
     public boolean setSafe(int index, DrillBuf value, int start, int length) {
       if (index >= nullableVarBinaryVector.getValueCapacity()) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/decimal/BinaryDecimalParquetValueWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/decimal/BinaryDecimalParquetValueWriter.java
new file mode 100644
index 0000000000..85d9f80ffe
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/decimal/BinaryDecimalParquetValueWriter.java
@@ -0,0 +1,34 @@
+/*
+ * 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.exec.store.parquet.decimal;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.io.api.RecordConsumer;
+
+/**
+ * Parquet value writer for passing decimal values
+ * into {@code RecordConsumer} to be stored as BINARY type.
+ */
+public class BinaryDecimalParquetValueWriter extends DecimalValueWriter {
+
+  @Override
+  public void writeValue(RecordConsumer consumer, DrillBuf buffer, int start, int end, int precision) {
+    consumer.addBinary(Binary.fromReusedByteBuffer(buffer.nioBuffer(start, end - start)));
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/decimal/DecimalValueWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/decimal/DecimalValueWriter.java
new file mode 100644
index 0000000000..ed70a41756
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/decimal/DecimalValueWriter.java
@@ -0,0 +1,61 @@
+/*
+ * 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.exec.store.parquet.decimal;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.parquet.io.api.RecordConsumer;
+import org.apache.parquet.schema.PrimitiveType;
+
+public abstract class DecimalValueWriter {
+
+  /**
+   * Takes value from specified {@code DrillBuf buffer}, converts it into required format
+   * if needed and passes the value into specified {@code RecordConsumer consumer}.
+   *
+   * @param consumer  abstraction for writing records
+   * @param buffer    the source of data that should be written
+   * @param start     start position of data in buffer
+   * @param end       end position of data in buffer
+   * @param precision precision of the value that should be stored
+   */
+  public abstract void writeValue(RecordConsumer consumer, DrillBuf buffer,
+      int start, int end, int precision);
+
+  /**
+   * Creates and returns writer suitable for specified {@code PrimitiveType.PrimitiveTypeName type}.
+   *
+   * @param type the type of the value in output file.
+   * @return writer for decimal values.
+   */
+  public static DecimalValueWriter getDecimalValueWriterForType(
+      PrimitiveType.PrimitiveTypeName type) {
+    switch (type) {
+      case INT32:
+        return new Int32DecimalParquetValueWriter();
+      case INT64:
+        return new Int64DecimalParquetValueWriter();
+      case FIXED_LEN_BYTE_ARRAY:
+        return new FixedLenDecimalParquetValueWriter();
+      case BINARY:
+        return new BinaryDecimalParquetValueWriter();
+      default:
+        throw new UnsupportedOperationException(
+            String.format("Specified PrimitiveTypeName %s cannot be used to store decimal values", type));
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/decimal/FixedLenDecimalParquetValueWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/decimal/FixedLenDecimalParquetValueWriter.java
new file mode 100644
index 0000000000..893d9dcd35
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/decimal/FixedLenDecimalParquetValueWriter.java
@@ -0,0 +1,50 @@
+/*
+ * 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.exec.store.parquet.decimal;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.util.DecimalUtility;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.io.api.RecordConsumer;
+
+import java.util.Arrays;
+
+/**
+ * Parquet value writer for passing decimal values
+ * into {@code RecordConsumer} to be stored as FIXED_LEN_BYTE_ARRAY type.
+ */
+public class FixedLenDecimalParquetValueWriter extends DecimalValueWriter {
+
+  @Override
+  public void writeValue(RecordConsumer consumer, DrillBuf buffer, int start, int end, int precision) {
+    int typeLength = DecimalUtility.getMaxBytesSizeForPrecision(precision);
+    int length = end - start;
+    int startPos = typeLength - length;
+    byte[] output = new byte[typeLength];
+    if (startPos >= 0) {
+      buffer.getBytes(start, output, startPos, length);
+      if (output[startPos] < 0) {
+        Arrays.fill(output, 0, output.length - length, (byte) -1);
+      }
+    } else {
+      // in this case value from FIXED_LEN_BYTE_ARRAY or BINARY field with greater length was taken, ignore leading bytes
+      buffer.getBytes(start - startPos, output, 0, length + startPos);
+    }
+    consumer.addBinary(Binary.fromReusedByteArray(output));
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/decimal/Int32DecimalParquetValueWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/decimal/Int32DecimalParquetValueWriter.java
new file mode 100644
index 0000000000..1dc90a09b3
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/decimal/Int32DecimalParquetValueWriter.java
@@ -0,0 +1,51 @@
+/*
+ * 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.exec.store.parquet.decimal;
+
+import com.google.common.primitives.Ints;
+import io.netty.buffer.DrillBuf;
+import org.apache.parquet.io.api.RecordConsumer;
+
+import java.util.Arrays;
+
+/**
+ * Parquet value writer for passing decimal values
+ * into {@code RecordConsumer} to be stored as INT32 type.
+ */
+public class Int32DecimalParquetValueWriter extends DecimalValueWriter {
+
+  @Override
+  public void writeValue(RecordConsumer consumer, DrillBuf buffer, int start, int end, int precision) {
+    byte[] output;
+    int startPos;
+    int length = end - start;
+    startPos = Ints.BYTES - length;
+    output = new byte[Ints.BYTES];
+    if (startPos >= 0) {
+      buffer.getBytes(start, output, startPos, length);
+    } else {
+      // in this case value from FIXED_LEN_BYTE_ARRAY or BINARY field was taken, ignore leading bytes
+      buffer.getBytes(start - startPos, output, 0, length + startPos);
+      startPos = 0;
+    }
+    if (output[startPos] < 0) {
+      Arrays.fill(output, 0, output.length - length, (byte) -1);
+    }
+    consumer.addInteger(Ints.fromByteArray(output));
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/decimal/Int64DecimalParquetValueWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/decimal/Int64DecimalParquetValueWriter.java
new file mode 100644
index 0000000000..f6b83970ec
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/decimal/Int64DecimalParquetValueWriter.java
@@ -0,0 +1,50 @@
+/*
+ * 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.exec.store.parquet.decimal;
+
+import com.google.common.primitives.Longs;
+import io.netty.buffer.DrillBuf;
+import org.apache.parquet.io.api.RecordConsumer;
+
+import java.util.Arrays;
+
+/**
+ * Parquet value writer for passing decimal values
+ * into {@code RecordConsumer} to be stored as INT32 type.
+ */
+public class Int64DecimalParquetValueWriter extends DecimalValueWriter {
+
+  @Override
+  public void writeValue(RecordConsumer consumer, DrillBuf buffer, int start, int end, int precision) {
+    byte[] output;
+    int startPos;
+    int length = end - start;
+    startPos = Longs.BYTES - length;
+    output = new byte[Longs.BYTES];
+    if (startPos > 0) {
+      buffer.getBytes(start, output, startPos, length);
+      if (output[startPos] < 0) {
+        Arrays.fill(output, 0, output.length - length, (byte) -1);
+      }
+    } else {
+      // in this case value from FIXED_LEN_BYTE_ARRAY or BINARY field was taken, ignore leading bytes
+      buffer.getBytes(start - startPos, output, 0, length + startPos);
+    }
+    consumer.addLong(Longs.fromByteArray(output));
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java
index 4363a7a452..59302d067a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java
@@ -19,12 +19,13 @@
 
 import static org.apache.drill.exec.store.parquet.ParquetReaderUtility.NanoTimeUtils.getDateTimeValueFromBinary;
 
-import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.expression.PathSegment;
 import org.apache.drill.common.expression.SchemaPath;
@@ -32,10 +33,6 @@
 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.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;
@@ -44,20 +41,16 @@
 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 org.apache.drill.exec.expr.holders.VarDecimalHolder;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.store.parquet.ParquetReaderUtility;
 import org.apache.drill.exec.store.parquet.columnreaders.ParquetColumnMetadata;
-import org.apache.drill.exec.util.DecimalUtility;
 import org.apache.drill.exec.vector.complex.impl.ComplexWriterImpl;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
 import org.apache.drill.exec.vector.complex.writer.BigIntWriter;
 import org.apache.drill.exec.vector.complex.writer.BitWriter;
 import org.apache.drill.exec.vector.complex.writer.DateWriter;
-import org.apache.drill.exec.vector.complex.writer.Decimal18Writer;
-import org.apache.drill.exec.vector.complex.writer.Decimal28SparseWriter;
-import org.apache.drill.exec.vector.complex.writer.Decimal38SparseWriter;
-import org.apache.drill.exec.vector.complex.writer.Decimal9Writer;
 import org.apache.drill.exec.vector.complex.writer.Float4Writer;
 import org.apache.drill.exec.vector.complex.writer.Float8Writer;
 import org.apache.drill.exec.vector.complex.writer.IntWriter;
@@ -66,6 +59,7 @@
 import org.apache.drill.exec.vector.complex.writer.TimeWriter;
 import org.apache.drill.exec.vector.complex.writer.VarBinaryWriter;
 import org.apache.drill.exec.vector.complex.writer.VarCharWriter;
+import org.apache.drill.exec.vector.complex.writer.VarDecimalWriter;
 import org.apache.parquet.io.api.Binary;
 import org.apache.parquet.io.api.Converter;
 import org.apache.parquet.io.api.GroupConverter;
@@ -73,7 +67,6 @@
 import org.apache.parquet.schema.DecimalMetadata;
 import org.apache.parquet.schema.GroupType;
 import org.apache.parquet.schema.MessageType;
-import org.apache.parquet.schema.OriginalType;
 import org.apache.parquet.schema.PrimitiveType;
 import org.apache.parquet.schema.Type;
 import org.apache.parquet.schema.Type.Repetition;
@@ -189,10 +182,11 @@ private PrimitiveConverter getConverterForType(String name, PrimitiveType type)
           }
           case DECIMAL: {
             ParquetReaderUtility.checkDecimalTypeEnabled(options);
-            Decimal9Writer writer = type.getRepetition() == Repetition.REPEATED
-                ? mapWriter.list(name).decimal9()
-                : mapWriter.decimal9(name, type.getDecimalMetadata().getScale(), type.getDecimalMetadata().getPrecision());
-            return new DrillDecimal9Converter(writer, type.getDecimalMetadata().getPrecision(), type.getDecimalMetadata().getScale());
+            VarDecimalWriter writer = type.getRepetition() == Repetition.REPEATED
+                ? mapWriter.list(name).varDecimal(type.getDecimalMetadata().getScale(), type.getDecimalMetadata().getPrecision())
+                : mapWriter.varDecimal(name, type.getDecimalMetadata().getScale(), type.getDecimalMetadata().getPrecision());
+            return new DrillVarDecimalConverter(writer, type.getDecimalMetadata().getPrecision(),
+                type.getDecimalMetadata().getScale(), mutator.getManagedBuffer());
           }
           case DATE: {
             DateWriter writer = type.getRepetition() == Repetition.REPEATED ? mapWriter.list(name).date() : mapWriter.date(name);
@@ -232,10 +226,11 @@ private PrimitiveConverter getConverterForType(String name, PrimitiveType type)
           }
           case DECIMAL: {
             ParquetReaderUtility.checkDecimalTypeEnabled(options);
-            Decimal18Writer writer = type.getRepetition() == Repetition.REPEATED
-                ? mapWriter.list(name).decimal18()
-                : mapWriter.decimal18(name, type.getDecimalMetadata().getScale(), type.getDecimalMetadata().getPrecision());
-            return new DrillDecimal18Converter(writer, type.getDecimalMetadata().getPrecision(), type.getDecimalMetadata().getScale());
+            VarDecimalWriter writer = type.getRepetition() == Repetition.REPEATED
+                ? mapWriter.list(name).varDecimal(type.getDecimalMetadata().getScale(), type.getDecimalMetadata().getPrecision())
+                : mapWriter.varDecimal(name, type.getDecimalMetadata().getScale(), type.getDecimalMetadata().getPrecision());
+            return new DrillVarDecimalConverter(writer, type.getDecimalMetadata().getPrecision(),
+                type.getDecimalMetadata().getScale(), mutator.getManagedBuffer());
           }
           case TIMESTAMP_MILLIS: {
             TimeStampWriter writer = type.getRepetition() == Repetition.REPEATED ? mapWriter.list(name).timeStamp() : mapWriter.timeStamp(name);
@@ -285,21 +280,15 @@ private PrimitiveConverter getConverterForType(String name, PrimitiveType type)
             VarCharWriter writer = type.getRepetition() == Repetition.REPEATED ? mapWriter.list(name).varChar() : mapWriter.varChar(name);
             return new DrillVarCharConverter(writer, mutator.getManagedBuffer());
           }
-          //TODO not sure if BINARY/DECIMAL is actually supported
+          // See DRILL-4184 and DRILL-4834. Support for this is added using new VarDecimal type.
           case DECIMAL: {
             ParquetReaderUtility.checkDecimalTypeEnabled(options);
             DecimalMetadata metadata = type.getDecimalMetadata();
-            if (metadata.getPrecision() <= 28) {
-              Decimal28SparseWriter writer = type.getRepetition() == Repetition.REPEATED
-                  ? mapWriter.list(name).decimal28Sparse()
-                  : mapWriter.decimal28Sparse(name, metadata.getScale(), metadata.getPrecision());
-              return new DrillBinaryToDecimal28Converter(writer, metadata.getPrecision(), metadata.getScale(), mutator.getManagedBuffer());
-            } else {
-              Decimal38SparseWriter writer = type.getRepetition() == Repetition.REPEATED
-                  ? mapWriter.list(name).decimal38Sparse()
-                  : mapWriter.decimal38Sparse(name, metadata.getScale(), metadata.getPrecision());
-              return new DrillBinaryToDecimal38Converter(writer, metadata.getPrecision(), metadata.getScale(), mutator.getManagedBuffer());
-            }
+            VarDecimalWriter writer =
+                type.getRepetition() == Repetition.REPEATED
+                    ? mapWriter.list(name).varDecimal(metadata.getScale(), metadata.getPrecision())
+                    : mapWriter.varDecimal(name, metadata.getScale(), metadata.getPrecision());
+            return new DrillVarDecimalConverter(writer, metadata.getPrecision(), metadata.getScale(), mutator.getManagedBuffer());
           }
           default: {
             throw new UnsupportedOperationException("Unsupported type " + type.getOriginalType());
@@ -307,28 +296,28 @@ private PrimitiveConverter getConverterForType(String name, PrimitiveType type)
         }
       }
       case FIXED_LEN_BYTE_ARRAY:
-        if (type.getOriginalType() == OriginalType.DECIMAL) {
-          ParquetReaderUtility.checkDecimalTypeEnabled(options);
-          DecimalMetadata metadata = type.getDecimalMetadata();
-          if (metadata.getPrecision() <= 28) {
-            Decimal28SparseWriter writer = type.getRepetition() == Repetition.REPEATED
-                ? mapWriter.list(name).decimal28Sparse()
-                : mapWriter.decimal28Sparse(name, metadata.getScale(), metadata.getPrecision());
-            return new DrillBinaryToDecimal28Converter(writer, metadata.getPrecision(), metadata.getScale(), mutator.getManagedBuffer());
-          } else {
-            Decimal38SparseWriter writer = type.getRepetition() == Repetition.REPEATED
-                ? mapWriter.list(name).decimal38Sparse()
-                : mapWriter.decimal38Sparse(name, metadata.getScale(), metadata.getPrecision());
-            return new DrillBinaryToDecimal38Converter(writer, metadata.getPrecision(), metadata.getScale(), mutator.getManagedBuffer());
+        switch (type.getOriginalType()) {
+          case DECIMAL: {
+            ParquetReaderUtility.checkDecimalTypeEnabled(options);
+            DecimalMetadata metadata = type.getDecimalMetadata();
+            VarDecimalWriter writer = type.getRepetition() == Repetition.REPEATED
+                ? mapWriter.list(name).varDecimal(metadata.getScale(), metadata.getPrecision())
+                : mapWriter.varDecimal(name, metadata.getScale(), metadata.getPrecision());
+            return new DrillVarDecimalConverter(writer, metadata.getPrecision(), metadata.getScale(), mutator.getManagedBuffer());
           }
-        } else if (type.getOriginalType() == OriginalType.INTERVAL) {
-          IntervalWriter writer = type.getRepetition() == Repetition.REPEATED ? mapWriter.list(name).interval()
-              : mapWriter.interval(name);
-          return new DrillFixedLengthByteArrayToInterval(writer);
+          case INTERVAL: {
+            IntervalWriter writer = type.getRepetition() == Repetition.REPEATED
+                ? mapWriter.list(name).interval()
+                : mapWriter.interval(name);
+            return new DrillFixedLengthByteArrayToInterval(writer);
 
-        } else {
-          VarBinaryWriter writer = type.getRepetition() == Repetition.REPEATED ? mapWriter.list(name).varBinary() : mapWriter.varBinary(name);
-          return new DrillFixedBinaryToVarbinaryConverter(writer, type.getTypeLength(), mutator.getManagedBuffer());
+          }
+          default: {
+            VarBinaryWriter writer = type.getRepetition() == Repetition.REPEATED
+                ? mapWriter.list(name).varBinary()
+                : mapWriter.varBinary(name);
+            return new DrillFixedBinaryToVarbinaryConverter(writer, type.getTypeLength(), mutator.getManagedBuffer());
+          }
         }
       default:
         throw new UnsupportedOperationException("Unsupported type: " + type.getPrimitiveTypeName());
@@ -366,25 +355,6 @@ public void addInt(int value) {
     }
   }
 
-  public static class DrillDecimal9Converter extends PrimitiveConverter {
-    private Decimal9Writer writer;
-    private Decimal9Holder holder = new Decimal9Holder();
-    int precision;
-    int scale;
-
-    public DrillDecimal9Converter(Decimal9Writer writer, int precision, int scale) {
-      this.writer = writer;
-      this.scale = scale;
-      this.precision = precision;
-    }
-
-    @Override
-    public void addInt(int value) {
-      holder.value = value;
-      writer.write(holder);
-    }
-  }
-
   public static class CorruptionDetectingDateConverter extends PrimitiveConverter {
     private DateWriter writer;
     private DateHolder holder = new DateHolder();
@@ -479,23 +449,6 @@ public void addLong(long value) {
     }
   }
 
-  public static class DrillDecimal18Converter extends PrimitiveConverter {
-    private Decimal18Writer writer;
-    private Decimal18Holder holder = new Decimal18Holder();
-
-    public DrillDecimal18Converter(Decimal18Writer writer, int precision, int scale) {
-      this.writer = writer;
-      holder.precision = precision;
-      holder.scale = scale;
-    }
-
-    @Override
-    public void addLong(long value) {
-      holder.value = value;
-      writer.write(holder);
-    }
-  }
-
   public static class DrillFloat4Converter extends PrimitiveConverter {
     private Float4Writer writer;
     private Float4Holder holder = new Float4Holder();
@@ -581,44 +534,44 @@ public void addBinary(Binary value) {
     }
   }
 
-  public static class DrillBinaryToDecimal28Converter extends PrimitiveConverter {
-    private Decimal28SparseWriter writer;
-    private Decimal28SparseHolder holder = new Decimal28SparseHolder();
+  public static class DrillVarDecimalConverter extends PrimitiveConverter {
+    private VarDecimalWriter writer;
+    private VarDecimalHolder holder = new VarDecimalHolder();
     private DrillBuf buf;
 
-    public DrillBinaryToDecimal28Converter(Decimal28SparseWriter writer, int precision, int scale,  DrillBuf buf) {
+    public DrillVarDecimalConverter(VarDecimalWriter writer, int precision, int scale, DrillBuf buf) {
       this.writer = writer;
-      this.buf = buf.reallocIfNeeded(28);
-      holder.precision = precision;
       holder.scale = scale;
+      holder.precision = precision;
+      this.buf = buf;
     }
 
     @Override
     public void addBinary(Binary value) {
-      BigDecimal bigDecimal = DecimalUtility.getBigDecimalFromByteArray(value.getBytes(), 0, value.length(), holder.scale);
-      DecimalUtility.getSparseFromBigDecimal(bigDecimal, buf, 0, holder.scale, holder.precision, Decimal28SparseHolder.nDecimalDigits);
-      holder.buffer = buf;
+      holder.buffer = buf.reallocIfNeeded(value.length());
+      holder.buffer.setBytes(0, value.toByteBuffer());
+      holder.start = 0;
+      holder.end = value.length();
       writer.write(holder);
     }
-  }
-
-  public static class DrillBinaryToDecimal38Converter extends PrimitiveConverter {
-    private Decimal38SparseWriter writer;
-    private Decimal38SparseHolder holder = new Decimal38SparseHolder();
-    private DrillBuf buf;
 
-    public DrillBinaryToDecimal38Converter(Decimal38SparseWriter writer, int precision, int scale,  DrillBuf buf) {
-      this.writer = writer;
-      this.buf = buf.reallocIfNeeded(38);
-      holder.precision = precision;
-      holder.scale = scale;
+    @Override
+    public void addInt(int value) {
+      byte[] bytes = Ints.toByteArray(value);
+      holder.buffer = buf.reallocIfNeeded(bytes.length);
+      holder.buffer.setBytes(0, bytes);
+      holder.start = 0;
+      holder.end = bytes.length;
+      writer.write(holder);
     }
 
     @Override
-    public void addBinary(Binary value) {
-       BigDecimal bigDecimal = DecimalUtility.getBigDecimalFromByteArray(value.getBytes(), 0, value.length(), holder.scale);
-      DecimalUtility.getSparseFromBigDecimal(bigDecimal, buf, 0, holder.scale, holder.precision, Decimal38SparseHolder.nDecimalDigits);
-      holder.buffer = buf;
+    public void addLong(long value) {
+      byte[] bytes = Longs.toByteArray(value);
+      holder.buffer = buf.reallocIfNeeded(bytes.length);
+      holder.buffer.setBytes(0, bytes);
+      holder.start = 0;
+      holder.end = bytes.length;
       writer.write(holder);
     }
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoDataType.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoDataType.java
index cb68efcb9a..7d8a8e4920 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoDataType.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoDataType.java
@@ -19,11 +19,10 @@
 
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
+import java.math.BigDecimal;
 import java.sql.Timestamp;
 import java.util.List;
 
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.type.SqlTypeName;
 
 import com.google.common.collect.Lists;
@@ -57,6 +56,8 @@ public PojoDataType(Class<?> pojoClass) {
         types.add(SqlTypeName.BIGINT);
       } else if(type == double.class || type == Double.class) {
         types.add(SqlTypeName.DOUBLE);
+      } else if(type == BigDecimal.class) {
+        types.add(SqlTypeName.DECIMAL);
       } else if(type == String.class) {
         types.add(SqlTypeName.VARCHAR);
       } else if(type.isEnum()) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriters.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriters.java
index 090f32f6f8..19c2365851 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriters.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriters.java
@@ -19,6 +19,7 @@
 
 import io.netty.buffer.DrillBuf;
 
+import java.math.BigDecimal;
 import java.sql.Timestamp;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -37,6 +38,7 @@
 import org.apache.drill.exec.vector.NullableVarCharVector;
 
 import com.google.common.base.Charsets;
+import org.apache.drill.exec.vector.VarDecimalVector;
 
 public class PojoWriters {
 
@@ -63,6 +65,8 @@ public static PojoWriter getWriter(Class<?> type, String fieldName, DrillBuf buf
       return new EnumWriter(fieldName, buffer);
     } else if (type == String.class) {
       return new StringWriter(fieldName, buffer);
+    } else if (type == BigDecimal.class) {
+      return new DecimalWriter(fieldName);
     } else if (type == Timestamp.class) {
       return new NTimeStampWriter(fieldName);
       // primitives
@@ -142,6 +146,24 @@ public void writeField(Object value, int outboundIndex) {
 
   }
 
+  /**
+   * Pojo writer for decimal. If null is encountered does not write it.
+   */
+  public static class DecimalWriter extends AbstractPojoWriter<VarDecimalVector> {
+
+    public DecimalWriter(String fieldName) {
+      super(fieldName, Types.optional(MinorType.VARDECIMAL));
+    }
+
+    @Override
+    public void writeField(Object value, int outboundIndex) {
+      if (value != null) {
+        vector.getMutator().setSafe(outboundIndex, (BigDecimal) value);
+      }
+    }
+
+  }
+
   /**
    * Parent class for String and Enum writers. Writes data using nullable varchar holder.
    */
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapUtility.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapUtility.java
index 72c094a83e..543a6db98c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapUtility.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapUtility.java
@@ -130,6 +130,16 @@ public static void writeToMapFromReader(FieldReader fieldReader, BaseWriter.MapW
             fieldReader.copyAsValue(mapWriter.decimal38Sparse(MappifyUtility.fieldValue));
           }
           break;
+        case VARDECIMAL:
+          if (repeated) {
+            fieldReader.copyAsValue(
+                mapWriter.list(MappifyUtility.fieldValue)
+                    .varDecimal(valueMajorType.getScale(), valueMajorType.getPrecision()));
+          } else {
+            fieldReader.copyAsValue(
+                mapWriter.varDecimal(MappifyUtility.fieldValue, valueMajorType.getScale(), valueMajorType.getPrecision()));
+          }
+          break;
         case DATE:
           if (repeated) {
             fieldReader.copyAsValue(mapWriter.list(MappifyUtility.fieldValue).date());
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonWriter.java
index 7377184a8b..4d844023da 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/JsonWriter.java
@@ -110,6 +110,7 @@ private void writeValue(FieldReader reader) throws JsonGenerationException, IOEx
       case DECIMAL38SPARSE:
       case DECIMAL9:
       case DECIMAL18:
+      case VARDECIMAL:
         gen.writeDecimal(reader);
         break;
 
@@ -220,6 +221,7 @@ private void writeValue(FieldReader reader) throws JsonGenerationException, IOEx
       case DECIMAL38SPARSE:
       case DECIMAL9:
       case DECIMAL18:
+      case VARDECIMAL:
         for(int i = 0; i < reader.size(); i++){
           gen.writeDecimal(i, reader);
         }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/ExecErrorConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/ExecErrorConstants.java
index 9c7b9b90a3..f1ebffbb2f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/ExecErrorConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/ExecErrorConstants.java
@@ -22,7 +22,6 @@
 public interface ExecErrorConstants {
 
   // Error message when decimal data type is disabled
-  public static final String DECIMAL_DISABLE_ERR_MSG = String.format("Decimal data type is disabled. \n" +
-      "As of this release decimal data type is a beta level feature and should not be used in production \n" +
+  String DECIMAL_DISABLE_ERR_MSG = String.format("Decimal data type is disabled. \n" +
       "Use option '%s' to enable decimal data type", PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/prepare/PreparedStatementProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/prepare/PreparedStatementProvider.java
index b748cdb597..4c757ef210 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/prepare/PreparedStatementProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/prepare/PreparedStatementProvider.java
@@ -91,6 +91,7 @@
       .put(MinorType.DECIMAL18, BigDecimal.class.getName())
       .put(MinorType.DECIMAL28SPARSE, BigDecimal.class.getName())
       .put(MinorType.DECIMAL38SPARSE, BigDecimal.class.getName())
+      .put(MinorType.VARDECIMAL, BigDecimal.class.getName())
       .put(MinorType.TIME, Time.class.getName())
       .put(MinorType.TIMESTAMP, Timestamp.class.getName())
       .put(MinorType.VARBINARY, byte[].class.getName())
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index 60fd63928a..8874f92042 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -466,7 +466,7 @@ drill.exec.options: {
     planner.disable_exchanges: false,
     planner.enable_broadcast_join: true,
     planner.enable_constant_folding: true,
-    planner.enable_decimal_data_type: false,
+    planner.enable_decimal_data_type: true,
     planner.enable_demux_exchange: false,
     planner.enable_hash_single_key: true,
     planner.enable_hashagg: true,
@@ -548,6 +548,8 @@ drill.exec.options: {
     store.parquet.use_new_reader: false,
     store.parquet.vector_fill_check_threshold: 10,
     store.parquet.vector_fill_threshold: 85,
+    store.parquet.writer.use_primitive_types_for_decimals: true,
+    store.parquet.writer.logical_type_for_decimals: "fixed_len_byte_array",
     store.parquet.writer.use_single_fs_block: false,
     store.partition.hash_distribute: false,
     store.text.estimated_row_size_bytes: 100.0,
diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
index 0dc62c637d..18fc4e1375 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
@@ -20,12 +20,17 @@
 import java.io.IOException;
 import java.net.URL;
 
+import com.google.common.base.Function;
+import io.netty.buffer.DrillBuf;
 import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.config.LogicalPlanPersistence;
 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.exec.expr.holders.ValueHolder;
+import org.apache.drill.exec.vector.ValueHolderHelper;
 import org.apache.drill.test.TestTools;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
@@ -54,7 +59,9 @@
 import com.google.common.base.Charsets;
 import com.google.common.collect.ImmutableList;
 import com.google.common.io.Resources;
+import org.mockito.Matchers;
 
+import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -116,6 +123,18 @@ protected void testSqlPlan(String sqlCommands) throws Exception {
     when(context.getAllocator()).thenReturn(allocator);
     when(context.getExecutionControls()).thenReturn(executionControls);
     when(context.getLpPersistence()).thenReturn(logicalPlanPersistence);
+    // mocks for org.apache.drill.TestTpchPlanning#tpch06 test.
+    // With changes for decimal types, subtract udf for decimals is used.
+    when(context.getManagedBuffer()).thenReturn(allocator.buffer(4));
+    when(context.getConstantValueHolder(eq("0.03"),
+        eq(TypeProtos.MinorType.VARDECIMAL),
+        Matchers.<Function<DrillBuf, ValueHolder>>any()))
+      .thenReturn(ValueHolderHelper.getVarDecimalHolder(allocator.buffer(4), "0.03"));
+    when(context.getConstantValueHolder(eq("0.01"),
+        eq(TypeProtos.MinorType.VARDECIMAL),
+        Matchers.<Function<DrillBuf, ValueHolder>>any()))
+      .thenReturn(ValueHolderHelper.getVarDecimalHolder(allocator.buffer(4), "0.01"));
+
 
     for (final String sql : sqlStrings) {
       if (sql.trim().isEmpty()) {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestDisabledFunctionality.java b/exec/java-exec/src/test/java/org/apache/drill/TestDisabledFunctionality.java
index bfbc174107..781fce31bf 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestDisabledFunctionality.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestDisabledFunctionality.java
@@ -18,6 +18,7 @@
 package org.apache.drill;
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.work.ExecErrorConstants;
 import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
 import org.apache.drill.exec.work.foreman.UnsupportedDataTypeException;
@@ -307,14 +308,24 @@ public void testFlattenWithinDistinct() throws Exception {
 
   @Test // DRILL-2848
   public void testDisableDecimalCasts() throws Exception {
-    final String query = "select cast('1.2' as decimal(9, 2)) from cp.`employee.json` limit 1";
-    errorMsgTestHelper(query, ExecErrorConstants.DECIMAL_DISABLE_ERR_MSG);
+    try {
+      alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, false);
+      final String query = "select cast('1.2' as decimal(9, 2)) from cp.`employee.json` limit 1";
+      errorMsgTestHelper(query, ExecErrorConstants.DECIMAL_DISABLE_ERR_MSG);
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    }
   }
 
   @Test // DRILL-2848
   public void testDisableDecimalFromParquet() throws Exception {
-    final String query = "select * from cp.`parquet/decimal_dictionary.parquet`";
-    errorMsgTestHelper(query, ExecErrorConstants.DECIMAL_DISABLE_ERR_MSG);
+    try {
+      alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, false);
+      final String query = "select * from cp.`parquet/decimal_dictionary.parquet`";
+      errorMsgTestHelper(query, ExecErrorConstants.DECIMAL_DISABLE_ERR_MSG);
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    }
   }
 
   @Test (expected = UnsupportedFunctionException.class) //DRILL-3802
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestFrameworkTest.java b/exec/java-exec/src/test/java/org/apache/drill/TestFrameworkTest.java
index 419e639e45..5c36ca8db9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestFrameworkTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestFrameworkTest.java
@@ -144,7 +144,7 @@ public void testDecimalBaseline() throws  Exception {
           .sqlQuery("select cast(dec_col as decimal(38,2)) dec_col from cp.`testframework/decimal_test.json`")
           .unOrdered()
           .csvBaselineFile("testframework/decimal_test.tsv")
-          .baselineTypes(Types.withScaleAndPrecision(TypeProtos.MinorType.DECIMAL38SPARSE, TypeProtos.DataMode.REQUIRED, 2, 38))
+          .baselineTypes(Types.withScaleAndPrecision(TypeProtos.MinorType.VARDECIMAL, TypeProtos.DataMode.REQUIRED, 2, 38))
           .baselineColumns("dec_col")
           .build().run();
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java b/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
index 5ae0c380dc..9986579c1e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestFunctionsQuery.java
@@ -42,8 +42,8 @@ public static void enableDecimalDataType() throws Exception {
   }
 
   @AfterClass
-  public static void disableDecimalDataType() throws Exception {
-    test(String.format("alter session set `%s` = false", PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY));
+  public static void disableDecimalDataType() {
+    resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
   }
 
   @Test
@@ -402,7 +402,7 @@ public void testDecimalMultiplicationOverflowHandling() throws Exception {
         .sqlQuery(query)
         .unOrdered()
         .baselineColumns("DEC38_1", "DEC38_2", "DEC38_3")
-        .baselineValues(new BigDecimal("1000000000000000000000000000.00000"), new BigDecimal("1000000000000000000"), new BigDecimal("1000000000.000000000000000000"))
+        .baselineValues(new BigDecimal("1000000000000000000000000000.00000"), new BigDecimal("1.0000000000000000E+18"), new BigDecimal("1000000000.000000000000000000"))
         .go();
   }
 
@@ -429,7 +429,7 @@ public void testDecimalDownwardCast() throws Exception {
     String query = "select cast((cast('12345.6789' as decimal(18, 4))) as decimal(9, 4)) as DEC18_DEC9_1, " +
         "cast((cast('12345.6789' as decimal(18, 4))) as decimal(9, 2)) as DEC18_DEC9_2, " +
         "cast((cast('-12345.6789' as decimal(18, 4))) as decimal(9, 0)) as DEC18_DEC9_3, " +
-        "cast((cast('999999999.6789' as decimal(38, 4))) as decimal(9, 0)) as DEC38_DEC19_1, " +
+        "cast((cast('99999999.6789' as decimal(38, 4))) as decimal(9, 0)) as DEC38_DEC19_1, " +
         "cast((cast('-999999999999999.6789' as decimal(38, 4))) as decimal(18, 2)) as DEC38_DEC18_1, " +
         "cast((cast('-999999999999999.6789' as decimal(38, 4))) as decimal(18, 0)) as DEC38_DEC18_2, " +
         "cast((cast('100000000999999999.6789' as decimal(38, 4))) as decimal(28, 0)) as DEC38_DEC28_1 " +
@@ -438,24 +438,25 @@ public void testDecimalDownwardCast() throws Exception {
         .sqlQuery(query)
         .unOrdered()
         .baselineColumns("DEC18_DEC9_1", "DEC18_DEC9_2", "DEC18_DEC9_3", "DEC38_DEC19_1", "DEC38_DEC18_1", "DEC38_DEC18_2", "DEC38_DEC28_1")
-        .baselineValues(new BigDecimal("12345.6789"), new BigDecimal("12345.68"), new BigDecimal("-12346"), new BigDecimal("1000000000"),
+        .baselineValues(new BigDecimal("12345.6789"), new BigDecimal("12345.68"), new BigDecimal("-12346"), new BigDecimal("100000000"),
             new BigDecimal("-999999999999999.68"), new BigDecimal("-1000000000000000"), new BigDecimal("100000001000000000"))
         .go();
   }
 
   @Test
   public void testTruncateWithParamFunction() throws Exception {
-    String query = "SELECT " +
-        "trunc(1234.4567, 2) as T_1, " +
-        "trunc(-1234.4567, 2) as T_2, " +
-        "trunc(1234.4567, -2) as T_3, " +
-        "trunc(-1234.4567, -2) as T_4, " +
-        "trunc(1234, 4) as T_5, " +
-        "trunc(-1234, 4) as T_6, " +
-        "trunc(1234, -4) as T_7, " +
-        "trunc(-1234, -4) as T_8, " +
-        "trunc(8124674407369523212, 0) as T_9, " +
-        "trunc(81246744073695.395, 1) as T_10 " +
+    String query =
+        "SELECT\n" +
+            "trunc(cast('1234.4567' as double), 2) as T_1,\n" +
+            "trunc(cast('-1234.4567' as double), 2) as T_2,\n" +
+            "trunc(cast('1234.4567' as double), -2) as T_3,\n" +
+            "trunc(cast('-1234.4567' as double), -2) as T_4,\n" +
+            "trunc(cast('1234' as double), 4) as T_5,\n" +
+            "trunc(cast('-1234' as double), 4) as T_6,\n" +
+            "trunc(cast('1234' as double), -4) as T_7,\n" +
+            "trunc(cast('-1234' as double), -4) as T_8,\n" +
+            "trunc(cast('8124674407369523212' as double), 0) as T_9,\n" +
+            "trunc(cast('81246744073695.395' as double), 1) as T_10\n" +
         "FROM cp.`tpch/region.parquet` limit 1";
 
     testBuilder()
@@ -469,17 +470,18 @@ public void testTruncateWithParamFunction() throws Exception {
 
   @Test
   public void testRoundWithParamFunction() throws Exception {
-    String query = "SELECT " +
-        "round(1234.4567, 2) as T_1, " +
-        "round(-1234.4567, 2) as T_2, " +
-        "round(1234.4567, -2) as T_3, " +
-        "round(-1234.4567, -2) as T_4, " +
-        "round(1234, 4) as T_5, " +
-        "round(-1234, 4) as T_6, " +
-        "round(1234, -4) as T_7, " +
-        "round(-1234, -4) as T_8, " +
-        "round(8124674407369523212, -4) as T_9, " +
-        "round(81246744073695.395, 1) as T_10 " +
+    String query =
+        "SELECT\n" +
+            "round(cast('1234.4567' as double), 2) as T_1,\n" +
+            "round(cast('-1234.4567' as double), 2) as T_2,\n" +
+            "round(cast('1234.4567' as double), -2) as T_3,\n" +
+            "round(cast('-1234.4567' as double), -2) as T_4,\n" +
+            "round(cast('1234' as double), 4) as T_5,\n" +
+            "round(cast('-1234' as double), 4) as T_6,\n" +
+            "round(cast('1234' as double), -4) as T_7,\n" +
+            "round(cast('-1234' as double), -4) as T_8,\n" +
+            "round(cast('8124674407369523212' as double), -4) as T_9,\n" +
+            "round(cast('81246744073695.395' as double), 1) as T_10\n" +
         "FROM cp.`tpch/region.parquet` limit 1";
 
     testBuilder()
@@ -494,19 +496,20 @@ public void testRoundWithParamFunction() throws Exception {
 
   @Test
   public void testRoundWithOneParam() throws Exception {
-    String query = "select " +
-        "round(8124674407369523212) round_bigint," +
-        "round(9999999) round_int, " +
-        "round(cast('23.45' as float)) round_float_1, " +
-        "round(cast('23.55' as float)) round_float_2, " +
-        "round(8124674407369.2345) round_double_1, " +
-        "round(8124674407369.589) round_double_2 " +
-        " from cp.`tpch/region.parquet` limit 1";
+    String query =
+        "select\n" +
+            "round(8124674407369523212) round_bigint,\n" +
+            "round(9999999) round_int,\n" +
+            "round(cast('23.45' as float)) round_float_1,\n" +
+            "round(cast('23.55' as float)) round_float_2,\n" +
+            "round(cast('8124674407369.2345' as double)) round_double_1,\n" +
+            "round(cast('8124674407369.589' as double)) round_double_2\n" +
+        "from cp.`tpch/region.parquet` limit 1";
     testBuilder()
         .sqlQuery(query)
         .unOrdered()
         .baselineColumns("round_bigint", "round_int", "round_float_1", "round_float_2", "round_double_1", "round_double_2")
-        .baselineValues(8124674407369523212l, 9999999, 23.0f, 24.0f, 8124674407369.0d, 8124674407370.0d)
+        .baselineValues(8124674407369523212L, 9999999, 23.0f, 24.0f, 8124674407369.0d, 8124674407370.0d)
         .go();
   }
 
@@ -729,14 +732,14 @@ public void testOptiqDecimalCapping() throws Exception {
 
   @Test
   public void testNegative() throws Exception {
-    String query = "select  negative(cast(2 as bigint)) as NEG " +
+    String query = "select  negative(cast(2 as bigint)) as NEG\n" +
         "from cp.`employee.json` where employee_id = 1";
 
     testBuilder()
         .sqlQuery(query)
         .unOrdered()
         .baselineColumns("NEG")
-        .baselineValues(-2l)
+        .baselineValues(-2L)
         .go();
   }
 
@@ -755,11 +758,11 @@ public void testOptiqValidationFunctions() throws Exception {
 
   @Test
   public void testToTimeStamp() throws Exception {
-    String query = "select to_timestamp(cast('800120400.12312' as decimal(38, 5))) as DEC38_TS, to_timestamp(200120400) as INT_TS " +
+    String query = "select to_timestamp(cast('800120400.12312' as decimal(38, 5))) as DEC38_TS, to_timestamp(200120400) as INT_TS\n" +
         "from cp.`employee.json` where employee_id < 2";
 
-    DateTime result1 = new DateTime(800120400123l);
-    DateTime result2 = new DateTime(200120400000l);
+    DateTime result1 = new DateTime(800120400123L);
+    DateTime result2 = new DateTime(200120400000L);
 
     testBuilder()
         .sqlQuery(query)
@@ -883,7 +886,7 @@ public void testImplicitCastVarcharToDouble() throws Exception {
         .sqlQuery("select `integer` i, `float` f from cp.`jsoninput/input1.json` where `float` = '1.2'")
         .unOrdered()
         .baselineColumns("i", "f")
-        .baselineValues(2001l, 1.2d)
+        .baselineValues(2001L, 1.2d)
         .go();
   }
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestEmptyInputSql.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestEmptyInputSql.java
index e79d75de4d..84da7a6e88 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/TestEmptyInputSql.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestEmptyInputSql.java
@@ -19,6 +19,7 @@
 
 import com.google.common.collect.Lists;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.test.rowSet.schema.SchemaBuilder;
 import org.apache.drill.categories.UnlikelyTest;
@@ -128,26 +129,53 @@ public void testQueryMapArrayEmptyJson() throws Exception {
 
   /**
    * Test with query against an empty file. Select clause has three expressions.
-   * 1.0 + 100.0 as constant expression, is resolved to required FLOAT8
+   * 1.0 + 100.0 as constant expression, is resolved to required FLOAT8/VARDECIMAL
    * cast(100 as varchar(100) is resolved to required varchar(100)
    * cast(columns as varchar(100)) is resolved to nullable varchar(100).
    */
   @Test
   public void testQueryConstExprEmptyJson() throws Exception {
-    final BatchSchema expectedSchema = new SchemaBuilder()
-        .add("key", TypeProtos.MinorType.FLOAT8)
-        .add("name", TypeProtos.MinorType.VARCHAR, 100)
-        .addNullable("name2", TypeProtos.MinorType.VARCHAR, 100)
-        .build();
-
-    testBuilder()
-        .sqlQuery("select 1.0 + 100.0 as key, "
-          + " cast(100 as varchar(100)) as name, "
-          + " cast(columns as varchar(100)) as name2 "
-          + " from cp.`%s` ", SINGLE_EMPTY_JSON)
-        .schemaBaseLine(expectedSchema)
-        .build()
-        .run();
+    try {
+      alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, false);
+      BatchSchema expectedSchema = new SchemaBuilder()
+          .add("key", TypeProtos.MinorType.FLOAT8)
+          .add("name", TypeProtos.MinorType.VARCHAR, 100)
+          .addNullable("name2", TypeProtos.MinorType.VARCHAR, 100)
+          .build();
+
+      testBuilder()
+          .sqlQuery("select 1.0 + 100.0 as key, "
+            + " cast(100 as varchar(100)) as name, "
+            + " cast(columns as varchar(100)) as name2 "
+            + " from cp.`%s` ", SINGLE_EMPTY_JSON)
+          .schemaBaseLine(expectedSchema)
+          .build()
+          .run();
+
+      alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+      expectedSchema = new SchemaBuilder()
+          .add("key",
+              TypeProtos.MajorType.newBuilder()
+                  .setMinorType(TypeProtos.MinorType.VARDECIMAL)
+                  .setMode(TypeProtos.DataMode.REQUIRED)
+                  .setPrecision(5)
+                  .setScale(1)
+                  .build())
+          .add("name", TypeProtos.MinorType.VARCHAR, 100)
+          .addNullable("name2", TypeProtos.MinorType.VARCHAR, 100)
+          .build();
+
+      testBuilder()
+          .sqlQuery("select 1.0 + 100.0 as key, "
+            + " cast(100 as varchar(100)) as name, "
+            + " cast(columns as varchar(100)) as name2 "
+            + " from cp.`%s` ", SINGLE_EMPTY_JSON)
+          .schemaBaseLine(expectedSchema)
+          .build()
+          .run();
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    }
   }
 
   /**
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
index e2aec04ac7..fc2f639579 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
@@ -575,13 +575,15 @@ public void testWindowSumAggIsNotNull() throws Exception {
       .sqlQuery(query)
       .ordered()
       .baselineColumns("cnt")
-      .baselineValues(26l)
+      .baselineValues(26L)
       .build().run();
   }
 
   @Test // DRILL-3292
   public void testWindowConstants() throws Exception {
-    String query = "select rank() over w fn, sum(2) over w sumINTEGER, sum(employee_id) over w sumEmpId, sum(0.5) over w sumFLOAT \n" +
+    String query =
+        "select rank() over w fn, sum(2) over w sumINTEGER, sum(employee_id) over w sumEmpId,\n" +
+            "sum(cast(0.5 as double)) over w sumFLOAT \n" +
         "from cp.`employee.json` \n" +
         "where position_id = 2 \n" +
         "window w as(partition by position_id order by employee_id)";
@@ -596,12 +598,12 @@ public void testWindowConstants() throws Exception {
         .sqlQuery(query)
         .ordered()
         .baselineColumns("fn", "sumINTEGER", "sumEmpId", "sumFLOAT")
-        .baselineValues(1l, 2l, 2l, 0.5)
-        .baselineValues(2l, 4l, 6l, 1.0)
-        .baselineValues(3l, 6l, 11l, 1.5)
-        .baselineValues(4l, 8l, 31l, 2.0)
-        .baselineValues(5l, 10l, 52l, 2.5)
-        .baselineValues(6l, 12l, 74l, 3.0)
+        .baselineValues(1L, 2L, 2L, 0.5)
+        .baselineValues(2L, 4L, 6L, 1.0)
+        .baselineValues(3L, 6L, 11L, 1.5)
+        .baselineValues(4L, 8L, 31L, 2.0)
+        .baselineValues(5L, 10L, 52L, 2.5)
+        .baselineValues(6L, 12L, 74L, 3.0)
         .build()
         .run();
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
index 684c79a3b7..6a4ec52dbc 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestBatchSerialization.java
@@ -169,6 +169,7 @@ public void testTypes() throws IOException {
     testType(MinorType.DECIMAL18);
     testType(MinorType.DECIMAL28SPARSE);
     testType(MinorType.DECIMAL38SPARSE);
+    testType(MinorType.VARDECIMAL);
 //  testType(MinorType.DECIMAL28DENSE); No writer
 //  testType(MinorType.DECIMAL38DENSE); No writer
     testType(MinorType.DATE);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
index 120f1e63d0..0a5b7cd6ec 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
@@ -17,10 +17,14 @@
  */
 package org.apache.drill.exec.fn.impl;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.common.exceptions.UserRemoteException;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.OperatorTest;
 import org.apache.drill.PlanTestBase;
@@ -39,6 +43,7 @@
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileWriter;
+import java.math.BigDecimal;
 import java.nio.file.Paths;
 import java.util.List;
 import java.util.Map;
@@ -65,8 +70,8 @@ public void testCountOnNullableColumn() throws Exception {
         .sqlQuery("select count(t.x.y)  as cnt1, count(`integer`) as cnt2 from cp.`jsoninput/input2.json` t")
         .ordered()
         .baselineColumns("cnt1", "cnt2")
-        .baselineValues(3l, 4l)
-        .build().run();
+        .baselineValues(3L, 4L)
+        .go();
   }
 
   @Test
@@ -75,18 +80,23 @@ public void testCountDistinctOnBoolColumn() throws Exception {
         .sqlQuery("select count(distinct `bool_val`) as cnt from `sys`.`options`")
         .ordered()
         .baselineColumns("cnt")
-        .baselineValues(2l)
-        .build().run();
+        .baselineValues(2L)
+        .go();
   }
 
   @Test
   public void testMaxWithZeroInput() throws Exception {
-    testBuilder()
-        .sqlQuery("select max(employee_id * 0.0) as max_val from cp.`employee.json`")
-        .unOrdered()
-        .baselineColumns("max_val")
-        .baselineValues(0.0d)
-        .go();
+    try {
+      alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, false);
+      testBuilder()
+          .sqlQuery("select max(employee_id * 0.0) as max_val from cp.`employee.json`")
+          .unOrdered()
+          .baselineColumns("max_val")
+          .baselineValues(0.0)
+          .go();
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    }
   }
 
   @Ignore
@@ -142,8 +152,8 @@ public void testDrill2170() throws Exception {
     .ordered()
     .optionSettingQueriesForTestQuery("alter system set `planner.slice_target` = 1000")
     .baselineColumns("cnt")
-    .baselineValues(100l)
-    .build().run();
+    .baselineValues(100L)
+    .go();
   }
 
   @Test // DRILL-2168
@@ -209,14 +219,14 @@ public void testCountWithAvg() throws Exception {
         .sqlQuery("select count(a) col1, avg(b) col2 from cp.`jsoninput/nullable3.json`")
         .unOrdered()
         .baselineColumns("col1", "col2")
-        .baselineValues(2l, 3.0d)
+        .baselineValues(2L, 3.0d)
         .go();
 
     testBuilder()
         .sqlQuery("select count(a) col1, avg(a) col2 from cp.`jsoninput/nullable3.json`")
         .unOrdered()
         .baselineColumns("col1", "col2")
-        .baselineValues(2l, 1.0d)
+        .baselineValues(2L, 1.0d)
         .go();
   }
 
@@ -240,6 +250,211 @@ public void testStddevOnKnownType() throws Exception {
         .go();
   }
 
+  @Test
+  public void testVarSampDecimal() throws Exception {
+    try {
+      alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+      testBuilder()
+          .sqlQuery("select var_samp(cast(employee_id as decimal(28, 20))) as dec20,\n" +
+                "var_samp(cast(employee_id as decimal(28, 0))) as dec6,\n" +
+                "var_samp(cast(employee_id as integer)) as d\n" +
+                "from cp.`employee.json`")
+          .unOrdered()
+          .baselineColumns("dec20", "dec6", "d")
+          .baselineValues(new BigDecimal("111266.99999699895713760532"),
+              new BigDecimal("111266.999997"),
+              111266.99999699896)
+          .go();
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    }
+  }
+
+  @Test
+  public void testVarPopDecimal() throws Exception {
+    try {
+      alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+      testBuilder()
+          .sqlQuery("select var_pop(cast(employee_id as decimal(28, 20))) as dec20,\n" +
+              "var_pop(cast(employee_id as decimal(28, 0))) as dec6,\n" +
+              "var_pop(cast(employee_id as integer)) as d\n" +
+              "from cp.`employee.json`")
+          .unOrdered()
+          .baselineColumns("dec20", "dec6", "d")
+          .baselineValues(new BigDecimal("111170.66493206649050804895"),
+              new BigDecimal("111170.664932"),
+              111170.66493206649)
+          .go();
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    }
+  }
+
+  @Test
+  public void testStddevSampDecimal() throws Exception {
+    try {
+      alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+      testBuilder()
+          .sqlQuery("select stddev_samp(cast(employee_id as decimal(28, 20))) as dec20,\n" +
+              "stddev_samp(cast(employee_id as decimal(28, 0))) as dec6,\n" +
+              "stddev_samp(cast(employee_id as integer)) as d\n" +
+              "from cp.`employee.json`")
+          .unOrdered()
+          .baselineColumns("dec20", "dec6", "d")
+          .baselineValues(new BigDecimal("333.56708470261114349632"),
+              new BigDecimal("333.567085"),
+              333.56708470261117) // last number differs because of double precision.
+          // Was taken sqrt of 111266.99999699895713760531784795216338 and decimal result is correct
+          .go();
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    }
+  }
+
+  @Test
+  public void testStddevPopDecimal() throws Exception {
+    try {
+      alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+      testBuilder()
+          .sqlQuery("select stddev_pop(cast(employee_id as decimal(28, 20))) as dec20,\n" +
+              "stddev_pop(cast(employee_id as decimal(28, 0))) as dec6,\n" +
+              "stddev_pop(cast(employee_id as integer)) as d\n" +
+              "from cp.`employee.json`")
+          .unOrdered()
+          .baselineColumns("dec20", "dec6", "d")
+          .baselineValues(new BigDecimal("333.42265209800381903633"),
+              new BigDecimal("333.422652"),
+              333.4226520980038)
+          .go();
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    }
+  }
+
+  @Test
+  public void testSumDecimal() throws Exception {
+    try {
+      alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+      testBuilder()
+          .sqlQuery("select sum(cast(employee_id as decimal(9, 0))) as colDecS0,\n" +
+              "sum(cast(employee_id as decimal(12, 3))) as colDecS3,\n" +
+              "sum(cast(employee_id as integer)) as colInt\n" +
+              "from cp.`employee.json`")
+          .unOrdered()
+          .baselineColumns("colDecS0", "colDecS3", "colInt")
+          .baselineValues(BigDecimal.valueOf(668743), new BigDecimal("668743.000"), 668743L)
+          .go();
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    }
+  }
+
+  @Test
+  public void testAvgDecimal() throws Exception {
+    try {
+      alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+      testBuilder()
+          .sqlQuery("select avg(cast(employee_id as decimal(28, 20))) as colDec20,\n" +
+              "avg(cast(employee_id as decimal(28, 0))) as colDec6,\n" +
+              "avg(cast(employee_id as integer)) as colInt\n" +
+              "from cp.`employee.json`")
+          .unOrdered()
+          .baselineColumns("colDec20", "colDec6", "colInt")
+          .baselineValues(new BigDecimal("578.99826839826839826840"),
+              new BigDecimal("578.998268"),
+              578.9982683982684)
+          .go();
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    }
+  }
+
+  @Test
+  public void testSumAvgDecimalLimit0() throws Exception {
+    final List<Pair<SchemaPath, TypeProtos.MajorType>> expectedSchema =
+        ImmutableList.of(
+            Pair.of(SchemaPath.getSimplePath("sum_col"),
+                Types.withScaleAndPrecision(TypeProtos.MinorType.VARDECIMAL, TypeProtos.DataMode.OPTIONAL, 3, 38)),
+            Pair.of(SchemaPath.getSimplePath("avg_col"),
+                Types.withScaleAndPrecision(TypeProtos.MinorType.VARDECIMAL, TypeProtos.DataMode.OPTIONAL, 6, 38)),
+            Pair.of(SchemaPath.getSimplePath("stddev_pop_col"),
+                Types.withScaleAndPrecision(TypeProtos.MinorType.VARDECIMAL, TypeProtos.DataMode.OPTIONAL, 6, 38)),
+            Pair.of(SchemaPath.getSimplePath("stddev_samp_col"),
+                Types.withScaleAndPrecision(TypeProtos.MinorType.VARDECIMAL, TypeProtos.DataMode.OPTIONAL, 6, 38)),
+            Pair.of(SchemaPath.getSimplePath("var_pop_col"),
+                Types.withScaleAndPrecision(TypeProtos.MinorType.VARDECIMAL, TypeProtos.DataMode.OPTIONAL, 6, 38)),
+            Pair.of(SchemaPath.getSimplePath("var_samp_col"),
+                Types.withScaleAndPrecision(TypeProtos.MinorType.VARDECIMAL, TypeProtos.DataMode.OPTIONAL, 6, 38)),
+            Pair.of(SchemaPath.getSimplePath("max_col"),
+                Types.withScaleAndPrecision(TypeProtos.MinorType.VARDECIMAL, TypeProtos.DataMode.OPTIONAL, 3, 9)),
+            Pair.of(SchemaPath.getSimplePath("min_col"),
+                Types.withScaleAndPrecision(TypeProtos.MinorType.VARDECIMAL, TypeProtos.DataMode.OPTIONAL, 3, 9)));
+
+    String query =
+        "select\n" +
+            "sum(cast(employee_id as decimal(9, 3))) sum_col,\n" +
+            "avg(cast(employee_id as decimal(9, 3))) avg_col,\n" +
+            "stddev_pop(cast(employee_id as decimal(9, 3))) stddev_pop_col,\n" +
+            "stddev_samp(cast(employee_id as decimal(9, 3))) stddev_samp_col,\n" +
+            "var_pop(cast(employee_id as decimal(9, 3))) var_pop_col,\n" +
+            "var_samp(cast(employee_id as decimal(9, 3))) var_samp_col,\n" +
+            "max(cast(employee_id as decimal(9, 3))) max_col,\n" +
+            "min(cast(employee_id as decimal(9, 3))) min_col\n" +
+            "from cp.`employee.json` limit 0";
+    try {
+      alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+      alterSession(ExecConstants.EARLY_LIMIT0_OPT_KEY, true);
+
+      testBuilder()
+          .sqlQuery(query)
+          .schemaBaseLine(expectedSchema)
+          .go();
+
+      alterSession(ExecConstants.EARLY_LIMIT0_OPT_KEY, false);
+
+      testBuilder()
+        .sqlQuery(query)
+        .schemaBaseLine(expectedSchema)
+        .go();
+
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+      resetSessionOption(ExecConstants.EARLY_LIMIT0_OPT_KEY);
+    }
+  }
+
+  @Test // DRILL-6221
+  public void testAggGroupByWithNullDecimal() throws Exception {
+    String fileName = "table.json";
+    try (BufferedWriter writer = new BufferedWriter(new FileWriter(new File(dirTestWatcher.getRootDir(), fileName)))) {
+      writer.write("{\"a\": 1, \"b\": 0}");
+      writer.write("{\"b\": 2}");
+    }
+
+    try {
+      alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+      testBuilder()
+          .sqlQuery("select sum(cast(a as decimal(9,0))) as s,\n" +
+              "avg(cast(a as decimal(9,0))) as a,\n" +
+              "var_samp(cast(a as decimal(9,0))) as varSamp,\n" +
+              "var_pop(cast(a as decimal(9,0))) as varPop,\n" +
+              "stddev_pop(cast(a as decimal(9,0))) as stddevPop,\n" +
+              "stddev_samp(cast(a as decimal(9,0))) as stddevSamp," +
+              "max(cast(a as decimal(9,0))) as mx," +
+            "min(cast(a as decimal(9,0))) as mn from dfs.`%s` t group by a", fileName)
+          .unOrdered()
+          .baselineColumns("s", "a", "varSamp", "varPop", "stddevPop", "stddevSamp", "mx", "mn")
+          .baselineValues(BigDecimal.valueOf(1), new BigDecimal("1.000000"), new BigDecimal("0.000000"),
+              new BigDecimal("0.000000"), new BigDecimal("0.000000"), new BigDecimal("0.000000"),
+              BigDecimal.valueOf(1), BigDecimal.valueOf(1))
+          .baselineValues(null, null, null, null, null, null, null, null)
+          .go();
+
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    }
+  }
+
   @Test
   // test aggregates when input is empty and data type is optional
   public void countEmptyNullableInput() throws Exception {
@@ -251,7 +466,7 @@ public void countEmptyNullableInput() throws Exception {
         .sqlQuery(query)
         .unOrdered()
         .baselineColumns("col1", "col2", "col3")
-        .baselineValues(0l, null, null)
+        .baselineValues(0L, null, null)
         .go();
   }
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
index 14a49b5200..4eb3917f99 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
@@ -20,6 +20,7 @@
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import mockit.integration.junit4.JMockit;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.categories.UnlikelyTest;
@@ -28,6 +29,8 @@
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import java.math.BigDecimal;
+import java.math.MathContext;
+import java.math.RoundingMode;
 import java.util.List;
 import java.util.Map;
 
@@ -41,8 +44,8 @@ public void testVarbinaryToDate() throws Exception {
       .sqlQuery("select count(*) as cnt from cp.`employee.json` where (cast(convert_to(birth_date, 'utf8') as date)) = date '1961-08-26'")
       .unOrdered()
       .baselineColumns("cnt")
-      .baselineValues(1l)
-      .build().run();
+      .baselineValues(1L)
+      .go();
   }
 
   @Test // DRILL-2827
@@ -66,9 +69,8 @@ public void testCastByConstantFolding() throws Exception {
         .sqlQuery(query)
         .ordered()
         .baselineColumns("col1", "col2")
-        .baselineValues(1155l, 1155l)
-        .build()
-        .run();
+        .baselineValues(1155L, 1155L)
+        .go();
   }
 
   @Test // DRILL-3769
@@ -116,8 +118,7 @@ public void testCastFloatToInt() throws Exception {
           .unOrdered()
           .baselineColumns("col1")
           .baselineValues(values.get(value))
-          .build()
-          .run();
+          .go();
       } finally {
         test("drop table if exists dfs.tmp.table_with_float");
       }
@@ -148,8 +149,7 @@ public void testCastIntToFloatAndDouble() throws Exception {
           .unOrdered()
           .baselineColumns("col1", "col2")
           .baselineValues((float) value, (double) value)
-          .build()
-          .run();
+          .go();
       } finally {
         test("drop table if exists dfs.tmp.table_with_int");
       }
@@ -185,8 +185,7 @@ public void testCastFloatToBigInt() throws Exception {
           .unOrdered()
           .baselineColumns("col1")
           .baselineValues(values.get(value))
-          .build()
-          .run();
+          .go();
       } finally {
         test("drop table if exists dfs.tmp.table_with_float");
       }
@@ -219,8 +218,7 @@ public void testCastBigIntToFloatAndDouble() throws Exception {
           .unOrdered()
           .baselineColumns("col1", "col2")
           .baselineValues((float) value, (double) value)
-          .build()
-          .run();
+          .go();
       } finally {
         test("drop table if exists dfs.tmp.table_with_bigint");
       }
@@ -256,8 +254,7 @@ public void testCastDoubleToInt() throws Exception {
           .unOrdered()
           .baselineColumns("col1")
           .baselineValues(values.get(value))
-          .build()
-          .run();
+          .go();
       } finally {
         test("drop table if exists dfs.tmp.table_with_double");
       }
@@ -293,8 +290,7 @@ public void testCastDoubleToBigInt() throws Exception {
           .unOrdered()
           .baselineColumns("col1")
           .baselineValues(values.get(value))
-          .build()
-          .run();
+          .go();
       } finally {
         test("drop table if exists dfs.tmp.table_with_double");
       }
@@ -324,8 +320,7 @@ public void testCastIntAndBigInt() throws Exception {
           .unOrdered()
           .baselineColumns("col1", "col2")
           .baselineValues((long) value, value)
-          .build()
-          .run();
+          .go();
       } finally {
         test("drop table if exists dfs.tmp.table_with_int");
       }
@@ -362,8 +357,7 @@ public void testCastFloatAndDouble() throws Exception {
           .unOrdered()
           .baselineColumns("col1", "col2")
           .baselineValues((double) ((float) (value)), (float) value)
-          .build()
-          .run();
+          .go();
       } finally {
         test("drop table if exists dfs.tmp.table_with_float");
       }
@@ -383,22 +377,29 @@ public void testCastIntAndBigIntToDecimal() throws Exception {
           .baselineValues(new BigDecimal(1), new BigDecimal(1), new BigDecimal(1), new BigDecimal(1))
           .baselineValues(new BigDecimal(-1), new BigDecimal(-1), new BigDecimal(-1), new BigDecimal(-1))
 
-          .baselineValues(new BigDecimal(Integer.MAX_VALUE),
+          .baselineValues(new BigDecimal(Integer.MAX_VALUE)
+                              .round(new MathContext(9, RoundingMode.HALF_UP))
+                              .setScale(0, RoundingMode.HALF_UP),
                           new BigDecimal(Integer.MAX_VALUE),
-                          new BigDecimal((int) Long.MAX_VALUE),
+                          new BigDecimal(Long.MAX_VALUE)
+                              .round(new MathContext(9, RoundingMode.HALF_UP))
+                              .setScale(0, RoundingMode.HALF_UP),
                           new BigDecimal(Long.MAX_VALUE))
 
-          .baselineValues(new BigDecimal(Integer.MIN_VALUE),
+          .baselineValues(new BigDecimal(Integer.MIN_VALUE)
+                              .round(new MathContext(9, RoundingMode.HALF_UP))
+                              .setScale(0, RoundingMode.HALF_UP),
                           new BigDecimal(Integer.MIN_VALUE),
-                          new BigDecimal((int) Long.MIN_VALUE),
+                          new BigDecimal(Long.MIN_VALUE)
+                              .round(new MathContext(9, RoundingMode.HALF_UP))
+                              .setScale(0, RoundingMode.HALF_UP),
                           new BigDecimal(Long.MIN_VALUE))
 
           .baselineValues(new BigDecimal(123456789),
                           new BigDecimal(123456789),
                           new BigDecimal(123456789),
                           new BigDecimal(123456789))
-          .build()
-          .run();
+          .go();
       } finally {
         test("drop table if exists dfs.tmp.table_with_int");
         test("alter session reset planner.enable_decimal_data_type");
@@ -417,11 +418,24 @@ public void testCastDecimalToIntAndBigInt() throws Exception {
         .baselineValues(0, 0, 0L, 0L)
         .baselineValues(1, 1, 1L, 1L)
         .baselineValues(-1, -1, -1L, -1L)
-        .baselineValues(Integer.MAX_VALUE, (int) Long.MAX_VALUE, (long) Integer.MAX_VALUE, Long.MAX_VALUE)
-        .baselineValues(Integer.MIN_VALUE, (int) Long.MIN_VALUE, (long) Integer.MIN_VALUE, Long.MIN_VALUE)
+        .baselineValues(new BigDecimal(Integer.MAX_VALUE)
+                          .round(new MathContext(9, RoundingMode.HALF_UP))
+                          .setScale(0, RoundingMode.HALF_UP).intValue(),
+                        (int) Long.MAX_VALUE,
+                        new BigDecimal(Integer.MAX_VALUE)
+                          .round(new MathContext(9, RoundingMode.HALF_UP))
+                          .setScale(0, RoundingMode.HALF_UP).longValue(),
+                        Long.MAX_VALUE)
+        .baselineValues(new BigDecimal(Integer.MIN_VALUE)
+                          .round(new MathContext(9, RoundingMode.HALF_UP))
+                          .setScale(0, RoundingMode.HALF_UP).intValue(),
+                        (int) Long.MIN_VALUE,
+                        new BigDecimal(Integer.MIN_VALUE)
+                          .round(new MathContext(9, RoundingMode.HALF_UP))
+                          .setScale(0, RoundingMode.HALF_UP).longValue(),
+                        Long.MIN_VALUE)
         .baselineValues(123456789, 123456789, 123456789L, 123456789L)
-        .build()
-        .run();
+        .go();
     } finally {
       test("drop table if exists dfs.tmp.table_with_int");
       test("alter session reset planner.enable_decimal_data_type");
@@ -443,14 +457,73 @@ public void testCastDecimalToFloatAndDouble() throws Exception {
         .baselineValues(-0.12f, -0.1004f, -0.12, -0.1004)
         .baselineValues(-123.1234f, -987654321.1234567891f, -123.1234, -987654321.1235)
         .baselineValues(-1.0001f, -2.0301f, -1.0001, -2.0301)
-        .build()
-        .run();
+        .go();
     } finally {
       test("drop table if exists dfs.tmp.table_with_int");
       test("alter session reset planner.enable_decimal_data_type");
     }
   }
 
+  @Test
+  public void testCastDecimalToVarDecimal() throws Exception {
+    try {
+      setSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+
+      testBuilder()
+        .physicalPlanFromFile("decimal/cast_decimal_vardecimal.json")
+        .unOrdered()
+        .baselineColumns("DEC28_COL", "DEC38_COL", "DEC9_COL", "DEC18_COL")
+        .baselineValues(new BigDecimal("-100000000001.0000000000000000"), new BigDecimal("1123.3000000000000000"),
+            new BigDecimal("1123"), new BigDecimal("-100000000001"))
+        .baselineValues(new BigDecimal("11.1234567890123456"), new BigDecimal("0.3000000000000000"),
+            new BigDecimal("0"), new BigDecimal("11"))
+        .baselineValues(new BigDecimal("0.1000000000010000"), new BigDecimal("123456789.0000000000000000"),
+            new BigDecimal("123456789"), new BigDecimal("0"))
+        .baselineValues(new BigDecimal("-0.1200000000000000"), new BigDecimal("0.0000020000000000"),
+            new BigDecimal("0"), new BigDecimal("0"))
+        .baselineValues(new BigDecimal("100000000001.1234567890010000"), new BigDecimal("111.3000000000000000"),
+            new BigDecimal("111"), new BigDecimal("100000000001"))
+        .baselineValues(new BigDecimal("-100000000001.0000000000000000"), new BigDecimal("121.0930000000000000"),
+            new BigDecimal("121"), new BigDecimal("-100000000001"))
+        .baselineValues(new BigDecimal("123456789123456789.0000000000000000"), new BigDecimal("12.3000000000000000"),
+            new BigDecimal("12"), new BigDecimal("123456789123456789"))
+        .go();
+    } finally {
+      test("drop table if exists dfs.tmp.table_with_int");
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    }
+  }
+
+  @Test
+  public void testCastVarDecimalToDecimal() throws Exception {
+    try {
+      setSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+
+      testBuilder()
+        .physicalPlanFromFile("decimal/cast_vardecimal_decimal.json")
+        .unOrdered()
+        .baselineColumns("DEC28_COL", "DEC38_COL", "DEC9_COL", "DEC18_COL")
+        .baselineValues(new BigDecimal("-100000000001.0000000000000000"), new BigDecimal("1123.3000000000000000"),
+          new BigDecimal("1123"), new BigDecimal("-100000000001"))
+        .baselineValues(new BigDecimal("11.1234567890123456"), new BigDecimal("0.3000000000000000"),
+          new BigDecimal("0"), new BigDecimal("11"))
+        .baselineValues(new BigDecimal("0.1000000000010000"), new BigDecimal("123456789.0000000000000000"),
+          new BigDecimal("123456789"), new BigDecimal("0"))
+        .baselineValues(new BigDecimal("-0.1200000000000000"), new BigDecimal("0.0000020000000000"),
+          new BigDecimal("0"), new BigDecimal("0"))
+        .baselineValues(new BigDecimal("100000000001.1234567890010000"), new BigDecimal("111.3000000000000000"),
+          new BigDecimal("111"), new BigDecimal("100000000001"))
+        .baselineValues(new BigDecimal("-100000000001.0000000000000000"), new BigDecimal("121.0930000000000000"),
+          new BigDecimal("121"), new BigDecimal("-100000000001"))
+        .baselineValues(new BigDecimal("123456789123456789.0000000000000000"), new BigDecimal("12.3000000000000000"),
+          new BigDecimal("12"), new BigDecimal("123456789123456789"))
+        .go();
+    } finally {
+      test("drop table if exists dfs.tmp.table_with_int");
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    }
+  }
+
   @Test // DRILL-4970
   public void testCastNegativeFloatToInt() throws Exception {
     try {
@@ -477,8 +550,7 @@ public void testCastNegativeFloatToInt() throws Exception {
             .unOrdered()
             .baselineColumns("c")
             .baselineValues(1L)
-            .build()
-            .run();
+            .go();
         }
       }
     } finally {
@@ -509,12 +581,24 @@ public void testCastNegativeDecimalToVarChar() throws Exception {
           .unOrdered()
           .baselineColumns("c")
           .baselineValues(1L)
-          .build()
-          .run();
+          .go();
       }
     } finally {
       test("drop table if exists dfs.tmp.table_with_decimal");
       test("alter session reset planner.enable_decimal_data_type");
     }
   }
+
+  @Test
+  public void testCastDecimalLiteral() throws Exception {
+    String query =
+        "select case when true then cast(100.0 as decimal(38,2)) else cast('123.0' as decimal(38,2)) end as c1";
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("c1")
+        .baselineValues(new BigDecimal("100.00"))
+        .go();
+  }
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestVarDecimalFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestVarDecimalFunctions.java
new file mode 100644
index 0000000000..b74a860707
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestVarDecimalFunctions.java
@@ -0,0 +1,743 @@
+/*
+ * 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.exec.fn.impl;
+
+import org.apache.drill.categories.SqlFunctionTest;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.test.BaseTestQuery;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.math.RoundingMode;
+
+@Category(SqlFunctionTest.class)
+public class TestVarDecimalFunctions extends BaseTestQuery {
+
+  @BeforeClass
+  public static void enableDecimalDataType() {
+    setSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+  }
+
+  @AfterClass
+  public static void disableDecimalDataType() {
+    resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+  }
+
+  // Tests for math functions
+
+  @Test
+  public void testDecimalAdd() throws Exception {
+    String query =
+        "select\n" +
+            // checks trimming of scale
+            "cast('999999999999999999999999999.92345678912' as DECIMAL(38, 11))\n" +
+            "+ cast('0.32345678912345678912345678912345678912' as DECIMAL(38, 38)) as s1,\n" +
+            // sanitary checks
+            "cast('1234567891234567891234567891234567.89' as DECIMAL(36, 2))\n" +
+            "+ cast('123456789123456789123456789123456.789' as DECIMAL(36, 3)) as s2,\n" +
+            "cast('1234567891234567891234567891234567.89' as DECIMAL(36, 2))\n" +
+            "+ cast('0' as DECIMAL(36, 3)) as s3,\n" +
+            "cast('15.02' as DECIMAL(4, 2)) - cast('12.93' as DECIMAL(4, 2)) as s4,\n" +
+            "cast('11.02' as DECIMAL(4, 2)) - cast('12.93' as DECIMAL(4, 2)) as s5,\n" +
+            "cast('0' as DECIMAL(36, 2)) - cast('12.93' as DECIMAL(36, 2)) as s6,\n" +
+            // check trimming (negative scale)
+            "cast('99999999999999999999999999992345678912' as DECIMAL(38, 0))\n" +
+            "+ cast('32345678912345678912345678912345678912' as DECIMAL(38, 0)) as s7";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6", "s7")
+        .baselineValues(
+            new BigDecimal("999999999999999999999999999.92345678912")
+                .add(new BigDecimal("0.32345678912345678912345678912345678912"))
+                .round(new MathContext(38, RoundingMode.HALF_UP)),
+            new BigDecimal("1358024680358024680358024680358024.679"),
+            new BigDecimal("1234567891234567891234567891234567.890"),
+            new BigDecimal("2.09"), new BigDecimal("-1.91"), new BigDecimal("-12.93"),
+            new BigDecimal("1.3234567891234567891234567890469135782E+38"))
+        .go();
+  }
+
+  @Test
+  public void testDecimalMultiply() throws Exception {
+    String query =
+        "select\n" +
+            // checks trimming of scale
+            "cast('999999999999999999999999999.92345678912' as DECIMAL(38, 11))\n" +
+            "* cast('0.32345678912345678912345678912345678912' as DECIMAL(38, 38)) as s1,\n" +
+            // sanitary checks
+            "cast('1234567.89' as DECIMAL(9, 2))\n" +
+            "* cast('-1.789' as DECIMAL(4, 3)) as s2,\n" +
+            "cast('15.02' as DECIMAL(4, 2)) * cast('0' as DECIMAL(4, 2)) as s3,\n" +
+            "cast('12.93123456789' as DECIMAL(13, 11)) * cast('1' as DECIMAL(1, 0)) as s4";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4")
+        .baselineValues(new BigDecimal("999999999999999999999999999.92345678912")
+                          .multiply(new BigDecimal("0.32345678912345678912345678912345678912"))
+                          .round(new MathContext(38, RoundingMode.HALF_UP)),
+            new BigDecimal("-2208641.95521"),
+            new BigDecimal("0.0000"), new BigDecimal("12.93123456789"))
+        .go();
+  }
+
+  @Test
+  public void testDecimalDivide() throws Exception {
+    String query =
+        "select\n" +
+            // checks trimming of scale
+            "cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37))\n" +
+            "/ cast('0.00000000000000000000000000000000000001' as DECIMAL(38, 38)) as s1,\n" +
+            // sanitary checks
+            "cast('1234567.89' as DECIMAL(9, 2))\n" +
+            "/ cast('-1.789' as DECIMAL(4, 3)) as s2,\n" +
+            "cast('15.02' as DECIMAL(4, 2)) / cast('15.02' as DECIMAL(4, 2)) as s3,\n" +
+            "cast('12.93123456789' as DECIMAL(13, 11)) / cast('1' as DECIMAL(1, 0)) as s4,\n" +
+            "cast('0' as DECIMAL(0, 0)) / cast('15.02' as DECIMAL(4, 2)) as s5";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5")
+        .baselineValues(new BigDecimal("199999999999999999999999999992345678910"),
+            new BigDecimal("-690088.2560089"),
+            new BigDecimal("1.0000000"), new BigDecimal("12.9312345678900"), new BigDecimal("0.000000"))
+        .go();
+  }
+
+  @Test
+  public void testDecimalMod() throws Exception {
+    String query =
+        "select\n" +
+            "mod(cast('1111' as DECIMAL(4, 0)), cast('12' as DECIMAL(2, 0))) as s1,\n" +
+            "mod(cast('1234567' as DECIMAL(7, 0)),\n" +
+            "cast('-9' as DECIMAL(1, 0))) as s2,\n" +
+            "mod(cast('-1502' as DECIMAL(4, 0)), cast('15' as DECIMAL(2, 0))) as s3,\n" +
+            "mod(cast('-987654' as DECIMAL(6, 0)), cast('-31' as DECIMAL(2, 0))) as s4";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4")
+        .baselineValues(new BigDecimal("7"), new BigDecimal("1"),
+            new BigDecimal("-2"), new BigDecimal("-25"))
+        .go();
+  }
+
+  @Test
+  public void testDecimalAbs() throws Exception {
+    String query =
+        "select\n" +
+            "abs(cast('1111' as DECIMAL(4, 0))) as s1,\n" +
+            "abs(cast('-1234567.123456' as DECIMAL(13, 6))) as s2,\n" +
+            "abs(cast('-1502' as DECIMAL(4, 0))) as s3,\n" +
+            "abs(cast('0' as DECIMAL(4, 0))) as s4,\n" +
+            "abs(cast('-987654' as DECIMAL(6, 0))) as s5";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5")
+        .baselineValues(new BigDecimal("1111"), new BigDecimal("1234567.123456"),
+            new BigDecimal("1502"), new BigDecimal("0"), new BigDecimal("987654"))
+        .go();
+  }
+
+  @Test
+  public void testDecimalCeil() throws Exception {
+    String query =
+        "select\n" +
+            "ceil(cast('1111.35' as DECIMAL(6, 2))) as s1,\n" +
+            "ceiling(cast('1234567.123456' as DECIMAL(13, 6))) as s2,\n" +
+            "ceil(cast('-1502.5' as DECIMAL(5, 1))) as s3,\n" +
+            "ceiling(cast('987654.5' as DECIMAL(7, 1))) as s4,\n" +
+            "ceil(cast('987654.00' as DECIMAL(8, 2))) as s5";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5")
+        .baselineValues(new BigDecimal("1112"), new BigDecimal("1234568"),
+            new BigDecimal("-1502"), new BigDecimal("987655"), new BigDecimal("987654"))
+        .go();
+  }
+
+  @Test
+  public void testDecimalFloor() throws Exception {
+    String query =
+        "select\n" +
+            "floor(cast('1111.35' as DECIMAL(6, 2))) as s1,\n" +
+            "floor(cast('1234567.123456' as DECIMAL(13, 6))) as s2,\n" +
+            "floor(cast('-1502.5' as DECIMAL(5, 1))) as s3,\n" +
+            "floor(cast('987654.5' as DECIMAL(7, 1))) as s4,\n" +
+            "floor(cast('987654.00' as DECIMAL(8, 2))) as s5";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5")
+        .baselineValues(new BigDecimal("1111"), new BigDecimal("1234567"),
+            new BigDecimal("-1503"), new BigDecimal("987654"), new BigDecimal("987654"))
+        .go();
+  }
+
+  @Test
+  public void testDecimalTrunc() throws Exception {
+    String query =
+        "select\n" +
+            "trunc(cast('1111.35' as DECIMAL(6, 2))) as s1,\n" +
+            "truncate(cast('1234567.123456' as DECIMAL(13, 6))) as s2,\n" +
+            "trunc(cast('-1502.5' as DECIMAL(5, 1))) as s3,\n" +
+            "truncate(cast('987654.5' as DECIMAL(7, 1))) as s4,\n" +
+            "trunc(cast('987654.00' as DECIMAL(8, 2))) as s5";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5")
+        .baselineValues(new BigDecimal("1111"), new BigDecimal("1234567"),
+            new BigDecimal("-1502"), new BigDecimal("987654"), new BigDecimal("987654"))
+        .go();
+  }
+
+  @Test
+  public void testDecimalRound() throws Exception {
+    String query =
+        "select\n" +
+            "round(cast('1111.45' as DECIMAL(6, 2))) as s1,\n" +
+            "round(cast('1234567.523456' as DECIMAL(13, 6))) as s2,\n" +
+            "round(cast('-1502.5' as DECIMAL(5, 1))) as s3,\n" +
+            "round(cast('-987654.4' as DECIMAL(7, 1))) as s4,\n" +
+            "round(cast('987654.00' as DECIMAL(8, 2))) as s5";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5")
+        .baselineValues(new BigDecimal("1111"), new BigDecimal("1234568"),
+            new BigDecimal("-1503"), new BigDecimal("-987654"), new BigDecimal("987654"))
+        .go();
+  }
+
+  @Test
+  public void testDecimalSign() throws Exception {
+    String query =
+        "select\n" +
+            "sign(cast('+1111.45' as DECIMAL(6, 2))) as s1,\n" +
+            "sign(cast('-1234567.523456' as DECIMAL(13, 6))) as s2,\n" +
+            "sign(cast('-1502.5' as DECIMAL(5, 1))) as s3,\n" +
+            "sign(cast('987654.4' as DECIMAL(7, 1))) as s4,\n" +
+            "sign(cast('0' as DECIMAL(8, 2))) as s5";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5")
+        .baselineValues(1, -1, -1, 1, 0)
+        .go();
+  }
+
+  @Test
+  public void testDecimalRoundWithScale() throws Exception {
+    String query =
+        "select\n" +
+            "round(cast('1111.45' as DECIMAL(6, 2)), 1) as s1,\n" +
+            "round(cast('1234567.523456' as DECIMAL(13, 6)), 5) as s2,\n" +
+            "round(cast('-1502.5' as DECIMAL(5, 1)), 0) as s3,\n" +
+            "round(cast('-987654.4' as DECIMAL(7, 1)), 1) as s4,\n" +
+            "round(cast('987654.00' as DECIMAL(8, 2)), 2) as s5";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5")
+        .baselineValues(new BigDecimal("1111.5"), new BigDecimal("1234567.52346"),
+            new BigDecimal("-1503"), new BigDecimal("-987654.4"), new BigDecimal("987654.00"))
+        .go();
+  }
+
+  @Test
+  public void testDecimalTruncWithScale() throws Exception {
+    String query =
+        "select\n" +
+            "trunc(cast('1111.45' as DECIMAL(6, 2)), 1) as s1,\n" +
+            "truncate(cast('1234567.523456' as DECIMAL(13, 6)), 5) as s2,\n" +
+            "trunc(cast('-1502.5' as DECIMAL(5, 1)), 0) as s3,\n" +
+            "truncate(cast('-987654.4' as DECIMAL(7, 1)), 1) as s4,\n" +
+            "trunc(cast('987654.00' as DECIMAL(8, 2)), 2) as s5";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5")
+        .baselineValues(new BigDecimal("1111.4"), new BigDecimal("1234567.52345"),
+            new BigDecimal("-1502"), new BigDecimal("-987654.4"), new BigDecimal("987654.00"))
+        .go();
+  }
+
+  // Tests for comparison functions
+
+  @Test
+  public void testDecimalEquals() throws Exception {
+    String query =
+        "select\n" +
+            "cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37))\n" +
+            "= cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37)) as s1,\n" +
+            "cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37))\n" +
+            "<> cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37)) as s2,\n" +
+            // the same value but different scale and precision
+            "cast('1234567.89' as DECIMAL(9, 2)) = cast('1234567.890' as DECIMAL(10, 3)) as s3,\n" +
+            "cast('0' as DECIMAL(4, 2)) = cast('0' as DECIMAL(0, 0)) as s4,\n" +
+            "cast('0' as DECIMAL(4, 2)) <> cast('0' as DECIMAL(0, 0)) as s5,\n" +
+            "cast('12.93123456789' as DECIMAL(13, 11)) = cast('12.93123456788' as DECIMAL(13, 11)) as s6";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6")
+        .baselineValues(true, false, true, true, false, false)
+        .go();
+  }
+
+  @Test
+  public void testDecimalLessThan() throws Exception {
+    String query =
+        "select\n" +
+            "cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37))\n" +
+            "< cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37)) as s1,\n" +
+            "cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37))\n" +
+            "< cast('1.9999999999999999999999999999234567892' as DECIMAL(38, 37)) as s2,\n" +
+            // the same value but different scale and precision
+            "cast('1234567.89' as DECIMAL(9, 2)) < cast('1234567.890' as DECIMAL(10, 3)) as s3,\n" +
+            "cast('0' as DECIMAL(4, 2)) < cast('0' as DECIMAL(0, 0)) as s4,\n" +
+            "cast('12.93123456789' as DECIMAL(13, 11)) < cast('12.93123456788' as DECIMAL(13, 11)) as s5";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5")
+        .baselineValues(false, true, false, false, false)
+        .go();
+  }
+
+  @Test
+  public void testDecimalLessThanEquals() throws Exception {
+    String query =
+        "select\n" +
+            "cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37))\n" +
+            "<= cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37)) as s1,\n" +
+            "cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37))\n" +
+            "<= cast('1.9999999999999999999999999999234567892' as DECIMAL(38, 37)) as s2,\n" +
+            // the same value but different scale and precision
+            "cast('1234567.89' as DECIMAL(9, 2)) <= cast('1234567.890' as DECIMAL(10, 3)) as s3,\n" +
+            "cast('0' as DECIMAL(4, 2)) <= cast('0' as DECIMAL(0, 0)) as s4,\n" +
+            "cast('12.93123456789' as DECIMAL(13, 11)) <= cast('12.93123456788' as DECIMAL(13, 11)) as s5";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5")
+        .baselineValues(true, true, true, true, false)
+        .go();
+  }
+
+  @Test
+  public void testDecimalGreaterThan() throws Exception {
+    String query =
+        "select\n" +
+            "cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37))\n" +
+            "> cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37)) as s1,\n" +
+            "cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37))\n" +
+            "> cast('1.9999999999999999999999999999234567892' as DECIMAL(38, 37)) as s2,\n" +
+            // the same value but different scale and precision
+            "cast('1234567.89' as DECIMAL(9, 2)) > cast('1234567.890' as DECIMAL(10, 3)) as s3,\n" +
+            "cast('0' as DECIMAL(4, 2)) > cast('0' as DECIMAL(0, 0)) as s4,\n" +
+            "cast('12.93123456789' as DECIMAL(13, 11)) > cast('12.93123456788' as DECIMAL(13, 11)) as s5";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5")
+        .baselineValues(false, false, false, false, true)
+        .go();
+  }
+
+  @Test
+  public void testDecimalGreaterThanEquals() throws Exception {
+    String query =
+        "select\n" +
+            "cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37))\n" +
+            ">= cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37)) as s1,\n" +
+            "cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37))\n" +
+            ">= cast('1.9999999999999999999999999999234567892' as DECIMAL(38, 37)) as s2,\n" +
+            // the same value but different scale and precision
+            "cast('1234567.89' as DECIMAL(9, 2)) >= cast('1234567.890' as DECIMAL(10, 3)) as s3,\n" +
+            "cast('0' as DECIMAL(4, 2)) >= cast('0' as DECIMAL(0, 0)) as s4,\n" +
+            "cast('12.93123456789' as DECIMAL(13, 11)) >= cast('12.93123456788' as DECIMAL(13, 11)) as s5";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5")
+        .baselineValues(true, false, true, true, true)
+        .go();
+  }
+
+  @Test
+  public void testDecimalCompareToNullsHigh() throws Exception {
+    String query =
+        "select\n" +
+            "compare_to_nulls_high(cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37)),\n" +
+            "cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37))) as s1,\n" +
+            "compare_to_nulls_high(cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37)),\n" +
+            "cast('1.9999999999999999999999999999234567892' as DECIMAL(38, 37))) as s2,\n" +
+            // the same value but different scale and precision
+            "compare_to_nulls_high(cast('1234567.89' as DECIMAL(9, 2)), cast('1234567.890' as DECIMAL(10, 3))) as s3,\n" +
+            "compare_to_nulls_high(cast('0' as DECIMAL(4, 2)), cast('0' as DECIMAL(0, 0))) as s4,\n" +
+            "compare_to_nulls_high(cast('0' as DECIMAL(4, 2)), cast(null as DECIMAL(0, 0))) as s5,\n" +
+            "compare_to_nulls_high(cast('12.93123456789' as DECIMAL(13, 11)), " +
+            "cast('12.93123456788' as DECIMAL(13, 11))) as s6";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6")
+        .baselineValues(0, -1, 0, 0, -1, 1)
+        .go();
+  }
+
+  @Test
+  public void testDecimalCompareToNullsLow() throws Exception {
+    String query =
+        "select\n" +
+            "compare_to_nulls_low(cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37)),\n" +
+            "cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37))) as s1,\n" +
+            "compare_to_nulls_low(cast('1.9999999999999999999999999999234567891' as DECIMAL(38, 37)),\n" +
+            "cast('1.9999999999999999999999999999234567892' as DECIMAL(38, 37))) as s2,\n" +
+            // the same value but different scale and precision
+            "compare_to_nulls_low(cast('1234567.89' as DECIMAL(9, 2)), cast('1234567.890' as DECIMAL(10, 3))) as s3,\n" +
+            "compare_to_nulls_low(cast('0' as DECIMAL(4, 2)), cast('0' as DECIMAL(0, 0))) as s4,\n" +
+            "compare_to_nulls_low(cast('0' as DECIMAL(4, 2)), cast(null as DECIMAL(0, 0))) as s5,\n" +
+            "compare_to_nulls_low(cast('12.93123456789' as DECIMAL(13, 11)), " +
+            "cast('12.93123456788' as DECIMAL(13, 11))) as s6";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6")
+        .baselineValues(0, -1, 0, 0, 1, 1)
+        .go();
+  }
+
+  // Tests for cast functions
+
+  @Test
+  public void testCastIntDecimal() throws Exception {
+    String query =
+        "select\n" +
+            "cast(i1 as DECIMAL(4, 0)) as s1,\n" +
+            "cast(i2 as DECIMAL(7, 0)) as s2,\n" +
+            "cast(i3 as DECIMAL(8, 0)) as s3,\n" +
+            "cast(i4 as DECIMAL(6, 0)) as s4,\n" +
+            "cast(i5 as DECIMAL(6, 0)) as s5,\n" +
+            "cast(i6 as DECIMAL(10, 0)) as s6,\n" +
+            "cast(i7 as DECIMAL(10, 0)) as s7\n" +
+        "from (" +
+            "select\n" +
+                "cast(0 as int) as i1,\n" +
+                "cast(1234567 as int) as i2,\n" +
+                "cast(-15022222 as int) as i3,\n" +
+                "cast(-987654 as int) as i4,\n" +
+                "cast(987654 as int) as i5,\n" +
+                "cast(%s as int) as i6,\n" +
+                "cast(%s as int) as i7)";
+    testBuilder()
+        .sqlQuery(query, Integer.MAX_VALUE, Integer.MIN_VALUE)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6", "s7")
+        .baselineValues(BigDecimal.valueOf(0), BigDecimal.valueOf(1234567),
+            BigDecimal.valueOf(-15022222), BigDecimal.valueOf(-987654),BigDecimal.valueOf(987654),
+            BigDecimal.valueOf(Integer.MAX_VALUE), BigDecimal.valueOf(Integer.MIN_VALUE))
+        .go();
+  }
+
+  @Test
+  public void testCastDecimalInt() throws Exception {
+    String query =
+        "select\n" +
+            "cast(i1 as int) as s1,\n" +
+            "cast(i2 as int) as s2,\n" +
+            "cast(i3 as int) as s3,\n" +
+            "cast(i4 as int) as s4,\n" +
+            "cast(i5 as int) as s5,\n" +
+            "cast(i6 as int) as s6,\n" +
+            "cast(i7 as int) as s7\n" +
+        "from (" +
+            "select\n" +
+                "cast('999999999999999999999999999.92345678912' as DECIMAL(38, 11)) as i1,\n" +
+                "cast('0.32345678912345678912345678912345678912' as DECIMAL(38, 38)) as i2,\n" +
+                "cast('-1234567891234567891234567891234567.89' as DECIMAL(36, 2)) as i3,\n" +
+                "cast('0' as DECIMAL(36, 3)) as i4,\n" +
+                "cast('15.02' as DECIMAL(4, 2)) as i5,\n" +
+                "cast('-15.02' as DECIMAL(4, 2)) as i6,\n" +
+                "cast('0.7877' as DECIMAL(4, 4)) as i7)";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6", "s7")
+        .baselineValues(
+            new BigDecimal("999999999999999999999999999.92345678912")
+                .setScale(0, BigDecimal.ROUND_HALF_UP).intValue(),
+            0,
+            new BigDecimal("-1234567891234567891234567891234567.89")
+                .setScale(0, BigDecimal.ROUND_HALF_UP).intValue(),
+            0, 15, -15, 1)
+        .go();
+  }
+
+  @Test
+  public void testCastBigIntDecimal() throws Exception {
+    String query =
+        "select\n" +
+            "cast(i1 as DECIMAL(4, 0)) as s1,\n" +
+            "cast(i2 as DECIMAL(7, 0)) as s2,\n" +
+            "cast(i3 as DECIMAL(8, 0)) as s3,\n" +
+            "cast(i4 as DECIMAL(6, 0)) as s4,\n" +
+            "cast(i5 as DECIMAL(6, 0)) as s5,\n" +
+            "cast(i6 as DECIMAL(19, 0)) as s6,\n" +
+            "cast(i7 as DECIMAL(19, 0)) as s7\n" +
+        "from (" +
+            "select " +
+                "cast(0 as bigint) as i1,\n" +
+                "cast(1234567 as bigint) as i2,\n" +
+                "cast(-15022222 as bigint) as i3,\n" +
+                "cast(-987654 as bigint) as i4,\n" +
+                "cast(987654 as bigint) as i5,\n" +
+                "cast(%s as bigint) as i6,\n" +
+                "cast(%s as bigint) as i7)";
+    testBuilder()
+        .sqlQuery(query, Long.MAX_VALUE, Long.MIN_VALUE)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6", "s7")
+        .baselineValues(new BigDecimal("0"), new BigDecimal("1234567"),
+            new BigDecimal("-15022222"), new BigDecimal("-987654"), new BigDecimal("987654"),
+            BigDecimal.valueOf(Long.MAX_VALUE), BigDecimal.valueOf(Long.MIN_VALUE))
+        .go();
+  }
+
+  @Test
+  public void testCastDecimalBigInt() throws Exception {
+    String query =
+        "select\n" +
+            "cast(i1 as bigint) as s1,\n" +
+            "cast(i2 as bigint) as s2,\n" +
+            "cast(i3 as bigint) as s3,\n" +
+            "cast(i4 as bigint) as s4,\n" +
+            "cast(i5 as bigint) as s5,\n" +
+            "cast(i6 as bigint) as s6,\n" +
+            "cast(i7 as bigint) as s7\n" +
+        "from (" +
+            "select\n" +
+                "cast('999999999999999999999999999.92345678912' as DECIMAL(38, 11)) as i1,\n" +
+                "cast('0.32345678912345678912345678912345678912' as DECIMAL(38, 38)) as i2,\n" +
+                "cast('-1234567891234567891234567891234567.89' as DECIMAL(36, 2)) as i3,\n" +
+                "cast('0' as DECIMAL(36, 3)) as i4,\n" +
+                "cast('15.02' as DECIMAL(4, 2)) as i5,\n" +
+                "cast('-15.02' as DECIMAL(4, 2)) as i6,\n" +
+                "cast('0.7877' as DECIMAL(4, 4)) as i7)";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6", "s7")
+        .baselineValues(
+            new BigDecimal("999999999999999999999999999.92345678912")
+                .setScale(0, BigDecimal.ROUND_HALF_UP).longValue(),
+            0L,
+            new BigDecimal("-1234567891234567891234567891234567.89")
+                .setScale(0, BigDecimal.ROUND_HALF_UP).longValue(),
+            0L, 15L, -15L, 1L)
+        .go();
+  }
+
+  @Test
+  public void testCastFloatDecimal() throws Exception {
+    String query =
+        "select\n" +
+            "cast(i1 as DECIMAL(4, 0)) as s1,\n" +
+            "cast(i2 as DECIMAL(7, 6)) as s2,\n" +
+            "cast(i3 as DECIMAL(8, 7)) as s3,\n" +
+            "cast(i4 as DECIMAL(6, 6)) as s4,\n" +
+            "cast(i5 as DECIMAL(7, 0)) as s5,\n" +
+            "cast(i6 as DECIMAL(7, 46)) as s6,\n" +
+            "cast(i7 as DECIMAL(17, 0)) as s7\n" +
+        "from (" +
+            "select\n" +
+                "cast(0 as float) as i1,\n" +
+                "cast(1.234567 as float) as i2,\n" +
+                "cast(-1.5022222 as float) as i3,\n" +
+                "cast(-0.987654 as float) as i4,\n" +
+                "cast(9999999 as float) as i5,\n" +
+                "cast('%s' as float) as i6,\n" +
+                "cast('%s' as float) as i7)";
+
+    testBuilder()
+        .sqlQuery(query, Float.MIN_VALUE, Float.MAX_VALUE)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6", "s7")
+        .baselineValues(BigDecimal.valueOf(0), new BigDecimal("1.234567"),
+            new BigDecimal("-1.5022222"), new BigDecimal("-0.987654"), BigDecimal.valueOf(9999999),
+            new BigDecimal(String.format("%s", Float.MIN_VALUE)),
+            new BigDecimal("340282350000000000000000000000000000000")) // Float.MAX_VALUE in non-scientific format
+        .go();
+  }
+
+  @Test
+  public void testCastDecimalFloat() throws Exception {
+    String query =
+        "select\n" +
+            "cast(i1 as float) as s1,\n" +
+            "cast(i2 as float) as s2,\n" +
+            "cast(i3 as float) as s3,\n" +
+            "cast(i4 as float) as s4,\n" +
+            "cast(i5 as float) as s5,\n" +
+            "cast(i6 as float) as s6,\n" +
+            "cast(i7 as float) as s7\n" +
+        "from (" +
+            "select\n" +
+                "cast('999999999999999999999999999.92345678912' as DECIMAL(38, 11)) as i1,\n" +
+                "cast('0.32345678912345678912345678912345678912' as DECIMAL(38, 38)) as i2,\n" +
+                "cast('-1234567891234567891234567891234567.89' as DECIMAL(36, 2)) as i3,\n" +
+                "cast('0' as DECIMAL(36, 3)) as i4,\n" +
+                "cast('15.02' as DECIMAL(4, 2)) as i5,\n" +
+                "cast('%s' as DECIMAL(2, 46)) as i6,\n" +
+                "cast('%s' as DECIMAL(8, 0)) as i7)";
+
+    testBuilder()
+        .sqlQuery(query, Float.MIN_VALUE, Float.MAX_VALUE)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6", "s7")
+        .baselineValues(new BigDecimal("999999999999999999999999999.92345678912").floatValue(),
+            new BigDecimal("0.32345678912345678912345678912345678912").floatValue(),
+            new BigDecimal("-1234567891234567891234567891234567.89").floatValue(),
+            0f, 15.02f, Float.MIN_VALUE, Float.MAX_VALUE)
+        .go();
+  }
+
+  @Test
+  public void testCastDoubleDecimal() throws Exception {
+    String query =
+        "select\n" +
+            "cast(i1 as DECIMAL(4, 0)) as s1,\n" +
+            "cast(i2 as DECIMAL(7, 6)) as s2,\n" +
+            "cast(i3 as DECIMAL(8, 7)) as s3,\n" +
+            "cast(i4 as DECIMAL(6, 6)) as s4,\n" +
+            "cast(i5 as DECIMAL(7, 0)) as s5,\n" +
+            "cast(i6 as DECIMAL(17, 325)) as s6,\n" +
+            "cast(i7 as DECIMAL(17, 0)) as s7\n" +
+        "from (" +
+            "select\n" +
+                "cast(0 as double) as i1,\n" +
+                "cast(1.234567 as double) as i2,\n" +
+                "cast(-1.5022222 as double) as i3,\n" +
+                "cast(-0.987654 as double) as i4,\n" +
+                "cast(9999999 as double) as i5,\n" +
+                "cast('%e' as double) as i6,\n" +
+                "cast('%f' as double) as i7)";
+
+    testBuilder()
+        .sqlQuery(query, Double.MIN_VALUE, Double.MAX_VALUE)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6", "s7")
+        .baselineValues(BigDecimal.valueOf(0), new BigDecimal("1.234567"),
+            new BigDecimal("-1.5022222"), new BigDecimal("-0.987654"), BigDecimal.valueOf(9999999),
+            new BigDecimal(String.valueOf(Double.MIN_VALUE)),
+            new BigDecimal(String.format("%1.0f", Double.MAX_VALUE))) // non-scientific format
+        .go();
+  }
+
+  @Test
+  public void testCastDecimalDouble() throws Exception {
+    String query =
+        "select\n" +
+            "cast(i1 as double) as s1,\n" +
+            "cast(i2 as double) as s2,\n" +
+            "cast(i3 as double) as s3,\n" +
+            "cast(i4 as double) as s4,\n" +
+            "cast(i5 as double) as s5,\n" +
+            "cast(i6 as double) as s6,\n" +
+            "cast(i7 as double) as s7\n" +
+        "from (" +
+            "select\n" +
+                "cast('999999999999999999999999999.92345678912' as DECIMAL(38, 11)) as i1,\n" +
+                "cast('0.32345678912345678912345678912345678912' as DECIMAL(38, 38)) as i2,\n" +
+                "cast('-1234567891234567891234567891234567.89' as DECIMAL(36, 2)) as i3,\n" +
+                "cast('0' as DECIMAL(36, 3)) as i4,\n" +
+                "cast('15.02' as DECIMAL(4, 2)) as i5,\n" +
+                "cast('%e' as DECIMAL(17, 325)) as i6,\n" +
+                "cast('%f' as DECIMAL(17, 0)) as i7)";
+
+    testBuilder()
+        .sqlQuery(query, Double.MIN_VALUE, Double.MAX_VALUE)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5", "s6", "s7")
+        .baselineValues(new BigDecimal("999999999999999999999999999.92345678912").doubleValue(),
+            new BigDecimal("0.32345678912345678912345678912345678912").doubleValue(),
+            new BigDecimal("-1234567891234567891234567891234567.89").doubleValue(),
+            0d, 15.02, Double.MIN_VALUE, Double.MAX_VALUE)
+        .go();
+  }
+
+  @Test
+  public void testCastDecimalVarchar() throws Exception {
+    String query =
+        "select\n" +
+            "cast(i1 as varchar) as s1,\n" +
+            "cast(i2 as varchar) as s2,\n" +
+            "cast(i3 as varchar) as s3,\n" +
+            "cast(i4 as varchar) as s4,\n" +
+            "cast(i5 as varchar) as s5\n" +
+        "from (" +
+            "select\n" +
+                "cast('999999999999999999999999999.92345678912' as DECIMAL(38, 11)) as i1,\n" +
+                "cast('0.32345678912345678912345678912345678912' as DECIMAL(38, 38)) as i2,\n" +
+                "cast('-1234567891234567891234567891234567.89' as DECIMAL(36, 2)) as i3,\n" +
+                "cast('0' as DECIMAL(36, 3)) as i4,\n" +
+                "cast('15.02' as DECIMAL(4, 2)) as i5)";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5")
+        .baselineValues("999999999999999999999999999.92345678912",
+            "0.32345678912345678912345678912345678912",
+            "-1234567891234567891234567891234567.89", "0.000", "15.02")
+        .go();
+  }
+
+  @Test
+  public void testDecimalToChar() throws Exception {
+    String query =
+        "select\n" +
+            "to_char(i1, '#.#') as s1,\n" +
+            "to_char(i2, '#.#') as s2,\n" +
+            "to_char(i3, '#.#') as s3,\n" +
+            "to_char(i4, '#.#') as s4,\n" +
+            "to_char(i5, '#.#') as s5\n" +
+            "from (" +
+            "select\n" +
+            "cast('999999999999999999999999999.92345678912' as DECIMAL(38, 11)) as i1,\n" +
+            "cast('0.32345678912345678912345678912345678912' as DECIMAL(38, 38)) as i2,\n" +
+            "cast('-1234567891234567891234567891234567.89' as DECIMAL(36, 2)) as i3,\n" +
+            "cast('0' as DECIMAL(36, 3)) as i4,\n" +
+            "cast('15.02' as DECIMAL(4, 2)) as i5)";
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s1", "s2", "s3", "s4", "s5")
+        .baselineValues("999999999999999999999999999.9", "0.3",
+            "-1234567891234567891234567891234567.9", "0", "15")
+        .go();
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDecimal.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDecimal.java
index 1bdcda5c3b..ff1edb932e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDecimal.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDecimal.java
@@ -20,10 +20,10 @@
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.math.BigInteger;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.DrillFileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
@@ -32,6 +32,7 @@
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.util.DecimalUtility;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.categories.SlowTest;
 import org.junit.Test;
@@ -41,8 +42,7 @@
 import org.junit.experimental.categories.Category;
 
 @Category({SlowTest.class})
-public class TestDecimal extends PopUnitTestBase{
-    DrillConfig c = DrillConfig.create();
+public class TestDecimal extends PopUnitTestBase {
 
     @Test
     public void testSimpleDecimal() throws Exception {
@@ -314,4 +314,24 @@ public void testSimpleDecimalMathFunc() throws Exception {
     }
   }
 
+  @Test
+  public void testGetMaxBytesSizeForPrecision() {
+    for (int i = 0; i < 10_000; i++) {
+      assertEquals(String.format("Bytes size does not match for precision %s", i),
+          getMaxBytesSizeForPrecisionFromBigInteger(i),
+          DecimalUtility.getMaxBytesSizeForPrecision(i));
+    }
+  }
+
+  private static int getMaxBytesSizeForPrecisionFromBigInteger(int precision) {
+    if (precision == 0) {
+      return 0;
+    }
+
+    StringBuilder sb = new StringBuilder("-");
+    for (int i = 0; i < precision; i++) {
+      sb.append(9);
+    }
+    return new BigInteger(sb.toString()).toByteArray().length;
+  }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestNestedLoopJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestNestedLoopJoin.java
index 78f76d363a..fb41df5e10 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestNestedLoopJoin.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestNestedLoopJoin.java
@@ -409,4 +409,30 @@ public void testNLJoinCorrectnessRightMultipleBatches() throws Exception {
       setSessionOption(ExecConstants.SLICE_TARGET, 100000);
     }
   }
+
+  @Test
+  public void testNlJoinWithStringsInCondition() throws Exception {
+    try {
+      test(DISABLE_NLJ_SCALAR);
+      test(DISABLE_JOIN_OPTIMIZATION);
+
+      final String query =
+          "select v.employee_id\n" +
+          "from cp.`employee.json` v\n" +
+          "left outer join cp.`employee.json` s\n" +
+          "on v.employee_id <> s.employee_id\n" +
+          "and (v.position_id <= '-1' or s.department_id > '5000')\n" +
+          "order by v.employee_id limit 1";
+
+      testBuilder()
+          .sqlQuery(query)
+          .unOrdered()
+          .baselineColumns("employee_id")
+          .baselineValues(1L)
+          .go();
+    } finally {
+      resetJoinOptions();
+      test(RESET_JOIN_OPTIMIZATION);
+    }
+  }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
index c359e69b6d..c1dc643e81 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
@@ -19,11 +19,17 @@
 
 import static org.apache.drill.exec.store.parquet.ParquetRecordWriter.DRILL_VERSION_PROPERTY;
 import static org.apache.drill.test.TestBuilder.convertToLocalTimestamp;
+import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER;
 import static org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
 import static org.junit.Assert.assertEquals;
 
 import java.io.File;
 import java.io.FileWriter;
+import java.io.IOException;
 import java.math.BigDecimal;
 import java.nio.file.Paths;
 import java.sql.Date;
@@ -35,6 +41,9 @@
 import java.util.Map;
 
 import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.util.Pair;
+import org.apache.drill.exec.util.JsonStringArrayList;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.ParquetTest;
 import org.apache.drill.categories.SlowTest;
@@ -49,6 +58,9 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
 import org.joda.time.DateTime;
 import org.joda.time.Period;
 import org.junit.AfterClass;
@@ -72,7 +84,7 @@ public static void setupTestFiles() {
     dirTestWatcher.copyResourceToRoot(Paths.get("parquet", "int96_dict_change"));
   }
 
-  static FileSystem fs;
+  private static FileSystem fs;
 
   // Map storing a convenient name as well as the cast type necessary
   // to produce it casting from a varchar
@@ -85,11 +97,11 @@ public static void setupTestFiles() {
   static {
     allTypes.put("int",                "int");
     allTypes.put("bigint",             "bigint");
-    // TODO(DRILL-3367)
-//    allTypes.put("decimal(9, 4)",      "decimal9");
-//    allTypes.put("decimal(18,9)",      "decimal18");
-//    allTypes.put("decimal(28, 14)",    "decimal28sparse");
-//    allTypes.put("decimal(38, 19)",    "decimal38sparse");
+    allTypes.put("decimal(9, 4)",      "decimal9");
+    allTypes.put("decimal(18,9)",      "decimal18");
+    allTypes.put("decimal(28, 14)",    "decimal28sparse");
+    allTypes.put("decimal(38, 19)",    "decimal38sparse");
+    allTypes.put("decimal(38, 15)",    "vardecimal");
     allTypes.put("date",               "date");
     allTypes.put("timestamp",          "timestamp");
     allTypes.put("float",              "float4");
@@ -126,8 +138,8 @@ public static void initFs() throws Exception {
   }
 
   @AfterClass
-  public static void disableDecimalDataType() throws Exception {
-    alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, false);
+  public static void disableDecimalDataType() {
+    resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
   }
 
   @Test
@@ -146,7 +158,7 @@ public void testSimple() throws Exception {
 
   @Test
   public void testLargeFooter() throws Exception {
-    StringBuffer sb = new StringBuffer();
+    StringBuilder sb = new StringBuilder();
     // create a JSON document with a lot of columns
     sb.append("{");
     final int numCols = 1000;
@@ -155,13 +167,13 @@ public void testLargeFooter() throws Exception {
     for (int i = 0 ; i < numCols - 1; i++) {
       sb.append(String.format("\"col_%d\" : 100,", i));
       colNames[i] = "col_" + i;
-      values[i] = 100l;
+      values[i] = 100L;
     }
     // add one column without a comma after it
     sb.append(String.format("\"col_%d\" : 100", numCols - 1));
     sb.append("}");
     colNames[numCols - 1] = "col_" + (numCols - 1);
-    values[numCols - 1] = 100l;
+    values[numCols - 1] = 100L;
 
     String path = "test";
     File pathDir = dirTestWatcher.makeRootSubDir(Paths.get(path));
@@ -1002,5 +1014,295 @@ public void testInt96TimeStampValueWidth() throws Exception {
       resetSessionOption(ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP);
     }
   }
-}
 
+  @Test
+  public void testWriteDecimalIntBigIntFixedLen() throws Exception {
+    String tableName = "decimalIntBigIntFixedLen";
+    try {
+      alterSession(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS, FIXED_LEN_BYTE_ARRAY.name());
+      test(
+          "create table dfs.tmp.%s as\n" +
+              "select cast('123456.789' as decimal(9, 3)) as decInt,\n" +
+                     "cast('123456.789123456789' as decimal(18, 12)) as decBigInt,\n" +
+                     "cast('123456.789123456789' as decimal(19, 12)) as fixedLen", tableName);
+      checkTableTypes(tableName,
+          ImmutableList.of(
+              Pair.of("decInt", INT32),
+              Pair.of("decBigInt", INT64),
+              Pair.of("fixedLen", FIXED_LEN_BYTE_ARRAY)),
+          true);
+      testBuilder()
+          .sqlQuery("select * from dfs.tmp.%s", tableName)
+          .unOrdered()
+          .baselineColumns("decInt", "decBigInt", "fixedLen")
+          .baselineValues(new BigDecimal("123456.789"),
+              new BigDecimal("123456.789123456789"),
+              new BigDecimal("123456.789123456789"))
+          .go();
+    } finally {
+      resetSessionOption(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS);
+      resetSessionOption(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS);
+      test("drop table if exists dfs.tmp.%s", tableName);
+    }
+  }
+
+  @Test
+  public void testWriteDecimalIntBigIntBinary() throws Exception {
+    String tableName = "decimalIntBigIntBinary";
+    try {
+      alterSession(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS, true);
+      alterSession(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS, BINARY.name());
+      test(
+          "create table dfs.tmp.%s as\n" +
+              "select cast('123456.789' as decimal(9, 3)) as decInt,\n" +
+                     "cast('123456.789123456789' as decimal(18, 12)) as decBigInt,\n" +
+                     "cast('123456.789123456789' as decimal(19, 12)) as binCol", tableName);
+      checkTableTypes(tableName,
+          ImmutableList.of(
+              Pair.of("decInt", INT32),
+              Pair.of("decBigInt", INT64),
+              Pair.of("binCol", BINARY)),
+          true);
+      testBuilder()
+          .sqlQuery("select * from dfs.tmp.%s", tableName)
+          .unOrdered()
+          .baselineColumns("decInt", "decBigInt", "binCol")
+          .baselineValues(new BigDecimal("123456.789"),
+            new BigDecimal("123456.789123456789"),
+            new BigDecimal("123456.789123456789"))
+          .go();
+    } finally {
+      resetSessionOption(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS);
+      resetSessionOption(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS);
+      test("drop table if exists dfs.tmp.%s", tableName);
+    }
+  }
+
+  @Test
+  public void testWriteDecimalFixedLenOnly() throws Exception {
+    String tableName = "decimalFixedLenOnly";
+    try {
+      alterSession(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS, false);
+      alterSession(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS, FIXED_LEN_BYTE_ARRAY.name());
+      test(
+          "create table dfs.tmp.%s as\n" +
+              "select cast('123456.789' as decimal(9, 3)) as decInt,\n" +
+                     "cast('123456.789123456789' as decimal(18, 12)) as decBigInt,\n" +
+                     "cast('123456.789123456789' as decimal(19, 12)) as fixedLen", tableName);
+      checkTableTypes(tableName,
+          ImmutableList.of(
+              Pair.of("decInt", FIXED_LEN_BYTE_ARRAY),
+              Pair.of("decBigInt", FIXED_LEN_BYTE_ARRAY),
+              Pair.of("fixedLen", FIXED_LEN_BYTE_ARRAY)),
+          true);
+      testBuilder()
+          .sqlQuery("select * from dfs.tmp.%s", tableName)
+          .unOrdered()
+          .baselineColumns("decInt", "decBigInt", "fixedLen")
+          .baselineValues(new BigDecimal("123456.789"),
+            new BigDecimal("123456.789123456789"),
+            new BigDecimal("123456.789123456789"))
+          .go();
+    } finally {
+      resetSessionOption(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS);
+      resetSessionOption(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS);
+      test("drop table if exists dfs.tmp.%s", tableName);
+    }
+  }
+
+  @Test
+  public void testWriteDecimalBinaryOnly() throws Exception {
+    String tableName = "decimalBinaryOnly";
+    try {
+      alterSession(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS, false);
+      alterSession(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS, BINARY.name());
+      test(
+        "create table dfs.tmp.%s as\n" +
+            "select cast('123456.789' as decimal(9, 3)) as decInt,\n" +
+                   "cast('123456.789123456789' as decimal(18, 12)) as decBigInt,\n" +
+                   "cast('123456.789123456789' as decimal(19, 12)) as binCol", tableName);
+      checkTableTypes(tableName,
+        ImmutableList.of(
+          Pair.of("decInt", BINARY),
+          Pair.of("decBigInt", BINARY),
+          Pair.of("binCol", BINARY)),
+        true);
+      testBuilder()
+          .sqlQuery("select * from dfs.tmp.%s", tableName)
+          .unOrdered()
+          .baselineColumns("decInt", "decBigInt", "binCol")
+          .baselineValues(new BigDecimal("123456.789"),
+            new BigDecimal("123456.789123456789"),
+            new BigDecimal("123456.789123456789"))
+          .go();
+    } finally {
+      resetSessionOption(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS);
+      resetSessionOption(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS);
+      test("drop table if exists dfs.tmp.%s", tableName);
+    }
+  }
+
+  @Test
+  public void testWriteDecimalIntBigIntRepeated() throws Exception {
+    String tableName = "decimalIntBigIntRepeated";
+
+    JsonStringArrayList<BigDecimal> ints = new JsonStringArrayList<>();
+    ints.add(new BigDecimal("999999.999"));
+    ints.add(new BigDecimal("-999999.999"));
+    ints.add(new BigDecimal("0.000"));
+
+    JsonStringArrayList<BigDecimal> longs = new JsonStringArrayList<>();
+    longs.add(new BigDecimal("999999999.999999999"));
+    longs.add(new BigDecimal("-999999999.999999999"));
+    longs.add(new BigDecimal("0.000000000"));
+
+    JsonStringArrayList<BigDecimal> fixedLen = new JsonStringArrayList<>();
+    fixedLen.add(new BigDecimal("999999999999.999999"));
+    fixedLen.add(new BigDecimal("-999999999999.999999"));
+    fixedLen.add(new BigDecimal("0.000000"));
+
+    try {
+      alterSession(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS, true);
+      alterSession(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS, FIXED_LEN_BYTE_ARRAY.name());
+      test(
+        "create table dfs.tmp.%s as\n" +
+            "select * from cp.`parquet/repeatedIntLondFixedLenBinaryDecimal.parquet`", tableName);
+      checkTableTypes(tableName,
+          ImmutableList.of(
+              Pair.of("decimal_int32", INT32),
+              Pair.of("decimal_int64", INT64),
+              Pair.of("decimal_fixedLen", INT64),
+              Pair.of("decimal_binary", INT64)),
+          true);
+      testBuilder()
+          .sqlQuery("select * from dfs.tmp.%s", tableName)
+          .unOrdered()
+          .baselineColumns("decimal_int32", "decimal_int64", "decimal_fixedLen", "decimal_binary")
+          .baselineValues(ints, longs, fixedLen, fixedLen)
+          .go();
+    } finally {
+      resetSessionOption(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS);
+      resetSessionOption(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS);
+      test("drop table if exists dfs.tmp.%s", tableName);
+    }
+  }
+
+  @Test
+  public void testWriteDecimalFixedLenRepeated() throws Exception {
+    String tableName = "decimalFixedLenRepeated";
+
+    JsonStringArrayList<BigDecimal> ints = new JsonStringArrayList<>();
+    ints.add(new BigDecimal("999999.999"));
+    ints.add(new BigDecimal("-999999.999"));
+    ints.add(new BigDecimal("0.000"));
+
+    JsonStringArrayList<BigDecimal> longs = new JsonStringArrayList<>();
+    longs.add(new BigDecimal("999999999.999999999"));
+    longs.add(new BigDecimal("-999999999.999999999"));
+    longs.add(new BigDecimal("0.000000000"));
+
+    JsonStringArrayList<BigDecimal> fixedLen = new JsonStringArrayList<>();
+    fixedLen.add(new BigDecimal("999999999999.999999"));
+    fixedLen.add(new BigDecimal("-999999999999.999999"));
+    fixedLen.add(new BigDecimal("0.000000"));
+
+    try {
+      alterSession(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS, false);
+      alterSession(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS, FIXED_LEN_BYTE_ARRAY.name());
+      test(
+          "create table dfs.tmp.%s as\n" +
+              "select * from cp.`parquet/repeatedIntLondFixedLenBinaryDecimal.parquet`", tableName);
+      checkTableTypes(tableName,
+          ImmutableList.of(
+              Pair.of("decimal_int32", FIXED_LEN_BYTE_ARRAY),
+              Pair.of("decimal_int64", FIXED_LEN_BYTE_ARRAY),
+              Pair.of("decimal_fixedLen", FIXED_LEN_BYTE_ARRAY),
+              Pair.of("decimal_binary", FIXED_LEN_BYTE_ARRAY)),
+          true);
+      testBuilder()
+          .sqlQuery("select * from dfs.tmp.%s", tableName)
+          .unOrdered()
+          .baselineColumns("decimal_int32", "decimal_int64", "decimal_fixedLen", "decimal_binary")
+          .baselineValues(ints, longs, fixedLen, fixedLen)
+          .go();
+    } finally {
+      resetSessionOption(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS);
+      resetSessionOption(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS);
+      test("drop table if exists dfs.tmp.%s", tableName);
+    }
+  }
+
+  @Test
+  public void testWriteDecimalBinaryRepeated() throws Exception {
+    String tableName = "decimalBinaryRepeated";
+
+    JsonStringArrayList<BigDecimal> ints = new JsonStringArrayList<>();
+    ints.add(new BigDecimal("999999.999"));
+    ints.add(new BigDecimal("-999999.999"));
+    ints.add(new BigDecimal("0.000"));
+
+    JsonStringArrayList<BigDecimal> longs = new JsonStringArrayList<>();
+    longs.add(new BigDecimal("999999999.999999999"));
+    longs.add(new BigDecimal("-999999999.999999999"));
+    longs.add(new BigDecimal("0.000000000"));
+
+    JsonStringArrayList<BigDecimal> fixedLen = new JsonStringArrayList<>();
+    fixedLen.add(new BigDecimal("999999999999.999999"));
+    fixedLen.add(new BigDecimal("-999999999999.999999"));
+    fixedLen.add(new BigDecimal("0.000000"));
+    try {
+      alterSession(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS, false);
+      alterSession(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS, BINARY.name());
+      test(
+        "create table dfs.tmp.%s as\n" +
+            "select * from cp.`parquet/repeatedIntLondFixedLenBinaryDecimal.parquet`", tableName);
+      checkTableTypes(tableName,
+          ImmutableList.of(
+              Pair.of("decimal_int32", BINARY),
+              Pair.of("decimal_int64", BINARY),
+              Pair.of("decimal_fixedLen", BINARY),
+              Pair.of("decimal_binary", BINARY)),
+          true);
+      testBuilder()
+          .sqlQuery("select * from dfs.tmp.%s", tableName)
+          .unOrdered()
+          .baselineColumns("decimal_int32", "decimal_int64", "decimal_fixedLen", "decimal_binary")
+          .baselineValues(ints, longs, fixedLen, fixedLen)
+          .go();
+    } finally {
+      resetSessionOption(ExecConstants.PARQUET_WRITER_USE_PRIMITIVE_TYPES_FOR_DECIMALS);
+      resetSessionOption(ExecConstants.PARQUET_WRITER_LOGICAL_TYPE_FOR_DECIMALS);
+      test("drop table if exists dfs.tmp.%s", tableName);
+    }
+  }
+
+  /**
+   * Checks that specified parquet table contains specified columns with specified types.
+   *
+   * @param tableName      name of the table that should be checked.
+   * @param columnsToCheck pair of column name and column type that should be checked in the table.
+   * @param isDecimalType  is should be specified columns annotated ad DECIMAL.
+   * @throws IOException If table file was not found.
+   */
+  private void checkTableTypes(String tableName,
+      List<Pair<String, PrimitiveType.PrimitiveTypeName>> columnsToCheck,
+      boolean isDecimalType) throws IOException {
+    MessageType schema = ParquetFileReader.readFooter(
+        new Configuration(),
+        new Path(Paths.get(dirTestWatcher.getDfsTestTmpDir().getPath(), tableName, "0_0_0.parquet").toUri().getPath()),
+        NO_FILTER).getFileMetaData().getSchema();
+
+    for (Pair<String, PrimitiveType.PrimitiveTypeName> nameType : columnsToCheck) {
+      assertEquals(
+          String.format("Table %s does not contain column %s with type %s",
+              tableName, nameType.getKey(), nameType.getValue()),
+          nameType.getValue(),
+          schema.getType(nameType.getKey()).asPrimitiveType().getPrimitiveTypeName());
+
+      assertEquals(
+        String.format("Table %s %s column %s with DECIMAL type", tableName,
+            isDecimalType ? "does not contain" : "contains unexpected", nameType.getKey()),
+        isDecimalType, schema.getType(nameType.getKey()).getOriginalType() == OriginalType.DECIMAL);
+    }
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
index 6de16bfd17..3138eede53 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestCopier.java
@@ -55,7 +55,7 @@
 public class TestCopier extends SubOperatorTest {
 
   @Test
-  public void testEmptyInput() throws Exception {
+  public void testEmptyInput() {
     BatchSchema schema = SortTestUtilities.nonNullSchema();
     List<BatchGroup> batches = new ArrayList<>();
     Sort popConfig = SortTestUtilities.makeCopierConfig(Ordering.ORDER_ASC, Ordering.NULLS_UNSPECIFIED);
@@ -323,8 +323,7 @@ public static void testAllTypes(OperatorFixture fixture) throws Exception {
     runTypeTest(fixture, MinorType.BIGINT);
     runTypeTest(fixture, MinorType.FLOAT4);
     runTypeTest(fixture, MinorType.FLOAT8);
-    runTypeTest(fixture, MinorType.DECIMAL9);
-    runTypeTest(fixture, MinorType.DECIMAL18);
+    runTypeTest(fixture, MinorType.VARDECIMAL);
     runTypeTest(fixture, MinorType.VARCHAR);
     runTypeTest(fixture, MinorType.VARBINARY);
     runTypeTest(fixture, MinorType.DATE);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
index 49667ab81f..88a7b99099 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/managed/TestSorter.java
@@ -493,10 +493,11 @@ public void testNumericTypes() throws Exception {
 //      tester1.test(MinorType.UINT4); DRILL-5329
       tester.test(MinorType.BIGINT);
 //      tester1.test(MinorType.UINT8); DRILL-5329
-      tester.test(MinorType.FLOAT4);
-      tester.test(MinorType.FLOAT8);
-      tester.test(MinorType.DECIMAL9);
-      tester.test(MinorType.DECIMAL18);
+    tester.test(MinorType.FLOAT4);
+    tester.test(MinorType.FLOAT8);
+    tester.test(MinorType.VARDECIMAL);
+//      tester1.test(MinorType.DECIMAL9);
+//      tester1.test(MinorType.DECIMAL18);
 //      tester1.test(MinorType.DECIMAL28SPARSE); DRILL-5329
 //      tester1.test(MinorType.DECIMAL38SPARSE); DRILL-5329
 //    tester1.test(MinorType.DECIMAL28DENSE); No writer
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/TestCaseNullableTypes.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/TestCaseNullableTypes.java
index f18821bd0e..b8c7275582 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/TestCaseNullableTypes.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/TestCaseNullableTypes.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.planner.logical;
 
+import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.SqlTest;
 import org.joda.time.DateTime;
@@ -24,6 +25,8 @@
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import java.math.BigDecimal;
+
 /**
  * DRILL-4906
  * Tests for handling nullable types in CASE function
@@ -143,4 +146,19 @@ public void testCaseNullableTimestamp() throws Exception {
       .baselineValues(date)
       .go();
   }
+
+  @Test
+  public void testCaseNullableTypesDecimal() throws Exception {
+    try {
+      alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+      testBuilder()
+          .sqlQuery("select (case when (false) then null else cast(0.1 as decimal(2, 1)) end) res1")
+          .ordered()
+          .baselineColumns("res1")
+          .baselineValues(new BigDecimal("0.1"))
+          .go();
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    }
+  }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java
index 930c41b10f..466a3f1237 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java
@@ -18,17 +18,28 @@
 package org.apache.drill.exec.store.avro;
 
 import com.google.common.collect.Lists;
+import mockit.integration.junit4.JMockit;
+import org.apache.avro.specific.TestRecordWithLogicalTypes;
 import org.apache.commons.io.FileUtils;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.ExecTest;
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.work.ExecErrorConstants;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.test.TestBuilder;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.exec.util.JsonStringHashMap;
+import org.joda.time.DateTime;
+import org.joda.time.LocalDate;
+import org.joda.time.LocalTime;
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.runner.RunWith;
 
 import java.io.File;
+import java.math.BigDecimal;
 import java.util.List;
 import java.util.Map;
 
@@ -46,11 +57,13 @@
 import static org.apache.drill.exec.store.avro.AvroTestUtil.generateUnionNestedSchema_withNullValues;
 import static org.apache.drill.exec.store.avro.AvroTestUtil.generateUnionSchema_WithNonNullValues;
 import static org.apache.drill.exec.store.avro.AvroTestUtil.generateUnionSchema_WithNullValues;
+import static org.apache.drill.exec.store.avro.AvroTestUtil.write;
 import static org.apache.drill.test.TestBuilder.listOf;
 
 /**
  * Unit tests for Avro record reader.
  */
+@RunWith(JMockit.class)
 public class AvroFormatTest extends BaseTestQuery {
 
   // XXX
@@ -293,6 +306,65 @@ public void testSelectAllWithPartitionColumn() throws Exception {
     }
   }
 
+  @Test
+  public void testAvroTableWithLogicalTypesDecimal() throws Exception {
+    ExecTest.mockUtcDateTimeZone();
+    DateTime date = DateUtility.getDateTimeFormatter().parseDateTime("2018-02-03");
+    DateTime time = DateUtility.getTimeFormatter().parseDateTime("19:25:03");
+    DateTime timestamp = DateUtility.getDateTimeFormatter().parseDateTime("2018-02-03 19:25:03");
+    BigDecimal bigDecimal = new BigDecimal("123.45");
+
+    TestRecordWithLogicalTypes record = new TestRecordWithLogicalTypes(
+        true,
+        34,
+        35L,
+        3.14F,
+        3019.34,
+        "abc",
+        timestamp.toLocalDate(),
+        timestamp.toLocalTime(),
+        timestamp,
+        bigDecimal
+    );
+
+    File data = write(TestRecordWithLogicalTypes.getClassSchema(), record);
+
+    final String query = "select * from dfs.`%s`";
+
+    testBuilder()
+        .sqlQuery(query, data.getName())
+        .unOrdered()
+        .baselineColumns("b", "i32", "i64", "f32", "f64", "s", "d", "t", "ts", "dec")
+        .baselineValues(true, 34, 35L, 3.14F, 3019.34, "abc", date, time, timestamp, bigDecimal)
+        .go();
+  }
+
+  @Test
+  public void testAvroWithDisabledDecimalType() throws Exception {
+    TestRecordWithLogicalTypes record = new TestRecordWithLogicalTypes(
+        true,
+        34,
+        35L,
+        3.14F,
+        3019.34,
+        "abc",
+        LocalDate.now(),
+        LocalTime.now(),
+        DateTime.now(),
+        new BigDecimal("123.45")
+    );
+
+    File data = write(TestRecordWithLogicalTypes.getClassSchema(), record);
+    final String query = String.format("select * from dfs.`%s`", data.getName());
+
+    try {
+      alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, false);
+      errorMsgTestHelper(query, ExecErrorConstants.DECIMAL_DISABLE_ERR_MSG);
+    } finally {
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    }
+  }
+
   @Test
   public void testSimpleArraySchema_NoNullValues() throws Exception {
     final String file = generateSimpleArraySchema_NoNullValues().getFileName();
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroTestUtil.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroTestUtil.java
index 11b3d3902b..c85cdf6783 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroTestUtil.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroTestUtil.java
@@ -38,6 +38,9 @@
 import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.generic.GenericRecord;
 
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.specific.SpecificData;
+import org.apache.avro.specific.SpecificRecord;
 import org.apache.drill.exec.util.JsonStringArrayList;
 import org.apache.drill.exec.util.JsonStringHashMap;
 import org.apache.drill.exec.util.Text;
@@ -716,4 +719,27 @@ public static AvroTestRecordWriter generateStringAndUtf8Data() throws Exception
 
     return record;
   }
+
+  /**
+   * Creates Avro table with specified schema and specified data
+   * @param schema table schema
+   * @param data table data
+   * @param <D> record type
+   * @return file with newly created Avro table.
+   * @throws IOException if an error is appeared during creation or filling the file.
+   */
+  public static <D extends SpecificRecord> File write(Schema schema, D... data) throws IOException {
+    File file = File.createTempFile("avro-primitive-test", ".avro", BaseTestQuery.dirTestWatcher.getRootDir());
+
+    DatumWriter writer = SpecificData.get().createDatumWriter(schema);
+
+    try (DataFileWriter<D> fileWriter = new DataFileWriter<>(writer)) {
+      fileWriter.create(schema, file);
+      for (D datum : data) {
+        fileWriter.append(datum);
+      }
+    }
+
+    return file;
+  }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFixedlenDecimal.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFixedlenDecimal.java
index e6d7c10efa..678930c7dd 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFixedlenDecimal.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFixedlenDecimal.java
@@ -20,16 +20,23 @@
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({UnlikelyTest.class})
 public class TestFixedlenDecimal extends BaseTestQuery {
-  // enable decimal data type
+
+
   @BeforeClass
-  public static void enableDecimalDataType() throws Exception {
-    test(String.format("alter session set `%s` = true", PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY));
+  public static void enableDecimalDataType() {
+    setSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+  }
+
+  @AfterClass
+  public static void disableDecimalDataType() {
+    resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
   }
 
   private static final String DATAFILE = "cp.`parquet/fixedlenDecimal.parquet`";
@@ -41,8 +48,7 @@ public void testNullCount() throws Exception {
         .unOrdered()
         .baselineColumns("c")
         .baselineValues(1L)
-        .build()
-        .run();
+        .go();
   }
 
   @Test
@@ -52,8 +58,7 @@ public void testNotNullCount() throws Exception {
         .unOrdered()
         .baselineColumns("c")
         .baselineValues(106L)
-        .build()
-        .run();
+        .go();
   }
 
   @Test
@@ -63,8 +68,7 @@ public void testSimpleQueryWithCast() throws Exception {
         .unOrdered()
         .baselineColumns("c")
         .baselineValues(80L)
-        .build()
-        .run();
+        .go();
   }
 
   @Test
@@ -74,7 +78,6 @@ public void testSimpleQueryDrill4704Fix() throws Exception {
         .unOrdered()
         .baselineColumns("c")
         .baselineValues(80L)
-        .build()
-        .run();
+        .go();
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java
index 8252834d2f..a7653b8410 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java
@@ -17,12 +17,14 @@
  */
 package org.apache.drill.exec.store.parquet;
 
+import org.apache.drill.exec.util.JsonStringArrayList;
 import org.apache.drill.test.BaseTestQuery;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 import org.joda.time.Period;
 import org.junit.Test;
 
+import java.math.BigDecimal;
 import java.util.Arrays;
 
 import static org.apache.drill.test.TestBuilder.mapOf;
@@ -202,10 +204,10 @@ public void notxistsField() throws Exception {
 
   @Test //DRILL-5971
   public void testComplexLogicalIntTypes() throws Exception {
-    String query = String.format("select t.complextype as complextype,  " +
+    String query = "select t.complextype as complextype,  " +
             "t.uint_64 as uint_64, t.uint_32 as uint_32, t.uint_16 as uint_16, t.uint_8 as uint_8,  " +
             "t.int_64 as int_64, t.int_32 as int_32, t.int_16 as int_16, t.int_8 as int_8  " +
-            "from cp.`store/parquet/complex/logical_int_complex.parquet` t" );
+            "from cp.`store/parquet/complex/logical_int_complex.parquet` t";
     String[] columns = {"complextype", "uint_64", "uint_32", "uint_16", "uint_8", "int_64", "int_32", "int_16", "int_8" };
     testBuilder()
         .sqlQuery(query)
@@ -225,7 +227,7 @@ public void testComplexLogicalIntTypes2() throws Exception {
     byte[] bytesOnes = new byte[12];
     byte[] bytesZeros = new byte[12];
     Arrays.fill(bytesOnes, (byte) 1);
-    String query = String.format(
+    String query =
         " select " +
         " t.rowKey as rowKey, " +
         " t.StringTypes._UTF8 as _UTF8, " +
@@ -247,8 +249,7 @@ public void testComplexLogicalIntTypes2() throws Exception {
         " t.NumericTypes.Int96._INT96_RAW as _INT96_RAW " +
         " from " +
         " cp.`store/parquet/complex/parquet_logical_types_complex.parquet` t " +
-        " order by t.rowKey "
-    );
+        " order by t.rowKey ";
     String[] columns = {
         "rowKey " ,
         "_UTF8" ,
@@ -300,7 +301,7 @@ public void testComplexLogicalIntTypes3() throws Exception {
     byte[] bytesOnes = new byte[12];
     byte[] bytesZeros = new byte[12];
     Arrays.fill(bytesOnes, (byte) 1);
-    String query = String.format(
+    String query =
         " select " +
             " t.rowKey as rowKey, " +
             " t.StringTypes._UTF8 as _UTF8, " +
@@ -322,8 +323,7 @@ public void testComplexLogicalIntTypes3() throws Exception {
             " t.NumericTypes.Int96._INT96_RAW as _INT96_RAW " +
             " from " +
             " cp.`store/parquet/complex/parquet_logical_types_complex_nullable.parquet` t " +
-            " order by t.rowKey "
-    );
+            " order by t.rowKey ";
     String[] columns = {
         "rowKey " ,
         "_UTF8" ,
@@ -370,4 +370,29 @@ public void testComplexLogicalIntTypes3() throws Exception {
         .build().run();
   }
 
+  @Test
+  public void testReadRepeatedDecimals() throws Exception {
+
+    JsonStringArrayList<BigDecimal> ints = new JsonStringArrayList<>();
+    ints.add(new BigDecimal("999999.999"));
+    ints.add(new BigDecimal("-999999.999"));
+    ints.add(new BigDecimal("0.000"));
+
+    JsonStringArrayList<BigDecimal> longs = new JsonStringArrayList<>();
+    longs.add(new BigDecimal("999999999.999999999"));
+    longs.add(new BigDecimal("-999999999.999999999"));
+    longs.add(new BigDecimal("0.000000000"));
+
+    JsonStringArrayList<BigDecimal> fixedLen = new JsonStringArrayList<>();
+    fixedLen.add(new BigDecimal("999999999999.999999"));
+    fixedLen.add(new BigDecimal("-999999999999.999999"));
+    fixedLen.add(new BigDecimal("0.000000"));
+
+    testBuilder()
+        .sqlQuery("select * from cp.`parquet/repeatedIntLondFixedLenBinaryDecimal.parquet`")
+        .unOrdered()
+        .baselineColumns("decimal_int32", "decimal_int64", "decimal_fixedLen", "decimal_binary")
+        .baselineValues(ints, longs, fixedLen, fixedLen)
+        .go();
+  }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestVarlenDecimal.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestVarlenDecimal.java
new file mode 100644
index 0000000000..eb5c900c3c
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestVarlenDecimal.java
@@ -0,0 +1,142 @@
+/*
+ * 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.exec.store.parquet;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.test.BaseTestQuery;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.hamcrest.CoreMatchers;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.AfterClass;
+import org.junit.Test;
+
+import java.math.BigDecimal;
+import java.nio.file.Paths;
+
+public class TestVarlenDecimal extends BaseTestQuery {
+
+  @BeforeClass
+  public static void enableDecimalDataType() {
+    setSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+  }
+
+  @AfterClass
+  public static void disableDecimalDataType() {
+    resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+  }
+
+  private static final String DATAFILE = "cp.`parquet/varlenDecimal.parquet`";
+
+  @Test
+  public void testNullCount() throws Exception {
+    String query = "select count(*) as c from %s where department_id is null";
+    testBuilder()
+        .sqlQuery(query, DATAFILE)
+        .unOrdered()
+        .baselineColumns("c")
+        .baselineValues(1L)
+        .go();
+  }
+
+  @Test
+  public void testNotNullCount() throws Exception {
+    String query = "select count(*) as c from %s where department_id is not null";
+    testBuilder()
+        .sqlQuery(query, DATAFILE)
+        .unOrdered()
+        .baselineColumns("c")
+        .baselineValues(106L)
+        .go();
+  }
+
+  @Test
+  public void testSimpleQuery() throws Exception {
+    String query = "select cast(department_id as bigint) as c from %s where cast(employee_id as decimal) = 170";
+
+    testBuilder()
+        .sqlQuery(query, DATAFILE)
+        .unOrdered()
+        .baselineColumns("c")
+        .baselineValues(80L)
+        .go();
+  }
+
+  @Test
+  public void testWriteReadJson() throws Exception {
+    // Drill stores decimal values in JSON files correctly, but it can read only double values, but not big decimal.
+    // See JsonToken class.
+
+    String tableName = "jsonWithDecimals";
+    try {
+      alterSession(ExecConstants.OUTPUT_FORMAT_OPTION, "json");
+
+      String bigDecimalValue = "987654321987654321987654321.987654321";
+
+      test(
+          "create table dfs.tmp.%s as\n" +
+              "select cast('%s' as decimal(36, 9)) dec36", tableName, bigDecimalValue);
+
+      String json = FileUtils.readFileToString(
+          Paths.get(dirTestWatcher.getDfsTestTmpDir().getPath(), tableName, "0_0_0.json").toFile());
+
+      Assert.assertThat(json, CoreMatchers.containsString(bigDecimalValue));
+
+      // checks that decimal value may be read as a double value
+      testBuilder()
+          .sqlQuery("select dec36 from dfs.tmp.%s", tableName)
+          .unOrdered()
+          .baselineColumns("dec36")
+          .baselineValues(new BigDecimal(bigDecimalValue).doubleValue())
+          .go();
+    } finally {
+      resetSessionOption(ExecConstants.OUTPUT_FORMAT_OPTION);
+      test("drop table if exists dfs.tmp.%s", tableName);
+    }
+  }
+
+  @Test
+  public void testWriteReadCsv() throws Exception {
+    String tableName = "csvWithDecimals";
+    try {
+      alterSession(ExecConstants.OUTPUT_FORMAT_OPTION, "csvh");
+
+      String bigDecimalValue = "987654321987654321987654321.987654321";
+
+      test(
+          "create table dfs.tmp.%s as\n" +
+              "select cast('%s' as decimal(36, 9)) dec36", tableName, bigDecimalValue);
+
+      String csv = FileUtils.readFileToString(
+          Paths.get(dirTestWatcher.getDfsTestTmpDir().getPath(), tableName, "0_0_0.csvh").toFile());
+
+      Assert.assertThat(csv, CoreMatchers.containsString(bigDecimalValue));
+
+      testBuilder()
+          .sqlQuery("select cast(dec36 as decimal(36, 9)) as dec36 from dfs.tmp.%s", tableName)
+          .ordered()
+          .baselineColumns("dec36")
+          .baselineValues(new BigDecimal(bigDecimalValue))
+          .go();
+    } finally {
+      resetSessionOption(ExecConstants.OUTPUT_FORMAT_OPTION);
+      test("drop table if exists dfs.tmp.%s", tableName);
+    }
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/columnreaders/TestColumnReaderFactory.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/columnreaders/TestColumnReaderFactory.java
index 87adc2654d..a6a3bed96e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/columnreaders/TestColumnReaderFactory.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/columnreaders/TestColumnReaderFactory.java
@@ -24,15 +24,15 @@
 import org.junit.Test;
 
 public class TestColumnReaderFactory extends BaseTestQuery {
-  // enable decimal data type
+
   @BeforeClass
-  public static void enableDecimalDataType() throws Exception {
-    test(String.format("alter session set `%s` = true", PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY));
+  public static void enableDecimalDataType() {
+    alterSession(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
   }
 
   @AfterClass
-  public static void disableDecimalDataType() throws Exception {
-    test(String.format("alter session set `%s` = false", PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY));
+  public static void disableDecimalDataType() {
+    resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
   }
 
   /**
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
index 798764794b..696b6b2bd4 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
@@ -433,6 +433,7 @@ public BaselineQueryTestBuilder physicalPlanBaselineQueryFromFile(String baselin
   private String getDecimalPrecisionScaleInfo(TypeProtos.MajorType type) {
     String precision = "";
     switch(type.getMinorType()) {
+      case VARDECIMAL:
       case DECIMAL18:
       case DECIMAL28SPARSE:
       case DECIMAL38SPARSE:
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFillEmpties.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFillEmpties.java
index f4c00ca1fe..63a98b3221 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFillEmpties.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/test/TestFillEmpties.java
@@ -114,6 +114,7 @@ private void testFillEmpties(DataMode mode) {
       case DECIMAL28SPARSE:
       case DECIMAL9:
       case DECIMAL38SPARSE:
+      case VARDECIMAL:
         doFillEmptiesTest(type, mode, 9, 2);
         break;
       default:
diff --git a/exec/java-exec/src/test/resources/decimal/cast_decimal_float.json b/exec/java-exec/src/test/resources/decimal/cast_decimal_float.json
index 3fc94a0d26..af155045be 100644
--- a/exec/java-exec/src/test/resources/decimal/cast_decimal_float.json
+++ b/exec/java-exec/src/test/resources/decimal/cast_decimal_float.json
@@ -19,8 +19,8 @@
       "pop" : "project",
       "@id" : 2,
       "exprs" : [
-        { "ref" : "DEC9_COL", "expr": "(cast(DEC9 as decimal9(9, 4)))" },
-        { "ref" : "DEC38_COL", "expr": "(cast(DEC18 as decimal38sparse(38, 4)))" }
+        { "ref" : "DEC9_COL", "expr": "(cast(DEC9 as vardecimal(9, 4)))" },
+        { "ref" : "DEC38_COL", "expr": "(cast(DEC18 as vardecimal(38, 4)))" }
       ],
 
       "child" : 1
diff --git a/exec/java-exec/src/test/resources/decimal/cast_decimal_int.json b/exec/java-exec/src/test/resources/decimal/cast_decimal_int.json
index 2cca86541c..27df35aea2 100644
--- a/exec/java-exec/src/test/resources/decimal/cast_decimal_int.json
+++ b/exec/java-exec/src/test/resources/decimal/cast_decimal_int.json
@@ -19,8 +19,8 @@
       "pop" : "project",
       "@id" : 2,
       "exprs" : [
-        { "ref" : "DEC9_COL", "expr": "(cast(cast(INT_COL as int) as decimal9(9, 0)))" },
-        { "ref" : "DEC38_COL", "expr": "(cast(BIGINT_COL as decimal38sparse(38, 0)))" }
+        { "ref" : "DEC9_COL", "expr": "(cast(cast(INT_COL as int) as vardecimal(9, 0)))" },
+        { "ref" : "DEC38_COL", "expr": "(cast(BIGINT_COL as vardecimal(38, 0)))" }
       ],
 
       "child" : 1
diff --git a/exec/java-exec/src/test/resources/decimal/cast_decimal_vardecimal.json b/exec/java-exec/src/test/resources/decimal/cast_decimal_vardecimal.json
new file mode 100644
index 0000000000..92c8237b42
--- /dev/null
+++ b/exec/java-exec/src/test/resources/decimal/cast_decimal_vardecimal.json
@@ -0,0 +1,56 @@
+{
+  "head" : {
+    "version" : 1,
+    "generator" : {
+      "type" : "org.apache.drill.exec.planner.logical.DrillImplementor",
+      "info" : ""
+    },
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "resultMode" : "EXEC"
+  },
+  "graph":[
+    {
+      "pop" : "fs-scan",
+      "@id" : 4,
+      "format": {"type": "json"},
+      "storage":{"type": "file", "connection": "classpath:///"},
+      "files":["/input_complex_decimal.json"]
+    }, {
+      "pop" : "project",
+      "@id" : 3,
+      "exprs" : [
+        { "ref" : "DEC28", "expr": "(cast(B as vardecimal(38, 20)))" },
+        { "ref" : "DEC38", "expr": "(cast(A as vardecimal(28, 16)))" },
+        { "ref" : "DEC18", "expr": "(cast(B as vardecimal(18, 9)))" },
+        { "ref" : "DEC9", "expr": "(cast(A as vardecimal(9, 0)))" }
+      ],
+
+      "child" : 4
+    }, {
+      "pop" : "project",
+      "@id" : 2,
+      "exprs" : [
+        {"ref": "DEC28_COL", "expr" : "cast(DEC28 as decimal38sparse(38, 20))"},
+        {"ref": "DEC38_COL", "expr" : "cast(DEC38 as decimal28sparse(28, 16))"},
+        {"ref": "DEC18_COL", "expr" : "cast(DEC18 as decimal18(18, 0))"},
+        {"ref": "DEC9_COL", "expr" : "cast(DEC9 as decimal9(9, 0))"}
+      ],
+
+      "child" : 3
+    }, {
+      "pop" : "project",
+      "@id" : 1,
+      "exprs" : [
+        {"ref": "DEC28_COL", "expr" : "cast(DEC28_COL as decimal38sparse(38, 16))"},
+        {"ref": "DEC38_COL", "expr" : "cast(DEC38_COL as decimal28sparse(28, 16))"},
+        {"ref": "DEC18_COL", "expr" : "cast(DEC18_COL as decimal18(18, 0))"},
+        {"ref": "DEC9_COL", "expr" : "cast(DEC9_COL as decimal9(9, 0))"}
+      ],
+
+      "child" : 2
+    }, {
+      "pop" : "screen",
+      "@id" : 0,
+      "child" : 1
+    } ]
+}
diff --git a/exec/java-exec/src/test/resources/decimal/cast_float_decimal.json b/exec/java-exec/src/test/resources/decimal/cast_float_decimal.json
index b9e05d36c5..e8f318a246 100644
--- a/exec/java-exec/src/test/resources/decimal/cast_float_decimal.json
+++ b/exec/java-exec/src/test/resources/decimal/cast_float_decimal.json
@@ -32,9 +32,9 @@
     "@id" : 4,
     "exprs" : [ {
       "ref" : "DECIMAL_9",
-      "expr" : " cast(F4 as decimal9(9, 4))  "
+      "expr" : " cast(F4 as vardecimal(9, 4))  "
     },
-    {"ref": "DECIMAL38", "expr" : "cast(F8 as decimal38sparse(38, 4))"}
+    {"ref": "DECIMAL38", "expr" : "cast(F8 as vardecimal(38, 4))"}
     ],
 
     "child" : 2
diff --git a/exec/java-exec/src/test/resources/decimal/cast_int_decimal.json b/exec/java-exec/src/test/resources/decimal/cast_int_decimal.json
index 885ff534b0..2f0999289a 100644
--- a/exec/java-exec/src/test/resources/decimal/cast_int_decimal.json
+++ b/exec/java-exec/src/test/resources/decimal/cast_int_decimal.json
@@ -29,10 +29,10 @@
       "pop" : "project",
       "@id" : 4,
       "exprs" : [
-        {"ref": "DEC9_INT", "expr" : "cast(INT_COL as decimal9(9, 0))"},
-        {"ref": "DEC38_INT", "expr" : "cast(INT_COL as decimal38sparse(38, 0))"},
-        {"ref": "DEC9_BIGINT", "expr" : "cast(BIGINT_COL as decimal9(9, 0))"},
-        {"ref": "DEC38_BIGINT", "expr" : "cast(BIGINT_COL as decimal38sparse(38, 0))"}
+        {"ref": "DEC9_INT", "expr" : "cast(INT_COL as vardecimal(9, 0))"},
+        {"ref": "DEC38_INT", "expr" : "cast(INT_COL as vardecimal(38, 0))"},
+        {"ref": "DEC9_BIGINT", "expr" : "cast(BIGINT_COL as vardecimal(9, 0))"},
+        {"ref": "DEC38_BIGINT", "expr" : "cast(BIGINT_COL as vardecimal(38, 0))"}
       ],
 
       "child" : 2
diff --git a/exec/java-exec/src/test/resources/decimal/cast_simple_decimal.json b/exec/java-exec/src/test/resources/decimal/cast_simple_decimal.json
index 0291724659..53ff360c80 100644
--- a/exec/java-exec/src/test/resources/decimal/cast_simple_decimal.json
+++ b/exec/java-exec/src/test/resources/decimal/cast_simple_decimal.json
@@ -20,9 +20,9 @@
     "@id" : 2,
     "exprs" : [ {
       "ref" : "DECIMAL_9",
-      "expr" : " (cast(DEC9 as decimal9(9, 4))) "
+      "expr" : " (cast(DEC9 as vardecimal(9, 4))) "
     },
-    { "ref" : "DECIMAL_18", "expr": "(cast(DEC18 as decimal18(18, 9)))" }
+    { "ref" : "DECIMAL_18", "expr": "(cast(DEC18 as vardecimal(18, 9)))" }
     ],
 
     "child" : 1
diff --git a/exec/java-exec/src/test/resources/decimal/cast_vardecimal_decimal.json b/exec/java-exec/src/test/resources/decimal/cast_vardecimal_decimal.json
new file mode 100644
index 0000000000..4a4c5c90f6
--- /dev/null
+++ b/exec/java-exec/src/test/resources/decimal/cast_vardecimal_decimal.json
@@ -0,0 +1,47 @@
+{
+  "head" : {
+    "version" : 1,
+    "generator" : {
+      "type" : "org.apache.drill.exec.planner.logical.DrillImplementor",
+      "info" : ""
+    },
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "resultMode" : "EXEC"
+  },
+  graph:[
+    {
+      @id:1,
+      pop:"fs-scan",
+      format: {type: "json"},
+      storage:{type: "file", connection: "classpath:///"},
+      files:["/input_complex_decimal.json"]
+    }, {
+      "pop" : "project",
+      "@id" : 2,
+      "exprs" : [
+        { "ref" : "DEC28", "expr": "(cast(B as vardecimal(38, 16)))" },
+        { "ref" : "DEC38", "expr": "(cast(A as vardecimal(28, 16)))" },
+        { "ref" : "DEC18", "expr": "(cast(B as vardecimal(18, 9)))" },
+        { "ref" : "DEC9", "expr": "(cast(A as vardecimal(9, 0)))" }
+      ],
+
+      "child" : 1
+    },
+    {
+      "pop" : "project",
+      "@id" : 4,
+      "exprs" : [
+        {"ref": "DEC28_COL", "expr" : "cast(DEC28 as decimal38sparse(38, 16))"},
+        {"ref": "DEC38_COL", "expr" : "cast(DEC38 as decimal28sparse(28, 16))"},
+        {"ref": "DEC18_COL", "expr" : "cast(DEC18 as decimal18(18, 0))"},
+        {"ref": "DEC9_COL", "expr" : "cast(DEC9 as decimal9(9, 0))"}
+      ],
+
+      "child" : 2
+    },
+    {
+      "pop" : "screen",
+      "@id" : 5,
+      "child" : 4
+    } ]
+}
diff --git a/exec/java-exec/src/test/resources/decimal/simple_decimal_arithmetic.json b/exec/java-exec/src/test/resources/decimal/simple_decimal_arithmetic.json
index ca31d88659..c774230f8b 100644
--- a/exec/java-exec/src/test/resources/decimal/simple_decimal_arithmetic.json
+++ b/exec/java-exec/src/test/resources/decimal/simple_decimal_arithmetic.json
@@ -19,8 +19,8 @@
           "pop" : "project",
           "@id" : 2,
           "exprs" : [
-          { "ref" : "DECIMAL18_1", "expr": "(cast(DEC18 as decimal18(18, 1)))" },
-          { "ref" : "DECIMAL18_2", "expr": "(cast(DEC9 as decimal18(18, 1)))" }
+          { "ref" : "DECIMAL18_1", "expr": "(cast(DEC18 as vardecimal(18, 1)))" },
+          { "ref" : "DECIMAL18_2", "expr": "(cast(DEC9 as vardecimal(18, 1)))" }
           ],
 
           "child" : 1
diff --git a/exec/java-exec/src/test/resources/decimal/simple_decimal_math.json b/exec/java-exec/src/test/resources/decimal/simple_decimal_math.json
index 42fd78eafa..f7eb7f2c5e 100644
--- a/exec/java-exec/src/test/resources/decimal/simple_decimal_math.json
+++ b/exec/java-exec/src/test/resources/decimal/simple_decimal_math.json
@@ -19,7 +19,7 @@
     "pop" : "project",
     "@id" : 2,
     "exprs" : [
-    { "ref" : "DECIMAL_18", "expr": "log((cast(DEC18 as decimal18(18, 9))))" }
+    { "ref" : "DECIMAL_18", "expr": "log((cast(DEC18 as vardecimal(18, 9))))" }
     ],
 
     "child" : 1
diff --git a/exec/java-exec/src/test/resources/decimal/test_decimal_complex.json b/exec/java-exec/src/test/resources/decimal/test_decimal_complex.json
index b909411abc..f341421b26 100644
--- a/exec/java-exec/src/test/resources/decimal/test_decimal_complex.json
+++ b/exec/java-exec/src/test/resources/decimal/test_decimal_complex.json
@@ -20,9 +20,9 @@
         "@id" : 2,
         "exprs" : [ {
           "ref" : "DE",
-          "expr" : " (cast(B as decimal38sparse(28, 9))) "
+          "expr" : " (cast(B as vardecimal(28, 9))) "
         },
-        {"ref" : "DE1", "expr": " cast(A as decimal18(15, 6))" }
+        {"ref" : "DE1", "expr": " cast(A as vardecimal(15, 6))" }
         ],
 
         "child" : 1
diff --git a/exec/java-exec/src/test/resources/decimal/test_decimal_sort_complex.json b/exec/java-exec/src/test/resources/decimal/test_decimal_sort_complex.json
index 70a7a10040..8ff0253f2d 100644
--- a/exec/java-exec/src/test/resources/decimal/test_decimal_sort_complex.json
+++ b/exec/java-exec/src/test/resources/decimal/test_decimal_sort_complex.json
@@ -20,7 +20,7 @@
         "@id" : 2,
         "exprs" : [ {
           "ref" : "DEC",
-          "expr" : " (cast(B as decimal38sparse(38, 12)))"
+          "expr" : " (cast(B as vardecimal(38, 12)))"
         }
         ],
         "child" : 1
diff --git a/exec/java-exec/src/test/resources/parquet/repeatedIntLondFixedLenBinaryDecimal.parquet b/exec/java-exec/src/test/resources/parquet/repeatedIntLondFixedLenBinaryDecimal.parquet
new file mode 100644
index 0000000000..62795b2920
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquet/repeatedIntLondFixedLenBinaryDecimal.parquet differ
diff --git a/exec/java-exec/src/test/resources/parquet/varlenDecimal.parquet b/exec/java-exec/src/test/resources/parquet/varlenDecimal.parquet
new file mode 100755
index 0000000000..c531ef13eb
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquet/varlenDecimal.parquet differ
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java
index b08c0f5649..f3d1e38698 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java
@@ -19,6 +19,7 @@
 
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
+import java.math.BigDecimal;
 import java.sql.DatabaseMetaData;
 import java.sql.SQLException;
 import java.sql.Time;
@@ -132,6 +133,7 @@ private MetaResultSet s(String s) {
         .put(Double.TYPE,  of(Types.DOUBLE, "DOUBLE"))
         .put(Double.class, of(Types.DOUBLE, "DOUBLE"))
         .put(String.class, of(Types.VARCHAR, "CHARACTER VARYING"))
+        .put(BigDecimal.class, of(Types.DECIMAL, "DECIMAL"))
         .put(java.sql.Date.class, of(Types.DATE, "DATE"))
         .put(Time.class, of(Types.TIME, "TIME"))
         .put(Timestamp.class, of(Types.TIMESTAMP, "TIMESTAMP"))
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/TypeConvertingSqlAccessor.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/TypeConvertingSqlAccessor.java
index 204d7d862b..f3d24fc9f2 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/TypeConvertingSqlAccessor.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/TypeConvertingSqlAccessor.java
@@ -301,6 +301,11 @@ public byte getByte( int rowOffset ) throws InvalidAccessException {
         result = getByteValueOrThrow( innerAccessor.getDouble( rowOffset ),
                                       "Java double / SQL DOUBLE PRECISION" );
         break;
+      case VARDECIMAL:
+        result = getByteValueOrThrow(
+            innerAccessor.getBigDecimal(rowOffset).byteValue(),
+            "Java BigDecimal / SQL DECIMAL PRECISION");
+        break;
 
       // 3. Not-yet-converted and unconvertible types:
       default:
@@ -365,6 +370,11 @@ public short getShort( int rowOffset ) throws InvalidAccessException {
         result = getShortValueOrThrow( innerAccessor.getDouble( rowOffset ),
                                        "Java double / SQL DOUBLE PRECISION" );
         break;
+      case VARDECIMAL:
+        result = getShortValueOrThrow(
+            innerAccessor.getBigDecimal(rowOffset).shortValue(),
+            "Java BigDecimal / SQL DECIMAL PRECISION");
+        break;
 
       // 3. Not-yet-converted and unconvertible types:
       default:
@@ -428,6 +438,11 @@ public int getInt( int rowOffset ) throws InvalidAccessException {
         result = getIntValueOrThrow( innerAccessor.getDouble( rowOffset ),
                                      "Java double / SQL DOUBLE PRECISION" );
         break;
+      case VARDECIMAL:
+        result = getIntValueOrThrow(
+            innerAccessor.getBigDecimal(rowOffset).intValue(),
+            "Java BigDecimal / SQL DECIMAL PRECISION");
+        break;
 
       // 3. Not-yet-converted and unconvertible types:
       default:
@@ -481,6 +496,11 @@ public long getLong( int rowOffset ) throws InvalidAccessException {
         result = getLongValueOrThrow( innerAccessor.getDouble( rowOffset ),
                                       "Java double / SQL DOUBLE PRECISION" );
         break;
+      case VARDECIMAL:
+        result = getLongValueOrThrow(
+            innerAccessor.getBigDecimal(rowOffset).longValue(),
+            "Java BigDecimal / SQL DECIMAL PRECISION");
+        break;
 
       // 3. Not-yet-converted and unconvertible types:
       default:
@@ -490,6 +510,14 @@ public long getLong( int rowOffset ) throws InvalidAccessException {
     return result;
   }
 
+  private static float getFloatValueOrThrow(double value, String typeLabel)
+      throws SQLConversionOverflowException {
+    if (Float.MIN_VALUE <= value && value <= Float.MAX_VALUE) {
+      return (float) value;
+    } else {
+      throw newOverflowException("getFloat(...)", typeLabel, value);
+    }
+  }
 
   ////////////////////////////////////////
   // - getFloat:
@@ -516,14 +544,13 @@ public float getFloat( int rowOffset ) throws InvalidAccessException {
         result = innerAccessor.getLong( rowOffset );
         break;
       case FLOAT8:
-        final double value = innerAccessor.getDouble( rowOffset );
-        if ( Float.MIN_VALUE <= value && value <= Float.MAX_VALUE ) {
-          result = (float) value;
-        } else {
-          throw newOverflowException( "getFloat(...)",
-                                      "Java double / SQL DOUBLE PRECISION",
-                                      value );
-        }
+        result = getFloatValueOrThrow(innerAccessor.getDouble(rowOffset),
+            "Java double / SQL DOUBLE PRECISION");
+        break;
+      case VARDECIMAL:
+        result = getFloatValueOrThrow(
+            innerAccessor.getBigDecimal(rowOffset).floatValue(),
+            "Java BigDecimal / SQL DECIMAL PRECISION");
         break;
 
       // 3. Not-yet-converted and unconvertible types:
@@ -561,6 +588,9 @@ public double getDouble( int rowOffset ) throws InvalidAccessException {
       case FLOAT4:
         result = innerAccessor.getFloat( rowOffset );
         break;
+      case VARDECIMAL:
+        result = innerAccessor.getBigDecimal(rowOffset).doubleValue();
+        break;
 
       // 3. Not-yet-converted and unconvertible types:
       default:
@@ -588,6 +618,7 @@ public BigDecimal getBigDecimal( int rowOffset ) throws InvalidAccessException {
       case DECIMAL18:
       case DECIMAL28SPARSE:
       case DECIMAL38SPARSE:
+      case VARDECIMAL:
         result = innerAccessor.getBigDecimal( rowOffset );
         break;
 
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/ResultSetGetMethodConversionsTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/ResultSetGetMethodConversionsTest.java
index 0af4c25770..e47c826561 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/ResultSetGetMethodConversionsTest.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/ResultSetGetMethodConversionsTest.java
@@ -92,8 +92,8 @@ public static void setUpConnectionAndMetadataToCheck() throws SQLException {
         //+ "\n  CAST(  5.5 AS REAL             ) AS `C_REAL_5.5`, "
         + "\n  CAST(  6.6 AS DOUBLE PRECISION ) AS `C_DOUBLE_PREC._6.6`, "
         + "\n  CAST(  7.7 AS FLOAT            ) AS `C_FLOAT_7.7`, "
-        + "\n  CAST( 10.10 AS DECIMAL         ) AS `C_DECIMAL_10.10`, "
-        + "\n  CAST( 10.5  AS DECIMAL         ) AS `C_DECIMAL_10.5`, "
+        + "\n  CAST( 10.10 AS DECIMAL(4,2)         ) AS `C_DECIMAL_10.10`, "
+        + "\n  CAST( 10.5  AS DECIMAL (3,1)        ) AS `C_DECIMAL_10.5`, "
         + "\n  CAST( 11.11 AS DECIMAL(9,2)    ) AS `C_DECIMAL(9,2)_11.11`, "
         + "\n  CAST( 12.12 AS DECIMAL(18,2)   ) AS `C_DECIMAL(18,2)_12.12`, "
         + "\n  CAST( 13.13 AS DECIMAL(28,2)   ) AS `C_DECIMAL(28,2)_13.13`, "
@@ -494,7 +494,6 @@ public void test_getBigDecimal_handles_DECIMAL_1() throws SQLException {
     assertThat( testDataRow.getBigDecimal( "C_DECIMAL_10.5" ), equalTo( new BigDecimal( "10.5") ) );
   }
 
-  @Ignore( "TODO(DRILL-3367): unignore when DECIMAL is no longer DOUBLE" )
   @Test
   public void test_getBigDecimal_handles_DECIMAL_2() throws SQLException {
     assertThat( testDataRow.getBigDecimal( "C_DECIMAL_10.10" ), equalTo( new BigDecimal( "10.10") ) );
@@ -566,7 +565,7 @@ public void test_getString_handles_FLOAT() throws SQLException {
 
   @Test
   public void test_getString_handles_DECIMAL() throws SQLException {
-    assertThat( testDataRow.getString( "C_DECIMAL_10.10" ), equalTo( "10.1" ) );
+    assertThat( testDataRow.getString( "C_DECIMAL_10.10" ), equalTo( "10.10" ) );
     assertThat( testDataRowWithNulls.getString( "C_DECIMAL_10.10" ), equalTo( null ) );
   }
 
@@ -724,14 +723,12 @@ public void test_getObject_handles_FLOAT() throws SQLException {
     assertThat( testDataRow.getObject( "C_FLOAT_7.7" ), equalTo( (Object) 7.7f ) );
   }
 
-  @Ignore( "TODO(DRILL-3367): unignore when DECIMAL is no longer DOUBLE" )
   @Test
   public void test_getObject_handles_DECIMAL_1() throws SQLException {
     assertThat( testDataRow.getObject( "C_DECIMAL_10.5" ),
                 equalTo( (Object) new BigDecimal( "10.5" ) ) );
   }
 
-  @Ignore( "TODO(DRILL-3367): unignore when DECIMAL is no longer DOUBLE" )
   @Test
   public void test_getObject_handles_DECIMAL_2() throws SQLException {
     assertThat( testDataRow.getObject( "C_DECIMAL_10.10" ),
diff --git a/exec/vector/src/main/codegen/data/ValueVectorTypes.tdd b/exec/vector/src/main/codegen/data/ValueVectorTypes.tdd
index 4d719b4f53..f6e1f1d09d 100644
--- a/exec/vector/src/main/codegen/data/ValueVectorTypes.tdd
+++ b/exec/vector/src/main/codegen/data/ValueVectorTypes.tdd
@@ -177,6 +177,17 @@
         { class: "Var16Char" , friendlyType: "String" }
       ]
     },
+    {
+      major: "VarLen",
+      width: 4,
+      javaType: "int",
+      boxedType: "DrillBuf",
+      minor: [
+        { class: "VarDecimal", friendlyType: "BigDecimal", fields: [{name: "start", type: "int"},
+            {name: "end", type: "int"}, {name: "buffer", type: "DrillBuf"},
+            {name: "scale", type: "int", include: false}, {name: "precision", type: "int", include: false}] }
+      ]
+    },
     {
       major: "Bit",
       width: 1,
diff --git a/exec/vector/src/main/codegen/templates/AbstractFieldReader.java b/exec/vector/src/main/codegen/templates/AbstractFieldReader.java
index f2e1eb0dd8..2ed5a3b6d1 100644
--- a/exec/vector/src/main/codegen/templates/AbstractFieldReader.java
+++ b/exec/vector/src/main/codegen/templates/AbstractFieldReader.java
@@ -29,10 +29,9 @@
  * This class is generated using freemarker and the ${.template_name} template.
  */
 @SuppressWarnings("unused")
-abstract class AbstractFieldReader extends AbstractBaseReader implements FieldReader{
+abstract class AbstractFieldReader extends AbstractBaseReader implements FieldReader {
 
-  AbstractFieldReader(){
-    super();
+  AbstractFieldReader() {
   }
 
   /**
@@ -49,78 +48,77 @@ public boolean isSet() {
   <#assign safeType=friendlyType />
   <#if safeType=="byte[]"><#assign safeType="ByteArray" /></#if>
 
-  public ${friendlyType} read${safeType}(int arrayIndex){
+  public ${friendlyType} read${safeType}(int arrayIndex) {
     fail("read${safeType}(int arrayIndex)");
     return null;
   }
 
-  public ${friendlyType} read${safeType}(){
+  public ${friendlyType} read${safeType}() {
     fail("read${safeType}()");
     return null;
   }
 
   </#list>
 
-  public void copyAsValue(MapWriter writer){
+  public void copyAsValue(MapWriter writer) {
     fail("CopyAsValue MapWriter");
   }
-  public void copyAsField(String name, MapWriter writer){
+  public void copyAsField(String name, MapWriter writer) {
     fail("CopyAsField MapWriter");
   }
 
-  public void copyAsField(String name, ListWriter writer){
+  public void copyAsField(String name, ListWriter writer) {
     fail("CopyAsFieldList");
   }
 
   <#list vv.types as type><#list type.minor as minor><#assign name = minor.class?cap_first />
   <#assign boxedType = (minor.boxedType!type.boxedType) />
 
-  public void read(${name}Holder holder){
+  public void read(${name}Holder holder) {
     fail("${name}");
   }
 
-  public void read(Nullable${name}Holder holder){
+  public void read(Nullable${name}Holder holder) {
     fail("${name}");
   }
 
-  public void read(int arrayIndex, ${name}Holder holder){
+  public void read(int arrayIndex, ${name}Holder holder) {
     fail("Repeated${name}");
   }
 
-  public void read(int arrayIndex, Nullable${name}Holder holder){
+  public void read(int arrayIndex, Nullable${name}Holder holder) {
     fail("Repeated${name}");
   }
 
-  public void copyAsValue(${name}Writer writer){
+  public void copyAsValue(${name}Writer writer) {
     fail("CopyAsValue${name}");
   }
-  public void copyAsField(String name, ${name}Writer writer){
+
+  <#if minor.class == "VarDecimal">
+  public void copyAsField(String name, ${name}Writer writer, int scale, int precision) {
+  <#else>
+  public void copyAsField(String name, ${name}Writer writer) {
+  </#if>
     fail("CopyAsField${name}");
   }
   </#list></#list>
 
-  public FieldReader reader(String name){
+  public FieldReader reader(String name) {
     fail("reader(String name)");
     return null;
   }
 
-  public FieldReader reader(){
+  public FieldReader reader() {
     fail("reader()");
     return null;
-
   }
 
-  public int size(){
+  public int size() {
     fail("size()");
     return -1;
   }
 
-  private void fail(String name){
+  private void fail(String name) {
     throw new IllegalArgumentException(String.format("You tried to read a [%s] type when you are using a field reader of type [%s].", name, this.getClass().getSimpleName()));
   }
-
-
 }
-
-
-
diff --git a/exec/vector/src/main/codegen/templates/AbstractFieldWriter.java b/exec/vector/src/main/codegen/templates/AbstractFieldWriter.java
index 608420cce8..dcf99d34b8 100644
--- a/exec/vector/src/main/codegen/templates/AbstractFieldWriter.java
+++ b/exec/vector/src/main/codegen/templates/AbstractFieldWriter.java
@@ -65,6 +65,12 @@ public void write(${name}Holder holder) {
     fail("${name}");
   }
 
+  <#if minor.class?contains("Decimal") >
+  public void write${minor.class}(BigDecimal value) {
+    fail("${name}");
+  }
+  </#if>
+
   </#list></#list>
 
   public void writeNull() {
@@ -111,11 +117,18 @@ public ListWriter list(String name) {
   <#if lowerName == "int" ><#assign lowerName = "integer" /></#if>
   <#assign upperName = minor.class?upper_case />
   <#assign capName = minor.class?cap_first />
-  <#if minor.class?starts_with("Decimal") >
+  <#if minor.class?contains("Decimal") >
+  @Override
   public ${capName}Writer ${lowerName}(String name, int scale, int precision) {
     fail("${capName}");
     return null;
   }
+
+  @Override
+  public ${capName}Writer ${lowerName}(int scale, int precision) {
+    fail("${capName}");
+    return null;
+  }
   </#if>
 
   @Override
diff --git a/exec/vector/src/main/codegen/templates/AbstractPromotableFieldWriter.java b/exec/vector/src/main/codegen/templates/AbstractPromotableFieldWriter.java
index df7be39bd6..96b45d329d 100644
--- a/exec/vector/src/main/codegen/templates/AbstractPromotableFieldWriter.java
+++ b/exec/vector/src/main/codegen/templates/AbstractPromotableFieldWriter.java
@@ -75,12 +75,20 @@ public void endList() {
 
   <#list vv.types as type><#list type.minor as minor><#assign name = minor.class?cap_first />
   <#assign fields = minor.fields!type.fields />
-  <#if !minor.class?starts_with("Decimal") >
+  <#if minor.class?contains("VarDecimal")>
+  @Override
+  public void write${minor.class}(BigDecimal value) {
+    getWriter(MinorType.${name?upper_case}).write${minor.class}(value);
+  }
+  </#if>
+
   @Override
   public void write(${name}Holder holder) {
     getWriter(MinorType.${name?upper_case}).write(holder);
   }
 
+  <#-- This condition was added to cover previous decimal functionality for new VarDecimal type -->
+  <#if !minor.class?contains("Decimal") || minor.class?contains("VarDecimal")>
   public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, </#if></#list>) {
     getWriter(MinorType.${name?upper_case}).write${minor.class}(<#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
   }
@@ -116,7 +124,19 @@ public ListWriter list(String name) {
   <#if lowerName == "int" ><#assign lowerName = "integer" /></#if>
   <#assign upperName = minor.class?upper_case />
   <#assign capName = minor.class?cap_first />
-  <#if !minor.class?starts_with("Decimal") >
+  <#if minor.class?contains("Decimal")>
+
+  @Override
+  public ${capName}Writer ${lowerName}(String name, int scale, int precision) {
+    return getWriter(MinorType.MAP).${lowerName}(name, scale, precision);
+  }
+
+  @Override
+  public ${capName}Writer ${lowerName}(int scale, int precision) {
+    return getWriter(MinorType.LIST).${lowerName}(scale, precision);
+  }
+
+  <#else>
 
   @Override
   public ${capName}Writer ${lowerName}(String name) {
diff --git a/exec/vector/src/main/codegen/templates/BaseWriter.java b/exec/vector/src/main/codegen/templates/BaseWriter.java
index c65a758090..e650609760 100644
--- a/exec/vector/src/main/codegen/templates/BaseWriter.java
+++ b/exec/vector/src/main/codegen/templates/BaseWriter.java
@@ -53,7 +53,7 @@
     <#if lowerName == "int" ><#assign lowerName = "integer" /></#if>
     <#assign upperName = minor.class?upper_case />
     <#assign capName = minor.class?cap_first />
-    <#if minor.class?starts_with("Decimal") >
+    <#if minor.class?contains("Decimal") >
     ${capName}Writer ${lowerName}(String name, int scale, int precision);
     </#if>
     ${capName}Writer ${lowerName}(String name);
@@ -78,6 +78,9 @@
     <#if lowerName == "int" ><#assign lowerName = "integer" /></#if>
     <#assign upperName = minor.class?upper_case />
     <#assign capName = minor.class?cap_first />
+    <#if minor.class?contains("Decimal") >
+    ${capName}Writer ${lowerName}(int scale, int precision);
+    </#if>
     ${capName}Writer ${lowerName}();
     </#list></#list>
   }
@@ -111,6 +114,8 @@
     UInt8Writer uInt8(String name);
     VarCharWriter varChar(String name);
     Var16CharWriter var16Char(String name);
+    VarDecimalWriter varDecimal(String name);
+    VarDecimalWriter varDecimal(String name, int scale, int precision);
     TinyIntWriter tinyInt(String name);
     SmallIntWriter smallInt(String name);
     IntWriter integer(String name);
diff --git a/exec/vector/src/main/codegen/templates/BasicTypeHelper.java b/exec/vector/src/main/codegen/templates/BasicTypeHelper.java
index 8160dd59ae..f818c236f6 100644
--- a/exec/vector/src/main/codegen/templates/BasicTypeHelper.java
+++ b/exec/vector/src/main/codegen/templates/BasicTypeHelper.java
@@ -395,7 +395,9 @@ public static boolean compareValues(ValueVector v1, int v1index, ValueVector v2,
     MajorType type1 = v1.getField().getType();
     MajorType type2 = v2.getField().getType();
 
-    if (type1.getMinorType() != type2.getMinorType()) {
+    if (type1.getMinorType() != type2.getMinorType()
+        || type1.getScale() != type1.getScale()
+        || type1.getPrecision() != type1.getPrecision()) {
       return false;
     }
 
@@ -539,11 +541,19 @@ public static MajorType getValueHolderType(ValueHolder holder) {
     }
     <#list vv.types as type>
     <#list type.minor as minor>
+    <#if minor.class.contains("Decimal")>
+      else if (holder instanceof ${minor.class}Holder) {
+        return  getType((${minor.class}Holder) holder);
+      } else if (holder instanceof Nullable${minor.class}Holder) {
+        return  getType((Nullable${minor.class}Holder) holder);
+      }
+    <#else>
       else if (holder instanceof ${minor.class}Holder) {
         return ((${minor.class}Holder) holder).TYPE;
       } else if (holder instanceof Nullable${minor.class}Holder) {
       return ((Nullable${minor.class}Holder) holder).TYPE;
       }
+    </#if>
     </#list>
     </#list>
     else if (holder instanceof UntypedNullHolder) {
@@ -553,4 +563,31 @@ else if (holder instanceof UntypedNullHolder) {
 
   }
 
+  <#list vv.types as type>
+  <#list type.minor as minor>
+  <#if minor.class.contains("Decimal")>
+  <#list ["Nullable", "", "Repeated"] as dataMode>
+  public static MajorType getType(${dataMode}${minor.class}Holder holder) {
+    return MajorType.newBuilder()
+        .setMinorType(MinorType.${minor.class?upper_case})
+      <#if dataMode == "Nullable">
+        .setMode(DataMode.OPTIONAL)
+        .setScale(holder.scale)
+        .setPrecision(holder.precision)
+      <#elseif dataMode == "Repeated">
+        .setMode(DataMode.REPEATED)
+        .setScale(holder.vector.getField().getScale())
+        .setPrecision(holder.vector.getField().getPrecision())
+      <#else>
+        .setMode(DataMode.REQUIRED)
+        .setScale(holder.scale)
+        .setPrecision(holder.precision)
+      </#if>
+        .build();
+  }
+  </#list>
+  </#if>
+  </#list>
+  </#list>
+
 }
diff --git a/exec/vector/src/main/codegen/templates/ColumnAccessors.java b/exec/vector/src/main/codegen/templates/ColumnAccessors.java
index d0a2ace6f1..accf1b28d3 100644
--- a/exec/vector/src/main/codegen/templates/ColumnAccessors.java
+++ b/exec/vector/src/main/codegen/templates/ColumnAccessors.java
@@ -28,6 +28,8 @@ public ValueType valueType() {
       return ValueType.INTEGER;
   <#elseif drillType == "VarChar" || drillType == "Var16Char">
       return ValueType.STRING;
+  <#elseif drillType == "VarDecimal">
+    return ValueType.DECIMAL;
   <#else>
       return ValueType.${label?upper_case};
   </#if>
@@ -65,6 +67,7 @@ public ValueType valueType() {
 package org.apache.drill.exec.vector.accessor;
 
 import java.math.BigDecimal;
+import java.math.BigInteger;
 
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.vector.DateUtilities;
@@ -112,9 +115,9 @@ public ValueType valueType() {
     <#if accessorType=="BigDecimal">
       <#assign label="Decimal">
     </#if>
-    <#assign varWidth = drillType == "VarChar" || drillType == "Var16Char" || drillType == "VarBinary" />
+    <#assign varWidth = drillType == "VarChar" || drillType == "Var16Char" || drillType == "VarBinary"  || drillType == "VarDecimal"/>
     <#assign decimal = drillType == "Decimal9" || drillType == "Decimal18" ||
-                       drillType == "Decimal28Sparse" || drillType == "Decimal38Sparse" />
+                       drillType == "Decimal28Sparse" || drillType == "Decimal38Sparse"  || drillType == "VarDecimal"/>
     <#if varWidth>
       <#assign accessorType = "byte[]">
       <#assign label = "Bytes">
@@ -135,18 +138,16 @@ public ValueType valueType() {
 
     <#if varWidth>
   public static class ${drillType}ColumnReader extends BaseVarWidthReader {
-   
+
     <#else>
   public static class ${drillType}ColumnReader extends BaseFixedWidthReader {
-    
+
     private static final int VALUE_WIDTH = ${drillType}Vector.VALUE_WIDTH;
 
-      <#if decimal>
-    private MajorType type;
-    
-      </#if>
     </#if>
     <#if decimal>
+    private MajorType type;
+
     @Override
     public void bindVector(ColumnMetadata schema, VectorAccessor va) {
       super.bindVector(schema, va);
@@ -190,12 +191,12 @@ public void bindVector(ColumnMetadata schema, VectorAccessor va) {
     <#elseif drillType == "IntervalDay">
       final int offset = ${getOffset};
       return DateUtilities.fromIntervalDay(
-          buf.getInt(offset), 
+          buf.getInt(offset),
           buf.getInt(offset + ${minor.millisecondsOffset}));
     <#elseif drillType == "Interval">
       final int offset = ${getOffset};
       return DateUtilities.fromInterval(
-          buf.getInt(offset), 
+          buf.getInt(offset),
           buf.getInt(offset + ${minor.daysOffset}),
           buf.getInt(offset + ${minor.millisecondsOffset}));
     <#elseif drillType == "Decimal28Sparse" || drillType == "Decimal38Sparse">
@@ -234,6 +235,14 @@ public String getString() {
     public String getString() {
       return new String(getBytes(${indexVar}), Charsets.UTF_16);
     }
+  <#elseif drillType == "VarDecimal">
+
+    @Override
+    public BigDecimal getDecimal() {
+      byte[] bytes = getBytes();
+      BigInteger unscaledValue = bytes.length == 0 ? BigInteger.ZERO : new BigInteger(bytes);
+      return new BigDecimal(unscaledValue, type.getScale());
+    }
   </#if>
   }
 
@@ -241,9 +250,9 @@ public String getString() {
   public static class ${drillType}ColumnWriter extends BaseVarWidthWriter {
       <#else>
   public static class ${drillType}ColumnWriter extends BaseFixedWidthWriter {
-    
+
     private static final int VALUE_WIDTH = ${drillType}Vector.VALUE_WIDTH;
-    
+
         <#if decimal>
     private MajorType type;
         </#if>
@@ -269,8 +278,8 @@ public String getString() {
       </#if>
       <@getType drillType label />
       <#if ! varWidth>
-
     </#if>
+
     @Override
     public final void set${label}(final ${accessorType} value${putArgs}) {
       <#-- Must compute the write offset first; can't be inline because the
@@ -337,6 +346,15 @@ public final void setString(String value) {
       final byte bytes[] = value.getBytes(Charsets.UTF_16);
       setBytes(bytes, bytes.length);
     }
+
+    <#elseif drillType = "VarDecimal">
+
+    @Override
+    public final void setDecimal(final BigDecimal bd) {
+      byte[] barr = bd.unscaledValue().toByteArray();
+      int len = barr.length;
+      setBytes(barr, len);
+    }
     </#if>
   }
 
@@ -355,7 +373,7 @@ public final void setString(String value) {
 import org.apache.drill.exec.vector.accessor.writer.BaseScalarWriter;
 
 public class ColumnAccessorUtils {
-  
+
   private ColumnAccessorUtils() { }
 
 <@build vv.types "Required" "Reader" />
diff --git a/exec/vector/src/main/codegen/templates/ComplexCopier.java b/exec/vector/src/main/codegen/templates/ComplexCopier.java
index 91e3ff6454..6d7c7e6076 100644
--- a/exec/vector/src/main/codegen/templates/ComplexCopier.java
+++ b/exec/vector/src/main/codegen/templates/ComplexCopier.java
@@ -97,9 +97,12 @@ private static FieldWriter getMapWriterForReader(FieldReader reader, MapWriter w
     <#list vv.types as type><#list type.minor as minor><#assign name = minor.class?cap_first />
     <#assign fields = minor.fields!type.fields />
     <#assign uncappedName = name?uncap_first/>
-    <#if !minor.class?starts_with("Decimal")>
+    <#if !minor.class?contains("Decimal")>
     case ${name?upper_case}:
       return (FieldWriter) writer.<#if name == "Int">integer<#else>${uncappedName}</#if>(name);
+    <#elseif minor.class?contains("VarDecimal")>
+    case ${name?upper_case}:
+      return (FieldWriter) writer.${uncappedName}(name, reader.getType().getScale(), reader.getType().getPrecision());
     </#if>
     </#list></#list>
     case MAP:
@@ -116,9 +119,12 @@ private static FieldWriter getListWriterForReader(FieldReader reader, ListWriter
     <#list vv.types as type><#list type.minor as minor><#assign name = minor.class?cap_first />
     <#assign fields = minor.fields!type.fields />
     <#assign uncappedName = name?uncap_first/>
-    <#if !minor.class?starts_with("Decimal")>
+    <#if !minor.class?contains("Decimal")>
+    case ${name?upper_case}:
+      return (FieldWriter) writer.<#if name == "Int">integer<#else>${uncappedName}</#if>();
+    <#elseif minor.class?contains("VarDecimal")>
     case ${name?upper_case}:
-    return (FieldWriter) writer.<#if name == "Int">integer<#else>${uncappedName}</#if>();
+      return (FieldWriter) writer.${uncappedName}(reader.getType().getScale(), reader.getType().getPrecision());
     </#if>
     </#list></#list>
     case MAP:
diff --git a/exec/vector/src/main/codegen/templates/ComplexReaders.java b/exec/vector/src/main/codegen/templates/ComplexReaders.java
index 1f0b92b021..d72c7c6969 100644
--- a/exec/vector/src/main/codegen/templates/ComplexReaders.java
+++ b/exec/vector/src/main/codegen/templates/ComplexReaders.java
@@ -53,20 +53,20 @@
   
   private final ${nullMode}${name}Vector vector;
   
-  public ${nullMode}${name}ReaderImpl(${nullMode}${name}Vector vector){
+  public ${nullMode}${name}ReaderImpl(${nullMode}${name}Vector vector) {
     super();
     this.vector = vector;
   }
 
-  public MajorType getType(){
+  public MajorType getType() {
     return vector.getField().getType();
   }
 
-  public MaterializedField getField(){
+  public MaterializedField getField() {
     return vector.getField();
   }
   
-  public boolean isSet(){
+  public boolean isSet() {
     <#if nullMode == "Nullable">
     return !vector.getAccessor().isNull(idx());
     <#else>
@@ -74,76 +74,85 @@ public boolean isSet(){
     </#if>
   }
 
-
-  
-  
   <#if mode == "Repeated">
 
-  public void copyAsValue(${minor.class?cap_first}Writer writer){
+  public void copyAsValue(${minor.class?cap_first}Writer writer) {
     Repeated${minor.class?cap_first}WriterImpl impl = (Repeated${minor.class?cap_first}WriterImpl) writer;
     impl.vector.copyFromSafe(idx(), impl.idx(), vector);
   }
-  
-  public void copyAsField(String name, MapWriter writer){
-    Repeated${minor.class?cap_first}WriterImpl impl = (Repeated${minor.class?cap_first}WriterImpl)  writer.list(name).${lowerName}();
+
+  <#if minor.class == "VarDecimal">
+  public void copyAsField(String name, MapWriter writer, int scale, int precision) {
+    Repeated${minor.class?cap_first}WriterImpl impl
+        = (Repeated${minor.class?cap_first}WriterImpl) writer.list(name).${lowerName}(scale, precision);
+  <#else>
+  public void copyAsField(String name, MapWriter writer) {
+    Repeated${minor.class?cap_first}WriterImpl impl = (Repeated${minor.class?cap_first}WriterImpl) writer.list(name).${lowerName}();
+  </#if>
     impl.vector.copyFromSafe(idx(), impl.idx(), vector);
   }
   
-  public int size(){
+  public int size() {
     return vector.getAccessor().getInnerValueCountAt(idx());
   }
   
-  public void read(int arrayIndex, ${minor.class?cap_first}Holder h){
+  public void read(int arrayIndex, ${minor.class?cap_first}Holder h) {
     vector.getAccessor().get(idx(), arrayIndex, h);
   }
-  public void read(int arrayIndex, Nullable${minor.class?cap_first}Holder h){
+
+  public void read(int arrayIndex, Nullable${minor.class?cap_first}Holder h) {
     vector.getAccessor().get(idx(), arrayIndex, h);
   }
   
-  public ${friendlyType} read${safeType}(int arrayIndex){
+  public ${friendlyType} read${safeType}(int arrayIndex) {
     return vector.getAccessor().getSingleObject(idx(), arrayIndex);
   }
 
   
-  public List<Object> readObject(){
+  public List<Object> readObject() {
     return (List<Object>) (Object) vector.getAccessor().getObject(idx());
   }
   
   <#else>
   
-  public void copyAsValue(${minor.class?cap_first}Writer writer){
+  public void copyAsValue(${minor.class?cap_first}Writer writer) {
     ${nullMode}${minor.class?cap_first}WriterImpl impl = (${nullMode}${minor.class?cap_first}WriterImpl) writer;
     impl.vector.copyFromSafe(idx(), impl.idx(), vector);
   }
-  
-  public void copyAsField(String name, MapWriter writer){
+
+  <#if minor.class == "VarDecimal">
+  public void copyAsField(String name, MapWriter writer, int scale, int precision) {
+    ${nullMode}${minor.class?cap_first}WriterImpl impl
+        = (${nullMode}${minor.class?cap_first}WriterImpl) writer.${lowerName}(name, scale, precision);
+<#else>
+  public void copyAsField(String name, MapWriter writer) {
     ${nullMode}${minor.class?cap_first}WriterImpl impl = (${nullMode}${minor.class?cap_first}WriterImpl) writer.${lowerName}(name);
+  </#if>
     impl.vector.copyFromSafe(idx(), impl.idx(), vector);
   }
 
   <#if nullMode != "Nullable">
-  public void read(${minor.class?cap_first}Holder h){
+  public void read(${minor.class?cap_first}Holder h) {
     vector.getAccessor().get(idx(), h);
   }
   </#if>
 
-  public void read(Nullable${minor.class?cap_first}Holder h){
+  public void read(Nullable${minor.class?cap_first}Holder h) {
     vector.getAccessor().get(idx(), h);
   }
   
-  public ${friendlyType} read${safeType}(){
+  public ${friendlyType} read${safeType}() {
     return vector.getAccessor().getObject(idx());
   }
   
-  public void copyValue(FieldWriter w){
+  public void copyValue(FieldWriter w) {
     
   }
   
-  public Object readObject(){
+  public Object readObject() {
     return vector.getAccessor().getObject(idx());
   }
 
-  
   </#if>
 }
 </#if>
@@ -155,7 +164,7 @@ public Object readObject(){
 
 <#include "/@includes/vv_imports.ftl" />
 @SuppressWarnings("unused")
-public interface ${name}Reader extends BaseReader{
+public interface ${name}Reader extends BaseReader {
   
   <#if mode == "Repeated">
   public int size();
@@ -171,8 +180,11 @@ public Object readObject(){
   </#if>  
   public boolean isSet();
   public void copyAsValue(${minor.class}Writer writer);
+  <#if minor.class == "VarDecimal">
+  public void copyAsField(String name, ${minor.class}Writer writer, int scale, int precision);
+  <#else>
   public void copyAsField(String name, ${minor.class}Writer writer);
-  
+  </#if>
 }
 
 
diff --git a/exec/vector/src/main/codegen/templates/ComplexWriters.java b/exec/vector/src/main/codegen/templates/ComplexWriters.java
index cfa049d12f..f46b7958d5 100644
--- a/exec/vector/src/main/codegen/templates/ComplexWriters.java
+++ b/exec/vector/src/main/codegen/templates/ComplexWriters.java
@@ -96,10 +96,10 @@ public void write(Nullable${minor.class?cap_first}Holder h) {
     vector.getMutator().setValueCount(idx()+1);
   }
 
-  <#if !(minor.class == "Decimal9" || minor.class == "Decimal18" || minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense")>
+  <#if !minor.class?contains("Decimal") || minor.class?contains("VarDecimal")>
   public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, </#if></#list>) {
-    mutator.addSafe(idx(), <#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
-    vector.getMutator().setValueCount(idx()+1);
+    mutator.addSafe(idx()<#list fields as field><#if field.include!true>, ${field.name}</#if></#list>);
+    vector.getMutator().setValueCount(idx() + 1);
   }
   </#if>
 
@@ -121,17 +121,23 @@ public void write(Nullable${minor.class}Holder h) {
     vector.getMutator().setValueCount(idx()+1);
   }
 
-  <#if !(minor.class == "Decimal9" || minor.class == "Decimal18" || minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense")>
-  public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, </#if></#list>) {
-    mutator.setSafe(idx(), <#if mode == "Nullable">1, </#if><#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
-    vector.getMutator().setValueCount(idx()+1);
-  }
-
   <#if mode == "Nullable">
-
   public void writeNull() {
     mutator.setNull(idx());
-    vector.getMutator().setValueCount(idx()+1);
+    vector.getMutator().setValueCount(idx() + 1);
+  }
+  </#if>
+
+  <#if !(minor.class == "Decimal9" || minor.class == "Decimal18")>
+  public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, </#if></#list>) {
+    mutator.setSafe(idx()<#if mode == "Nullable">, 1</#if><#list fields as field><#if field.include!true>, ${field.name}</#if></#list>);
+    vector.getMutator().setValueCount(idx() + 1);
+  }
+
+  <#if minor.class?contains("VarDecimal")>
+  public void write${minor.class}(BigDecimal value) {
+    mutator.setSafe(idx(), value);
+    vector.getMutator().setValueCount(idx() + 1);
   }
   </#if>
   </#if>
@@ -148,9 +154,13 @@ public void writeNull() {
 public interface ${eName}Writer extends BaseWriter {
   public void write(${minor.class}Holder h);
 
-  <#if !(minor.class == "Decimal9" || minor.class == "Decimal18" || minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense")>
+  <#if !(minor.class == "Decimal9" || minor.class == "Decimal18")>
   public void write${minor.class}(<#list fields as field>${field.type} ${field.name}<#if field_has_next>, </#if></#list>);
   </#if>
+
+  <#if minor.class?contains("VarDecimal")>
+  public void write${minor.class}(BigDecimal value);
+  </#if>
 }
 
 </#list>
diff --git a/exec/vector/src/main/codegen/templates/HolderReaderImpl.java b/exec/vector/src/main/codegen/templates/HolderReaderImpl.java
index 0eca723ce8..4b7be4fd73 100644
--- a/exec/vector/src/main/codegen/templates/HolderReaderImpl.java
+++ b/exec/vector/src/main/codegen/templates/HolderReaderImpl.java
@@ -42,6 +42,7 @@
 import java.math.BigInteger;
 
 import org.apache.drill.exec.expr.holders.*;
+import org.apache.drill.exec.expr.BasicTypeHelper;
 import org.joda.time.Period;
 
 // Source code generated using FreeMarker template ${.template_name}
@@ -96,10 +97,14 @@ public void setPosition(int index) {
 
   @Override
   public MajorType getType() {
-<#if holderMode == "Repeated">
-    return this.repeatedHolder.TYPE;
+<#if name?contains("Decimal")>
+    return BasicTypeHelper.getType(holder);
 <#else>
+  <#if holderMode == "Repeated">
+    return this.repeatedHolder.TYPE;
+  <#else>
     return this.holder.TYPE;
+  </#if>
 </#if>
   }
 
@@ -160,6 +165,8 @@ public void read(Nullable${name}Holder h) {
 
 <#if minor.class == "VarBinary">
       return value;
+<#elseif minor.class == "VarDecimal">
+      return org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(holder.buffer, holder.start, holder.end - holder.start, holder.scale);
 <#elseif minor.class == "Var16Char">
       return new String(value);
 <#elseif minor.class == "VarChar">
@@ -233,6 +240,8 @@ private Object readSingleObject() {
 
 <#if minor.class == "VarBinary">
       return value;
+<#elseif minor.class == "VarDecimal">
+      return org.apache.drill.exec.util.DecimalUtility.getBigDecimalFromDrillBuf(holder.buffer, holder.start, holder.end - holder.start, holder.scale);
 <#elseif minor.class == "Var16Char">
       return new String(value);
 <#elseif minor.class == "VarChar">
diff --git a/exec/vector/src/main/codegen/templates/ListWriters.java b/exec/vector/src/main/codegen/templates/ListWriters.java
index d7a66f75d0..cab8772a74 100644
--- a/exec/vector/src/main/codegen/templates/ListWriters.java
+++ b/exec/vector/src/main/codegen/templates/ListWriters.java
@@ -156,10 +156,24 @@ public ListWriter list() {
   <#assign upperName = minor.class?upper_case />
   <#assign capName = minor.class?cap_first />
   <#if lowerName == "int" ><#assign lowerName = "integer" /></#if>
+
+  <#if minor.class?contains("Decimal") >
+  @Override
+  public ${capName}Writer ${lowerName}() {
+    // returns existing writer
+    assert mode == Mode.IN_${upperName};
+    return writer;
+  }
+
+  @Override
+  public ${capName}Writer ${lowerName}(int scale, int precision) {
+    final MajorType ${upperName}_TYPE = Types.withScaleAndPrecision(MinorType.${upperName}, DataMode.REPEATED, scale, precision);
+  <#else>
   private static final MajorType ${upperName}_TYPE = Types.repeated(MinorType.${upperName});
 
   @Override
   public ${capName}Writer ${lowerName}() {
+  </#if>
     switch (mode) {
     case INIT:
       final int vectorCount = container.size();
@@ -181,14 +195,14 @@ public ListWriter list() {
          .build(logger);
     }
   }
-  
+
   </#list></#list>
   @Override
   public MaterializedField getField() {
     return container.getField();
   }
   <#if mode == "Repeated">
-  
+
   @Override
   public void startList() {
     final RepeatedListVector list = (RepeatedListVector) container;
diff --git a/exec/vector/src/main/codegen/templates/MapWriters.java b/exec/vector/src/main/codegen/templates/MapWriters.java
index 0f05528177..9074293d7a 100644
--- a/exec/vector/src/main/codegen/templates/MapWriters.java
+++ b/exec/vector/src/main/codegen/templates/MapWriters.java
@@ -197,7 +197,8 @@ public void end() {
   <#assign vectName = capName />
   <#assign vectName = "Nullable${capName}" />
 
-  <#if minor.class?starts_with("Decimal") >
+  <#if minor.class?contains("Decimal") >
+  @Override
   public ${minor.class}Writer ${lowerName}(String name) {
     // returns existing writer
     final FieldWriter writer = fields.get(name.toLowerCase());
@@ -205,6 +206,7 @@ public void end() {
     return writer;
   }
 
+  @Override
   public ${minor.class}Writer ${lowerName}(String name, int scale, int precision) {
     final MajorType ${upperName}_TYPE = Types.withScaleAndPrecision(MinorType.${upperName}, DataMode.OPTIONAL, scale, precision);
   <#else>
diff --git a/exec/vector/src/main/codegen/templates/NullReader.java b/exec/vector/src/main/codegen/templates/NullReader.java
index 32ee9b9b9a..4d867baa45 100644
--- a/exec/vector/src/main/codegen/templates/NullReader.java
+++ b/exec/vector/src/main/codegen/templates/NullReader.java
@@ -31,20 +31,18 @@
  * This class is generated using freemarker and the ${.template_name} template.
  */
 @SuppressWarnings("unused")
-public class NullReader extends AbstractBaseReader implements FieldReader{
+public class NullReader extends AbstractBaseReader implements FieldReader {
   
   public static final NullReader INSTANCE = new NullReader();
   public static final NullReader EMPTY_LIST_INSTANCE = new NullReader(Types.repeated(TypeProtos.MinorType.NULL));
   public static final NullReader EMPTY_MAP_INSTANCE = new NullReader(Types.required(TypeProtos.MinorType.MAP));
   private MajorType type;
   
-  private NullReader(){
-    super();
+  private NullReader() {
     type = Types.NULL;
   }
 
-  private NullReader(MajorType type){
-    super();
+  private NullReader(MajorType type) {
     this.type = type;
   }
 
@@ -60,63 +58,67 @@ public void copyAsValue(ListWriter writer) {}
   public void copyAsValue(UnionWriter writer) {}
 
   <#list vv.types as type><#list type.minor as minor><#assign name = minor.class?cap_first />
-  public void read(${name}Holder holder){
+  public void read(${name}Holder holder) {
     throw new UnsupportedOperationException("NullReader cannot write into non-nullable holder");
   }
 
-  public void read(Nullable${name}Holder holder){
+  public void read(Nullable${name}Holder holder) {
     holder.isSet = 0;
   }
 
-  public void read(int arrayIndex, ${name}Holder holder){
+  public void read(int arrayIndex, ${name}Holder holder) {
     throw new ArrayIndexOutOfBoundsException();
   }
   
-  public void copyAsValue(${minor.class}Writer writer){}
-  public void copyAsField(String name, ${minor.class}Writer writer){}
+  public void copyAsValue(${minor.class}Writer writer) {}
+  <#if minor.class == "VarDecimal">
+  public void copyAsField(String name, ${minor.class}Writer writer, int scale, int precision) {}
+  <#else>
+  public void copyAsField(String name, ${minor.class}Writer writer) {}
+  </#if>
 
-  public void read(int arrayIndex, Nullable${name}Holder holder){
+  public void read(int arrayIndex, Nullable${name}Holder holder) {
     throw new ArrayIndexOutOfBoundsException();
   }
   </#list></#list>
   
-  public int size(){
+  public int size() {
     return 0;
   }
   
-  public boolean isSet(){
+  public boolean isSet() {
     return false;
   }
   
-  public boolean next(){
+  public boolean next() {
     return false;
   }
   
-  public RepeatedMapReader map(){
+  public RepeatedMapReader map() {
     return this;
   }
   
-  public RepeatedListReader list(){
+  public RepeatedListReader list() {
     return this;
   }
   
-  public MapReader map(String name){
+  public MapReader map(String name) {
     return this;
   }
   
-  public ListReader list(String name){
+  public ListReader list(String name) {
     return this;
   }
   
-  public FieldReader reader(String name){
+  public FieldReader reader(String name) {
     return this;
   }
   
-  public FieldReader reader(){
+  public FieldReader reader() {
     return this;
   }
   
-  private void fail(String name){
+  private void fail(String name) {
     throw new IllegalArgumentException(String.format("You tried to read a %s type when you are using a ValueReader of type %s.", name, this.getClass().getSimpleName()));
   }
   
@@ -126,11 +128,11 @@ private void fail(String name){
   <#assign safeType=friendlyType />
   <#if safeType=="byte[]"><#assign safeType="ByteArray" /></#if>
   
-  public ${friendlyType} read${safeType}(int arrayIndex){
+  public ${friendlyType} read${safeType}(int arrayIndex) {
     return null;
   }
   
-  public ${friendlyType} read${safeType}(){
+  public ${friendlyType} read${safeType}() {
     return null;
   }
   </#list>
diff --git a/exec/vector/src/main/codegen/templates/NullableValueVectors.java b/exec/vector/src/main/codegen/templates/NullableValueVectors.java
index 8be5c8b867..c9c0987431 100644
--- a/exec/vector/src/main/codegen/templates/NullableValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/NullableValueVectors.java
@@ -482,7 +482,7 @@ public void get(int index, Nullable${minor.class}Holder holder){
       vAccessor.get(index, holder);
       holder.isSet = bAccessor.get(index);
 
-      <#if minor.class.startsWith("Decimal")>
+      <#if minor.class.contains("Decimal")>
       holder.scale = getField().getScale();
       holder.precision = getField().getPrecision();
       </#if>
@@ -694,17 +694,29 @@ public void setSafe(int index, ${minor.javaType!type.javaType} value) {
     }
 
     </#if>
-    <#if minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse">
+    <#if minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "VarDecimal">
     public void set(int index, BigDecimal value) {
+      <#if type.major == "VarLen">
+      if (index > lastSet + 1) {
+        fillEmpties(index);
+      }
+      </#if>
       bits.getMutator().set(index, 1);
       values.getMutator().set(index, value);
       setCount++;
+      <#if type.major == "VarLen">lastSet = index;</#if>
     }
 
     public void setSafe(int index, BigDecimal value) {
+      <#if type.major == "VarLen">
+      if (index > lastSet + 1) {
+        fillEmpties(index);
+      }
+      </#if>
       bits.getMutator().setSafe(index, 1);
       values.getMutator().setSafe(index, value);
       setCount++;
+      <#if type.major == "VarLen">lastSet = index;</#if>
     }
 
     </#if>
diff --git a/exec/vector/src/main/codegen/templates/RepeatedValueVectors.java b/exec/vector/src/main/codegen/templates/RepeatedValueVectors.java
index 412f498c22..e9884208fe 100644
--- a/exec/vector/src/main/codegen/templates/RepeatedValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/RepeatedValueVectors.java
@@ -404,11 +404,13 @@ public void fillEmpties(int lastWrite, int index) {
       }
     }
 
-    <#if (fields?size > 1) && !(minor.class == "Decimal9" || minor.class == "Decimal18" || minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense" || minor.class == "Decimal38Dense")>
-    public void addSafe(int rowIndex, <#list fields as field>${field.type} ${field.name}<#if field_has_next>, </#if></#list>) {
+    <#if (fields?size > 1) && !(minor.class == "Decimal9" || minor.class == "Decimal18"
+        || minor.class == "Decimal28Sparse" || minor.class == "Decimal38Sparse" || minor.class == "Decimal28Dense"
+        || minor.class == "Decimal38Dense") || minor.class == "VarDecimal">
+    public void addSafe(int rowIndex<#list fields as field><#if field.include!true>, ${field.type} ${field.name}</#if></#list>) {
       final int nextOffset = offsets.getAccessor().get(rowIndex+1);
-      values.getMutator().setSafe(nextOffset, <#list fields as field>${field.name}<#if field_has_next>, </#if></#list>);
-      offsets.getMutator().setSafe(rowIndex+1, nextOffset+1);
+      values.getMutator().setSafe(nextOffset<#list fields as field><#if field.include!true>, ${field.name}</#if></#list>);
+      offsets.getMutator().setSafe(rowIndex + 1, nextOffset + 1);
     }
 
     </#if>
diff --git a/exec/vector/src/main/codegen/templates/UnionListWriter.java b/exec/vector/src/main/codegen/templates/UnionListWriter.java
index a0d26a020a..4ea907d1b6 100644
--- a/exec/vector/src/main/codegen/templates/UnionListWriter.java
+++ b/exec/vector/src/main/codegen/templates/UnionListWriter.java
@@ -81,6 +81,17 @@ public void close() throws Exception { }
     return this;
   }
 
+  <#if minor.class == "VarDecimal">
+  @Override
+  public ${name}Writer <#if uncappedName == "int">integer<#else>${uncappedName}</#if>(String name, int scale, int precision) {
+    assert inMap;
+    final int nextOffset = offsets.getAccessor().get(idx() + 1);
+    vector.getMutator().setNotNull(idx());
+    writer.setPosition(nextOffset);
+    ${name}Writer ${uncappedName}Writer = writer.${uncappedName}(name, scale, precision);
+    return ${uncappedName}Writer;
+  }
+  <#else>
   @Override
   public ${name}Writer <#if uncappedName == "int">integer<#else>${uncappedName}</#if>(String name) {
     assert inMap;
@@ -91,6 +102,7 @@ public void close() throws Exception { }
     return ${uncappedName}Writer;
   }
   </#if>
+  </#if>
   </#list></#list>
 
   @Override
diff --git a/exec/vector/src/main/codegen/templates/UnionReader.java b/exec/vector/src/main/codegen/templates/UnionReader.java
index 40ad89b82a..54276f5734 100644
--- a/exec/vector/src/main/codegen/templates/UnionReader.java
+++ b/exec/vector/src/main/codegen/templates/UnionReader.java
@@ -34,14 +34,14 @@
 @SuppressWarnings("unused")
 public class UnionReader extends AbstractFieldReader {
 
-  private BaseReader[] readers = new BaseReader[43];
+  private BaseReader[] readers = new BaseReader[44];
   public UnionVector data;
   
   public UnionReader(UnionVector data) {
     this.data = data;
   }
 
-  private static MajorType[] TYPES = new MajorType[43];
+  private static MajorType[] TYPES = new MajorType[44];
 
   static {
     for (MinorType minorType : MinorType.values()) {
diff --git a/exec/vector/src/main/codegen/templates/UnionWriter.java b/exec/vector/src/main/codegen/templates/UnionWriter.java
index 961810e3a4..13c093acf8 100644
--- a/exec/vector/src/main/codegen/templates/UnionWriter.java
+++ b/exec/vector/src/main/codegen/templates/UnionWriter.java
@@ -181,7 +181,21 @@ public MapWriter map(String name) {
   <#if lowerName == "int" ><#assign lowerName = "integer" /></#if>
   <#assign upperName = minor.class?upper_case />
   <#assign capName = minor.class?cap_first />
-  <#if !minor.class?starts_with("Decimal")>
+  <#if minor.class == "VarDecimal">
+  @Override
+  public ${capName}Writer ${lowerName}(String name, int scale, int precision) {
+    data.getMutator().setType(idx(), MinorType.MAP);
+    getMapWriter().setPosition(idx());
+    return getMapWriter().${lowerName}(name, scale, precision);
+  }
+
+  @Override
+  public ${capName}Writer ${lowerName}(int scale, int precision) {
+    data.getMutator().setType(idx(), MinorType.LIST);
+    getListWriter().setPosition(idx());
+    return getListWriter().${lowerName}(scale, precision);
+  }
+  <#else>
   @Override
   public ${capName}Writer ${lowerName}(String name) {
     data.getMutator().setType(idx(), MinorType.MAP);
diff --git a/exec/vector/src/main/codegen/templates/ValueHolders.java b/exec/vector/src/main/codegen/templates/ValueHolders.java
index 9982bd4190..7635895be3 100644
--- a/exec/vector/src/main/codegen/templates/ValueHolders.java
+++ b/exec/vector/src/main/codegen/templates/ValueHolders.java
@@ -32,88 +32,90 @@
 /*
  * This class is generated using freemarker and the ${.template_name} template.
  */
-public final class ${className} implements ValueHolder{
-  
+<#if minor.class.contains("Decimal") && !minor.class.contains("VarDecimal")>
+/**
+ *  Old decimal types are deprecated. Please use {@link VarDecimalHolder} holder instead.
+ */
+@Deprecated
+</#if>
+public final class ${className} implements ValueHolder {
+
+  <#if minor.class.contains("Decimal")>
+  @Deprecated
+  </#if>
   public static final MajorType TYPE = Types.${mode.name?lower_case}(MinorType.${minor.class?upper_case});
 
-    <#if mode.name == "Repeated">
+  <#if mode.name == "Repeated">
     
-    /** The first index (inclusive) into the Vector. **/
-    public int start;
+  /** The first index (inclusive) into the Vector. **/
+  public int start;
     
-    /** The last index (exclusive) into the Vector. **/
-    public int end;
+  /** The last index (exclusive) into the Vector. **/
+  public int end;
     
-    /** The Vector holding the actual values. **/
-    public ${minor.class}Vector vector;
+  /** The Vector holding the actual values. **/
+  public ${minor.class}Vector vector;
     
-    <#else>
-    public static final int WIDTH = ${type.width};
+  <#else>
+  public static final int WIDTH = ${type.width};
     
-    <#if mode.name == "Optional">public int isSet;</#if>
-    <#assign fields = minor.fields!type.fields />
-    <#list fields as field>
-    public ${field.type} ${field.name};
-    </#list>
+  <#if mode.name == "Optional">public int isSet;</#if>
+  <#assign fields = minor.fields!type.fields />
+  <#list fields as field>
+  public ${field.type} ${field.name};
+  </#list>
     
-    <#if minor.class.startsWith("Decimal")>
-    public static final int maxPrecision = ${minor.maxPrecisionDigits};
-    <#if minor.class.startsWith("Decimal28") || minor.class.startsWith("Decimal38")>
-    public static final int nDecimalDigits = ${minor.nDecimalDigits};
+  <#if minor.class.startsWith("Decimal")>
+  public static final int maxPrecision = ${minor.maxPrecisionDigits};
+  <#if minor.class.startsWith("Decimal28") || minor.class.startsWith("Decimal38")>
+  public static final int nDecimalDigits = ${minor.nDecimalDigits};
     
-    public static int getInteger(int index, int start, DrillBuf buffer) {
-      int value = buffer.getInt(start + (index * 4));
+  public static int getInteger(int index, int start, DrillBuf buffer) {
+    int value = buffer.getInt(start + (index * 4));
 
-      if (index == 0) {
-          /* the first byte contains sign bit, return value without it */
-          <#if minor.class.endsWith("Sparse")>
-          value = (value & 0x7FFFFFFF);
-          <#elseif minor.class.endsWith("Dense")>
-          value = (value & 0x0000007F);
-          </#if>
-      }
-      return value;
+    if (index == 0) {
+      /* the first byte contains sign bit, return value without it */
+      <#if minor.class.endsWith("Sparse")>
+      value = (value & 0x7FFFFFFF);
+      <#elseif minor.class.endsWith("Dense")>
+      value = (value & 0x0000007F);
+      </#if>
     }
+    return value;
+  }
 
-    public static void setInteger(int index, int value, int start, DrillBuf buffer) {
-        buffer.setInt(start + (index * 4), value);
-    }
+  public static void setInteger(int index, int value, int start, DrillBuf buffer) {
+    buffer.setInt(start + (index * 4), value);
+  }
   
-    public static void setSign(boolean sign, int start, DrillBuf buffer) {
-      // Set MSB to 1 if sign is negative
-      if (sign == true) {
-        int value = getInteger(0, start, buffer);
-        setInteger(0, (value | 0x80000000), start, buffer);
-      }
+  public static void setSign(boolean sign, int start, DrillBuf buffer) {
+    // Set MSB to 1 if sign is negative
+    if (sign == true) {
+      int value = getInteger(0, start, buffer);
+      setInteger(0, (value | 0x80000000), start, buffer);
     }
+  }
   
-    public static boolean getSign(int start, DrillBuf buffer) {
-      return ((buffer.getInt(start) & 0x80000000) != 0);
-    }
-    </#if></#if>
-
-    public MajorType getType() {return TYPE;}
+  public static boolean getSign(int start, DrillBuf buffer) {
+    return ((buffer.getInt(start) & 0x80000000) != 0);
+  }
 
-    @Deprecated
-    public int hashCode(){
-      throw new UnsupportedOperationException();
-    }
+  </#if></#if>
+  @Deprecated
+  public int hashCode() {
+    throw new UnsupportedOperationException();
+  }
 
-    /*
-     * Reason for deprecation is that ValueHolders are potential scalar replacements
-     * and hence we don't want any methods to be invoked on them.
-     */
-    @Deprecated
-    public String toString(){
-      throw new UnsupportedOperationException();
-    }
-    </#if>
-    
-    
-    
-    
+  /*
+   * Reason for deprecation is that ValueHolders are potential scalar replacements
+   * and hence we don't want any methods to be invoked on them.
+   */
+  @Deprecated
+  public String toString() {
+    throw new UnsupportedOperationException();
+  }
+  </#if>
 }
-
 </#list>
 </#list>
-</#list>
\ No newline at end of file
+</#list>
diff --git a/exec/vector/src/main/codegen/templates/VariableLengthVectors.java b/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
index 876d6880dc..6a67772ce3 100644
--- a/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
+++ b/exec/vector/src/main/codegen/templates/VariableLengthVectors.java
@@ -464,6 +464,10 @@ public void get(int index, ${minor.class}Holder holder){
       holder.start = oAccessor.get(index);
       holder.end = oAccessor.get(index + 1);
       holder.buffer = data;
+      <#if minor.class.contains("Decimal")>
+      holder.scale = field.getScale();
+      holder.precision = field.getPrecision();
+      </#if>
     }
 
     public void get(int index, Nullable${minor.class}Holder holder){
@@ -471,9 +475,22 @@ public void get(int index, Nullable${minor.class}Holder holder){
       holder.start = oAccessor.get(index);
       holder.end = oAccessor.get(index + 1);
       holder.buffer = data;
+      <#if minor.class.contains("Decimal")>
+      holder.scale = field.getScale();
+      holder.precision = field.getPrecision();
+      </#if>
     }
 
     <#switch minor.class>
+    <#case "VarDecimal">
+    @Override
+    public ${friendlyType} getObject(int index) {
+      byte[] b = get(index);
+      BigInteger bi = b.length == 0 ? BigInteger.ZERO : new BigInteger(b);
+      BigDecimal bd = new BigDecimal(bi, getField().getScale());
+      return bd;
+    }
+    <#break>
     <#case "VarChar">
     @Override
     public ${friendlyType} getObject(int index) {
@@ -663,6 +680,18 @@ public void setSafe(int index, Nullable${minor.class}Holder holder) {
       offsetVector.getMutator().setSafe(index + 1, outputStart + len);
     }
 
+    <#if minor.class == "VarDecimal">
+    public void set(int index, BigDecimal value) {
+      byte[] bytes = value.unscaledValue().toByteArray();
+      set(index, bytes, 0, bytes.length);
+    }
+
+    public void setSafe(int index, BigDecimal value) {
+      byte[] bytes = value.unscaledValue().toByteArray();
+      setSafe(index, bytes, 0, bytes.length);
+    }
+    </#if>
+
     public void setSafe(int index, ${minor.class}Holder holder) {
       final int start = holder.start;
       final int end =   holder.end;
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java b/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java
index 914d68dee7..a451b974f6 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java
@@ -17,20 +17,23 @@
  */
 package org.apache.drill.exec.util;
 
+import com.google.common.math.BigIntegerMath;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.DrillBuf;
 import io.netty.buffer.UnpooledByteBufAllocator;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
+import java.math.RoundingMode;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 
+import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.util.CoreDecimalUtility;
 import org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers;
 import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
 
-public class DecimalUtility extends CoreDecimalUtility{
+public class DecimalUtility extends CoreDecimalUtility {
 
   public final static int MAX_DIGITS = 9;
   public final static int MAX_DIGITS_INT = 10;
@@ -356,11 +359,27 @@ public static void getSparseFromBigDecimal(BigDecimal input, ByteBuf data, int s
     }
 
     // Set the negative sign
-    if (sign == true) {
-        data.setInt(startIndex, data.getInt(startIndex) | 0x80000000);
+    if (sign) {
+      data.setInt(startIndex, data.getInt(startIndex) | 0x80000000);
     }
   }
 
+  /**
+   * Converts from an input BigDecimal into varying width "VarDecimal" representation.
+   * The object that manages the "data" is assumed to already have the proper scale set,
+   * matching that of input.scale().
+   * @param input      input decimal number to be stored
+   * @param data       destination buffer to store the byte array representation of input
+   * @param startIndex starting index in data to hold the bytes
+   * @return startIndex + length of bytes stored (i.e., the next startIndex in the data buffer)
+   */
+  public static int getVarDecimalFromBigDecimal(BigDecimal input, ByteBuf data, int startIndex) {
+    byte[] bytes = input.unscaledValue().toByteArray();
+    int len = bytes.length;
+    data.setBytes(startIndex, bytes);
+    return startIndex + len;
+  }
+
   public static long getDecimal18FromBigDecimal(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);
@@ -433,6 +452,51 @@ public static void setInteger(DrillBuf buffer, int start, int index, int value)
     buffer.setInt(start + (index * 4), value);
   }
 
+  /**
+   * Compares two VarDecimal values, still stored in their respective Drill buffers
+   *
+   * @param left       left value Drill buffer
+   * @param leftStart  start offset of left value
+   * @param leftEnd    end offset of left value
+   * @param leftScale  scale of left value
+   * @param right      right value Drill buffer
+   * @param rightStart start offset of right value
+   * @param rightEnd   end offset of right value
+   * @param rightScale scale of right value
+   * @param absCompare comparison of absolute values is done iff this is true
+   * @return 1 if left > right, 0 if left = right, -1 if left < right.  two values that are numerically equal, but with different
+   * scales (e.g., 2.00 and 2), are considered equal.
+   */
+  public static int compareVarLenBytes(DrillBuf left, int leftStart, int leftEnd, int leftScale, DrillBuf right, int rightStart, int rightEnd, int rightScale, boolean absCompare) {
+    byte[] rightBytes = new byte[rightEnd - rightStart];
+    right.getBytes(rightStart, rightBytes, 0, rightEnd - rightStart);
+
+    return compareVarLenBytes(left, leftStart, leftEnd, leftScale, rightBytes, rightScale, absCompare);
+  }
+
+  /**
+   * Compares two VarDecimal values, still stored in Drill buffer and byte array
+   *
+   * @param left       left value Drill buffer
+   * @param leftStart  start offset of left value
+   * @param leftEnd    end offset of left value
+   * @param leftScale  scale of left value
+   * @param right      right value byte array
+   * @param rightScale scale of right value
+   * @param absCompare comparison of absolute values is done iff this is true
+   * @return 1 if left > right, 0 if left = right, -1 if left < right.  two values that are numerically equal, but with different
+   * scales (e.g., 2.00 and 2), are considered equal.
+   */
+  public static int compareVarLenBytes(DrillBuf left, int leftStart, int leftEnd, int leftScale, byte right[], int rightScale, boolean absCompare) {
+    java.math.BigDecimal bdLeft = getBigDecimalFromDrillBuf(left, leftStart, leftEnd - leftStart, leftScale);
+    java.math.BigDecimal bdRight = new BigDecimal(new BigInteger(right), rightScale);
+    if (absCompare) {
+      bdLeft = bdLeft.abs();
+      bdRight = bdRight.abs();
+    }
+    return bdLeft.compareTo(bdRight);
+  }
+
   public static int compareSparseBytes(DrillBuf left, int leftStart, boolean leftSign, int leftScale, int leftPrecision, DrillBuf right, int rightStart, boolean rightSign, int rightPrecision, int rightScale, int width, int nDecimalDigits, boolean absCompare) {
 
     int invert = 1;
@@ -740,5 +804,73 @@ public static int compareSparseSamePrecScale(DrillBuf left, int lStart, byte[] r
     }
     return cmp * invert;
   }
-}
 
+  /**
+   * Returns max length of byte array, required to store value with specified precision.
+   *
+   * @param precision the precision of value
+   *
+   * @return max length of byte array
+   */
+  public static int getMaxBytesSizeForPrecision(int precision) {
+    if (precision == 0) {
+      return 0;
+    }
+    if (precision < 300) { // normal case, use exact heuristic formula
+      return (int) Math.ceil((Math.log(Math.pow(10, precision) - 1) / Math.log(2) + 1) / Byte.SIZE);
+    } else {
+      // for values greater than 304 Math.pow(10, precision) returns Infinity, therefore 1 is neglected in Math.log()
+      return (int) Math.ceil((precision * Math.log(10) / Math.log(2) + 1) / Byte.SIZE);
+    }
+  }
+
+  /**
+   * Calculates and returns square root for specified BigDecimal
+   * with specified number of digits alter decimal point.
+   *
+   * @param in    BigDecimal which square root should be calculated
+   * @param scale number of digits alter decimal point in the result value.
+   * @return square root for specified BigDecimal
+   */
+  public static BigDecimal sqrt(BigDecimal in, int scale) {
+    // unscaled BigInteger value from specified BigDecimal with doubled number of digits after decimal point
+    // was used to calculate sqrt using Guava's BigIntegerMath.
+    BigInteger valueWithDoubleMaxPrecision =
+        in.multiply(BigDecimal.TEN.pow(scale * 2)).setScale(0, RoundingMode.HALF_UP).unscaledValue();
+    return new BigDecimal(
+        BigIntegerMath.sqrt(valueWithDoubleMaxPrecision, RoundingMode.HALF_UP), scale);
+  }
+
+  /**
+   * Checks that specified decimal minorType is obsolete.
+   *
+   * @param minorType type to check
+   * @return true if specified decimal minorType is obsolete.
+   */
+  public static boolean isObsoleteDecimalType(TypeProtos.MinorType minorType) {
+    return minorType == TypeProtos.MinorType.DECIMAL9 ||
+        minorType == TypeProtos.MinorType.DECIMAL18 ||
+        minorType == TypeProtos.MinorType.DECIMAL28SPARSE ||
+        minorType == TypeProtos.MinorType.DECIMAL38SPARSE;
+  }
+
+  /**
+   * Returns default precision for specified {@link TypeProtos.MinorType}
+   * or returns specified defaultPrecision if {@link TypeProtos.MinorType} isn't
+   * {@link TypeProtos.MinorType#INT} or {@link TypeProtos.MinorType#BIGINT}.
+   *
+   * @param minorType        type wich precision should be received
+   * @param defaultPrecision default value for precision
+   * @return default precision for specified {@link TypeProtos.MinorType}
+   */
+  public static int getDefaultPrecision(TypeProtos.MinorType minorType, int defaultPrecision) {
+    switch (minorType) {
+      case INT:
+        return MAX_DIGITS_INT;
+      case BIGINT:
+        return MAX_DIGITS_BIGINT;
+      default:
+        return defaultPrecision;
+    }
+  }
+}
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueHolderHelper.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueHolderHelper.java
index 73b2b551e9..d1cc3b4a10 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueHolderHelper.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/ValueHolderHelper.java
@@ -37,6 +37,7 @@
 import org.apache.drill.exec.expr.holders.TimeHolder;
 import org.apache.drill.exec.expr.holders.TimeStampHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.expr.holders.VarDecimalHolder;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.util.DecimalUtility;
 
@@ -200,4 +201,22 @@ public static Decimal38SparseHolder getDecimal38Holder(DrillBuf buf, String deci
 
       return dch;
   }
+
+  public static VarDecimalHolder getVarDecimalHolder(DrillBuf buf, String decimal) {
+    VarDecimalHolder dch = new VarDecimalHolder();
+
+    BigDecimal bigDecimal = new BigDecimal(decimal);
+
+    byte[] bytes = bigDecimal.unscaledValue().toByteArray();
+    int length = bytes.length;
+
+    dch.scale = bigDecimal.scale();
+    dch.precision = bigDecimal.precision();
+    dch.start = 0;
+    dch.end = length;
+    dch.buffer = buf.reallocIfNeeded(length);
+    dch.buffer.setBytes(0, bytes);
+
+    return dch;
+  }
 }
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/impl/MapOrListWriterImpl.java b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/impl/MapOrListWriterImpl.java
index b074abd179..937df8603b 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/impl/MapOrListWriterImpl.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/vector/complex/impl/MapOrListWriterImpl.java
@@ -28,6 +28,7 @@
 import org.apache.drill.exec.vector.complex.writer.Decimal38DenseWriter;
 import org.apache.drill.exec.vector.complex.writer.Decimal38SparseWriter;
 import org.apache.drill.exec.vector.complex.writer.Decimal9Writer;
+import org.apache.drill.exec.vector.complex.writer.VarDecimalWriter;
 import org.apache.drill.exec.vector.complex.writer.Float4Writer;
 import org.apache.drill.exec.vector.complex.writer.Float8Writer;
 import org.apache.drill.exec.vector.complex.writer.IntWriter;
@@ -222,6 +223,16 @@ public Decimal38DenseWriter decimal38Dense(String name) {
     return (map != null) ? map.decimal38Dense(name) : list.decimal38Dense();
   }
 
+  @Override
+  public VarDecimalWriter varDecimal(String name) {
+    return (map != null) ? map.varDecimal(name) : list.varDecimal();
+  }
+
+  @Override
+  public VarDecimalWriter varDecimal(String name, int scale, int precision) {
+    return (map != null) ? map.varDecimal(name, scale, precision) : list.varDecimal(scale, precision);
+  }
+
   @Override
   public Decimal38SparseWriter decimal38Sparse(String name) {
     return (map != null) ? map.decimal38Sparse(name) : list.decimal38Sparse();
diff --git a/logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g b/logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
index a881b5cb06..2b497a1959 100644
--- a/logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
+++ b/logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
@@ -62,6 +62,7 @@ DECIMAL28DENSE : 'decimal28dense' | 'DECIMAL28DENSE';
 DECIMAL28SPARSE : 'decimal28sparse' | 'DECIMAL28SPARSE';
 DECIMAL38DENSE : 'decimal38dense' | 'DECIMAL38DENSE';
 DECIMAL38SPARSE : 'decimal38sparse' | 'DECIMAL38SPARSE';
+VARDECIMAL : 'vardecimal' | 'VARDECIMAL';
 Or       : 'or' | 'OR' | 'Or';
 And      : 'and' | 'AND' ;
 Equals   : '==' | '=';
diff --git a/logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g b/logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
index 12048b0c4e..e73bdea00e 100644
--- a/logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
+++ b/logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
@@ -119,6 +119,7 @@ numType returns [MajorType type]
 	| DECIMAL28SPARSE OParen precision Comma scale CParen { $type = TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.DECIMAL28SPARSE).setMode(DataMode.REQUIRED).setPrecision($precision.value.intValue()).setScale($scale.value.intValue()).build(); }
 	| DECIMAL38DENSE OParen precision Comma scale CParen { $type = TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.DECIMAL38DENSE).setMode(DataMode.REQUIRED).setPrecision($precision.value.intValue()).setScale($scale.value.intValue()).build(); }
 	| DECIMAL38SPARSE OParen precision Comma scale CParen { $type = TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.DECIMAL38SPARSE).setMode(DataMode.REQUIRED).setPrecision($precision.value.intValue()).setScale($scale.value.intValue()).build(); }
+	| VARDECIMAL OParen precision Comma scale CParen { $type = TypeProtos.MajorType.newBuilder().setMinorType(TypeProtos.MinorType.VARDECIMAL).setMode(DataMode.REQUIRED).setPrecision($precision.value.intValue()).setScale($scale.value.intValue()).build(); }
 	;
 
 charType returns [MajorType type]
diff --git a/logical/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java b/logical/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
index 34736df324..f09f887412 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
@@ -35,6 +35,7 @@
 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.joda.time.Period;
@@ -216,6 +217,12 @@ public Void visitDecimal38Constant(Decimal38Expression decExpr, StringBuilder sb
     return null;
   }
 
+  @Override
+  public Void visitVarDecimalConstant(VarDecimalExpression decExpr, StringBuilder sb) throws RuntimeException {
+    sb.append(decExpr.getBigDecimal().toString());
+    return null;
+  }
+
   @Override
   public Void visitDoubleConstant(DoubleExpression dExpr, StringBuilder sb) throws RuntimeException {
     sb.append(dExpr.getDouble());
@@ -292,7 +299,7 @@ public Void visitCastExpression(CastExpression e, StringBuilder sb) throws Runti
     case DECIMAL28SPARSE:
     case DECIMAL38DENSE:
     case DECIMAL38SPARSE:
-
+    case VARDECIMAL:
       // add scale and precision
       sb.append("(");
       sb.append(mt.getPrecision());
diff --git a/logical/src/main/java/org/apache/drill/common/expression/ValueExpressions.java b/logical/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
index 527b6b73bb..29e9c05197 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
@@ -18,6 +18,7 @@
 package org.apache.drill.common.expression;
 
 import java.math.BigDecimal;
+import java.util.Collections;
 import java.util.GregorianCalendar;
 import java.util.Iterator;
 
@@ -109,6 +110,10 @@ public static LogicalExpression getDecimal38(BigDecimal i) {
       return new Decimal38Expression(i, ExpressionPosition.UNKNOWN);
   }
 
+  public static LogicalExpression getVarDecimal(BigDecimal i) {
+    return new VarDecimalExpression(i, ExpressionPosition.UNKNOWN);
+  }
+
   public static LogicalExpression getNumericExpression(String sign, String s, ExpressionPosition ep) {
     String numStr = (sign == null) ? s : sign+s;
     try {
@@ -391,11 +396,45 @@ public MajorType getMajorType() {
 
     @Override
     public Iterator<LogicalExpression> iterator() {
-      return Iterators.emptyIterator();
+      return Collections.emptyIterator();
     }
 
   }
 
+  public static class VarDecimalExpression extends LogicalExpressionBase {
+
+    private BigDecimal bigDecimal;
+
+    public VarDecimalExpression(BigDecimal input, ExpressionPosition pos) {
+      super(pos);
+      this.bigDecimal = input;
+    }
+
+    public BigDecimal getBigDecimal() {
+      return bigDecimal;
+    }
+
+    @Override
+    public MajorType getMajorType() {
+      return MajorType
+          .newBuilder()
+          .setMinorType(MinorType.VARDECIMAL)
+          .setScale(bigDecimal.scale())
+          .setPrecision(bigDecimal.precision())
+          .setMode(DataMode.REQUIRED)
+          .build();
+    }
+
+    @Override
+    public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+      return visitor.visitVarDecimalConstant(this, value);
+    }
+
+    @Override
+    public Iterator<LogicalExpression> iterator() {
+      return Collections.emptyIterator();
+    }
+  }
 
   public static class DoubleExpression extends LogicalExpressionBase {
     private double d;
diff --git a/logical/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java b/logical/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java
index 62f9f3d7d9..b5ed5b6a2f 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java
@@ -67,6 +67,7 @@
     TYPE2FUNC.put(MinorType.DECIMAL28DENSE, "castDECIMAL28DENSE");
     TYPE2FUNC.put(MinorType.DECIMAL38SPARSE, "castDECIMAL38SPARSE");
     TYPE2FUNC.put(MinorType.DECIMAL38DENSE, "castDECIMAL38DENSE");
+    TYPE2FUNC.put(MinorType.VARDECIMAL, "castVARDECIMAL");
 
     CAST_FUNC_REPLACEMENT_NEEDED.add(TYPE2FUNC.get(MinorType.INT));
     CAST_FUNC_REPLACEMENT_NEEDED.add(TYPE2FUNC.get(MinorType.BIGINT));
@@ -76,6 +77,7 @@
     CAST_FUNC_REPLACEMENT_NEEDED.add(TYPE2FUNC.get(MinorType.DECIMAL18));
     CAST_FUNC_REPLACEMENT_NEEDED.add(TYPE2FUNC.get(MinorType.DECIMAL28SPARSE));
     CAST_FUNC_REPLACEMENT_NEEDED.add(TYPE2FUNC.get(MinorType.DECIMAL38SPARSE));
+    CAST_FUNC_REPLACEMENT_NEEDED.add(TYPE2FUNC.get(MinorType.VARDECIMAL));
 
     CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.INT), "castEmptyStringVarCharToNullableINT");
     CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.BIGINT), "castEmptyStringVarCharToNullableBIGINT");
@@ -85,6 +87,7 @@
     CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.DECIMAL18), "castEmptyStringVarCharToNullableDECIMAL18");
     CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.DECIMAL28SPARSE), "castEmptyStringVarCharToNullableDECIMAL28SPARSE");
     CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.DECIMAL38SPARSE), "castEmptyStringVarCharToNullableDECIMAL38SPARSE");
+    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.VARDECIMAL), "castEmptyStringVarCharToNullableVARDECIMAL");
 
     CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.INT), "castEmptyStringVar16CharToNullableINT");
     CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.BIGINT), "castEmptyStringVar16CharToNullableBIGINT");
@@ -94,6 +97,7 @@
     CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.DECIMAL18), "castEmptyStringVar16CharToNullableDECIMAL18");
     CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.DECIMAL28SPARSE), "castEmptyStringVar16CharToNullableDECIMAL28SPARSE");
     CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.DECIMAL38SPARSE), "castEmptyStringVar16CharToNullableDECIMAL38SPARSE");
+    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.VARDECIMAL), "castEmptyStringVar16CharToNullableVARDECIMAL");
 
     CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.INT), "castEmptyStringVarBinaryToNullableINT");
     CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.BIGINT), "castEmptyStringVarBinaryToNullableBIGINT");
@@ -103,6 +107,7 @@
     CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.DECIMAL18), "castEmptyStringVarBinaryToNullableDECIMAL18");
     CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.DECIMAL28SPARSE), "castEmptyStringVarBinaryToNullableDECIMAL28SPARSE");
     CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.DECIMAL38SPARSE), "castEmptyStringVarBinaryToNullableDECIMAL38SPARSE");
+    CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.VARDECIMAL), "castEmptyStringVarBinaryToNullableVARDECIMAL");
 
     CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.INT), "castEmptyStringNullableVarCharToNullableINT");
     CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.BIGINT), "castEmptyStringNullableVarCharToNullableBIGINT");
@@ -112,6 +117,7 @@
     CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.DECIMAL18), "castEmptyStringNullableVarCharToNullableDECIMAL18");
     CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.DECIMAL28SPARSE), "castEmptyStringNullableVarCharToNullableDECIMAL28SPARSE");
     CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.DECIMAL38SPARSE), "castEmptyStringNullableVarCharToNullableDECIMAL38SPARSE");
+    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARCHAR.put(TYPE2FUNC.get(MinorType.VARDECIMAL), "castEmptyStringNullableVarCharToNullableVARDECIMAL");
 
     CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.INT), "castEmptyStringNullableVar16CharToNullableINT");
     CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.BIGINT), "castEmptyStringNullableVar16CharToNullableBIGINT");
@@ -121,6 +127,7 @@
     CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.DECIMAL18), "castEmptyStringNullableVar16CharToNullableDECIMAL18");
     CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.DECIMAL28SPARSE), "castEmptyStringNullableVar16CharToNullableDECIMAL28SPARSE");
     CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.DECIMAL38SPARSE), "castEmptyStringNullableVar16CharToNullableDECIMAL38SPARSE");
+    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VAR16CHAR.put(TYPE2FUNC.get(MinorType.VARDECIMAL), "castEmptyStringNullableVar16CharToNullableVARDECIMAL");
 
     CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.INT), "castEmptyStringNullableVarBinaryToNullableINT");
     CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.BIGINT), "castEmptyStringNullableVarBinaryToNullableBIGINT");
@@ -130,6 +137,7 @@
     CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.DECIMAL18), "castEmptyStringNullableVarBinaryToNullableDECIMAL18");
     CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.DECIMAL28SPARSE), "castEmptyStringNullableVarBinaryToNullableDECIMAL28SPARSE");
     CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.DECIMAL38SPARSE), "castEmptyStringNullableVarBinaryToNullableDECIMAL38SPARSE");
+    CAST_FUNC_REPLACEMENT_FROM_NULLABLE_VARBINARY.put(TYPE2FUNC.get(MinorType.VARDECIMAL), "castEmptyStringNullableVarBinaryToNullableVARDECIMAL");
   }
 
   /**
diff --git a/logical/src/main/java/org/apache/drill/common/expression/visitors/AbstractExprVisitor.java b/logical/src/main/java/org/apache/drill/common/expression/visitors/AbstractExprVisitor.java
index 5356813eb5..8458968493 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/visitors/AbstractExprVisitor.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/visitors/AbstractExprVisitor.java
@@ -44,6 +44,7 @@
 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;
 
 public abstract class AbstractExprVisitor<T, VAL, EXCEP extends Exception> implements ExprVisitor<T, VAL, EXCEP> {
 
@@ -108,6 +109,11 @@ public T visitDecimal38Constant(Decimal38Expression decExpr, VAL value) throws E
     return visitUnknown(decExpr, value);
   }
 
+  @Override
+  public T visitVarDecimalConstant(VarDecimalExpression decExpr, VAL value) throws EXCEP {
+    return visitUnknown(decExpr, value);
+  }
+
   @Override
   public T visitDateConstant(DateExpression intExpr, VAL value) throws EXCEP {
     return visitUnknown(intExpr, value);
diff --git a/logical/src/main/java/org/apache/drill/common/expression/visitors/AggregateChecker.java b/logical/src/main/java/org/apache/drill/common/expression/visitors/AggregateChecker.java
index f6fe89dac4..2e6b60b5e3 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/visitors/AggregateChecker.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/visitors/AggregateChecker.java
@@ -46,6 +46,7 @@
 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;
 
 public final class AggregateChecker implements ExprVisitor<Boolean, ErrorCollector, RuntimeException>{
 
@@ -150,6 +151,11 @@ public Boolean visitDecimal38Constant(Decimal38Expression decExpr, ErrorCollecto
     return false;
   }
 
+  @Override
+  public Boolean visitVarDecimalConstant(VarDecimalExpression decExpr, ErrorCollector errors) {
+    return false;
+  }
+
   @Override
   public Boolean visitQuotedStringConstant(QuotedString e, ErrorCollector errors) {
     return false;
diff --git a/logical/src/main/java/org/apache/drill/common/expression/visitors/ConstantChecker.java b/logical/src/main/java/org/apache/drill/common/expression/visitors/ConstantChecker.java
index 0468bc2196..fbe7d721b7 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/visitors/ConstantChecker.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/visitors/ConstantChecker.java
@@ -98,10 +98,7 @@ public Boolean visitIfExpression(IfExpression ifExpr, ErrorCollector errors) {
       return false;
     }
 
-    if (!ifExpr.elseExpression.accept(this, errors)) {
-      return false;
-    }
-    return true;
+    return ifExpr.elseExpression.accept(this, errors);
   }
 
   @Override
@@ -169,6 +166,11 @@ public Boolean visitDecimal38Constant(Decimal38Expression decExpr, ErrorCollecto
     return false;
   }
 
+  @Override
+  public Boolean visitVarDecimalConstant(ValueExpressions.VarDecimalExpression decExpr, ErrorCollector errors) {
+    return true;
+  }
+
   @Override
   public Boolean visitDoubleConstant(DoubleExpression dExpr, ErrorCollector errors) {
     return true;
diff --git a/logical/src/main/java/org/apache/drill/common/expression/visitors/ExprVisitor.java b/logical/src/main/java/org/apache/drill/common/expression/visitors/ExprVisitor.java
index e6198ae6bc..c065bc8835 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/visitors/ExprVisitor.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/visitors/ExprVisitor.java
@@ -44,6 +44,7 @@
 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;
 
 public interface ExprVisitor<T, VAL, EXCEP extends Exception> {
   T visitFunctionCall(FunctionCall call, VAL value) throws EXCEP;
@@ -63,6 +64,7 @@
   T visitDecimal18Constant(Decimal18Expression decExpr, VAL value) throws EXCEP;
   T visitDecimal28Constant(Decimal28Expression decExpr, VAL value) throws EXCEP;
   T visitDecimal38Constant(Decimal38Expression decExpr, VAL value) throws EXCEP;
+  T visitVarDecimalConstant(VarDecimalExpression decExpr, VAL value) throws EXCEP;
   T visitDoubleConstant(DoubleExpression dExpr, VAL value) throws EXCEP;
   T visitBooleanConstant(BooleanExpression e, VAL value) throws EXCEP;
   T visitQuotedStringConstant(QuotedString e, VAL value) throws EXCEP;
diff --git a/logical/src/main/java/org/apache/drill/common/expression/visitors/ExpressionValidator.java b/logical/src/main/java/org/apache/drill/common/expression/visitors/ExpressionValidator.java
index c32825abb1..b3074fcb1d 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/visitors/ExpressionValidator.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/visitors/ExpressionValidator.java
@@ -44,6 +44,7 @@
 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.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
@@ -166,6 +167,11 @@ public Void visitDecimal38Constant(Decimal38Expression decExpr, ErrorCollector e
     return null;
   }
 
+  @Override
+  public Void visitVarDecimalConstant(VarDecimalExpression decExpr, ErrorCollector errors) throws RuntimeException {
+    return null;
+  }
+
   @Override
   public Void visitDateConstant(DateExpression intExpr, ErrorCollector errors) throws RuntimeException {
     return null;
diff --git a/logical/src/main/java/org/apache/drill/common/expression/visitors/SimpleExprVisitor.java b/logical/src/main/java/org/apache/drill/common/expression/visitors/SimpleExprVisitor.java
index 6655986f14..c025dc87e9 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/visitors/SimpleExprVisitor.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/visitors/SimpleExprVisitor.java
@@ -36,6 +36,7 @@
 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;
 
 public abstract class SimpleExprVisitor<T> implements ExprVisitor<T, Void, RuntimeException>{
 
@@ -120,6 +121,11 @@ public T visitDecimal38Constant(Decimal38Expression decExpr, Void value) throws
     return visitDecimal38Constant(decExpr);
   }
 
+  @Override
+  public T visitVarDecimalConstant(VarDecimalExpression decExpr, Void value) throws RuntimeException {
+    return visitVarDecimalConstant(decExpr);
+  }
+
   @Override
   public T visitDoubleConstant(DoubleExpression dExpr, Void value) throws RuntimeException {
     return visitDoubleConstant(dExpr);
@@ -152,6 +158,7 @@ public T visitQuotedStringConstant(QuotedString e, Void value) throws RuntimeExc
   public abstract T visitDecimal18Constant(Decimal18Expression intExpr);
   public abstract T visitDecimal28Constant(Decimal28Expression intExpr);
   public abstract T visitDecimal38Constant(Decimal38Expression intExpr);
+  public abstract T visitVarDecimalConstant(VarDecimalExpression intExpr);
   public abstract T visitDoubleConstant(DoubleExpression dExpr);
   public abstract T visitBooleanConstant(BooleanExpression e);
   public abstract T visitQuotedStringConstant(QuotedString e);
diff --git a/pom.xml b/pom.xml
index 8233d1c18a..42729781b0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -72,6 +72,7 @@
     <javassist.version>3.22.0-GA</javassist.version>
     <msgpack.version>0.6.6</msgpack.version>
     <reflections.version>0.9.10</reflections.version>
+    <avro.version>1.8.2</avro.version>
     <metrics.version>4.0.2</metrics.version>
     <excludedGroups/>
     <memoryMb>4096</memoryMb>
@@ -965,6 +966,44 @@
           </exclusion>
         </exclusions>
       </dependency>
+      <dependency>
+        <groupId>org.apache.avro</groupId>
+        <artifactId>avro</artifactId>
+        <version>${avro.version}</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>${avro.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.mortbay.jetty</groupId>
+            <artifactId>servlet-api</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.hive</groupId>
         <artifactId>hive-exec</artifactId>
diff --git a/protocol/src/main/java/org/apache/drill/common/types/MinorType.java b/protocol/src/main/java/org/apache/drill/common/types/MinorType.java
index dffc912e0d..b26cb087d2 100644
--- a/protocol/src/main/java/org/apache/drill/common/types/MinorType.java
+++ b/protocol/src/main/java/org/apache/drill/common/types/MinorType.java
@@ -59,7 +59,8 @@
     INTERVALDAY(39),
     LIST(40),
     GENERIC_OBJECT(41),
-    UNION(42);
+    UNION(42),
+    VARDECIMAL(43);
     
     public final int number;
     
@@ -115,6 +116,7 @@ public static MinorType valueOf(int number)
             case 40: return LIST;
             case 41: return GENERIC_OBJECT;
             case 42: return UNION;
+            case 43: return VARDECIMAL;
             default: return null;
         }
     }
diff --git a/protocol/src/main/java/org/apache/drill/common/types/TypeProtos.java b/protocol/src/main/java/org/apache/drill/common/types/TypeProtos.java
index b02298a70e..c22b135fa2 100644
--- a/protocol/src/main/java/org/apache/drill/common/types/TypeProtos.java
+++ b/protocol/src/main/java/org/apache/drill/common/types/TypeProtos.java
@@ -321,6 +321,14 @@ public static void registerAllExtensions(
      * <code>UNION = 42;</code>
      */
     UNION(37, 42),
+    /**
+     * <code>VARDECIMAL = 43;</code>
+     *
+     * <pre>
+     * variable width decimal (arbitrary precision)
+     * </pre>
+     */
+    VARDECIMAL(38, 43),
     ;
 
     /**
@@ -614,6 +622,14 @@ public static void registerAllExtensions(
      * <code>UNION = 42;</code>
      */
     public static final int UNION_VALUE = 42;
+    /**
+     * <code>VARDECIMAL = 43;</code>
+     *
+     * <pre>
+     * variable width decimal (arbitrary precision)
+     * </pre>
+     */
+    public static final int VARDECIMAL_VALUE = 43;
 
 
     public final int getNumber() { return value; }
@@ -658,6 +674,7 @@ public static MinorType valueOf(int value) {
         case 40: return LIST;
         case 41: return GENERIC_OBJECT;
         case 42: return UNION;
+        case 43: return VARDECIMAL;
         default: return null;
       }
     }
@@ -1984,7 +2001,7 @@ public Builder clearSubType() {
       "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*\225\004\n\tMinorType\022\010\n\004LATE\020\000\022\007\n\003MA" +
+      ".MinorType*\245\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" +
@@ -1997,10 +2014,10 @@ public Builder clearSubType() {
       "\n\005UINT4\020\037\022\t\n\005UINT8\020 \022\022\n\016DECIMAL28DENSE\020!" +
       "\022\022\n\016DECIMAL38DENSE\020\"\022\010\n\004NULL\020%\022\020\n\014INTERV" +
       "ALYEAR\020&\022\017\n\013INTERVALDAY\020\'\022\010\n\004LIST\020(\022\022\n\016G" +
-      "ENERIC_OBJECT\020)\022\t\n\005UNION\020**4\n\010DataMode\022\014" +
-      "\n\010OPTIONAL\020\000\022\014\n\010REQUIRED\020\001\022\014\n\010REPEATED\020\002",
-      "B-\n\035org.apache.drill.common.typesB\nTypeP" +
-      "rotosH\001"
+      "ENERIC_OBJECT\020)\022\t\n\005UNION\020*\022\016\n\nVARDECIMAL" +
+      "\020+*4\n\010DataMode\022\014\n\010OPTIONAL\020\000\022\014\n\010REQUIRED",
+      "\020\001\022\014\n\010REPEATED\020\002B-\n\035org.apache.drill.com" +
+      "mon.typesB\nTypeProtosH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
diff --git a/protocol/src/main/protobuf/Types.proto b/protocol/src/main/protobuf/Types.proto
index b2b29f0850..6e4b66e86b 100644
--- a/protocol/src/main/protobuf/Types.proto
+++ b/protocol/src/main/protobuf/Types.proto
@@ -65,6 +65,7 @@ enum MinorType {
     LIST = 40;
     GENERIC_OBJECT = 41;
     UNION = 42;
+    VARDECIMAL = 43;  // variable width decimal (arbitrary precision)
 }
 
 message MajorType {


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> Decimal data type enhancements
> ------------------------------
>
>                 Key: DRILL-6094
>                 URL: https://issues.apache.org/jira/browse/DRILL-6094
>             Project: Apache Drill
>          Issue Type: Improvement
>    Affects Versions: 1.12.0
>            Reporter: Volodymyr Vysotskyi
>            Assignee: Volodymyr Vysotskyi
>            Priority: Major
>              Labels: doc-impacting
>             Fix For: 1.14.0
>
>
> Currently, Decimal types are disabled by default since existing Decimal implementation has a lot of flaws and performance problems. The goal of this Jira to describe majority of them and possible ways of improving existing implementation to be able to enable Decimal data types by default.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)