You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by gp...@apache.org on 2019/02/28 21:06:06 UTC

[drill] branch master updated (4627973 -> 469be17)

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

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


    from 4627973  DRILL-6950: Row set-based scan framework
     new 3233d8a  DRILL-1328: Support table statistics
     new 469be17  DRILL-1328: Support table statistics - Part 2

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../exec/store/mapr/db/MapRDBFormatPlugin.java     |   16 +
 .../drill/exec/store/mapr/db/MapRDBGroupScan.java  |    1 +
 .../store/mapr/streams/StreamsFormatPlugin.java    |   18 +
 .../exec/store/syslog/SyslogFormatPlugin.java      |   19 +
 .../native/client/src/protobuf/UserBitShared.pb.cc |   15 +-
 .../native/client/src/protobuf/UserBitShared.pb.h  |    7 +-
 .../drill/exec/store/hbase/HBaseGroupScan.java     |    1 +
 .../org/apache/drill/exec/store/hive/HiveScan.java |    1 +
 .../drill/exec/store/jdbc/JdbcGroupScan.java       |    9 +-
 .../org/apache/drill/exec/store/jdbc/JdbcPrel.java |    9 +-
 .../drill/exec/store/jdbc/JdbcRecordReader.java    |    7 +-
 .../apache/drill/exec/store/jdbc/JdbcSubScan.java  |    9 +-
 .../drill/exec/store/kafka/KafkaGroupScan.java     |    1 +
 .../drill/exec/store/kudu/KuduGroupScan.java       |    1 +
 .../drill/exec/store/mongo/MongoGroupScan.java     |    1 +
 .../exec/store/openTSDB/OpenTSDBGroupScan.java     |    1 +
 exec/java-exec/pom.xml                             |    5 +
 exec/java-exec/src/main/codegen/data/Parser.tdd    |    8 +-
 .../src/main/codegen/includes/parserImpls.ftl      |   48 +-
 .../codegen/templates/AbstractRecordWriter.java    |    5 +
 .../templates/JsonBaseStatisticsRecordWriter.java  |  139 +
 .../src/main/codegen/templates/RecordWriter.java   |    6 +-
 ...cordWriter.java => StatisticsRecordWriter.java} |   44 +-
 .../templates/StatisticsRecordWriterImpl.java      |  119 +
 .../java/org/apache/drill/exec/ExecConstants.java  |   42 +-
 .../apache/drill/exec/dotdrill/DotDrillType.java   |    6 +-
 .../apache/drill/exec/dotdrill/DotDrillUtil.java   |   14 +-
 .../drill/exec/expr/fn/impl/MathFunctions.java     |   28 +
 .../exec/expr/fn/impl/StatisticsAggrFunctions.java | 5740 ++++++++++++++++++++
 .../apache/drill/exec/ops/BaseFragmentContext.java |   24 +-
 .../apache/drill/exec/ops/ContextInformation.java  |    1 +
 .../org/apache/drill/exec/ops/FragmentContext.java |   29 +-
 .../apache/drill/exec/ops/FragmentContextImpl.java |   20 +-
 .../org/apache/drill/exec/ops/QueryContext.java    |   34 +
 .../org/apache/drill/exec/ops/UdfUtilities.java    |    8 +-
 .../exec/physical/base/AbstractDbGroupScan.java    |    2 -
 .../exec/physical/base/AbstractGroupScan.java      |    5 +
 .../physical/base/AbstractPhysicalVisitor.java     |   18 +
 .../apache/drill/exec/physical/base/GroupScan.java |   12 +-
 .../exec/physical/base/PhysicalOperatorUtil.java   |   26 +-
 .../drill/exec/physical/base/PhysicalVisitor.java  |    6 +
 .../drill/exec/physical/base/SchemalessScan.java   |    6 +
 .../config/StatisticsAggregate.java}               |   48 +-
 .../exec/physical/config/StatisticsMerge.java      |   69 +
 .../config/UnpivotMaps.java}                       |   45 +-
 ...Batch.java => StatisticsWriterRecordBatch.java} |   70 +-
 .../exec/physical/impl/WriterRecordBatch.java      |    3 +
 .../physical/impl/aggregate/InternalBatch.java     |    6 +
 .../physical/impl/aggregate/StreamingAggBatch.java |   18 +-
 .../impl/statistics/AbstractMergedStatistic.java   |   48 +
 .../impl/statistics/AvgWidthMergedStatistic.java   |  150 +
 .../impl/statistics/CntDupsMergedStatistic.java    |  103 +
 .../impl/statistics/ColTypeMergedStatistic.java    |   87 +
 .../impl/statistics/ColumnMergedStatistic.java     |   68 +
 .../impl/statistics/HLLMergedStatistic.java        |  130 +
 .../physical/impl/statistics/MergedStatistic.java  |   62 +
 .../impl/statistics/MergedStatisticFactory.java    |   62 +
 .../impl/statistics/NDVMergedStatistic.java        |  198 +
 .../impl/statistics/NNRowCountMergedStatistic.java |   95 +
 .../impl/statistics/RowCountMergedStatistic.java   |   96 +
 .../exec/physical/impl/statistics/Statistic.java   |   44 +
 .../impl/statistics/StatisticsAggBatch.java        |  249 +
 .../impl/statistics/StatisticsAggBatchCreator.java |   37 +
 .../impl/statistics/StatisticsMergeBatch.java      |  405 ++
 .../statistics/StatisticsMergeBatchCreator.java    |   38 +
 .../impl/unpivot/UnpivotMapsBatchCreator.java      |   38 +
 .../impl/unpivot/UnpivotMapsRecordBatch.java       |  280 +
 .../planner/FileSystemPartitionDescriptor.java     |   11 +-
 .../apache/drill/exec/planner/PlannerPhase.java    |    2 +
 .../exec/planner/common/DrillAggregateRelBase.java |    9 +
 .../exec/planner/common/DrillFilterRelBase.java    |    4 +-
 .../exec/planner/common/DrillJoinRelBase.java      |   53 +-
 .../drill/exec/planner/common/DrillRelOptUtil.java |  102 +
 .../exec/planner/common/DrillScanRelBase.java      |   14 +
 .../drill/exec/planner/common/DrillStatsTable.java |  424 ++
 .../planner/cost/DrillRelMdDistinctRowCount.java   |  179 +-
 .../exec/planner/cost/DrillRelMdRowCount.java      |   62 +-
 .../exec/planner/cost/DrillRelMdSelectivity.java   |  201 +-
 .../exec/planner/logical/DrillAnalyzeRel.java      |   77 +
 .../drill/exec/planner/logical/DrillTable.java     |   11 +
 .../drill/exec/planner/physical/AggPruleBase.java  |    2 +-
 .../drill/exec/planner/physical/AnalyzePrule.java  |  131 +
 .../planner/physical/ConvertCountToDirectScan.java |    4 +-
 .../exec/planner/physical/DirectScanPrel.java      |   33 +-
 .../exec/planner/physical/DirectScanPrule.java     |    3 -
 .../exec/planner/physical/NestedLoopJoinPrule.java |   20 +-
 .../exec/planner/physical/PlannerSettings.java     |    6 +
 .../drill/exec/planner/physical/ScanPrel.java      |   29 +-
 .../drill/exec/planner/physical/StatsAggPrel.java  |   83 +
 .../exec/planner/physical/StatsMergePrel.java      |   89 +
 .../exec/planner/physical/UnpivotMapsPrel.java     |   85 +
 .../planner/physical/visitor/BasePrelVisitor.java  |    6 +
 .../exec/planner/physical/visitor/PrelVisitor.java |    2 +
 .../physical/visitor/PrelVisualizerVisitor.java    |    8 +
 .../drill/exec/planner/sql/DrillSqlWorker.java     |   20 +
 .../drill/exec/planner/sql/SchemaUtilites.java     |   19 +
 .../planner/sql/handlers/AnalyzeTableHandler.java  |  284 +
 .../planner/sql/handlers/DefaultSqlHandler.java    |    5 +-
 .../planner/sql/handlers/FindLimit0Visitor.java    |    8 +-
 .../sql/parser/CompoundIdentifierConverter.java    |    8 +-
 .../sql/parser/DrillCompoundIdentifier.java        |   27 +-
 .../exec/planner/sql/parser/SqlAnalyzeTable.java   |  156 +
 .../impl/DrillParserWithCompoundIdConverter.java   |    9 +-
 .../exec/server/options/SystemOptionManager.java   |    7 +-
 .../drill/exec/server/options/TypeValidators.java  |   19 +
 .../apache/drill/exec/store/AbstractSchema.java    |   34 +
 .../org/apache/drill/exec/store/RecordReader.java  |   11 +
 .../apache/drill/exec/store/SubSchemaWrapper.java  |   15 +
 .../drill/exec/store/avro/AvroFormatPlugin.java    |   18 +
 .../exec/store/dfs/FileSystemSchemaFactory.java    |   15 +
 .../apache/drill/exec/store/dfs/FormatPlugin.java  |   11 +-
 .../exec/store/dfs/WorkspaceSchemaFactory.java     |  123 +-
 .../exec/store/dfs/easy/EasyFormatPlugin.java      |   17 +-
 .../drill/exec/store/dfs/easy/EasyGroupScan.java   |    1 +
 .../drill/exec/store/direct/DirectSubScan.java     |    9 +-
 .../exec/store/easy/json/JSONFormatPlugin.java     |  100 +-
 .../exec/store/easy/json/JsonRecordWriter.java     |    1 +
 .../easy/json/JsonStatisticsRecordWriter.java      |  482 ++
 .../sequencefile/SequenceFileFormatPlugin.java     |   17 +
 .../exec/store/easy/text/TextFormatPlugin.java     |   17 +
 .../exec/store/httpd/HttpdLogFormatPlugin.java     |   17 +
 .../drill/exec/store/image/ImageFormatPlugin.java  |   18 +
 .../drill/exec/store/log/LogFormatPlugin.java      |   19 +
 .../store/parquet/AbstractParquetGroupScan.java    |    1 +
 .../exec/store/parquet/ParquetFormatPlugin.java    |   46 +-
 .../drill/exec/store/pcap/PcapFormatPlugin.java    |   18 +
 .../exec/store/pcapng/PcapngFormatPlugin.java      |   19 +
 .../drill/exec/store/pojo/PojoRecordReader.java    |   13 +-
 .../drill/exec/store/sys/SystemTableScan.java      |    1 +
 .../java/org/apache/drill/exec/util/Utilities.java |   13 +-
 .../java-exec/src/main/resources/drill-module.conf |    9 +-
 .../java/org/apache/drill/TestSelectivity.java     |    4 +-
 .../impersonation/TestImpersonationMetadata.java   |   43 +-
 .../exec/physical/impl/TestLocalExchange.java      |    3 +-
 .../impl/partitionsender/TestPartitionSender.java  |    3 +-
 .../apache/drill/exec/pop/TestFragmentChecker.java |    3 +-
 .../org/apache/drill/exec/sql/TestAnalyze.java     |  390 ++
 .../drill/exec/sql/TestSqlBracketlessSyntax.java   |    2 +-
 .../org/apache/drill/test/DrillTestWrapper.java    |   36 +-
 .../java/org/apache/drill/test/TestBuilder.java    |   10 +-
 exec/jdbc-all/pom.xml                              |    4 +
 .../apache/drill/common/logical/data/Analyze.java  |   46 +
 .../data/visitors/AbstractLogicalVisitor.java      |    6 +
 .../logical/data/visitors/LogicalVisitor.java      |    2 +
 .../org/apache/drill/exec/proto/UserBitShared.java |   39 +-
 .../drill/exec/proto/beans/CoreOperatorType.java   |    8 +-
 protocol/src/main/protobuf/UserBitShared.proto     |    3 +
 147 files changed, 12483 insertions(+), 336 deletions(-)
 create mode 100644 exec/java-exec/src/main/codegen/templates/JsonBaseStatisticsRecordWriter.java
 copy exec/java-exec/src/main/codegen/templates/{RecordWriter.java => StatisticsRecordWriter.java} (67%)
 create mode 100644 exec/java-exec/src/main/codegen/templates/StatisticsRecordWriterImpl.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StatisticsAggrFunctions.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/{store/direct/DirectSubScan.java => physical/config/StatisticsAggregate.java} (50%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsMerge.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/{store/direct/DirectSubScan.java => physical/config/UnpivotMaps.java} (53%)
 copy exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/{WriterRecordBatch.java => StatisticsWriterRecordBatch.java} (75%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/CntDupsMergedStatistic.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColTypeMergedStatistic.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColumnMergedStatistic.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/HLLMergedStatistic.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatistic.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NDVMergedStatistic.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NNRowCountMergedStatistic.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/RowCountMergedStatistic.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/Statistic.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatchCreator.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatchCreator.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsBatchCreator.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsRecordBatch.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAnalyzeRel.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AnalyzePrule.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StatsAggPrel.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StatsMergePrel.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnpivotMapsPrel.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlAnalyzeTable.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonStatisticsRecordWriter.java
 create mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestAnalyze.java
 create mode 100644 logical/src/main/java/org/apache/drill/common/logical/data/Analyze.java


[drill] 01/02: DRILL-1328: Support table statistics

Posted by gp...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 3233d8aaff57ac71bd3b726efcd5fdaa92aef861
Author: Cliff Buchanan <cb...@maprtech.com>
AuthorDate: Tue Oct 16 15:17:43 2018 -0700

    DRILL-1328: Support table statistics
---
 .../drill/exec/store/mapr/TableFormatPlugin.java   |   2 +-
 .../store/mapr/streams/StreamsFormatPlugin.java    |   2 +-
 exec/java-exec/pom.xml                             |   5 +
 exec/java-exec/src/main/codegen/data/Parser.tdd    |  28 +-
 .../src/main/codegen/includes/parserImpls.ftl      |  46 +++-
 .../apache/drill/exec/dotdrill/DotDrillType.java   |   5 +-
 .../apache/drill/exec/dotdrill/DotDrillUtil.java   |  14 +-
 .../exec/expr/fn/impl/StatisticsAggrFunctions.java | 285 +++++++++++++++++++++
 .../org/apache/drill/exec/ops/QueryContext.java    |   4 +
 .../physical/base/AbstractPhysicalVisitor.java     |  12 +
 .../drill/exec/physical/base/PhysicalVisitor.java  |   4 +
 .../exec/physical/config/StatisticsAggregate.java  |  66 +++++
 .../drill/exec/physical/config/UnpivotMaps.java    |  59 +++++
 .../physical/impl/aggregate/InternalBatch.java     |   6 +
 .../impl/aggregate/StatisticsAggBatch.java         | 223 ++++++++++++++++
 .../impl/aggregate/StatisticsAggBatchCreator.java  |  38 +++
 .../physical/impl/aggregate/StreamingAggBatch.java |  16 +-
 .../impl/unpivot/UnpivotMapsBatchCreator.java      |  38 +++
 .../impl/unpivot/UnpivotMapsRecordBatch.java       | 244 ++++++++++++++++++
 .../apache/drill/exec/planner/PlannerPhase.java    |   2 +
 .../exec/planner/common/DrillJoinRelBase.java      |  51 +++-
 .../drill/exec/planner/common/DrillStatsTable.java | 158 ++++++++++++
 .../planner/cost/DrillRelMdDistinctRowCount.java   |  52 ++++
 .../exec/planner/cost/DrillRelMdRowCount.java      |  19 ++
 .../exec/planner/logical/DrillAnalyzeRel.java      |  71 +++++
 .../drill/exec/planner/logical/DrillTable.java     |  10 +
 .../logical/FileSystemCreateTableEntry.java        |   6 +-
 .../drill/exec/planner/physical/AnalyzePrule.java  |  64 +++++
 .../drill/exec/planner/physical/StatsAggPrel.java  |  86 +++++++
 .../exec/planner/physical/UnpivotMapsPrel.java     |  85 ++++++
 .../planner/sql/handlers/AnalyzeTableHandler.java  | 157 ++++++++++++
 .../planner/sql/handlers/DefaultSqlHandler.java    |   2 +
 .../sql/parser/CompoundIdentifierConverter.java    |   1 +
 .../exec/planner/sql/parser/SqlAnalyzeTable.java   | 157 ++++++++++++
 .../apache/drill/exec/store/AbstractSchema.java    |  34 +++
 .../apache/drill/exec/store/SubSchemaWrapper.java  |  15 ++
 .../exec/store/dfs/FileSystemSchemaFactory.java    |  15 ++
 .../apache/drill/exec/store/dfs/FormatPlugin.java  |   2 +-
 .../exec/store/dfs/WorkspaceSchemaFactory.java     |  90 ++++++-
 .../exec/store/dfs/easy/EasyFormatPlugin.java      |   4 +-
 .../drill/exec/store/dfs/easy/EasyWriter.java      |  12 +-
 .../exec/store/easy/json/JSONFormatPlugin.java     |   2 +
 .../exec/store/easy/json/JsonRecordWriter.java     |   8 +-
 .../exec/store/parquet/ParquetFormatPlugin.java    |   8 +-
 .../drill/exec/store/parquet/ParquetWriter.java    |  12 +-
 .../org/apache/drill/exec/sql/TestAnalyze.java     | 105 ++++++++
 exec/jdbc-all/pom.xml                              |   4 +
 .../apache/drill/common/logical/data/Analyze.java  |  35 +++
 .../data/visitors/AbstractLogicalVisitor.java      |   6 +
 .../logical/data/visitors/LogicalVisitor.java      |   2 +
 .../org/apache/drill/exec/proto/UserBitShared.java |  29 ++-
 .../drill/exec/proto/beans/CoreOperatorType.java   |   6 +-
 protocol/src/main/protobuf/UserBitShared.proto     |   2 +
 53 files changed, 2356 insertions(+), 53 deletions(-)

diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java
index aeb117a..1c30264 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java
@@ -94,7 +94,7 @@ public abstract class TableFormatPlugin implements FormatPlugin {
 
   @Override
   public AbstractWriter getWriter(PhysicalOperator child, String location,
-      List<String> partitionColumns) throws IOException {
+      boolean append, List<String> partitionColumns) throws IOException {
     throw new UnsupportedOperationException();
   }
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java
index 206954b..76466ab 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java
@@ -64,7 +64,7 @@ public class StreamsFormatPlugin extends TableFormatPlugin {
 
   @Override
   public AbstractWriter getWriter(PhysicalOperator child, String location,
-      List<String> partitionColumns) throws IOException {
+      boolean append, List<String> partitionColumns) throws IOException {
     throw new UnsupportedOperationException();
   }
 
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index df150e5..d7f7393 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -352,6 +352,11 @@
       <artifactId>commons-compiler</artifactId>
     </dependency>
     <dependency>
+      <groupId>com.clearspring.analytics</groupId>
+      <artifactId>stream</artifactId>
+      <version>2.7.0</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <exclusions>
diff --git a/exec/java-exec/src/main/codegen/data/Parser.tdd b/exec/java-exec/src/main/codegen/data/Parser.tdd
index 820ecb5..ec56af4 100644
--- a/exec/java-exec/src/main/codegen/data/Parser.tdd
+++ b/exec/java-exec/src/main/codegen/data/Parser.tdd
@@ -38,6 +38,30 @@
     "IF",
     "JAR",
     "PROPERTIES"
+    "ANALYZE",
+    "COMPUTE",
+    "ESTIMATE",
+    "STATISTICS",
+    "COLUMNS",
+    "SAMPLE"
+  ]
+
+  # List of keywords from "keywords" section that are not reserved by SQL:2003 standard.
+  # Example: "DATABASES", "TABLES" are keywords but are not reserved by SQL:2003 standard.
+  # First keyword that starts the statement should be a reserved keyword, otherwise the current parser
+  # ends up considering it as a expression and fails.
+  nonReservedKeywords: [
+    "DATABASES",
+    "REPLACE",
+    "SCHEMAS",
+    "TABLES",
+    "FILES",
+    "METADATA",
+    "COMPUTE",
+    "ESTIMATE",
+    "STATISTICS",
+    "COLUMNS",
+    "SAMPLE"
   ]
 
   # List of methods for parsing custom SQL statements.
@@ -52,7 +76,9 @@
     "SqlShowFiles()",
     "SqlRefreshMetadata()",
     "SqlCreateFunction()",
-    "SqlDropFunction()"
+    "SqlDropFunction()",
+    "SqlRefreshMetadata()",
+    "SqlAnalyzeTable()"
   ]
 
   # List of methods for parsing custom literals.
diff --git a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
index 8afc8f8..2606006 100644
--- a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
+++ b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
@@ -557,4 +557,48 @@ Pair<SqlNodeList, SqlNodeList> ParenthesizedCompoundIdentifierList() :
        return Pair.of(new SqlNodeList(list, getPos()), null);
     }
 }
