You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pa...@apache.org on 2015/02/23 20:53:13 UTC

[4/4] drill git commit: DRILL-1062: Implemented null ordering (NULLS FIRST/NULLS LAST).

DRILL-1062: Implemented null ordering (NULLS FIRST/NULLS LAST).

Primary:
- Split "compare_to" function templates (for sorting) into
  "compare_to_nulls_high" and "compare_to_nulls_low" versions.
- Added tests to verify ORDER BY ordering.
- Added tests to verify merge join order correctness.
- Implemented java.sql.DatabaseMetaData.nullsAreSortedHigh(), etc.
Secondary:
- Eliminated DateInterfaceFunctions.java template (merged into other).
- Renamed comparison-related template data objects and file names.
- Eliminated unused template macros, function template classes.
- Overhauled Order.Ordering; added unit test.
- Regularized some generated-class names.
Miscellaneous:
- Added toString() to ExpressionPosition, Order.Ordering, JoinStatus.
- Fixed some typos.
- Fixed some comment syntax.


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/5efc7e68
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/5efc7e68
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/5efc7e68

Branch: refs/heads/master
Commit: 5efc7e686d04ebff7c651270961ec7f5ca09c93d
Parents: 3c85bd8
Author: dbarclay <db...@maprtech.com>
Authored: Tue Feb 17 13:50:16 2015 -0800
Committer: Parth Chandra <pc...@maprtech.com>
Committed: Mon Feb 23 11:51:33 2015 -0800

----------------------------------------------------------------------
 .gitignore                                      |    1 +
 .../common/expression/ExpressionPosition.java   |    6 +
 .../common/expression/fn/CastFunctions.java     |   10 +-
 .../apache/drill/common/logical/data/Order.java |  147 +-
 .../drill/common/logical/data/OrderTest.java    |   93 ++
 .../drill/exec/TestHiveProjectPushDown.java     |    2 +-
 exec/java-exec/src/main/codegen/config.fmpp     |   51 +-
 .../src/main/codegen/data/CompareTypes.tdd      |   21 -
 .../codegen/data/ComparisonTypesDecimal.tdd     |   26 +
 .../main/codegen/data/ComparisonTypesMain.tdd   |   69 +
 .../src/main/codegen/data/DateTypes.tdd         |   27 -
 .../src/main/codegen/data/DecimalTypes.tdd      |   26 -
 .../codegen/templates/ComparisonFunctions.java  |  410 ++++--
 .../templates/DateIntervalFunctions.java        |  355 -----
 .../templates/Decimal/CastVarCharDecimal.java   |   15 +-
 .../templates/Decimal/DecimalFunctions.java     |  627 +++++----
 .../exec/client/PrintingResultsListener.java    |    3 +-
 .../exec/expr/ExpressionTreeMaterializer.java   |    2 +-
 .../exec/expr/annotations/FunctionTemplate.java |   35 +-
 .../exec/expr/fn/FunctionGenerationHelper.java  |   58 +-
 .../expr/fn/FunctionImplementationRegistry.java |   10 +-
 .../drill/exec/expr/fn/impl/BitFunctions.java   |   46 +-
 .../exec/expr/fn/impl/ComparisonFunctions.java  |  572 --------
 .../fn/impl/ComparisonFunctionsNullable.java    |  250 ----
 .../exec/physical/impl/TopN/TopNBatch.java      |    5 +-
 .../impl/aggregate/StreamingAggBatch.java       |    8 +-
 .../physical/impl/common/ChainedHashTable.java  |    6 +-
 .../exec/physical/impl/join/JoinStatus.java     |   23 +
 .../exec/physical/impl/join/JoinTemplate.java   |   12 +-
 .../exec/physical/impl/join/MergeJoinBatch.java |   18 +-
 .../impl/mergereceiver/MergingRecordBatch.java  |    5 +-
 .../OrderedPartitionRecordBatch.java            |    6 +-
 .../exec/physical/impl/sort/SortBatch.java      |    5 +-
 .../window/StreamingWindowFrameRecordBatch.java |    8 +-
 .../physical/impl/xsort/ExternalSortBatch.java  |    9 +-
 .../drill/exec/physical/impl/xsort/MSorter.java |    2 +
 .../exec/planner/physical/ProjectPrule.java     |    2 +-
 .../drill/exec/rpc/user/QueryResultHandler.java |    2 +-
 .../java/org/apache/drill/TestAggNullable.java  |    8 +-
 .../org/apache/drill/TestExampleQueries.java    |   28 +-
 .../test/java/org/apache/drill/TestInList.java  |    2 +-
 .../java/org/apache/drill/TestJoinNullable.java |  390 +++++-
 .../resources/jsoninput/nullableOrdered1.json   |   24 +
 .../resources/jsoninput/nullableOrdered2.json   |   16 +
 .../drill/jdbc/DrillDatabaseMetaData.java       |   53 +
 .../org/apache/drill/jdbc/DrillFactory.java     |    3 +
 .../apache/drill/jdbc/DrillJdbc41Factory.java   |    8 +-
 .../test/JdbcNullOrderingAndGroupingTest.java   | 1290 ++++++++++++++++++
 exec/jdbc/src/test/resources/donuts.json        |  130 ++
 .../null_ordering_and_grouping_data.json        |   63 +
 pom.xml                                         |    1 +
 51 files changed, 3127 insertions(+), 1862 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/5efc7e68/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 838ea6b..73c1be4 100644
--- a/.gitignore
+++ b/.gitignore
@@ -4,6 +4,7 @@
 .checkstyle
 .settings/
 .idea/
+TAGS
 *.log
 *.lck
 *.iml

http://git-wip-us.apache.org/repos/asf/drill/blob/5efc7e68/common/src/main/java/org/apache/drill/common/expression/ExpressionPosition.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/ExpressionPosition.java b/common/src/main/java/org/apache/drill/common/expression/ExpressionPosition.java
index 010e440..e3f34f4 100644
--- a/common/src/main/java/org/apache/drill/common/expression/ExpressionPosition.java
+++ b/common/src/main/java/org/apache/drill/common/expression/ExpressionPosition.java
@@ -31,6 +31,12 @@ public class ExpressionPosition {
     this.charIndex = charIndex;
   }
 
