You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by jn...@apache.org on 2016/10/30 15:23:21 UTC

[1/5] drill git commit: DRILL-4964: Make Drill reconnect to hive metastore after hive metastore is restarted.

Repository: drill
Updated Branches:
  refs/heads/master 03e8f9f3e -> 1e6fa00cd


DRILL-4964: Make Drill reconnect to hive metastore after hive metastore is restarted.

Drill fails to connect to hive metastore after hive metastore is restarted unless drillbits are restarted.

Changes: For methods DrillHiveMetaStoreClient.getAllDatabases() and DrillHiveMetaStoreClient.getAllTables(),
the HiveMetaStoreClient wraps MetaException and TException both into MetaException. In case of connection
failure which is thrown as TException it is difficult to categorize at DrillClient level. The fix is to
close older connection and reconnect in case of these 2 api's. In all other cases proper set of exceptions
are thrown where we can handle each one individually.

close apache/drill#628


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

Branch: refs/heads/master
Commit: ca3733e38197f77c6bd7f8b7e03cdd9f5dd87f18
Parents: 03e8f9f
Author: Sorabh Hamirwasia <sh...@maprtech.com>
Authored: Tue Oct 25 18:52:01 2016 -0700
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Sat Oct 29 21:52:21 2016 -0700

----------------------------------------------------------------------
 .../exec/store/hive/DrillHiveMetaStoreClient.java | 18 +++++++++++++-----
 1 file changed, 13 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/ca3733e3/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/DrillHiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/DrillHiveMetaStoreClient.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/DrillHiveMetaStoreClient.java