-</#if>
\ No newline at end of file
+</#if>
+/**
+ * Parses a analyze statement.
+ * ANALYZE TABLE tblname {COMPUTE | ESTIMATE} | STATISTICS FOR
+ *      {ALL COLUMNS | COLUMNS (field1, field2, ...)} [ SAMPLE numeric PERCENT ]
+ */
+SqlNode SqlAnalyzeTable() :
+{
+    SqlParserPos pos;
+    SqlIdentifier tblName;
+    SqlLiteral estimate = null;
+    SqlNodeList fieldList = null;
+    SqlNumericLiteral percent = null;
+}
+{
+    <ANALYZE> { pos = getPos(); }
+    <TABLE>
+    tblName = CompoundIdentifier()
+    (
+        <COMPUTE> { estimate = SqlLiteral.createBoolean(false, pos); }
+        |
+        <ESTIMATE> { estimate = SqlLiteral.createBoolean(true, pos); }
+    )
+    <STATISTICS> <FOR>
+    (
+        ( <ALL> <COLUMNS> )
+        |
+        ( <COLUMNS> fieldList = ParseRequiredFieldList("Table") )
+    )
+    [
+        <SAMPLE> percent = UnsignedNumericLiteral() <PERCENT>
+        {
+            BigDecimal rate = percent.bigDecimalValue();
+            if (rate.compareTo(BigDecimal.ZERO) <= 0 ||
+                rate.compareTo(BigDecimal.valueOf(100L)) > 0)
+            {
+                throw new ParseException("Invalid percentage for ANALYZE TABLE");
+            }
+        }
+    ]
+    {
+        if (percent == null) { percent = SqlLiteral.createExactNumeric("100.0", pos); }
+        return new SqlAnalyzeTable(pos, tblName, estimate, fieldList, percent);
+    }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillType.java b/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillType.java
index 673e1c7..589e982 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillType.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillType.java
@@ -21,10 +21,9 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 
 public enum DotDrillType {
-  VIEW;
+  VIEW,
   // ,FORMAT
-  // ,STATS
-
+  STATS;
 
   private final String ending;
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillUtil.java
index b6571df..32759d2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillUtil.java
@@ -17,19 +17,18 @@
  */
 package org.apache.drill.exec.dotdrill;
 
-import java.io.IOException;
 import java.io.FileNotFoundException;
-import java.util.List;
-import java.util.Arrays;
+import java.io.IOException;
 import java.util.ArrayList;
-
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.GlobPattern;
 import org.apache.hadoop.fs.Path;
 
-import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
-
 public class DotDrillUtil {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DotDrillUtil.class);
 
@@ -42,6 +41,9 @@ public class DotDrillUtil {
    * @return List of matched DotDrillFile objects
    */
   private static List<DotDrillFile> getDrillFiles(DrillFileSystem fs, List<FileStatus> statuses, DotDrillType... types){
+    if (statuses == null) {
+      return Collections.emptyList();
+    }
     List<DotDrillFile> files = Lists.newArrayList();
     for(FileStatus s : statuses){
       DotDrillFile f = DotDrillFile.create(fs, s);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StatisticsAggrFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StatisticsAggrFunctions.java
new file mode 100644
index 0000000..c6430dd
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StatisticsAggrFunctions.java
@@ -0,0 +1,285 @@
+/*******************************************************************************
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ ******************************************************************************/
+
+/*
+ * This class is automatically generated from AggrTypeFunctions2.tdd using FreeMarker.
+ */
+
+package org.apache.drill.exec.expr.fn.impl;
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillAggFunc;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.NullableBigIntHolder;
+import org.apache.drill.exec.expr.holders.NullableVarBinaryHolder;
+import org.apache.drill.exec.expr.holders.ObjectHolder;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+
+import javax.inject.Inject;
+
+@SuppressWarnings("unused")
+public class StatisticsAggrFunctions {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsAggrFunctions.class);
+
+  @FunctionTemplate(name = "statcount", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class StatCount implements DrillAggFunc {
+    @Param FieldReader in;
+    @Workspace BigIntHolder count;
+    @Output NullableBigIntHolder out;
+
+    @Override
+    public void setup() {
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.isSet = 1;
+      out.value = count.value;
+    }
+
+    @Override
+    public void reset() {
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "nonnullstatcount", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NonNullStatCount implements DrillAggFunc {
+    @Param FieldReader in;
+    @Workspace BigIntHolder count;
+    @Output NullableBigIntHolder out;
+
+    @Override
+    public void setup() {
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      if (in.isSet()) {
+        count.value++;
+      }
+    }
+
+    @Override
+    public void output() {
+      out.isSet = 1;
+      out.value = count.value;
+    }
+
+    @Override
+    public void reset() {
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class HllFieldReader implements DrillAggFunc {
+    @Param FieldReader in;
+    @Workspace ObjectHolder work;
+    @Output NullableVarBinaryHolder out;
+    @Inject DrillBuf buffer;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(10);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+            (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        int mode = in.getType().getMode().getNumber();
+        int type = in.getType().getMinorType().getNumber();
+
+        switch (mode) {
+          case org.apache.drill.common.types.TypeProtos.DataMode.OPTIONAL_VALUE:
+            if (!in.isSet()) {
+              hll.offer(null);
+              break;
+            }
+            // fall through //
+          case org.apache.drill.common.types.TypeProtos.DataMode.REQUIRED_VALUE:
+            switch (type) {
+              case org.apache.drill.common.types.TypeProtos.MinorType.VARCHAR_VALUE:
+                hll.offer(in.readText().toString());
+                break;
+              default:
+                work.obj = null;
+            }
+            break;
+          default:
+            work.obj = null;
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+            (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(10);
+    }
+  }
+
+
+  @FunctionTemplate(name = "ndv", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NdvVarBinary implements DrillAggFunc {
+    @Param
+    FieldReader in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableBigIntHolder out;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(10);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+            (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        int mode = in.getType().getMode().getNumber();
+        int type = in.getType().getMinorType().getNumber();
+
+        switch (mode) {
+          case org.apache.drill.common.types.TypeProtos.DataMode.OPTIONAL_VALUE:
+            if (!in.isSet()) {
+              hll.offer(null);
+              break;
+            }
+            // fall through //
+          case org.apache.drill.common.types.TypeProtos.DataMode.REQUIRED_VALUE:
+            switch (type) {
+              case org.apache.drill.common.types.TypeProtos.MinorType.VARCHAR_VALUE:
+                hll.offer(in.readText().toString());
+                break;
+              case org.apache.drill.common.types.TypeProtos.MinorType.FLOAT8_VALUE:
+                hll.offer(in.readDouble());
+                break;
+              case org.apache.drill.common.types.TypeProtos.MinorType.INT_VALUE:
+                hll.offer(in.readInteger());
+                break;
+              case org.apache.drill.common.types.TypeProtos.MinorType.BIGINT_VALUE:
+                hll.offer(in.readLong());
+                break;
+              case org.apache.drill.common.types.TypeProtos.MinorType.DATE_VALUE:
+              case org.apache.drill.common.types.TypeProtos.MinorType.TIMESTAMP_VALUE:
+              case org.apache.drill.common.types.TypeProtos.MinorType.TIME_VALUE:
+              case org.apache.drill.common.types.TypeProtos.MinorType.TIMETZ_VALUE:
+                hll.offer(in.readLocalDateTime());
+                break;
+              case org.apache.drill.common.types.TypeProtos.MinorType.VARBINARY_VALUE:
+                hll.offer(in.readByteArray());
+                break;
+              default:
+                work.obj = null;
+            }
+            break;
+          default:
+            work.obj = null;
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+            (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        out.isSet = 1;
+        out.value = hll.cardinality();
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(10);
+    }
+  }
+
+
+  @FunctionTemplate(name = "hll_decode", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class HllDecode implements DrillSimpleFunc {
+
+    @Param
+    NullableVarBinaryHolder in;
+    @Output
+    BigIntHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    public void eval() {
+      out.value = -1;
+
+      if (in.isSet != 0) {
+        byte[] din = new byte[in.end - in.start];
+        in.buffer.getBytes(in.start, din);
+        try {
+          out.value = com.clearspring.analytics.stream.cardinality.HyperLogLog.Builder.build(din).cardinality();
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failure evaluation hll_decode", e);
+        }
+      }
+    }
+  }
+
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index c3e6bda..d770636 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -288,6 +288,10 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
     return new PartitionExplorerImpl(getRootSchema());
   }
 
+  public DrillbitContext getDrillbitContext() {
+    return drillbitContext;
+  }
+
   @Override
   public ValueHolder getConstantValueHolder(String value, MinorType type, Function<DrillBuf, ValueHolder> holderInitializer) {
     if (!constantValueHolderCache.containsKey(value)) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
index 262e7e5..85d2a2b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
@@ -35,11 +35,13 @@ import org.apache.drill.exec.physical.config.RowKeyJoinPOP;
 import org.apache.drill.exec.physical.config.Screen;
 import org.apache.drill.exec.physical.config.SingleSender;
 import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.physical.config.StatisticsAggregate;
 import org.apache.drill.exec.physical.config.StreamingAggregate;
 import org.apache.drill.exec.physical.config.Trace;
 import org.apache.drill.exec.physical.config.UnionAll;
 import org.apache.drill.exec.physical.config.UnnestPOP;
 import org.apache.drill.exec.physical.config.UnorderedReceiver;
+import org.apache.drill.exec.physical.config.UnpivotMaps;
 import org.apache.drill.exec.physical.config.Values;
 import org.apache.drill.exec.physical.config.WindowPOP;
 
@@ -97,6 +99,11 @@ public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> impleme
   }
 
   @Override
+  public T visitStatisticsAggregate(StatisticsAggregate agg, X value) throws E {
+    return visitOp(agg, value);
+  }
+
+  @Override
   public T visitHashAggregate(HashAggregate agg, X value) throws E {
     return visitOp(agg, value);
   }
@@ -215,6 +222,11 @@ public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> impleme
   }
 
   @Override
+  public T visitUnpivot(UnpivotMaps op, X value) throws E {
+    return visitOp(op, value);
+  }
+
+  @Override
   public T visitOp(PhysicalOperator op, X value) throws E{
     throw new UnsupportedOperationException(String.format(
         "The PhysicalVisitor of type %s does not currently support visiting the PhysicalOperator type %s.", this
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
index 77a87c2..a21f578 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
@@ -35,11 +35,13 @@ import org.apache.drill.exec.physical.config.RowKeyJoinPOP;
 import org.apache.drill.exec.physical.config.Screen;
 import org.apache.drill.exec.physical.config.SingleSender;
 import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.physical.config.StatisticsAggregate;
 import org.apache.drill.exec.physical.config.StreamingAggregate;
 import org.apache.drill.exec.physical.config.Trace;
 import org.apache.drill.exec.physical.config.UnionAll;
 import org.apache.drill.exec.physical.config.UnnestPOP;
 import org.apache.drill.exec.physical.config.UnorderedReceiver;
+import org.apache.drill.exec.physical.config.UnpivotMaps;
 import org.apache.drill.exec.physical.config.Values;
 import org.apache.drill.exec.physical.config.WindowPOP;
 
@@ -68,8 +70,10 @@ public interface PhysicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
   public RETURN visitRowKeyJoin(RowKeyJoinPOP join, EXTRA value) throws EXCEP;
   public RETURN visitReceiver(Receiver receiver, EXTRA value) throws EXCEP;
   public RETURN visitStreamingAggregate(StreamingAggregate agg, EXTRA value) throws EXCEP;
+  public RETURN visitStatisticsAggregate(StatisticsAggregate agg, EXTRA value) throws EXCEP;
   public RETURN visitHashAggregate(HashAggregate agg, EXTRA value) throws EXCEP;
   public RETURN visitWriter(Writer op, EXTRA value) throws EXCEP;
+  public RETURN visitUnpivot(UnpivotMaps op, EXTRA value) throws EXCEP;
   public RETURN visitValues(Values op, EXTRA value) throws EXCEP;
   public RETURN visitOp(PhysicalOperator op, EXTRA value) throws EXCEP;
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java
new file mode 100644
index 0000000..95ee6bf
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java
@@ -0,0 +1,66 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.config;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+import java.util.List;
+
+@JsonTypeName("statistics-aggregate")
+public class StatisticsAggregate extends StreamingAggregate {
+  // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsAggregate.class);
+
+  private final List<String> functions;
+
+  @JsonCreator
+  public StatisticsAggregate(
+      @JsonProperty("child") PhysicalOperator child,
+      @JsonProperty("functions") List<String> functions) {
+    super(child, null, null, 0.f);
+    this.functions = ImmutableList.copyOf(functions);
+  }
+
+  public List<String> getFunctions() {
+    return functions;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value)
+      throws E {
+    return physicalVisitor.visitStatisticsAggregate(this, value);
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new StatisticsAggregate(child, functions);
+  }
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.STATISTICS_AGGREGATE_VALUE;
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java
new file mode 100644
index 0000000..ac71b11
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java
@@ -0,0 +1,59 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.config;
+
+import java.util.List;
+
+import org.apache.drill.exec.physical.base.AbstractSingle;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("unpivot-maps")
+public class UnpivotMaps extends AbstractSingle {
+  private final List<String> mapFieldsNames;
+
+  @JsonCreator
+  public UnpivotMaps(@JsonProperty("child") PhysicalOperator child, List<String> mapFieldsNames) {
+    super(child);
+    this.mapFieldsNames = mapFieldsNames;
+  }
+
+  public List<String> getMapFieldNames() {
+    return mapFieldsNames;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitUnpivot(this, value);
+  }
+
+  @Override
+  protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+    return new UnpivotMaps(child, mapFieldsNames);
+  }
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.UNPIVOT_MAPS_VALUE;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java
index 1d1d3cb..396fd36 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.physical.impl.aggregate;
 
 import java.util.Iterator;
 
+import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.RecordBatch;
@@ -31,6 +32,7 @@ public class InternalBatch implements Iterable<VectorWrapper<?>>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(InternalBatch.class);
 
   private final VectorContainer container;
+  private final RecordBatch incoming;
   private final BatchSchema schema;
   private final SelectionVector2 sv2;
   private final SelectionVector4 sv4;
@@ -54,6 +56,7 @@ public class InternalBatch implements Iterable<VectorWrapper<?>>{
       this.sv2 = null;
     }
     this.schema = incoming.getSchema();
+    this.incoming = incoming;
     this.container = VectorContainer.getTransferClone(incoming, ignoreWrappers, oContext);
   }
 
@@ -88,4 +91,7 @@ public class InternalBatch implements Iterable<VectorWrapper<?>>{
     return container.getValueAccessorById(clazz, fieldIds);
   }
 
+  public FragmentContext getContext() {
+    return incoming.getContext();
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatch.java
new file mode 100644
index 0000000..ea5a7b3
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatch.java
@@ -0,0 +1,223 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.aggregate;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import com.sun.codemodel.JExpr;
+import org.apache.drill.common.expression.ErrorCollector;
+import org.apache.drill.common.expression.ErrorCollectorImpl;
+import org.apache.drill.common.expression.FunctionCallFactory;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.ValueExpressions;
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.ClassGenerator;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.expr.ValueVectorWriteExpression;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.StatisticsAggregate;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.FieldIdUtil;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * TODO: This needs cleanup. Currently the key values are constants and we compare the constants for
+ * every record. Seems unnecessary.
+ *
+ * Example input and output:
+ * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
+ * Schema of output:
+ *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
+ *    "computed" : BIGINT - What time is it computed?
+ *    "columns" : MAP - Column names
+ *       "region_id" : VARCHAR
+ *       "sales_city" : VARCHAR
+ *       "cnt" : VARCHAR
+ *    "statscount" : MAP
+ *       "region_id" : BIGINT - statscount(region_id) - aggregation over all values of region_id in incoming batch
+ *       "sales_city" : BIGINT - statscount(sales_city)
+ *       "cnt" : BIGINT - statscount(cnt)
+ *    "nonnullstatcount" : MAP
+ *       "region_id" : BIGINT - nonnullstatcount(region_id)
+ *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
+ *       "cnt" : BIGINT - nonnullstatcount(cnt)
+ *   .... another map for next stats function ....
+ */
+public class StatisticsAggBatch extends StreamingAggBatch {
+  private List<String> functions;
+  private int schema = 0;
+
+  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming, FragmentContext context)
+      throws OutOfMemoryException {
+    super(popConfig, incoming, context);
+    this.functions = popConfig.getFunctions();
+  }
+
+  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
+      throws SchemaChangeException {
+    ErrorCollector collector = new ErrorCollectorImpl();
+
+    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector, context.getFunctionRegistry());
+
+    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
+    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
+
+    keyExprs.add(mle);
+    keyOutputIds.add(container.add(vector));
+
+    if (collector.hasErrors()) {
+      throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString());
+    }
+  }
+
+  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr, List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
+      throws SchemaChangeException {
+    ErrorCollector collector = new ErrorCollectorImpl();
+
+    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector, context.getFunctionRegistry());
+
+    Class<? extends ValueVector> vvc =
+        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
+
+    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
+
+    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getName()));
+    assert pfid.getFieldIds().length == 1;
+    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
+    builder.addId(pfid.getFieldIds()[0]);
+    TypedFieldId id =
+        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
+            SchemaPath.getSimplePath(vv.getField().getName()).getRootSegment());
+
+    keyExprs.add(mle);
+    keyOutputIds.add(id);
+
+    if (collector.hasErrors()) {
+      throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString());
+    }
+  }
+
+  private void addMapVector(String name, MapVector parent, LogicalExpression expr, List<LogicalExpression> valueExprs)
+      throws SchemaChangeException {
+    ErrorCollector collector = new ErrorCollectorImpl();
+
+    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector, context.getFunctionRegistry());
+
+    Class<? extends ValueVector> vvc =
+        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
+    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
+
+    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getName()));
+    assert pfid.getFieldIds().length == 1;
+    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
+    builder.addId(pfid.getFieldIds()[0]);
+    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
+        SchemaPath.getSimplePath(vv.getField().getName()).getRootSegment());
+
+    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
+
+    if (collector.hasErrors()) {
+      throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString());
+    }
+  }
+
+  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs, List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
+      throws SchemaChangeException, ClassTransformationException, IOException {
+    ClassGenerator<StreamingAggregator> cg = CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getOptions());
+
+    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
+    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
+    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
+
+    keyExprs.toArray(keyExprsArray);
+    valueExprs.toArray(valueExprsArray);
+    keyOutputIds.toArray(keyOutputIdsArray);
+
+    setupIsSame(cg, keyExprsArray);
+    setupIsSameApart(cg, keyExprsArray);
+    addRecordValues(cg, valueExprsArray);
+    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
+    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
+
+    cg.getBlock("resetValues")._return(JExpr.TRUE);
+    getIndex(cg);
+
+    container.buildSchema(SelectionVectorMode.NONE);
+    StreamingAggregator agg = context.getImplementationClass(cg);
+    agg.setup(oContext, incoming, this, ValueVector.MAX_ROW_COUNT);
+    return agg;
+  }
+
+  protected StreamingAggregator createAggregatorInternal()
+      throws SchemaChangeException, ClassTransformationException, IOException {
+    container.clear();
+
+    List<LogicalExpression> keyExprs = Lists.newArrayList();
+    List<LogicalExpression> valueExprs = Lists.newArrayList();
+    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
+
+    createKeyColumn("schema",
+        ValueExpressions.getBigInt(schema++),
+        keyExprs,
+        keyOutputIds
+    );
+    createKeyColumn("computed",
+        ValueExpressions.getBigInt(System.currentTimeMillis()),
+        keyExprs,
+        keyOutputIds
+    );
+
+    MapVector cparent = new MapVector("column", oContext.getAllocator(), null);
+    container.add(cparent);
+    for (MaterializedField mf : incoming.getSchema()) {
+      createNestedKeyColumn(
+          cparent,
+          mf.getName(),
+          ValueExpressions.getChar(mf.getName(), 0),
+          keyExprs,
+          keyOutputIds
+      );
+    }
+
+    for (String func : functions) {
+      MapVector parent = new MapVector(func, oContext.getAllocator(), null);
+      container.add(parent);
+
+      for (MaterializedField mf : incoming.getSchema()) {
+        List<LogicalExpression> args = Lists.newArrayList();
+        args.add(SchemaPath.getSimplePath(mf.getName()));
+        LogicalExpression call = FunctionCallFactory.createExpression(func, args);
+
+        addMapVector(mf.getName(), parent, call, valueExprs);
+      }
+    }
+
+    return codegenAggregator(keyExprs, valueExprs, keyOutputIds);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatchCreator.java
new file mode 100644
index 0000000..aba325c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatchCreator.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.aggregate;
+
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import java.util.List;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.ExecutorFragmentContext;
+import org.apache.drill.exec.physical.config.StatisticsAggregate;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.record.CloseableRecordBatch;
+import org.apache.drill.exec.record.RecordBatch;
+
+@SuppressWarnings("unused")
+public class StatisticsAggBatchCreator implements BatchCreator<StatisticsAggregate>{
+
+  @Override
+  public CloseableRecordBatch getBatch(ExecutorFragmentContext context, StatisticsAggregate config, List<RecordBatch> children)
+      throws ExecutionSetupException {
+    Preconditions.checkArgument(children.size() == 1);
+    return new StatisticsAggBatch(config, children.iterator().next(), context);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
index ffcfa78..e1e43bd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
@@ -77,8 +77,8 @@ import static org.apache.drill.exec.record.RecordBatch.IterOutcome.STOP;
 public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StreamingAggBatch.class);
 
-  private StreamingAggregator aggregator;
-  private final RecordBatch incoming;
+  protected StreamingAggregator aggregator;
+  protected final RecordBatch incoming;
   private List<BaseWriter.ComplexWriter> complexWriters;
   //
   // Streaming agg can be in (a) a normal pipeline or (b) it may be in a pipeline that is part of a subquery involving
@@ -533,7 +533,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
   private final MappingSet IS_SAME_I1 = new MappingSet("index1", null, IS_SAME, IS_SAME);
   private final MappingSet IS_SAME_I2 = new MappingSet("index2", null, IS_SAME, IS_SAME);
 
-  private void setupIsSame(ClassGenerator<StreamingAggregator> cg, LogicalExpression[] keyExprs) {
+  protected void setupIsSame(ClassGenerator<StreamingAggregator> cg, LogicalExpression[] keyExprs) {
     cg.setMappingSet(IS_SAME_I1);
     for (final LogicalExpression expr : keyExprs) {
       // first, we rewrite the evaluation stack for each side of the comparison.
@@ -556,7 +556,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
   private final MappingSet ISA_B1 = new MappingSet("b1Index", null, "b1", null, IS_SAME_PREV_INTERNAL_BATCH_READ, IS_SAME_PREV_INTERNAL_BATCH_READ);
   private final MappingSet ISA_B2 = new MappingSet("b2Index", null, "incoming", null, IS_SAME_PREV, IS_SAME_PREV);
 
-  private void setupIsSameApart(ClassGenerator<StreamingAggregator> cg, LogicalExpression[] keyExprs) {
+  protected void setupIsSameApart(ClassGenerator<StreamingAggregator> cg, LogicalExpression[] keyExprs) {
     cg.setMappingSet(ISA_B1);
     for (final LogicalExpression expr : keyExprs) {
       // first, we rewrite the evaluation stack for each side of the comparison.
@@ -578,7 +578,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
   private final GeneratorMapping EVAL_OUTSIDE = GeneratorMapping.create("setupInterior", "outputRecordValues", "resetValues", "cleanup");
   private final MappingSet EVAL = new MappingSet("index", "outIndex", "incoming", "outgoing", EVAL_INSIDE, EVAL_OUTSIDE, EVAL_INSIDE);
 
-  private void addRecordValues(ClassGenerator<StreamingAggregator> cg, LogicalExpression[] valueExprs) {
+  protected void addRecordValues(ClassGenerator<StreamingAggregator> cg, LogicalExpression[] valueExprs) {
     cg.setMappingSet(EVAL);
     for (final LogicalExpression ex : valueExprs) {
       cg.addExpr(ex);
@@ -587,7 +587,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
 
   private final MappingSet RECORD_KEYS = new MappingSet(GeneratorMapping.create("setupInterior", "outputRecordKeys", null, null));
 
-  private void outputRecordKeys(ClassGenerator<StreamingAggregator> cg, TypedFieldId[] keyOutputIds, LogicalExpression[] keyExprs) {
+  protected void outputRecordKeys(ClassGenerator<StreamingAggregator> cg, TypedFieldId[] keyOutputIds, LogicalExpression[] keyExprs) {
     cg.setMappingSet(RECORD_KEYS);
     for (int i = 0; i < keyExprs.length; i++) {
       cg.addExpr(new ValueVectorWriteExpression(keyOutputIds[i], keyExprs[i], true));
@@ -600,7 +600,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
   private final GeneratorMapping PREVIOUS_KEYS = GeneratorMapping.create("outputRecordKeysPrev", "outputRecordKeysPrev", null, null);
   private final MappingSet RECORD_KEYS_PREV = new MappingSet("previousIndex", "outIndex", "previous", null, PREVIOUS_KEYS, PREVIOUS_KEYS);
 
-  private void outputRecordKeysPrev(ClassGenerator<StreamingAggregator> cg, TypedFieldId[] keyOutputIds, LogicalExpression[] keyExprs) {
+  protected void outputRecordKeysPrev(ClassGenerator<StreamingAggregator> cg, TypedFieldId[] keyOutputIds, LogicalExpression[] keyExprs) {
     cg.setMappingSet(RECORD_KEYS_PREV);
 
     for (int i = 0; i < keyExprs.length; i++) {
@@ -614,7 +614,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
     }
   }
 
-  private void getIndex(ClassGenerator<StreamingAggregator> g) {
+  protected void getIndex(ClassGenerator<StreamingAggregator> g) {
     switch (incoming.getSchema().getSelectionVectorMode()) {
     case FOUR_BYTE: {
       JVar var = g.declareClassField("sv4_", g.getModel()._ref(SelectionVector4.class));
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsBatchCreator.java
new file mode 100644
index 0000000..733524f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsBatchCreator.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.unpivot;
+
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import java.util.List;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.ExecutorFragmentContext;
+import org.apache.drill.exec.physical.config.UnpivotMaps;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.record.CloseableRecordBatch;
+import org.apache.drill.exec.record.RecordBatch;
+
+@SuppressWarnings("unused")
+public class UnpivotMapsBatchCreator implements BatchCreator<UnpivotMaps>{
+
+  @Override
+  public CloseableRecordBatch getBatch(ExecutorFragmentContext context, UnpivotMaps config, List<RecordBatch> children)
+      throws ExecutionSetupException {
+    Preconditions.checkArgument(children.size() == 1);
+    return new UnpivotMapsRecordBatch(config, children.iterator().next(), context);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsRecordBatch.java
new file mode 100644
index 0000000..e98d70e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unpivot/UnpivotMapsRecordBatch.java
@@ -0,0 +1,244 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.unpivot;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.UnpivotMaps;
+import org.apache.drill.exec.record.AbstractSingleRecordBatch;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+
+/**
+ * TODO: This needs cleanup, especially in state transitions.
+ *
+ * Unpivot maps. Assumptions are:
+ *  1) all child vectors in a map are of same type.
+ *  2) Each map contains the same number of fields and field names are also same (types could be different).
+ *
+ * Example input and output:
+ * Schema of input:
+ *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
+ *    "computed" : BIGINT - What time is it computed?
+ *    "columns" : MAP - Column names
+ *       "region_id" : VARCHAR
+ *       "sales_city" : VARCHAR
+ *       "cnt" : VARCHAR
+ *    "statscount" : MAP
+ *       "region_id" : BIGINT - statscount(region_id) - aggregation over all values of region_id in incoming batch
+ *       "sales_city" : BIGINT - statscount(sales_city)
+ *       "cnt" : BIGINT - statscount(cnt)
+ *    "nonnullstatcount" : MAP
+ *       "region_id" : BIGINT - nonnullstatcount(region_id)
+ *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
+ *       "cnt" : BIGINT - nonnullstatcount(cnt)
+ *   .... another map for next stats function ....
+ *
+ * Schema of output:
+ *  "schema" : BIGINT - Schema number. For each schema change this number is incremented.
+ *  "computed" : BIGINT - What time is this computed?
+ *  "column" : column name
+ *  "statscount" : BIGINT
+ *  "nonnullstatcount" : BIGINT
+ *  .... one column for each map type ...
+ */
+public class UnpivotMapsRecordBatch extends AbstractSingleRecordBatch<UnpivotMaps> {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnpivotMapsRecordBatch.class);
+
+  private final List<String> mapFieldsNames;
+
+  private int keyIndex = 0;
+  private List<String> keyList = null;
+
+  private Map<MaterializedField, Map<String, ValueVector>> dataSrcVecMap = null;
+
+  // Map of non-map fields to VV in the incoming schema
+  private Map<MaterializedField, ValueVector> copySrcVecMap = null;
+
+  private List<TransferPair> transferList;
+  private int recordCount = 0;
+
+  public UnpivotMapsRecordBatch(UnpivotMaps pop, RecordBatch incoming, FragmentContext context)
+      throws OutOfMemoryException {
+    super(pop, context, incoming);
+    this.mapFieldsNames = pop.getMapFieldNames();
+  }
+
+  @Override
+  public int getRecordCount() {
+    return recordCount;
+  }
+
+  @Override
+  public IterOutcome innerNext() {
+    if (keyIndex != 0) {
+      doWork();
+      return IterOutcome.OK;
+    } else {
+      return super.innerNext();
+    }
+  }
+
+  public VectorContainer getOutgoingContainer() {
+    return this.container;
+  }
+
+  private void doTransfer() {
+    final int inputCount = incoming.getRecordCount();
+
+    for (TransferPair tp : transferList) {
+      tp.splitAndTransfer(0, inputCount);
+    }
+  }
+
+  @Override
+  protected IterOutcome doWork() {
+    int outRecordCount = incoming.getRecordCount();
+
+    prepareTransfers();
+    doTransfer();
+
+    keyIndex = (keyIndex + 1) % keyList.size();
+    recordCount = outRecordCount;
+
+    if (keyIndex == 0) {
+      for (VectorWrapper w : incoming) {
+        w.clear();
+      }
+    }
+    return IterOutcome.OK;
+  }
+
+  /**
+   * Identify the list of fields within a map which are unpivoted as columns in output
+   */
+  private void buildKeyList() {
+    List<String> lastMapKeyList = null;
+    for (VectorWrapper<?> vw : incoming) {
+      if (vw.getField().getType().getMinorType() != MinorType.MAP) {
+        continue;
+      }
+
+      keyList = Lists.newArrayList();
+
+      for (ValueVector vv : vw.getValueVector()) {
+        keyList.add(vv.getField().getName());
+      }
+
+      if (lastMapKeyList == null) {
+        lastMapKeyList = keyList;
+      } else {
+        if (keyList.size() != lastMapKeyList.size() || !lastMapKeyList.containsAll(keyList)) {
+          throw new UnsupportedOperationException("Maps have different fields");
+        }
+      }
+    }
+  }
+
+  private void buildOutputContainer() {
+    dataSrcVecMap = Maps.newHashMap();
+    copySrcVecMap = Maps.newHashMap();
+    for (VectorWrapper<?> vw : incoming) {
+      MaterializedField ds = vw.getField();
+      String col = vw.getField().getName();
+
+      if (!mapFieldsNames.contains(col)) {
+        MajorType mt = vw.getValueVector().getField().getType();
+        MaterializedField mf = MaterializedField.create(col, mt);
+        container.add(TypeHelper.getNewVector(mf, oContext.getAllocator()));
+        copySrcVecMap.put(mf, vw.getValueVector());
+        continue;
+      }
+
+      MapVector mapVector = (MapVector) vw.getValueVector();
+      assert mapVector.getPrimitiveVectors().size() > 0;
+
+      MajorType mt = mapVector.iterator().next().getField().getType();
+      MaterializedField mf = MaterializedField.create(col, mt);
+      assert !dataSrcVecMap.containsKey(mf);
+      container.add(TypeHelper.getNewVector(mf, oContext.getAllocator()));
+
+      Map<String, ValueVector> m = Maps.newHashMap();
+      dataSrcVecMap.put(mf, m);
+
+      for (ValueVector vv : mapVector) {
+        String fieldName = vv.getField().getName();
+
+        if (!keyList.contains(fieldName)) {
+          throw new UnsupportedOperationException("Unpivot data vector " +
+              ds + " contains key " + fieldName + " not contained in key source!");
+        }
+
+        if (vv.getField().getType().getMinorType() == MinorType.MAP) {
+          throw new UnsupportedOperationException("Unpivot of nested map is not supported!");
+        }
+
+        m.put(fieldName, vv);
+      }
+    }
+
+    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
+  }
+
+  private void prepareTransfers() {
+    transferList = Lists.newArrayList();
+    for (VectorWrapper<?> vw : container) {
+      MaterializedField mf = vw.getField();
+
+      ValueVector vv;
+      TransferPair tp;
+      if (dataSrcVecMap.containsKey(mf)) {
+        String k = keyList.get(keyIndex);
+        vv = dataSrcVecMap.get(mf).get(k);
+        tp = vv.makeTransferPair(vw.getValueVector());
+      } else {
+        vv = copySrcVecMap.get(mf);
+        tp = vv.makeTransferPair(vw.getValueVector());
+      }
+
+      transferList.add(tp);
+    }
+  }
+
+  @Override
+  protected boolean setupNewSchema() throws SchemaChangeException {
+    container.clear();
+
+    buildKeyList();
+    buildOutputContainer();
+    return true;
+  }
+
+  @Override
+  public void dump() {
+    logger.error("UnpivotMapsRecordbatch[recordCount={}, container={}]", recordCount, container);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
index 6cdb18d..b6b0f58 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
@@ -60,6 +60,7 @@ import org.apache.drill.exec.planner.logical.DrillValuesRule;
 import org.apache.drill.exec.planner.logical.DrillWindowRule;
 import org.apache.drill.exec.planner.logical.partition.ParquetPruneScanRule;
 import org.apache.drill.exec.planner.logical.partition.PruneScanRule;
+import org.apache.drill.exec.planner.physical.AnalyzePrule;
 import org.apache.drill.exec.planner.physical.ConvertCountToDirectScan;
 import org.apache.drill.exec.planner.physical.LateralJoinPrule;
 import org.apache.drill.exec.planner.physical.DirectScanPrule;
@@ -516,6 +517,7 @@ public enum PlannerPhase {
     ruleList.add(ValuesPrule.INSTANCE);
     ruleList.add(DirectScanPrule.INSTANCE);
     ruleList.add(RowKeyJoinPrule.INSTANCE);
+    ruleList.add(AnalyzePrule.INSTANCE);
 
     ruleList.add(UnnestPrule.INSTANCE);
     ruleList.add(LateralJoinPrule.INSTANCE);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
index cde49e4..42fbedb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
@@ -20,27 +20,28 @@ package org.apache.drill.exec.planner.common;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
-
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.expr.holders.IntHolder;
-import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.physical.impl.join.JoinUtils;
 import org.apache.drill.exec.physical.impl.join.JoinUtils.JoinCategory;
+import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
 import org.apache.drill.exec.planner.logical.DrillJoin;
 import org.apache.drill.exec.planner.physical.PrelUtil;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexNode;
-
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
 /**
@@ -100,9 +101,31 @@ public abstract class DrillJoinRelBase extends Join implements DrillJoin {
   public double estimateRowCount(RelMetadataQuery mq) {
     if (this.condition.isAlwaysTrue()) {
       return joinRowFactor * this.getLeft().estimateRowCount(mq) * this.getRight().estimateRowCount(mq);
-    } else {
-      return joinRowFactor * Math.max(this.getLeft().estimateRowCount(mq), this.getRight().estimateRowCount(mq));
     }
+
+    int[] joinFields = new int[2];
+
+    LogicalJoin jr = LogicalJoin.create(this.getLeft(), this.getRight(), this.getCondition(),
+            this.getVariablesSet(), this.getJoinType());
+
+    if (RelOptUtil.analyzeSimpleEquiJoin(jr, joinFields)) {
+      ImmutableBitSet leq = ImmutableBitSet.of(joinFields[0]);
+      ImmutableBitSet req = ImmutableBitSet.of(joinFields[1]);
+
+      Double ldrc = mq.getDistinctRowCount(this.getLeft(), leq, null);
+      Double rdrc = mq.getDistinctRowCount(this.getRight(), req, null);
+
+      Double lrc = mq.getRowCount(this.getLeft());
+      Double rrc = mq.getRowCount(this.getRight());
+
+      if (ldrc != null && rdrc != null && lrc != null && rrc != null) {
+        return (lrc * rrc) / Math.max(ldrc, rdrc);
+      }
+    }
+
+    return joinRowFactor * Math.max(
+        mq.getRowCount(left),
+        mq.getRowCount(right));
   }
 
   /**
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java
new file mode 100644
index 0000000..a22552b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java
@@ -0,0 +1,158 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.common;
+
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import java.util.Map;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelVisitor;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.server.DrillbitContext;
+
+/**
+ * Wraps the stats table info including schema and tableName. Also materializes stats from storage and keeps them in
+ * memory.
+ */
+public class DrillStatsTable {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillStatsTable.class);
+
+  /**
+   * List of columns in stats table.
+   */
+  public static final String COL_COLUMN = "column";
+  public static final String COL_COMPUTED = "computed";
+  public static final String COL_STATCOUNT = "statcount";
+  public static final String COL_NDV = "ndv";
+
+  private final String schemaName;
+  private final String tableName;
+
+  private final Map<String, Long> ndv = Maps.newHashMap();
+  private double rowCount = -1;
+
+  private boolean materialized = false;
+
+  public DrillStatsTable(String schemaName, String tableName) {
+    this.schemaName = schemaName;
+    this.tableName = tableName;
+  }
+
+  /**
+   * Get number of distinct values of given column. If stats are not present for the given column, a null is returned.
+   *
+   * Note: returned data may not be accurate. Accuracy depends on whether the table data has changed after the
+   * stats are computed.
+   *
+   * @param col
+   * @return
+   */
+  public Double getNdv(String col) {
+    Preconditions.checkState(materialized, "Stats are not yet materialized.");
+
+    final String upperCol = col.toUpperCase();
+    final Long ndvCol = ndv.get(upperCol);
+    if (ndvCol != null) {
+      return Math.min(ndvCol, rowCount);
+    }
+
+    return null;
+  }
+
+  /**
+   * Get row count of the table. Returns null if stats are not present.
+   *
+   * Note: returned data may not be accurate. Accuracy depends on whether the table data has changed after the
+   * stats are computed.
+   *
+   * @return
+   */
+  public Double getRowCount() {
+    Preconditions.checkState(materialized, "Stats are not yet materialized.");
+    return rowCount > 0 ? rowCount : null;
+  }
+
+  /**
+   * Read the stats from storage and keep them in memory.
+   * @param context
+   * @throws Exception
+   */
+  public void materialize(final QueryContext context) throws Exception {
+    if (materialized) {
+      return;
+    }
+
+    final String fullTableName = "`" + schemaName + "`.`" + tableName + "`";
+    final String sql = "SELECT a.* FROM " + fullTableName + " AS a INNER JOIN " +
+        "(SELECT `" + COL_COLUMN + "`, max(`" + COL_COMPUTED +"`) AS `" + COL_COMPUTED + "` " +
+        "FROM " + fullTableName + " GROUP BY `" + COL_COLUMN + "`) AS b " +
+        "ON a.`" + COL_COLUMN + "` = b.`" + COL_COLUMN +"` and a.`" + COL_COMPUTED + "` = b.`" + COL_COMPUTED + "`";
+
+    final DrillbitContext dc = context.getDrillbitContext();
+    try(final DrillClient client = new DrillClient(dc.getConfig(), dc.getClusterCoordinator(), dc.getAllocator())) {
+      /*final Listener listener = new Listener(dc.getAllocator());
+
+      client.connect();
+      client.runQuery(UserBitShared.QueryType.SQL, sql, listener);
+
+      listener.waitForCompletion();
+
+      for (Map<String, String> r : listener.results) {
+        ndv.put(r.get(COL_COLUMN).toUpperCase(), Long.valueOf(r.get(COL_NDV)));
+        rowCount = Math.max(rowCount, Long.valueOf(r.get(COL_STATCOUNT)));
+      }*/
+    }
+
+    materialized = true;
+  }
+
+  /**
+   * materialize on nodes that have an attached stats table
+   */
+  public static class StatsMaterializationVisitor extends RelVisitor {
+    private QueryContext context;
+
+    public static void materialize(final RelNode relNode, final QueryContext context) {
+      new StatsMaterializationVisitor(context).go(relNode);
+    }
+
+    private StatsMaterializationVisitor(QueryContext context) {
+      this.context = context;
+    }
+
+    @Override
+    public void visit(RelNode node, int ordinal, RelNode parent) {
+      if (node instanceof TableScan) {
+        try {
+          final DrillTable drillTable = node.getTable().unwrap(DrillTable.class);
+          final DrillStatsTable statsTable = drillTable.getStatsTable();
+          if (statsTable != null) {
+            statsTable.materialize(context);
+          }
+        } catch (Exception e) {
+          // Log a warning and proceed. We don't want to fail a query.
+          logger.warn("Failed to materialize the stats. Continuing without stats.", e);
+        }
+      }
+      super.visit(node, ordinal, parent);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java
index 373683f..13f1600 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java
@@ -17,17 +17,22 @@
  */
 package org.apache.drill.exec.planner.cost;
 
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
+import org.apache.calcite.rel.metadata.RelColumnOrigin;
 import org.apache.calcite.rel.metadata.RelMdDistinctRowCount;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
 import org.apache.drill.exec.planner.logical.DrillScanRel;
 
+import java.util.List;
+
 public class DrillRelMdDistinctRowCount extends RelMdDistinctRowCount {
   private static final DrillRelMdDistinctRowCount INSTANCE =
       new DrillRelMdDistinctRowCount();
@@ -61,4 +66,51 @@ public class DrillRelMdDistinctRowCount extends RelMdDistinctRowCount {
     // Consistent with the estimation of Aggregate row count in RelMdRowCount : distinctRowCount = rowCount * 10%.
     return scan.estimateRowCount(mq) * 0.1;
   }
+
+  public Double getDistinctRowCount(RelNode rel, RelMetadataQuery mq, ImmutableBitSet groupKey, RexNode predicate) {
+    if (rel instanceof DrillScanRel) {
+      return getDistinctRowCount((DrillScanRel) rel, mq, groupKey);
+    } else {
+      return super.getDistinctRowCount(rel, mq, groupKey, predicate);
+    }
+  }
+
+  /**
+   * Estimates the number of rows which would be produced by a GROUP BY on the
+   * set of columns indicated by groupKey.
+   * column").
+   */
+  private Double getDistinctRowCount(DrillScanRel scan, RelMetadataQuery mq, ImmutableBitSet groupKey) {
+    if (scan.getDrillTable() == null || scan.getDrillTable().getStatsTable() == null) {
+      // If there is no table or metadata (stats) table associated with scan, estimate the distinct row count.
+      // Consistent with the estimation of Aggregate row count in RelMdRowCount : distinctRowCount = rowCount * 10%.
+      return scan.getRows() * 0.1;
+    }
+
+    // TODO: may be we should get the column origin of each group by key before we look up it in metadata table?
+    List<RelColumnOrigin> cols = Lists.newArrayList();
+
+    if (groupKey.length() == 0) {
+      return new Double(0);
+    }
+
+    DrillStatsTable md = scan.getDrillTable().getStatsTable();
+
+    final double rc = mq.getRowCount(scan);
+    double s = 1.0;
+    for (int i = 0; i < groupKey.length(); i++) {
+      final String colName = scan.getRowType().getFieldNames().get(i);
+      if (!groupKey.get(i) && colName.equals("*")) {
+        continue;
+      }
+
+      Double d = md.getNdv(colName);
+      if (d == null) {
+        continue;
+      }
+
+      s *= 1 - d / rc;
+    }
+    return new Double((1 - s) * rc);
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java
index 7f15fb3..343affb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.planner.cost;
 
+import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.SingleRel;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.Filter;
@@ -31,6 +32,8 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.drill.exec.planner.common.DrillLimitRelBase;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
+import org.apache.drill.exec.planner.logical.DrillScanRel;
 
 public class DrillRelMdRowCount extends RelMdRowCount {
   private static final DrillRelMdRowCount INSTANCE = new DrillRelMdRowCount();
@@ -81,4 +84,20 @@ public class DrillRelMdRowCount extends RelMdRowCount {
   public Double getRowCount(Join rel, RelMetadataQuery mq) {
     return rel.estimateRowCount(mq);
   }
+
+  public Double getRowCount(RelNode rel, RelMetadataQuery mq) {
+    if (rel instanceof DrillScanRel) {
+      return getRowCount((DrillScanRel)rel, mq);
+    }
+    return super.getRowCount(rel, mq);
+  }
+
+  private Double getRowCount(DrillScanRel scanRel, RelMetadataQuery mq) {
+    final DrillStatsTable md = scanRel.getDrillTable().getStatsTable();
+    if (md != null) {
+      return md.getRowCount();
+    }
+
+    return super.getRowCount(scanRel, mq);
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAnalyzeRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAnalyzeRel.java
new file mode 100644
index 0000000..5d570f3
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAnalyzeRel.java
@@ -0,0 +1,71 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.logical;
+
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.drill.common.logical.data.Analyze;
+import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.exec.planner.cost.DrillCostBase;
+import org.apache.drill.exec.planner.torel.ConversionContext;
+
+import java.util.List;
+
+/**
+ * Drill logical node for "Analyze".
+ */
+public class DrillAnalyzeRel extends SingleRel implements DrillRel {
+
+  public DrillAnalyzeRel(RelOptCluster cluster, RelTraitSet traits, RelNode child) {
+    super(cluster, traits, child);
+  }
+
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+    final double dRows = mq.getRowCount(getInput());
+    final double dCpu = dRows * DrillCostBase.COMPARE_CPU_COST;
+    final double dIo = 0;
+    return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new DrillAnalyzeRel(getCluster(), traitSet, sole(inputs));
+  }
+
+  @Override
+  public LogicalOperator implement(DrillImplementor implementor) {
+    final LogicalOperator inputOp = implementor.visitChild(this, 0, getInput());
+    final Analyze rel = new Analyze();
+    rel.setInput(inputOp);
+
+    return rel;
+  }
+
+  public static DrillAnalyzeRel convert(Analyze analyze, ConversionContext context) throws InvalidRelException {
+    RelNode input = context.toRel(analyze.getInput());
+    return new DrillAnalyzeRel(context.getCluster(), context.getLogicalTraits(), input);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
index ed9b32f..afddbfc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
@@ -30,6 +30,7 @@ import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
 import org.apache.drill.exec.physical.base.SchemalessScan;
 import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.server.options.SessionOptionManager;
@@ -45,6 +46,7 @@ public abstract class DrillTable implements Table {
   private final Object selection;
   private final StoragePlugin plugin;
   private final String userName;
+  private DrillStatsTable statsTable;
   private GroupScan scan;
   private SessionOptionManager options;
 
@@ -131,6 +133,14 @@ public abstract class DrillTable implements Table {
     return Statistics.UNKNOWN;
   }
 
+  public DrillStatsTable getStatsTable() {
+    return statsTable;
+  }
+
+  public void setStatsTable(DrillStatsTable statsTable) {
+    this.statsTable = statsTable;
+  }
+
   public RelNode toRel(RelOptTable.ToRelContext context, RelOptTable table) {
     return new DrillScanRel(context.getCluster(),
         context.getCluster().traitSetOf(DrillRel.DRILL_LOGICAL),
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/FileSystemCreateTableEntry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/FileSystemCreateTableEntry.java
index 23ea23f..6869616 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/FileSystemCreateTableEntry.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/FileSystemCreateTableEntry.java
@@ -46,6 +46,7 @@ public class FileSystemCreateTableEntry implements CreateTableEntry {
   private FileSystemConfig storageConfig;
   private FormatPlugin formatPlugin;
   private String location;
+  private boolean append;
   private final List<String> partitionColumns;
   private final StorageStrategy storageStrategy;
 
@@ -53,6 +54,7 @@ public class FileSystemCreateTableEntry implements CreateTableEntry {
   public FileSystemCreateTableEntry(@JsonProperty("storageConfig") FileSystemConfig storageConfig,
                                     @JsonProperty("formatConfig") FormatPluginConfig formatConfig,
                                     @JsonProperty("location") String location,
+                                    @JsonProperty("append") boolean append,
                                     @JsonProperty("partitionColumn") List<String> partitionColumns,
                                     @JsonProperty("storageStrategy") StorageStrategy storageStrategy,
                                     @JacksonInject StoragePluginRegistry engineRegistry)
@@ -67,11 +69,13 @@ public class FileSystemCreateTableEntry implements CreateTableEntry {
   public FileSystemCreateTableEntry(FileSystemConfig storageConfig,
                                     FormatPlugin formatPlugin,
                                     String location,
+                                    boolean append,
                                     List<String> partitionColumns,
                                     StorageStrategy storageStrategy) {
     this.storageConfig = storageConfig;
     this.formatPlugin = formatPlugin;
     this.location = location;
+    this.append = append;
     this.partitionColumns = partitionColumns;
     this.storageStrategy = storageStrategy;
   }
@@ -94,7 +98,7 @@ public class FileSystemCreateTableEntry implements CreateTableEntry {
           formatPlugin.getName())).build(logger);
     }
 
-    AbstractWriter writer = formatPlugin.getWriter(child, location, partitionColumns);
+    AbstractWriter writer = formatPlugin.getWriter(child, location, append, partitionColumns);
     writer.setStorageStrategy(storageStrategy);
     return writer;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AnalyzePrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AnalyzePrule.java
new file mode 100644
index 0000000..4cac5d9
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AnalyzePrule.java
@@ -0,0 +1,64 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.physical;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
+import org.apache.drill.exec.planner.logical.DrillAnalyzeRel;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+
+import java.util.List;
+
+public class AnalyzePrule extends Prule {
+  public static final RelOptRule INSTANCE = new AnalyzePrule();
+
+  private static final List<String> FUNCTIONS = ImmutableList.of(
+      "statcount", // total number of entries in the table
+      "nonnullstatcount", // total number of non-null entries in the table
+      "ndv",  // total distinctive values in table
+      "hll" // HyperLogLog
+  );
+
+  public AnalyzePrule() {
+    super(RelOptHelper.some(DrillAnalyzeRel.class, DrillRel.DRILL_LOGICAL, RelOptHelper.any(RelNode.class)), "Prel.AnalyzePrule");
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final DrillAnalyzeRel analyze = (DrillAnalyzeRel) call.rel(0);
+    final RelNode input = call.rel(1);
+
+    final RelTraitSet traits = input.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON);
+    final RelNode convertedInput = convert(input, traits);
+
+    final StatsAggPrel statsAggPrel = new StatsAggPrel(convertedInput, analyze.getCluster(), FUNCTIONS);
+
+    final List<String> mapFileds = Lists.newArrayList(FUNCTIONS);
+    mapFileds.add(DrillStatsTable.COL_COLUMN);
+    final SingleRel newAnalyze = new UnpivotMapsPrel(statsAggPrel, analyze.getCluster(), mapFileds);
+
+    call.transformTo(newAnalyze);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StatsAggPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StatsAggPrel.java
new file mode 100644
index 0000000..124246b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StatsAggPrel.java
@@ -0,0 +1,86 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.physical;
+
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.StatisticsAggregate;
+import org.apache.drill.exec.planner.common.DrillRelNode;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+public class StatsAggPrel extends SingleRel implements DrillRelNode, Prel {
+
+  private List<String> functions;
+
+  public StatsAggPrel(RelNode child, RelOptCluster cluster, List<String> functions) {
+    super(cluster, child.getTraitSet(), child);
+    this.functions = ImmutableList.copyOf(functions);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new StatsAggPrel(sole(inputs), getCluster(), ImmutableList.copyOf(functions));
+  }
+
+  @Override
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator)
+      throws IOException {
+    Prel child = (Prel) this.getInput();
+
+    PhysicalOperator childPOP = child.getPhysicalOperator(creator);
+
+    StatisticsAggregate g = new StatisticsAggregate(childPOP, functions);
+
+    return creator.addMetadata(this, g);
+  }
+
+  @Override
+  public Iterator<Prel> iterator() {
+    return PrelUtil.iter(getInput());
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value)
+      throws E {
+    return logicalVisitor.visitPrel(this, value);
+  }
+
+  @Override
+  public SelectionVectorMode[] getSupportedEncodings() {
+    return SelectionVectorMode.ALL;
+  }
+
+  @Override
+  public SelectionVectorMode getEncoding() {
+    return SelectionVectorMode.NONE;
+  }
+
+  @Override
+  public boolean needsFinalColumnReordering() {
+    return true;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnpivotMapsPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnpivotMapsPrel.java
new file mode 100644
index 0000000..4fc7aae
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnpivotMapsPrel.java
@@ -0,0 +1,85 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.physical;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.UnpivotMaps;
+import org.apache.drill.exec.planner.common.DrillRelNode;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+
+public class UnpivotMapsPrel extends SingleRel implements Prel, DrillRelNode {
+
+  private List<String> mapFieldsNames;
+
+  public UnpivotMapsPrel(RelNode child, RelOptCluster cluster, List<String> mapFieldsNames) {
+    super(cluster, child.getTraitSet(), child);
+    this.mapFieldsNames = mapFieldsNames;
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new UnpivotMapsPrel(sole(inputs), getCluster(), mapFieldsNames);
+  }
+
+  @Override
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator)
+      throws IOException {
+    Prel child = (Prel) this.getInput();
+
+    PhysicalOperator childPOP = child.getPhysicalOperator(creator);
+
+    UnpivotMaps um = new UnpivotMaps(childPOP, mapFieldsNames);
+    return creator.addMetadata(this, um);
+  }
+
+  @Override
+  public Iterator<Prel> iterator() {
+    return PrelUtil.iter(getInput());
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value)
+      throws E {
+    return logicalVisitor.visitPrel(this, value);
+  }
+
+  @Override
+  public SelectionVectorMode[] getSupportedEncodings() {
+    return SelectionVectorMode.DEFAULT;
+  }
+
+  @Override
+  public SelectionVectorMode getEncoding() {
+    return SelectionVectorMode.NONE;
+  }
+
+  @Override
+  public boolean needsFinalColumnReordering() {
+    return false;
+  }
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java
new file mode 100644
index 0000000..cdfe31b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/AnalyzeTableHandler.java
@@ -0,0 +1,157 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.sql.handlers;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.tools.RelConversionException;
+import org.apache.calcite.tools.ValidationException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.logical.DrillAnalyzeRel;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.DrillScreenRel;
+import org.apache.drill.exec.planner.logical.DrillStoreRel;
+import org.apache.drill.exec.planner.logical.DrillWriterRel;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.sql.SchemaUtilites;
+import org.apache.drill.exec.planner.sql.parser.SqlAnalyzeTable;
+import org.apache.drill.exec.store.AbstractSchema;
+import org.apache.drill.exec.util.Pointer;
+import org.apache.drill.exec.work.foreman.ForemanSetupException;
+import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
+
+import java.io.IOException;
+import java.util.List;
+
+public class AnalyzeTableHandler extends DefaultSqlHandler {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AnalyzeTableHandler.class);
+
+  public AnalyzeTableHandler(SqlHandlerConfig config, Pointer<String> textPlan) {
+    super(config, textPlan);
+  }
+
+  @Override
+  public PhysicalPlan getPlan(SqlNode sqlNode)
+      throws ValidationException, RelConversionException, IOException, ForemanSetupException {
+    final SqlAnalyzeTable sqlAnalyzeTable = unwrap(sqlNode, SqlAnalyzeTable.class);
+
+    verifyNoUnsupportedFunctions(sqlAnalyzeTable);
+
+    SqlIdentifier tableIdentifier = sqlAnalyzeTable.getTableIdentifier();
+    SqlSelect scanSql = new SqlSelect(
+        SqlParserPos.ZERO, /* position */
+        SqlNodeList.EMPTY, /* keyword list */
+        getColumnList(sqlAnalyzeTable), /*select list */
+        tableIdentifier, /* from */
+        null, /* where */
+        null, /* group by */
+        null, /* having */
+        null, /* windowDecls */
+        null, /* orderBy */
+        null, /* offset */
+        null /* fetch */
+    );
+
+    final ConvertedRelNode convertedRelNode = validateAndConvert(rewrite(scanSql));
+    final RelDataType validatedRowType = convertedRelNode.getValidatedRowType();
+
+    final RelNode relScan = convertedRelNode.getConvertedNode();
+
+    final String tableName = sqlAnalyzeTable.getName();
+    final AbstractSchema drillSchema = SchemaUtilites.resolveToMutableDrillSchema(
+        config.getConverter().getDefaultSchema(), sqlAnalyzeTable.getSchemaPath());
+
+    if (SqlHandlerUtil.getTableFromSchema(drillSchema, tableName) == null) {
+      throw UserException.validationError()
+          .message("No table with given name [%s] exists in schema [%s]", tableName, drillSchema.getFullSchemaName())
+          .build(logger);
+    }
+
+    // Convert the query to Drill Logical plan and insert a writer operator on top.
+    DrillRel drel = convertToDrel(relScan, drillSchema, tableName);
+    Prel prel = convertToPrel(drel, validatedRowType);
+    logAndSetTextPlan("Drill Physical", prel, logger);
+    PhysicalOperator pop = convertToPop(prel);
+    PhysicalPlan plan = convertToPlan(pop);
+    log("Drill Plan", plan, logger);
+
+    return plan;
+  }
+
+  private SqlNodeList getColumnList(final SqlAnalyzeTable sqlAnalyzeTable) {
+    final SqlNodeList columnList = new SqlNodeList(SqlParserPos.ZERO);
+
+    final List<String> fields = sqlAnalyzeTable.getFieldNames();
+    if (fields == null || fields.size() <= 0) {
+      columnList.add(new SqlIdentifier("*", SqlParserPos.ZERO));
+    } else {
+      for(String field : fields) {
+        columnList.add(new SqlIdentifier(field, SqlParserPos.ZERO));
+      }
+    }
+
+    return columnList;
+  }
+
+  protected DrillRel convertToDrel(RelNode relNode, AbstractSchema schema, String analyzeTableName)
+      throws RelConversionException, SqlUnsupportedException {
+    final DrillRel convertedRelNode = convertToDrel(relNode);
+
+    if (convertedRelNode instanceof DrillStoreRel) {
+      throw new UnsupportedOperationException();
+    }
+
+    final RelNode analyzeRel = new DrillAnalyzeRel(
+        convertedRelNode.getCluster(),
+        convertedRelNode.getTraitSet(),
+        convertedRelNode
+    );
+
+    final RelNode writerRel = new DrillWriterRel(
+        analyzeRel.getCluster(),
+        analyzeRel.getTraitSet(),
+        analyzeRel,
+        schema.appendToStatsTable(analyzeTableName)
+    );
+
+    return new DrillScreenRel(writerRel.getCluster(), writerRel.getTraitSet(), writerRel);
+  }
+
+  // make sure no unsupported features in ANALYZE statement are used
+  private static void verifyNoUnsupportedFunctions(final SqlAnalyzeTable analyzeTable) {
+    // throw unsupported error for functions that are not yet implemented
+    if (analyzeTable.getEstimate()) {
+      throw UserException.unsupportedError()
+          .message("Statistics estimation is not yet supported.")
+          .build(logger);
+    }
+
+    if (analyzeTable.getPercent() != 100) {
+      throw UserException.unsupportedError()
+          .message("Statistics from sampling is not yet supported.")
+          .build(logger);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
index 0881dc1..52ae7b1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
@@ -73,6 +73,7 @@ import org.apache.drill.exec.physical.impl.join.JoinUtils;
 import org.apache.drill.exec.planner.PlannerPhase;
 import org.apache.drill.exec.planner.PlannerType;
 import org.apache.drill.exec.planner.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.common.DrillStatsTable.StatsMaterializationVisitor;
 import org.apache.drill.exec.planner.cost.DrillDefaultRelMetadataProvider;
 import org.apache.drill.exec.planner.logical.DrillProjectRel;
 import org.apache.drill.exec.planner.logical.DrillRel;
@@ -230,6 +231,7 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
     }
 
     try {
+      StatsMaterializationVisitor.materialize(relNode, context);
 
       // HEP for rules, which are failed at the LOGICAL_PLANNING stage for Volcano planner
       final RelNode setOpTransposeNode = transform(PlannerType.HEP, PlannerPhase.PRE_LOGICAL_PLANNING, relNode);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
index 57a7e17..210f43b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
@@ -64,6 +64,7 @@ public class CompoundIdentifierConverter extends SqlShuttle {
     // Every element of the array corresponds to the item in the list
     // returned by getOperandList() method for concrete SqlCall implementation.
     REWRITE_RULES = ImmutableMap.<Class<? extends SqlCall>, RewriteType[]>builder()
+        .put(SqlAnalyzeTable.class, arrayOf(D, D, E, D))
         .put(SqlSelect.class, arrayOf(D, E, D, E, E, E, E, E, D, D))
         .put(SqlCreateTable.class, arrayOf(D, D, D, E, D, D))
         .put(SqlCreateView.class, arrayOf(D, E, E, D))
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlAnalyzeTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlAnalyzeTable.java
new file mode 100644
index 0000000..91f83bf
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlAnalyzeTable.java
@@ -0,0 +1,157 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.sql.parser;
+
+import java.util.List;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlNumericLiteral;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.Util;
+import org.apache.drill.exec.planner.sql.handlers.AbstractSqlHandler;
+import org.apache.drill.exec.planner.sql.handlers.AnalyzeTableHandler;
+import org.apache.drill.exec.planner.sql.handlers.SqlHandlerConfig;
+import org.apache.drill.exec.util.Pointer;
+
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+/**
+ * SQL tree for ANALYZE statement.
+ */
+public class SqlAnalyzeTable extends DrillSqlCall {
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("ANALYZE_TABLE", SqlKind.OTHER) {
+    public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
+      Preconditions.checkArgument(operands.length == 4, "SqlAnalyzeTable.createCall() has to get 4 operands!");
+      return new SqlAnalyzeTable(pos, (SqlIdentifier) operands[0], (SqlLiteral) operands[1],
+          (SqlNodeList) operands[2], (SqlNumericLiteral) operands[3]
+      );
+    }
+  };
+
+  private final SqlIdentifier tblName;
+  private final SqlLiteral estimate;
+  private final SqlNodeList fieldList;
+  private final SqlNumericLiteral percent;
+
+  public SqlAnalyzeTable(SqlParserPos pos, SqlIdentifier tblName, SqlLiteral estimate,
+      SqlNodeList fieldList, SqlNumericLiteral percent) {
+    super(pos);
+    this.tblName = tblName;
+    this.estimate = estimate;
+    this.fieldList = fieldList;
+    this.percent = percent;
+  }
+
+  @Override
+  public SqlOperator getOperator() {
+    return OPERATOR;
+  }
+
+  @Override
+  public List<SqlNode> getOperandList() {
+    final List<SqlNode> operands = Lists.newArrayListWithCapacity(4);
+    operands.add(tblName);
+    operands.add(estimate);
+    operands.add(fieldList);
+    operands.add(percent);
+    return operands;
+  }
+
+  @Override
+  public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+    writer.keyword("ANALYZE");
+    writer.keyword("TABLE");
+    tblName.unparse(writer, leftPrec, rightPrec);
+    writer.keyword(estimate.booleanValue() ? "ESTIMATE" : "COMPUTE");
+    writer.keyword("STATISTICS");
+    writer.keyword("FOR");
+
+    if (fieldList != null && fieldList.size() > 0) {
+      writer.keyword("COLUMNS");
+      writer.keyword("(");
+      fieldList.get(0).unparse(writer, leftPrec, rightPrec);
+      for (int i = 1; i < fieldList.size(); i++) {
+        writer.keyword(",");
+        fieldList.get(i).unparse(writer, leftPrec, rightPrec);
+      }
+      writer.keyword(")");
+    } else {
+      writer.keyword("ALL");
+      writer.keyword("COLUMNS");
+    }
+    writer.keyword("SAMPLE");
+    percent.unparse(writer, leftPrec, rightPrec);
+    writer.keyword("PERCENT");
+  }
+
+  @Override
+  public AbstractSqlHandler getSqlHandler(SqlHandlerConfig config, Pointer<String> textPlan) {
+    return new AnalyzeTableHandler(config, textPlan);
+  }
+
+  @Override
+  public AbstractSqlHandler getSqlHandler(SqlHandlerConfig config) {
+    return getSqlHandler(config, null);
+  }
+
+  public List<String> getSchemaPath() {
+    if (tblName.isSimple()) {
+      return ImmutableList.of();
+    }
+
+    return tblName.names.subList(0, tblName.names.size() - 1);
+  }
+
+  public SqlIdentifier getTableIdentifier() {
+    return tblName;
+  }
+
+  public String getName() {
+    return Util.last(tblName.names);
+  }
+
+  public List<String> getFieldNames() {
+    if (fieldList == null) {
+      return ImmutableList.of();
+    }
+
+    List<String> columnNames = Lists.newArrayList();
+    for (SqlNode node : fieldList.getList()) {
+      columnNames.add(node.toString());
+    }
+    return columnNames;
+  }
+
+  public boolean getEstimate() {
+    return estimate.booleanValue();
+  }
+
+  public int getPercent() {
+    return percent.intValue(true);
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
index 8f4f067..c510525 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
@@ -149,6 +149,40 @@ public abstract class AbstractSchema implements Schema, SchemaPartitionExplorer,
   }
 
   /**
+   * Create stats table entry for given <i>tableName</i>.
+   * @param tableName
+   * @return
+   */
+  public CreateTableEntry createStatsTable(String tableName) {
+    throw UserException.unsupportedError()
+        .message("Statistics tables are not supported in schema [%s]", getSchemaPath())
+        .build(logger);
+  }
+
+  /**
+   * Create an append statistics table entry for given <i>tableName</i>. If there is not existing
+   * statistics table, a new one is created.
+   * @param tableName
+   * @return
+   */
+  public CreateTableEntry appendToStatsTable(String tableName) {
+    throw UserException.unsupportedError()
+        .message("Statistics tables are not supported in schema [%s]", getSchemaPath())
+        .build(logger);
+  }
+
+  /**
+   * Get the statistics table for given <i>tableName</i>
+   * @param tableName
+   * @return
+   */
+  public Table getStatsTable(String tableName) {
+    throw UserException.unsupportedError()
+        .message("Statistics tables are not supported in schema [%s]", getSchemaPath())
+        .build(logger);
+  }
+
+  /**
    * Reports whether to show items from this schema in INFORMATION_SCHEMA
    * tables.
    * (Controls ... TODO:  Doc.:  Mention what this typically controls or
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SubSchemaWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SubSchemaWrapper.java
index 3a747a6..2539c64 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SubSchemaWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SubSchemaWrapper.java
@@ -68,6 +68,21 @@ public class SubSchemaWrapper extends AbstractSchema {
   }
 
   @Override
+  public CreateTableEntry createStatsTable(String tableName) {
+    return innerSchema.createStatsTable(tableName);
+  }
+
+  @Override
+  public CreateTableEntry appendToStatsTable(String tableName) {
+    return innerSchema.appendToStatsTable(tableName);
+  }
+
+  @Override
+  public Table getStatsTable(String tableName) {
+    return innerSchema.getStatsTable(tableName);
+  }
+
+  @Override
   public Collection<Function> getFunctions(String name) {
     return innerSchema.getFunctions(name);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
index 795cbd2..45dea13 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
@@ -164,6 +164,21 @@ public class FileSystemSchemaFactory extends AbstractSchemaFactory {
     }
 
     @Override
+    public CreateTableEntry createStatsTable(String tableName) {
+      return defaultSchema.createStatsTable(tableName);
+    }
+
+    @Override
+    public CreateTableEntry appendToStatsTable(String tableName) {
+      return defaultSchema.appendToStatsTable(tableName);
+    }
+
+    @Override
+    public Table getStatsTable(String tableName) {
+      return defaultSchema.getStatsTable(tableName);
+    }
+
+    @Override
     public AbstractSchema getDefaultSchema() {
       return defaultSchema;
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java
index 27a72e3..bf258c2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java
@@ -49,7 +49,7 @@ public interface FormatPlugin {
 
   FormatMatcher getMatcher();
 
-  AbstractWriter getWriter(PhysicalOperator child, String location, List<String> partitionColumns) throws IOException;
+  public AbstractWriter getWriter(PhysicalOperator child, String location, boolean append, List<String> partitionColumns) throws IOException;
 
   Set<StoragePluginOptimizerRule> getOptimizerRules();
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
index 493278c..fc51221 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.store.dfs;
 
 import static java.util.Collections.unmodifiableList;
+import static org.apache.drill.exec.dotdrill.DotDrillType.STATS;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -44,6 +45,7 @@ import org.apache.calcite.schema.TranslatableTable;
 import org.apache.commons.lang3.SystemUtils;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.common.config.LogicalPlanPersistence;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.logical.FormatPluginConfig;
@@ -53,7 +55,7 @@ import org.apache.drill.exec.dotdrill.DotDrillFile;
 import org.apache.drill.exec.dotdrill.DotDrillType;
 import org.apache.drill.exec.dotdrill.DotDrillUtil;
 import org.apache.drill.exec.dotdrill.View;
-import org.apache.drill.exec.store.StorageStrategy;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
 import org.apache.drill.exec.planner.logical.CreateTableEntry;
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.planner.logical.DrillTranslatableTable;
@@ -65,6 +67,8 @@ import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.PartitionNotFoundException;
 import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.drill.exec.util.DrillFileSystemUtil;
+import org.apache.drill.exec.store.StorageStrategy;
+import org.apache.drill.exec.store.easy.json.JSONFormatPlugin;
 import org.apache.drill.exec.util.ImpersonationUtil;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -202,6 +206,17 @@ public class WorkspaceSchemaFactory {
     return plugin;
   }
 
+  // Ensure given tableName is not a stats table
+  private static void ensureNotStatsTable(final String tableName) {
+    if (tableName.toLowerCase().endsWith(STATS.getEnding())) {
+      throw UserException
+          .validationError()
+          .message("Given table [%s] is already a stats table. " +
+              "Cannot perform stats operations on a stats table.", tableName)
+          .build(logger);
+    }
+  }
+
   /**
    * Implementation of a table macro that generates a table based on parameters
    */
@@ -551,7 +566,47 @@ public class WorkspaceSchemaFactory {
       } catch (UnsupportedOperationException e) {
         logger.debug("The filesystem for this workspace does not support this operation.", e);
       }
-      return tables.get(tableKey);
+      final DrillTable table = tables.get(tableKey);
+      setMetadataTable(table, tableName);
+      return table;
+    }
+
+    private void setMetadataTable(final DrillTable table, final String tableName) {
+      if (table == null) {
+        return;
+      }
+
+      // If this itself is the stats table, then skip it.
+      if (tableName.toLowerCase().endsWith(STATS.getEnding())) {
+        return;
+      }
+
+      try {
+        if (table.getStatsTable() == null) {
+          Table statsTable = getStatsTable(tableName);
+          if (statsTable != null) {
+            table.setStatsTable(new DrillStatsTable(getFullSchemaName(), getStatsTableName(tableName)));
+          }
+        }
+      } catch (final Exception e) {
+        logger.warn("Failed to find the stats table for table [{}] in schema [{}]", tableName, getFullSchemaName());
+      }
+    }
+
+    // Get stats table name for a given table name.
+    private String getStatsTableName(final String tableName) {
+      final Path tablePath = new Path(config.getLocation(), tableName);
+      try {
+        if (fs.isDirectory(tablePath)) {
+          return tableName + Path.SEPARATOR + STATS.getEnding();
+        } else {
+          return tableName + STATS.getEnding();
+        }
+      } catch (final Exception e) {
+        throw new DrillRuntimeException(
+            String.format("Failed to find the location of the stats for table [%s] in schema [%s]",
+                tableName, getFullSchemaName()));
+      }
     }
 
     @Override
@@ -571,6 +626,34 @@ public class WorkspaceSchemaFactory {
     public CreateTableEntry createNewTable(String tableName, List<String> partitionColumns, StorageStrategy storageStrategy) {
       String storage = schemaConfig.getOption(ExecConstants.OUTPUT_FORMAT_OPTION).string_val;
       FormatPlugin formatPlugin = plugin.getFormatPlugin(storage);
+      return createOrAppendToTable(tableName, false, formatPlugin, partitionColumns, storageStrategy);
+    }
+
+    @Override
+    public CreateTableEntry createStatsTable(String tableName) {
+      ensureNotStatsTable(tableName);
+      final String statsTableName = getStatsTableName(tableName);
+      FormatPlugin formatPlugin = plugin.getFormatPlugin(JSONFormatPlugin.DEFAULT_NAME);
+      return createOrAppendToTable(statsTableName, false, formatPlugin, ImmutableList.<String>of(),
+          StorageStrategy.DEFAULT);
+    }
+
+    @Override
+    public CreateTableEntry appendToStatsTable(String tableName) {
+      ensureNotStatsTable(tableName);
+      final String statsTableName = getStatsTableName(tableName);
+      FormatPlugin formatPlugin = plugin.getFormatPlugin(JSONFormatPlugin.DEFAULT_NAME);
+      return createOrAppendToTable(statsTableName, true, formatPlugin, ImmutableList.<String>of(),
+          StorageStrategy.DEFAULT);
+    }
+
+    @Override
+    public Table getStatsTable(String tableName) {
+      return getTable(getStatsTableName(tableName));
+    }
+
+    private CreateTableEntry createOrAppendToTable(String tableName, boolean append, FormatPlugin formatPlugin,
+        List<String> partitonColumns, StorageStrategy storageStrategy) {
       if (formatPlugin == null) {
         throw new UnsupportedOperationException(
           String.format("Unsupported format '%s' in workspace '%s'", config.getDefaultInputFormat(),
@@ -581,7 +664,8 @@ public class WorkspaceSchemaFactory {
           (FileSystemConfig) plugin.getConfig(),
           formatPlugin,
           config.getLocation() + Path.SEPARATOR + tableName,
-          partitionColumns,
+          append,
+          partitonColumns,
           storageStrategy);
     }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
index 4c550c3..ed1651e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
@@ -193,8 +193,8 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
   }
 
   @Override
-  public AbstractWriter getWriter(PhysicalOperator child, String location, List<String> partitionColumns) throws IOException {
-    return new EasyWriter(child, location, partitionColumns, this);
+  public AbstractWriter getWriter(PhysicalOperator child, String location, boolean append, List<String> partitionColumns) throws IOException {
+    return new EasyWriter(child, location, append, partitionColumns, this);
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyWriter.java
index 379e2c9..9f41206 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyWriter.java
@@ -40,6 +40,7 @@ public class EasyWriter extends AbstractWriter {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(EasyWriter.class);
 
   private final String location;
+  private final boolean append;
   private final List<String> partitionColumns;
   private final EasyFormatPlugin<?> formatPlugin;
 
@@ -47,6 +48,7 @@ public class EasyWriter extends AbstractWriter {
   public EasyWriter(
       @JsonProperty("child") PhysicalOperator child,
       @JsonProperty("location") String location,
+      @JsonProperty("append") boolean append,
       @JsonProperty("partitionColumns") List<String> partitionColumns,
       @JsonProperty("storageStrategy") StorageStrategy storageStrategy,
       @JsonProperty("storage") StoragePluginConfig storageConfig,
@@ -57,18 +59,21 @@ public class EasyWriter extends AbstractWriter {
     this.formatPlugin = (EasyFormatPlugin<?>) engineRegistry.getFormatPlugin(storageConfig, formatConfig);
     Preconditions.checkNotNull(formatPlugin, "Unable to load format plugin for provided format config.");
     this.location = location;
+    this.append = append;
     this.partitionColumns = partitionColumns;
     setStorageStrategy(storageStrategy);
   }
 
   public EasyWriter(PhysicalOperator child,
                          String location,
+                         boolean append,
                          List<String> partitionColumns,
                          EasyFormatPlugin<?> formatPlugin) {
 
     super(child);
     this.formatPlugin = formatPlugin;
     this.location = location;
+    this.append = append;
     this.partitionColumns = partitionColumns;
   }
 
@@ -77,6 +82,11 @@ public class EasyWriter extends AbstractWriter {
     return location;
   }
 
+  @JsonProperty("append")
+  public boolean getAppend() {
+    return append;
+  }
+
   @JsonProperty("storage")
   public StoragePluginConfig getStorageConfig(){
     return formatPlugin.getStorageConfig();
@@ -94,7 +104,7 @@ public class EasyWriter extends AbstractWriter {
 
   @Override
   protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
-    EasyWriter writer = new EasyWriter(child, location, partitionColumns, formatPlugin);
+    EasyWriter writer = new EasyWriter(child, location, append, partitionColumns, formatPlugin);
     writer.setStorageStrategy(getStorageStrategy());
     return writer;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
index 11dc204..ab90cda 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
@@ -70,6 +70,8 @@ public class JSONFormatPlugin extends EasyFormatPlugin<JSONFormatConfig> {
     Map<String, String> options = new HashMap<>();
 
     options.put("location", writer.getLocation());
+    options.put("append", Boolean.toString(writer.getAppend()));
+
     FragmentHandle handle = context.getHandle();
     String fragmentId = String.format("%d_%d", handle.getMajorFragmentId(), handle.getMinorFragmentId());
     options.put("prefix", fragmentId);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonRecordWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonRecordWriter.java
index 2e80b3f..d533c0a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonRecordWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JsonRecordWriter.java
@@ -48,6 +48,7 @@ public class JsonRecordWriter extends JSONOutputRecordWriter implements RecordWr
 
   private Path cleanUpLocation;
   private String location;
+  private boolean append;
   private String prefix;
 
   private String fieldDelimiter;
@@ -74,6 +75,7 @@ public class JsonRecordWriter extends JSONOutputRecordWriter implements RecordWr
   @Override
   public void init(Map<String, String> writerOptions) throws IOException {
     this.location = writerOptions.get("location");
+    this.append = writerOptions.get("append").equalsIgnoreCase("true") ? true : false;
     this.prefix = writerOptions.get("prefix");
     this.fieldDelimiter = writerOptions.get("separator");
     this.extension = writerOptions.get("extension");
@@ -83,7 +85,11 @@ public class JsonRecordWriter extends JSONOutputRecordWriter implements RecordWr
 
     this.fs = FileSystem.get(fsConf);
 
-    Path fileName = new Path(location, prefix + "_" + index + "." + extension);
+    Path fileName;
+    do {
+      fileName = new Path(location, prefix + "_" + (index++) + "." + extension);
+    } while (append && fs.exists(fileName));
+
     try {
       // json writer does not support partitions, so only one file can be created
       // and thus only one location should be deleted in case of abort
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
index f46cc1c..876cd5b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
@@ -126,14 +126,15 @@ public class ParquetFormatPlugin implements FormatPlugin {
   }
 
   @Override
-  public AbstractWriter getWriter(PhysicalOperator child, String location, List<String> partitionColumns) {
-    return new ParquetWriter(child, location, partitionColumns, this);
+  public AbstractWriter getWriter(PhysicalOperator child, String location, boolean append, List<String> partitionColumns) throws IOException {
+    return new ParquetWriter(child, location, append, partitionColumns, this);
   }
 
   public RecordWriter getRecordWriter(FragmentContext context, ParquetWriter writer) throws IOException, OutOfMemoryException {
     Map<String, String> options = new HashMap<>();
 
     options.put("location", writer.getLocation());
+    options.put("append", Boolean.toString(writer.getAppend()));
 
     FragmentHandle handle = context.getHandle();
     String fragmentId = String.format("%d_%d", handle.getMajorFragmentId(), handle.getMinorFragmentId());
@@ -261,6 +262,9 @@ public class ParquetFormatPlugin implements FormatPlugin {
               new FormatSelection(plugin.getConfig(), selection));
         }
       }
+      if (!super.supportDirectoryReads() && selection.containsDirectories(fs)) {
+        return null;
+      }
       return super.isReadable(fs, selection, fsPlugin, storageEngineName, schemaConfig);
     }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
index aea3218..6298c1a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
@@ -53,6 +53,7 @@ public class ParquetWriter extends AbstractWriter {
   public static final int WRITER_VERSION = 2;
 
   private final String location;
+  private final boolean append;
   private final List<String> partitionColumns;
   private final ParquetFormatPlugin formatPlugin;
 
@@ -60,6 +61,7 @@ public class ParquetWriter extends AbstractWriter {
   public ParquetWriter(
           @JsonProperty("child") PhysicalOperator child,
           @JsonProperty("location") String location,
+          @JsonProperty("append") boolean append,
           @JsonProperty("partitionColumns") List<String> partitionColumns,
           @JsonProperty("storageStrategy") StorageStrategy storageStrategy,
           @JsonProperty("storage") StoragePluginConfig storageConfig,
@@ -69,18 +71,21 @@ public class ParquetWriter extends AbstractWriter {
     this.formatPlugin = (ParquetFormatPlugin) engineRegistry.getFormatPlugin(storageConfig, new ParquetFormatConfig());
     Preconditions.checkNotNull(formatPlugin, "Unable to load format plugin for provided format config.");
     this.location = location;
+    this.append = append;
     this.partitionColumns = partitionColumns;
     setStorageStrategy(storageStrategy);
   }
 
   public ParquetWriter(PhysicalOperator child,
                        String location,
+                       boolean append,
                        List<String> partitionColumns,
                        ParquetFormatPlugin formatPlugin) {
 
     super(child);
     this.formatPlugin = formatPlugin;
     this.location = location;
+    this.append = append;
     this.partitionColumns = partitionColumns;
   }
 
@@ -89,6 +94,11 @@ public class ParquetWriter extends AbstractWriter {
     return location;
   }
 
+  @JsonProperty("append")
+  public boolean getAppend() {
+    return append;
+  }
+
   @JsonProperty("storage")
   public StoragePluginConfig getStorageConfig(){
     return formatPlugin.getStorageConfig();
@@ -111,7 +121,7 @@ public class ParquetWriter extends AbstractWriter {
 
   @Override
   protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
-    ParquetWriter writer = new ParquetWriter(child, location, partitionColumns, formatPlugin);
+    ParquetWriter writer = new ParquetWriter(child, location, append, partitionColumns, formatPlugin);
     writer.setStorageStrategy(getStorageStrategy());
     return writer;
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestAnalyze.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestAnalyze.java
new file mode 100644
index 0000000..0f15fb3
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestAnalyze.java
@@ -0,0 +1,105 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.sql;
+
+import org.apache.drill.PlanTestBase;
+import org.apache.drill.exec.ExecConstants;
+import org.junit.Test;
+
+public class TestAnalyze extends PlanTestBase {
+
+  // Analyze for all columns
+  @Test
+  public void basic1() throws Exception {
+    try {
+      test("ALTER SESSION SET `planner.slice_target` = 1");
+      test("CREATE TABLE dfs_test.tmp.region_basic1 AS SELECT * from cp.`region.json`");
+      test("ANALYZE TABLE dfs_test.tmp.region_basic1 COMPUTE STATISTICS FOR ALL COLUMNS");
+      test("SELECT * FROM dfs_test.tmp.`region_basic1/.stats.drill`");
+
+      testBuilder()
+          .sqlQuery("SELECT `column`, statcount, nonnullstatcount, ndv FROM dfs_test.tmp.`region_basic1/.stats.drill`")
+          .unOrdered()
+          .baselineColumns("column", "statcount", "nonnullstatcount", "ndv")
+          .baselineValues("region_id", 110L, 110L, 107L)
+          .baselineValues("sales_city", 110L, 110L, 111L)
+          .baselineValues("sales_state_province", 110L, 110L, 13L)
+          .baselineValues("sales_district", 110L, 110L, 22L)
+          .baselineValues("sales_region", 110L, 110L, 8L)
+          .baselineValues("sales_country", 110L, 110L, 4L)
+          .baselineValues("sales_district_id", 110L, 110L, 23L)
+          .go();
+
+      // we can't compare the ndv for correctness as it is an estimate and not accurate
+      testBuilder()
+          .sqlQuery("SELECT statcount FROM dfs_test.tmp.`region_basic1/.stats.drill` WHERE `column` = 'region_id'")
+          .unOrdered()
+          .sqlBaselineQuery("SELECT count(region_id) AS statcount FROM dfs_test.tmp.region_basic1")
+          .go();
+
+    } finally {
+      test("ALTER SESSION SET `planner.slice_target` = " + ExecConstants.SLICE_TARGET_DEFAULT);
+    }
+  }
+
+  // Analyze for only a subset of the columns in table
+  @Test
+  public void basic2() throws Exception {
+    try {
+      test("ALTER SESSION SET `planner.slice_target` = 1");
+      test("CREATE TABLE dfs_test.tmp.employee_basic2 AS SELECT * from cp.`employee.json`");
+      test("ANALYZE TABLE dfs_test.tmp.employee_basic2 COMPUTE STATISTICS FOR COLUMNS (employee_id, birth_date)");
+      test("SELECT * FROM dfs_test.tmp.`employee_basic2/.stats.drill`");
+
+      testBuilder()
+          .sqlQuery("SELECT `column`, statcount, nonnullstatcount, ndv FROM dfs_test.tmp.`employee_basic2/.stats.drill`")
+          .unOrdered()
+          .baselineColumns("column", "statcount", "nonnullstatcount", "ndv")
+          .baselineValues("employee_id", 1155L, 1155L, 1144L)
+          .baselineValues("birth_date", 1155L, 1155L, 53L)
+          .go();
+
+      // we can't compare the ndv for correctness as it is an estimate and not accurate
+      testBuilder()
+          .sqlQuery("SELECT statcount FROM dfs_test.tmp.`employee_basic2/.stats.drill` WHERE `column` = 'birth_date'")
+          .unOrdered()
+          .sqlBaselineQuery("SELECT count(birth_date) AS statcount FROM dfs_test.tmp.employee_basic2")
+          .go();
+
+    } finally {
+      test("ALTER SESSION SET `planner.slice_target` = " + ExecConstants.SLICE_TARGET_DEFAULT);
+    }
+  }
+
+  @Test
+  public void join() throws Exception {
+    try {
+      test("ALTER SESSION SET `planner.slice_target` = 1");
+      test("CREATE TABLE dfs_test.tmp.lineitem AS SELECT * FROM cp.`tpch/lineitem.parquet`");
+      test("CREATE TABLE dfs_test.tmp.orders AS select * FROM cp.`tpch/orders.parquet`");
+      test("ANALYZE TABLE dfs_test.tmp.lineitem COMPUTE STATISTICS FOR ALL COLUMNS");
+      test("ANALYZE TABLE dfs_test.tmp.orders COMPUTE STATISTICS FOR ALL COLUMNS");
+      test("SELECT * FROM dfs_test.tmp.`lineitem/.stats.drill`");
+      test("SELECT * FROM dfs_test.tmp.`orders/.stats.drill`");
+
+      test("SELECT * FROM dfs_test.tmp.`lineitem` l JOIN dfs_test.tmp.`orders` o ON l.l_orderkey = o.o_orderkey");
+    } finally {
+      test("ALTER SESSION SET `planner.slice_target` = " + ExecConstants.SLICE_TARGET_DEFAULT);
+    }
+  }
+}
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index 67a3bb8..32c2a90 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -172,6 +172,10 @@
           <groupId>sqlline</groupId>
           <artifactId>sqlline</artifactId>
         </exclusion>
+        <exclusion>
+          <artifactId>stream</artifactId>
+          <groupId>com.clearspring.analytics</groupId>
+        </exclusion>
       </exclusions>
     </dependency>
     <dependency>
diff --git a/logical/src/main/java/org/apache/drill/common/logical/data/Analyze.java b/logical/src/main/java/org/apache/drill/common/logical/data/Analyze.java
new file mode 100644
index 0000000..711050d
--- /dev/null
+++ b/logical/src/main/java/org/apache/drill/common/logical/data/Analyze.java
@@ -0,0 +1,35 @@
+/**
+ * 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 org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("analyze")
+public class Analyze extends SingleInputOperator {
+
+  @JsonCreator
+  public Analyze() { }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
+    return logicalVisitor.visitAnalyze(this, value);
+  }
+}
diff --git a/logical/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java b/logical/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
index 482146f..c46dc43 100644
--- a/logical/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
+++ b/logical/src/main/java/org/apache/drill/common/logical/data/visitors/AbstractLogicalVisitor.java
@@ -19,6 +19,7 @@ package org.apache.drill.common.logical.data.visitors;
 
 import org.apache.drill.common.logical.data.LateralJoin;
 import org.apache.drill.common.logical.data.Unnest;
+import org.apache.drill.common.logical.data.Analyze;
 import org.apache.drill.common.logical.data.Values;
 import org.apache.drill.common.logical.data.Filter;
 import org.apache.drill.common.logical.data.Flatten;
@@ -46,6 +47,11 @@ public abstract class AbstractLogicalVisitor<T, X, E extends Throwable> implemen
     }
 
     @Override
+    public T visitAnalyze(Analyze analyze, X value) throws E {
+      return visitOp(analyze, value);
+    }
+
+    @Override
     public T visitScan(Scan scan, X value) throws E {
         return visitOp(scan, value);
     }
diff --git a/logical/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java b/logical/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
index 9d9013e..ee9036c 100644
--- a/logical/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
+++ b/logical/src/main/java/org/apache/drill/common/logical/data/visitors/LogicalVisitor.java
@@ -20,6 +20,7 @@ package org.apache.drill.common.logical.data.visitors;
 
 import org.apache.drill.common.logical.data.LateralJoin;
 import org.apache.drill.common.logical.data.Unnest;
+import org.apache.drill.common.logical.data.Analyze;
 import org.apache.drill.common.logical.data.Values;
 import org.apache.drill.common.logical.data.Filter;
 import org.apache.drill.common.logical.data.Flatten;
@@ -51,6 +52,7 @@ public interface LogicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
     public RETURN visitGroupingAggregate(GroupingAggregate groupBy, EXTRA value) throws EXCEP;
     public RETURN visitFilter(Filter filter, EXTRA value) throws EXCEP;
     public RETURN visitFlatten(Flatten flatten, EXTRA value) throws EXCEP;
+    public RETURN visitAnalyze(Analyze analyze, EXTRA value) throws EXCEP;
 
     public RETURN visitProject(Project project, EXTRA value) throws EXCEP;
     public RETURN visitValues(Values constant, EXTRA value) throws EXCEP;
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
index c540c8f..635f972 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
@@ -601,6 +601,14 @@ public final class UserBitShared {
      * <code>SYSLOG_SUB_SCAN = 58;</code>
      */
     SYSLOG_SUB_SCAN(58, 58),
+    /**
+     * <code>STATISTICS_AGGREGATE = 59;</code>
+     */
+    STATISTICS_AGGREGATE(59, 59),
+    /**
+     * <code>UNPIVOT_MAPS = 60;</code>
+     */
+    UNPIVOT_MAPS(60, 60),
     ;
 
     /**
@@ -839,6 +847,14 @@ public final class UserBitShared {
      * <code>SYSLOG_SUB_SCAN = 58;</code>
      */
     public static final int SYSLOG_SUB_SCAN_VALUE = 58;
+    /**
+     * <code>STATISTICS_AGGREGATE = 59;</code>
+     */
+    public static final int STATISTICS_AGGREGATE_VALUE = 59;
+    /**
+     * <code>UNPIVOT_MAPS = 60;</code>
+     */
+    public static final int UNPIVOT_MAPS_VALUE = 60;
 
 
     public final int getNumber() { return value; }
@@ -904,6 +920,8 @@ public final class UserBitShared {
         case 56: return RUNTIME_FILTER;
         case 57: return ROWKEY_JOIN;
         case 58: return SYSLOG_SUB_SCAN;
+        case 59: return STATISTICS_AGGREGATE;
+        case 60: return UNPIVOT_MAPS;
         default: return null;
       }
     }
@@ -24644,7 +24662,7 @@ public final class UserBitShared {
       "entState\022\013\n\007SENDING\020\000\022\027\n\023AWAITING_ALLOCA" +
       "TION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHED\020\003\022\r\n\tCAN" +
       "CELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCELLATION_REQ" +
-      "UESTED\020\006*\247\t\n\020CoreOperatorType\022\021\n\rSINGLE_" +
+      "UESTED\020\006*\323\t\n\020CoreOperatorType\022\021\n\rSINGLE_" +
       "SENDER\020\000\022\024\n\020BROADCAST_SENDER\020\001\022\n\n\006FILTER" +
       "\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHASH_JOIN\020\004\022\016\n" +
       "\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITION_SENDER\020\006" +
@@ -24674,10 +24692,11 @@ public final class UserBitShared {
       "\025\n\021SEQUENCE_SUB_SCAN\0205\022\023\n\017PARTITION_LIMI" +
       "T\0206\022\023\n\017PCAPNG_SUB_SCAN\0207\022\022\n\016RUNTIME_FILT" +
       "ER\0208\022\017\n\013ROWKEY_JOIN\0209\022\023\n\017SYSLOG_SUB_SCAN" +
-      "\020:*g\n\nSaslStatus\022\020\n\014SASL_UNKNOWN\020\000\022\016\n\nSA" +
-      "SL_START\020\001\022\024\n\020SASL_IN_PROGRESS\020\002\022\020\n\014SASL" +
-      "_SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B.\n\033org.apach" +
-      "e.drill.exec.protoB\rUserBitSharedH\001"
+      "\020:\022\030\n\024STATISTICS_AGGREGATE\020;\022\020\n\014UNPIVOT_" +
+      "MAPS\020<*g\n\nSaslStatus\022\020\n\014SASL_UNKNOWN\020\000\022\016" +
+      "\n\nSASL_START\020\001\022\024\n\020SASL_IN_PROGRESS\020\002\022\020\n\014" +
+      "SASL_SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B.\n\033org.a" +
+      "pache.drill.exec.protoB\rUserBitSharedH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java
index 7d5041c..051f82f 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/CoreOperatorType.java
@@ -80,7 +80,9 @@ public enum CoreOperatorType implements com.dyuproject.protostuff.EnumLite<CoreO
     PCAPNG_SUB_SCAN(55),
     RUNTIME_FILTER(56),
     ROWKEY_JOIN(57),
-    SYSLOG_SUB_SCAN(58);
+    SYSLOG_SUB_SCAN(58),
+    STATISTICS_AGGREGATE(59),
+    UNPIVOT_MAPS(60);
     
     public final int number;
     
@@ -157,6 +159,8 @@ public enum CoreOperatorType implements com.dyuproject.protostuff.EnumLite<CoreO
             case 56: return RUNTIME_FILTER;
             case 57: return ROWKEY_JOIN;
             case 58: return SYSLOG_SUB_SCAN;
+            case 59: return STATISTICS_AGGREGATE;
+            case 60: return UNPIVOT_MAPS;
             default: return null;
         }
     }
diff --git a/protocol/src/main/protobuf/UserBitShared.proto b/protocol/src/main/protobuf/UserBitShared.proto
index a0438b7..ca4e273 100644
--- a/protocol/src/main/protobuf/UserBitShared.proto
+++ b/protocol/src/main/protobuf/UserBitShared.proto
@@ -351,6 +351,8 @@ enum CoreOperatorType {
   RUNTIME_FILTER = 56;
   ROWKEY_JOIN = 57;
   SYSLOG_SUB_SCAN = 58;
+  STATISTICS_AGGREGATE = 59;
+  UNPIVOT_MAPS = 60;
 }
 
 /* Registry that contains list of jars, each jar contains its name and list of function signatures.


[drill] 02/02: DRILL-1328: Support table statistics - Part 2

Posted by gp...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 469be17597e7b7c6bc1de9863dcb6c5604a55f0c
Author: Gautam Parai <gp...@maprtech.com>
AuthorDate: Thu Aug 21 14:59:53 2014 -0700

    DRILL-1328: Support table statistics - Part 2
    
    Add support for avg row-width and major type statistics.
    Parallelize the ANALYZE implementation and stats UDF implementation to improve stats collection performance.
    Update/fix rowcount, selectivity and ndv computations to improve plan costing.
    Add options for configuring collection/usage of statistics.
    Add new APIs and implementation for stats writer (as a precursor to Drill Metastore APIs).
    Fix several stats/costing related issues identified while running TPC-H nad TPC-DS queries.
    Add support for CPU sampling and nested scalar columns.
    Add more testcases for collection and usage of statistics and fix remaining unit/functional test failures.
    
    Thanks to Venki Korukanti (@vkorukanti) for the description below (modified to account for new changes). He graciously agreed to rebase the patch to latest master, fixed few issues and added few tests.
    
    FUNCS: Statistics functions as UDFs:
    Separate
    Currently using FieldReader to ensure consistent output type so that Unpivot doesn't get confused. All stats columns should be Nullable, so that stats functions can return NULL when N/A.
    * custom versions of "count" that always return BigInt
    * HyperLogLog based NDV that returns BigInt that works only on VarChars
    * HyperLogLog with binary output that only works on VarChars
    
    OPS: Updated protobufs for new ops
    
    OPS: Implemented StatisticsMerge
    
    OPS: Implemented StatisticsUnpivot
    
    ANALYZE: AnalyzeTable functionality
    * JavaCC syntax more-or-less copied from LucidDB.
    * (Basic) AnalyzePrule: DrillAnalyzeRel -> UnpivotPrel StatsMergePrel FilterPrel(for sampling) StatsAggPrel ScanPrel
    
    ANALYZE: Add getMetadataTable() to AbstractSchema
    
    USAGE: Change field access in QueryWrapper
    
    USAGE: Add getDrillTable() to DrillScanRelBase and ScanPrel
    * since ScanPrel does not inherit from DrillScanRelBase, this requires adding a DrillTable to the constructor
    * This is done so that a custom ReflectiveRelMetadataProvider can access the DrillTable associated with Logical/Physical scans.
    
    USAGE: Attach DrillStatsTable to DrillTable.
    * DrillStatsTable represents the data scanned from a corresponding ".stats.drill" table
    * In order to avoid doing query execution right after the ".stats.drill" table is found, metadata is not actually collected until the MaterializationVisitor is used.
    ** Currently, the metadata source must be a string (so that a SQL query can be created). Doing this with a table is probably more complicated.
    ** Query is set up to extract only the most recent statistics results for each column.
    
    closes #729
---
 .../drill/exec/store/mapr/TableFormatPlugin.java   |    2 +-
 .../exec/store/mapr/db/MapRDBFormatPlugin.java     |   16 +
 .../drill/exec/store/mapr/db/MapRDBGroupScan.java  |    1 +
 .../store/mapr/streams/StreamsFormatPlugin.java    |   20 +-
 .../exec/store/syslog/SyslogFormatPlugin.java      |   19 +
 .../native/client/src/protobuf/UserBitShared.pb.cc |   15 +-
 .../native/client/src/protobuf/UserBitShared.pb.h  |    7 +-
 .../drill/exec/store/hbase/HBaseGroupScan.java     |    1 +
 .../org/apache/drill/exec/store/hive/HiveScan.java |    1 +
 .../drill/exec/store/jdbc/JdbcGroupScan.java       |    9 +-
 .../org/apache/drill/exec/store/jdbc/JdbcPrel.java |    9 +-
 .../drill/exec/store/jdbc/JdbcRecordReader.java    |    7 +-
 .../apache/drill/exec/store/jdbc/JdbcSubScan.java  |    9 +-
 .../drill/exec/store/kafka/KafkaGroupScan.java     |    1 +
 .../drill/exec/store/kudu/KuduGroupScan.java       |    1 +
 .../drill/exec/store/mongo/MongoGroupScan.java     |    1 +
 .../exec/store/openTSDB/OpenTSDBGroupScan.java     |    1 +
 exec/java-exec/src/main/codegen/data/Parser.tdd    |   20 -
 .../src/main/codegen/includes/parserImpls.ftl      |   18 +-
 .../codegen/templates/AbstractRecordWriter.java    |    5 +
 .../templates/JsonBaseStatisticsRecordWriter.java  |  139 +
 .../src/main/codegen/templates/RecordWriter.java   |    6 +-
 ...cordWriter.java => StatisticsRecordWriter.java} |   44 +-
 .../templates/StatisticsRecordWriterImpl.java      |  119 +
 .../java/org/apache/drill/exec/ExecConstants.java  |   42 +-
 .../apache/drill/exec/dotdrill/DotDrillType.java   |    1 -
 .../drill/exec/expr/fn/impl/MathFunctions.java     |   28 +
 .../exec/expr/fn/impl/StatisticsAggrFunctions.java | 5599 +++++++++++++++++++-
 .../apache/drill/exec/ops/BaseFragmentContext.java |   24 +-
 .../apache/drill/exec/ops/ContextInformation.java  |    1 +
 .../org/apache/drill/exec/ops/FragmentContext.java |   29 +-
 .../apache/drill/exec/ops/FragmentContextImpl.java |   20 +-
 .../org/apache/drill/exec/ops/QueryContext.java    |   38 +-
 .../org/apache/drill/exec/ops/UdfUtilities.java    |    8 +-
 .../exec/physical/base/AbstractDbGroupScan.java    |    2 -
 .../exec/physical/base/AbstractGroupScan.java      |    5 +
 .../physical/base/AbstractPhysicalVisitor.java     |    6 +
 .../apache/drill/exec/physical/base/GroupScan.java |   12 +-
 .../exec/physical/base/PhysicalOperatorUtil.java   |   26 +-
 .../drill/exec/physical/base/PhysicalVisitor.java  |    2 +
 .../drill/exec/physical/base/SchemalessScan.java   |    6 +
 .../exec/physical/config/StatisticsAggregate.java  |    8 +-
 ...atisticsAggregate.java => StatisticsMerge.java} |   51 +-
 .../drill/exec/physical/config/UnpivotMaps.java    |    6 +-
 ...Batch.java => StatisticsWriterRecordBatch.java} |   70 +-
 .../exec/physical/impl/WriterRecordBatch.java      |    3 +
 .../physical/impl/aggregate/InternalBatch.java     |    6 +-
 .../impl/aggregate/StatisticsAggBatch.java         |  223 -
 .../physical/impl/aggregate/StreamingAggBatch.java |    2 +-
 .../impl/statistics/AbstractMergedStatistic.java   |   37 +-
 .../impl/statistics/AvgWidthMergedStatistic.java   |  150 +
 .../impl/statistics/CntDupsMergedStatistic.java    |  103 +
 .../impl/statistics/ColTypeMergedStatistic.java    |   87 +
 .../impl/statistics/ColumnMergedStatistic.java     |   68 +
 .../impl/statistics/HLLMergedStatistic.java        |  130 +
 .../physical/impl/statistics/MergedStatistic.java  |   62 +
 .../impl/statistics/MergedStatisticFactory.java    |   62 +
 .../impl/statistics/NDVMergedStatistic.java        |  198 +
 .../impl/statistics/NNRowCountMergedStatistic.java |   95 +
 .../impl/statistics/RowCountMergedStatistic.java   |   96 +
 .../exec/physical/impl/statistics/Statistic.java   |   44 +
 .../impl/statistics/StatisticsAggBatch.java        |  249 +
 .../StatisticsAggBatchCreator.java                 |    5 +-
 .../impl/statistics/StatisticsMergeBatch.java      |  405 ++
 .../StatisticsMergeBatchCreator.java}              |   16 +-
 .../impl/unpivot/UnpivotMapsBatchCreator.java      |    6 +-
 .../impl/unpivot/UnpivotMapsRecordBatch.java       |  114 +-
 .../planner/FileSystemPartitionDescriptor.java     |   11 +-
 .../exec/planner/common/DrillAggregateRelBase.java |    9 +
 .../exec/planner/common/DrillFilterRelBase.java    |    4 +-
 .../exec/planner/common/DrillJoinRelBase.java      |   12 +-
 .../drill/exec/planner/common/DrillRelOptUtil.java |  102 +
 .../exec/planner/common/DrillScanRelBase.java      |   14 +
 .../drill/exec/planner/common/DrillStatsTable.java |  386 +-
 .../planner/cost/DrillRelMdDistinctRowCount.java   |  171 +-
 .../exec/planner/cost/DrillRelMdRowCount.java      |   63 +-
 .../exec/planner/cost/DrillRelMdSelectivity.java   |  201 +-
 .../exec/planner/logical/DrillAnalyzeRel.java      |   26 +-
 .../drill/exec/planner/logical/DrillTable.java     |    3 +-
 .../logical/FileSystemCreateTableEntry.java        |    6 +-
 .../drill/exec/planner/physical/AggPruleBase.java  |    2 +-
 .../drill/exec/planner/physical/AnalyzePrule.java  |  113 +-
 .../planner/physical/ConvertCountToDirectScan.java |    4 +-
 .../exec/planner/physical/DirectScanPrel.java      |   33 +-
 .../exec/planner/physical/DirectScanPrule.java     |    3 -
 .../exec/planner/physical/NestedLoopJoinPrule.java |   20 +-
 .../exec/planner/physical/PlannerSettings.java     |    6 +
 .../drill/exec/planner/physical/ScanPrel.java      |   29 +-
 .../drill/exec/planner/physical/StatsAggPrel.java  |   25 +-
 .../{StatsAggPrel.java => StatsMergePrel.java}     |   33 +-
 .../exec/planner/physical/UnpivotMapsPrel.java     |   12 +-
 .../planner/physical/visitor/BasePrelVisitor.java  |    6 +
 .../exec/planner/physical/visitor/PrelVisitor.java |    2 +
 .../physical/visitor/PrelVisualizerVisitor.java    |    8 +
 .../drill/exec/planner/sql/DrillSqlWorker.java     |   20 +
 .../drill/exec/planner/sql/SchemaUtilites.java     |   19 +
 .../planner/sql/handlers/AnalyzeTableHandler.java  |  199 +-
 .../planner/sql/handlers/DefaultSqlHandler.java    |    3 +-
 .../planner/sql/handlers/FindLimit0Visitor.java    |    8 +-
 .../sql/parser/CompoundIdentifierConverter.java    |    7 +-
 .../sql/parser/DrillCompoundIdentifier.java        |   27 +-
 .../exec/planner/sql/parser/SqlAnalyzeTable.java   |   29 +-
 .../impl/DrillParserWithCompoundIdConverter.java   |    9 +-
 .../exec/server/options/SystemOptionManager.java   |    7 +-
 .../drill/exec/server/options/TypeValidators.java  |   19 +
 .../org/apache/drill/exec/store/RecordReader.java  |   11 +
 .../drill/exec/store/avro/AvroFormatPlugin.java    |   18 +
 .../apache/drill/exec/store/dfs/FormatPlugin.java  |   11 +-
 .../exec/store/dfs/WorkspaceSchemaFactory.java     |   67 +-
 .../exec/store/dfs/easy/EasyFormatPlugin.java      |   21 +-
 .../drill/exec/store/dfs/easy/EasyGroupScan.java   |    1 +
 .../drill/exec/store/dfs/easy/EasyWriter.java      |   12 +-
 .../drill/exec/store/direct/DirectSubScan.java     |    9 +-
 .../exec/store/easy/json/JSONFormatPlugin.java     |  100 +-
 .../exec/store/easy/json/JsonRecordWriter.java     |    7 +-
 .../easy/json/JsonStatisticsRecordWriter.java      |  482 ++
 .../sequencefile/SequenceFileFormatPlugin.java     |   17 +
 .../exec/store/easy/text/TextFormatPlugin.java     |   17 +
 .../exec/store/httpd/HttpdLogFormatPlugin.java     |   17 +
 .../drill/exec/store/image/ImageFormatPlugin.java  |   18 +
 .../drill/exec/store/log/LogFormatPlugin.java      |   19 +
 .../store/parquet/AbstractParquetGroupScan.java    |    1 +
 .../exec/store/parquet/ParquetFormatPlugin.java    |   50 +-
 .../drill/exec/store/parquet/ParquetWriter.java    |   12 +-
 .../drill/exec/store/pcap/PcapFormatPlugin.java    |   18 +
 .../exec/store/pcapng/PcapngFormatPlugin.java      |   19 +
 .../drill/exec/store/pojo/PojoRecordReader.java    |   13 +-
 .../drill/exec/store/sys/SystemTableScan.java      |    1 +
 .../java/org/apache/drill/exec/util/Utilities.java |   13 +-
 .../java-exec/src/main/resources/drill-module.conf |    9 +-
 .../java/org/apache/drill/TestSelectivity.java     |    4 +-
 .../impersonation/TestImpersonationMetadata.java   |   43 +-
 .../exec/physical/impl/TestLocalExchange.java      |    3 +-
 .../impl/partitionsender/TestPartitionSender.java  |    3 +-
 .../apache/drill/exec/pop/TestFragmentChecker.java |    3 +-
 .../org/apache/drill/exec/sql/TestAnalyze.java     |  375 +-
 .../drill/exec/sql/TestSqlBracketlessSyntax.java   |    2 +-
 .../org/apache/drill/test/DrillTestWrapper.java    |   36 +-
 .../java/org/apache/drill/test/TestBuilder.java    |   10 +-
 .../apache/drill/common/logical/data/Analyze.java  |   13 +-
 .../org/apache/drill/exec/proto/UserBitShared.java |   20 +-
 .../drill/exec/proto/beans/CoreOperatorType.java   |    4 +-
 protocol/src/main/protobuf/UserBitShared.proto     |    1 +
 143 files changed, 10739 insertions(+), 1008 deletions(-)

diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java
index 1c30264..aeb117a 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java
@@ -94,7 +94,7 @@ public abstract class TableFormatPlugin implements FormatPlugin {
 
   @Override
   public AbstractWriter getWriter(PhysicalOperator child, String location,
-      boolean append, List<String> partitionColumns) throws IOException {
+      List<String> partitionColumns) throws IOException {
     throw new UnsupportedOperationException();
   }
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
index b5cff58..3011f4e 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBFormatPlugin.java
@@ -24,6 +24,7 @@ import java.util.Set;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.planner.common.DrillStatsTable.TableStatistics;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.StoragePluginOptimizerRule;
 import org.apache.drill.exec.store.dfs.FileSelection;
@@ -35,6 +36,7 @@ import org.apache.drill.exec.store.mapr.db.binary.BinaryTableGroupScan;
 import org.apache.drill.exec.store.mapr.db.json.JsonScanSpec;
 import org.apache.drill.exec.store.mapr.db.json.JsonTableGroupScan;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.client.Connection;
@@ -134,6 +136,20 @@ public class MapRDBFormatPlugin extends TableFormatPlugin {
     return getGroupScan(userName, selection, columns, (IndexDesc) null /* indexDesc */);
   }
 
+  public boolean supportsStatistics() {
+    return false;
+  }
+
+  @Override
+  public TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+    throw new UnsupportedOperationException("unimplemented");
+  }
+
+  @Override
+  public void writeStatistics(TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+    throw new UnsupportedOperationException("unimplemented");
+  }
+
   @JsonIgnore
   public Configuration getHBaseConf() {
     return hbaseConf;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java
index 422a269..07943d9 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBGroupScan.java
@@ -279,6 +279,7 @@ public abstract class MapRDBGroupScan extends AbstractDbGroupScan {
     return storagePlugin;
   }
 
+  @Override
   @JsonProperty
   public List<SchemaPath> getColumns() {
     return columns;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java
index 76466ab..92f134f 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/streams/StreamsFormatPlugin.java
@@ -26,11 +26,14 @@ import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.physical.base.AbstractWriter;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.dfs.FileSelection;
 import org.apache.drill.exec.store.dfs.FormatMatcher;
 import org.apache.drill.exec.store.mapr.TableFormatPlugin;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 
 public class StreamsFormatPlugin extends TableFormatPlugin {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StreamsFormatPlugin.class);
@@ -64,7 +67,7 @@ public class StreamsFormatPlugin extends TableFormatPlugin {
 
   @Override
   public AbstractWriter getWriter(PhysicalOperator child, String location,
-      boolean append, List<String> partitionColumns) throws IOException {
+      List<String> partitionColumns) throws IOException {
     throw new UnsupportedOperationException();
   }
 
@@ -77,4 +80,19 @@ public class StreamsFormatPlugin extends TableFormatPlugin {
     throw UserException.unsupportedError().message("MapR streams can not be querried at this time.").build(logger);
   }
 
+  @Override
+  public boolean supportsStatistics() {
+    return false;
+  }
+
+  @Override
+  public DrillStatsTable.TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+    throw new UnsupportedOperationException("unimplemented");
+  }
+
+  @Override
+  public void writeStatistics(DrillStatsTable.TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+    throw new UnsupportedOperationException("unimplemented");
+  }
+
 }
diff --git a/contrib/format-syslog/src/main/java/org/apache/drill/exec/store/syslog/SyslogFormatPlugin.java b/contrib/format-syslog/src/main/java/org/apache/drill/exec/store/syslog/SyslogFormatPlugin.java
index bf4b4b4..6f81ac6 100644
--- a/contrib/format-syslog/src/main/java/org/apache/drill/exec/store/syslog/SyslogFormatPlugin.java
+++ b/contrib/format-syslog/src/main/java/org/apache/drill/exec/store/syslog/SyslogFormatPlugin.java
@@ -17,6 +17,8 @@
  */
 
 package org.apache.drill.exec.store.syslog;
+import java.io.IOException;
+import org.apache.drill.exec.planner.common.DrillStatsTable.TableStatistics;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
@@ -34,6 +36,8 @@ import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 
 import java.util.List;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 
 public class SyslogFormatPlugin extends EasyFormatPlugin<SyslogFormatConfig> {
 
@@ -79,4 +83,19 @@ public class SyslogFormatPlugin extends EasyFormatPlugin<SyslogFormatConfig> {
   public int getWriterOperatorType() {
     throw new UnsupportedOperationException("Drill does not support writing records to Syslog format.");
   }
+
+  @Override
+  public boolean supportsStatistics() {
+    return false;
+  }
+
+  @Override
+  public TableStatistics readStatistics(FileSystem fs, Path statsTablePath) throws IOException {
+    throw new UnsupportedOperationException("unimplemented");
+  }
+
+  @Override
+  public void writeStatistics(TableStatistics statistics, FileSystem fs, Path statsTablePath) throws IOException {
+    throw new UnsupportedOperationException("unimplemented");
+  }
 }
diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.cc b/contrib/native/client/src/protobuf/UserBitShared.pb.cc
index 1e07120..0db64d3 100644
--- a/contrib/native/client/src/protobuf/UserBitShared.pb.cc
+++ b/contrib/native/client/src/protobuf/UserBitShared.pb.cc
@@ -754,7 +754,7 @@ void protobuf_AddDesc_UserBitShared_2eproto() {
     "entState\022\013\n\007SENDING\020\000\022\027\n\023AWAITING_ALLOCA"
     "TION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHED\020\003\022\r\n\tCAN"
     "CELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCELLATION_REQ"
-    "UESTED\020\006*\247\t\n\020CoreOperatorType\022\021\n\rSINGLE_"
+    "UESTED\020\006*\351\t\n\020CoreOperatorType\022\021\n\rSINGLE_"
     "SENDER\020\000\022\024\n\020BROADCAST_SENDER\020\001\022\n\n\006FILTER"
     "\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHASH_JOIN\020\004\022\016\n"
     "\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITION_SENDER\020\006"
@@ -784,10 +784,12 @@ void protobuf_AddDesc_UserBitShared_2eproto() {
     "\025\n\021SEQUENCE_SUB_SCAN\0205\022\023\n\017PARTITION_LIMI"
     "T\0206\022\023\n\017PCAPNG_SUB_SCAN\0207\022\022\n\016RUNTIME_FILT"
     "ER\0208\022\017\n\013ROWKEY_JOIN\0209\022\023\n\017SYSLOG_SUB_SCAN"
-    "\020:*g\n\nSaslStatus\022\020\n\014SASL_UNKNOWN\020\000\022\016\n\nSA"
-    "SL_START\020\001\022\024\n\020SASL_IN_PROGRESS\020\002\022\020\n\014SASL"
-    "_SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B.\n\033org.apach"
-    "e.drill.exec.protoB\rUserBitSharedH\001", 5555);
+    "\020:\022\030\n\024STATISTICS_AGGREGATE\020;\022\020\n\014UNPIVOT_"
+    "MAPS\020<\022\024\n\020STATISTICS_MERGE\020=*g\n\nSaslStat"
+    "us\022\020\n\014SASL_UNKNOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020"
+    "SASL_IN_PROGRESS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013"
+    "SASL_FAILED\020\004B.\n\033org.apache.drill.exec.p"
+    "rotoB\rUserBitSharedH\001", 5621);
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile(
     "UserBitShared.proto", &protobuf_RegisterTypes);
   UserCredentials::default_instance_ = new UserCredentials();
@@ -967,6 +969,9 @@ bool CoreOperatorType_IsValid(int value) {
     case 56:
     case 57:
     case 58:
+    case 59:
+    case 60:
+    case 61:
       return true;
     default:
       return false;
diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.h b/contrib/native/client/src/protobuf/UserBitShared.pb.h
index b95b311..a8e6ccb 100644
--- a/contrib/native/client/src/protobuf/UserBitShared.pb.h
+++ b/contrib/native/client/src/protobuf/UserBitShared.pb.h
@@ -262,11 +262,14 @@ enum CoreOperatorType {
   PCAPNG_SUB_SCAN = 55,
   RUNTIME_FILTER = 56,
   ROWKEY_JOIN = 57,
-  SYSLOG_SUB_SCAN = 58
+  SYSLOG_SUB_SCAN = 58,
+  STATISTICS_AGGREGATE = 59,
+  UNPIVOT_MAPS = 60,
+  STATISTICS_MERGE = 61
 };
 bool CoreOperatorType_IsValid(int value);
 const CoreOperatorType CoreOperatorType_MIN = SINGLE_SENDER;
-const CoreOperatorType CoreOperatorType_MAX = SYSLOG_SUB_SCAN;
+const CoreOperatorType CoreOperatorType_MAX = STATISTICS_MERGE;
 const int CoreOperatorType_ARRAYSIZE = CoreOperatorType_MAX + 1;
 
 const ::google::protobuf::EnumDescriptor* CoreOperatorType_descriptor();
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java
index 22e318b..e1b41e8 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java
@@ -388,6 +388,7 @@ public class HBaseGroupScan extends AbstractGroupScan implements DrillHBaseConst
     return this.storagePluginConfig;
   }
 
+  @Override
   @JsonProperty
   public List<SchemaPath> getColumns() {
     return columns;
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveScan.java
index d631740..dfa2ee3 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveScan.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveScan.java
@@ -120,6 +120,7 @@ public class HiveScan extends AbstractGroupScan {
     return hiveStoragePlugin.getConfig();
   }
 
+  @Override
   @JsonProperty
   public List<SchemaPath> getColumns() {
     return columns;
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcGroupScan.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcGroupScan.java
index a981939..199d922 100644
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcGroupScan.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcGroupScan.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.store.jdbc;
 import java.util.List;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
@@ -38,14 +39,14 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
 public class JdbcGroupScan extends AbstractGroupScan {
 
   private final String sql;
-  private final List<String> columns;
+  private final List<SchemaPath> columns;
   private final JdbcStoragePlugin plugin;
   private final double rows;
 
   @JsonCreator
   public JdbcGroupScan(
       @JsonProperty("sql") String sql,
-      @JsonProperty("columns") List<String> columns,
+      @JsonProperty("columns") List<SchemaPath> columns,
       @JsonProperty("config") StoragePluginConfig config,
       @JsonProperty("rows") double rows,
       @JacksonInject StoragePluginRegistry plugins) throws ExecutionSetupException {
@@ -56,7 +57,7 @@ public class JdbcGroupScan extends AbstractGroupScan {
     this.rows = rows;
   }
 
-  JdbcGroupScan(String sql, List<String> columns, JdbcStoragePlugin plugin, double rows) {
+  JdbcGroupScan(String sql, List<SchemaPath> columns, JdbcStoragePlugin plugin, double rows) {
     super("");
     this.sql = sql;
     this.columns = columns;
@@ -91,7 +92,7 @@ public class JdbcGroupScan extends AbstractGroupScan {
     return sql;
   }
 
-  public List<String> getColumns() {
+  public List<SchemaPath> getColumns() {
     return columns;
   }
 
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
index b822940..85f88a8 100644
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
@@ -17,9 +17,11 @@
  */
 package org.apache.drill.exec.store.jdbc;
 
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Iterator;
 
+import java.util.List;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.adapter.jdbc.JdbcImplementor;
 import org.apache.calcite.plan.ConventionTraitDef;
@@ -32,6 +34,7 @@ import org.apache.calcite.rel.RelShuttleImpl;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.sql.SqlDialect;
+import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.planner.physical.PhysicalPlanCreator;
 import org.apache.drill.exec.planner.physical.Prel;
@@ -91,7 +94,11 @@ public class JdbcPrel extends AbstractRelNode implements Prel {
 
   @Override
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) {
-    JdbcGroupScan output = new JdbcGroupScan(sql, rowType.getFieldNames(), convention.getPlugin(), rows);
+    List<SchemaPath> columns = new ArrayList<>();
+    for (String col : rowType.getFieldNames()) {
+      columns.add(SchemaPath.getSimplePath(col));
+    }
+    JdbcGroupScan output = new JdbcGroupScan(sql, columns, convention.getPlugin(), rows);
     return creator.addMetadata(this, output);
   }
 
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java
index 011c9bc..5c6def2 100755
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java
@@ -35,6 +35,7 @@ import javax.sql.DataSource;
 
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
@@ -75,9 +76,9 @@ class JdbcRecordReader extends AbstractRecordReader {
   private final String sql;
   private ImmutableList<ValueVector> vectors;
   private ImmutableList<Copier<?>> copiers;
-  private final List<String> columns;
+  private final List<SchemaPath> columns;
 
-  public JdbcRecordReader(DataSource source, String sql, String storagePluginName, List<String> columns) {
+  public JdbcRecordReader(DataSource source, String sql, String storagePluginName, List<SchemaPath> columns) {
     this.source = source;
     this.sql = sql;
     this.storagePluginName = storagePluginName;
@@ -206,7 +207,7 @@ class JdbcRecordReader extends AbstractRecordReader {
       ImmutableList.Builder<Copier<?>> copierBuilder = ImmutableList.builder();
 
       for (int i = 1; i <= columnsCount; i++) {
-        String name = columns.get(i - 1);
+        String name = columns.get(i - 1).getRootSegmentPath();
         // column index in ResultSetMetaData starts from 1
         int jdbcType = meta.getColumnType(i);
         int width = meta.getPrecision(i);
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java
index 9bc6de8..c9d5f0d 100755
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.store.jdbc;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.AbstractSubScan;
 import org.apache.drill.exec.proto.beans.CoreOperatorType;
@@ -36,12 +37,12 @@ public class JdbcSubScan extends AbstractSubScan {
 
   private final String sql;
   private final JdbcStoragePlugin plugin;
-  private final List<String> columns;
+  private final List<SchemaPath> columns;
 
   @JsonCreator
   public JdbcSubScan(
       @JsonProperty("sql") String sql,
-      @JsonProperty("columns") List<String> columns,
+      @JsonProperty("columns") List<SchemaPath> columns,
       @JsonProperty("config") StoragePluginConfig config,
       @JacksonInject StoragePluginRegistry plugins) throws ExecutionSetupException {
     super("");
@@ -50,7 +51,7 @@ public class JdbcSubScan extends AbstractSubScan {
     this.plugin = (JdbcStoragePlugin) plugins.getPlugin(config);
   }
 
-  JdbcSubScan(String sql, List<String> columns, JdbcStoragePlugin plugin) {
+  JdbcSubScan(String sql, List<SchemaPath> columns, JdbcStoragePlugin plugin) {
     super("");
     this.sql = sql;
     this.columns = columns;
@@ -66,7 +67,7 @@ public class JdbcSubScan extends AbstractSubScan {
     return sql;
   }
 
-  public List<String> getColumns() {
+  public List<SchemaPath> getColumns() {
     return columns;
   }
 
diff --git a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaGroupScan.java b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaGroupScan.java
index 510d3ad..dded560 100644
--- a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaGroupScan.java
+++ b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaGroupScan.java
@@ -304,6 +304,7 @@ public class KafkaGroupScan extends AbstractGroupScan {
     return kafkaStoragePlugin.getConfig();
   }
 
+  @Override
   @JsonProperty
   public List<SchemaPath> getColumns() {
     return columns;
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java
index 1045950..593ca12 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduGroupScan.java
@@ -246,6 +246,7 @@ public class KuduGroupScan extends AbstractGroupScan {
     return kuduStoragePlugin.getConfig();
   }
 
+  @Override
   @JsonProperty
   public List<SchemaPath> getColumns() {
     return columns;
diff --git a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoGroupScan.java b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoGroupScan.java
index e33acd4..46ea567 100644
--- a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoGroupScan.java
+++ b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoGroupScan.java
@@ -561,6 +561,7 @@ public class MongoGroupScan extends AbstractGroupScan implements
     return Lists.newArrayList(affinityMap.values());
   }
 
+  @Override
   @JsonProperty
   public List<SchemaPath> getColumns() {
     return columns;
diff --git a/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBGroupScan.java b/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBGroupScan.java
index ad258df..8a09637 100644
--- a/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBGroupScan.java
+++ b/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBGroupScan.java
@@ -149,6 +149,7 @@ public class OpenTSDBGroupScan extends AbstractGroupScan {
     return storagePluginConfig;
   }
 
+  @Override
   @JsonProperty
   public List<SchemaPath> getColumns() {
     return columns;
diff --git a/exec/java-exec/src/main/codegen/data/Parser.tdd b/exec/java-exec/src/main/codegen/data/Parser.tdd
index ec56af4..4461f6d 100644
--- a/exec/java-exec/src/main/codegen/data/Parser.tdd
+++ b/exec/java-exec/src/main/codegen/data/Parser.tdd
@@ -42,25 +42,6 @@
     "COMPUTE",
     "ESTIMATE",
     "STATISTICS",
-    "COLUMNS",
-    "SAMPLE"
-  ]
-
-  # List of keywords from "keywords" section that are not reserved by SQL:2003 standard.
-  # Example: "DATABASES", "TABLES" are keywords but are not reserved by SQL:2003 standard.
-  # First keyword that starts the statement should be a reserved keyword, otherwise the current parser
-  # ends up considering it as a expression and fails.
-  nonReservedKeywords: [
-    "DATABASES",
-    "REPLACE",
-    "SCHEMAS",
-    "TABLES",
-    "FILES",
-    "METADATA",
-    "COMPUTE",
-    "ESTIMATE",
-    "STATISTICS",
-    "COLUMNS",
     "SAMPLE"
   ]
 
@@ -77,7 +58,6 @@
     "SqlRefreshMetadata()",
     "SqlCreateFunction()",
     "SqlDropFunction()",
-    "SqlRefreshMetadata()",
     "SqlAnalyzeTable()"
   ]
 
diff --git a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
index 2606006..2da1c4a 100644
--- a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
+++ b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
@@ -550,9 +550,9 @@ Pair<SqlNodeList, SqlNodeList> ParenthesizedCompoundIdentifierList() :
     SqlIdentifier id;
 }
 {
-    id = SimpleIdentifier() {list.add(id);}
+    id = CompoundIdentifier() {list.add(id);}
     (
-   <COMMA> id = SimpleIdentifier() {list.add(id);}) *
+   <COMMA> id = CompoundIdentifier() {list.add(id);}) *
     {
        return Pair.of(new SqlNodeList(list, getPos()), null);
     }
@@ -560,8 +560,8 @@ Pair<SqlNodeList, SqlNodeList> ParenthesizedCompoundIdentifierList() :
 </#if>
 /**
  * Parses a analyze statement.
- * ANALYZE TABLE tblname {COMPUTE | ESTIMATE} | STATISTICS FOR
- *      {ALL COLUMNS | COLUMNS (field1, field2, ...)} [ SAMPLE numeric PERCENT ]
+ * ANALYZE TABLE tblname {COMPUTE | ESTIMATE} | STATISTICS
+ *      [(column1, column2, ...)] [ SAMPLE numeric PERCENT ]
  */
 SqlNode SqlAnalyzeTable() :
 {
@@ -580,12 +580,10 @@ SqlNode SqlAnalyzeTable() :
         |
         <ESTIMATE> { estimate = SqlLiteral.createBoolean(true, pos); }
     )
-    <STATISTICS> <FOR>
-    (
-        ( <ALL> <COLUMNS> )
-        |
-        ( <COLUMNS> fieldList = ParseRequiredFieldList("Table") )
-    )
+    <STATISTICS>
+    [
+        (fieldList = ParseRequiredFieldList("Table"))
+    ]
     [
         <SAMPLE> percent = UnsignedNumericLiteral() <PERCENT>
         {
diff --git a/exec/java-exec/src/main/codegen/templates/AbstractRecordWriter.java b/exec/java-exec/src/main/codegen/templates/AbstractRecordWriter.java
index 070e947..2bd5faf 100644
--- a/exec/java-exec/src/main/codegen/templates/AbstractRecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/AbstractRecordWriter.java
@@ -81,4 +81,9 @@ public abstract class AbstractRecordWriter implements RecordWriter {
     </#list>
   </#list>
 </#list>
+
+  @Override
+  public void postProcessing() throws IOException {
+    // no op
+  }
 }
diff --git a/exec/java-exec/src/main/codegen/templates/JsonBaseStatisticsRecordWriter.java b/exec/java-exec/src/main/codegen/templates/JsonBaseStatisticsRecordWriter.java
new file mode 100644
index 0000000..1485127
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/JsonBaseStatisticsRecordWriter.java
@@ -0,0 +1,139 @@
+/*
+ * 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 />
+<@pp.changeOutputFile name="org/apache/drill/exec/store/JSONBaseStatisticsRecordWriter.java" />
+<#include "/@includes/license.ftl" />
+
+package org.apache.drill.exec.store;
+
+import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+import org.apache.drill.exec.vector.complex.fn.JsonOutput;
+
+import java.io.IOException;
+import java.lang.UnsupportedOperationException;
+import java.util.List;
+
+/**
+ * Abstract implementation of StatisticsRecordWriter interface which exposes interface:
+ *    {@link #writeHeader(List)}
+ *    {@link #addField(int,String)}
+ * to output the data in string format instead of implementing addField for each type holder.
+ *
+ * This is useful for text format writers such as CSV, TSV etc.
+ *
+ * NB: Source code generated using FreeMarker template ${.template_name}
+ */
+public abstract class JSONBaseStatisticsRecordWriter implements StatisticsRecordWriter {
+
+  protected JsonOutput gen;
+  protected boolean skipNullFields = true;
+
+  <#list vv.types as type>
+  <#list type.minor as minor>
+  <#list vv.modes as mode>
+  <#assign friendlyType = (minor.friendlyType!minor.boxedType!type.boxedType) />
+  @Override
+  public FieldConverter getNew${mode.prefix}${minor.class}Converter(int fieldId, String fieldName, FieldReader reader) {
+    return new ${mode.prefix}${minor.class}JsonConverter(fieldId, fieldName, reader);
+  }
+
+  public class ${mode.prefix}${minor.class}JsonConverter extends FieldConverter {
+
+    public ${mode.prefix}${minor.class}JsonConverter(int fieldId, String fieldName, FieldReader reader) {
+      super(fieldId, fieldName, reader);
+    }
+
+    @Override
+    public void startField() throws IOException {
+      <#if mode.prefix == "Nullable" >
+      if (!skipNullFields || this.reader.isSet()) {
+        gen.writeFieldName(fieldName);
+      }
+      <#else>
+      gen.writeFieldName(fieldName);
+      </#if>
+    }
+
+    @Override
+    public void writeField() throws IOException {
+
+      <#assign typeName = minor.class >
+
+      <#switch minor.class>
+      <#case "UInt1">
+        <#case "UInt2">
+        <#case "UInt4">
+        <#case "UInt8">
+        <#assign typeName = "unsupported">
+        <#break>
+
+      <#case "Decimal9">
+        <#case "Decimal18">
+        <#case "Decimal28Sparse">
+        <#case "Decimal28Dense">
+        <#case "Decimal38Dense">
+        <#case "Decimal38Sparse">
+        <#case "VarDecimal">
+        <#assign typeName = "Decimal">
+        <#break>
+      <#case "Float4">
+        <#assign typeName = "Float">
+        <#break>
+      <#case "Float8">
+        <#assign typeName = "Double">
+        <#break>
+
+      <#case "IntervalDay">
+        <#case "IntervalYear">
+        <#assign typeName = "Interval">
+        <#break>
+
+      <#case "Bit">
+        <#assign typeName = "Boolean">
+        <#break>
+
+      <#case "TimeStamp">
+        <#assign typeName = "Timestamp">
+        <#break>
+
+      <#case "VarBinary">
+        <#assign typeName = "Binary">
+        <#break>
+
+      </#switch>
+
+      <#if typeName == "unsupported">
+      throw new UnsupportedOperationException("Unable to currently write ${minor.class} type to JSON.");
+      <#else>
+      <#if mode.prefix == "Nullable" >
+      if (!skipNullFields || this.reader.isSet()) {
+        gen.write${typeName}(reader);
+      }
+      <#else>
+      gen.write${typeName}(reader);
+      </#if>
+      </#if>
+    }
+  }
+  </#list>
+  </#list>
+  </#list>
+
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/main/codegen/templates/RecordWriter.java b/exec/java-exec/src/main/codegen/templates/RecordWriter.java
index 470c5f2..e07ab7c 100644
--- a/exec/java-exec/src/main/codegen/templates/RecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/RecordWriter.java
@@ -85,7 +85,11 @@ public interface RecordWriter {
    * @throws IOException
    */
   void endRecord() throws IOException;
-
+  /**
+   * Called after adding all the records to perform any post processing related tasks
+   * @throws IOException
+   */
+  void postProcessing() throws IOException;
   void abort() throws IOException;
   void cleanup() throws IOException;
 }
diff --git a/exec/java-exec/src/main/codegen/templates/RecordWriter.java b/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriter.java
similarity index 67%
copy from exec/java-exec/src/main/codegen/templates/RecordWriter.java
copy to exec/java-exec/src/main/codegen/templates/StatisticsRecordWriter.java
index 470c5f2..85c5bde 100644
--- a/exec/java-exec/src/main/codegen/templates/RecordWriter.java
+++ b/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriter.java
@@ -16,27 +16,24 @@
  * limitations under the License.
  */
 <@pp.dropOutputFile />
-<@pp.changeOutputFile name="org/apache/drill/exec/store/RecordWriter.java" />
+<@pp.changeOutputFile name="org/apache/drill/exec/store/StatisticsRecordWriter.java" />
 <#include "/@includes/license.ftl" />
 
 package org.apache.drill.exec.store;
 
-import org.apache.drill.exec.expr.holders.*;
-import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
 import org.apache.drill.exec.vector.complex.reader.FieldReader;
 
 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 {
+/** StatisticsRecordWriter interface. */
+public interface StatisticsRecordWriter {
 
   /**
    * Initialize the writer.
@@ -59,33 +56,38 @@ public interface RecordWriter {
   public void checkForNewPartition(int index);
 
   /**
+   * Returns if the writer is a blocking writer i.e. consumes all input before writing it out
+   * @return TRUE, if writer is blocking. FALSE, otherwise
+   */
+  boolean isBlockingWriter();
+
+  /**
    * Called before starting writing fields in a record.
    * @throws IOException
    */
-  void startRecord() throws IOException;
+  void startStatisticsRecord() throws IOException;
 
-  /** Add the field value given in <code>valueHolder</code> at the given column number <code>fieldId</code>. */
-  public FieldConverter getNewMapConverter(int fieldId, String fieldName, FieldReader reader);
-  public FieldConverter getNewUnionConverter(int fieldId, String fieldName, FieldReader reader);
-  public FieldConverter getNewRepeatedMapConverter(int fieldId, String fieldName, FieldReader reader);
-  public FieldConverter getNewRepeatedListConverter(int fieldId, String fieldName, FieldReader reader);
-
-<#list vv.types as type>
+  <#list vv.types as type>
   <#list type.minor as minor>
-    <#list vv.modes as mode>
+  <#list vv.modes as mode>
   /** Add the field value given in <code>valueHolder</code> at the given column number <code>fieldId</code>. */
   public FieldConverter getNew${mode.prefix}${minor.class}Converter(int fieldId, String fieldName, FieldReader reader);
 
-    </#list>
   </#list>
-</#list>
+  </#list>
+  </#list>
 
   /**
-   * Called after adding all fields in a particular record are added using add{TypeHolder}(fieldId, TypeHolder) methods.
+   * Called after adding all fields in a particular statistics record are added using
+   * add{TypeHolder}(fieldId, TypeHolder) methods.
    * @throws IOException
    */
-  void endRecord() throws IOException;
-
+  void endStatisticsRecord() throws IOException;
+  /**
+   * For a blocking writer, called after processing all the records to flush out the writes
+   * @throws IOException
+   */
+  void flushBlockingWriter() throws IOException;
   void abort() throws IOException;
   void cleanup() throws IOException;
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriterImpl.java b/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriterImpl.java
new file mode 100644
index 0000000..dd3ab7a
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/StatisticsRecordWriterImpl.java
@@ -0,0 +1,119 @@
+/*
+ * 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.
+ */
+
+import org.apache.drill.exec.planner.physical.WriterPrel;
+
+<@pp.dropOutputFile />
+<@pp.changeOutputFile name="org/apache/drill/exec/store/StatisticsRecordWriterImpl.java" />
+<#include "/@includes/license.ftl" />
+
+package org.apache.drill.exec.store;
+
+import com.google.common.collect.Lists;
+
+import org.apache.drill.exec.planner.physical.WriterPrel;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.store.EventBasedRecordWriter.FieldConverter;
+import org.apache.drill.exec.vector.complex.reader.FieldReader;
+
+import java.io.IOException;
+import java.util.List;
+
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
+/** Reads records from the RecordValueAccessor and writes into StatisticsRecordWriter. */
+public class StatisticsRecordWriterImpl {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsRecordWriterImpl.class);
+
+  private VectorAccessible batch;
+  private StatisticsRecordWriter recordWriter;
+  private List<FieldConverter> fieldConverters;
+
+  public StatisticsRecordWriterImpl(VectorAccessible batch, StatisticsRecordWriter recordWriter)
+      throws IOException {
+    this.batch = batch;
+    this.recordWriter = recordWriter;
+    initFieldWriters();
+  }
+
+  public int writeStatistics(int recordCount) throws IOException {
+    int counter = 0;
+
+    for (; counter < recordCount; counter++) {
+      recordWriter.checkForNewPartition(counter);
+      recordWriter.startStatisticsRecord();
+      // write the current record
+      for (FieldConverter converter : fieldConverters) {
+        converter.setPosition(counter);
+        converter.startField();
+        converter.writeField();
+        converter.endField();
+      }
+      recordWriter.endStatisticsRecord();
+    }
+
+    return counter;
+  }
+
+  public void flushBlockingWriter() throws IOException {
+    if (recordWriter.isBlockingWriter()) {
+      recordWriter.flushBlockingWriter();
+    }
+  }
+
+  private void initFieldWriters() throws IOException {
+    fieldConverters = Lists.newArrayList();
+    try {
+      int fieldId = 0;
+      for (VectorWrapper w : batch) {
+        if (w.getField().getName().equalsIgnoreCase(WriterPrel.PARTITION_COMPARATOR_FIELD)) {
+          continue;
+        }
+        FieldReader reader = w.getValueVector().getReader();
+        FieldConverter converter = getConverter(recordWriter, fieldId++, w.getField().getName(), reader);
+        fieldConverters.add(converter);
+      }
+    } catch(Exception e) {
+      logger.error("Failed to create FieldWriter.", e);
+      throw new IOException("Failed to initialize FieldWriters.", e);
+    }
+  }
+
+  public static FieldConverter getConverter(StatisticsRecordWriter recordWriter, int fieldId, String fieldName,
+      FieldReader reader) {
+    switch (reader.getType().getMinorType()) {
+      <#list vv.types as type>
+      <#list type.minor as minor>
+      case ${minor.class?upper_case}:
+      switch (reader.getType().getMode()) {
+        case REQUIRED:
+          return recordWriter.getNew${minor.class}Converter(fieldId, fieldName, reader);
+        case OPTIONAL:
+          return recordWriter.getNewNullable${minor.class}Converter(fieldId, fieldName, reader);
+        case REPEATED:
+          return recordWriter.getNewRepeated${minor.class}Converter(fieldId, fieldName, reader);
+      }
+      </#list>
+      </#list>
+    }
+    throw new UnsupportedOperationException();
+  }
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index bab94a1..ac3252a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -22,11 +22,13 @@ import org.apache.drill.exec.physical.impl.common.HashTable;
 import org.apache.drill.exec.rpc.user.InboundImpersonationManager;
 import org.apache.drill.exec.server.options.OptionValidator;
 import org.apache.drill.exec.server.options.OptionValidator.OptionDescription;
-import org.apache.drill.exec.server.options.TypeValidators.DateTimeFormatValidator;
-import org.apache.drill.exec.server.options.TypeValidators.IntegerValidator;
+import org.apache.drill.exec.server.options.TypeValidators.AdminUserGroupsValidator;
+import org.apache.drill.exec.server.options.TypeValidators.AdminUsersValidator;
 import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
+import org.apache.drill.exec.server.options.TypeValidators.DateTimeFormatValidator;
 import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
 import org.apache.drill.exec.server.options.TypeValidators.EnumeratedStringValidator;
+import org.apache.drill.exec.server.options.TypeValidators.IntegerValidator;
 import org.apache.drill.exec.server.options.TypeValidators.LongValidator;
 import org.apache.drill.exec.server.options.TypeValidators.MaxWidthValidator;
 import org.apache.drill.exec.server.options.TypeValidators.PositiveLongValidator;
@@ -34,8 +36,6 @@ import org.apache.drill.exec.server.options.TypeValidators.PowerOfTwoLongValidat
 import org.apache.drill.exec.server.options.TypeValidators.RangeDoubleValidator;
 import org.apache.drill.exec.server.options.TypeValidators.RangeLongValidator;
 import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
-import org.apache.drill.exec.server.options.TypeValidators.AdminUsersValidator;
-import org.apache.drill.exec.server.options.TypeValidators.AdminUserGroupsValidator;
 import org.apache.drill.exec.testing.ExecutionControls;
 import org.apache.drill.exec.vector.ValueVector;
 
@@ -911,4 +911,38 @@ public final class ExecConstants {
       new OptionDescription("Controls whether to return result set for CREATE TABLE / VIEW / FUNCTION, DROP TABLE / VIEW / FUNCTION, " +
           "SET, USE, REFRESH METADATA TABLE queries. If set to false affected rows count will be returned instead and result set will be null. " +
           "Affects JDBC connections only. Default is true. (Drill 1.15+)"));
+
+  /**
+   * Option whose value is a long value representing the number of bits required for computing ndv (using HLL).
+   * Controls the trade-off between accuracy and memory requirements. The number of bits correlates positively with accuracy
+   */
+  public static final String HLL_ACCURACY = "exec.statistics.ndv_accuracy";
+  public static final LongValidator HLL_ACCURACY_VALIDATOR = new PositiveLongValidator(HLL_ACCURACY, 30,
+      new OptionDescription("Controls trade-off between NDV statistic computation memory cost and accuracy"));
+
+  /**
+   * Option whose value is a boolean value representing whether to perform deterministic sampling. It translates to using
+   * the same (pre-defined) seed for the underlying pseudo-random number generator.
+   */
+  public static final String DETERMINISTIC_SAMPLING = "exec.statistics.deterministic_sampling";
+  public static final BooleanValidator DETERMINISTIC_SAMPLING_VALIDATOR = new BooleanValidator(DETERMINISTIC_SAMPLING,
+      new OptionDescription("Deterministic sampling"));
+
+  /**
+   * Option whose value is a long value representing the expected number of elements in the bloom filter. The bloom filter
+   * computes the number of duplicates which is used for extrapolating the NDV when using sampling. Controls the trade-off
+   * between accuracy and memory requirements. The number of elements correlates positively with accuracy.
+   */
+  public static final String NDV_BLOOM_FILTER_ELEMENTS = "exec.statistics.ndv_extrapolation_bf_elements";
+  public static final LongValidator NDV_BLOOM_FILTER_ELEMENTS_VALIDATOR = new PositiveLongValidator(NDV_BLOOM_FILTER_ELEMENTS, Integer.MAX_VALUE,
+          new OptionDescription("Controls trade-off between NDV statistic computation memory cost and sampling extrapolation accuracy"));
+
+  /**
+   * Option whose value is a double value representing the desired max false positive probability in the bloom filter. The bloom filter
+   * computes the number of duplicates which is used for extrapolating the NDV when using sampling. Controls the trade-off
+   * between accuracy and memory requirements. The probability correlates negatively with the accuracy.
+   */
+  public static final String NDV_BLOOM_FILTER_FPOS_PROB = "exec.statistics.ndv_extrapolation_bf_fpprobability";
+  public static final LongValidator NDV_BLOOM_FILTER_FPOS_PROB_VALIDATOR = new PositiveLongValidator(NDV_BLOOM_FILTER_FPOS_PROB,
+          100, new OptionDescription("Controls trade-off between NDV statistic computation memory cost and sampling extrapolation accuracy"));
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillType.java b/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillType.java
index 589e982..f4d0601 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillType.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/DotDrillType.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.fs.Path;
 
 public enum DotDrillType {
   VIEW,
-  // ,FORMAT
   STATS;
 
   private final String ending;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
index 8470e1f..dd5b359 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/MathFunctions.java
@@ -26,6 +26,7 @@ import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
 import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
 import org.apache.drill.exec.expr.holders.Float8Holder;
 import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
 import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
@@ -61,7 +62,34 @@ public class MathFunctions{
     public void eval(){
       out.value = java.lang.Math.random();
     }
+  }
+
+  @FunctionTemplate(name = "rand", isRandom = true,
+          scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class RandomWithoutSeed implements DrillSimpleFunc{
+    @Output  Float8Holder out;
+
+    public void setup(){}
+
+    public void eval(){
+      out.value = java.lang.Math.random();
+    }
+  }
+
+  @FunctionTemplate(name = "rand", isRandom = true,
+          scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class RandomWithSeed implements DrillSimpleFunc{
+    @Param BigIntHolder seed;
+    @Workspace java.util.Random rand;
+    @Output  Float8Holder out;
 
+    public void setup(){
+      rand = new java.util.Random(seed.value);
+    }
+
+    public void eval(){
+      out.value = rand.nextDouble();
+    }
   }
 
   @FunctionTemplate(name = "to_number", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StatisticsAggrFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StatisticsAggrFunctions.java
index c6430dd..79765ed 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StatisticsAggrFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StatisticsAggrFunctions.java
@@ -1,4 +1,4 @@
-/*******************************************************************************
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -6,15 +6,15 @@
  * 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
- * <p/>
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
+ *
  * 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.
- ******************************************************************************/
+ */
 
 /*
  * This class is automatically generated from AggrTypeFunctions2.tdd using FreeMarker.
@@ -23,28 +23,65 @@
 package org.apache.drill.exec.expr.fn.impl;
 
 import io.netty.buffer.DrillBuf;
+import javax.inject.Inject;
 import org.apache.drill.exec.expr.DrillAggFunc;
 import org.apache.drill.exec.expr.DrillSimpleFunc;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
 import org.apache.drill.exec.expr.annotations.Output;
 import org.apache.drill.exec.expr.annotations.Param;
 import org.apache.drill.exec.expr.annotations.Workspace;
 import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.DateHolder;
+import org.apache.drill.exec.expr.holders.Decimal18Holder;
+import org.apache.drill.exec.expr.holders.Decimal28DenseHolder;
+import org.apache.drill.exec.expr.holders.Decimal28SparseHolder;
+import org.apache.drill.exec.expr.holders.Decimal38DenseHolder;
+import org.apache.drill.exec.expr.holders.Decimal38SparseHolder;
+import org.apache.drill.exec.expr.holders.Decimal9Holder;
+import org.apache.drill.exec.expr.holders.Float4Holder;
+import org.apache.drill.exec.expr.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.IntHolder;
+import org.apache.drill.exec.expr.holders.IntervalHolder;
 import org.apache.drill.exec.expr.holders.NullableBigIntHolder;
+import org.apache.drill.exec.expr.holders.NullableBitHolder;
+import org.apache.drill.exec.expr.holders.NullableDateHolder;
+import org.apache.drill.exec.expr.holders.NullableDecimal18Holder;
+import org.apache.drill.exec.expr.holders.NullableDecimal28DenseHolder;
+import org.apache.drill.exec.expr.holders.NullableDecimal28SparseHolder;
+import org.apache.drill.exec.expr.holders.NullableDecimal38DenseHolder;
+import org.apache.drill.exec.expr.holders.NullableDecimal38SparseHolder;
+import org.apache.drill.exec.expr.holders.NullableDecimal9Holder;
+import org.apache.drill.exec.expr.holders.NullableFloat4Holder;
+import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
+import org.apache.drill.exec.expr.holders.NullableIntHolder;
+import org.apache.drill.exec.expr.holders.NullableIntervalHolder;
+import org.apache.drill.exec.expr.holders.NullableTimeHolder;
+import org.apache.drill.exec.expr.holders.NullableTimeStampHolder;
+import org.apache.drill.exec.expr.holders.NullableVar16CharHolder;
 import org.apache.drill.exec.expr.holders.NullableVarBinaryHolder;
+import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
 import org.apache.drill.exec.expr.holders.ObjectHolder;
+import org.apache.drill.exec.expr.holders.TimeHolder;
+import org.apache.drill.exec.expr.holders.TimeStampHolder;
+import org.apache.drill.exec.expr.holders.Var16CharHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.vector.complex.reader.FieldReader;
 
-import javax.inject.Inject;
-
 @SuppressWarnings("unused")
 public class StatisticsAggrFunctions {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsAggrFunctions.class);
 
-  @FunctionTemplate(name = "statcount", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
-  public static class StatCount implements DrillAggFunc {
+  /* IMPORTANT NOTE: Please make sure to create a new function for each datatype. See the examples below.
+  * This will result in more performant generated code. Use switch-case/if-else statements judiciously
+  * as it MAY cause the generated code to slow down considerably.
+  * */
+  @FunctionTemplate(name = "rowcount", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class RowCount implements DrillAggFunc {
     @Param FieldReader in;
     @Workspace BigIntHolder count;
     @Output NullableBigIntHolder out;
@@ -71,8 +108,8 @@ public class StatisticsAggrFunctions {
     }
   }
 
-  @FunctionTemplate(name = "nonnullstatcount", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
-  public static class NonNullStatCount implements DrillAggFunc {
+  @FunctionTemplate(name = "nonnullrowcount", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NonNullRowCount implements DrillAggFunc {
     @Param FieldReader in;
     @Workspace BigIntHolder count;
     @Output NullableBigIntHolder out;
@@ -101,17 +138,27 @@ public class StatisticsAggrFunctions {
     }
   }
 
+  /**
+   * The log2m parameter defines the accuracy of the counter.  The larger the
+   * log2m the better the accuracy.
+   * accuracy = 1.04/sqrt(2^log2m)
+   * where
+   * log2m - the number of bits to use as the basis for the HLL instance
+   */
   @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
   public static class HllFieldReader implements DrillAggFunc {
     @Param FieldReader in;
     @Workspace ObjectHolder work;
     @Output NullableVarBinaryHolder out;
+    @Inject OptionManager options;
     @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
 
     @Override
     public void setup() {
       work = new ObjectHolder();
-      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(10);
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
     }
 
     @Override
@@ -134,6 +181,9 @@ public class StatisticsAggrFunctions {
               case org.apache.drill.common.types.TypeProtos.MinorType.VARCHAR_VALUE:
                 hll.offer(in.readText().toString());
                 break;
+              case org.apache.drill.common.types.TypeProtos.MinorType.BIGINT_VALUE:
+                hll.offer(in.readLong());
+                break;
               default:
                 work.obj = null;
             }
@@ -167,71 +217,197 @@ public class StatisticsAggrFunctions {
 
     @Override
     public void reset() {
-      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(10);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  /**
+   * The log2m parameter defines the accuracy of the counter.  The larger the log2m the better the accuracy where:
+   * accuracy = 1.04/sqrt(2^log2m)
+   * log2m - the number of bits to use as the basis for the HLL instance
+   * The parameter accepts integers in the range [0, 30]
+   */
+  @FunctionTemplate(name = "hll_decode", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class HllDecode implements DrillSimpleFunc {
+
+    @Param
+    NullableVarBinaryHolder in;
+    @Output
+    BigIntHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    public void eval() {
+      out.value = -1;
+
+      if (in.isSet != 0) {
+        byte[] din = new byte[in.end - in.start];
+        in.buffer.getBytes(in.start, din);
+        try {
+          out.value = com.clearspring.analytics.stream.cardinality.HyperLogLog.Builder.build(din).cardinality();
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failure evaluating hll_decode", e);
+        }
+      }
     }
   }
 
+  /**
+   * The log2m parameter defines the accuracy of the counter.  The larger the log2m the better the accuracy where:
+   * accuracy = 1.04/sqrt(2^log2m)
+   * log2m - the number of bits to use as the basis for the HLL instance
+   * The parameter accepts integers in the range [0, 30]
+   */
+  @FunctionTemplate(name = "hll_merge", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class HllMerge implements DrillAggFunc {
+    @Param NullableVarBinaryHolder in;
+    @Workspace ObjectHolder work;
+    @Output NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        try {
+          if (in.isSet != 0) {
+            byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+                in.start, in.end, in.buffer).getBytes();
+            com.clearspring.analytics.stream.cardinality.HyperLogLog other =
+                com.clearspring.analytics.stream.cardinality.HyperLogLog.Builder.build(buf);
+            hll.addAll(other);
+          }
+        } catch (Exception e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to merge HyperLogLog output", e);
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
 
-  @FunctionTemplate(name = "ndv", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
-  public static class NdvVarBinary implements DrillAggFunc {
+  /**
+   * The log2m parameter defines the accuracy of the counter.  The larger the log2m the better the accuracy where:
+   * accuracy = 1.04/sqrt(2^log2m)
+   * log2m - the number of bits to use as the basis for the HLL instance
+   * The parameter accepts integers in the range [0, 30]
+   */
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class BitHLLFunction implements DrillAggFunc {
     @Param
-    FieldReader in;
+    BitHolder in;
     @Workspace
     ObjectHolder work;
     @Output
-    NullableBigIntHolder out;
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
 
     @Override
     public void setup() {
       work = new ObjectHolder();
-      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(10);
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
     }
 
     @Override
     public void add() {
       if (work.obj != null) {
         com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
-            (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
-        int mode = in.getType().getMode().getNumber();
-        int type = in.getType().getMinorType().getNumber();
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        hll.offer(in.value);
+      }
+    }
 
-        switch (mode) {
-          case org.apache.drill.common.types.TypeProtos.DataMode.OPTIONAL_VALUE:
-            if (!in.isSet()) {
-              hll.offer(null);
-              break;
-            }
-            // fall through //
-          case org.apache.drill.common.types.TypeProtos.DataMode.REQUIRED_VALUE:
-            switch (type) {
-              case org.apache.drill.common.types.TypeProtos.MinorType.VARCHAR_VALUE:
-                hll.offer(in.readText().toString());
-                break;
-              case org.apache.drill.common.types.TypeProtos.MinorType.FLOAT8_VALUE:
-                hll.offer(in.readDouble());
-                break;
-              case org.apache.drill.common.types.TypeProtos.MinorType.INT_VALUE:
-                hll.offer(in.readInteger());
-                break;
-              case org.apache.drill.common.types.TypeProtos.MinorType.BIGINT_VALUE:
-                hll.offer(in.readLong());
-                break;
-              case org.apache.drill.common.types.TypeProtos.MinorType.DATE_VALUE:
-              case org.apache.drill.common.types.TypeProtos.MinorType.TIMESTAMP_VALUE:
-              case org.apache.drill.common.types.TypeProtos.MinorType.TIME_VALUE:
-              case org.apache.drill.common.types.TypeProtos.MinorType.TIMETZ_VALUE:
-                hll.offer(in.readLocalDateTime());
-                break;
-              case org.apache.drill.common.types.TypeProtos.MinorType.VARBINARY_VALUE:
-                hll.offer(in.readByteArray());
-                break;
-              default:
-                work.obj = null;
-            }
-            break;
-          default:
-            work.obj = null;
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
         }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableBitHLLFunction implements DrillAggFunc {
+    @Param
+    NullableBitHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        hll.offer(in.value);
       }
     }
 
@@ -239,10 +415,18 @@ public class StatisticsAggrFunctions {
     public void output() {
       if (work.obj != null) {
         com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
-            (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
 
-        out.isSet = 1;
-        out.value = hll.cardinality();
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
       } else {
         out.isSet = 0;
       }
@@ -250,36 +434,5307 @@ public class StatisticsAggrFunctions {
 
     @Override
     public void reset() {
-      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(10);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
     }
   }
 
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class IntHLLFunction implements DrillAggFunc {
+    @Param
+    IntHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
 
-  @FunctionTemplate(name = "hll_decode", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-  public static class HllDecode implements DrillSimpleFunc {
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        hll.offer(in.value);
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
 
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableIntHLLFunction implements DrillAggFunc {
     @Param
-    NullableVarBinaryHolder in;
+    NullableIntHolder in;
+    @Workspace
+    ObjectHolder work;
     @Output
-    BigIntHolder out;
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
 
     @Override
     public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
     }
 
-    public void eval() {
-      out.value = -1;
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        if (in.isSet == 1) {
+          hll.offer(in.value);
+        } else {
+          hll.offer(null);
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
 
-      if (in.isSet != 0) {
-        byte[] din = new byte[in.end - in.start];
-        in.buffer.getBytes(in.start, din);
         try {
-          out.value = com.clearspring.analytics.stream.cardinality.HyperLogLog.Builder.build(din).cardinality();
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
         } catch (java.io.IOException e) {
-          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failure evaluation hll_decode", e);
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
         }
+      } else {
+        out.isSet = 0;
       }
     }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
   }
 
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class BigIntHLLFunction implements DrillAggFunc {
+    @Param
+    BigIntHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        hll.offer(in.value);
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableBigIntHLLFunction implements DrillAggFunc {
+    @Param
+    NullableBigIntHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        if (in.isSet == 1) {
+          hll.offer(in.value);
+        } else {
+          hll.offer(null);
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Float4HLLFunction implements DrillAggFunc {
+    @Param
+    Float4Holder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        hll.offer(in.value);
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableFloat4HLLFunction implements DrillAggFunc {
+    @Param
+    NullableFloat4Holder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        if (in.isSet == 1) {
+          hll.offer(in.value);
+        } else {
+          hll.offer(null);
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Float8HLLFunction implements DrillAggFunc {
+    @Param
+    Float8Holder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        hll.offer(in.value);
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableFloat8HLLFunction implements DrillAggFunc {
+    @Param
+    NullableFloat8Holder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        if (in.isSet == 1) {
+          hll.offer(in.value);
+        } else {
+          hll.offer(null);
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Decimal9HLLFunction implements DrillAggFunc {
+    @Param
+    Decimal9Holder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        hll.offer(in.value);
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDecimal9HLLFunction implements DrillAggFunc {
+    @Param
+    NullableDecimal9Holder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        if (in.isSet == 1) {
+          hll.offer(in.value);
+        } else {
+          hll.offer(null);
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Decimal18HLLFunction implements DrillAggFunc {
+    @Param
+    Decimal18Holder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        hll.offer(in.value);
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDecimal18HLLFunction implements DrillAggFunc {
+    @Param
+    NullableDecimal18Holder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        if (in.isSet == 1) {
+          hll.offer(in.value);
+        } else {
+          hll.offer(null);
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class DateHLLFunction implements DrillAggFunc {
+    @Param
+    DateHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        hll.offer(in.value);
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDateHLLFunction implements DrillAggFunc {
+    @Param
+    NullableDateHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        if (in.isSet == 1) {
+          hll.offer(in.value);
+        } else {
+          hll.offer(null);
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class TimeHLLFunction implements DrillAggFunc {
+    @Param
+    TimeHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        hll.offer(in.value);
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableTimeHLLFunction implements DrillAggFunc {
+    @Param
+    NullableTimeHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        if (in.isSet == 1) {
+          hll.offer(in.value);
+        } else {
+          hll.offer(null);
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class TimeStampHLLFunction implements DrillAggFunc {
+    @Param
+    TimeStampHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        hll.offer(in.value);
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableTimeStampHLLFunction implements DrillAggFunc {
+    @Param
+    NullableTimeStampHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        if (in.isSet == 1) {
+          hll.offer(in.value);
+        } else {
+          hll.offer(null);
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class IntervalHLLFunction implements DrillAggFunc {
+    @Param
+    IntervalHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace
+    ObjectHolder interval;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      interval = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+      interval.obj = new java.util.ArrayList<Integer>(3);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null
+          && interval.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+            (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        java.util.ArrayList<Integer> intervalList = (java.util.ArrayList<Integer>)interval.obj;
+        intervalList.clear();
+        intervalList.add(in.days);
+        intervalList.add(in.months);
+        intervalList.add(in.milliseconds);
+        hll.offer(interval.obj);
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+      interval.obj = new java.util.ArrayList<Integer>(3);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableIntervalHLLFunction implements DrillAggFunc {
+    @Param
+    NullableIntervalHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace
+    ObjectHolder interval;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      interval = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+      interval.obj = new java.util.ArrayList<Integer>(3);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        if (in.isSet == 1) {
+          if (interval.obj != null) {
+            java.util.ArrayList<Integer> intervalList = (java.util.ArrayList<Integer>)interval.obj;
+            intervalList.clear();
+            intervalList.add(in.days);
+            intervalList.add(in.months);
+            intervalList.add(in.milliseconds);
+            hll.offer(interval.obj);
+          }
+        } else {
+          hll.offer(null);
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+      interval.obj = new java.util.ArrayList<Integer>(3);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class VarCharHLLFunction implements DrillAggFunc {
+    @Param
+    VarCharHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+                in.start, in.end, in.buffer).getBytes();
+        hll.offer(buf);
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableVarCharHLLFunction implements DrillAggFunc {
+    @Param
+    NullableVarCharHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        if (in.isSet == 1) {
+          byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+                  in.start, in.end, in.buffer).getBytes();
+          hll.offer(buf);
+        } else {
+          hll.offer(null);
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Var16CharHLLFunction implements DrillAggFunc {
+    @Param
+    Var16CharHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF16
+                (in.start, in.end, in.buffer).getBytes();
+        hll.offer(buf);
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableVar16CharHLLFunction implements DrillAggFunc {
+    @Param
+    NullableVar16CharHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        if (in.isSet == 1) {
+          byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF16
+                  (in.start, in.end, in.buffer).getBytes();
+          hll.offer(buf);
+        } else {
+          hll.offer(null);
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class VarBinaryHLLFunction implements DrillAggFunc {
+    @Param
+    VarBinaryHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8
+                (in.start, in.end, in.buffer).getBytes();
+        hll.offer(buf);
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "hll", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableVarBinaryHLLFunction implements DrillAggFunc {
+    @Param
+    NullableVarBinaryHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Output
+    NullableVarBinaryHolder out;
+    @Inject OptionManager options;
+    @Inject DrillBuf buffer;
+    @Workspace IntHolder hllAccuracy;
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      hllAccuracy.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.HLL_ACCURACY);
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+        if (in.isSet == 1) {
+          byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8
+                  (in.start, in.end, in.buffer).getBytes();
+          hll.offer(buf);
+        } else {
+          hll.offer(null);
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.cardinality.HyperLogLog hll =
+                (com.clearspring.analytics.stream.cardinality.HyperLogLog) work.obj;
+
+        try {
+          byte[] ba = hll.getBytes();
+          out.buffer = buffer.reallocIfNeeded(ba.length);
+          out.start = 0;
+          out.end = ba.length;
+          out.buffer.setBytes(0, ba);
+          out.isSet = 1;
+        } catch (java.io.IOException e) {
+          throw new org.apache.drill.common.exceptions.DrillRuntimeException("Failed to get HyperLogLog output", e);
+        }
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      work.obj = new com.clearspring.analytics.stream.cardinality.HyperLogLog(hllAccuracy.value);
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class BitAvgWidthFunction implements DrillAggFunc {
+    @Param BitHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 1;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableBitAvgWidthFunction implements DrillAggFunc {
+    @Param NullableBitHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 8;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class IntAvgWidthFunction implements DrillAggFunc {
+    @Param IntHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Integer.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableIntAvgWidthFunction implements DrillAggFunc {
+    @Param NullableIntHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Integer.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class BigIntAvgWidthFunction implements DrillAggFunc {
+    @Param BigIntHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Long.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableBigIntAvgWidthFunction implements DrillAggFunc {
+    @Param NullableBigIntHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Long.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Decimal9AvgWidthFunction implements DrillAggFunc {
+    @Param Decimal9Holder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Integer.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDecimal9AvgWidthFunction implements DrillAggFunc {
+    @Param NullableDecimal9Holder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Integer.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Decimal18AvgWidthFunction implements DrillAggFunc {
+    @Param Decimal18Holder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Long.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDecimal18AvgWidthFunction implements DrillAggFunc {
+    @Param NullableDecimal18Holder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Long.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Decimal28DenseAvgWidthFunction implements DrillAggFunc {
+    @Param Decimal28DenseHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 12;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDecimal28DenseAvgWidthFunction implements DrillAggFunc {
+    @Param NullableDecimal28DenseHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 12;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Decimal28SparseAvgWidthFunction implements DrillAggFunc {
+    @Param Decimal28SparseHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 12;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDecimal28SparseAvgWidthFunction implements DrillAggFunc {
+    @Param NullableDecimal28SparseHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 12;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Decimal38DenseAvgWidthFunction implements DrillAggFunc {
+    @Param Decimal38DenseHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 16;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDecimal38DenseAvgWidthFunction implements DrillAggFunc {
+    @Param NullableDecimal38DenseHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 16;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Decimal38SparseAvgWidthFunction implements DrillAggFunc {
+    @Param Decimal38SparseHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 16;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDecimal38SparseAvgWidthFunction implements DrillAggFunc {
+    @Param NullableDecimal38SparseHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 16;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Float4AvgWidthFunction implements DrillAggFunc {
+    @Param Float4Holder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Float.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableFloat4AvgWidthFunction implements DrillAggFunc {
+    @Param NullableFloat4Holder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Float.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Float8AvgWidthFunction implements DrillAggFunc {
+    @Param Float8Holder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Double.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableFloat8AvgWidthFunction implements DrillAggFunc {
+    @Param NullableFloat8Holder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Double.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class DateAvgWidthFunction implements DrillAggFunc {
+    @Param DateHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Integer.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDateAvgWidthFunction implements DrillAggFunc {
+    @Param NullableDateHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Integer.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class TimeAvgWidthFunction implements DrillAggFunc {
+    @Param TimeHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Integer.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableTimeAvgWidthFunction implements DrillAggFunc {
+    @Param NullableTimeHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Integer.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class TimeStampAvgWidthFunction implements DrillAggFunc {
+    @Param TimeStampHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Long.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableTimeStampAvgWidthFunction implements DrillAggFunc {
+    @Param NullableTimeStampHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Long.SIZE;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value * 8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class IntervalAvgWidthFunction implements DrillAggFunc {
+    @Param IntervalHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 12;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableIntervalAvgWidthFunction implements DrillAggFunc {
+    @Param NullableIntervalHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 12;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class VarCharAvgWidthFunction implements DrillAggFunc {
+    @Param VarCharHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+              in.start, in.end, in.buffer).getBytes().length;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableVarCharAvgWidthFunction implements DrillAggFunc {
+    @Param NullableVarCharHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Workspace BigIntHolder nonNullCount;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+      nonNullCount = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      if (in.isSet == 1) {
+        totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+                in.start, in.end, in.buffer).getBytes().length;
+        nonNullCount.value += 1;
+      }
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      if (nonNullCount.value > 0) {
+        out.value = totWidth.value/((double)count.value);
+      } else {
+        out.value = 0;
+      }
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+      nonNullCount.value = 0;
+    }
+  }
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Var16CharAvgWidthFunction implements DrillAggFunc {
+    @Param Var16CharHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF16(
+              in.start, in.end, in.buffer).getBytes().length;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableVar16CharAvgWidthFunction implements DrillAggFunc {
+    @Param NullableVar16CharHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Workspace BigIntHolder nonNullCount;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+      nonNullCount = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      if (in.isSet == 1) {
+        totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF16(
+                in.start, in.end, in.buffer).getBytes().length;
+        nonNullCount.value += 1;
+      }
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      if (nonNullCount.value > 0) {
+        out.value = totWidth.value/((double)count.value);
+      } else {
+        out.value = 0;
+      }
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+      nonNullCount.value = 0;
+    }
+  }
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class VarBinaryAvgWidthFunction implements DrillAggFunc {
+    @Param VarBinaryHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+              in.start, in.end, in.buffer).getBytes().length;
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/((double)count.value);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "avg_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableVarBinaryAvgWidthFunction implements DrillAggFunc {
+    @Param NullableVarBinaryHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder count;
+    @Workspace BigIntHolder nonNullCount;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      count = new BigIntHolder();
+      nonNullCount = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      if (in.isSet == 1) {
+        totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+                in.start, in.end, in.buffer).getBytes().length;
+        nonNullCount.value += 1;
+      }
+      count.value++;
+    }
+
+    @Override
+    public void output() {
+      if (nonNullCount.value > 0) {
+        out.value = totWidth.value/((double)count.value);
+      } else {
+        out.value = 0;
+      }
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      count.value = 0;
+      nonNullCount.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class BitSumWidthFunction implements DrillAggFunc {
+    @Param BitHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 8;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableBitSumWidthFunction implements DrillAggFunc {
+    @Param NullableBitHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 8;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class IntSumWidthFunction implements DrillAggFunc {
+    @Param IntHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Integer.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableIntSumWidthFunction implements DrillAggFunc {
+    @Param NullableIntHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Integer.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class BigIntSumWidthFunction implements DrillAggFunc {
+    @Param BigIntHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Long.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableBigIntSumWidthFunction implements DrillAggFunc {
+    @Param NullableBigIntHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Long.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Decimal9SumWidthFunction implements DrillAggFunc {
+    @Param Decimal9Holder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Integer.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDecimal9SumWidthFunction implements DrillAggFunc {
+    @Param NullableDecimal9Holder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Integer.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Decimal18SumWidthFunction implements DrillAggFunc {
+    @Param Decimal18Holder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Long.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDecimal18SumWidthFunction implements DrillAggFunc {
+    @Param NullableDecimal18Holder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Long.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Decimal28DenseSumWidthFunction implements DrillAggFunc {
+    @Param Decimal28DenseHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 12;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value;
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDecimal28DenseSumWidthFunction implements DrillAggFunc {
+    @Param NullableDecimal28DenseHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 12;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value;
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Decimal28SparseSumWidthFunction implements DrillAggFunc {
+    @Param Decimal28SparseHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 12;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value;
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDecimal28SparseSumWidthFunction implements DrillAggFunc {
+    @Param NullableDecimal28SparseHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 12;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value;
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Decimal38DenseSumWidthFunction implements DrillAggFunc {
+    @Param Decimal38DenseHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 16;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value;
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDecimal38DenseSumWidthFunction implements DrillAggFunc {
+    @Param NullableDecimal38DenseHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 16;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value;
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Decimal38SparseSumWidthFunction implements DrillAggFunc {
+    @Param Decimal38SparseHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 16;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value;
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDecimal38SparseSumWidthFunction implements DrillAggFunc {
+    @Param NullableDecimal38SparseHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 16;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value;
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Float4SumWidthFunction implements DrillAggFunc {
+    @Param Float4Holder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Float.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableFloat4SumWidthFunction implements DrillAggFunc {
+    @Param NullableFloat4Holder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Float.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Float8SumWidthFunction implements DrillAggFunc {
+    @Param Float8Holder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Double.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableFloat8SumWidthFunction implements DrillAggFunc {
+    @Param NullableFloat8Holder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Double.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class DateSumWidthFunction implements DrillAggFunc {
+    @Param DateHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Integer.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDateSumWidthFunction implements DrillAggFunc {
+    @Param NullableDateHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Integer.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class TimeSumWidthFunction implements DrillAggFunc {
+    @Param TimeHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Integer.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableTimeSumWidthFunction implements DrillAggFunc {
+    @Param NullableTimeHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Integer.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class TimeStampSumWidthFunction implements DrillAggFunc {
+    @Param TimeStampHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Long.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableTimeStampSumWidthFunction implements DrillAggFunc {
+    @Param NullableTimeStampHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += Long.SIZE;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value/(8.0);
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class IntervalSumWidthFunction implements DrillAggFunc {
+    @Param IntervalHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 12;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value;
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableIntervalSumWidthFunction implements DrillAggFunc {
+    @Param NullableIntervalHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += 12;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value;
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class VarCharSumWidthFunction implements DrillAggFunc {
+    @Param VarCharHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+              in.start, in.end, in.buffer).getBytes().length;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value;
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableVarCharSumWidthFunction implements DrillAggFunc {
+    @Param NullableVarCharHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder nonNullCount;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      nonNullCount = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      if (in.isSet == 1) {
+        totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+            in.start, in.end, in.buffer).getBytes().length;
+        nonNullCount.value += 1;
+      }
+    }
+
+    @Override
+    public void output() {
+      if (nonNullCount.value > 0) {
+        out.value = totWidth.value;
+      } else {
+        out.value = 0;
+      }
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      nonNullCount.value = 0;
+    }
+  }
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Var16CharSumWidthFunction implements DrillAggFunc {
+    @Param Var16CharHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF16(
+              in.start, in.end, in.buffer).getBytes().length;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value;
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableVar16CharSumWidthFunction implements DrillAggFunc {
+    @Param NullableVar16CharHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder nonNullCount;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      nonNullCount = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      if (in.isSet == 1) {
+        totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF16(
+                in.start, in.end, in.buffer).getBytes().length;
+        nonNullCount.value += 1;
+      }
+    }
+
+    @Override
+    public void output() {
+      if (nonNullCount.value > 0) {
+        out.value = totWidth.value;
+      } else {
+        out.value = 0;
+      }
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      nonNullCount.value = 0;
+    }
+  }
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class VarBinarySumWidthFunction implements DrillAggFunc {
+    @Param VarBinaryHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+              in.start, in.end, in.buffer).getBytes().length;
+    }
+
+    @Override
+    public void output() {
+      out.value = totWidth.value;
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "sum_width", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableVarBinarySumWidthFunction implements DrillAggFunc {
+    @Param NullableVarBinaryHolder in;
+    @Workspace BigIntHolder totWidth;
+    @Workspace BigIntHolder nonNullCount;
+    @Output NullableFloat8Holder out;
+
+    @Override
+    public void setup() {
+      totWidth = new BigIntHolder();
+      nonNullCount = new BigIntHolder();
+    }
+
+    @Override
+    public void add() {
+      if (in.isSet == 1) {
+        totWidth.value += org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+                in.start, in.end, in.buffer).getBytes().length;
+        nonNullCount.value += 1;
+      }
+    }
+
+    @Override
+    public void output() {
+      if (nonNullCount.value > 0) {
+        out.value = totWidth.value;
+      } else {
+        out.value = 0;
+      }
+      out.isSet = 1;
+    }
+
+    @Override
+    public void reset() {
+      totWidth.value = 0;
+      nonNullCount.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class BitCntDupsFunction implements DrillAggFunc {
+    @Param
+    BitHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (!filter.isPresent(String.valueOf(in.value))) {
+          filter.add(String.valueOf(in.value));
+        } else {
+          dups.value++;
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableBitCntDupsFunction implements DrillAggFunc {
+    @Param
+    NullableBitHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (in.isSet == 1) {
+          if (!filter.isPresent(String.valueOf(in.value))) {
+            filter.add(String.valueOf(in.value));
+          } else {
+            dups.value++;
+          }
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class IntCntDupsFunction implements DrillAggFunc {
+    @Param
+    IntHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (!filter.isPresent(String.valueOf(in.value))) {
+          filter.add(String.valueOf(in.value));
+        } else {
+          dups.value++;
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableIntCntDupsFunction implements DrillAggFunc {
+    @Param
+    NullableIntHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (in.isSet == 1) {
+          if (!filter.isPresent(String.valueOf(in.value))) {
+            filter.add(String.valueOf(in.value));
+          } else {
+            dups.value++;
+          }
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class BigIntCntDupsFunction implements DrillAggFunc {
+    @Param
+    BigIntHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (!filter.isPresent(String.valueOf(in.value))) {
+          filter.add(String.valueOf(in.value));
+        } else {
+          dups.value++;
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableBigIntCntDupsFunction implements DrillAggFunc {
+    @Param
+    NullableBigIntHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (in.isSet == 1) {
+          if (!filter.isPresent(String.valueOf(in.value))) {
+            filter.add(String.valueOf(in.value));
+          } else {
+            dups.value++;
+          }
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Float4CntDupsFunction implements DrillAggFunc {
+    @Param
+    Float4Holder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (!filter.isPresent(String.valueOf(in.value))) {
+          filter.add(String.valueOf(in.value));
+        } else {
+          dups.value++;
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableFloat4CntDupsFunction implements DrillAggFunc {
+    @Param
+    NullableFloat4Holder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (in.isSet == 1) {
+          if (!filter.isPresent(String.valueOf(in.value))) {
+            filter.add(String.valueOf(in.value));
+          } else {
+            dups.value++;
+          }
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Float8CntDupsFunction implements DrillAggFunc {
+    @Param
+    Float8Holder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (!filter.isPresent(String.valueOf(in.value))) {
+          filter.add(String.valueOf(in.value));
+        } else {
+          dups.value++;
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableFloat8CntDupsFunction implements DrillAggFunc {
+    @Param
+    NullableFloat8Holder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (in.isSet == 1) {
+          if (!filter.isPresent(String.valueOf(in.value))) {
+            filter.add(String.valueOf(in.value));
+          } else {
+            dups.value++;
+          }
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Decimal9CntDupsFunction implements DrillAggFunc {
+    @Param
+    Decimal9Holder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (!filter.isPresent(String.valueOf(in.value))) {
+          filter.add(String.valueOf(in.value));
+        } else {
+          dups.value++;
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDecimal9CntDupsFunction implements DrillAggFunc {
+    @Param
+    NullableDecimal9Holder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (in.isSet == 1) {
+          if (!filter.isPresent(String.valueOf(in.value))) {
+            filter.add(String.valueOf(in.value));
+          } else {
+            dups.value++;
+          }
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Decimal18CntDupsFunction implements DrillAggFunc {
+    @Param
+    Decimal18Holder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (!filter.isPresent(String.valueOf(in.value))) {
+          filter.add(String.valueOf(in.value));
+        } else {
+          dups.value++;
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDecimal18CntDupsFunction implements DrillAggFunc {
+    @Param
+    NullableDecimal18Holder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (in.isSet == 1) {
+          if (!filter.isPresent(String.valueOf(in.value))) {
+            filter.add(String.valueOf(in.value));
+          } else {
+            dups.value++;
+          }
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class DateCntDupsFunction implements DrillAggFunc {
+    @Param
+    DateHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (!filter.isPresent(String.valueOf(in.value))) {
+          filter.add(String.valueOf(in.value));
+        } else {
+          dups.value++;
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableDateCntDupsFunction implements DrillAggFunc {
+    @Param
+    NullableDateHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (in.isSet == 1) {
+          if (!filter.isPresent(String.valueOf(in.value))) {
+            filter.add(String.valueOf(in.value));
+          } else {
+            dups.value++;
+          }
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class TimeCntDupsFunction implements DrillAggFunc {
+    @Param
+    TimeHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (!filter.isPresent(String.valueOf(in.value))) {
+          filter.add(String.valueOf(in.value));
+        } else {
+          dups.value++;
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableTimeCntDupsFunction implements DrillAggFunc {
+    @Param
+    NullableTimeHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (in.isSet == 1) {
+          if (!filter.isPresent(String.valueOf(in.value))) {
+            filter.add(String.valueOf(in.value));
+          } else {
+            dups.value++;
+          }
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class TimeStampCntDupsFunction implements DrillAggFunc {
+    @Param
+    TimeStampHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (!filter.isPresent(String.valueOf(in.value))) {
+          filter.add(String.valueOf(in.value));
+        } else {
+          dups.value++;
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableTimeStampCntDupsFunction implements DrillAggFunc {
+    @Param
+    NullableTimeStampHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (in.isSet == 1) {
+          if (!filter.isPresent(String.valueOf(in.value))) {
+            filter.add(String.valueOf(in.value));
+          } else {
+            dups.value++;
+          }
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class IntervalCntDupsFunction implements DrillAggFunc {
+    @Param
+    IntervalHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Workspace
+    ObjectHolder interval;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      interval = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+      interval.obj = new int[3];
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null
+              && interval.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        ((int[])interval.obj)[0] = in.days;
+        ((int[])interval.obj)[1] = in.months;
+        ((int[])interval.obj)[2] = in.milliseconds;
+        if (!filter.isPresent(String.valueOf(interval.obj))) {
+          filter.add(String.valueOf(interval.obj));
+        } else {
+          dups.value++;
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+      interval.obj = new int[3];
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableIntervalCntDupsFunction implements DrillAggFunc {
+    @Param
+    NullableIntervalHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Workspace
+    ObjectHolder interval;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      interval = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+      interval.obj = new int[3];
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (in.isSet == 1) {
+          if (interval.obj != null) {
+            ((int[]) interval.obj)[0] = in.days;
+            ((int[]) interval.obj)[1] = in.months;
+            ((int[]) interval.obj)[2] = in.milliseconds;
+            if (!filter.isPresent(String.valueOf(interval.obj))) {
+              filter.add(String.valueOf(interval.obj));
+            } else {
+              dups.value++;
+            }
+          }
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+      interval.obj = new int[3];
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class VarCharCntDupsFunction implements DrillAggFunc {
+    @Param
+    VarCharHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+                in.start, in.end, in.buffer).getBytes();
+        if (!filter.isPresent(buf)) {
+          filter.add(buf);
+        } else {
+          dups.value++;
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableVarCharCntDupsFunction implements DrillAggFunc {
+    @Param
+    NullableVarCharHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (in.isSet == 1) {
+          byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(
+                  in.start, in.end, in.buffer).getBytes();
+          if (!filter.isPresent(buf)) {
+            filter.add(buf);
+          } else {
+            dups.value++;
+          }
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class Var16CharCntDupsFunction implements DrillAggFunc {
+    @Param
+    Var16CharHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF16
+                (in.start, in.end, in.buffer).getBytes();
+        if (!filter.isPresent(buf)) {
+          filter.add(buf);
+        } else {
+          dups.value++;
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableVar16CharCntDupsFunction implements DrillAggFunc {
+    @Param
+    NullableVar16CharHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (in.isSet == 1) {
+          byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF16
+                  (in.start, in.end, in.buffer).getBytes();
+          if (!filter.isPresent(buf)) {
+            filter.add(buf);
+          } else {
+            dups.value++;
+          }
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class VarBinaryCntDupsFunction implements DrillAggFunc {
+    @Param
+    VarBinaryHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8
+                (in.start, in.end, in.buffer).getBytes();
+        if (!filter.isPresent(buf)) {
+          filter.add(buf);
+        } else {
+          dups.value++;
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
+
+  @FunctionTemplate(name = "approx_count_dups", scope = FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+  public static class NullableVarBinaryCntDupsFunction implements DrillAggFunc {
+    @Param
+    NullableVarBinaryHolder in;
+    @Workspace
+    ObjectHolder work;
+    @Workspace BigIntHolder dups;
+    @Output
+    NullableBigIntHolder out;
+    @Inject OptionManager options;
+    @Workspace IntHolder ndvBloomFilterElts;
+    @Workspace IntHolder ndvBloomFilterFPProb;
+
+    @Override
+    public void setup() {
+      work = new ObjectHolder();
+      dups.value = 0;
+      ndvBloomFilterElts.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_ELEMENTS);
+      ndvBloomFilterFPProb.value = (int) options.getLong(org.apache.drill.exec.ExecConstants.NDV_BLOOM_FILTER_FPOS_PROB);
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+
+    @Override
+    public void add() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        if (in.isSet == 1) {
+          byte[] buf = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8
+                  (in.start, in.end, in.buffer).getBytes();
+          if (!filter.isPresent(buf)) {
+            filter.add(buf);
+          } else {
+            dups.value++;
+          }
+        }
+      }
+    }
+
+    @Override
+    public void output() {
+      if (work.obj != null) {
+        com.clearspring.analytics.stream.membership.BloomFilter filter =
+                (com.clearspring.analytics.stream.membership.BloomFilter ) work.obj;
+        out.isSet = 1;
+        out.value = dups.value;
+      } else {
+        out.isSet = 0;
+      }
+    }
+
+    @Override
+    public void reset() {
+      dups.value = 0;
+      work.obj = new com.clearspring.analytics.stream.membership.BloomFilter(ndvBloomFilterElts.value, ndvBloomFilterFPProb.value);
+    }
+  }
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BaseFragmentContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BaseFragmentContext.java
index 8005f04..9d30031 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BaseFragmentContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BaseFragmentContext.java
@@ -17,16 +17,15 @@
  */
 package org.apache.drill.exec.ops;
 
+import io.netty.buffer.DrillBuf;
 import java.io.IOException;
 import java.util.List;
-
 import org.apache.drill.exec.exception.ClassTransformationException;
 import org.apache.drill.exec.expr.ClassGenerator;
 import org.apache.drill.exec.expr.CodeGenerator;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.physical.impl.common.CodeGenMemberInjector;
-
-import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.proto.UserBitShared;
 
 /**
  * Common implementation for both the test and production versions
@@ -88,4 +87,23 @@ public abstract class BaseFragmentContext implements FragmentContext {
     return getBufferManager().getManagedBuffer(size);
   }
 
+  @Override
+  public String getQueryUserName() {
+    return null;
+  }
+
+  @Override
+  public UserBitShared.QueryId getQueryId() {
+    return null;
+  }
+
+  @Override
+  public String getQueryIdString() {
+    return null;
+  }
+
+  @Override
+  public QueryContext.SqlStatementType getSQLStatementType() {
+    return null;
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java
index a7e0dc2..1972cb7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java
@@ -72,4 +72,5 @@ public class ContextInformation {
   public String getSessionId() {
     return sessionId;
   }
+
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
index 5125f72..46f94f0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
@@ -30,13 +30,15 @@ import org.apache.drill.exec.expr.ClassGenerator;
 import org.apache.drill.exec.expr.CodeGenerator;
 import org.apache.drill.exec.expr.fn.FunctionLookupContext;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.QueryContext.SqlStatementType;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.proto.ExecProtos;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.testing.ExecutionControls;
+import org.apache.drill.exec.work.filter.RuntimeFilterWritable;
 
 import io.netty.buffer.DrillBuf;
-import org.apache.drill.exec.work.filter.RuntimeFilterWritable;
 
 /**
  * Provides the resources required by a non-exchange operator to execute.
@@ -93,13 +95,14 @@ public interface FragmentContext extends UdfUtilities, AutoCloseable {
       throws ClassTransformationException, IOException;
 
   /**
-   * Generates code for a class given a {@link CodeGenerator}, and returns the
-   * specified number of instances of the generated class. (Note that the name
-   * is a misnomer, it would be better called
-   * <tt>getImplementationInstances</tt>.)
-   *
-   * @param cg the code generator
-   * @return list of instances of the generated class
+   * Returns the statement type (e.g. SELECT, CTAS, ANALYZE) from the query context.
+   * @return query statement type {@link SqlStatementType}, if known.
+   */
+  public SqlStatementType getSQLStatementType();
+
+  /**
+   * Get this node's identity.
+   * @return A DrillbitEndpoint object.
    */
   <T> List<T> getImplementationClass(final CodeGenerator<T> cg, final int instanceCount)
       throws ClassTransformationException, IOException;
@@ -139,6 +142,16 @@ public interface FragmentContext extends UdfUtilities, AutoCloseable {
 
   BufferAllocator getAllocator();
 
+  /**
+   * @return ID {@link java.util.UUID} of the current query
+   */
+  public QueryId getQueryId();
+
+  /**
+   * @return The string representation of the ID {@link java.util.UUID} of the current query
+   */
+  public String getQueryIdString();
+
   OperatorContext newOperatorContext(PhysicalOperator popConfig);
 
   OperatorContext newOperatorContext(PhysicalOperator popConfig, OperatorStats stats);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContextImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContextImpl.java
index b740c92..afbcd1c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContextImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContextImpl.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.ops;
 
+import io.netty.buffer.DrillBuf;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
@@ -40,6 +41,7 @@ import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.expr.holders.ValueHolder;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.QueryContext.SqlStatementType;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.impl.OperatorCreatorRegistry;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
@@ -48,6 +50,7 @@ import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
@@ -65,14 +68,12 @@ import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.drill.exec.testing.ExecutionControls;
 import org.apache.drill.exec.util.ImpersonationUtil;
 import org.apache.drill.exec.work.batch.IncomingBuffers;
+import org.apache.drill.exec.work.filter.RuntimeFilterWritable;
 import org.apache.drill.shaded.guava.com.google.common.base.Function;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
 
-import io.netty.buffer.DrillBuf;
-import org.apache.drill.exec.work.filter.RuntimeFilterWritable;
-
 /**
  * <p>
  *   This is the core Context which implements all the Context interfaces:
@@ -513,6 +514,12 @@ public class FragmentContextImpl extends BaseFragmentContext implements Executor
   }
 
   @Override
+  public QueryId getQueryId() { return fragment.getHandle().getQueryId();}
+
+  @Override
+  public String getQueryIdString() { return QueryIdHelper.getQueryId(getQueryId()); }
+
+  @Override
   public boolean isImpersonationEnabled() {
     // TODO(DRILL-2097): Until SimpleRootExec tests are removed, we need to consider impersonation disabled if there is
     // no config
@@ -606,4 +613,11 @@ public class FragmentContextImpl extends BaseFragmentContext implements Executor
       }
     }
   }
+
+  @Override
+  public SqlStatementType getSQLStatementType() {
+    Preconditions.checkNotNull(queryContext, "Statement type is only valid for root fragment."
+            + " Calling from non-root fragment");
+    return queryContext.getSQLStatementType();
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index d770636..03ed9fc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -61,6 +61,9 @@ import io.netty.buffer.DrillBuf;
 // in fragment contexts
 public class QueryContext implements AutoCloseable, OptimizerRulesContext, SchemaConfigInfoProvider {
 
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryContext.class);
+  public enum SqlStatementType {OTHER, ANALYZE, CTAS, EXPLAIN, DESCRIBE_TABLE, DESCRIBE_SCHEMA, REFRESH, SELECT, SETOPTION};
+
   private final DrillbitContext drillbitContext;
   private final UserSession session;
   private final QueryId queryId;
@@ -76,6 +79,7 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
   private final SchemaTreeProvider schemaTreeProvider;
   /** Stores constants and their holders by type */
   private final Map<String, Map<MinorType, ValueHolder>> constantValueHolderCache;
+  private SqlStatementType stmtType;
 
   /*
    * Flag to indicate if close has been called, after calling close the first
@@ -102,6 +106,7 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
     }
 
     queryContextInfo = Utilities.createQueryContextInfo(session.getDefaultSchemaPath(), session.getSessionId());
+
     contextInformation = new ContextInformation(session.getCredentials(), queryContextInfo);
 
     allocator = drillbitContext.getAllocator().newChildAllocator(
@@ -112,6 +117,7 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
     viewExpansionContext = new ViewExpansionContext(this);
     schemaTreeProvider = new SchemaTreeProvider(drillbitContext);
     constantValueHolderCache = Maps.newHashMap();
+    stmtType = null;
   }
 
   @Override
@@ -260,6 +266,8 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
    * Re-creates drill operator table to refresh functions list from local function registry.
    */
   public void reloadDrillOperatorTable() {
+    // This is re-trying the query plan on failure so qualifies to reset the SQL statement.
+    clearSQLStatementType();
     table = new DrillOperatorTable(
         drillbitContext.getFunctionImplementationRegistry(),
         drillbitContext.getOptionManager());
@@ -288,10 +296,6 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
     return new PartitionExplorerImpl(getRootSchema());
   }
 
-  public DrillbitContext getDrillbitContext() {
-    return drillbitContext;
-  }
-
   @Override
   public ValueHolder getConstantValueHolder(String value, MinorType type, Function<DrillBuf, ValueHolder> holderInitializer) {
     if (!constantValueHolderCache.containsKey(value)) {
@@ -325,4 +329,30 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
       closed = true;
     }
   }
+
+  /**
+  * @param stmtType : Sets the type {@link SqlStatementType} of the statement e.g. CTAS, ANALYZE
+  */
+  public void setSQLStatementType(SqlStatementType stmtType) {
+    if (this.stmtType == null) {
+      this.stmtType = stmtType;
+    } else {
+      throw new IllegalStateException(String.format("SQL Statement type is already set to %s", this.stmtType));
+    }
+  }
+
+  /*
+   * Clears the type {@link SqlStatementType} of the statement. Ideally we should not clear the statement type
+   * so this should never be exposed outside the QueryContext
+   */
+  private void clearSQLStatementType() {
+    this.stmtType = null;
+  }
+
+  /**
+   * @return Get the type {@link SqlStatementType} of the statement e.g. CTAS, ANALYZE
+   */
+  public SqlStatementType getSQLStatementType() {
+    return stmtType;
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/UdfUtilities.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/UdfUtilities.java
index b1b2bec..96b91da 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/UdfUtilities.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/UdfUtilities.java
@@ -17,15 +17,14 @@
  */
 package org.apache.drill.exec.ops;
 
-import org.apache.drill.shaded.guava.com.google.common.base.Function;
+import io.netty.buffer.DrillBuf;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.expr.holders.ValueHolder;
+import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.store.PartitionExplorer;
-
+import org.apache.drill.shaded.guava.com.google.common.base.Function;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
 
-import io.netty.buffer.DrillBuf;
-
 /**
  * Defines the query state and shared resources available to UDFs through
  * injectables. For use in a function, include a {@link javax.inject.Inject}
@@ -41,6 +40,7 @@ public interface UdfUtilities {
           .put(DrillBuf.class, "getManagedBuffer")
           .put(PartitionExplorer.class, "getPartitionExplorer")
           .put(ContextInformation.class, "getContextInformation")
+          .put(OptionManager.class, "getOptions")
           .build();
 
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractDbGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractDbGroupScan.java
index 42e4bb9..403492c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractDbGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractDbGroupScan.java
@@ -46,8 +46,6 @@ public abstract class AbstractDbGroupScan extends AbstractGroupScan implements D
 
   public abstract StoragePluginConfig getStorageConfig();
 
-  public abstract List<SchemaPath> getColumns();
-
   @Override
   public boolean supportsSecondaryIndex() {
     return false;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
index a4eb369..7e2623a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
@@ -137,6 +137,11 @@ public abstract class AbstractGroupScan extends AbstractBase implements GroupSca
   }
 
   @Override
+  public List<SchemaPath> getColumns() {
+    return GroupScan.ALL_COLUMNS;
+  }
+
+  @Override
   public List<SchemaPath> getPartitionColumns() {
     return Lists.newArrayList();
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
index 85d2a2b..212581d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
@@ -36,6 +36,7 @@ import org.apache.drill.exec.physical.config.Screen;
 import org.apache.drill.exec.physical.config.SingleSender;
 import org.apache.drill.exec.physical.config.Sort;
 import org.apache.drill.exec.physical.config.StatisticsAggregate;
+import org.apache.drill.exec.physical.config.StatisticsMerge;
 import org.apache.drill.exec.physical.config.StreamingAggregate;
 import org.apache.drill.exec.physical.config.Trace;
 import org.apache.drill.exec.physical.config.UnionAll;
@@ -104,6 +105,11 @@ public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> impleme
   }
 
   @Override
+  public T visitStatisticsMerge(StatisticsMerge agg, X value) throws E {
+    return visitOp(agg, value);
+  }
+
+  @Override
   public T visitHashAggregate(HashAggregate agg, X value) throws E {
     return visitOp(agg, value);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
index 263ef05..e42ae2d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
@@ -17,9 +17,9 @@
  */
 package org.apache.drill.exec.physical.base;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
 import java.util.Collection;
 import java.util.List;
-
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
@@ -28,10 +28,8 @@ import org.apache.drill.exec.ops.UdfUtilities;
 import org.apache.drill.exec.physical.PhysicalOperatorSetupException;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 
 /**
  * A GroupScan operator represents all data which will be scanned by a given physical
@@ -110,6 +108,12 @@ public interface GroupScan extends Scan, HasAffinity{
   boolean supportsPartitionFilterPushdown();
 
   /**
+   * Returns a list of columns scanned by this group scan
+   *
+   */
+  List<SchemaPath> getColumns();
+
+  /**
    * Returns a list of columns that can be used for partition pruning
    *
    */
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java
index 5065da5..f26e78d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java
@@ -20,10 +20,16 @@ package org.apache.drill.exec.physical.base;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.drill.common.expression.ErrorCollector;
+import org.apache.drill.common.expression.ErrorCollectorImpl;
+import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
+import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.MinorFragmentEndpoint;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-
+import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
 public class PhysicalOperatorUtil {
@@ -56,4 +62,22 @@ public class PhysicalOperatorUtil {
 
     return destinations;
   }
+
+  /**
+   * Helper method tp materialize the given logical expression using the ExpressionTreeMaterializer
+   * @param expr Logical expression to materialize
+   * @param incoming Incoming record batch
+   * @param context Fragment context
+   */
+  public static LogicalExpression materializeExpression(LogicalExpression expr,
+      VectorAccessible incoming, FragmentContext context) throws SchemaChangeException {
+    ErrorCollector collector = new ErrorCollectorImpl();
+    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector,
+            context.getFunctionRegistry());
+    if (collector.hasErrors()) {
+      throw new SchemaChangeException("Failure while materializing expression. "
+          + collector.toErrorString());
+    }
+    return mle;
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
index a21f578..1a63538 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
@@ -36,6 +36,7 @@ import org.apache.drill.exec.physical.config.Screen;
 import org.apache.drill.exec.physical.config.SingleSender;
 import org.apache.drill.exec.physical.config.Sort;
 import org.apache.drill.exec.physical.config.StatisticsAggregate;
+import org.apache.drill.exec.physical.config.StatisticsMerge;
 import org.apache.drill.exec.physical.config.StreamingAggregate;
 import org.apache.drill.exec.physical.config.Trace;
 import org.apache.drill.exec.physical.config.UnionAll;
@@ -71,6 +72,7 @@ public interface PhysicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
   public RETURN visitReceiver(Receiver receiver, EXTRA value) throws EXCEP;
   public RETURN visitStreamingAggregate(StreamingAggregate agg, EXTRA value) throws EXCEP;
   public RETURN visitStatisticsAggregate(StatisticsAggregate agg, EXTRA value) throws EXCEP;
+  public RETURN visitStatisticsMerge(StatisticsMerge agg, EXTRA value) throws EXCEP;
   public RETURN visitHashAggregate(HashAggregate agg, EXTRA value) throws EXCEP;
   public RETURN visitWriter(Writer op, EXTRA value) throws EXCEP;
   public RETURN visitUnpivot(UnpivotMaps op, EXTRA value) throws EXCEP;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SchemalessScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SchemalessScan.java
index d227b04..999c417 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SchemalessScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/SchemalessScan.java
@@ -36,6 +36,12 @@ public class SchemalessScan extends AbstractFileGroupScan implements SubScan {
 
   @JsonCreator
   public SchemalessScan(@JsonProperty("userName") String userName,
+                        @JsonProperty("selectionRoot") String selectionRoot,
+                        @JsonProperty("columns") List<SchemaPath> columns) {
+    this(userName, selectionRoot);
+  }
+
+  public SchemalessScan(@JsonProperty("userName") String userName,
                         @JsonProperty("selectionRoot") String selectionRoot) {
     super(userName);
     this.selectionRoot = selectionRoot;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java
index 95ee6bf..1fa30ab 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -6,9 +6,9 @@
  * 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
- * <p/>
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
+ *
  * 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.
@@ -31,8 +31,6 @@ import java.util.List;
 
 @JsonTypeName("statistics-aggregate")
 public class StatisticsAggregate extends StreamingAggregate {
-  // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsAggregate.class);
-
   private final List<String> functions;
 
   @JsonCreator
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsMerge.java
similarity index 66%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsMerge.java
index 95ee6bf..1570660 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsAggregate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StatisticsMerge.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -6,9 +6,9 @@
  * 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
- * <p/>
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
+ *
  * 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.
@@ -17,50 +17,53 @@
  */
 package org.apache.drill.exec.physical.config;
 
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
-
-import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.physical.base.PhysicalVisitor;
-import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
-
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import java.util.Map;
+import org.apache.drill.exec.physical.base.AbstractSingle;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
 
-import java.util.List;
-
-@JsonTypeName("statistics-aggregate")
-public class StatisticsAggregate extends StreamingAggregate {
-  // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsAggregate.class);
+@JsonTypeName("statistics-merge")
+public class StatisticsMerge extends AbstractSingle {
 
-  private final List<String> functions;
+  private final Map<String, String> functions;
+  private final double samplePercent;
 
   @JsonCreator
-  public StatisticsAggregate(
+  public StatisticsMerge(
       @JsonProperty("child") PhysicalOperator child,
-      @JsonProperty("functions") List<String> functions) {
-    super(child, null, null, 0.f);
-    this.functions = ImmutableList.copyOf(functions);
+      @JsonProperty("functions") Map<String, String> functions,
+      @JsonProperty("samplePercent") double samplePercent) {
+    super(child);
+    this.functions = ImmutableMap.copyOf(functions);
+    this.samplePercent = samplePercent;
   }
 
-  public List<String> getFunctions() {
+  public Map<String, String> getFunctions() {
     return functions;
   }
 
+  public double getSamplePercent() {
+    return samplePercent;
+  }
+
   @Override
   public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value)
       throws E {
-    return physicalVisitor.visitStatisticsAggregate(this, value);
+    return physicalVisitor.visitStatisticsMerge(this, value);
   }
 
   @Override
   protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
-    return new StatisticsAggregate(child, functions);
+    return new StatisticsMerge(child, functions, samplePercent);
   }
 
   @Override
   public int getOperatorType() {
-    return CoreOperatorType.STATISTICS_AGGREGATE_VALUE;
+    return CoreOperatorType.STATISTICS_MERGE_VALUE;
   }
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java
index ac71b11..4a561bf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnpivotMaps.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -6,9 +6,9 @@
  * 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
- * <p/>
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
+ *
  * 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.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/StatisticsWriterRecordBatch.java
similarity index 75%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/StatisticsWriterRecordBatch.java
index 3a8485a..6cad071 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/StatisticsWriterRecordBatch.java
@@ -15,46 +15,48 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.drill.exec.physical.impl;
 
-import java.io.IOException;
 
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.Writer;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos;
 import org.apache.drill.exec.record.AbstractRecordBatch;
 import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.store.EventBasedRecordWriter;
-import org.apache.drill.exec.store.RecordWriter;
+import org.apache.drill.exec.store.StatisticsRecordWriter;
+import org.apache.drill.exec.store.StatisticsRecordWriterImpl;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.BigIntVector;
 import org.apache.drill.exec.vector.VarCharVector;
 
-/* Write the RecordBatch to the given RecordWriter. */
-public class WriterRecordBatch extends AbstractRecordBatch<Writer> {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WriterRecordBatch.class);
+import java.io.IOException;
+
+public class StatisticsWriterRecordBatch extends AbstractRecordBatch<Writer> {
+
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsWriterRecordBatch.class);
 
-  private EventBasedRecordWriter eventBasedRecordWriter;
-  private RecordWriter recordWriter;
+  private StatisticsRecordWriterImpl statsRecordWriterImpl;
+  private StatisticsRecordWriter recordWriter;
   private long counter = 0;
   private final RecordBatch incoming;
   private boolean processed = false;
   private final String fragmentUniqueId;
   private BatchSchema schema;
 
-  public WriterRecordBatch(Writer writer, RecordBatch incoming, FragmentContext context, RecordWriter recordWriter) throws OutOfMemoryException {
+  public StatisticsWriterRecordBatch(Writer writer, RecordBatch incoming, FragmentContext context,
+                                     StatisticsRecordWriter recordWriter) throws OutOfMemoryException {
     super(writer, context, false);
     this.incoming = incoming;
 
-    final FragmentHandle handle = context.getHandle();
+    final ExecProtos.FragmentHandle handle = context.getHandle();
     fragmentUniqueId = String.format("%d_%d", handle.getMajorFragmentId(), handle.getMinorFragmentId());
     this.recordWriter = recordWriter;
   }
@@ -75,14 +77,18 @@ public class WriterRecordBatch extends AbstractRecordBatch<Writer> {
   }
 
   @Override
+  public void dump() {
+    logger.error("StatisticsWriterRecordBatch[container={}, popConfig={}, counter={}, fragmentUniqueId={}, schema={}]",
+        container, popConfig, counter, fragmentUniqueId, schema);
+  }
+
+  @Override
   public IterOutcome innerNext() {
     if (processed) {
-//      cleanup();
       // if the upstream record batch is already processed and next() is called by
       // downstream then return NONE to indicate completion
       return IterOutcome.NONE;
     }
-
     // process the complete upstream in one next() call
     IterOutcome upstream;
     try {
@@ -95,20 +101,14 @@ public class WriterRecordBatch extends AbstractRecordBatch<Writer> {
             return upstream;
 
           case NOT_YET:
-            break;
           case NONE:
-            if (schema != null) {
-              // Schema is for the output batch schema which is setup in setupNewSchema(). Since the output
-              // schema is fixed ((Fragment(VARCHAR), Number of records written (BIGINT)) we should set it
-              // up even with 0 records for it to be reported back to the client.
-              break;
-            }
+            break;
 
           case OK_NEW_SCHEMA:
             setupNewSchema();
             // $FALL-THROUGH$
           case OK:
-            counter += eventBasedRecordWriter.write(incoming.getRecordCount());
+            counter += statsRecordWriterImpl.writeStatistics(incoming.getRecordCount());
             logger.debug("Total records written so far: {}", counter);
 
             for(final VectorWrapper<?> v : incoming) {
@@ -120,6 +120,8 @@ public class WriterRecordBatch extends AbstractRecordBatch<Writer> {
             throw new UnsupportedOperationException();
         }
       } while(upstream != IterOutcome.NONE);
+      // Flush blocking writers now
+      statsRecordWriterImpl.flushBlockingWriter();
     } catch(IOException ex) {
       logger.error("Failure during query", ex);
       kill(false);
@@ -136,16 +138,14 @@ public class WriterRecordBatch extends AbstractRecordBatch<Writer> {
   }
 
   private void addOutputContainerData() {
-    @SuppressWarnings("resource")
     final VarCharVector fragmentIdVector = (VarCharVector) container.getValueAccessorById(
         VarCharVector.class,
         container.getValueVectorId(SchemaPath.getSimplePath("Fragment")).getFieldIds())
-      .getValueVector();
+        .getValueVector();
     AllocationHelper.allocate(fragmentIdVector, 1, 50);
-    @SuppressWarnings("resource")
     final BigIntVector summaryVector = (BigIntVector) container.getValueAccessorById(BigIntVector.class,
-            container.getValueVectorId(SchemaPath.getSimplePath("Number of records written")).getFieldIds())
-          .getValueVector();
+        container.getValueVectorId(SchemaPath.getSimplePath("Number of records written")).getFieldIds())
+        .getValueVector();
     AllocationHelper.allocate(summaryVector, 1, 8);
     fragmentIdVector.getMutator().setSafe(0, fragmentUniqueId.getBytes());
     fragmentIdVector.getMutator().setValueCount(1);
@@ -164,10 +164,10 @@ public class WriterRecordBatch extends AbstractRecordBatch<Writer> {
       //   1. Fragment unique id.
       //   2. Summary: currently contains number of records written.
       final MaterializedField fragmentIdField =
-          MaterializedField.create("Fragment", Types.required(MinorType.VARCHAR));
+          MaterializedField.create("Fragment", Types.required(TypeProtos.MinorType.VARCHAR));
       final MaterializedField summaryField =
           MaterializedField.create("Number of records written",
-              Types.required(MinorType.BIGINT));
+              Types.required(TypeProtos.MinorType.BIGINT));
 
       container.addOrGet(fragmentIdField);
       container.addOrGet(summaryField);
@@ -176,8 +176,8 @@ public class WriterRecordBatch extends AbstractRecordBatch<Writer> {
       stats.stopSetup();
     }
 
-    eventBasedRecordWriter = new EventBasedRecordWriter(incoming, recordWriter);
-    container.buildSchema(SelectionVectorMode.NONE);
+    statsRecordWriterImpl = new StatisticsRecordWriterImpl(incoming, recordWriter);
+    container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
     schema = container.getSchema();
   }
 
@@ -188,6 +188,7 @@ public class WriterRecordBatch extends AbstractRecordBatch<Writer> {
     }
 
     try {
+      //Perform any cleanup prior to closing the writer
       recordWriter.cleanup();
     } catch(IOException ex) {
       context.getExecutorState().fail(ex);
@@ -210,9 +211,4 @@ public class WriterRecordBatch extends AbstractRecordBatch<Writer> {
     super.close();
   }
 
-  @Override
-  public void dump() {
-    logger.error("WriterRecordBatch[container={}, popConfig={}, counter={}, fragmentUniqueId={}, schema={}]",
-        container, popConfig, counter, fragmentUniqueId, schema);
-  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
index 3a8485a..7c13f72 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
@@ -188,6 +188,9 @@ public class WriterRecordBatch extends AbstractRecordBatch<Writer> {
     }
 
     try {
+      //Perform any post processing tasks prior to cleaning up the writer
+      recordWriter.postProcessing();
+      //Perform any cleanup prior to closing the writer
       recordWriter.cleanup();
     } catch(IOException ex) {
       context.getExecutorState().fail(ex);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java
index 396fd36..ebdd4a4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/InternalBatch.java
@@ -32,7 +32,7 @@ public class InternalBatch implements Iterable<VectorWrapper<?>>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(InternalBatch.class);
 
   private final VectorContainer container;
-  private final RecordBatch incoming;
+  private final FragmentContext context;
   private final BatchSchema schema;
   private final SelectionVector2 sv2;
   private final SelectionVector4 sv4;
@@ -56,7 +56,7 @@ public class InternalBatch implements Iterable<VectorWrapper<?>>{
       this.sv2 = null;
     }
     this.schema = incoming.getSchema();
-    this.incoming = incoming;
+    this.context = incoming.getContext();
     this.container = VectorContainer.getTransferClone(incoming, ignoreWrappers, oContext);
   }
 
@@ -92,6 +92,6 @@ public class InternalBatch implements Iterable<VectorWrapper<?>>{
   }
 
   public FragmentContext getContext() {
-    return incoming.getContext();
+    return context;
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatch.java
deleted file mode 100644
index ea5a7b3..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatch.java
+++ /dev/null
@@ -1,223 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.physical.impl.aggregate;
-
-import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
-import com.sun.codemodel.JExpr;
-import org.apache.drill.common.expression.ErrorCollector;
-import org.apache.drill.common.expression.ErrorCollectorImpl;
-import org.apache.drill.common.expression.FunctionCallFactory;
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.ValueExpressions;
-import org.apache.drill.exec.exception.ClassTransformationException;
-import org.apache.drill.exec.exception.OutOfMemoryException;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.expr.ClassGenerator;
-import org.apache.drill.exec.expr.CodeGenerator;
-import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
-import org.apache.drill.exec.expr.TypeHelper;
-import org.apache.drill.exec.expr.ValueVectorWriteExpression;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.physical.config.StatisticsAggregate;
-import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.TypedFieldId;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.complex.FieldIdUtil;
-import org.apache.drill.exec.vector.complex.MapVector;
-
-import java.io.IOException;
-import java.util.List;
-
-/**
- * TODO: This needs cleanup. Currently the key values are constants and we compare the constants for
- * every record. Seems unnecessary.
- *
- * Example input and output:
- * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
- * Schema of output:
- *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
- *    "computed" : BIGINT - What time is it computed?
- *    "columns" : MAP - Column names
- *       "region_id" : VARCHAR
- *       "sales_city" : VARCHAR
- *       "cnt" : VARCHAR
- *    "statscount" : MAP
- *       "region_id" : BIGINT - statscount(region_id) - aggregation over all values of region_id in incoming batch
- *       "sales_city" : BIGINT - statscount(sales_city)
- *       "cnt" : BIGINT - statscount(cnt)
- *    "nonnullstatcount" : MAP
- *       "region_id" : BIGINT - nonnullstatcount(region_id)
- *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
- *       "cnt" : BIGINT - nonnullstatcount(cnt)
- *   .... another map for next stats function ....
- */
-public class StatisticsAggBatch extends StreamingAggBatch {
-  private List<String> functions;
-  private int schema = 0;
-
-  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming, FragmentContext context)
-      throws OutOfMemoryException {
-    super(popConfig, incoming, context);
-    this.functions = popConfig.getFunctions();
-  }
-
-  private void createKeyColumn(String name, LogicalExpression expr, List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
-      throws SchemaChangeException {
-    ErrorCollector collector = new ErrorCollectorImpl();
-
-    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector, context.getFunctionRegistry());
-
-    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
-    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
-
-    keyExprs.add(mle);
-    keyOutputIds.add(container.add(vector));
-
-    if (collector.hasErrors()) {
-      throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString());
-    }
-  }
-
-  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr, List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
-      throws SchemaChangeException {
-    ErrorCollector collector = new ErrorCollectorImpl();
-
-    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector, context.getFunctionRegistry());
-
-    Class<? extends ValueVector> vvc =
-        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
-
-    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
-
-    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getName()));
-    assert pfid.getFieldIds().length == 1;
-    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
-    builder.addId(pfid.getFieldIds()[0]);
-    TypedFieldId id =
-        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
-            SchemaPath.getSimplePath(vv.getField().getName()).getRootSegment());
-
-    keyExprs.add(mle);
-    keyOutputIds.add(id);
-
-    if (collector.hasErrors()) {
-      throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString());
-    }
-  }
-
-  private void addMapVector(String name, MapVector parent, LogicalExpression expr, List<LogicalExpression> valueExprs)
-      throws SchemaChangeException {
-    ErrorCollector collector = new ErrorCollectorImpl();
-
-    LogicalExpression mle = ExpressionTreeMaterializer.materialize(expr, incoming, collector, context.getFunctionRegistry());
-
-    Class<? extends ValueVector> vvc =
-        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(), mle.getMajorType().getMode());
-    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
-
-    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getName()));
-    assert pfid.getFieldIds().length == 1;
-    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
-    builder.addId(pfid.getFieldIds()[0]);
-    TypedFieldId id = FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
-        SchemaPath.getSimplePath(vv.getField().getName()).getRootSegment());
-
-    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
-
-    if (collector.hasErrors()) {
-      throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString());
-    }
-  }
-
-  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs, List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
-      throws SchemaChangeException, ClassTransformationException, IOException {
-    ClassGenerator<StreamingAggregator> cg = CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getOptions());
-
-    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
-    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
-    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
-
-    keyExprs.toArray(keyExprsArray);
-    valueExprs.toArray(valueExprsArray);
-    keyOutputIds.toArray(keyOutputIdsArray);
-
-    setupIsSame(cg, keyExprsArray);
-    setupIsSameApart(cg, keyExprsArray);
-    addRecordValues(cg, valueExprsArray);
-    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
-    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
-
-    cg.getBlock("resetValues")._return(JExpr.TRUE);
-    getIndex(cg);
-
-    container.buildSchema(SelectionVectorMode.NONE);
-    StreamingAggregator agg = context.getImplementationClass(cg);
-    agg.setup(oContext, incoming, this, ValueVector.MAX_ROW_COUNT);
-    return agg;
-  }
-
-  protected StreamingAggregator createAggregatorInternal()
-      throws SchemaChangeException, ClassTransformationException, IOException {
-    container.clear();
-
-    List<LogicalExpression> keyExprs = Lists.newArrayList();
-    List<LogicalExpression> valueExprs = Lists.newArrayList();
-    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
-
-    createKeyColumn("schema",
-        ValueExpressions.getBigInt(schema++),
-        keyExprs,
-        keyOutputIds
-    );
-    createKeyColumn("computed",
-        ValueExpressions.getBigInt(System.currentTimeMillis()),
-        keyExprs,
-        keyOutputIds
-    );
-
-    MapVector cparent = new MapVector("column", oContext.getAllocator(), null);
-    container.add(cparent);
-    for (MaterializedField mf : incoming.getSchema()) {
-      createNestedKeyColumn(
-          cparent,
-          mf.getName(),
-          ValueExpressions.getChar(mf.getName(), 0),
-          keyExprs,
-          keyOutputIds
-      );
-    }
-
-    for (String func : functions) {
-      MapVector parent = new MapVector(func, oContext.getAllocator(), null);
-      container.add(parent);
-
-      for (MaterializedField mf : incoming.getSchema()) {
-        List<LogicalExpression> args = Lists.newArrayList();
-        args.add(SchemaPath.getSimplePath(mf.getName()));
-        LogicalExpression call = FunctionCallFactory.createExpression(func, args);
-
-        addMapVector(mf.getName(), parent, call, valueExprs);
-      }
-    }
-
-    return codegenAggregator(keyExprs, valueExprs, keyOutputIds);
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
index e1e43bd..7570016 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
@@ -447,7 +447,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
     complexWriters.add(writer);
   }
 
-  private StreamingAggregator createAggregatorInternal() throws SchemaChangeException, ClassTransformationException, IOException{
+  protected StreamingAggregator createAggregatorInternal() throws SchemaChangeException, ClassTransformationException, IOException{
     ClassGenerator<StreamingAggregator> cg = CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getOptions());
     cg.getCodeGenerator().plainJavaCapable(true);
     // Uncomment out this line to debug the generated code.
diff --git a/logical/src/main/java/org/apache/drill/common/logical/data/Analyze.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java
similarity index 50%
copy from logical/src/main/java/org/apache/drill/common/logical/data/Analyze.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java
index 711050d..a4ee74f 100644
--- a/logical/src/main/java/org/apache/drill/common/logical/data/Analyze.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AbstractMergedStatistic.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,21 +15,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.common.logical.data;
+package org.apache.drill.exec.physical.impl.statistics;
 
-import org.apache.drill.common.logical.data.visitors.LogicalVisitor;
+import org.apache.drill.exec.vector.complex.MapVector;
 
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonTypeName;
+public abstract class AbstractMergedStatistic extends Statistic implements MergedStatistic {
+  protected String name;
+  protected String inputName;
+  protected double samplePercent;
+  protected State state;
 
-@JsonTypeName("analyze")
-public class Analyze extends SingleInputOperator {
+  public void initialize(String name, String inputName, double samplePercent) {
+    this.name = name;
+    this.inputName = inputName;
+    this.samplePercent = samplePercent;
+  }
 
-  @JsonCreator
-  public Analyze() { }
+  @Override
+  public abstract void initialize(String inputName, double samplePercent);
 
   @Override
-  public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
-    return logicalVisitor.visitAnalyze(this, value);
-  }
+  public abstract String getName();
+
+  @Override
+  public abstract String getInput();
+
+  @Override
+  public abstract void merge(MapVector input);
+
+  @Override
+  public abstract void setOutput(MapVector output);
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java
new file mode 100644
index 0000000..6a9660d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.statistics;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.drill.common.types.MinorType;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.record.MajorTypeSerDe;
+import org.apache.drill.exec.vector.NullableFloat8Vector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
+  private Map<String, Double> sumHolder;
+  ColTypeMergedStatistic types;
+  NNRowCountMergedStatistic nonNullStatCounts;
+  RowCountMergedStatistic statCounts;
+
+  public AvgWidthMergedStatistic () {
+    this.sumHolder = new HashMap<>();
+    types = null;
+    nonNullStatCounts = null;
+    statCounts = null;
+    state = State.INIT;
+  }
+
+  @Override
+  public void initialize(String inputName, double samplePercent) {
+    super.initialize(Statistic.AVG_WIDTH, inputName, samplePercent);
+    state = State.CONFIG;
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public String getInput() {
+    return inputName;
+  }
+
+  @Override
+  public void merge(MapVector input) {
+    // Check the input is a Map Vector
+    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+    for (ValueVector vv : input) {
+      NullableFloat8Vector fv = (NullableFloat8Vector) vv;
+      NullableFloat8Vector.Accessor accessor = fv.getAccessor();
+      String colName = vv.getField().getName();
+      double sum = 0;
+      if (sumHolder.get(colName) != null) {
+        sum = sumHolder.get(colName);
+      }
+      if (!accessor.isNull(0)) {
+        sum += accessor.get(0);
+        sumHolder.put(colName, sum);
+      }
+    }
+  }
+
+  public double getStat(String colName) {
+    if (state != State.COMPLETE) {
+      throw new IllegalStateException(
+          String.format("Statistic `%s` has not completed merging statistics", name));
+    }
+    return sumHolder.get(colName)/((samplePercent/100.0) *getRowCount(colName));
+  }
+
+  @Override
+  public void setOutput(MapVector output) {
+    // Check the input is a Map Vector
+    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+    // Dependencies have been configured correctly
+    assert (state == State.MERGE);
+    for (ValueVector outMapCol : output) {
+      String colName = outMapCol.getField().getName();
+      NullableFloat8Vector vv = (NullableFloat8Vector) outMapCol;
+      vv.allocateNewSafe();
+      // For variable-length columns, we divide by non-null rows since NULL values do not
+      // take up space. For fixed-length columns NULL values take up space.
+      if (sumHolder.get(colName) != null
+          && getRowCount(colName) > 0) {
+        vv.getMutator().setSafe(0, sumHolder.get(colName)/((samplePercent/100.0) *getRowCount(colName)));
+      } else {
+        vv.getMutator().setNull(0);
+      }
+    }
+    state = State.COMPLETE;
+  }
+
+  public void configure(List<MergedStatistic> statisticList) {
+    assert (state == State.CONFIG);
+    for (MergedStatistic statistic : statisticList) {
+      if (statistic.getName().equals(Statistic.COLTYPE)) {
+        types = (ColTypeMergedStatistic) statistic;
+      } else if (statistic.getName().equals(Statistic.ROWCOUNT)) {
+        statCounts = (RowCountMergedStatistic) statistic;
+      } else if (statistic.getName().equals(Statistic.NNROWCOUNT)) {
+        nonNullStatCounts = (NNRowCountMergedStatistic) statistic;
+      }
+    }
+    assert (types != null && statCounts != null && nonNullStatCounts != null);
+    // Now config complete - moving to MERGE state
+    state = State.MERGE;
+  }
+
+  private long getRowCount(String colName) {
+    byte[] typeAsBytes = types.getStat(colName);
+    int type  = -1;
+    ObjectMapper mapper = new ObjectMapper();
+    SimpleModule deModule = new SimpleModule("StatisticsSerDeModeule") //
+            .addDeserializer(TypeProtos.MajorType.class, new MajorTypeSerDe.De());
+    mapper.registerModule(deModule);
+    try {
+      type = mapper.readValue(typeAsBytes, TypeProtos.MajorType.class).getMinorType().getNumber();
+    } catch (IOException ex) {
+      //Ignore exception
+    }
+    // If variable length type - then use the nonNullCount. Otherwise, use the Count,
+    // since even NULL values take up the same space.
+    if (type == MinorType.VAR16CHAR.getNumber()
+        || type == MinorType.VARCHAR.getNumber()
+        || type == MinorType.VARBINARY.getNumber()) {
+      return nonNullStatCounts.getStat(colName);
+    } else {
+      return statCounts.getStat(colName);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/CntDupsMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/CntDupsMergedStatistic.java
new file mode 100644
index 0000000..2d7194d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/CntDupsMergedStatistic.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.statistics;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public class CntDupsMergedStatistic extends AbstractMergedStatistic {
+    private Map<String, Long> sumHolder;
+
+    public CntDupsMergedStatistic () {
+        this.sumHolder = new HashMap<>();
+        state = State.INIT;
+    }
+
+    @Override
+    public void initialize(String inputName, double samplePercent) {
+        super.initialize(Statistic.SUM_DUPS, inputName, samplePercent);
+        state = State.CONFIG;
+    }
+
+    @Override
+    public String getName() {
+        return name;
+    }
+
+    @Override
+    public String getInput() {
+        return inputName;
+    }
+
+    @Override
+    public void merge(MapVector input) {
+        // Check the input is a Map Vector
+        assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+        for (ValueVector vv : input) {
+            NullableBigIntVector fv = (NullableBigIntVector) vv;
+            NullableBigIntVector.Accessor accessor = fv.getAccessor();
+            String colName = vv.getField().getName();
+            long sum = 0;
+            if (sumHolder.get(colName) != null) {
+                sum = sumHolder.get(colName);
+            }
+            if (!accessor.isNull(0)) {
+                sum += accessor.get(0);
+                sumHolder.put(colName, sum);
+            }
+        }
+    }
+
+    public double getStat(String colName) {
+        if (state != State.COMPLETE) {
+            throw new IllegalStateException(
+                    String.format("Statistic `%s` has not completed merging statistics", name));
+        }
+        return sumHolder.get(colName);
+    }
+
+    @Override
+    public void setOutput(MapVector output) {
+        // Check the input is a Map Vector
+        assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+        // Dependencies have been configured correctly
+        assert (state == State.MERGE);
+        for (ValueVector outMapCol : output) {
+            String colName = outMapCol.getField().getName();
+            NullableBigIntVector vv = (NullableBigIntVector) outMapCol;
+            vv.allocateNewSafe();
+            // For variable-length columns, we divide by non-null rows since NULL values do not
+            // take up space. For fixed-length columns NULL values take up space.
+            if (sumHolder.get(colName) != null) {
+                vv.getMutator().setSafe(0, sumHolder.get(colName));
+            }
+        }
+        state = State.COMPLETE;
+    }
+
+    public void configure(List<MergedStatistic> statisticList) {
+        assert (state == State.CONFIG);
+        // Now config complete - moving to MERGE state
+        state = State.MERGE;
+    }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColTypeMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColTypeMergedStatistic.java
new file mode 100644
index 0000000..296d055
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColTypeMergedStatistic.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.statistics;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarCharVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public class ColTypeMergedStatistic extends AbstractMergedStatistic {
+  private Map<String, byte[]> typeHolder;
+
+  public ColTypeMergedStatistic () {
+    typeHolder = new HashMap<>();
+    state = State.INIT;
+  }
+
+  @Override
+  public void initialize(String inputName, double samplePercent) {
+    super.initialize(Statistic.COLTYPE, inputName, samplePercent);
+    state = State.MERGE;
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public String getInput() {
+    return inputName;
+  }
+
+  @Override
+  public void merge(MapVector input) {
+    // Check the input is a Map Vector
+    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+    for (ValueVector vv : input) {
+      String colName = vv.getField().getName();
+      if (typeHolder.get(colName) == null) {
+        VarCharVector iv = (VarCharVector) vv;
+        VarCharVector.Accessor accessor = iv.getAccessor();
+        typeHolder.put(colName, accessor.get(0));
+      }
+    }
+  }
+
+  public byte[] getStat(String colName) {
+    if (state != State.COMPLETE) {
+      throw new IllegalStateException(String.format("Statistic `%s` has not completed merging statistics",
+          name));
+    }
+    return typeHolder.get(colName);
+  }
+
+  @Override
+  public void setOutput(MapVector output) {
+    // Check the input is a Map Vector
+    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+    for (ValueVector outMapCol : output) {
+      String colName = outMapCol.getField().getName();
+      VarCharVector vv = (VarCharVector) outMapCol;
+      vv.allocateNewSafe();
+      // Set column name in ValueVector
+      vv.getMutator().setSafe(0, typeHolder.get(colName));
+    }
+    // Now moving to COMPLETE state
+    state = State.COMPLETE;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColumnMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColumnMergedStatistic.java
new file mode 100644
index 0000000..90d3b27
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/ColumnMergedStatistic.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.statistics;
+
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarCharVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public class ColumnMergedStatistic extends AbstractMergedStatistic {
+
+  public ColumnMergedStatistic () {
+    state = State.INIT;
+  }
+
+  @Override
+  public void initialize(String inputName, double samplePercent) {
+    super.initialize(Statistic.COLNAME, inputName, samplePercent);
+    state = State.MERGE;
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public String getInput() {
+    return inputName;
+  }
+
+  @Override
+  public void merge(MapVector input) {
+    // Check the input is a Map Vector
+    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+  }
+
+  @Override
+  public void setOutput(MapVector output) {
+    // Check the input is a Map Vector
+    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+    MapVector outputMap = (MapVector) output;
+    for (ValueVector outMapCol : outputMap) {
+      String colName = outMapCol.getField().getName();
+      VarCharVector vv = (VarCharVector) outMapCol;
+      vv.allocateNewSafe();
+      // Set column name in ValueVector
+      vv.getMutator().setSafe(0, colName.getBytes(), 0, colName.length());
+    }
+    // Now moving to COMPLETE state
+    state = State.COMPLETE;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/HLLMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/HLLMergedStatistic.java
new file mode 100644
index 0000000..68854f7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/HLLMergedStatistic.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.statistics;
+
+// Library implementing HLL algorithm to derive approximate #distinct values(NDV). Please refer:
+// 'HyperLogLog: the analysis of a near-optimal cardinality estimation algorithm.' Flajolet et. al.
+import com.clearspring.analytics.stream.cardinality.HyperLogLog;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.NullableVarBinaryVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public class HLLMergedStatistic extends AbstractMergedStatistic {
+  private Map<String, HyperLogLog> hllHolder;
+  private long accuracy;
+
+  public HLLMergedStatistic () {
+    this.hllHolder = new HashMap<>();
+    state = State.INIT;
+  }
+
+  @Override
+  public void initialize(String inputName, double samplePercent) {
+    super.initialize(Statistic.HLL_MERGE, inputName, samplePercent);
+    state = State.CONFIG;
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public String getInput() {
+    return inputName;
+  }
+
+  @Override
+  public void merge(MapVector input) {
+    // Check the input is a Map Vector
+    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+    for (ValueVector vv : input) {
+      String colName = vv.getField().getName();
+      HyperLogLog colHLLHolder = null;
+      if (hllHolder.get(colName) != null) {
+        colHLLHolder = hllHolder.get(colName);
+      }
+      NullableVarBinaryVector hllVector = (NullableVarBinaryVector) vv;
+      NullableVarBinaryVector.Accessor accessor = hllVector.getAccessor();
+
+      try {
+        if (!accessor.isNull(0)) {
+          ByteArrayInputStream bais = new ByteArrayInputStream(accessor.get(0), 0, vv.getBufferSize());
+          HyperLogLog other = HyperLogLog.Builder.build(new DataInputStream(bais));
+          if (colHLLHolder != null) {
+            colHLLHolder.addAll(other);
+            hllHolder.put(colName, colHLLHolder);
+          } else {
+            hllHolder.put(colName, other);
+          }
+        }
+      } catch (Exception ex) {
+        //TODO: Catch IOException/CardinalityMergeException
+        //TODO: logger
+      }
+    }
+  }
+
+  public HyperLogLog getStat(String colName) {
+    if (state != State.COMPLETE) {
+      throw new IllegalStateException(String.format("Statistic `%s` has not completed merging statistics",
+          name));
+    }
+    return hllHolder.get(colName);
+  }
+
+  @Override
+  public void setOutput(MapVector output) {
+    // Check the input is a Map Vector
+    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+    // Dependencies have been configured correctly
+    assert (state == State.MERGE);
+    for (ValueVector outMapCol : output) {
+      String colName = outMapCol.getField().getName();
+      HyperLogLog colHLLHolder = hllHolder.get(colName);
+      NullableVarBinaryVector vv = (NullableVarBinaryVector) outMapCol;
+      vv.allocateNewSafe();
+      try {
+        if (colHLLHolder != null) {
+          vv.getMutator().setSafe(0, colHLLHolder.getBytes(),
+              0, colHLLHolder.getBytes().length);
+        } else {
+          vv.getMutator().setNull(0);
+        }
+      } catch (IOException ex) {
+        // TODO: logger
+      }
+    }
+    state = State.COMPLETE;
+  }
+
+  public void configure(OptionManager optionsManager) {
+    assert (state == State.CONFIG);
+    accuracy = optionsManager.getLong(ExecConstants.HLL_ACCURACY);
+    // Now config complete - moving to MERGE state
+    state = State.MERGE;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatistic.java
new file mode 100644
index 0000000..26d30f5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatistic.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.statistics;
+
+import org.apache.drill.exec.vector.complex.MapVector;
+
+/*
+ * Interface for implementing a merged statistic. A merged statistic can merge
+ * the input statistics to get the overall value. e.g. `rowcount` merged statistic
+ * should merge all `rowcount` input statistic and return the overall `rowcount`.
+ * Given `rowcount`s 10 and 20, the `rowcount` merged statistic will return 30.
+ */
+public interface MergedStatistic {
+
+  /** Initialize the merged statistic
+   *
+   *  @param inputName - the input {@link StatisticsAggBatch} statistic for this merged statistic
+   *  @param samplePercent - the sample percentage used for extrapolation post merge phase
+   */
+  void initialize(String inputName, double samplePercent);
+
+  /** Gets the name of the merged statistic
+   *
+   * @return - name of this merged statistic
+   */
+  String getName();
+
+  /**
+   * Gets the name of the input statistic
+   *
+   *  @return - name of the input {@link StatisticsAggBatch} statistic for this merged statistic
+   */
+  String getInput();
+
+  /** Merges the input statistic (incoming value vector) into the existing
+   * merged statistic
+   *
+   * @param input - the input value vector to merge
+   */
+  void merge(MapVector input);
+
+  /** Sets the merged statistic value in the output (outgoing value vector)
+   *
+   * @param output - the output vector where to populate the statistic value
+   */
+  void setOutput(MapVector output);
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java
new file mode 100644
index 0000000..c921394
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/MergedStatisticFactory.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.statistics;
+
+import java.util.HashMap;
+
+public class MergedStatisticFactory {
+  private HashMap<String,Class<? extends MergedStatistic>> statsClasses = new HashMap<>( );
+  /*
+   * Creates the appropriate statistics object given the name of the statistics and the input statistic
+   */
+  private static MergedStatisticFactory instance = new MergedStatisticFactory();
+  //Can not instantiate
+  private MergedStatisticFactory() {
+    statsClasses.put(Statistic.COLNAME, ColumnMergedStatistic.class);
+    statsClasses.put(Statistic.COLTYPE, ColTypeMergedStatistic.class);
+    statsClasses.put(Statistic.ROWCOUNT, RowCountMergedStatistic.class);
+    statsClasses.put(Statistic.NNROWCOUNT, NNRowCountMergedStatistic.class);
+    statsClasses.put(Statistic.AVG_WIDTH, AvgWidthMergedStatistic.class);
+    statsClasses.put(Statistic.HLL_MERGE, HLLMergedStatistic.class);
+    statsClasses.put(Statistic.NDV, NDVMergedStatistic.class);
+    statsClasses.put(Statistic.SUM_DUPS, CntDupsMergedStatistic.class);
+  }
+
+  private MergedStatistic newMergedStatistic(String outputStatName)
+      throws InstantiationException, IllegalAccessException {
+    MergedStatistic stat = statsClasses.get(outputStatName).newInstance();
+    return stat;
+  }
+
+  public static MergedStatistic getMergedStatistic(String outputStatName, String inputStatName, double samplePercent) {
+    try {
+      MergedStatistic statistic = instance.newMergedStatistic(outputStatName);
+      if (statistic == null) {
+        throw new IllegalArgumentException("No implementation found for " + outputStatName);
+      } else {
+        statistic.initialize(inputStatName, samplePercent);
+        return statistic;
+      }
+    } catch (InstantiationException ex) {
+      throw new IllegalArgumentException("Cannot instantiate class for " + outputStatName);
+    } catch (IllegalAccessException ex) {
+      throw new IllegalArgumentException("Cannot access class for " + outputStatName);
+    }
+  }
+}
+
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NDVMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NDVMergedStatistic.java
new file mode 100644
index 0000000..fc87fbd
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NDVMergedStatistic.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.statistics;
+
+// Library implementing HLL algorithm to derive approximate #distinct values(NDV). Please refer:
+// 'HyperLogLog: the analysis of a near-optimal cardinality estimation algorithm.' Flajolet et. al.
+import com.clearspring.analytics.stream.cardinality.CardinalityMergeException;
+import com.clearspring.analytics.stream.cardinality.HyperLogLog;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.drill.common.types.MinorType;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.record.MajorTypeSerDe;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableVarBinaryVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public class NDVMergedStatistic extends AbstractMergedStatistic {
+  private Map<String, HyperLogLog> hllHolder;
+  ColTypeMergedStatistic types;
+  NNRowCountMergedStatistic nonNullStatCounts;
+  RowCountMergedStatistic statCounts;
+  CntDupsMergedStatistic sumDups;
+
+  public NDVMergedStatistic () {
+    this.hllHolder = new HashMap<>();
+    types = null;
+    nonNullStatCounts = null;
+    statCounts = null;
+    sumDups = null;
+    state = State.INIT;
+  }
+
+  public static class NDVConfiguration {
+    private final OptionManager optionManager;
+    private final List<MergedStatistic> dependencies;
+
+    public NDVConfiguration (OptionManager optionsManager, List<MergedStatistic> statistics) {
+      this.optionManager = optionsManager;
+      this.dependencies = statistics;
+    }
+  }
+
+  @Override
+  public void initialize(String inputName, double samplePercent) {
+    super.initialize(Statistic.NDV, inputName, samplePercent);
+    state = State.CONFIG;
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public String getInput() {
+    return inputName;
+  }
+
+  @Override
+  public void merge(MapVector input) {
+    // Check the input is a Map Vector
+    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+    // Dependencies have been configured correctly
+    assert (state == State.MERGE);
+    for (ValueVector vv : input) {
+      String colName = vv.getField().getName();
+      HyperLogLog colHLLHolder = null;
+      if (hllHolder.get(colName) != null) {
+        colHLLHolder = hllHolder.get(colName);
+      }
+      NullableVarBinaryVector hllVector = (NullableVarBinaryVector) vv;
+      NullableVarBinaryVector.Accessor accessor = hllVector.getAccessor();
+
+      try {
+        if (!accessor.isNull(0)) {
+          ByteArrayInputStream bais = new ByteArrayInputStream(accessor.get(0), 0, vv.getBufferSize());
+          HyperLogLog other = HyperLogLog.Builder.build(new DataInputStream(bais));
+          if (colHLLHolder != null) {
+            colHLLHolder.addAll(other);
+            hllHolder.put(colName, colHLLHolder);
+          } else {
+            hllHolder.put(colName, other);
+          }
+        }
+      } catch (CardinalityMergeException ex) {
+        throw new IllegalStateException("Failed to merge the NDV statistics");
+      } catch (Exception ex) {
+        throw new IllegalStateException(ex);
+      }
+    }
+  }
+
+  public long getStat(String colName) {
+    if (state != State.COMPLETE) {
+      throw new IllegalStateException(String.format("Statistic `%s` has not completed merging statistics", name));
+    }
+    return hllHolder.get(colName).cardinality();
+  }
+
+  @Override
+  public void setOutput(MapVector output) {
+    // Check the input is a Map Vector
+    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+    // Dependencies have been configured correctly
+    assert (state == State.MERGE);
+    for (ValueVector outMapCol : output) {
+      String colName = outMapCol.getField().getName();
+      HyperLogLog colHLLHolder = hllHolder.get(colName);
+      NullableBigIntVector vv = (NullableBigIntVector) outMapCol;
+      vv.allocateNewSafe();
+      if (colHLLHolder != null) {
+        /* Duj1 estimator - Peter J. Haas & Lynne Stokes (1998) Estimating the Number of Classes in a Finite Population,
+         * Journal of the American Statistical Association, 93:444, 1475-1487
+         * n*d / (n - f1 + f1*n/N) where
+         * n  - sample rows
+         * N  - total rows
+         * d  - ndv of sample
+         * f1 - number of singletons
+         * Cap estimate at N
+         */
+        double sampleRows = (samplePercent/100.0)*getRowCount(colName);
+        double sampleSingletons = sampleRows - sumDups.getStat(colName);
+        double estNdv = (sampleRows * colHLLHolder.cardinality()) /
+                (sampleRows - sampleSingletons + sampleSingletons* samplePercent/100.0);
+        estNdv = Math.min(estNdv, 100.0*sampleRows/samplePercent);
+        vv.getMutator().setSafe(0, 1, (long) estNdv);
+      } else {
+        vv.getMutator().setNull(0);
+      }
+    }
+    state = State.COMPLETE;
+  }
+
+  public void configure(NDVConfiguration ndvConfig) {
+    assert (state == State.CONFIG);
+    for (MergedStatistic statistic : ndvConfig.dependencies) {
+      if (statistic.getName().equals(Statistic.COLTYPE)) {
+        types = (ColTypeMergedStatistic) statistic;
+      } else if (statistic.getName().equals(Statistic.ROWCOUNT)) {
+        statCounts = (RowCountMergedStatistic) statistic;
+      } else if (statistic.getName().equals(Statistic.NNROWCOUNT)) {
+        nonNullStatCounts = (NNRowCountMergedStatistic) statistic;
+      } else if (statistic.getName().equals(Statistic.SUM_DUPS)) {
+        sumDups = (CntDupsMergedStatistic) statistic;
+      }
+    }
+    assert (types != null && statCounts != null && nonNullStatCounts != null && sumDups != null);
+    // Now config complete - moving to MERGE state
+    state = State.MERGE;
+  }
+
+  private long getRowCount(String colName) {
+    byte[] typeAsBytes = types.getStat(colName);
+    int type  = -1;
+    ObjectMapper mapper = new ObjectMapper();
+    SimpleModule deModule = new SimpleModule("StatisticsSerDeModule") //
+            .addDeserializer(TypeProtos.MajorType.class, new MajorTypeSerDe.De());
+    mapper.registerModule(deModule);
+    try {
+      type = mapper.readValue(typeAsBytes, TypeProtos.MajorType.class).getMinorType().getNumber();
+    } catch (IOException ex) {
+      //Ignore exception
+    }
+    // If variable length type - then use the nonNullCount. Otherwise, use the Count,
+    // since even NULL values take up the same space.
+    if (type == MinorType.VAR16CHAR.getNumber()
+            || type == MinorType.VARCHAR.getNumber()
+            || type == MinorType.VARBINARY.getNumber()) {
+      return nonNullStatCounts.getStat(colName);
+    } else {
+      return statCounts.getStat(colName);
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NNRowCountMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NNRowCountMergedStatistic.java
new file mode 100644
index 0000000..a930502
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NNRowCountMergedStatistic.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.statistics;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public class NNRowCountMergedStatistic extends AbstractMergedStatistic {
+
+  private Map<String, Long> sumHolder;
+
+  public NNRowCountMergedStatistic() {
+    this.sumHolder = new HashMap<>();
+    state = State.INIT;
+  }
+
+  @Override
+  public void initialize(String inputName, double samplePercent) {
+    super.initialize(Statistic.NNROWCOUNT, inputName, samplePercent);
+    state = State.MERGE;
+  }
+
+  @Override
+  public String getName() {
+      return name;
+  }
+
+  @Override
+  public String getInput() {
+    return inputName;
+  }
+
+  @Override
+  public void merge(MapVector input) {
+    // Check the input is a Map Vector
+    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+    for (ValueVector vv : input) {
+      String colName = vv.getField().getName();
+      NullableBigIntVector biv = (NullableBigIntVector) vv;
+      NullableBigIntVector.Accessor accessor = biv.getAccessor();
+      long sum = 0;
+      if (sumHolder.get(colName) != null) {
+        sum = sumHolder.get(colName);
+      }
+      if (!accessor.isNull(0)) {
+        sum += accessor.get(0);
+        sumHolder.put(colName, sum);
+      }
+    }
+  }
+
+  public long getStat(String colName) {
+    if (state != State.COMPLETE) {
+      throw new IllegalStateException(String.format("Statistic `%s` has not completed merging statistics",
+          name));
+    }
+    return (long)(100.0*sumHolder.get(colName)/ samplePercent);
+  }
+
+  @Override
+  public void setOutput(MapVector output) {
+    // Check the input is a Map Vector
+    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+    for (ValueVector outMapCol : output) {
+      String colName = outMapCol.getField().getName();
+      NullableBigIntVector vv = (NullableBigIntVector) outMapCol;
+      vv.allocateNewSafe();
+      if (sumHolder.get(colName) != null) {
+        vv.getMutator().setSafe(0, (long)(100.0*sumHolder.get(colName)/ samplePercent));
+      } else {
+        vv.getMutator().setNull(0);
+      }
+    }
+    state = State.COMPLETE;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/RowCountMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/RowCountMergedStatistic.java
new file mode 100644
index 0000000..9993031
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/RowCountMergedStatistic.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.statistics;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+
+public class RowCountMergedStatistic extends AbstractMergedStatistic {
+
+  private Map<String, Long> sumHolder;
+
+  public RowCountMergedStatistic() {
+    this.sumHolder = new HashMap<>();
+    //No CONFIG state so move directly to MERGE state
+    state = State.INIT;
+  }
+
+  @Override
+  public void initialize(String inputName, double samplePercent) {
+    super.initialize(Statistic.ROWCOUNT, inputName, samplePercent);
+    state = State.MERGE;
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public String getInput() {
+    return inputName;
+  }
+
+  @Override
+  public void merge(MapVector input) {
+    // Check the input is a Map Vector
+    assert (input.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+    for (ValueVector vv : input) {
+      String colName = vv.getField().getName();
+      NullableBigIntVector biv = (NullableBigIntVector) vv;
+      NullableBigIntVector.Accessor accessor = biv.getAccessor();
+      long sum = 0;
+      if (sumHolder.get(colName) != null) {
+        sum = sumHolder.get(colName);
+      }
+      if (!accessor.isNull(0)) {
+        sum += accessor.get(0);
+        sumHolder.put(colName, sum);
+      }
+    }
+  }
+
+  public long getStat(String colName) {
+    if (state != State.COMPLETE) {
+      throw new IllegalStateException(String.format("Statistic `%s` has not completed merging statistics",
+          name));
+    }
+    return (long)(100.0*sumHolder.get(colName)/ samplePercent);
+  }
+
+  @Override
+  public void setOutput(MapVector output) {
+    // Check the input is a Map Vector
+    assert (output.getField().getType().getMinorType() == TypeProtos.MinorType.MAP);
+    for (ValueVector outMapCol : output) {
+      String colName = outMapCol.getField().getName();
+      NullableBigIntVector vv = (NullableBigIntVector) outMapCol;
+      vv.allocateNewSafe();
+      if (sumHolder.get(colName) != null) {
+        vv.getMutator().setSafe(0, (long)(100.0*sumHolder.get(colName)/ samplePercent));
+      } else {
+        vv.getMutator().setNull(0);
+      }
+    }
+    state = State.COMPLETE;
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/Statistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/Statistic.java
new file mode 100644
index 0000000..b9f905d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/Statistic.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.statistics;
+
+/*
+ * Base Statistics class - all statistics classes should extend this class
+ */
+public abstract class Statistic {
+  /*
+   * The lifecycle states for statistics
+   */
+  public enum State {INIT, CONFIG, MERGE, COMPLETE};
+  /*
+   * List of statistics used in Drill.
+   */
+  public static final String COLNAME = "column";
+  public static final String COLTYPE = "majortype";
+  public static final String SCHEMA = "schema";
+  public static final String COMPUTED = "computed";
+  public static final String ROWCOUNT = "rowcount";
+  public static final String NNROWCOUNT = "nonnullrowcount";
+  public static final String NDV = "approx_count_distinct";
+  public static final String HLL_MERGE = "hll_merge";
+  public static final String HLL = "hll";
+  public static final String AVG_WIDTH = "avg_width";
+  public static final String SUM_WIDTH = "sum_width";
+  public static final String CNT_DUPS = "approx_count_dups";
+  public static final String SUM_DUPS = "sum";
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java
new file mode 100644
index 0000000..df64a5f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatch.java
@@ -0,0 +1,249 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.statistics;
+
+import com.sun.codemodel.JExpr;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.apache.drill.common.expression.FunctionCallFactory;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.ValueExpressions;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.exception.ClassTransformationException;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.ClassGenerator;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.expr.ValueVectorWriteExpression;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.PhysicalOperatorUtil;
+import org.apache.drill.exec.physical.config.StatisticsAggregate;
+import org.apache.drill.exec.physical.impl.aggregate.StreamingAggBatch;
+import org.apache.drill.exec.physical.impl.aggregate.StreamingAggTemplate;
+import org.apache.drill.exec.physical.impl.aggregate.StreamingAggregator;
+import org.apache.drill.exec.planner.common.DrillStatsTable;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.store.ColumnExplorer;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.FieldIdUtil;
+import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+/*
+ * TODO: This needs cleanup. Currently the key values are constants and we compare the constants
+ * for every record. Seems unnecessary.
+ *
+ * Example input and output:
+ * Schema of incoming batch: region_id (VARCHAR), sales_city (VARCHAR), cnt (BIGINT)
+ * Schema of outgoing batch:
+ *    "columns"       : MAP - Column names
+ *       "region_id"  : VARCHAR
+ *       "sales_city" : VARCHAR
+ *       "cnt"        : VARCHAR
+ *    "statscount" : MAP
+ *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
+ *                      in incoming batch
+ *       "sales_city" : BIGINT - statscount(sales_city)
+ *       "cnt"        : BIGINT - statscount(cnt)
+ *    "nonnullstatcount" : MAP
+ *       "region_id"  : BIGINT - nonnullstatcount(region_id)
+ *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
+ *       "cnt"        : BIGINT - nonnullstatcount(cnt)
+ *   .... another map for next stats function ....
+ */
+
+public class StatisticsAggBatch extends StreamingAggBatch {
+  // List of statistics functions e.g. rowcount, ndv output by StatisticsAggBatch
+  private List<String> functions;
+  // List of implicit columns for which we do NOT want to compute statistics
+  private Map<String, ColumnExplorer.ImplicitFileColumns> implicitFileColumnsMap;
+
+  public StatisticsAggBatch(StatisticsAggregate popConfig, RecordBatch incoming,
+      FragmentContext context) throws OutOfMemoryException {
+    super(popConfig, incoming, context);
+    // Get the list from the physical operator configuration
+    functions = popConfig.getFunctions();
+    implicitFileColumnsMap = ColumnExplorer.initImplicitFileColumns(context.getOptions());
+  }
+
+  /*
+   * Returns whether the given column is an implicit column
+   */
+  private boolean isImplicitFileColumn(MaterializedField mf) {
+    return implicitFileColumnsMap.get(SchemaPath.getSimplePath(mf.getName()).toString()) != null;
+  }
+
+  /*
+   * Create the field id for the value vector corresponding to the materialized expression
+   */
+  private TypedFieldId createVVFieldId(LogicalExpression mle, String name, MapVector parent) {
+    Class<? extends ValueVector> vvc =
+            TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
+                    mle.getMajorType().getMode());
+    ValueVector vv = parent.addOrGet(name, mle.getMajorType(), vvc);
+    TypedFieldId pfid = container.getValueVectorId(SchemaPath.getSimplePath(parent.getField().getName()));
+    assert pfid.getFieldIds().length == 1;
+    TypedFieldId.Builder builder = TypedFieldId.newBuilder();
+    builder.addId(pfid.getFieldIds()[0]);
+    TypedFieldId id =
+        FieldIdUtil.getFieldIdIfMatches(parent, builder, true,
+            SchemaPath.getSimplePath(vv.getField().getName()).getRootSegment());
+    return id;
+  }
+
+  /*
+   * Creates the key column within the parent value vector
+   */
+  private void createNestedKeyColumn(MapVector parent, String name, LogicalExpression expr,
+      List<LogicalExpression> keyExprs, List<TypedFieldId> keyOutputIds)
+          throws SchemaChangeException {
+    LogicalExpression mle = PhysicalOperatorUtil.materializeExpression(expr, incoming, context);
+    TypedFieldId id = createVVFieldId(mle, name, parent);
+    keyExprs.add(mle);
+    keyOutputIds.add(id);
+  }
+
+  /*
+   * Creates the value vector within the parent value vector. The map vector key is
+   * is the column name and value is the statistic expression e.g. "salary" : NDV(emp.salary)
+   */
+  private void addMapVector(String name, MapVector parent, LogicalExpression expr,
+      List<LogicalExpression> valueExprs) throws SchemaChangeException {
+    LogicalExpression mle = PhysicalOperatorUtil.materializeExpression(expr, incoming, context);
+    TypedFieldId id = createVVFieldId(mle, name, parent);
+    valueExprs.add(new ValueVectorWriteExpression(id, mle, true));
+  }
+
+  /*
+   * Generates the code for the statistics aggregate which is subclassed from StreamingAggregator
+   */
+  private StreamingAggregator codegenAggregator(List<LogicalExpression> keyExprs,
+      List<LogicalExpression> valueExprs, List<TypedFieldId> keyOutputIds)
+          throws SchemaChangeException, ClassTransformationException, IOException {
+
+    ClassGenerator<StreamingAggregator> cg = CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getOptions());
+    cg.getCodeGenerator().plainJavaCapable(true);
+    // Uncomment out this line to debug the generated code.
+    // cg.getCodeGenerator().saveCodeForDebugging(true);
+
+    LogicalExpression[] keyExprsArray = new LogicalExpression[keyExprs.size()];
+    LogicalExpression[] valueExprsArray = new LogicalExpression[valueExprs.size()];
+    TypedFieldId[] keyOutputIdsArray = new TypedFieldId[keyOutputIds.size()];
+
+    keyExprs.toArray(keyExprsArray);
+    valueExprs.toArray(valueExprsArray);
+    keyOutputIds.toArray(keyOutputIdsArray);
+
+    setupIsSame(cg, keyExprsArray);
+    setupIsSameApart(cg, keyExprsArray);
+    addRecordValues(cg, valueExprsArray);
+    outputRecordKeys(cg, keyOutputIdsArray, keyExprsArray);
+    outputRecordKeysPrev(cg, keyOutputIdsArray, keyExprsArray);
+
+    cg.getBlock("resetValues")._return(JExpr.TRUE);
+    getIndex(cg);
+
+    container.buildSchema(SelectionVectorMode.NONE);
+    StreamingAggregator agg = context.getImplementationClass(cg);
+    agg.setup(oContext, incoming, this, ValueVector.MAX_ROW_COUNT);
+    return agg;
+  }
+
+  @Override
+  protected StreamingAggregator createAggregatorInternal()
+      throws SchemaChangeException, ClassTransformationException, IOException {
+    List<LogicalExpression> keyExprs = Lists.newArrayList();
+    List<LogicalExpression> valueExprs = Lists.newArrayList();
+    List<TypedFieldId> keyOutputIds = Lists.newArrayList();
+    String [] colMeta = new String [] {Statistic.COLNAME, Statistic.COLTYPE};
+    container.clear();
+    // Generate the `column` map containing the columns in the incoming schema. Ignore
+    // the implicit columns
+    for (String col : colMeta) {
+      MapVector parent = new MapVector(col, oContext.getAllocator(), null);
+      container.add(parent);
+      for (MaterializedField mf : incoming.getSchema()) {
+        LogicalExpression expr;
+        if (col.equals(colMeta[0])) {
+          expr = ValueExpressions.getChar(SchemaPath.getSimplePath(mf.getName()).toString(), 0);
+        } else {
+          expr = ValueExpressions.getChar(DrillStatsTable.getMapper().writeValueAsString(mf.getType()), 0);
+        }
+        // Ignore implicit columns
+        if (!isImplicitFileColumn(mf)) {
+          createNestedKeyColumn(
+              parent,
+              SchemaPath.getSimplePath(mf.getName()).toString(),
+              expr,
+              keyExprs,
+              keyOutputIds
+          );
+        }
+      }
+    }
+    // Iterate over the list of statistics and generate a MAP whose key is the column
+    // and the value is the statistic for the column e.g.
+    // NDV <<"employee_id" : 500>, <"salary" : 10>> represents a MAP of NDVs (# distinct values)
+    // employee NDV = 500, salary NDV = 10
+    for (String func : functions) {
+      MapVector parent = new MapVector(func, oContext.getAllocator(), null);
+      container.add(parent);
+
+      for (MaterializedField mf : incoming.getSchema()) {
+        // Check stats collection is only being done for supported data-types. Complex types
+        // such as MAP, LIST are not supported!
+        if (isColMinorTypeValid(mf) && !isImplicitFileColumn(mf)) {
+          List<LogicalExpression> args = Lists.newArrayList();
+          args.add(SchemaPath.getSimplePath(mf.getName()));
+          LogicalExpression call = FunctionCallFactory.createExpression(func, args);
+          addMapVector(SchemaPath.getSimplePath(mf.getName()).toString(), parent, call, valueExprs);
+        }
+      }
+    }
+    // Now generate the code for the statistics aggregate
+    return codegenAggregator(keyExprs, valueExprs, keyOutputIds);
+  }
+
+  private boolean isColMinorTypeValid(MaterializedField mf) throws UnsupportedOperationException {
+    String mTypeStr = null;
+    if (mf.getType().getMinorType() == TypeProtos.MinorType.GENERIC_OBJECT) {
+      mTypeStr = "GENERIC OBJECT";
+    } else if (mf.getType().getMinorType() == TypeProtos.MinorType.LATE) {
+      mTypeStr = "LATE";
+    }else if (mf.getType().getMinorType() == TypeProtos.MinorType.LIST) {
+      mTypeStr = "LIST";
+    } else if (mf.getType().getMinorType() == TypeProtos.MinorType.MAP) {
+      mTypeStr = "MAP";
+    } else if (mf.getType().getMinorType() == TypeProtos.MinorType.UNION) {
+      mTypeStr = "UNION";
+    }
+    if (mTypeStr != null) {
+      return false;
+      //throw new UnsupportedOperationException(String.format("Column %s has data-type %s which is not supported",
+      //    mf.getName(), mTypeStr));
+    } else {
+      return true;
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatchCreator.java
similarity index 95%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatchCreator.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatchCreator.java
index aba325c..252b738 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StatisticsAggBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsAggBatchCreator.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.impl.aggregate;
+package org.apache.drill.exec.physical.impl.statistics;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import java.util.List;
@@ -26,7 +26,6 @@ import org.apache.drill.exec.physical.impl.BatchCreator;
 import org.apache.drill.exec.record.CloseableRecordBatch;
 import org.apache.drill.exec.record.RecordBatch;
 
-@SuppressWarnings("unused")
 public class StatisticsAggBatchCreator implements BatchCreator<StatisticsAggregate>{
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java
new file mode 100644
index 0000000..95982b7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/StatisticsMergeBatch.java
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.statistics;
+
+import java.util.ArrayList;
+import java.util.GregorianCalendar;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.ValueExpressions;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.PhysicalOperatorUtil;
+import org.apache.drill.exec.physical.config.StatisticsMerge;
+import org.apache.drill.exec.record.AbstractSingleRecordBatch;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.BigIntVector;
+import org.apache.drill.exec.vector.DateVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.complex.MapVector;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+
+/**
+ *
+ * Example input and output:
+ * Schema of incoming batch:
+ *    "columns"       : MAP - Column names
+ *       "region_id"  : VARCHAR
+ *       "sales_city" : VARCHAR
+ *       "cnt"        : VARCHAR
+ *    "statscount" : MAP - Number of entries (rows)
+ *       "region_id"  : BIGINT - statscount(region_id)
+ *                      in incoming batch
+ *       "sales_city" : BIGINT - statscount(sales_city)
+ *       "cnt"        : BIGINT - statscount(cnt)
+ *    "nonnullstatcount" : MAP - Number of non-null entries (rows)
+ *       "region_id"  : BIGINT - nonnullstatcount(region_id)
+ *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
+ *       "cnt"        : BIGINT - nonnullstatcount(cnt)
+ *   .... another map for next stats function ....
+ * Schema of outgoing batch:
+ *    "schema" : BIGINT - Schema number. For each schema change this number is incremented.
+ *    "computed" : DATE - What time is it computed?
+ *    "columns"       : MAP - Column names
+ *       "region_id"  : VARCHAR
+ *       "sales_city" : VARCHAR
+ *       "cnt"        : VARCHAR
+ *    "statscount" : MAP - Number of entries (rows)
+ *       "region_id"  : BIGINT - statscount(region_id) - aggregation over all values of region_id
+ *                      in incoming batch
+ *       "sales_city" : BIGINT - statscount(sales_city)
+ *       "cnt"        : BIGINT - statscount(cnt)
+ *    "nonnullstatcount" : MAP - Number of non-null entries (rows)
+ *       "region_id"  : BIGINT - nonnullstatcount(region_id)
+ *       "sales_city" : BIGINT - nonnullstatcount(sales_city)
+ *       "cnt"        : BIGINT - nonnullstatcount(cnt)
+ *   .... another map for next stats function ....
+ */
+public class StatisticsMergeBatch extends AbstractSingleRecordBatch<StatisticsMerge> {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatisticsMergeBatch.class);
+  private Map<String, String> functions;
+  private boolean first = true;
+  private boolean finished = false;
+  private int schema = 0;
+  private int recordCount = 0;
+  private List<String> columnsList = null;
+  private double samplePercent = 100.0;
+  private List<MergedStatistic> mergedStatisticList = null;
+
+  public StatisticsMergeBatch(StatisticsMerge popConfig, RecordBatch incoming,
+      FragmentContext context) throws OutOfMemoryException {
+    super(popConfig, context, incoming);
+    functions = popConfig.getFunctions();
+    samplePercent = popConfig.getSamplePercent();
+    mergedStatisticList = new ArrayList<>();
+  }
+
+  /*
+   * Creates key columns for the outgoing batch e.g. `schema`, `computed`. These columns are NOT
+   * table columns for which statistics will be computed.
+   */
+  private void createKeyColumn(String name, LogicalExpression expr)
+      throws SchemaChangeException {
+    LogicalExpression mle = PhysicalOperatorUtil.materializeExpression(expr, incoming, context);
+    MaterializedField outputField = MaterializedField.create(name, mle.getMajorType());
+    ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
+    container.add(vector);
+  }
+
+  /*
+   * Adds the `name` column value vector in the `parent` map vector. These `name` columns are
+   * table columns for which statistics will be computed.
+   */
+  private ValueVector addMapVector(String name, MapVector parent, LogicalExpression expr)
+      throws SchemaChangeException {
+    LogicalExpression mle = PhysicalOperatorUtil.materializeExpression(expr, incoming, context);
+    Class<? extends ValueVector> vvc =
+        TypeHelper.getValueVectorClass(mle.getMajorType().getMinorType(),
+            mle.getMajorType().getMode());
+    ValueVector vector = parent.addOrGet(name, mle.getMajorType(), vvc);
+    return vector;
+  }
+
+  /*
+   * Identify the list of fields within a map which are generated by StatisticsMerge. Perform
+   * basic sanity check i.e. all maps have the same number of columns and those columns are
+   * the same in each map
+   */
+  private void buildColumnsList() {
+    Map<String, Boolean> inputFunctions = new HashMap<>();
+    // Prepare map of input functions for verifying only they appear in the incoming batch
+    for (String inputFunc : functions.values()) {
+      inputFunctions.put(inputFunc, false);
+    }
+    List<String> lastMapColumnsList = null;
+    //Populate the columns list from the `columns` map
+    for (VectorWrapper<?> vw : incoming) {
+      String inputFunc = vw.getField().getName();
+      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
+        continue;
+      }
+      if (inputFunctions.get(inputFunc)) {
+        throw new IllegalArgumentException (String.format("The statistic `%s` appears more than once",
+            inputFunc));
+      } else {
+        inputFunctions.put(inputFunc, true);
+      }
+      if (vw.getField().getName().equals(Statistic.COLNAME)) {
+        columnsList = Lists.newArrayList();
+        for (ValueVector vv : vw.getValueVector()) {
+          if (vv.getField().getType().getMinorType() == TypeProtos.MinorType.MAP) {
+            throw new IllegalArgumentException("StatisticsMerge of nested map is not supported");
+          }
+          columnsList.add(vv.getField().getName());
+        }
+        lastMapColumnsList = columnsList;
+      }
+    }
+    // Verify the rest of the maps have the same columns
+    for (VectorWrapper<?> vw : incoming) {
+      String inputFunc = vw.getField().getName();
+      if (vw.getField().getType().getMinorType() != TypeProtos.MinorType.MAP) {
+        continue;
+      }
+      if (!inputFunctions.get(inputFunc)) {
+        throw new IllegalArgumentException (String.format("The statistic `%s` is not expected here",
+            inputFunc));
+      }
+      if (columnsList.size() != lastMapColumnsList.size()
+          || !lastMapColumnsList.containsAll(columnsList)) {
+        // Error!! Maps with different size and/or keys. The map for each statistics (e.g. NDV)
+        // should match exactly with the column map i.e. we did not run into any issues while
+        // generating statistics for all the specified columns
+        throw new IllegalStateException("StatisticsMerge Maps have different fields");
+      }
+    }
+  }
+
+  /* Prepare the outgoing container. Generates the outgoing record batch schema.
+   * Please look at the comments above the class definition which describes the
+   * incoming/outgoing batch schema
+   */
+  private void buildOutputContainer() throws SchemaChangeException {
+    // Populate the list of statistics which will be output in the schema
+    for (VectorWrapper<?> vw : incoming) {
+      for (String outputStatName : functions.keySet()) {
+        if (functions.get(outputStatName).equals(vw.getField().getName())) {
+          mergedStatisticList.add(MergedStatisticFactory.getMergedStatistic(outputStatName,
+              functions.get(outputStatName), samplePercent));
+        }
+      }
+    }
+    // Configure settings/dependencies for statistics, if needed
+    for (MergedStatistic statistic : mergedStatisticList) {
+      if (statistic.getName().equals(Statistic.AVG_WIDTH)) {
+        ((AvgWidthMergedStatistic)statistic).configure(mergedStatisticList);
+      } else if (statistic.getName().equals(Statistic.NDV)) {
+        NDVMergedStatistic.NDVConfiguration config =
+            new NDVMergedStatistic.NDVConfiguration(context.getOptions(),
+                mergedStatisticList);
+        ((NDVMergedStatistic)statistic).configure(config);
+      } else if (statistic.getName().equals(Statistic.SUM_DUPS)) {
+        ((CntDupsMergedStatistic)statistic).configure(mergedStatisticList);
+      } else if (statistic.getName().equals(Statistic.HLL_MERGE)) {
+        ((HLLMergedStatistic)statistic).configure(context.getOptions());
+      }
+    }
+    // Create the schema number and time when computed in the outgoing vector
+    createKeyColumn(Statistic.SCHEMA, ValueExpressions.getBigInt(schema++));
+    GregorianCalendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
+    calendar.setTimeInMillis(System.currentTimeMillis());
+    createKeyColumn(Statistic.COMPUTED, ValueExpressions.getDate(calendar));
+
+    // Create output map vectors corresponding to each statistic (e.g. rowcount)
+    for (MergedStatistic statistic : mergedStatisticList) {
+      String targetTypeStatistic = statistic.getInput();
+      for (VectorWrapper<?> vw : incoming) {
+        if (targetTypeStatistic.equals(vw.getField().getName())) {
+          addVectorToOutgoingContainer(statistic.getName(), vw);
+        }
+      }
+    }
+    container.setRecordCount(0);
+    recordCount = 0;
+    container.buildSchema(incoming.getSchema().getSelectionVectorMode());
+  }
+
+  /* Adds a value vector corresponding to the statistic in the outgoing record batch.
+   * Determines the MajorType based on the incoming value vector. Please look at the
+   * comments above the class definition which describes the incoming/outgoing batch schema
+   */
+  private void addVectorToOutgoingContainer(String outStatName, VectorWrapper vw)
+      throws SchemaChangeException {
+    // Input map vector
+    MapVector inputVector = (MapVector) vw.getValueVector();
+    assert inputVector.getPrimitiveVectors().size() > 0;
+    // Proceed to create output map vector with same name e.g. statcount etc.
+    MajorType mt = inputVector.getField().getType();
+    MaterializedField mf = MaterializedField.create(outStatName, mt);
+
+    ValueVector outputValueVector = TypeHelper.getNewVector(mf, oContext.getAllocator());
+    container.add(outputValueVector);
+    MapVector outputVector = (MapVector) outputValueVector;
+
+    for (ValueVector vv : inputVector) {
+      String columnName = vv.getField().getName();
+      // Except column name, type all the rest are actual statistic functions (rely on
+      // statistic calculation functions).
+      if (outStatName.equals(Statistic.COLNAME)
+          || outStatName.equals(Statistic.COLTYPE)) {
+        outputVector.addOrGet(columnName, vv.getField().getType(), vv.getClass());
+      } else {
+        TypeProtos.MinorType minorType;
+        if (outStatName.equals(Statistic.AVG_WIDTH)) {
+          minorType = TypeProtos.MinorType.FLOAT8;
+        } else if (outStatName.equals(Statistic.HLL_MERGE)) {
+          minorType = TypeProtos.MinorType.VARBINARY;
+        } else {
+          minorType = TypeProtos.MinorType.BIGINT;
+        }
+        Class<? extends ValueVector> vvc =
+                TypeHelper.getValueVectorClass(minorType,
+                        TypeProtos.DataMode.OPTIONAL);
+        outputVector.addOrGet(columnName, Types.optional(minorType), vvc);
+      }
+    }
+  }
+
+  /* Prepare the outgoing container. Populates the outgoing record batch data.
+   * Please look at the comments above the class definition which describes the
+   * incoming/outgoing batch schema
+   */
+  private IterOutcome buildOutgoingRecordBatch() {
+    for (VectorWrapper<?> vw : container) {
+      String outputStatName = vw.getField().getName();
+      // Populate the `schema` and `computed` fields
+      if (outputStatName.equals(Statistic.SCHEMA)) {
+        BigIntVector vv = (BigIntVector) vw.getValueVector();
+        vv.allocateNewSafe();
+        vv.getMutator().setSafe(0, schema);
+      } else if (outputStatName.equals(Statistic.COMPUTED)) {
+        GregorianCalendar cal = new GregorianCalendar(TimeZone.getTimeZone("UTC"));
+        DateVector vv = (DateVector) vw.getValueVector();
+        vv.allocateNewSafe();
+        vv.getMutator().setSafe(0, cal.getTimeInMillis());
+      } else {
+        // Populate the rest of the merged statistics. Each statistic is a map which
+        // contains <COL_NAME, STATS_VALUE> pairs
+        MapVector vv = (MapVector) vw.getValueVector();
+        for (MergedStatistic outputStat : mergedStatisticList) {
+          if (outputStatName.equals(outputStat.getName())) {
+            outputStat.setOutput(vv);
+            vv.getMutator().setValueCount(columnsList.size());
+            break;
+          }
+        }
+      }
+    }
+    ++recordCount;
+    // Populate the number of records (1) inside the outgoing batch.
+    container.setRecordCount(1);
+    return IterOutcome.OK;
+  }
+
+  @Override
+  protected boolean setupNewSchema() throws SchemaChangeException {
+    container.clear();
+    // Generate the list of fields for which statistics will be merged
+    buildColumnsList();
+    // Generate the schema for the outgoing record batch
+    buildOutputContainer();
+    return true;
+  }
+
+  @Override
+  protected IterOutcome doWork() {
+    for (MergedStatistic outputStat : mergedStatisticList) {
+      String inputStat = outputStat.getInput();
+      for (VectorWrapper<?> vw : incoming) {
+        MapVector vv = (MapVector) vw.getValueVector();
+        if (vv.getField().getName().equals(inputStat)) {
+          outputStat.merge(vv);
+          break;
+        }
+      }
+    }
+    return IterOutcome.OK;
+  }
+
+  @Override
+  public VectorContainer getOutgoingContainer() {
+    return this.container;
+  }
+
+  @Override
+  public void dump() {
+
+  }
+
+  @Override
+  public IterOutcome innerNext() {
+    IterOutcome outcome;
+    boolean didSomeWork = false;
+    if (finished) {
+      return IterOutcome.NONE;
+    }
+    try {
+      outer: while (true) {
+        outcome = next(incoming);
+        switch (outcome) {
+          case NONE:
+            break outer;
+          case OUT_OF_MEMORY:
+          case NOT_YET:
+          case STOP:
+            return outcome;
+          case OK_NEW_SCHEMA:
+            if (first) {
+              first = false;
+              if (!setupNewSchema()) {
+                outcome = IterOutcome.OK;
+              }
+              return outcome;
+            }
+            //fall through
+          case OK:
+            assert first == false : "First batch should be OK_NEW_SCHEMA";
+            IterOutcome out = doWork();
+            didSomeWork = true;
+            if (out != IterOutcome.OK) {
+              return out;
+            }
+            break;
+          default:
+            throw new UnsupportedOperationException("Unsupported upstream state " + outcome);
+        }
+      }
+    } catch (SchemaChangeException ex) {
+      kill(false);
+      context.getExecutorState().fail(UserException.unsupportedError(ex).build(logger));
+      return IterOutcome.STOP;
+    }
+
+    // We can only get here if upstream is NONE i.e. no more batches. If we did some work prior to
+    // exhausting all upstream, then return OK. Otherwise, return NONE.
+    if (didSomeWork) {
+      IterOutcome out = buildOutgoingRecordBatch();
+      finished = true;
+      return out;
+    } else {
+      return outcome;
+    }
... 5719 lines suppressed ...