+  @Override
+  public String toString() {
+    return super.toString()
+           + "[charIndex = " + charIndex + ", expression = " + expression + "]";
+  }
+
   public String getExpression() {
     return expression;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/5efc7e68/common/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java b/common/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java
index f1a3b37..c0eaa90 100644
--- a/common/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java
+++ b/common/src/main/java/org/apache/drill/common/expression/fn/CastFunctions.java
@@ -27,9 +27,13 @@ import org.apache.drill.common.types.TypeProtos.MinorType;
 public class CastFunctions {
 
   private static Map<MinorType, String> TYPE2FUNC = new HashMap<>();
-  private static Set<String> CAST_FUNC_REPLACEMENT_NEEDED = new HashSet<>();  // The cast fucntions which are needed to be replaced (if "drill.exec.functions.cast_empty_string_to_null"" is set as true)
-  private static Map<String, String> CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE = new HashMap<>(); // Map from the replaced functions to the new ones (for non-nullable varchar)
-  private static Map<String, String> CAST_FUNC_REPLACEMENT_FROM_NULLABLE = new HashMap<>();    // Map from the replaced functions to the new ones (for nullable varchar)
+  /** The cast functions that need to be replaced (if
+   * "drill.exec.functions.cast_empty_string_to_null" is set to true). */
+  private static Set<String> CAST_FUNC_REPLACEMENT_NEEDED = new HashSet<>();
+  /** Map from the replaced functions to the new ones (for non-nullable VARCHAR). */
+  private static Map<String, String> CAST_FUNC_REPLACEMENT_FROM_NONNULLABLE = new HashMap<>();
+  /** Map from the replaced functions to the new ones (for nullable VARCHAR). */
+  private static Map<String, String> CAST_FUNC_REPLACEMENT_FROM_NULLABLE = new HashMap<>();
 
   static {
     TYPE2FUNC.put(MinorType.BIGINT, "castBIGINT");

http://git-wip-us.apache.org/repos/asf/drill/blob/5efc7e68/common/src/main/java/org/apache/drill/common/logical/data/Order.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/logical/data/Order.java b/common/src/main/java/org/apache/drill/common/logical/data/Order.java
index dada606..ee6f9a9 100644
--- a/common/src/main/java/org/apache/drill/common/logical/data/Order.java
+++ b/common/src/main/java/org/apache/drill/common/logical/data/Order.java
@@ -20,6 +20,7 @@ package org.apache.drill.common.logical.data;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
@@ -65,29 +66,98 @@ public class Order extends SingleInputOperator {
     }
 
 
-    public static class Ordering {
+  /**
+   * Representation of a SQL &lt;sort specification>.
+   */
+  public static class Ordering {
 
-    private final RelFieldCollation.Direction direction;
     private final LogicalExpression expr;
-    private final RelFieldCollation.NullDirection nulls;
-
+    /** Net &lt;ordering specification>. */
+    private final Direction direction;
+    /** Net &lt;null ordering> */
+    private final NullDirection nullOrdering;
+
+    /**
+     * Constructs a sort specification.
+     * @param  expr  ...
+     * @param  strOrderingSpec  the &lt;ordering specification> as string;
+     *             allowed values: {@code "ASC"}, {@code "DESC"}, {@code null};
+     *             null specifies default &lt;ordering specification>
+     *                   ({@code "ASC"} / {@link Direction#ASCENDING})
+     * @param  strNullOrdering   the &lt;null ordering> as string;
+     *             allowed values: {@code "FIRST"}, {@code "LAST"},
+     *             {@code "UNSPECIFIED"}, {@code null};
+     *             null specifies default &lt;null ordering>
+     *             (omitted / {@link NullDirection#UNSPECIFIED}, interpreted later)
+     */
     @JsonCreator
-    public Ordering(@JsonProperty("order") String strOrder, @JsonProperty("expr") LogicalExpression expr, @JsonProperty("nullDirection") String nullCollation) {
+    public Ordering( @JsonProperty("expr") LogicalExpression expr,
+                     @JsonProperty("order") String strOrderingSpec,
+                     @JsonProperty("nullDirection") String strNullOrdering ) {
       this.expr = expr;
-      this.nulls = NullDirection.LAST.name().equalsIgnoreCase(nullCollation) ? NullDirection.LAST :  NullDirection.FIRST; // default first
-      this.direction = Order.getDirectionFromString(strOrder);
+      this.direction = getOrderingSpecFromString( strOrderingSpec );
+      this.nullOrdering = getNullOrderingFromString( strNullOrdering );
     }
 
-    public Ordering(Direction direction, LogicalExpression e, NullDirection nullCollation) {
+    public Ordering(Direction direction, LogicalExpression e, NullDirection nullOrdering) {
       this.expr = e;
-      this.nulls = nullCollation;
       this.direction = direction;
+      this.nullOrdering = nullOrdering;
     }
 
     public Ordering(Direction direction, LogicalExpression e) {
       this(direction, e, NullDirection.FIRST);
     }
 
+    private static Direction getOrderingSpecFromString( String strDirection ) {
+      final Direction direction;
+      if ( null == strDirection
+          || Direction.ASCENDING.shortString.equals( strDirection ) ) {
+        direction = Direction.ASCENDING;
+      }
+      else if ( Direction.DESCENDING.shortString.equals( strDirection ) ) {
+        direction = Direction.DESCENDING;
+      }
+      else {
+        throw new DrillRuntimeException(
+            "Unknown <ordering specification> string (not \"ASC\", \"DESC\", "
+            + "or null): \"" + strDirection + "\"" );
+      }
+      return direction;
+    }
+
+    private static NullDirection getNullOrderingFromString( String strNullOrdering ) {
+      final RelFieldCollation.NullDirection nullOrdering;
+      if ( null == strNullOrdering ) {
+        nullOrdering = NullDirection.UNSPECIFIED;
+      }
+      else {
+        try {
+          nullOrdering = NullDirection.valueOf( strNullOrdering );
+        }
+        catch ( IllegalArgumentException e ) {
+          throw new DrillRuntimeException(
+              "Internal error:  Unknown <null ordering> string (not "
+              + "\"" + NullDirection.FIRST.name() + "\", "
+              + "\"" + NullDirection.LAST.name() + "\", or "
+              + "\"" + NullDirection.UNSPECIFIED.name() + "\" or null): "
+              + "\"" + strNullOrdering + "\"" );
+        }
+      }
+      return nullOrdering;
+   }
+
+    @Override
+    public String toString() {
+      return
+          super.toString()
+          + "[ "
+          + " expr = " + expr
+          + ", direction = " + direction
+          + ", nullOrdering = " + nullOrdering
+          + "] ";
+    }
+
     @JsonIgnore
     public Direction getDirection() {
       return direction;
@@ -98,18 +168,59 @@ public class Order extends SingleInputOperator {
     }
 
     public String getOrder() {
-
-      switch(direction){
-      case DESCENDING: return "DESC";
-      default: return "ASC";
+      switch (direction) {
+      case ASCENDING:
+        return Direction.ASCENDING.shortString;
+      case DESCENDING:
+        return Direction.DESCENDING.shortString;
+      default:
+        throw new DrillRuntimeException(
+            "Unexpected " + Direction.class.getName() + " value other than "
+            + Direction.ASCENDING + " or " + Direction.DESCENDING + ": "
+            + direction );
       }
     }
 
     public NullDirection getNullDirection() {
-      return nulls;
+      return nullOrdering;
     }
 
+    /**
+     * Reports whether NULL sorts high or low in this ordering.
+     *
+     * @return
+     * {@code true}  if NULL sorts higher than any other value;
+     * {@code false} if NULL sorts lower  than any other value
+     */
+    public boolean nullsSortHigh() {
+      final boolean nullsHigh;
+
+      switch (nullOrdering) {
+
+      case UNSPECIFIED:
+        // Default:  NULL sorts high: like NULLS LAST if ASC, FIRST if DESC.
+        nullsHigh = true;
+        break;
+
+      case FIRST:
+        // FIRST: NULL sorts low with ASC, high with DESC.
+        nullsHigh = Direction.DESCENDING == getDirection();
+        break;
+
+      case LAST:
+        // LAST: NULL sorts high with ASC, low with DESC.
+        nullsHigh = Direction.ASCENDING == getDirection();
+        break;
+
+      default:
+        throw new DrillRuntimeException(
+            "Unexpected " + NullDirection.class.getName() + " value other than "
+            + NullDirection.FIRST + ", " + NullDirection.LAST + " or " + NullDirection.UNSPECIFIED + ": "
+            + nullOrdering );
+      }
 
+      return nullsHigh;
+    }
 
   }
 
@@ -138,12 +249,4 @@ public class Order extends SingleInputOperator {
 
 
   }
-
-  public static Direction getDirectionFromString(String direction){
-    return "DESC".equalsIgnoreCase(direction) ? Direction.DESCENDING : Direction.ASCENDING;
-  }
-
-  public static String getStringFromDirection(Direction direction){
-    return direction == Direction.DESCENDING ? "DESC" : "ASC";
-  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/5efc7e68/common/src/test/java/org/apache/drill/common/logical/data/OrderTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/drill/common/logical/data/OrderTest.java b/common/src/test/java/org/apache/drill/common/logical/data/OrderTest.java
new file mode 100644
index 0000000..144b35b
--- /dev/null
+++ b/common/src/test/java/org/apache/drill/common/logical/data/OrderTest.java
@@ -0,0 +1,93 @@
+/*
+ * 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.logical.data;
+
+import static org.junit.Assert.*;
+
+import java.sql.SQLException;
+
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.logical.data.Order.Ordering;
+import org.eigenbase.rel.RelFieldCollation;
+import org.eigenbase.rel.RelFieldCollation.Direction;
+import org.eigenbase.rel.RelFieldCollation.NullDirection;
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.*;
+
+public class OrderTest {
+
+  //////////
+  // Order.Ordering tests:
+
+  // "Round trip" tests that strings from output work as input:
+
+  @Test
+  public void test_Ordering_roundTripAscAndNullsFirst() {
+    Ordering src = new Ordering( Direction.ASCENDING, null, NullDirection.FIRST);
+    Ordering reconstituted =
+        new Ordering( (LogicalExpression) null, src.getOrder(), src.getNullDirection().toString() );
+    assertThat( reconstituted.getDirection(), equalTo( RelFieldCollation.Direction.ASCENDING  ) );
+    assertThat( reconstituted.getNullDirection(), equalTo( NullDirection.FIRST  ) );
+  }
+
+  @Test
+  public void test_Ordering_roundTripDescAndNullsLast() {
+    Ordering src = new Ordering( Direction.DESCENDING, null, NullDirection.LAST);
+    Ordering reconstituted =
+        new Ordering( (LogicalExpression) null, src.getOrder(), src.getNullDirection().toString() );
+    assertThat( reconstituted.getDirection(), equalTo( RelFieldCollation.Direction.DESCENDING  ) );
+    assertThat( reconstituted.getNullDirection(), equalTo( NullDirection.LAST  ) );
+  }
+
+  @Test
+  public void test_Ordering_roundTripDescAndNullsUnspecified() {
+    Ordering src = new Ordering( Direction.DESCENDING, null, NullDirection.UNSPECIFIED);
+    Ordering reconstituted =
+        new Ordering( (LogicalExpression) null, src.getOrder(), src.getNullDirection().toString() );
+    assertThat( reconstituted.getDirection(), equalTo( RelFieldCollation.Direction.DESCENDING  ) );
+    assertThat( reconstituted.getNullDirection(), equalTo( NullDirection.UNSPECIFIED  ) );
+  }
+
+
+  // Basic input validation:
+
+  @Test( expected = DrillRuntimeException.class )  // (Currently.)
+  public void test_Ordering_garbageOrderRejected() {
+    new Ordering( (LogicalExpression) null, "AS CE ND IN G", (String) null );
+  }
+
+  @Test( expected = DrillRuntimeException.class )  // (Currently.)
+  public void test_Ordering_garbageNullOrderingRejected() {
+    new Ordering( (LogicalExpression) null, (String) null, "HIGH" );
+  }
+
+
+  // Defaults-value/null-strings test:
+
+  @Test
+  public void testOrdering_nullStrings() {
+    Ordering ordering = new Ordering( (LogicalExpression) null, null, null );
+    assertThat( ordering.getDirection(), equalTo( RelFieldCollation.Direction.ASCENDING ) );
+    assertThat( ordering.getNullDirection(), equalTo( RelFieldCollation.NullDirection.UNSPECIFIED ) );
+    assertThat( ordering.getOrder(), equalTo( "ASC" ) );
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/5efc7e68/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveProjectPushDown.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveProjectPushDown.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveProjectPushDown.java
index 0246f54..2842976 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveProjectPushDown.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/TestHiveProjectPushDown.java
@@ -27,7 +27,7 @@ public class TestHiveProjectPushDown extends HiveTestBase {
     testPhysicalPlan(query, expectedColNamesInPlan);
 
     int actualRecordCount = testSql(query);
-    assertEquals(String.format("Received unexepcted number of rows in output: expected=%d, received=%s",
+    assertEquals(String.format("Received unexpected number of rows in output: expected=%d, received=%s",
         expectedRecordCount, actualRecordCount), expectedRecordCount, actualRecordCount);
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/5efc7e68/exec/java-exec/src/main/codegen/config.fmpp
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/config.fmpp b/exec/java-exec/src/main/codegen/config.fmpp
index aff6240..5dc7360 100644
--- a/exec/java-exec/src/main/codegen/config.fmpp
+++ b/exec/java-exec/src/main/codegen/config.fmpp
@@ -15,29 +15,34 @@
 # limitations under the License.
 
 data: {
-    vv: tdd(../data/ValueVectorTypes.tdd),
-    compare: tdd(../data/CompareTypes.tdd),
-    cast: tdd(../data/Casts.tdd),
-    MathFunctionTypes: tdd(../data/MathFunctionTypes.tdd),
-    mathFunc:tdd(../data/MathFunc.tdd),
-    aggrtypes1: tdd(../data/AggrTypes1.tdd),
-    decimalaggrtypes1: tdd(../data/DecimalAggrTypes1.tdd),
-    decimalaggrtypes2: tdd(../data/DecimalAggrTypes2.tdd),
-    aggrtypes2: tdd(../data/AggrTypes2.tdd),
-    aggrtypes3: tdd(../data/AggrTypes3.tdd),
-    covarTypes: tdd(../data/CovarTypes.tdd),
-    corrTypes: tdd(../data/CorrelationTypes.tdd),
-    logicalTypes: tdd(../data/AggrBitwiseLogicalTypes.tdd),
-    date: tdd(../data/DateTypes.tdd),
-    extract: tdd(../data/ExtractTypes.tdd),
-    parser: tdd(../data/Parser.tdd),
-    decimal: tdd(../data/DecimalTypes.tdd),
-    dateIntervalFunc: tdd(../data/DateIntervalFunc.tdd),
-    intervalNumericTypes: tdd(../data/IntervalNumericTypes.tdd),
-    extract: tdd(../data/ExtractTypes.tdd),
-    sumzero: tdd(../data/SumZero.tdd),
-    numericTypes: tdd(../data/NumericTypes.tdd),
-    casthigh: tdd(../data/CastHigh.tdd)
+    vv:                       tdd(../data/ValueVectorTypes.tdd),
+
+    # Most types for comparison functions (for templates/ComparisonFunctions.java).
+    comparisonTypesMain:      tdd(../data/ComparisonTypesMain.tdd),
+
+    # Decimal types for comparison function (for
+    # templates/DecimalFunctions/ComparisonFunctions.java).
+    comparisonTypesDecimal:   tdd(../data/ComparisonTypesDecimal.tdd),
+
+    cast:                     tdd(../data/Casts.tdd),
+    MathFunctionTypes:        tdd(../data/MathFunctionTypes.tdd),
+    mathFunc:                 tdd(../data/MathFunc.tdd),
+    aggrtypes1:               tdd(../data/AggrTypes1.tdd),
+    decimalaggrtypes1:        tdd(../data/DecimalAggrTypes1.tdd),
+    decimalaggrtypes2:        tdd(../data/DecimalAggrTypes2.tdd),
+    aggrtypes2:               tdd(../data/AggrTypes2.tdd),
+    aggrtypes3:               tdd(../data/AggrTypes3.tdd),
+    covarTypes:               tdd(../data/CovarTypes.tdd),
+    corrTypes:                tdd(../data/CorrelationTypes.tdd),
+    logicalTypes:             tdd(../data/AggrBitwiseLogicalTypes.tdd),
+    extract:                  tdd(../data/ExtractTypes.tdd),
+    parser:                   tdd(../data/Parser.tdd),
+    dateIntervalFunc:         tdd(../data/DateIntervalFunc.tdd),
+    intervalNumericTypes:     tdd(../data/IntervalNumericTypes.tdd),
+    extract:                  tdd(../data/ExtractTypes.tdd),
+    sumzero:                  tdd(../data/SumZero.tdd),
+    numericTypes:             tdd(../data/NumericTypes.tdd),
+    casthigh:                 tdd(../data/CastHigh.tdd)
 }
 freemarkerLinks: {
     includes: includes/

http://git-wip-us.apache.org/repos/asf/drill/blob/5efc7e68/exec/java-exec/src/main/codegen/data/CompareTypes.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/CompareTypes.tdd b/exec/java-exec/src/main/codegen/data/CompareTypes.tdd
deleted file mode 100644
index f384d52..0000000
--- a/exec/java-exec/src/main/codegen/data/CompareTypes.tdd
+++ /dev/null
@@ -1,21 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-# http:# www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-{
-  types: [
-    {comparables: ["Float4", "Float8", "Int", "BigInt", "NullableFloat4", "NullableFloat8", "NullableInt", "NullableBigInt"], mode: "fixed"}, 
-    {comparables: ["VarChar", "VarBinary", "NullableVarChar", "NullableVarBinary"], mode: "var"} 
-  ]
-} 

http://git-wip-us.apache.org/repos/asf/drill/blob/5efc7e68/exec/java-exec/src/main/codegen/data/ComparisonTypesDecimal.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/ComparisonTypesDecimal.tdd b/exec/java-exec/src/main/codegen/data/ComparisonTypesDecimal.tdd
new file mode 100644
index 0000000..423fe89
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/data/ComparisonTypesDecimal.tdd
@@ -0,0 +1,26 @@
+# 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.
+{
+  { decimalTypes: [
+      {name: "Decimal28Sparse", storage: "5"},
+      {name: "Decimal38Sparse", storage: "6"},
+      {name: "Decimal28Dense", storage: "4"},
+      {name: "Decimal38Dense", storage: "3"},
+      {name: "Decimal9", storage: "int"},
+      {name: "Decimal18", storage: "long"}
+    ]
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/5efc7e68/exec/java-exec/src/main/codegen/data/ComparisonTypesMain.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/ComparisonTypesMain.tdd b/exec/java-exec/src/main/codegen/data/ComparisonTypesMain.tdd
new file mode 100644
index 0000000..c4d924c
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/data/ComparisonTypesMain.tdd
@@ -0,0 +1,69 @@
+# 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.
+
+# Template data/configuration file for ComparisonFunctions.java for
+# comparison functions--currently, for primitive numeric types, for Bit/Boolean,
+# and for character and byte strings:
+
+# Each group (value in "typeGroups" array is for types that can be compared to each
+# other:
+# - "comparables" is types in group
+# - "mode" is comparison mode (case of code needed to compare):
+#   - "primitive" - comparable with Java primitive-type "<", "==", and ">" operators
+#   - "varString" - comparable with ByteFunctionHelpers.compare(...)
+
+{
+  typeGroups: [
+
+    # Group:  Drill "Bit"/SQL BOOLEAN (represented with Java primitive type)
+    {comparables: [
+      "Bit"
+      ],
+      mode: "primitive"},
+
+    # Group:  Numeric types cross-comparable with Java primitive-type operators:
+    {comparables: [
+      "Int",
+      "BigInt",
+      "Float4",
+      "Float8"
+      ],
+      mode: "primitive"},
+
+
+    # Date/time types other than IntervalDay or Interval (comparable with Java
+    # primitive-type operators):
+
+    {comparables: ["TimeStamp"], mode: "primitive"},
+    {comparables: ["Date"], mode: "primitive"},
+    {comparables: ["Time"], mode: "primitive"},
+    {comparables: ["TimeStampTZ"], mode: "primitive"},
+    {comparables: ["IntervalYear"], mode: "primitive"},
+
+    {comparables: ["IntervalDay"], mode: "intervalDay"},
+    {comparables: ["Interval"], mode: "intervalNameThis"},
+
+
+    # Group:  Variable-length string types whose values can be compared using
+    # ByteFunctionHelpers.compare(...)
+    {comparables: [
+      "VarChar",
+      "VarBinary"
+      ],
+      mode: "varString"}
+
+  ]
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/5efc7e68/exec/java-exec/src/main/codegen/data/DateTypes.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/DateTypes.tdd b/exec/java-exec/src/main/codegen/data/DateTypes.tdd
deleted file mode 100644
index 5802df0..0000000
--- a/exec/java-exec/src/main/codegen/data/DateTypes.tdd
+++ /dev/null
@@ -1,27 +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.
-
-{
-  dateTypes: [
-    {name: "Date"},
-    {name: "TimeStamp"},
-    {name: "TimeStampTZ"},
-    {name: "Time"},
-    {name: "Interval"},
-    {name: "IntervalYear"},
-    {name: "IntervalDay"}
-  ]
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/5efc7e68/exec/java-exec/src/main/codegen/data/DecimalTypes.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/DecimalTypes.tdd b/exec/java-exec/src/main/codegen/data/DecimalTypes.tdd
deleted file mode 100644
index 423fe89..0000000
--- a/exec/java-exec/src/main/codegen/data/DecimalTypes.tdd
+++ /dev/null
@@ -1,26 +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.
-{
-  { decimalTypes: [
-      {name: "Decimal28Sparse", storage: "5"},
-      {name: "Decimal38Sparse", storage: "6"},
-      {name: "Decimal28Dense", storage: "4"},
-      {name: "Decimal38Dense", storage: "3"},
-      {name: "Decimal9", storage: "int"},
-      {name: "Decimal18", storage: "long"}
-    ]
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/5efc7e68/exec/java-exec/src/main/codegen/templates/ComparisonFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/ComparisonFunctions.java b/exec/java-exec/src/main/codegen/templates/ComparisonFunctions.java
index 628277c..011d4d9 100644
--- a/exec/java-exec/src/main/codegen/templates/ComparisonFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/ComparisonFunctions.java
@@ -15,60 +15,143 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
+
+/**
+  NOTE: ComparisonFunctions.java does not contain/generate all comparison
+  functions.  DecimalFunctions.java and DateIntervalFunctions.java contain
+  some.
+*/
+<#-- TODO:  Refactor comparison code from DecimalFunctions.java and
+     DateIntervalFunctions.java into here (to eliminate duplicate template code
+     and so that ComparisonFunctions actually has all comparison functions. -->
+
+
 <@pp.dropOutputFile />
 
-<#macro compareBlock mode left right output nullCompare>
+<#macro intervalCompareBlock leftType rightType leftMonths leftDays leftMillis rightMonths rightDays rightMillis output>
+
+        org.joda.time.MutableDateTime leftDate  =
+            new org.joda.time.MutableDateTime(1970, 1, 1, 0, 0, 0, 0, org.joda.time.DateTimeZone.UTC);
+        org.joda.time.MutableDateTime rightDate =
+            new org.joda.time.MutableDateTime(1970, 1, 1, 0, 0, 0, 0, org.joda.time.DateTimeZone.UTC);
+
+        // Left and right date have the same starting point (epoch), add the interval period and compare the two
+        leftDate.addMonths(${leftMonths});
+        leftDate.addDays(${leftDays});
+        leftDate.add(${leftMillis});
+
+        rightDate.addMonths(${rightMonths});
+        rightDate.addDays(${rightDays});
+        rightDate.add(${rightMillis});
 
-outside:{
-  
+        long leftMS  = leftDate.getMillis();
+        long rightMS = rightDate.getMillis();
+
+        ${output} = leftMS < rightMS ? -1 : (leftMS > rightMS ? 1 : 0);
+
+</#macro>
+
+
+<#macro compareNullsSubblock leftType rightType output breakTarget nullCompare nullComparesHigh>
   <#if nullCompare>
-      <#if left?starts_with("Nullable")>
-        <#if right?starts_with("Nullable")>
+    <#if nullComparesHigh>
+      <#assign leftNullResult = 1> <#-- if only L is null and nulls are high, then "L > R" (1) -->
+      <#assign rightNullResult = -1>
+    <#else>
+      <#assign leftNullResult = -1> <#-- if only L is null and nulls are low, then "L < R" (-1) -->
+      <#assign rightNullResult = 1>
+    </#if>
+
+    <#if leftType?starts_with("Nullable")>
+      <#if rightType?starts_with("Nullable")>
         <#-- Both are nullable. -->
-        if(left.isSet == 0){
-          if(right.isSet == 0){
+        if ( left.isSet == 0 ) {
+          if ( right.isSet == 0 ) {
+            <#-- Both are null--result is "L = R". -->
             ${output} = 0;
-            break outside;
-          }else{
-            ${output} = 1;
-            break outside;
+            break ${breakTarget};
+          } else {
+            <#-- Only left is null--result is "L < R" or "L > R" per null ordering. -->
+            ${output} = ${leftNullResult};
+            break ${breakTarget};
           }
-        }else if(right.isSet == 0){
-          ${output} = -1;
-          break outside;
+        } else if ( right.isSet == 0 ) {
+          <#-- Only right is null--result is "L > R" or "L < R" per null ordering. -->
+          ${output} = ${rightNullResult};
+          break ${breakTarget};
         }
-        <#else>
+      <#else>
         <#-- Left is nullable but right is not. -->
-        if(left.isSet == 0){
-          ${output} = 1;
-          break outside;
+        if ( left.isSet == 0 ) {
+          <#-- Only left is null--result is "L < R" or "L > R" per null ordering. -->
+          ${output} = ${leftNullResult};
+          break ${breakTarget};
         }
-        </#if>
-    <#elseif right?starts_with("Nullable")>
-      if(right.isSet == 0){
-        ${output} = -1;
-        break outside;
-      }
       </#if>
+    <#elseif rightType?starts_with("Nullable")>
+      <#-- Left is not nullable but right is. -->
+      if ( right.isSet == 0 ) {
+        <#-- Only right is null--result is "L > R" or "L < R" per null ordering. -->
+        ${output} = ${rightNullResult};
+        break ${breakTarget};
+      }
     </#if>
-    
-    <#if mode == "var">
-      ${output} = org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.compare(left.buffer, left.start, left.end, right.buffer, right.start, right.end);
-    <#elseif mode == "fixed">
-      ${output} = left.value < right.value ? -1 : ((left.value == right.value)? 0 : 1);
-    </#if>    
-  
+  </#if>
 
-  
-  
+</#macro>
 
-}
+
+<#-- macro compareBlock: Generates block handling comparison, including NULL. -->
+
+<#-- Parameters: >
+<#-- - mode: selects case of comparison code -->
+<#-- - leftType: name of left argument's type  (e.g., NullableFloat4)  -->
+<#-- - rightType: name of right argument's type  -->
+<#-- - output: output variable name -->
+<#-- - nullCompare: whether to generate null-comparison code -->
+<#-- - nullComparesHigh:  whether NULL compares as the highest value or the lowest
+       value -->
+
+<#macro compareBlock mode leftType rightType output nullCompare nullComparesHigh>
+     outside:
+      {
+        <@compareNullsSubblock leftType=leftType rightType=rightType
+          output="out.value" breakTarget="outside" nullCompare=true nullComparesHigh=nullComparesHigh />
+
+    <#if mode == "primitive">
+      ${output} = left.value < right.value ? -1 : (left.value == right.value ? 0 : 1);
+    <#elseif mode == "varString">
+      ${output} = org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.compare(
+          left.buffer, left.start, left.end, right.buffer, right.start, right.end );
+    <#elseif mode == "intervalNameThis">
+      <@intervalCompareBlock leftType=leftType rightType=rightType
+        leftMonths ="left.months"  leftDays ="left.days"  leftMillis ="left.milliseconds"
+        rightMonths="right.months" rightDays="right.days" rightMillis="right.milliseconds"
+        output="${output}"/>
+    <#elseif mode == "intervalDay">
+      <@intervalCompareBlock leftType=leftType rightType=rightType
+        leftMonths ="0" leftDays ="left.days"  leftMillis ="left.milliseconds"
+        rightMonths="0" rightDays="right.days" rightMillis="right.milliseconds"
+        output="${output}"/>
+       <#-- TODO:  Refactor other comparison code to here. -->
+    <#else>
+      ${mode_HAS_BAD_VALUE}
+    </#if>
+
+      } // outside
 </#macro>
 
-<#list compare.types as type>
-<#list type.comparables as left>
-<#list type.comparables as right>
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/GCompare${left}${right}.java" />
+
+<#-- 1.  For each group of cross-comparable types: -->
+<#list comparisonTypesMain.typeGroups as typeGroup>
+
+<#-- 2.  For each pair of (cross-comparable) types in group: -->
+<#list typeGroup.comparables as leftTypeBase>
+<#list typeGroup.comparables as rightTypeBase>
+
+<#-- Generate one file for each pair of base types (includes Nullable cases). -->
+<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/GCompare${leftTypeBase}Vs${rightTypeBase}.java" />
 
 <#include "/@includes/license.ftl" />
 
@@ -79,6 +162,7 @@ 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.fn.impl.ByteFunctionHelpers;
 import org.apache.drill.exec.expr.holders.*;
 import org.apache.drill.exec.record.RecordBatch;
@@ -86,157 +170,249 @@ import javax.inject.Inject;
 import io.netty.buffer.DrillBuf;
 
 @SuppressWarnings("unused")
-public class GCompare${left}${right}{
+public class GCompare${leftTypeBase}Vs${rightTypeBase} {
 
-  @FunctionTemplate(name = "compare_to", scope = FunctionTemplate.FunctionScope.SIMPLE)
-  public static class Compare${left}${right} implements DrillSimpleFunc {
+<#-- 3.  For each combination of Nullable vs. not (of given non-nullable types):  -->
+<#list ["${leftTypeBase}", "Nullable${leftTypeBase}"] as leftType >
+<#list ["${rightTypeBase}", "Nullable${rightTypeBase}"] as rightType >
 
-      @Param ${left}Holder left;
-      @Param ${right}Holder right;
-      @Output IntHolder out;
 
-      public void setup(RecordBatch b) {}
+  <#-- 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,
+                    nulls = NullHandling.INTERNAL)
+  public static class GCompare${leftType}Vs${rightType}NullHigh implements DrillSimpleFunc {
 
-      public void eval() {
-        <@compareBlock mode=type.mode left=left right=right output="out.value" nullCompare=true />
-      }
+    @Param ${leftType}Holder left;
+    @Param ${rightType}Holder right;
+    @Output IntHolder out;
+
+    public void setup(RecordBatch b) {}
+
+    public void eval() {
+      <@compareBlock mode=typeGroup.mode leftType=leftType rightType=rightType
+                     output="out.value" nullCompare=true nullComparesHigh=true />
+    }
+  }
+
+  <#-- Comparison function for sorting and grouping relational operators
+       (not for comparison expression operators (=, <, etc.)). -->
+  @FunctionTemplate(name = FunctionGenerationHelper.COMPARE_TO_NULLS_LOW,
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.INTERNAL)
+  public static class GCompare${leftType}Vs${rightType}NullLow implements DrillSimpleFunc {
+
+    @Param ${leftType}Holder left;
+    @Param ${rightType}Holder right;
+    @Output IntHolder out;
+
+    public void setup(RecordBatch b) {}
+
+    public void eval() {
+      <@compareBlock mode=typeGroup.mode leftType=leftType rightType=rightType
+                     output="out.value" nullCompare=true nullComparesHigh=false />
+    }
   }
 
-  <#if ! left?starts_with("Nullable")  &&  ! right?starts_with("Nullable") >
+</#list>
+</#list> <#-- 3. Nullable combinations -->
+
 
-  @FunctionTemplate(names = {"less_than", "<"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-  public static class LessThan${left}${right} implements DrillSimpleFunc {
 
-      @Param ${left}Holder left;
-      @Param ${right}Holder right;
+  <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
+       not for sorting and grouping relational operators. -->
+  @FunctionTemplate(names = {"less_than", "<"},
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class LessThan${leftTypeBase}Vs${rightTypeBase} implements DrillSimpleFunc {
+
+      @Param ${leftTypeBase}Holder left;
+      @Param ${rightTypeBase}Holder right;
       @Output BitHolder out;
 
       public void setup(RecordBatch b) {}
 
       public void eval() {
-        
-        <#if type.mode == "var" >
-        int cmp;
-        <@compareBlock mode=type.mode left=left right=right output="cmp" nullCompare=false/>
-        out.value = cmp == -1 ? 1 : 0;
+
+        <#if typeGroup.mode == "primitive">
+          out.value = left.value < right.value ? 1 : 0;
+        <#elseif typeGroup.mode == "varString"
+            || typeGroup.mode == "intervalNameThis" || typeGroup.mode == "intervalDay" >
+          int cmp;
+          <@compareBlock mode=typeGroup.mode leftType=leftTypeBase rightType=rightTypeBase
+                         output="cmp" nullCompare=false nullComparesHigh=false />
+          out.value = cmp == -1 ? 1 : 0;
+        <#-- TODO:  Refactor other comparison code to here. -->
         <#else>
-        out.value = left.value < right.value ? 1 : 0;
+          ${mode_HAS_BAD_VALUE}
         </#if>
 
       }
   }
-  
-  @FunctionTemplate(names = {"less_than_or_equal_to", "<="}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-  public static class LessThanE${left}${right} implements DrillSimpleFunc {
 
-      @Param ${left}Holder left;
-      @Param ${right}Holder right;
+  <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
+       not for sorting and grouping relational operators. -->
+  @FunctionTemplate(names = {"less_than_or_equal_to", "<="},
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class LessThanEq${leftTypeBase}Vs${rightTypeBase} implements DrillSimpleFunc {
+
+      @Param ${leftTypeBase}Holder left;
+      @Param ${rightTypeBase}Holder right;
       @Output BitHolder out;
 
       public void setup(RecordBatch b) {}
 
       public void eval() {
-        
-        <#if type.mode == "var" >
-        int cmp;
-        <@compareBlock mode=type.mode left=left right=right output="cmp" nullCompare=false/>
-        out.value = cmp < 1 ? 1 : 0;
+
+        <#if typeGroup.mode == "primitive">
+          out.value = left.value <= right.value ? 1 : 0;
+        <#elseif typeGroup.mode == "varString"
+            || typeGroup.mode == "intervalNameThis" || typeGroup.mode == "intervalDay" >
+          int cmp;
+          <@compareBlock mode=typeGroup.mode leftType=leftTypeBase rightType=rightTypeBase
+                         output="cmp" nullCompare=false nullComparesHigh=false />
+          out.value = cmp < 1 ? 1 : 0;
+        <#-- TODO:  Refactor other comparison code to here. -->
         <#else>
-        out.value = left.value <= right.value ? 1 : 0;
+          ${mode_HAS_BAD_VALUE}
         </#if>
 
     }
   }
-  
-  @FunctionTemplate(names = {"greater_than", ">"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-  public static class GreaterThan${left}${right} implements DrillSimpleFunc {
 
-      @Param ${left}Holder left;
-      @Param ${right}Holder right;
+  <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
+       not for sorting and grouping relational operators. -->
+  @FunctionTemplate(names = {"greater_than", ">"},
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class GreaterThan${leftTypeBase}Vs${rightTypeBase} implements DrillSimpleFunc {
+
+      @Param ${leftTypeBase}Holder left;
+      @Param ${rightTypeBase}Holder right;
       @Output BitHolder out;
 
       public void setup(RecordBatch b) {}
 
       public void eval() {
-        
-        <#if type.mode == "var" >
-        int cmp;
-        <@compareBlock mode=type.mode left=left right=right output="cmp" nullCompare=false/>
-        out.value = cmp == 1 ? 1 : 0;
+
+        <#if typeGroup.mode == "primitive">
+          out.value = left.value > right.value ? 1 : 0;
+        <#elseif typeGroup.mode == "varString"
+            || typeGroup.mode == "intervalNameThis" || typeGroup.mode == "intervalDay" >
+          int cmp;
+          <@compareBlock mode=typeGroup.mode leftType=leftTypeBase rightType=rightTypeBase
+                         output="cmp" nullCompare=false nullComparesHigh=false />
+          out.value = cmp == 1 ? 1 : 0;
+        <#-- TODO:  Refactor other comparison code to here. -->
         <#else>
-        out.value = left.value > right.value ? 1 : 0;
+          ${mode_HAS_BAD_VALUE}
         </#if>
 
     }
   }
-  
-  @FunctionTemplate(names = {"greater_than_or_equal_to", ">="}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-  public static class GreaterThanE${left}${right} implements DrillSimpleFunc {
 
-      @Param ${left}Holder left;
-      @Param ${right}Holder right;
+  <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
+       not for sorting and grouping relational operators. -->
+  @FunctionTemplate(names = {"greater_than_or_equal_to", ">="},
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class GreaterThanEq${leftTypeBase}Vs${rightTypeBase} implements DrillSimpleFunc {
+
+      @Param ${leftTypeBase}Holder left;
+      @Param ${rightTypeBase}Holder right;
       @Output BitHolder out;
 
       public void setup(RecordBatch b) {}
 
       public void eval() {
-        
-        <#if type.mode == "var" >            
-        int cmp;
-        <@compareBlock mode=type.mode left=left right=right output="cmp" nullCompare=false/>
-        out.value = cmp > -1 ? 1 : 0;
+
+        <#if typeGroup.mode == "primitive">
+          out.value = left.value >= right.value ? 1 : 0;
+        <#elseif typeGroup.mode == "varString"
+            || typeGroup.mode == "intervalNameThis" || typeGroup.mode == "intervalDay" >
+          int cmp;
+          <@compareBlock mode=typeGroup.mode leftType=leftTypeBase rightType=rightTypeBase
+                         output="cmp" nullCompare=false nullComparesHigh=false />
+          out.value = cmp > -1 ? 1 : 0;
+        <#-- TODO:  Refactor other comparison code to here. -->
         <#else>
-        out.value = left.value >= right.value ? 1 : 0;
+          ${mode_HAS_BAD_VALUE}
         </#if>
 
       }
   }
-  
-  @FunctionTemplate(names = {"equal","==","="}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-  public static class Equals${left}${right} implements DrillSimpleFunc {
 
-      @Param ${left}Holder left;
-      @Param ${right}Holder right;
+  <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
+       not for sorting and grouping relational operators. -->
+  @FunctionTemplate(names = {"equal", "==", "="},
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class Equals${leftTypeBase}Vs${rightTypeBase} implements DrillSimpleFunc {
+
+      @Param ${leftTypeBase}Holder left;
+      @Param ${rightTypeBase}Holder right;
       @Output BitHolder out;
 
       public void setup(RecordBatch b) {}
 
       public void eval() {
-          <#if type.mode == "var" >
-          out.value = org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.equal(left.buffer, left.start, left.end, right.buffer, right.start, right.end);
-          <#else>
+
+        <#if typeGroup.mode == "primitive">
           out.value = left.value == right.value ? 1 : 0;
-          </#if>
+        <#elseif typeGroup.mode == "varString" >
+          out.value = org.apache.drill.exec.expr.fn.impl.ByteFunctionHelpers.equal(
+              left.buffer, left.start, left.end, right.buffer, right.start, right.end);
+        <#elseif typeGroup.mode == "intervalNameThis" || typeGroup.mode == "intervalDay" >
+          int cmp;
+          <@compareBlock mode=typeGroup.mode leftType=leftTypeBase rightType=rightTypeBase
+                         output="cmp" nullCompare=false nullComparesHigh=false />
+          out.value = cmp == 0 ? 1 : 0;
+        <#-- TODO:  Refactor other comparison code to here. -->
+        <#else>
+          ${mode_HAS_BAD_VALUE}
+        </#if>
 
       }
   }
-  
-  @FunctionTemplate(names = {"not_equal","<>","!="}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-  public static class NotEquals${left}${right} implements DrillSimpleFunc {
 
-      @Param ${left}Holder left;
-      @Param ${right}Holder right;
+  <#-- Comparison function for comparison expression operator (=, &lt;, etc.),
+       not for sorting and grouping relational operators. -->
+  @FunctionTemplate(names = {"not_equal", "<>", "!="},
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = NullHandling.NULL_IF_NULL)
+  public static class NotEquals${leftTypeBase}Vs${rightTypeBase} implements DrillSimpleFunc {
+
+      @Param ${leftTypeBase}Holder left;
+      @Param ${rightTypeBase}Holder right;
       @Output BitHolder out;
 
       public void setup(RecordBatch b) {}
 
       public void eval() {
-        
-        <#if type.mode == "var" >            
-        int cmp;
-        <@compareBlock mode=type.mode left=left right=right output="cmp" nullCompare=false/>
-        out.value = cmp == 0 ? 0 : 1;
+
+        <#if typeGroup.mode == "primitive">
+          out.value = left.value != right.value ? 1 : 0;
+        <#elseif typeGroup.mode == "varString"
+            || typeGroup.mode == "intervalNameThis" || typeGroup.mode == "intervalDay" >
+          int cmp;
+          <@compareBlock mode=typeGroup.mode leftType=leftTypeBase rightType=rightTypeBase
+                         output="cmp" nullCompare=false nullComparesHigh=false />
+          out.value = cmp == 0 ? 0 : 1;
+        <#-- TODO:  Refactor other comparison code to here. -->
         <#else>
-        out.value = left.value != right.value ? 1 : 0;
+          ${mode_HAS_BAD_VALUE}
         </#if>
-                
+
       }
   }
 
-  </#if>
-
 }
+
+
+</#list> <#-- 2.  Pair of types-->
 </#list>
-</#list>
-</#list>
+
+</#list> <#-- 1. Group -->
 

http://git-wip-us.apache.org/repos/asf/drill/blob/5efc7e68/exec/java-exec/src/main/codegen/templates/DateIntervalFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctions.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctions.java
deleted file mode 100644
index 8fe13bb..0000000
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctions.java
+++ /dev/null
@@ -1,355 +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 date.dateTypes as type>
-
-<#if type.name == "Date" || type.name == "TimeStamp" || type.name == "Time" || type.name == "TimeStampTZ" || type.name == "IntervalYear">       <#-- type.name is Date, TimeStamp, Time, TimeStampTZ or IntervalYear -->
-
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/GCompare${type.name}Functions.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl;
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionCostCategory;
-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;
-
-@SuppressWarnings("unused")
-public class GCompare${type.name}Functions {
-
-  @FunctionTemplate(name = "compare_to", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-  public static class Compare${type.name} implements DrillSimpleFunc {
-
-      @Param ${type.name}Holder left;
-      @Param ${type.name}Holder right;
-      @Output IntHolder out;
-
-      public void setup(RecordBatch b) {}
-
-      public void eval() {
-
-          out.value = (left.value > right.value) ? 1 : ((left.value < right.value) ? -1 : 0);
-      }
-  }
-
-  @FunctionTemplate(name = "less_than", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-  public static class LessThan${type.name} implements DrillSimpleFunc {
-
-      @Param ${type.name}Holder left;
-      @Param ${type.name}Holder right;
-      @Output BitHolder out;
-
-      public void setup(RecordBatch b) {}
-
-      public void eval() {
-        out.value = left.value < right.value ? 1 : 0;
-      }
-  }
-
-  @FunctionTemplate(name = "less_than_or_equal_to", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-  public static class LessThanE${type.name} implements DrillSimpleFunc {
-
-      @Param ${type.name}Holder left;
-      @Param ${type.name}Holder right;
-      @Output BitHolder out;
-
-      public void setup(RecordBatch b) {}
-
-      public void eval() {
-        out.value = left.value <= right.value ? 1 : 0;
-    }
-  }
-
-  @FunctionTemplate(name = "greater_than", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-  public static class GreaterThan${type.name} implements DrillSimpleFunc {
-
-      @Param ${type.name}Holder left;
-      @Param ${type.name}Holder right;
-      @Output BitHolder out;
-
-      public void setup(RecordBatch b) {}
-
-      public void eval() {
-        out.value = left.value > right.value ? 1 : 0;
-    }
-  }
-
-  @FunctionTemplate(name = "greater_than_or_equal_to", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-  public static class GreaterThanE${type.name} implements DrillSimpleFunc {
-
-      @Param ${type.name}Holder left;
-      @Param ${type.name}Holder right;
-      @Output BitHolder out;
-
-      public void setup(RecordBatch b) {}
-
-      public void eval() {
-        out.value = left.value >= right.value ? 1 : 0;
-      }
-  }
-
-  @FunctionTemplate(name = "equal", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-  public static class Equals${type.name} implements DrillSimpleFunc {
-
-      @Param ${type.name}Holder left;
-      @Param ${type.name}Holder right;
-      @Output BitHolder out;
-
-      public void setup(RecordBatch b) {}
-
-      public void eval() {
-        out.value = left.value == right.value ? 1 : 0;
-      }
-  }
-
-  @FunctionTemplate(name = "not_equal", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-  public static class NotEquals${type.name} implements DrillSimpleFunc {
-
-      @Param ${type.name}Holder left;
-      @Param ${type.name}Holder right;
-      @Output BitHolder out;
-
-      public void setup(RecordBatch b) {}
-
-      public void eval() {
-        out.value = left.value != right.value ? 1 : 0;
-      }
-  }
-
-}
-
-<#elseif type.name == "Interval" || type.name == "IntervalDay">
-
-<#macro intervalCompareBlock left right leftMonths leftDays leftMillis rightMonths rightDays rightMillis output>
-outside: {
-
-        org.joda.time.MutableDateTime leftDate  = new org.joda.time.MutableDateTime(1970, 1, 1, 0, 0, 0, 0, org.joda.time.DateTimeZone.UTC);
-        org.joda.time.MutableDateTime rightDate = new org.joda.time.MutableDateTime(1970, 1, 1, 0, 0, 0, 0, org.joda.time.DateTimeZone.UTC);
-
-        // Left and right date have the same starting point (epoch), add the interval period and compare the two
-        leftDate.addMonths(${leftMonths});
-        leftDate.addDays(${leftDays});
-        leftDate.add(${leftMillis});
-
-        rightDate.addMonths(${rightMonths});
-        rightDate.addDays(${rightDays});
-        rightDate.add(${rightMillis});
-
-        long leftMS  = leftDate.getMillis();
-        long rightMS = rightDate.getMillis();
-
-        ${output} = ((leftMS < rightMS) ? -1 : ((leftMS > rightMS) ? 1 : 0));
-
-    }
-</#macro>
-
-<#macro intervalConvertBlock left right leftMonths leftDays leftMillis rightMonths rightDays rightMillis>
-        org.joda.time.MutableDateTime leftDate  = new org.joda.time.MutableDateTime(1970, 1, 1, 0, 0, 0, 0, org.joda.time.DateTimeZone.UTC);
-        org.joda.time.MutableDateTime rightDate = new org.joda.time.MutableDateTime(1970, 1, 1, 0, 0, 0, 0, org.joda.time.DateTimeZone.UTC);
-
-        // Left and right date have the same starting point (epoch), add the interval period and compare the two
-        leftDate.addMonths(${leftMonths});
-        leftDate.addDays(${leftDays});
-        leftDate.add(${leftMillis});
-
-        rightDate.addMonths(${rightMonths});
-        rightDate.addDays(${rightDays});
-        rightDate.add(${rightMillis});
-
-        long leftMS  = leftDate.getMillis();
-        long rightMS = rightDate.getMillis();
-</#macro>
-
-<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/GCompare${type.name}Functions.java" />
-
-<#include "/@includes/license.ftl" />
-
-package org.apache.drill.exec.expr.fn.impl;
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionCostCategory;
-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;
-
-@SuppressWarnings("unused")
-public class GCompare${type.name}Functions {
-
-  @FunctionTemplate(name = "compare_to", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, 
-      costCategory = FunctionCostCategory.COMPLEX)
-  public static class GCCompare${type.name} implements DrillSimpleFunc {
-
-      @Param ${type.name}Holder left;
-      @Param ${type.name}Holder right;
-      @Output IntHolder out;
-
-      public void setup(RecordBatch b) {}
-
-      public void eval() {
-
-          <#if type.name == "Interval">
-          <@intervalCompareBlock left="left" right="right" leftMonths="left.months" leftDays="left.days" leftMillis="left.milliseconds" rightMonths="right.months" rightDays="right.days" rightMillis="right.milliseconds" output="out.value"/>
-          <#else>
-          <@intervalCompareBlock left="left" right="right" leftMonths="0" leftDays="left.days" leftMillis="left.milliseconds" rightMonths="0" rightDays="right.days" rightMillis="right.milliseconds" output="out.value"/>
-          </#if>
-
-      }
-  }
-
-  @FunctionTemplate(name = "less_than", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, 
-      costCategory = FunctionCostCategory.COMPLEX)
-  public static class LessThan${type.name} implements DrillSimpleFunc {
-
-      @Param ${type.name}Holder left;
-      @Param ${type.name}Holder right;
-      @Output BitHolder out;
-
-      public void setup(RecordBatch b) {}
-
-      public void eval() {
-
-          <#if type.name == "Interval">
-          <@intervalConvertBlock left="left" right="right" leftMonths="left.months" leftDays="left.days" leftMillis="left.milliseconds" rightMonths="right.months" rightDays="right.days" rightMillis="right.milliseconds"/>
-          <#else>
-          <@intervalConvertBlock left="left" right="right" leftMonths="0" leftDays="left.days" leftMillis="left.milliseconds" rightMonths="0" rightDays="right.days" rightMillis="right.milliseconds"/>
-          </#if>
-
-          out.value = leftMS < rightMS ? 1 : 0;
-      }
-  }
-
-  @FunctionTemplate(name = "less_than_or_equal_to", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, 
-      costCategory = FunctionCostCategory.COMPLEX)
-  public static class LessThanE${type.name} implements DrillSimpleFunc {
-
-      @Param ${type.name}Holder left;
-      @Param ${type.name}Holder right;
-      @Output BitHolder out;
-
-      public void setup(RecordBatch b) {}
-
-      public void eval() {
-
-          <#if type.name == "Interval">
-          <@intervalConvertBlock left="left" right="right" leftMonths="left.months" leftDays="left.days" leftMillis="left.milliseconds" rightMonths="right.months" rightDays="right.days" rightMillis="right.milliseconds"/>
-          <#else>
-          <@intervalConvertBlock left="left" right="right" leftMonths="0" leftDays="left.days" leftMillis="left.milliseconds" rightMonths="0" rightDays="right.days" rightMillis="right.milliseconds"/>
-          </#if>
-
-          out.value = leftMS <= rightMS ? 1 : 0;
-    }
-  }
-
-  @FunctionTemplate(name = "greater_than", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
-      costCategory = FunctionCostCategory.COMPLEX)
-  public static class GreaterThan${type.name} implements DrillSimpleFunc {
-
-      @Param ${type.name}Holder left;
-      @Param ${type.name}Holder right;
-      @Output BitHolder out;
-
-      public void setup(RecordBatch b) {}
-
-      public void eval() {
-
-          <#if type.name == "Interval">
-          <@intervalConvertBlock left="left" right="right" leftMonths="left.months" leftDays="left.days" leftMillis="left.milliseconds" rightMonths="right.months" rightDays="right.days" rightMillis="right.milliseconds"/>
-          <#else>
-          <@intervalConvertBlock left="left" right="right" leftMonths="0" leftDays="left.days" leftMillis="left.milliseconds" rightMonths="0" rightDays="right.days" rightMillis="right.milliseconds"/>
-          </#if>
-
-          out.value = leftMS > rightMS ? 1 : 0;
-    }
-  }
-
-  @FunctionTemplate(name = "greater_than_or_equal_to", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
-      costCategory = FunctionCostCategory.COMPLEX)
-  public static class GreaterThanE${type.name} implements DrillSimpleFunc {
-
-      @Param ${type.name}Holder left;
-      @Param ${type.name}Holder right;
-      @Output BitHolder out;
-
-      public void setup(RecordBatch b) {}
-
-      public void eval() {
-
-          <#if type.name == "Interval">
-          <@intervalConvertBlock left="left" right="right" leftMonths="left.months" leftDays="left.days" leftMillis="left.milliseconds" rightMonths="right.months" rightDays="right.days" rightMillis="right.milliseconds"/>
-          <#else>
-          <@intervalConvertBlock left="left" right="right" leftMonths="0" leftDays="left.days" leftMillis="left.milliseconds" rightMonths="0" rightDays="right.days" rightMillis="right.milliseconds"/>
-          </#if>
-
-          out.value = leftMS >= rightMS ? 1 : 0;
-      }
-  }
-
-  @FunctionTemplate(name = "equal", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
-      costCategory = FunctionCostCategory.COMPLEX)
-  public static class Equals${type.name} implements DrillSimpleFunc {
-
-      @Param ${type.name}Holder left;
-      @Param ${type.name}Holder right;
-      @Output BitHolder out;
-
-      public void setup(RecordBatch b) {}
-
-      public void eval() {
-          <#if type.name == "Interval">
-          <@intervalConvertBlock left="left" right="right" leftMonths="left.months" leftDays="left.days" leftMillis="left.milliseconds" rightMonths="right.months" rightDays="right.days" rightMillis="right.milliseconds"/>
-          <#else>
-          <@intervalConvertBlock left="left" right="right" leftMonths="0" leftDays="left.days" leftMillis="left.milliseconds" rightMonths="0" rightDays="right.days" rightMillis="right.milliseconds"/>
-          </#if>
-
-          out.value = leftMS == rightMS ? 1 : 0;
-      }
-  }
-
-  @FunctionTemplate(name = "not_equal", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL,
-      costCategory = FunctionCostCategory.COMPLEX)
-  public static class NotEquals${type.name} implements DrillSimpleFunc {
-
-      @Param ${type.name}Holder left;
-      @Param ${type.name}Holder right;
-      @Output BitHolder out;
-
-      public void setup(RecordBatch b) {}
-
-      public void eval() {
-          <#if type.name == "Interval">
-          <@intervalConvertBlock left="left" right="right" leftMonths="left.months" leftDays="left.days" leftMillis="left.milliseconds" rightMonths="right.months" rightDays="right.days" rightMillis="right.milliseconds"/>
-          <#else>
-          <@intervalConvertBlock left="left" right="right" leftMonths="0" leftDays="left.days" leftMillis="left.milliseconds" rightMonths="0" rightDays="right.days" rightMillis="right.milliseconds"/>
-          </#if>
-
-          out.value = leftMS != rightMS ? 1 : 0;
-      }
-  }
-}
-</#if>
-</#list>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/5efc7e68/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
index 960368a..84be392 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
@@ -160,6 +160,7 @@ public class CastEmptyString${type.from}ToNullable${type.to} implements DrillSim
             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
@@ -302,7 +303,7 @@ public class CastEmptyString${type.from}ToNullable${type.to} implements DrillSim
         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
          */
@@ -343,6 +344,9 @@ public class CastEmptyString${type.from}ToNullable${type.to} implements DrillSim
             }
         }
 
+        <#-- 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
          */
@@ -350,6 +354,13 @@ public class CastEmptyString${type.from}ToNullable${type.to} implements DrillSim
             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.)
+                 -->
         }
 
 
@@ -444,7 +455,7 @@ public class CastEmptyString${type.from}ToNullable${type.to} implements DrillSim
 
             int carry = 0;
             do {
-                // propogate the carry
+                // 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;