index 2fe291b..d7ba659 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/DrillHiveMetaStoreClient.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/DrillHiveMetaStoreClient.java
@@ -204,8 +204,12 @@ public abstract class DrillHiveMetaStoreClient extends HiveMetaStoreClient {
     try {
       return mClient.getAllDatabases();
     } catch (MetaException e) {
-      throw e;
-    } catch (TException e) {
+      /*
+         HiveMetaStoreClient is encapsulating both the MetaException/TExceptions inside MetaException.
+         Since we don't have good way to differentiate, we will close older connection and retry once.
+         This is only applicable for getAllTables and getAllDatabases method since other methods are
+         properly throwing correct exceptions.
+      */
       logger.warn("Failure while attempting to get hive databases. Retries once.", e);
       try {
         mClient.close();
@@ -222,9 +226,13 @@ public abstract class DrillHiveMetaStoreClient extends HiveMetaStoreClient {
       throws TException {
     try {
       return mClient.getAllTables(dbName);
-    } catch (MetaException | UnknownDBException e) {
-      throw e;
-    } catch (TException e) {
+    } catch (MetaException e) {
+      /*
+         HiveMetaStoreClient is encapsulating both the MetaException/TExceptions inside MetaException.
+         Since we don't have good way to differentiate, we will close older connection and retry once.
+         This is only applicable for getAllTables and getAllDatabases method since other methods are
+         properly throwing correct exceptions.
+      */
       logger.warn("Failure while attempting to get hive tables. Retries once.", e);
       try {
         mClient.close();


[2/5] drill git commit: DRILL-4972: Remove setDaemon(true) call in WorkManager.StatusThread

Posted by jn...@apache.org.
DRILL-4972: Remove setDaemon(true) call in WorkManager.StatusThread

closes #633


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

Branch: refs/heads/master
Commit: c17b7556a7a319c4d8c25bf542e94f937f59b976
Parents: ca3733e
Author: Sudheesh Katkam <sk...@maprtech.com>
Authored: Fri Oct 28 09:08:23 2016 -0700
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Sat Oct 29 21:58:27 2016 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/drill/exec/work/WorkManager.java      | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/c17b7556/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
index ee11592..697616e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
@@ -295,7 +295,7 @@ public class WorkManager implements AutoCloseable {
    */
   private class StatusThread extends Thread {
     public StatusThread() {
-      setDaemon(true);
+      // assume this thread is created by a non-daemon thread
       setName("WorkManager.StatusThread");
     }
 


[3/5] drill git commit: DRILL-4974: Add missing null check in FindPartitionConditions.analyzeCall()

Posted by jn...@apache.org.
DRILL-4974: Add missing null check in FindPartitionConditions.analyzeCall()

close apache/drill#634


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

Branch: refs/heads/master
Commit: 00ee1236e54a30268358de2810208509efbea9f0
Parents: c17b755
Author: karthik <km...@maprtech.com>
Authored: Thu Oct 20 13:43:17 2016 -0700
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Sat Oct 29 22:02:39 2016 -0700

----------------------------------------------------------------------
 .../logical/partition/FindPartitionConditions.java    | 14 ++++++++------
 1 file changed, 8 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/00ee1236/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/FindPartitionConditions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/FindPartitionConditions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/FindPartitionConditions.java
index 620b6b2..da90065 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/FindPartitionConditions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/FindPartitionConditions.java
@@ -312,12 +312,14 @@ public class FindPartitionConditions extends RexVisitorImpl<Void> {
 
     if (callPushDirFilter == PushDirFilter.NO_PUSH) {
       OpState currentOp = opStack.peek();
-      if (currentOp.sqlOperator.getKind() != SqlKind.AND) {
-        clearChildren();
-      } else {
-        // AND op, check if we pushed some children
-        if (currentOp.children.size() > 0) {
-          callPushDirFilter = PushDirFilter.PARTIAL_PUSH;
+      if (currentOp != null) {
+        if (currentOp.sqlOperator.getKind() != SqlKind.AND) {
+          clearChildren();
+        } else {
+          // AND op, check if we pushed some children
+          if (currentOp.children.size() > 0) {
+            callPushDirFilter = PushDirFilter.PARTIAL_PUSH;
+          }
         }
       }
     }


[4/5] drill git commit: DRILL-4967: Adding template_name to source code generated using freemarker template.

Posted by jn...@apache.org.
DRILL-4967: Adding template_name to source code generated using freemarker template.

close apache/drill#629


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

Branch: refs/heads/master
Commit: 2081d76c9cfa33a796dba8a2676747edeccd9dfe
Parents: 00ee123
Author: Jinfeng Ni <jn...@apache.org>
Authored: Thu Sep 8 15:50:43 2016 -0700
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Sat Oct 29 22:02:50 2016 -0700

----------------------------------------------------------------------
 .../main/codegen/templates/AbstractRecordWriter.java    |  3 +++
 .../templates/AggrBitwiseLogicalTypeFunctions.java      |  2 +-
 .../src/main/codegen/templates/AggrTypeFunctions1.java  |  4 +++-
 .../src/main/codegen/templates/AggrTypeFunctions2.java  |  4 +++-
 .../src/main/codegen/templates/AggrTypeFunctions3.java  |  4 +++-
 .../src/main/codegen/templates/CastDateDate.java        |  3 +++
 .../src/main/codegen/templates/CastDateVarChar.java     |  4 ++++
 .../src/main/codegen/templates/CastFunctions.java       |  4 ++++
 .../main/codegen/templates/CastFunctionsSrcVarLen.java  |  3 +++
 .../templates/CastFunctionsSrcVarLenTargetVarLen.java   |  4 ++++
 .../codegen/templates/CastFunctionsTargetVarLen.java    |  3 +++
 exec/java-exec/src/main/codegen/templates/CastHigh.java |  3 +++
 .../main/codegen/templates/CastIntervalInterval.java    |  3 +++
 .../src/main/codegen/templates/CastIntervalVarChar.java |  3 +++
 .../src/main/codegen/templates/CastVarCharDate.java     |  3 +++
 .../src/main/codegen/templates/CastVarCharInterval.java |  3 +++
 .../src/main/codegen/templates/ComparisonFunctions.java |  4 ++++
 .../main/codegen/templates/ConvertToNullableHolder.java |  4 ++++
 .../codegen/templates/CorrelationTypeFunctions.java     |  4 ++++
 .../main/codegen/templates/CountAggregateFunctions.java |  5 ++++-
 .../src/main/codegen/templates/CovarTypeFunctions.java  |  7 ++++---
 .../codegen/templates/DateIntervalAggrFunctions1.java   |  5 ++++-
 .../DateDateArithmeticFunctions.java                    |  4 ++++
 .../DateIntervalArithmeticFunctions.java                |  7 +++++++
 .../DateToCharFunctions.java                            |  4 ++++
 .../DateTruncFunctions.java                             |  4 ++++
 .../DateIntervalFunctionTemplates/Extract.java          |  4 ++++
 .../IntervalIntervalArithmetic.java                     |  3 +++
 .../IntervalNumericArithmetic.java                      |  3 +++
 .../ToDateTypeFunctions.java                            |  4 ++++
 .../ToTimeStampFunction.java                            |  4 +++-
 .../Decimal/CastDecimalDenseDecimalSparse.java          |  3 +++
 .../codegen/templates/Decimal/CastDecimalFloat.java     |  8 ++++++++
 .../main/codegen/templates/Decimal/CastDecimalInt.java  |  7 +++++++
 .../codegen/templates/Decimal/CastDecimalSimilar.java   |  4 ++++
 .../Decimal/CastDecimalSparseDecimalDense.java          |  4 ++++
 .../codegen/templates/Decimal/CastDecimalVarchar.java   |  8 ++++++++
 .../codegen/templates/Decimal/CastDownwardDecimal.java  | 12 ++++++++++++
 .../codegen/templates/Decimal/CastFloatDecimal.java     |  4 ++++
 .../main/codegen/templates/Decimal/CastIntDecimal.java  |  4 ++++
 .../codegen/templates/Decimal/CastSrcDecimalSimple.java | 12 ++++++++++++
 .../codegen/templates/Decimal/CastVarCharDecimal.java   |  8 ++++++++
 .../templates/Decimal/DecimalAggrTypeFunctions1.java    |  4 ++++
 .../templates/Decimal/DecimalAggrTypeFunctions2.java    |  4 ++++
 .../codegen/templates/Decimal/DecimalFunctions.java     |  7 +++++++
 .../src/main/codegen/templates/DirectoryExplorers.java  |  4 ++--
 .../main/codegen/templates/EventBasedRecordWriter.java  |  3 +++
 .../main/codegen/templates/IntervalAggrFunctions2.java  |  4 ++++
 .../main/codegen/templates/MathFunctionTemplates.java   |  3 +++
 .../src/main/codegen/templates/MathFunctions.java       |  3 +++
 .../src/main/codegen/templates/NewValueFunctions.java   |  3 +++
 .../src/main/codegen/templates/NullOperator.java        |  3 +++
 .../codegen/templates/NumericFunctionsTemplates.java    |  3 +++
 .../main/codegen/templates/NumericToCharFunctions.java  |  3 +++
 .../src/main/codegen/templates/ParquetTypeHelper.java   |  4 ++++
 .../src/main/codegen/templates/RecordValueAccessor.java |  4 ++++
 .../src/main/codegen/templates/RecordWriter.java        |  4 ++++
 .../src/main/codegen/templates/SqlAccessors.java        |  3 +++
 .../src/main/codegen/templates/SumZeroAggr.java         |  4 ++--
 .../src/main/codegen/templates/TypeHelper.java          |  3 +++
 .../main/codegen/templates/VarCharAggrFunctions1.java   |  4 +++-
 .../src/main/codegen/templates/AbstractFieldReader.java |  3 +++
 exec/vector/src/main/codegen/templates/BaseReader.java  |  4 +++-
 .../src/main/codegen/templates/BasicTypeHelper.java     |  4 +++-
 .../src/main/codegen/templates/ComplexReaders.java      |  4 +++-
 exec/vector/src/main/codegen/templates/NullReader.java  |  4 +++-
 exec/vector/src/main/codegen/templates/UnionReader.java |  3 +++
 .../vector/src/main/codegen/templates/ValueHolders.java |  3 +++
 68 files changed, 269 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/AbstractRecordWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/AbstractRecordWriter.java b/exec/java-exec/src/main/codegen/templates/AbstractRecordWriter.java
index 13f7482..83b811e 100644
--- a/exec/java-exec/src/main/codegen/templates/AbstractRecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/AbstractRecordWriter.java
@@ -33,6 +33,9 @@ import org.apache.drill.exec.vector.complex.reader.FieldReader;
 import java.io.IOException;
 import java.lang.UnsupportedOperationException;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 public abstract class AbstractRecordWriter implements RecordWriter {
 
   private Accessor newPartitionVector;

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/AggrBitwiseLogicalTypeFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/AggrBitwiseLogicalTypeFunctions.java b/exec/java-exec/src/main/codegen/templates/AggrBitwiseLogicalTypeFunctions.java
index 3453058..b6d3e73 100644
--- a/exec/java-exec/src/main/codegen/templates/AggrBitwiseLogicalTypeFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/AggrBitwiseLogicalTypeFunctions.java
@@ -27,7 +27,7 @@
 <#-- A utility class that is used to generate java code for aggr functions bit_and / bit_or -->
 
 /*
- * This class is automatically generated from AggrBitwiseLogicalTypes.tdd using FreeMarker.
+ * This class is generated using freemarker and the ${.template_name} template.
  */
 
 package org.apache.drill.exec.expr.fn.impl.gaggr;

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
index 1360f66..b363cd1 100644
--- a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
@@ -24,7 +24,9 @@
 
 <#include "/@includes/license.ftl" />
 
-// Source code generated using FreeMarker template ${.template_name}
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 
 <#-- A utility class that is used to generate java code for aggr functions that maintain a single -->
 <#-- running counter to hold the result.  This includes: MIN, MAX, SUM. -->

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions2.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions2.java b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions2.java
index fe76cbe..7c396d9 100644
--- a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions2.java
+++ b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions2.java
@@ -24,7 +24,9 @@
 
 <#include "/@includes/license.ftl" />
 
-// Source code generated using FreeMarker template ${.template_name}
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 
 <#-- A utility class that is used to generate java code for aggr functions that maintain a sum -->
 <#-- and a running count.  For now, this includes: AVG. -->

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions3.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions3.java b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions3.java
index ee158bd..9b73418 100644
--- a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions3.java
+++ b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions3.java
@@ -24,7 +24,9 @@
 
 <#include "/@includes/license.ftl" />
 
-// Source code generated using FreeMarker template ${.template_name}
+/*
+ * 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 -->
 

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/CastDateDate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CastDateDate.java b/exec/java-exec/src/main/codegen/templates/CastDateDate.java
index c686917..21e9c21 100644
--- a/exec/java-exec/src/main/codegen/templates/CastDateDate.java
+++ b/exec/java-exec/src/main/codegen/templates/CastDateDate.java
@@ -41,6 +41,9 @@ import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
 import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 @FunctionTemplate(
         <#if type.to == "Date">

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/CastDateVarChar.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CastDateVarChar.java b/exec/java-exec/src/main/codegen/templates/CastDateVarChar.java
index 0c31b96..7005977 100644
--- a/exec/java-exec/src/main/codegen/templates/CastDateVarChar.java
+++ b/exec/java-exec/src/main/codegen/templates/CastDateVarChar.java
@@ -47,6 +47,10 @@ import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
 import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 @SuppressWarnings("unused")
 @FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL, 
   costCategory = FunctionCostCategory.COMPLEX)

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/CastFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CastFunctions.java b/exec/java-exec/src/main/codegen/templates/CastFunctions.java
index 4e0b4d2..f68da06 100644
--- a/exec/java-exec/src/main/codegen/templates/CastFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/CastFunctions.java
@@ -38,6 +38,10 @@ import javax.inject.Inject;
 import io.netty.buffer.DrillBuf;
 import org.apache.drill.exec.record.RecordBatch;
 
+/*
+ * 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{

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/CastFunctionsSrcVarLen.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CastFunctionsSrcVarLen.java b/exec/java-exec/src/main/codegen/templates/CastFunctionsSrcVarLen.java
index 85768de..6f2266b 100644
--- a/exec/java-exec/src/main/codegen/templates/CastFunctionsSrcVarLen.java
+++ b/exec/java-exec/src/main/codegen/templates/CastFunctionsSrcVarLen.java
@@ -44,6 +44,9 @@ import org.apache.drill.exec.record.RecordBatch;
 import javax.inject.Inject;
 import io.netty.buffer.DrillBuf;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 @FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc{

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/CastFunctionsSrcVarLenTargetVarLen.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CastFunctionsSrcVarLenTargetVarLen.java b/exec/java-exec/src/main/codegen/templates/CastFunctionsSrcVarLenTargetVarLen.java
index 15d4436..4524f5b 100644
--- a/exec/java-exec/src/main/codegen/templates/CastFunctionsSrcVarLenTargetVarLen.java
+++ b/exec/java-exec/src/main/codegen/templates/CastFunctionsSrcVarLenTargetVarLen.java
@@ -41,6 +41,10 @@ import org.apache.drill.exec.record.RecordBatch;
 import javax.inject.Inject;
 import io.netty.buffer.DrillBuf;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 @SuppressWarnings("unused")
 @FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc{

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/CastFunctionsTargetVarLen.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CastFunctionsTargetVarLen.java b/exec/java-exec/src/main/codegen/templates/CastFunctionsTargetVarLen.java
index 0d6b76a..2a318a6 100644
--- a/exec/java-exec/src/main/codegen/templates/CastFunctionsTargetVarLen.java
+++ b/exec/java-exec/src/main/codegen/templates/CastFunctionsTargetVarLen.java
@@ -42,6 +42,9 @@ import org.apache.drill.exec.record.RecordBatch;
 import javax.inject.Inject;
 import io.netty.buffer.DrillBuf;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 @FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc{

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/CastHigh.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CastHigh.java b/exec/java-exec/src/main/codegen/templates/CastHigh.java
index 082417c..a3f9c91 100644
--- a/exec/java-exec/src/main/codegen/templates/CastHigh.java
+++ b/exec/java-exec/src/main/codegen/templates/CastHigh.java
@@ -33,6 +33,9 @@ import javax.inject.Inject;
 import io.netty.buffer.DrillBuf;
 import org.apache.drill.exec.record.RecordBatch;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 public class CastHighFunctions {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CastHighFunctions.class);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/CastIntervalInterval.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CastIntervalInterval.java b/exec/java-exec/src/main/codegen/templates/CastIntervalInterval.java
index dcc8bc7..ab3e378 100644
--- a/exec/java-exec/src/main/codegen/templates/CastIntervalInterval.java
+++ b/exec/java-exec/src/main/codegen/templates/CastIntervalInterval.java
@@ -43,6 +43,9 @@ import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
 import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 @FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL)
 public class Cast${type.from}To${type.to} implements DrillSimpleFunc {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/CastIntervalVarChar.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CastIntervalVarChar.java b/exec/java-exec/src/main/codegen/templates/CastIntervalVarChar.java
index ebc6005..6dedca7 100644
--- a/exec/java-exec/src/main/codegen/templates/CastIntervalVarChar.java
+++ b/exec/java-exec/src/main/codegen/templates/CastIntervalVarChar.java
@@ -46,6 +46,9 @@ import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
 import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 @FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL)
 public class Cast${type.from}To${type.to} implements DrillSimpleFunc {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/CastVarCharDate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CastVarCharDate.java b/exec/java-exec/src/main/codegen/templates/CastVarCharDate.java
index 68f9b8c..4c51ba8 100644
--- a/exec/java-exec/src/main/codegen/templates/CastVarCharDate.java
+++ b/exec/java-exec/src/main/codegen/templates/CastVarCharDate.java
@@ -43,6 +43,9 @@ import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import javax.inject.Inject;
 import io.netty.buffer.DrillBuf;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 @FunctionTemplate(names = {"cast${type.to?upper_case}", "${type.alias}"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL, 
   costCategory = FunctionCostCategory.COMPLEX)

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/CastVarCharInterval.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CastVarCharInterval.java b/exec/java-exec/src/main/codegen/templates/CastVarCharInterval.java
index 1304f63..d8b2024 100644
--- a/exec/java-exec/src/main/codegen/templates/CastVarCharInterval.java
+++ b/exec/java-exec/src/main/codegen/templates/CastVarCharInterval.java
@@ -43,6 +43,9 @@ import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import javax.inject.Inject;
 import io.netty.buffer.DrillBuf;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 @FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL)
 public class Cast${type.from}To${type.to} implements DrillSimpleFunc {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/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 aed10a2..633bb56 100644
--- a/exec/java-exec/src/main/codegen/templates/ComparisonFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/ComparisonFunctions.java
@@ -169,6 +169,10 @@ import org.apache.drill.exec.record.RecordBatch;
 import javax.inject.Inject;
 import io.netty.buffer.DrillBuf;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 @SuppressWarnings("unused")
 public class GCompare${leftTypeBase}Vs${rightTypeBase} {
 

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/ConvertToNullableHolder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/ConvertToNullableHolder.java b/exec/java-exec/src/main/codegen/templates/ConvertToNullableHolder.java
index 045c919..cb67251 100644
--- a/exec/java-exec/src/main/codegen/templates/ConvertToNullableHolder.java
+++ b/exec/java-exec/src/main/codegen/templates/ConvertToNullableHolder.java
@@ -31,6 +31,10 @@ import org.apache.drill.exec.expr.DrillSimpleFunc;
 import org.apache.drill.exec.expr.annotations.*;
 import org.apache.drill.exec.expr.holders.*;
 import org.apache.drill.exec.record.RecordBatch;
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 
 <#if minor.class.startsWith("Decimal")>
 @FunctionTemplate(name = "convertToNullable${minor.class?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_MAX_SCALE, nulls = FunctionTemplate.NullHandling.INTERNAL)

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/CorrelationTypeFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CorrelationTypeFunctions.java b/exec/java-exec/src/main/codegen/templates/CorrelationTypeFunctions.java
index 2a94730..133a17a 100644
--- a/exec/java-exec/src/main/codegen/templates/CorrelationTypeFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/CorrelationTypeFunctions.java
@@ -61,6 +61,10 @@ import org.apache.drill.exec.expr.holders.NullableFloat4Holder;
 import org.apache.drill.exec.expr.holders.Float8Holder;
 import org.apache.drill.exec.expr.holders.Float4Holder;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 @SuppressWarnings("unused")
 
 public class ${aggrtype.className}Functions {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/CountAggregateFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CountAggregateFunctions.java b/exec/java-exec/src/main/codegen/templates/CountAggregateFunctions.java
index 71ac6a7..b32bb28 100644
--- a/exec/java-exec/src/main/codegen/templates/CountAggregateFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/CountAggregateFunctions.java
@@ -26,7 +26,6 @@ import java.lang.Override;
 
 <#include "/@includes/license.ftl" />
 
-// Source code generated using FreeMarker template ${.template_name}
 
 package org.apache.drill.exec.expr.fn.impl.gaggr;
 
@@ -38,6 +37,10 @@ import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.annotations.Workspace;
 import org.apache.drill.exec.expr.holders.*;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 @SuppressWarnings("unused")
 
 public class CountFunctions {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/CovarTypeFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/CovarTypeFunctions.java b/exec/java-exec/src/main/codegen/templates/CovarTypeFunctions.java
index e3d2f4a..08f623d 100644
--- a/exec/java-exec/src/main/codegen/templates/CovarTypeFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/CovarTypeFunctions.java
@@ -26,9 +26,6 @@
 
 <#-- A utility class that is used to generate java code for covariance functions -->
 
-/*
- * This class is automatically generated from CovarType.tdd using FreeMarker.
- */
 
 package org.apache.drill.exec.expr.fn.impl.gaggr;
 
@@ -61,6 +58,10 @@ import org.apache.drill.exec.expr.holders.NullableFloat4Holder;
 import org.apache.drill.exec.expr.holders.Float8Holder;
 import org.apache.drill.exec.expr.holders.Float4Holder;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 @SuppressWarnings("unused")
 
 public class ${aggrtype.className}Functions {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java b/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
index 7480842..b2a0525 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalAggrFunctions1.java
@@ -24,7 +24,6 @@
 
 <#include "/@includes/license.ftl" />
 
-// Source code generated using FreeMarker template ${.template_name}
 
 <#-- A utility class that is used to generate java code for aggr functions for Date, Time, Interval types -->
 <#--  that maintain a single running counter to hold the result.  This includes: MIN, MAX, SUM, COUNT. -->
@@ -39,6 +38,10 @@ import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.annotations.Workspace;
 import org.apache.drill.exec.expr.holders.*;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 @SuppressWarnings("unused")
 
 public class ${aggrtype.className}DateTypeFunctions {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateDateArithmeticFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateDateArithmeticFunctions.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateDateArithmeticFunctions.java
index 5fb2136..04eb327 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateDateArithmeticFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateDateArithmeticFunctions.java
@@ -39,6 +39,10 @@ import org.apache.drill.exec.record.RecordBatch;
 
 import io.netty.buffer.ByteBuf;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 @SuppressWarnings("unused")
 public class G${type}Arithmetic {
 @SuppressWarnings("unused")

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateIntervalArithmeticFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateIntervalArithmeticFunctions.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateIntervalArithmeticFunctions.java
index 50d869e..57e7f68 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateIntervalArithmeticFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateIntervalArithmeticFunctions.java
@@ -43,6 +43,9 @@ import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
 import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 
 public class ${datetype}${intervaltype}Functions {
 
@@ -159,6 +162,10 @@ import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
 import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 public class ${datetype}${intervaltype}Functions {
 <#macro timeIntervalArithmeticBlock left right temp op output intervaltype>
     <#if intervaltype == "Int" || intervaltype == "BigInt">

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateToCharFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateToCharFunctions.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateToCharFunctions.java
index 22c14de..6ef2d3e 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateToCharFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateToCharFunctions.java
@@ -42,6 +42,10 @@ import org.apache.drill.exec.record.RecordBatch;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.DrillBuf;
 
+/*
+ * 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)
 public class G${type}ToChar implements DrillSimpleFunc {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateTruncFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateTruncFunctions.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateTruncFunctions.java
index ad2f4c8..702f717 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateTruncFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/DateTruncFunctions.java
@@ -37,6 +37,10 @@ import org.apache.drill.exec.record.RecordBatch;
 
 import io.netty.buffer.ByteBuf;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 public class GDateTimeTruncateFunctions {
 
 <#list dateIntervalFunc.truncInputTypes as type> <#-- Start InputType Loop -->

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/Extract.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/Extract.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/Extract.java
index e205247..a64d655 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/Extract.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/Extract.java
@@ -29,6 +29,10 @@ import org.apache.drill.exec.expr.annotations.*;
 import org.apache.drill.exec.expr.holders.*;
 import org.apache.drill.exec.record.RecordBatch;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 public class ${className} {
 
 <#list extract.fromTypes as fromUnit>

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java
index 3a343e5..b248c35 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalIntervalArithmetic.java
@@ -41,6 +41,9 @@ import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
 import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 
 public class ${intervaltype}Functions {
 

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalNumericArithmetic.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalNumericArithmetic.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalNumericArithmetic.java
index 8198ce6..8a8e966 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalNumericArithmetic.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/IntervalNumericArithmetic.java
@@ -41,6 +41,9 @@ import org.joda.time.DateTimeZone;
 import org.joda.time.DateMidnight;
 import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 
 public class ${intervaltype}${numerictype}Functions {
 

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java
index c2e5a37..3217617 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToDateTypeFunctions.java
@@ -37,6 +37,10 @@ import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.*;
 import org.apache.drill.exec.record.RecordBatch;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 @FunctionTemplate(name = "to_${type?lower_case}" , scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
 public class GTo${type} implements DrillSimpleFunc {
 

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToTimeStampFunction.java
----------------------------------------------------------------------
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 4d1d093..44db9df 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToTimeStampFunction.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/ToTimeStampFunction.java
@@ -37,7 +37,9 @@ import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.holders.*;
 import org.apache.drill.exec.record.RecordBatch;
 
-// This class is generated using freemarker template ToTimeStampFunction.java
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 
 @FunctionTemplate(name = "to_timestamp" , scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
 public class G${numerics}ToTimeStamp implements DrillSimpleFunc {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalDenseDecimalSparse.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalDenseDecimalSparse.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalDenseDecimalSparse.java
index a110ed7..dc26889 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalDenseDecimalSparse.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalDenseDecimalSparse.java
@@ -41,6 +41,9 @@ 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.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL)

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalFloat.java
----------------------------------------------------------------------
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 c85cc4f..c393255 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalFloat.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalFloat.java
@@ -39,6 +39,10 @@ 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 {
@@ -77,6 +81,10 @@ 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 {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalInt.java
----------------------------------------------------------------------
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 318a8d8..a13f0e7 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalInt.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalInt.java
@@ -42,6 +42,9 @@ 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 {
@@ -82,6 +85,10 @@ 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 {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSimilar.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSimilar.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSimilar.java
index 5ab34ad..e0fba0a 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSimilar.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSimilar.java
@@ -44,6 +44,10 @@ 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.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc{

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSparseDecimalDense.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSparseDecimalDense.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSparseDecimalDense.java
index 62b3229..5ea8a8d 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSparseDecimalDense.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalSparseDecimalDense.java
@@ -42,6 +42,10 @@ 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.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc{

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java
index d9d15e1..c5ca782 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDecimalVarchar.java
@@ -46,6 +46,10 @@ import io.netty.buffer.SwappedByteBuf;
 
 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 {
@@ -127,6 +131,10 @@ 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 {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/Decimal/CastDownwardDecimal.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastDownwardDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastDownwardDecimal.java
index d492392..467405e 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastDownwardDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastDownwardDecimal.java
@@ -43,6 +43,10 @@ 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.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc {
@@ -116,6 +120,10 @@ 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.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc {
@@ -159,6 +167,10 @@ 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.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/Decimal/CastFloatDecimal.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastFloatDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastFloatDecimal.java
index f0775d8..7d7ec35 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastFloatDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastFloatDecimal.java
@@ -46,6 +46,10 @@ 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.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
index 368eb8b..eb01566 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastIntDecimal.java
@@ -42,6 +42,10 @@ 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.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java b/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java
index fd0aba4..fca6e99 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastSrcDecimalSimple.java
@@ -44,6 +44,10 @@ 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.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc {
@@ -148,6 +152,10 @@ 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.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc{
@@ -248,6 +256,10 @@ 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.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL)
 public class Cast${type.from}${type.to} implements DrillSimpleFunc {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/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 e2dbaea..bb011cb 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/CastVarCharDecimal.java
@@ -48,6 +48,10 @@ 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.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL)
@@ -221,6 +225,10 @@ import io.netty.buffer.ByteBuf;
 
 import java.nio.ByteBuffer;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 @SuppressWarnings("unused")
 <#if type.major == "VarCharDecimalComplex">
 @FunctionTemplate(name = "cast${type.to?upper_case}", scope = FunctionTemplate.FunctionScope.DECIMAL_CAST, nulls=NullHandling.NULL_IF_NULL)

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java
index 22339da..7737356 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions1.java
@@ -49,6 +49,10 @@ import org.apache.drill.exec.expr.holders.*;
 import org.apache.drill.exec.record.RecordBatch;
 import io.netty.buffer.ByteBuf;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 @SuppressWarnings("unused")
 
 public class Decimal${aggrtype.className}Functions {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions2.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions2.java b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions2.java
index 5a1b04a..f885e08 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions2.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalAggrTypeFunctions2.java
@@ -50,6 +50,10 @@ import io.netty.buffer.DrillBuf;
 import org.apache.drill.exec.record.RecordBatch;
 import io.netty.buffer.ByteBuf;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 @SuppressWarnings("unused")
 
 public class Decimal${aggrtype.className}Functions {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java
index efa6354..a4bf06a 100644
--- a/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/Decimal/DecimalFunctions.java
@@ -126,6 +126,9 @@ import io.netty.buffer.ByteBuf;
 import io.netty.buffer.DrillBuf;
 
 import java.nio.ByteBuffer;
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 
 @SuppressWarnings("unused")
 public class ${type.name}Functions {
@@ -1015,6 +1018,10 @@ 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 {
 

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/DirectoryExplorers.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DirectoryExplorers.java b/exec/java-exec/src/main/codegen/templates/DirectoryExplorers.java
index a47a541..f0101ca 100644
--- a/exec/java-exec/src/main/codegen/templates/DirectoryExplorers.java
+++ b/exec/java-exec/src/main/codegen/templates/DirectoryExplorers.java
@@ -33,8 +33,8 @@ import org.apache.drill.exec.expr.holders.VarCharHolder;
 
 import javax.inject.Inject;
 
-/**
- * This file is generated with Freemarker using the template exec/java-exec/src/main/codegen/templates/DirectoryExplorers.java
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
  */
 public class DirectoryExplorers {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DirectoryExplorers.class);

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java b/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
index 584f4b6..a2428a7 100644
--- a/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/EventBasedRecordWriter.java
@@ -38,6 +38,9 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 
 /** Reads records from the RecordValueAccessor and writes into RecordWriter. */
 public class EventBasedRecordWriter {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/IntervalAggrFunctions2.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/IntervalAggrFunctions2.java b/exec/java-exec/src/main/codegen/templates/IntervalAggrFunctions2.java
index 6745097..8e7fed5 100644
--- a/exec/java-exec/src/main/codegen/templates/IntervalAggrFunctions2.java
+++ b/exec/java-exec/src/main/codegen/templates/IntervalAggrFunctions2.java
@@ -39,6 +39,10 @@ import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.annotations.Workspace;
 import org.apache.drill.exec.expr.holders.*;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 @SuppressWarnings("unused")
 
 public class ${aggrtype.className}IntervalTypeFunctions {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/MathFunctionTemplates.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/MathFunctionTemplates.java b/exec/java-exec/src/main/codegen/templates/MathFunctionTemplates.java
index a1e0375..4bf7e16 100644
--- a/exec/java-exec/src/main/codegen/templates/MathFunctionTemplates.java
+++ b/exec/java-exec/src/main/codegen/templates/MathFunctionTemplates.java
@@ -44,6 +44,9 @@ import org.apache.drill.exec.expr.annotations.Workspace;
 import org.apache.drill.exec.expr.holders.*;
 import org.apache.drill.exec.record.RecordBatch;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 
 public class ${inputType.className}Functions {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/MathFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/MathFunctions.java b/exec/java-exec/src/main/codegen/templates/MathFunctions.java
index ba5b76c..4144fcf 100644
--- a/exec/java-exec/src/main/codegen/templates/MathFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/MathFunctions.java
@@ -45,6 +45,9 @@ import org.apache.drill.exec.expr.fn.impl.StringFunctions;
 import org.apache.drill.exec.expr.holders.*;
 import org.apache.drill.exec.record.RecordBatch;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 
 public class GMathFunctions{

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/NewValueFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/NewValueFunctions.java b/exec/java-exec/src/main/codegen/templates/NewValueFunctions.java
index d0e99d4..3b1d86f 100644
--- a/exec/java-exec/src/main/codegen/templates/NewValueFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/NewValueFunctions.java
@@ -34,6 +34,9 @@ import javax.inject.Inject;
 import io.netty.buffer.DrillBuf;
 import org.apache.drill.exec.record.RecordBatch;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 public class GNewValueFunctions {
 <#list vv.types as type>
 <#if type.major == "Fixed" || type.major = "Bit">

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/NullOperator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/NullOperator.java b/exec/java-exec/src/main/codegen/templates/NullOperator.java
index 1c9ad32..9a92aac 100644
--- a/exec/java-exec/src/main/codegen/templates/NullOperator.java
+++ b/exec/java-exec/src/main/codegen/templates/NullOperator.java
@@ -33,6 +33,9 @@ import org.apache.drill.exec.expr.annotations.*;
 import org.apache.drill.exec.expr.holders.*;
 import org.apache.drill.exec.record.RecordBatch;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 public class ${className} {
 
   @FunctionTemplate(names = {"isnull", "is null"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/NumericFunctionsTemplates.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/NumericFunctionsTemplates.java b/exec/java-exec/src/main/codegen/templates/NumericFunctionsTemplates.java
index 4d06c68..e8ff431 100644
--- a/exec/java-exec/src/main/codegen/templates/NumericFunctionsTemplates.java
+++ b/exec/java-exec/src/main/codegen/templates/NumericFunctionsTemplates.java
@@ -60,6 +60,9 @@ import org.apache.drill.exec.expr.holders.VarCharHolder;
 import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
 import org.apache.drill.exec.record.RecordBatch;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 
 public class ${numericFunc.className}Functions {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/NumericToCharFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/NumericToCharFunctions.java b/exec/java-exec/src/main/codegen/templates/NumericToCharFunctions.java
index 8bfe512..78f4f7d 100644
--- a/exec/java-exec/src/main/codegen/templates/NumericToCharFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/NumericToCharFunctions.java
@@ -45,6 +45,9 @@ import io.netty.buffer.DrillBuf;
 import java.text.NumberFormat;
 import java.text.DecimalFormat;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 @FunctionTemplate(name = "to_char", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
 public class G${type}ToChar implements DrillSimpleFunc {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java b/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
index ecd1d3e..7584b82 100644
--- a/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
+++ b/exec/java-exec/src/main/codegen/templates/ParquetTypeHelper.java
@@ -39,6 +39,10 @@ import org.apache.parquet.schema.Type.Repetition;
 import java.util.HashMap;
 import java.util.Map;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 public class ParquetTypeHelper {
   private static Map<MinorType,PrimitiveTypeName> typeMap;
   private static Map<DataMode,Repetition> modeMap;

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/RecordValueAccessor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/RecordValueAccessor.java b/exec/java-exec/src/main/codegen/templates/RecordValueAccessor.java
index b91d383..5e7d87c 100644
--- a/exec/java-exec/src/main/codegen/templates/RecordValueAccessor.java
+++ b/exec/java-exec/src/main/codegen/templates/RecordValueAccessor.java
@@ -28,6 +28,10 @@ import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.expr.holders.*;
 import org.apache.drill.exec.vector.*;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 /** Wrapper around VectorAccessible to iterate over the records and fetch fields within a record. */
 public class RecordValueAccessor {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordValueAccessor.class);

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/RecordWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/RecordWriter.java b/exec/java-exec/src/main/codegen/templates/RecordWriter.java
index 24a94c4..4c7a129 100644
--- a/exec/java-exec/src/main/codegen/templates/RecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/RecordWriter.java
@@ -31,6 +31,10 @@ import java.io.IOException;
 import java.lang.UnsupportedOperationException;
 import java.util.Map;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
 /** RecordWriter interface. */
 public interface RecordWriter {
 

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/SqlAccessors.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/SqlAccessors.java b/exec/java-exec/src/main/codegen/templates/SqlAccessors.java
index 283c209..49a91c2 100644
--- a/exec/java-exec/src/main/codegen/templates/SqlAccessors.java
+++ b/exec/java-exec/src/main/codegen/templates/SqlAccessors.java
@@ -35,6 +35,9 @@ package org.apache.drill.exec.vector.accessor;
 
 <#include "/@includes/vv_imports.ftl" />
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 public class ${name}Accessor extends AbstractSqlAccessor {
  <#if mode == "Nullable">

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/SumZeroAggr.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/SumZeroAggr.java b/exec/java-exec/src/main/codegen/templates/SumZeroAggr.java
index 87393f7..bc16b0c 100644
--- a/exec/java-exec/src/main/codegen/templates/SumZeroAggr.java
+++ b/exec/java-exec/src/main/codegen/templates/SumZeroAggr.java
@@ -24,8 +24,8 @@
 
 <#include "/@includes/license.ftl" />
 
-/* 
- * This class is automatically generated from SumZero.tdd using FreeMarker.
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
  */
 
 package org.apache.drill.exec.expr.fn.impl.gaggr;

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/TypeHelper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/TypeHelper.java b/exec/java-exec/src/main/codegen/templates/TypeHelper.java
index 3f7b6e6..8390e30 100644
--- a/exec/java-exec/src/main/codegen/templates/TypeHelper.java
+++ b/exec/java-exec/src/main/codegen/templates/TypeHelper.java
@@ -35,6 +35,9 @@ import org.apache.drill.exec.vector.accessor.*;
 import org.apache.drill.exec.vector.complex.RepeatedMapVector;
 import org.apache.drill.exec.util.CallBack;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 public class TypeHelper extends BasicTypeHelper {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeHelper.class);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java b/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
index 11b20b1..dd73f79 100644
--- a/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/VarCharAggrFunctions1.java
@@ -24,7 +24,6 @@
 
 <#include "/@includes/license.ftl" />
 
-// Source code generated using FreeMarker template ${.template_name}
 
 <#-- A utility class that is used to generate java code for aggr functions that maintain a single -->
 <#-- running counter to hold the result.  This includes: MIN, MAX, COUNT. -->
@@ -46,6 +45,9 @@ import org.apache.drill.exec.record.RecordBatch;
 
 import io.netty.buffer.ByteBuf;
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 
 public class ${aggrtype.className}VarBytesFunctions {

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/vector/src/main/codegen/templates/AbstractFieldReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/AbstractFieldReader.java b/exec/vector/src/main/codegen/templates/AbstractFieldReader.java
index 2b7b305..289aa60 100644
--- a/exec/vector/src/main/codegen/templates/AbstractFieldReader.java
+++ b/exec/vector/src/main/codegen/templates/AbstractFieldReader.java
@@ -26,6 +26,9 @@ package org.apache.drill.exec.vector.complex.impl;
 
 <#include "/@includes/vv_imports.ftl" />
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 abstract class AbstractFieldReader extends AbstractBaseReader implements FieldReader{
   

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/vector/src/main/codegen/templates/BaseReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/BaseReader.java b/exec/vector/src/main/codegen/templates/BaseReader.java
index 78f32f4..771f9b2 100644
--- a/exec/vector/src/main/codegen/templates/BaseReader.java
+++ b/exec/vector/src/main/codegen/templates/BaseReader.java
@@ -27,7 +27,9 @@ package org.apache.drill.exec.vector.complex.reader;
 <#include "/@includes/vv_imports.ftl" />
 
 
-
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 public interface BaseReader extends Positionable{
   MajorType getType();

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/vector/src/main/codegen/templates/BasicTypeHelper.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/BasicTypeHelper.java b/exec/vector/src/main/codegen/templates/BasicTypeHelper.java
index a618cfd..452c331 100644
--- a/exec/vector/src/main/codegen/templates/BasicTypeHelper.java
+++ b/exec/vector/src/main/codegen/templates/BasicTypeHelper.java
@@ -31,7 +31,9 @@ import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.vector.complex.RepeatedMapVector;
 import org.apache.drill.exec.util.CallBack;
-
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 public class BasicTypeHelper {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicTypeHelper.class);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/vector/src/main/codegen/templates/ComplexReaders.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/ComplexReaders.java b/exec/vector/src/main/codegen/templates/ComplexReaders.java
index 607b71d..d662a6f 100644
--- a/exec/vector/src/main/codegen/templates/ComplexReaders.java
+++ b/exec/vector/src/main/codegen/templates/ComplexReaders.java
@@ -46,7 +46,9 @@ import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
 package org.apache.drill.exec.vector.complex.impl;
 
 <#include "/@includes/vv_imports.ftl" />
-
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 public class ${nullMode}${name}ReaderImpl extends AbstractFieldReader {
   

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/vector/src/main/codegen/templates/NullReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/NullReader.java b/exec/vector/src/main/codegen/templates/NullReader.java
index 62aa33e..4da7514 100644
--- a/exec/vector/src/main/codegen/templates/NullReader.java
+++ b/exec/vector/src/main/codegen/templates/NullReader.java
@@ -28,7 +28,9 @@ package org.apache.drill.exec.vector.complex.impl;
 
 import org.apache.drill.common.types.TypeProtos;
 
-
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 public class NullReader extends AbstractBaseReader implements FieldReader{
   

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/vector/src/main/codegen/templates/UnionReader.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/UnionReader.java b/exec/vector/src/main/codegen/templates/UnionReader.java
index 46a32ee..58485dd 100644
--- a/exec/vector/src/main/codegen/templates/UnionReader.java
+++ b/exec/vector/src/main/codegen/templates/UnionReader.java
@@ -29,6 +29,9 @@ package org.apache.drill.exec.vector.complex.impl;
 
 <#include "/@includes/vv_imports.ftl" />
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 @SuppressWarnings("unused")
 public class UnionReader extends AbstractFieldReader {
 

http://git-wip-us.apache.org/repos/asf/drill/blob/2081d76c/exec/vector/src/main/codegen/templates/ValueHolders.java
----------------------------------------------------------------------
diff --git a/exec/vector/src/main/codegen/templates/ValueHolders.java b/exec/vector/src/main/codegen/templates/ValueHolders.java
index 4151bbe..11607b4 100644
--- a/exec/vector/src/main/codegen/templates/ValueHolders.java
+++ b/exec/vector/src/main/codegen/templates/ValueHolders.java
@@ -29,6 +29,9 @@ package org.apache.drill.exec.expr.holders;
 
 <#include "/@includes/vv_imports.ftl" />
 
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
 public final class ${className} implements ValueHolder{
   
   public static final MajorType TYPE = Types.${mode.name?lower_case}(MinorType.${minor.class?upper_case});


[5/5] drill git commit: DRILL-4884: Fix IOB exception in limit n query when n is beyond 65535.

Posted by jn...@apache.org.
DRILL-4884: Fix IOB exception in limit n query when n is beyond 65535.

close apache/drill#584


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

Branch: refs/heads/master
Commit: 1e6fa00cd4b0b1db41614749f6d12c03f0ca7990
Parents: 2081d76
Author: hongze.zhz <ho...@alibaba-inc.com>
Authored: Fri Sep 9 16:19:16 2016 +0800
Committer: Jinfeng Ni <jn...@apache.org>
Committed: Sat Oct 29 22:12:36 2016 -0700

----------------------------------------------------------------------
 .../physical/impl/limit/LimitRecordBatch.java   |   4 ++--
 .../java/org/apache/drill/TestBugFixes.java     |  21 +++++++++++++++++++
 .../limit_test_parquet/test0_0_0.parquet        | Bin 0 -> 428815 bytes
 3 files changed, 23 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/1e6fa00c/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
index 176ee17..08ffc0b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
@@ -163,8 +163,8 @@ public class LimitRecordBatch extends AbstractSingleRecordBatch<Limit> {
     }
 
     int svIndex = 0;
-    for(char i = (char) offset; i < fetch; svIndex++, i++) {
-      outgoingSv.setIndex(svIndex, i);
+    for(int i = offset; i < fetch; svIndex++, i++) {
+      outgoingSv.setIndex(svIndex, (char) i);
     }
     outgoingSv.setRecordCount(svIndex);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/1e6fa00c/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java b/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java
index 5b736bc..03b1b61 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java
@@ -17,12 +17,17 @@
  */
 package org.apache.drill;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.util.TestTools;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
 public class TestBugFixes extends BaseTestQuery {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestBugFixes.class);
   private static final String WORKING_PATH = TestTools.getWorkingPath();
@@ -200,4 +205,20 @@ public class TestBugFixes extends BaseTestQuery {
             .baselineValues("M", 554L, 11.9)
             .build().run();
   }
+
+  @Test
+  public void testDRILL4884() throws Exception {
+    int limit = 65536;
+    ImmutableList.Builder<Map<String, Object>> baselineBuilder = ImmutableList.builder();
+    for (int i = 0; i < limit; i++) {
+      baselineBuilder.add(Collections.<String, Object>singletonMap("`id`", String.valueOf(i + 1)));
+    }
+    List<Map<String, Object>> baseline = baselineBuilder.build();
+
+    testBuilder()
+            .sqlQuery(String.format("select id from dfs_test.`%s/bugs/DRILL-4884/limit_test_parquet/test0_0_0.parquet` group by id limit %s", TEST_RES_PATH, limit))
+            .unOrdered()
+            .baselineRecords(baseline)
+            .go();
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/1e6fa00c/exec/java-exec/src/test/resources/bugs/DRILL-4884/limit_test_parquet/test0_0_0.parquet
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/bugs/DRILL-4884/limit_test_parquet/test0_0_0.parquet b/exec/java-exec/src/test/resources/bugs/DRILL-4884/limit_test_parquet/test0_0_0.parquet
new file mode 100644
index 0000000..15b1c25
Binary files /dev/null and b/exec/java-exec/src/test/resources/bugs/DRILL-4884/limit_test_parquet/test0_0_0.parquet differ