You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by vo...@apache.org on 2019/12/17 12:48:23 UTC

[drill] branch master updated (0acc652 -> 5339fc2)

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

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


    from 0acc652  DRILL-7471: DESCRIBE TABLE command fails with ClassCastException when Metastore is enabled
     new b2e3ba8  DRILL-6332: Allow to provide two-component Kerberos principals
     new 19b5f17  DRILL-7470: Remove conflicting logback-classic dependency in drill-yarn
     new 807a22f  DRILL-7472: Fix ser / de for sys and information_schema schemas queries
     new d27221f  DRILL-7473: Parquet reader failed to get field of repeated map
     new 64bb383  DRILL-7474: Reduce size of Drill's tar.gz file
     new 79ea7b1  DRILL-7476: Set lastSet on TransferPair copies
     new 2e8619c  DRILL-7481: Fix raw type warnings in Iceberg Metastore and related classes
     new 005f4d9  DRILL-7479: Partial fixes for metadata parameterized type issues
     new 435b6cb  DRILL-7482: Fix missing artifact and overlapping classes warnings in Drill build
     new d4c8f35  DRILL-7483: Add support for 12 and 13 java versions
     new 68dd10a  DRILL-7484: Malware found in the Drill test folder
     new 5339fc2  DRILL-7486: Refactor row set reader builders

The 12 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:
 .circleci/config.yml                               | 345 ++++++++++-----------
 .../java/org/apache/drill/common/KerberosUtil.java |  27 +-
 contrib/format-excel/pom.xml                       |  15 -
 .../apache/drill/exec/hive/HiveClusterTest.java    |  18 +-
 .../org/apache/drill/exec/hive/HiveTestBase.java   |  31 +-
 .../apache/drill/exec/hive/HiveTestUtilities.java  |  22 ++
 .../exec/hive/complex_types/TestHiveArrays.java    |   6 +-
 .../exec/hive/complex_types/TestHiveMaps.java      |   6 +-
 .../exec/hive/complex_types/TestHiveStructs.java   |   6 +-
 .../exec/hive/complex_types/TestHiveUnions.java    |   6 +-
 .../hive/BaseTestHiveImpersonation.java            |   7 +
 .../exec/sql/hive/TestViewSupportOnHiveTables.java |   6 +-
 contrib/storage-hive/hive-exec-shade/pom.xml       |   5 +-
 contrib/storage-mongo/pom.xml                      |   2 +-
 distribution/pom.xml                               |  33 ++
 drill-yarn/pom.xml                                 |   8 -
 .../org/apache/drill/exec/client/DrillClient.java  |   6 +-
 .../apache/drill/exec/compile/MergeAdapter.java    |  94 +++---
 .../drill/exec/expr/ComparisonPredicate.java       |  14 +-
 .../apache/drill/exec/expr/EvaluationVisitor.java  |   7 -
 .../org/apache/drill/exec/expr/IsPredicate.java    |   9 +-
 .../apache/drill/exec/expr/StatisticsProvider.java |  18 +-
 .../MetastoreParquetTableMetadataProvider.java     |   8 +-
 .../metastore/SimpleFileTableMetadataProvider.java |   8 +-
 .../base/AbstractGroupScanWithMetadata.java        |  10 +-
 .../apache/drill/exec/physical/base/GroupScan.java |   2 +-
 .../exec/physical/impl/limit/LimitRecordBatch.java |   5 +-
 .../impl/metadata/MetadataAggregateHelper.java     |   3 +-
 .../impl/metadata/MetadataControllerBatch.java     |  50 ++-
 .../impl/metadata/MetadataHandlerBatch.java        |   7 +-
 .../unorderedreceiver/UnorderedReceiverBatch.java  |   2 +-
 .../physical/impl/validate/BatchValidator.java     |  71 +++--
 .../physical/resultSet/impl/WriterIndexImpl.java   |   2 +-
 ...stractReaderBuilder.java => ReaderBuilder.java} |  29 +-
 ...eReaderBuilder.java => HyperReaderBuilder.java} |  49 +--
 .../resultSet/model/single/DirectRowIndex.java     |   5 +-
 ...ReaderBuilder.java => SimpleReaderBuilder.java} |  50 ++-
 .../exec/physical/rowSet/AbstractSingleRowSet.java |  19 +-
 .../drill/exec/physical/rowSet/DirectRowSet.java   |   2 +-
 .../HyperRowIndex.java}                            |  27 +-
 .../exec/physical/rowSet/HyperRowSetImpl.java      |  15 +-
 .../IndirectRowIndex.java}                         |  24 +-
 .../drill/exec/physical/rowSet/IndirectRowSet.java |  35 +--
 .../exec/physical/rowSet/RowSetFormatter.java      |   5 -
 .../drill/exec/planner/common/DrillStatsTable.java |  12 +-
 .../planner/cost/DrillRelMdDistinctRowCount.java   |   2 +-
 .../exec/planner/cost/DrillRelMdSelectivity.java   |   9 +-
 .../exec/planner/index/FunctionalIndexHelper.java  |   3 +-
 .../drill/exec/record/FragmentWritableBatch.java   |  80 +++--
 .../apache/drill/exec/record/WritableBatch.java    |   7 +-
 .../apache/drill/exec/server/BootStrapContext.java |   7 +-
 .../exec/store/ischema/InfoSchemaGroupScan.java    |  20 +-
 .../InfoSchemaPushFilterIntoRecordGenerator.java   |   5 +-
 .../drill/exec/store/ischema/RecordCollector.java  |   2 +-
 .../apache/drill/exec/store/ischema/Records.java   |   4 +-
 .../store/parquet/AbstractParquetGroupScan.java    |   4 +-
 .../parquet/AbstractParquetScanBatchCreator.java   |   6 +-
 .../store/parquet/BaseParquetMetadataProvider.java |  17 +-
 .../exec/store/parquet/FilterEvaluatorUtils.java   |  13 +-
 .../drill/exec/store/parquet/ParquetGroupScan.java |   9 +-
 .../store/parquet/ParquetGroupScanStatistics.java  |  13 +-
 .../exec/store/parquet/ParquetPushDownFilter.java  |   8 +-
 .../store/parquet/ParquetTableMetadataUtils.java   |  31 +-
 .../drill/exec/store/sys/SystemTableScan.java      |  11 +-
 .../drill/exec/vector/complex/FieldIdUtil.java     |   1 +
 .../drill/exec/work/fragment/FragmentExecutor.java |  95 ++++--
 .../test/java/org/apache/drill/PlanTestBase.java   |  10 +
 .../rest/spnego/TestDrillSpnegoAuthenticator.java  |  48 ++-
 .../org/apache/drill/exec/sql/TestInfoSchema.java  |  28 +-
 .../exec/sql/TestInfoSchemaWithMetastore.java      |  15 +-
 .../drill/exec/sql/TestMetastoreCommands.java      |  47 +--
 .../ischema/TestInfoSchemaFilterPushDown.java      |  20 +-
 .../exec/store/parquet/TestFileGenerator.java      |  29 --
 .../exec/store/parquet/TestParquetComplex.java     |  15 +
 .../store/parquet/TestParquetFilterPushDown.java   |  24 +-
 .../drill/exec/store/pcap/TestSessionizePCAP.java  | 168 +++++++---
 .../drill/exec/store/sys/TestSystemTable.java      |  10 +
 .../java/org/apache/drill/test/QueryBuilder.java   |  20 +-
 .../java/org/apache/drill/test/TestBuilder.java    |   5 +
 .../map/parquet/repeated_struct_with_dict.parquet  | Bin 0 -> 646 bytes
 .../test/resources/store/pcap/attack-trace.pcap    | Bin 189103 -> 0 bytes
 .../test/resources/store/pcap/dataFromRemote.txt   |   1 +
 .../src/test/resources/store/pcap/http.pcap        | Bin 0 -> 25803 bytes
 exec/jdbc-all/pom.xml                              |  10 +
 .../drill/exec/memory/BoundsCheckingTest.java      |  37 +--
 .../codegen/templates/NullableValueVectors.java    |   9 +
 metastore/iceberg-metastore/pom.xml                |  12 +-
 .../iceberg/schema/IcebergTableSchema.java         |   2 +-
 .../iceberg/transform/FilterExpressionVisitor.java |  16 +-
 .../iceberg/schema/TestIcebergTableSchema.java     | 241 ++++++++------
 .../components/tables/BasicTablesRequests.java     |   2 +-
 .../components/tables/BasicTablesTransformer.java  |   3 +
 .../components/tables/TableMetadataUnit.java       |   2 +
 .../metastore/exceptions/MetastoreException.java   |   6 +-
 .../metastore/expressions/FilterExpression.java    |  16 +-
 .../drill/metastore/metadata/BaseMetadata.java     |  28 +-
 .../metastore/metadata/BaseTableMetadata.java      |  12 +-
 .../apache/drill/metastore/metadata/Metadata.java  |   6 +-
 .../metadata/NonInterestingColumnsMetadata.java    |  13 +-
 .../drill/metastore/metadata/TableMetadata.java    |   3 +-
 .../CollectableColumnStatisticsKind.java           |   2 +-
 .../metastore/statistics/ColumnStatistics.java     |  30 +-
 .../metastore/statistics/ColumnStatisticsKind.java |  31 +-
 .../metastore/statistics/StatisticsHolder.java     |  12 +-
 .../drill/metastore/util/TableMetadataUtils.java   |  44 ++-
 .../tables/TestTableMetadataUnitConversion.java    |   4 +-
 .../metastore/metadata/MetadataSerDeTest.java      |  16 +-
 pom.xml                                            |  42 ++-
 tools/fmpp/pom.xml                                 |  10 +-
 109 files changed, 1439 insertions(+), 1088 deletions(-)
 copy drill-yarn/src/main/java/org/apache/drill/yarn/appMaster/YarnFacadeException.java => contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveClusterTest.java (73%)
 rename exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/{AbstractReaderBuilder.java => ReaderBuilder.java} (60%)
 rename exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/hyper/{BaseReaderBuilder.java => HyperReaderBuilder.java} (85%)
 rename exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/{BaseReaderBuilder.java => SimpleReaderBuilder.java} (81%)
 copy exec/java-exec/src/main/java/org/apache/drill/exec/physical/{resultSet/model/single/DirectRowIndex.java => rowSet/HyperRowIndex.java} (58%)
 copy exec/java-exec/src/main/java/org/apache/drill/exec/physical/{resultSet/model/single/DirectRowIndex.java => rowSet/IndirectRowIndex.java} (57%)
 create mode 100644 exec/java-exec/src/test/resources/store/parquet/complex/map/parquet/repeated_struct_with_dict.parquet
 delete mode 100644 exec/java-exec/src/test/resources/store/pcap/attack-trace.pcap
 create mode 100644 exec/java-exec/src/test/resources/store/pcap/dataFromRemote.txt
 create mode 100644 exec/java-exec/src/test/resources/store/pcap/http.pcap


[drill] 02/12: DRILL-7470: Remove conflicting logback-classic dependency in drill-yarn

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

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

commit 19b5f17df6769b70d9f9ed833b131400217d298e
Author: Anton Gozhiy <an...@gmail.com>
AuthorDate: Fri Dec 6 19:32:50 2019 +0200

    DRILL-7470: Remove conflicting logback-classic dependency in drill-yarn
    
    closes #1920
---
 drill-yarn/pom.xml | 8 --------
 1 file changed, 8 deletions(-)

diff --git a/drill-yarn/pom.xml b/drill-yarn/pom.xml
index 57d2073..454c2f8 100644
--- a/drill-yarn/pom.xml
+++ b/drill-yarn/pom.xml
@@ -125,14 +125,6 @@
       <artifactId>config</artifactId>
     </dependency>
 
-    <!-- Logging -->
-    <dependency>
-      <groupId>ch.qos.logback</groupId>
-      <artifactId>logback-classic</artifactId>
-      <version>1.0.13</version>
-      <scope>test</scope>
-    </dependency>
-
     <!--  Testing -->
     <dependency>
       <groupId>org.apache.drill</groupId>


[drill] 05/12: DRILL-7474: Reduce size of Drill's tar.gz file

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

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

commit 64bb383bc1551e8d1bf1999f7ab19125a41d38fd
Author: Anton Gozhiy <an...@gmail.com>
AuthorDate: Tue Dec 10 20:35:13 2019 +0200

    DRILL-7474: Reduce size of Drill's tar.gz file
    
    - Excluded aws-java-sdk-bundle dependency, only required aws libraries added instead.
    - Cleared format-excel module from unused dependencies.
    
    closes #1926
---
 contrib/format-excel/pom.xml | 15 ---------------
 distribution/pom.xml         | 33 +++++++++++++++++++++++++++++++++
 pom.xml                      | 12 ------------
 3 files changed, 33 insertions(+), 27 deletions(-)

diff --git a/contrib/format-excel/pom.xml b/contrib/format-excel/pom.xml
index 0b6d1db..1ee182a 100644
--- a/contrib/format-excel/pom.xml
+++ b/contrib/format-excel/pom.xml
@@ -50,21 +50,6 @@
       <artifactId>poi-ooxml</artifactId>
       <version>${poi.version}</version>
     </dependency>
-    <dependency>
-      <groupId>org.apache.poi</groupId>
-      <artifactId>poi-ooxml-schemas</artifactId>
-      <version>${poi.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.poi</groupId>
-      <artifactId>poi-scratchpad</artifactId>
-      <version>${poi.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.poi</groupId>
-      <artifactId>ooxml-schemas</artifactId>
-      <version>1.4</version>
-    </dependency>
     <!-- Test dependencies -->
     <dependency>
       <groupId>org.apache.drill.exec</groupId>
diff --git a/distribution/pom.xml b/distribution/pom.xml
index 2c80d4b..485922a 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -30,6 +30,9 @@
   <packaging>pom</packaging>
   <name>Packaging and Distribution Assembly</name>
 
+  <properties>
+    <aws.java.sdk.version>1.11.375</aws.java.sdk.version>
+  </properties>
 
   <dependencies>
     <dependency>
@@ -106,9 +109,39 @@
           <groupId>commons-codec</groupId>
           <artifactId>commons-codec</artifactId>
         </exclusion>
+        <exclusion>
+          <groupId>com.amazonaws</groupId>
+          <artifactId>aws-java-sdk-bundle</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>com.amazonaws</groupId>
+      <artifactId>aws-java-sdk-core</artifactId>
+      <version>${aws.java.sdk.version}</version>
+      <exclusions>
+        <exclusion>
+          <artifactId>commons-logging</artifactId>
+          <groupId>commons-logging</groupId>
+        </exclusion>
       </exclusions>
     </dependency>
     <dependency>
+      <groupId>com.amazonaws</groupId>
+      <artifactId>aws-java-sdk-dynamodb</artifactId>
+      <version>${aws.java.sdk.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.amazonaws</groupId>
+      <artifactId>aws-java-sdk-sts</artifactId>
+      <version>${aws.java.sdk.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.amazonaws</groupId>
+      <artifactId>aws-java-sdk-s3</artifactId>
+      <version>${aws.java.sdk.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-client</artifactId>
       <exclusions>
diff --git a/pom.xml b/pom.xml
index 8b2c132..3b90251 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1194,18 +1194,6 @@
       </dependency>
       <dependency>
         <groupId>org.apache.hadoop</groupId>
-        <artifactId>hadoop-aws</artifactId>
-        <version>${hadoop.version}</version>
-        <scope>test</scope>
-        <exclusions>
-          <exclusion>
-            <groupId>commons-logging</groupId>
-            <artifactId>commons-logging</artifactId>
-          </exclusion>
-        </exclusions>
-      </dependency>
-      <dependency>
-        <groupId>org.apache.hadoop</groupId>
         <artifactId>hadoop-yarn-api</artifactId>
         <version>${hadoop.version}</version>
         <scope>test</scope>


[drill] 03/12: DRILL-7472: Fix ser / de for sys and information_schema schemas queries

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

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

commit 807a22f6920e550649be9c1a7f836324eacbe9e9
Author: Arina Ielchiieva <ar...@gmail.com>
AuthorDate: Wed Dec 11 17:26:48 2019 +0200

    DRILL-7472: Fix ser / de for sys and information_schema schemas queries
    
    closes #1925
---
 .../apache/drill/exec/physical/base/GroupScan.java |  2 +-
 .../exec/store/ischema/InfoSchemaGroupScan.java    | 20 ++++++++--------
 .../InfoSchemaPushFilterIntoRecordGenerator.java   |  5 ++--
 .../drill/exec/store/sys/SystemTableScan.java      | 11 +++++----
 .../test/java/org/apache/drill/PlanTestBase.java   | 10 ++++++++
 .../org/apache/drill/exec/sql/TestInfoSchema.java  | 28 +++++++++++++++-------
 .../ischema/TestInfoSchemaFilterPushDown.java      | 20 ++++++++--------
 .../drill/exec/store/sys/TestSystemTable.java      | 10 ++++++++
 .../java/org/apache/drill/test/TestBuilder.java    |  5 ++++
 9 files changed, 75 insertions(+), 36 deletions(-)

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 72c26e9..40ce22f 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
@@ -37,7 +37,7 @@ import org.apache.hadoop.fs.Path;
 
 /**
  * A GroupScan operator represents all data which will be scanned by a given physical
- * plan.  It is the superset of all SubScans for the plan.
+ * plan. It is the superset of all SubScans for the plan.
  */
 public interface GroupScan extends Scan, HasAffinity {
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaGroupScan.java
index 7b44a4d..8ae95cf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaGroupScan.java
@@ -39,8 +39,6 @@ public class InfoSchemaGroupScan extends AbstractGroupScan {
   private final InfoSchemaTableType table;
   private final InfoSchemaFilter filter;
 
-  private boolean isFilterPushedDown = false;
-
   public InfoSchemaGroupScan(InfoSchemaTableType table) {
     this(table, null);
   }
@@ -57,7 +55,6 @@ public class InfoSchemaGroupScan extends AbstractGroupScan {
     super(that);
     this.table = that.table;
     this.filter = that.filter;
-    this.isFilterPushedDown = that.isFilterPushedDown;
   }
 
   @JsonProperty("table")
@@ -70,6 +67,12 @@ public class InfoSchemaGroupScan extends AbstractGroupScan {
     return filter;
   }
 
+  @JsonIgnore
+  @Override
+  public List<SchemaPath> getColumns() {
+    return super.getColumns();
+  }
+
   @Override
   public void applyAssignments(List<DrillbitEndpoint> endpoints) {
     Preconditions.checkArgument(endpoints.size() == 1);
@@ -81,7 +84,8 @@ public class InfoSchemaGroupScan extends AbstractGroupScan {
     return new InfoSchemaSubScan(table, filter);
   }
 
-  public ScanStats getScanStats(){
+  @Override
+  public ScanStats getScanStats() {
     if (filter == null) {
       return ScanStats.TRIVIAL_TABLE;
     } else {
@@ -103,7 +107,7 @@ public class InfoSchemaGroupScan extends AbstractGroupScan {
 
   @Override
   public String getDigest() {
-    return this.table.toString() + ", filter=" + filter;
+    return table.toString() + ", filter=" + filter;
   }
 
   @Override
@@ -111,12 +115,8 @@ public class InfoSchemaGroupScan extends AbstractGroupScan {
     return new InfoSchemaGroupScan(this);
   }
 
-  public void setFilterPushedDown(boolean status) {
-    this.isFilterPushedDown = status;
-  }
-
   @JsonIgnore
   public boolean isFilterPushedDown() {
-    return isFilterPushedDown;
+    return filter != null;
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaPushFilterIntoRecordGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaPushFilterIntoRecordGenerator.java
index 2ce0cc1..e53f7fb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaPushFilterIntoRecordGenerator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaPushFilterIntoRecordGenerator.java
@@ -80,7 +80,7 @@ public abstract class InfoSchemaPushFilterIntoRecordGenerator extends StoragePlu
   }
 
   protected void doMatch(RelOptRuleCall call, ScanPrel scan, ProjectPrel project, FilterPrel filter) {
-    final RexNode condition = filter.getCondition();
+    RexNode condition = filter.getCondition();
 
     InfoSchemaGroupScan groupScan = (InfoSchemaGroupScan) scan.getGroupScan();
     if (groupScan.isFilterPushedDown()) {
@@ -95,8 +95,7 @@ public abstract class InfoSchemaPushFilterIntoRecordGenerator extends StoragePlu
       return; //no filter push down ==> No transformation.
     }
 
-    final InfoSchemaGroupScan newGroupsScan = new InfoSchemaGroupScan(groupScan.getTable(), infoSchemaFilter);
-    newGroupsScan.setFilterPushedDown(true);
+    InfoSchemaGroupScan newGroupsScan = new InfoSchemaGroupScan(groupScan.getTable(), infoSchemaFilter);
 
     RelNode input = new ScanPrel(scan.getCluster(), filter.getTraitSet(), newGroupsScan, scan.getRowType(), scan.getTable());
     if (project != null) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableScan.java
index 787e243..50d65a2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableScan.java
@@ -42,7 +42,6 @@ import org.apache.drill.exec.store.StoragePluginRegistry;
 
 @JsonTypeName("sys")
 public class SystemTableScan extends AbstractGroupScan implements SubScan {
-  // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SystemTableScan.class);
 
   private final SystemTable table;
   private final SystemTablePlugin plugin;
@@ -51,7 +50,6 @@ public class SystemTableScan extends AbstractGroupScan implements SubScan {
   @JsonCreator
   public SystemTableScan(@JsonProperty("table") SystemTable table,
                          @JsonProperty("maxRecordsToRead") int maxRecordsToRead,
-                         @JsonProperty("columns") List<SchemaPath> columns,
                          @JacksonInject StoragePluginRegistry engineRegistry) throws ExecutionSetupException {
     this(table, maxRecordsToRead, (SystemTablePlugin) engineRegistry.getPlugin(SystemTablePluginConfig.INSTANCE));
   }
@@ -85,6 +83,12 @@ public class SystemTableScan extends AbstractGroupScan implements SubScan {
   public void applyAssignments(List<DrillbitEndpoint> endpoints) {
   }
 
+  @JsonIgnore
+  @Override
+  public List<SchemaPath> getColumns() {
+    return super.getColumns();
+  }
+
   @Override
   public SubScan getSpecificScan(int minorFragmentId) {
     return this;
@@ -178,7 +182,7 @@ public class SystemTableScan extends AbstractGroupScan implements SubScan {
 
   @Override
   public GroupScan applyLimit(int maxRecords) {
-    if (this.maxRecordsToRead == maxRecords) {
+    if (maxRecordsToRead == maxRecords) {
       return null;
     }
     return clone(this, maxRecords);
@@ -196,5 +200,4 @@ public class SystemTableScan extends AbstractGroupScan implements SubScan {
   public SystemTablePlugin getPlugin() {
     return plugin;
   }
-
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
index cba86ed..a2764c2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
@@ -72,6 +72,16 @@ public class PlanTestBase extends BaseTestQuery {
   }
 
   /**
+   * Converts given sql query into physical JSON plan representation.
+   *
+   * @param sql sql query
+   * @return physical plan in JSON format
+   */
+  public static String getPhysicalJsonPlan(String sql) throws Exception {
+    return getPlanInString("EXPLAIN PLAN for " + sql, JSON_FORMAT);
+  }
+
+  /**
    * Runs an explain plan query and check for expected regex patterns (in optiq
    * text format), also ensure excluded patterns are not found. Either list can
    * be empty or null to skip that part of the check.
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java
index f55ee1a..509809c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.sql;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.PlanTestBase;
 import org.apache.drill.categories.SqlTest;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.record.RecordBatchLoader;
@@ -435,25 +436,25 @@ public class TestInfoSchema extends BaseTestQuery {
 
   @Test
   public void describeSchemaOutput() throws Exception {
-    final List<QueryDataBatch> result = testSqlWithResults("describe schema dfs.tmp");
+    List<QueryDataBatch> result = testSqlWithResults("describe schema dfs.tmp");
     assertEquals(1, result.size());
-    final QueryDataBatch batch = result.get(0);
-    final RecordBatchLoader loader = new RecordBatchLoader(getDrillbitContext().getAllocator());
+    QueryDataBatch batch = result.get(0);
+    RecordBatchLoader loader = new RecordBatchLoader(getDrillbitContext().getAllocator());
     loader.load(batch.getHeader().getDef(), batch.getData());
 
     // check schema column value
-    final VectorWrapper schemaValueVector = loader.getValueAccessorById(
+    VectorWrapper<?> schemaValueVector = loader.getValueAccessorById(
         NullableVarCharVector.class,
         loader.getValueVectorId(SchemaPath.getCompoundPath("schema")).getFieldIds());
     String schema = schemaValueVector.getValueVector().getAccessor().getObject(0).toString();
     assertEquals("dfs.tmp", schema);
 
     // check properties column value
-    final VectorWrapper propertiesValueVector = loader.getValueAccessorById(
+    VectorWrapper<?> propertiesValueVector = loader.getValueAccessorById(
         NullableVarCharVector.class,
         loader.getValueVectorId(SchemaPath.getCompoundPath("properties")).getFieldIds());
     String properties = propertiesValueVector.getValueVector().getAccessor().getObject(0).toString();
-    final Map configMap = mapper.readValue(properties, Map.class);
+    Map<?, ?> configMap = mapper.readValue(properties, Map.class);
 
     // check some stable properties existence
     assertTrue(configMap.containsKey("connection"));
@@ -464,8 +465,8 @@ public class TestInfoSchema extends BaseTestQuery {
     // check some stable properties values
     assertEquals("file", configMap.get("type"));
 
-    final FileSystemConfig testConfig = (FileSystemConfig) bits[0].getContext().getStorage().getPlugin("dfs").getConfig();
-    final String tmpSchemaLocation = testConfig.getWorkspaces().get("tmp").getLocation();
+    FileSystemConfig testConfig = (FileSystemConfig) bits[0].getContext().getStorage().getPlugin("dfs").getConfig();
+    String tmpSchemaLocation = testConfig.getWorkspaces().get("tmp").getLocation();
     assertEquals(tmpSchemaLocation, configMap.get("location"));
 
     batch.release();
@@ -483,4 +484,15 @@ public class TestInfoSchema extends BaseTestQuery {
     test("desc information_schema.`catalogs`");
     test("desc table information_schema.`catalogs`");
   }
+
+  @Test
+  public void testSerDe() throws Exception {
+    String sql = "select * from information_schema.`tables` where table_name = 'schemata' order by 1";
+
+    testBuilder()
+      .sqlQuery(sql)
+      .unOrdered()
+      .physicalPlanBaseline(PlanTestBase.getPhysicalJsonPlan(sql))
+      .go();
+  }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestInfoSchemaFilterPushDown.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestInfoSchemaFilterPushDown.java
index a25bd1f..04a7635 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestInfoSchemaFilterPushDown.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestInfoSchemaFilterPushDown.java
@@ -25,7 +25,7 @@ public class TestInfoSchemaFilterPushDown extends PlanTestBase {
   @Test
   public void testFilterPushdown_Equal() throws Exception {
     final String query = "SELECT * FROM INFORMATION_SCHEMA.`TABLES` WHERE TABLE_SCHEMA='INFORMATION_SCHEMA'";
-    final String scan = "Scan.*groupscan=\\[TABLES, filter=equal\\(Field=TABLE_SCHEMA,Literal=INFORMATION_SCHEMA\\)\\]";
+    final String scan = "Scan.*groupscan=\\[TABLES, filter=equal\\(Field=TABLE_SCHEMA,Literal=INFORMATION_SCHEMA\\)";
 
     testHelper(query, scan, false);
   }
@@ -33,7 +33,7 @@ public class TestInfoSchemaFilterPushDown extends PlanTestBase {
   @Test
   public void testFilterPushdown_NonEqual() throws Exception {
     final String query = "SELECT * FROM INFORMATION_SCHEMA.`TABLES` WHERE TABLE_SCHEMA <> 'INFORMATION_SCHEMA'";
-    final String scan = "Scan.*groupscan=\\[TABLES, filter=not_equal\\(Field=TABLE_SCHEMA,Literal=INFORMATION_SCHEMA\\)\\]";
+    final String scan = "Scan.*groupscan=\\[TABLES, filter=not_equal\\(Field=TABLE_SCHEMA,Literal=INFORMATION_SCHEMA\\)";
 
     testHelper(query, scan, false);
   }
@@ -41,7 +41,7 @@ public class TestInfoSchemaFilterPushDown extends PlanTestBase {
   @Test
   public void testFilterPushdown_Like() throws Exception {
     final String query = "SELECT * FROM INFORMATION_SCHEMA.`TABLES` WHERE TABLE_SCHEMA LIKE '%SCH%'";
-    final String scan = "Scan.*groupscan=\\[TABLES, filter=like\\(Field=TABLE_SCHEMA,Literal=%SCH%\\)\\]";
+    final String scan = "Scan.*groupscan=\\[TABLES, filter=like\\(Field=TABLE_SCHEMA,Literal=%SCH%\\)";
 
     testHelper(query, scan, false);
   }
@@ -49,7 +49,7 @@ public class TestInfoSchemaFilterPushDown extends PlanTestBase {
   @Test
   public void testFilterPushdown_LikeWithEscape() throws Exception {
     final String query = "SELECT * FROM INFORMATION_SCHEMA.`TABLES` WHERE TABLE_SCHEMA LIKE '%\\\\SCH%' ESCAPE '\\'";
-    final String scan = "Scan.*groupscan=\\[TABLES, filter=like\\(Field=TABLE_SCHEMA,Literal=%\\\\\\\\SCH%,Literal=\\\\\\)\\]";
+    final String scan = "Scan.*groupscan=\\[TABLES, filter=like\\(Field=TABLE_SCHEMA,Literal=%\\\\\\\\SCH%,Literal=\\\\\\)";
 
     testHelper(query, scan, false);
   }
@@ -60,7 +60,7 @@ public class TestInfoSchemaFilterPushDown extends PlanTestBase {
         "TABLE_SCHEMA = 'sys' AND " +
         "TABLE_NAME <> 'version'";
     final String scan = "Scan.*groupscan=\\[COLUMNS, filter=booleanand\\(equal\\(Field=TABLE_SCHEMA,Literal=sys\\)," +
-        "not_equal\\(Field=TABLE_NAME,Literal=version\\)\\)\\]";
+        "not_equal\\(Field=TABLE_NAME,Literal=version\\)\\)";
 
     testHelper(query, scan, false);
   }
@@ -72,7 +72,7 @@ public class TestInfoSchemaFilterPushDown extends PlanTestBase {
         "TABLE_NAME <> 'version' OR " +
         "TABLE_SCHEMA like '%sdfgjk%'";
     final String scan = "Scan.*groupscan=\\[COLUMNS, filter=booleanor\\(equal\\(Field=TABLE_SCHEMA,Literal=sys\\)," +
-        "not_equal\\(Field=TABLE_NAME,Literal=version\\),like\\(Field=TABLE_SCHEMA,Literal=%sdfgjk%\\)\\)\\]";
+        "not_equal\\(Field=TABLE_NAME,Literal=version\\),like\\(Field=TABLE_SCHEMA,Literal=%sdfgjk%\\)\\)";
 
     testHelper(query, scan, false);
   }
@@ -80,21 +80,21 @@ public class TestInfoSchemaFilterPushDown extends PlanTestBase {
   @Test
   public void testFilterPushDownWithProject_Equal() throws Exception {
     final String query = "SELECT COLUMN_NAME from INFORMATION_SCHEMA.`COLUMNS` WHERE TABLE_SCHEMA = 'INFORMATION_SCHEMA'";
-    final String scan = "Scan.*groupscan=\\[COLUMNS, filter=equal\\(Field=TABLE_SCHEMA,Literal=INFORMATION_SCHEMA\\)\\]";
+    final String scan = "Scan.*groupscan=\\[COLUMNS, filter=equal\\(Field=TABLE_SCHEMA,Literal=INFORMATION_SCHEMA\\)";
     testHelper(query, scan, false);
   }
 
   @Test
   public void testFilterPushDownWithProject_NotEqual() throws Exception {
     final String query = "SELECT COLUMN_NAME from INFORMATION_SCHEMA.`COLUMNS` WHERE TABLE_NAME <> 'TABLES'";
-    final String scan = "Scan.*groupscan=\\[COLUMNS, filter=not_equal\\(Field=TABLE_NAME,Literal=TABLES\\)\\]";
+    final String scan = "Scan.*groupscan=\\[COLUMNS, filter=not_equal\\(Field=TABLE_NAME,Literal=TABLES\\)";
     testHelper(query, scan, false);
   }
 
   @Test
   public void testFilterPushDownWithProject_Like() throws Exception {
     final String query = "SELECT COLUMN_NAME from INFORMATION_SCHEMA.`COLUMNS` WHERE TABLE_NAME LIKE '%BL%'";
-    final String scan = "Scan.*groupscan=\\[COLUMNS, filter=like\\(Field=TABLE_NAME,Literal=%BL%\\)\\]";
+    final String scan = "Scan.*groupscan=\\[COLUMNS, filter=like\\(Field=TABLE_NAME,Literal=%BL%\\)";
     testHelper(query, scan, false);
   }
 
@@ -107,7 +107,7 @@ public class TestInfoSchemaFilterPushDown extends PlanTestBase {
         "IS_NULLABLE = 'YES'"; // this is not expected to pushdown into scan
     final String scan = "Scan.*groupscan=\\[COLUMNS, " +
         "filter=booleanand\\(equal\\(Field=TABLE_SCHEMA,Literal=sys\\),equal\\(Field=TABLE_NAME,Literal=version\\)," +
-        "like\\(Field=COLUMN_NAME,Literal=commit%s\\)\\)\\]";
+        "like\\(Field=COLUMN_NAME,Literal=commit%s\\)\\)";
 
     testHelper(query, scan, true);
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestSystemTable.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestSystemTable.java
index 2ed5f6e..0fbd242 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestSystemTable.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestSystemTable.java
@@ -62,6 +62,16 @@ public class TestSystemTable extends PlanTestBase {
   }
 
   @Test
+  public void testSerDe() throws Exception {
+    String sql = "select * from sys.functions where name = 'avg' limit 100";
+    testBuilder()
+      .sqlQuery(sql)
+      .unOrdered()
+      .physicalPlanBaseline(PlanTestBase.getPhysicalJsonPlan(sql))
+      .go();
+  }
+
+  @Test
   public void threadsTable() throws Exception {
     test("select * from sys.threads");
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
index 0b88670..029d259 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
@@ -441,6 +441,11 @@ public class TestBuilder {
         baselineTypeMap, baselineOptionSettingQueries, testOptionSettingQueries, highPerformanceComparison, expectedNumBatches);
   }
 
+  public BaselineQueryTestBuilder physicalPlanBaseline(String physicalPlan) {
+    return new BaselineQueryTestBuilder(physicalPlan, UserBitShared.QueryType.PHYSICAL, services, query, queryType, ordered, approximateEquality,
+      baselineTypeMap, baselineOptionSettingQueries, testOptionSettingQueries, highPerformanceComparison, expectedNumBatches);
+  }
+
   private String getDecimalPrecisionScaleInfo(TypeProtos.MajorType type) {
     String precision = "";
     switch(type.getMinorType()) {


[drill] 06/12: DRILL-7476: Set lastSet on TransferPair copies

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

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

commit 79ea7b1f13c4864bfc9a25049b25b3369fce07cc
Author: Paul Rogers <pa...@yahoo.com>
AuthorDate: Tue Dec 10 20:15:32 2019 -0800

    DRILL-7476: Set lastSet on TransferPair copies
    
    Variable-width nullable vectors maintain a "lastSet" field
    in the mutator. This field is used in "fill empties" logic
    when setting the vector's value count. This is true even
    if the vector is read-only, or has been transferred from
    another (read-only) vector. LastSet must be set to the
    row count or the code will helpfully overwrite existing
    offsets with 0.
    
    closes #1922
---
 .../org/apache/drill/exec/client/DrillClient.java  |  6 +-
 .../exec/physical/impl/limit/LimitRecordBatch.java |  5 +-
 .../unorderedreceiver/UnorderedReceiverBatch.java  |  2 +-
 .../physical/impl/validate/BatchValidator.java     | 43 +++++++++-
 .../drill/exec/record/FragmentWritableBatch.java   | 80 ++++++++++--------
 .../apache/drill/exec/record/WritableBatch.java    |  7 +-
 .../drill/exec/work/fragment/FragmentExecutor.java | 95 ++++++++++++++--------
 .../java/org/apache/drill/test/QueryBuilder.java   | 20 ++---
 .../codegen/templates/NullableValueVectors.java    |  9 ++
 9 files changed, 181 insertions(+), 86 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
index 7dc4d59..237aba1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
@@ -112,8 +112,8 @@ public class DrillClient implements Closeable, ConnectionThrottle {
   private volatile ClusterCoordinator clusterCoordinator;
   private volatile boolean connected = false;
   private final BufferAllocator allocator;
-  private int reconnectTimes;
-  private int reconnectDelay;
+  private final int reconnectTimes;
+  private final int reconnectDelay;
   private boolean supportComplexTypes;
   private final boolean ownsZkConnection;
   private final boolean ownsAllocator;
@@ -862,7 +862,7 @@ public class DrillClient implements Closeable, ConnectionThrottle {
 
     @Override
     public void dataArrived(QueryDataBatch result, ConnectionThrottle throttle) {
-      logger.debug("Result arrived:  Result: {}", result );
+      logger.debug("Result arrived:  Result: {}", result);
       results.add(result);
     }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
index e9d7dd3..25c79ac 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
@@ -41,7 +41,7 @@ import static org.apache.drill.exec.record.RecordBatch.IterOutcome.NONE;
 public class LimitRecordBatch extends AbstractSingleRecordBatch<Limit> {
   private static final Logger logger = LoggerFactory.getLogger(LimitRecordBatch.class);
 
-  private SelectionVector2 outgoingSv;
+  private final SelectionVector2 outgoingSv;
   private SelectionVector2 incomingSv;
 
   // Start offset of the records
@@ -234,7 +234,8 @@ public class LimitRecordBatch extends AbstractSingleRecordBatch<Limit> {
     outgoingSv.setRecordCount(svIndex);
     outgoingSv.setBatchActualRecordCount(inputRecordCount);
     // Actual number of values in the container; not the number in
-    // the SV.
+    // the SV. Set record count, not value count. Value count is
+    // carried over from input vectors.
     container.setRecordCount(inputRecordCount);
     // Update the start offset
     recordStartOffset = 0;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
index d40bd6d..f5fb77a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
@@ -206,7 +206,7 @@ public class UnorderedReceiverBatch implements CloseableRecordBatch {
       stats.addLongStat(Metric.BYTES_RECEIVED, batch.getByteCount());
 
       batch.release();
-      if(schemaChanged) {
+      if (schemaChanged) {
         this.schema = batchLoader.getSchema();
         stats.batchReceived(0, rbd.getRecordCount(), true);
         lastOutcome = IterOutcome.OK_NEW_SCHEMA;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java
index 36d9c8f..2c657e6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java
@@ -25,6 +25,10 @@ import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.vector.BitVector;
 import org.apache.drill.exec.vector.FixedWidthVector;
+import org.apache.drill.exec.vector.NullableVar16CharVector;
+import org.apache.drill.exec.vector.NullableVarBinaryVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.NullableVarDecimalVector;
 import org.apache.drill.exec.vector.NullableVector;
 import org.apache.drill.exec.vector.RepeatedBitVector;
 import org.apache.drill.exec.vector.UInt1Vector;
@@ -321,10 +325,38 @@ public class BatchValidator {
           "Outer value count = %d, but inner value count = %d",
           outerCount, valueCount));
     }
+    int lastSet = getLastSet(vector);
+    if (lastSet != -2) {
+      if (lastSet != valueCount - 1) {
+        error(name, vector, String.format(
+            "Value count = %d, but last set = %d",
+            valueCount, lastSet));
+      }
+    }
     verifyIsSetVector(vector, (UInt1Vector) vector.getBitsVector());
     validateVector(name + "-values", valuesVector);
   }
 
+  // getLastSet() is visible per vector type, not on a super class.
+  // There is no common nullable, variable width super class.
+
+  private int getLastSet(NullableVector vector) {
+    if (vector instanceof NullableVarCharVector) {
+      return ((NullableVarCharVector) vector).getMutator().getLastSet();
+    }
+    if (vector instanceof NullableVarBinaryVector) {
+      return ((NullableVarBinaryVector) vector).getMutator().getLastSet();
+    }
+    if (vector instanceof NullableVarDecimalVector) {
+      return ((NullableVarDecimalVector) vector).getMutator().getLastSet();
+    }
+    if (vector instanceof NullableVar16CharVector) {
+      return ((NullableVar16CharVector) vector).getMutator().getLastSet();
+    }
+    // Otherwise, return a value that is never legal for lastSet
+    return -2;
+  }
+
   private void validateVarCharVector(String name, VarCharVector vector) {
     int dataLength = vector.getBuffer().writerIndex();
     validateVarWidthVector(name, vector, dataLength);
@@ -332,7 +364,12 @@ public class BatchValidator {
 
   private void validateVarBinaryVector(String name, VarBinaryVector vector) {
     int dataLength = vector.getBuffer().writerIndex();
-    validateVarWidthVector(name, vector, dataLength);
+    int lastOffset = validateVarWidthVector(name, vector, dataLength);
+    if (lastOffset != dataLength) {
+      error(name, vector, String.format(
+          "Data vector has length %d, but offset vector has largest offset %d",
+          dataLength, lastOffset));
+    }
   }
 
   private void validateVarDecimalVector(String name, VarDecimalVector vector) {
@@ -340,9 +377,9 @@ public class BatchValidator {
     validateVarWidthVector(name, vector, dataLength);
   }
 
-  private void validateVarWidthVector(String name, VariableWidthVector vector, int dataLength) {
+  private int validateVarWidthVector(String name, VariableWidthVector vector, int dataLength) {
     int valueCount = vector.getAccessor().getValueCount();
-    validateOffsetVector(name + "-offsets", vector.getOffsetVector(),
+    return validateOffsetVector(name + "-offsets", vector.getOffsetVector(),
         valueCount, dataLength);
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java
index 5606d75..bbb988a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java
@@ -23,25 +23,36 @@ import org.apache.drill.exec.proto.BitData.FragmentRecordBatch;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
 
-public class FragmentWritableBatch{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentWritableBatch.class);
+public class FragmentWritableBatch {
 
   private static RecordBatchDef EMPTY_DEF = RecordBatchDef.newBuilder().setRecordCount(0).build();
 
   private final ByteBuf[] buffers;
   private final FragmentRecordBatch header;
 
-  public FragmentWritableBatch(final boolean isLast, final QueryId queryId, final int sendMajorFragmentId, final int sendMinorFragmentId, final int receiveMajorFragmentId, final int receiveMinorFragmentId, final WritableBatch batch){
-    this(isLast, queryId, sendMajorFragmentId, sendMinorFragmentId, receiveMajorFragmentId, new int[]{receiveMinorFragmentId}, batch.getDef(), batch.getBuffers());
+  public FragmentWritableBatch(boolean isLast, QueryId queryId,
+      int sendMajorFragmentId, int sendMinorFragmentId,
+      int receiveMajorFragmentId, int receiveMinorFragmentId,
+      WritableBatch batch) {
+    this(isLast, queryId, sendMajorFragmentId, sendMinorFragmentId, receiveMajorFragmentId,
+        new int[]{receiveMinorFragmentId}, batch.getDef(), batch.getBuffers());
   }
 
-  public FragmentWritableBatch(final boolean isLast, final QueryId queryId, final int sendMajorFragmentId, final int sendMinorFragmentId, final int receiveMajorFragmentId, final int[] receiveMinorFragmentIds, final WritableBatch batch){
-    this(isLast, queryId, sendMajorFragmentId, sendMinorFragmentId, receiveMajorFragmentId, receiveMinorFragmentIds, batch.getDef(), batch.getBuffers());
+  public FragmentWritableBatch(boolean isLast, QueryId queryId,
+      int sendMajorFragmentId, int sendMinorFragmentId,
+      int receiveMajorFragmentId, int[] receiveMinorFragmentIds,
+      WritableBatch batch) {
+    this(isLast, queryId, sendMajorFragmentId, sendMinorFragmentId,
+        receiveMajorFragmentId, receiveMinorFragmentIds, batch.getDef(),
+        batch.getBuffers());
   }
 
-  private FragmentWritableBatch(final boolean isLast, final QueryId queryId, final int sendMajorFragmentId, final int sendMinorFragmentId, final int receiveMajorFragmentId, final int[] receiveMinorFragmentId, final RecordBatchDef def, final ByteBuf... buffers){
+  private FragmentWritableBatch(boolean isLast, QueryId queryId,
+      int sendMajorFragmentId, int sendMinorFragmentId,
+      int receiveMajorFragmentId, int[] receiveMinorFragmentId,
+      RecordBatchDef def, ByteBuf... buffers) {
     this.buffers = buffers;
-    final FragmentRecordBatch.Builder builder = FragmentRecordBatch.newBuilder()
+    FragmentRecordBatch.Builder builder = FragmentRecordBatch.newBuilder()
         .setIsLastBatch(isLast)
         .setDef(def)
         .setQueryId(queryId)
@@ -49,49 +60,60 @@ public class FragmentWritableBatch{
         .setSendingMajorFragmentId(sendMajorFragmentId)
         .setSendingMinorFragmentId(sendMinorFragmentId);
 
-    for(final int i : receiveMinorFragmentId){
-      builder.addReceivingMinorFragmentId(i);
+    for (int fragmentId : receiveMinorFragmentId) {
+      builder.addReceivingMinorFragmentId(fragmentId);
     }
 
     this.header = builder.build();
   }
 
-
-  public static FragmentWritableBatch getEmptyLast(final QueryId queryId, final int sendMajorFragmentId, final int sendMinorFragmentId, final int receiveMajorFragmentId, final int receiveMinorFragmentId){
-    return getEmptyLast(queryId, sendMajorFragmentId, sendMinorFragmentId, receiveMajorFragmentId, new int[]{receiveMinorFragmentId});
+  public static FragmentWritableBatch getEmptyLast(QueryId queryId,
+      int sendMajorFragmentId, int sendMinorFragmentId,
+      int receiveMajorFragmentId, int receiveMinorFragmentId) {
+    return getEmptyLast(queryId, sendMajorFragmentId, sendMinorFragmentId,
+        receiveMajorFragmentId, new int[]{receiveMinorFragmentId});
   }
 
-  public static FragmentWritableBatch getEmptyLast(final QueryId queryId, final int sendMajorFragmentId, final int sendMinorFragmentId, final int receiveMajorFragmentId, final int[] receiveMinorFragmentIds){
-    return new FragmentWritableBatch(true, queryId, sendMajorFragmentId, sendMinorFragmentId, receiveMajorFragmentId, receiveMinorFragmentIds, EMPTY_DEF);
+  public static FragmentWritableBatch getEmptyLast(QueryId queryId,
+      int sendMajorFragmentId, int sendMinorFragmentId,
+      int receiveMajorFragmentId, int[] receiveMinorFragmentIds) {
+    return new FragmentWritableBatch(true, queryId, sendMajorFragmentId,
+        sendMinorFragmentId, receiveMajorFragmentId, receiveMinorFragmentIds,
+        EMPTY_DEF);
   }
 
-
-  public static FragmentWritableBatch getEmptyLastWithSchema(final QueryId queryId, final int sendMajorFragmentId, final int sendMinorFragmentId,
-                                                             final int receiveMajorFragmentId, final int receiveMinorFragmentId, final BatchSchema schema){
-    return getEmptyBatchWithSchema(true, queryId, sendMajorFragmentId, sendMinorFragmentId, receiveMajorFragmentId,
+  public static FragmentWritableBatch getEmptyLastWithSchema(
+      QueryId queryId, int sendMajorFragmentId,
+      int sendMinorFragmentId, int receiveMajorFragmentId,
+      int receiveMinorFragmentId, BatchSchema schema) {
+    return getEmptyBatchWithSchema(true, queryId, sendMajorFragmentId,
+        sendMinorFragmentId, receiveMajorFragmentId,
         receiveMinorFragmentId, schema);
   }
 
-  public static FragmentWritableBatch getEmptyBatchWithSchema(final boolean isLast, final QueryId queryId, final int sendMajorFragmentId,
-      final int sendMinorFragmentId, final int receiveMajorFragmentId, final int receiveMinorFragmentId, final BatchSchema schema){
+  public static FragmentWritableBatch getEmptyBatchWithSchema(
+      boolean isLast, QueryId queryId, int sendMajorFragmentId,
+      int sendMinorFragmentId, int receiveMajorFragmentId,
+      int receiveMinorFragmentId, BatchSchema schema) {
 
-    final RecordBatchDef.Builder def = RecordBatchDef.newBuilder();
+    RecordBatchDef.Builder def = RecordBatchDef.newBuilder();
     if (schema != null) {
-      for (final MaterializedField field : schema) {
+      for (MaterializedField field : schema) {
         def.addField(field.getSerializedField());
       }
     }
-    return new FragmentWritableBatch(isLast, queryId, sendMajorFragmentId, sendMinorFragmentId, receiveMajorFragmentId,
+    return new FragmentWritableBatch(isLast, queryId, sendMajorFragmentId,
+        sendMinorFragmentId, receiveMajorFragmentId,
         new int[] { receiveMinorFragmentId }, def.build());
   }
 
-  public ByteBuf[] getBuffers(){
+  public ByteBuf[] getBuffers() {
     return buffers;
   }
 
   public long getByteCount() {
     long n = 0;
-    for (final ByteBuf buf : buffers) {
+    for (ByteBuf buf : buffers) {
       n += buf.readableBytes();
     }
     return n;
@@ -99,11 +121,5 @@ public class FragmentWritableBatch{
 
   public FragmentRecordBatch getHeader() {
     return header;
-
   }
-
-
-
-
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
index 577517d..966ade7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
@@ -163,8 +163,11 @@ public class WritableBatch implements AutoCloseable {
       vv.clear();
     }
 
-    RecordBatchDef batchDef = RecordBatchDef.newBuilder().addAllField(metadata).setRecordCount(recordCount)
-        .setCarriesTwoByteSelectionVector(isSV2).build();
+    RecordBatchDef batchDef = RecordBatchDef.newBuilder()
+        .addAllField(metadata)
+        .setRecordCount(recordCount)
+        .setCarriesTwoByteSelectionVector(isSV2)
+        .build();
     WritableBatch b = new WritableBatch(batchDef, buffers);
     return b;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
index 3e4d94a..79884e8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
@@ -48,60 +48,89 @@ import org.apache.drill.exec.testing.ControlsInjectorFactory;
 import org.apache.drill.exec.util.ImpersonationUtil;
 import org.apache.drill.exec.work.foreman.DrillbitStatusListener;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static org.apache.drill.exec.server.FailureUtils.EXIT_CODE_HEAP_OOM;
 
 /**
  * <h2>Overview</h2>
  * <p>
- *   Responsible for running a single fragment on a single Drillbit. Listens/responds to status request and cancellation messages.
+ * Responsible for running a single fragment on a single Drillbit.
+ * Listens/responds to status request and cancellation messages.
  * </p>
  * <h2>Theory of Operation</h2>
  * <p>
- *  The {@link FragmentExecutor} runs a fragment's {@link RootExec} in the {@link FragmentExecutor#run()} method in a single thread. While a fragment is running
- *  it may be subject to termination requests. The {@link FragmentExecutor} is reponsible for gracefully handling termination requests for the {@link RootExec}. There
- *  are two types of termination messages:
- *  <ol>
- *    <li><b>Cancellation Request:</b> This signals that the fragment and therefore the {@link RootExec} need to terminate immediately.</li>
- *    <li><b>Receiver Finished:</b> This signals that a downstream receiver no longer needs anymore data. A fragment may recieve multiple receiver finished requests
- *    (one for each downstream receiver). The {@link RootExec} will only terminate once it has recieved {@link FragmentExecutor.EventType#RECEIVER_FINISHED} messages
- *    for all downstream receivers.</li>
- *  </ol>
+ * The {@link FragmentExecutor} runs a fragment's {@link RootExec} in the
+ * {@link FragmentExecutor#run()} method in a single thread. While a fragment is
+ * running it may be subject to termination requests. The
+ * {@link FragmentExecutor} is responsible for gracefully handling termination
+ * requests for the {@link RootExec}. There are two types of termination
+ * messages:
+ * <ol>
+ * <li><b>Cancellation Request:</b> This signals that the fragment and therefore
+ * the {@link RootExec} need to terminate immediately.</li>
+ * <li><b>Receiver Finished:</b> This signals that a downstream receiver no
+ * longer needs anymore data. A fragment may receive multiple receiver finished
+ * requests (one for each downstream receiver). The {@link RootExec} will only
+ * terminate once it has received
+ * {@link FragmentExecutor.EventType#RECEIVER_FINISHED} messages for all
+ * downstream receivers.</li>
+ * </ol>
  * </p>
  * <p>
- *   The {@link FragmentExecutor} processes termination requests appropriately for the {@link RootExec}. A <b>Cancellation Request</b> is signalled when
- *   {@link FragmentExecutor#cancel()} is called. A <b>Receiver Finished</b> event is signalled when {@link FragmentExecutor#receivingFragmentFinished(FragmentHandle)} is
- *   called. The way in which these signals are handled is the following:
+ * The {@link FragmentExecutor} processes termination requests appropriately for
+ * the {@link RootExec}. A <b>Cancellation Request</b> is signaled when
+ * {@link FragmentExecutor#cancel()} is called. A <b>Receiver Finished</b> event
+ * is signaled when
+ * {@link FragmentExecutor#receivingFragmentFinished(FragmentHandle)} is called.
+ * The way in which these signals are handled is the following:
  * </p>
  * <h3>Cancellation Request</h3>
  * <p>
- *   There are two ways in which a cancellation request can be handled when {@link FragmentExecutor#cancel()} is called.
- *   <ol>
- *     <li>The Cancellation Request is recieved before the {@link RootExec} for the fragment is even started. In this case we can cleanup resources allocated for the fragment
- *     and never start a {@link RootExec}</li>
- *     <li>The Cancellation Request is recieve after the {@link RootExec} for the fragment is started. In this the cancellation request is sent to the
- *     {@link FragmentEventProcessor}. If this is not the first cancellation request it is ignored. If this is the first cancellation request the {@link RootExec} for this
- *     fragment is terminated by interrupting it. Then the {@link FragmentExecutor#run()} thread proceeds to cleanup resources normally</li>
- *   </ol>
+ * There are two ways in which a cancellation request can be handled when
+ * {@link FragmentExecutor#cancel()} is called.
+ * <ol>
+ * <li>The Cancellation Request is received before the {@link RootExec} for the
+ * fragment is even started. In this case we can cleanup resources allocated for
+ * the fragment and never start a {@link RootExec}</li>
+ * <li>The Cancellation Request is receive after the {@link RootExec} for the
+ * fragment is started. In this the cancellation request is sent to the
+ * {@link FragmentEventProcessor}. If this is not the first cancellation request
+ * it is ignored. If this is the first cancellation request the {@link RootExec}
+ * for this fragment is terminated by interrupting it. Then the
+ * {@link FragmentExecutor#run()} thread proceeds to cleanup resources
+ * normally</li>
+ * </ol>
  * </p>
  * <h3>Receiver Finished</h3>
  * <p>
- *  When {@link FragmentExecutor#receivingFragmentFinished(FragmentHandle)} is called, the message is passed to the {@link FragmentEventProcessor} if we
- *  did not already recieve a Cancellation request. Then the finished message is queued in {@link FragmentExecutor#receiverFinishedQueue}. The {@link FragmentExecutor#run()} polls
- *  {@link FragmentExecutor#receiverFinishedQueue} and singlas the {@link RootExec} with {@link RootExec#receivingFragmentFinished(FragmentHandle)} appropriately.
+ * When {@link FragmentExecutor#receivingFragmentFinished(FragmentHandle)} is
+ * called, the message is passed to the {@link FragmentEventProcessor} if we did
+ * not already receive a Cancellation request. Then the finished message is
+ * queued in {@link FragmentExecutor#receiverFinishedQueue}. The
+ * {@link FragmentExecutor#run()} polls
+ * {@link FragmentExecutor#receiverFinishedQueue} and signals the
+ * {@link RootExec} with
+ * {@link RootExec#receivingFragmentFinished(FragmentHandle)} appropriately.
  * </p>
- * <h2>Possible Design Flaws / Poorly Defined Behavoir</h2>
+ * <h2>Possible Design Flaws / Poorly Defined Behavior</h2>
  * <p>
- *   There are still a few aspects of the {@link FragmentExecutor} design that are not clear.
- *   <ol>
- *     <li>If we get a <b>Receiver Finished</b> message for one downstream receiver, will we eventually get one from every downstream receiver?</li>
- *     <li>What happens when we process a <b>Receiver Finished</b> message for some (but not all) downstream receivers and then we cancel the fragment?</li>
- *     <li>What happens when we process a <b>Receiver Finished</b> message for some (but not all) downstream receivers and then we run out of data from the upstream?</li>
- *   </ol>
+ * There are still a few aspects of the {@link FragmentExecutor} design that are
+ * not clear.
+ * <ol>
+ * <li>If we get a <b>Receiver Finished</b> message for one downstream receiver,
+ * will we eventually get one from every downstream receiver?</li>
+ * <li>What happens when we process a <b>Receiver Finished</b> message for some
+ * (but not all) downstream receivers and then we cancel the fragment?</li>
+ * <li>What happens when we process a <b>Receiver Finished</b> message for some
+ * (but not all) downstream receivers and then we run out of data from the
+ * upstream?</li>
+ * </ol>
  * </p>
  */
 public class FragmentExecutor implements Runnable {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentExecutor.class);
+  private static final Logger logger = LoggerFactory.getLogger(FragmentExecutor.class);
   private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(FragmentExecutor.class);
 
   private final String fragmentName;
@@ -542,7 +571,7 @@ public class FragmentExecutor implements Runnable {
    * This is especially important as fragments can take longer to start
    */
   private class FragmentEventProcessor extends EventProcessor<FragmentEvent> {
-    private AtomicBoolean terminate = new AtomicBoolean(false);
+    private final AtomicBoolean terminate = new AtomicBoolean(false);
 
     void cancel() {
       sendEvent(new FragmentEvent(EventType.CANCEL, null));
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
index 6c7176b..2352d0a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/QueryBuilder.java
@@ -86,7 +86,7 @@ public class QueryBuilder {
     private QueryId queryId;
     private int recordCount;
     private int batchCount;
-    private long startTime;
+    private final long startTime;
 
     public SummaryOnlyQueryEventListener(QuerySummaryFuture future) {
       this.future = future;
@@ -132,7 +132,7 @@ public class QueryBuilder {
      * launched the query.
      */
 
-    private CountDownLatch lock = new CountDownLatch(1);
+    private final CountDownLatch lock = new CountDownLatch(1);
     private QuerySummary summary;
 
     /**
@@ -373,7 +373,7 @@ public class QueryBuilder {
 
     // Unload the batch and convert to a row set.
 
-    final RecordBatchLoader loader = new RecordBatchLoader(client.allocator());
+    RecordBatchLoader loader = new RecordBatchLoader(client.allocator());
     try {
       loader.load(resultBatch.getHeader().getDef(), resultBatch.getData());
       resultBatch.release();
@@ -760,18 +760,18 @@ public class QueryBuilder {
    */
   private String queryPlan(String columnName) throws Exception {
     Preconditions.checkArgument(queryType == QueryType.SQL, "Can only explain an SQL query.");
-    final List<QueryDataBatch> results = results();
-    final RecordBatchLoader loader = new RecordBatchLoader(client.allocator());
-    final StringBuilder builder = new StringBuilder();
+    List<QueryDataBatch> results = results();
+    RecordBatchLoader loader = new RecordBatchLoader(client.allocator());
+    StringBuilder builder = new StringBuilder();
 
-    for (final QueryDataBatch b : results) {
+    for (QueryDataBatch b : results) {
       if (!b.hasData()) {
         continue;
       }
 
       loader.load(b.getHeader().getDef(), b.getData());
 
-      final VectorWrapper<?> vw;
+      VectorWrapper<?> vw;
       try {
           vw = loader.getValueAccessorById(
               NullableVarCharVector.class,
@@ -780,9 +780,9 @@ public class QueryBuilder {
         throw new IllegalStateException("Looks like you did not provide an explain plan query, please add EXPLAIN PLAN FOR to the beginning of your query.");
       }
 
-      final ValueVector vv = vw.getValueVector();
+      ValueVector vv = vw.getValueVector();
       for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
-        final Object o = vv.getAccessor().getObject(i);
+        Object o = vv.getAccessor().getObject(i);
         builder.append(o);
       }
       loader.clear();
diff --git a/exec/vector/src/main/codegen/templates/NullableValueVectors.java b/exec/vector/src/main/codegen/templates/NullableValueVectors.java
index e32ecc9..d92cb5d 100644
--- a/exec/vector/src/main/codegen/templates/NullableValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/NullableValueVectors.java
@@ -299,6 +299,15 @@ public final class ${className} extends BaseDataValueVector implements <#if type
     int bitsLength = bitsField.getBufferLength();
     SerializedField valuesField = metadata.getChild(1);
     values.load(valuesField, buffer.slice(bitsLength, capacity - bitsLength));
+    <#if type.major == "VarLen">
+
+    // Though a loaded vector should be read only,
+    // it can have its values set such as when copying
+    // with transfer pairs. Since lastSet is used when
+    // setting values, it must be set on vector load.
+
+    mutator.lastSet = accessor.getValueCount() - 1;
+    </#if>
   }
 
   @Override


[drill] 04/12: DRILL-7473: Parquet reader failed to get field of repeated map

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

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

commit d27221fd7d060acc1e069db2597ca1b1727f0fa3
Author: Bohdan Kazydub <bo...@gmail.com>
AuthorDate: Fri Dec 13 16:27:07 2019 +0200

    DRILL-7473: Parquet reader failed to get field of repeated map
    
    closes #1933
---
 .../apache/drill/exec/expr/EvaluationVisitor.java  |   7 ------
 .../physical/impl/validate/BatchValidator.java     |  24 +++++++--------------
 .../exec/planner/index/FunctionalIndexHelper.java  |   3 +--
 .../drill/exec/vector/complex/FieldIdUtil.java     |   1 +
 .../exec/store/parquet/TestParquetComplex.java     |  15 +++++++++++++
 .../map/parquet/repeated_struct_with_dict.parquet  | Bin 0 -> 646 bytes
 6 files changed, 25 insertions(+), 25 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
index c3478f6..d8742ff 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
@@ -595,13 +595,6 @@ public class EvaluationVisitor {
 
         if (complex || repeated) {
 
-          if (e.getFieldId().isDict(depth)) {
-            JVar dictReader = generator.declareClassField("dictReader", generator.getModel()._ref(FieldReader.class));
-            eval.assign(dictReader, expr);
-
-            return new HoldingContainer(e.getMajorType(), dictReader, null, null, false, true);
-          }
-
           JVar complexReader = generator.declareClassField("reader", generator.getModel()._ref(FieldReader.class));
 
           if (isNullReaderLikely) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java
index e1ffd7a..36d9c8f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java
@@ -35,11 +35,10 @@ import org.apache.drill.exec.vector.VarCharVector;
 import org.apache.drill.exec.vector.VarDecimalVector;
 import org.apache.drill.exec.vector.VariableWidthVector;
 import org.apache.drill.exec.vector.ZeroVector;
+import org.apache.drill.exec.vector.complex.AbstractRepeatedMapVector;
 import org.apache.drill.exec.vector.complex.BaseRepeatedValueVector;
-import org.apache.drill.exec.vector.complex.DictVector;
 import org.apache.drill.exec.vector.complex.MapVector;
 import org.apache.drill.exec.vector.complex.RepeatedListVector;
-import org.apache.drill.exec.vector.complex.RepeatedMapVector;
 import org.apache.drill.exec.vector.complex.UnionVector;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -293,14 +292,16 @@ public class BatchValidator {
       // structure to check.
     } else if (vector instanceof BaseRepeatedValueVector) {
       validateRepeatedVector(name, (BaseRepeatedValueVector) vector);
-    } else if (vector instanceof RepeatedMapVector) {
-      validateRepeatedMapVector(name, (RepeatedMapVector) vector);
+    } else if (vector instanceof AbstractRepeatedMapVector) { // RepeatedMapVector or DictVector
+      // In case of DictVector, keys and values vectors are not validated explicitly to avoid NPE
+      // when keys and values vectors are not set. This happens when output dict vector's keys and
+      // values are not constructed while copying values from input reader to dict writer and the
+      // input reader is an instance of NullReader for all rows which does not have schema.
+      validateRepeatedMapVector(name, (AbstractRepeatedMapVector) vector);
     } else if (vector instanceof MapVector) {
       validateMapVector(name, (MapVector) vector);
     } else if (vector instanceof RepeatedListVector) {
       validateRepeatedListVector(name, (RepeatedListVector) vector);
-    } else if (vector instanceof DictVector) {
-      validateDictVector(name, (DictVector) vector);
     } else if (vector instanceof UnionVector) {
       validateUnionVector(name, (UnionVector) vector);
     } else if (vector instanceof VarDecimalVector) {
@@ -397,8 +398,7 @@ public class BatchValidator {
     }
   }
 
-  private void validateRepeatedMapVector(String name,
-      RepeatedMapVector vector) {
+  private void validateRepeatedMapVector(String name, AbstractRepeatedMapVector vector) {
     int valueCount = vector.getAccessor().getValueCount();
     int elementCount = validateOffsetVector(name + "-offsets",
         vector.getOffsetVector(), valueCount, Integer.MAX_VALUE);
@@ -407,14 +407,6 @@ public class BatchValidator {
     }
   }
 
-  private void validateDictVector(String name, DictVector vector) {
-    int valueCount = vector.getAccessor().getValueCount();
-    int elementCount = validateOffsetVector(name + "-offsets",
-        vector.getOffsetVector(), valueCount, Integer.MAX_VALUE);
-    validateVector(elementCount, vector.getKeys());
-    validateVector(elementCount, vector.getValues());
-  }
-
   private void validateRepeatedListVector(String name,
       RepeatedListVector vector) {
     int valueCount = vector.getAccessor().getValueCount();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/FunctionalIndexHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/FunctionalIndexHelper.java
index 3ff81b4..41de5dd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/FunctionalIndexHelper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/index/FunctionalIndexHelper.java
@@ -163,8 +163,7 @@ public class FunctionalIndexHelper {
       SchemaPath path = SchemaPath.parseFromString(f.getName());
       rowfields.add(new RelDataTypeFieldImpl(
           path.getRootSegmentPath(), rowfields.size(),
-          typeFactory.createMapType(typeFactory.createSqlType(SqlTypeName.VARCHAR),
-              typeFactory.createSqlType(SqlTypeName.ANY))
+          typeFactory.createSqlType(SqlTypeName.ANY)
       ));
       columns.add(path);
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/FieldIdUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/FieldIdUtil.java
index 09e2cfc..0b6e0ee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/FieldIdUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/FieldIdUtil.java
@@ -85,6 +85,7 @@ public class FieldIdUtil {
       }
       // skip the first array segment as there is no corresponding child vector.
       seg = seg.getChild();
+      depth++;
 
       // multi-level numbered access to a repeated map is not possible so return if the next part is also an array
       // segment.
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java
index 075f644..f5b9cd7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetComplex.java
@@ -835,4 +835,19 @@ public class TestParquetComplex extends BaseTestQuery {
         .baselineValues(2, TestBuilder.mapOfObject("a", 1, "b", 2, "c", 3))
         .go();
   }
+
+  @Test // DRILL-7473
+  public void testDictInRepeatedMap() throws Exception {
+    String query = "select struct_array[1].d as d from cp.`store/parquet/complex/map/parquet/repeated_struct_with_dict.parquet`";
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("d")
+        .baselineValuesForSingleColumn(
+            TestBuilder.mapOfObject(1, "a", 2, "b", 3, "c"),
+            TestBuilder.mapOfObject(),
+            TestBuilder.mapOfObject(1, "a", 2, "b")
+        )
+        .go();
+  }
 }
diff --git a/exec/java-exec/src/test/resources/store/parquet/complex/map/parquet/repeated_struct_with_dict.parquet b/exec/java-exec/src/test/resources/store/parquet/complex/map/parquet/repeated_struct_with_dict.parquet
new file mode 100644
index 0000000..8e1a8b6
Binary files /dev/null and b/exec/java-exec/src/test/resources/store/parquet/complex/map/parquet/repeated_struct_with_dict.parquet differ


[drill] 10/12: DRILL-7483: Add support for 12 and 13 java versions

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

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

commit d4c8f35f508e3e8e517a69975028bec3c7975b25
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Wed Dec 11 15:11:31 2019 +0200

    DRILL-7483: Add support for 12 and 13 java versions
    
    closes #1935
---
 .circleci/config.yml                               | 345 ++++++++++-----------
 .../apache/drill/exec/hive/HiveClusterTest.java    |  32 ++
 .../org/apache/drill/exec/hive/HiveTestBase.java   |  31 +-
 .../apache/drill/exec/hive/HiveTestUtilities.java  |  22 ++
 .../exec/hive/complex_types/TestHiveArrays.java    |   6 +-
 .../exec/hive/complex_types/TestHiveMaps.java      |   6 +-
 .../exec/hive/complex_types/TestHiveStructs.java   |   6 +-
 .../exec/hive/complex_types/TestHiveUnions.java    |   6 +-
 .../hive/BaseTestHiveImpersonation.java            |   7 +
 .../exec/sql/hive/TestViewSupportOnHiveTables.java |   6 +-
 contrib/storage-mongo/pom.xml                      |   2 +-
 .../apache/drill/exec/compile/MergeAdapter.java    |  94 +++---
 .../impl/metadata/MetadataAggregateHelper.java     |   3 +-
 .../physical/impl/validate/BatchValidator.java     |   4 +
 .../rest/spnego/TestDrillSpnegoAuthenticator.java  |  48 ++-
 .../exec/sql/TestInfoSchemaWithMetastore.java      |   5 +-
 .../drill/exec/sql/TestMetastoreCommands.java      |   3 +-
 .../drill/exec/memory/BoundsCheckingTest.java      |  37 +--
 metastore/iceberg-metastore/pom.xml                |  12 +-
 .../iceberg/schema/TestIcebergTableSchema.java     | 241 ++++++++------
 pom.xml                                            |  19 +-
 tools/fmpp/pom.xml                                 |  10 +-
 22 files changed, 523 insertions(+), 422 deletions(-)

diff --git a/.circleci/config.yml b/.circleci/config.yml
index 31cb2cc..71d424f 100644
--- a/.circleci/config.yml
+++ b/.circleci/config.yml
@@ -25,45 +25,46 @@ jobs:
     working_directory: ~/drill
 
     steps:
-    - checkout
-    - restore_cache:
-        keys:
-        - m2-{{ checksum "pom.xml" }}
-        - m2- # used if checksum fails
-    - run:
-        name: Update maven version
-        # TODO: Could be removed, once Machine Executor image is updated https://github.com/circleci/image-builder/issues/140
-        # and the possibility of specifying Maven version is added https://github.com/circleci/image-builder/issues/143
-        command:
-          curl -fsSL https://git.io/vpDIf | bash -s -- 3.6.0
-    - run:
-        name: Update packages list
-        command:
-          sudo apt-get update
-    - run:
-        name: Install libaio1.so library for MySQL integration tests
-        command:
-          sudo apt-get install libaio1 libaio-dev
-    - run:
-        name: Drill project build
-        # TODO: 2. Optimizing Maven Builds on CircleCI - https://circleci.com/blog/optimizing-maven-builds-on-circleci/
-        # TODO: 3. Resolving memory issues without "SlowTest" and "UnlikelyTest" excludedGroups in the build
-        command: >
-          mvn install -Drat.skip=false -Dlicense.skip=false -DmemoryMb=2560 -DdirectMemoryMb=4608
-          -DexcludedGroups="org.apache.drill.categories.SlowTest,org.apache.drill.categories.UnlikelyTest" --batch-mode
-    - run:
-        name: Save test results
-        command: |
-          mkdir -p ~/test-results/junit/
-          find . -type f -regex ".*/target/surefire-reports/.*xml" -exec cp {} ~/test-results/junit/ \;
-        when: always
-    - store_test_results:
-        path: ~/test-results
-    - save_cache:
-        paths:
-          - ~/.m2
-        key: m2-{{ checksum "pom.xml" }}
-  build_jdk9:
+      - checkout
+      - restore_cache:
+          keys:
+            - m2-{{ checksum "pom.xml" }}
+            - m2- # used if checksum fails
+      - run:
+          name: Update maven version
+          # TODO: Could be removed, once Machine Executor image is updated https://github.com/circleci/image-builder/issues/140
+          # and the possibility of specifying Maven version is added https://github.com/circleci/image-builder/issues/143
+          command:
+            curl -fsSL https://git.io/vpDIf | bash -s -- 3.6.3
+      - run:
+          name: Update packages list
+          command:
+            sudo apt-get update
+      - run:
+          name: Install libaio1.so library for MySQL integration tests
+          command:
+            sudo apt-get install libaio1 libaio-dev
+      - run:
+          name: Drill project build
+          # TODO: 2. Optimizing Maven Builds on CircleCI - https://circleci.com/blog/optimizing-maven-builds-on-circleci/
+          # TODO: 3. Resolving memory issues without "SlowTest" and "UnlikelyTest" excludedGroups in the build
+          command: >
+            mvn install -Drat.skip=false -Dlicense.skip=false -DmemoryMb=2560 -DdirectMemoryMb=4608
+            -DexcludedGroups="org.apache.drill.categories.SlowTest,org.apache.drill.categories.UnlikelyTest" --batch-mode
+      - run:
+          name: Save test results
+          command: |
+            mkdir -p ~/test-results/junit/
+            find . -type f -regex ".*/target/surefire-reports/.*xml" -exec cp {} ~/test-results/junit/ \;
+          when: always
+      - store_test_results:
+          path: ~/test-results
+      - save_cache:
+          paths:
+            - ~/.m2
+          key: m2-{{ checksum "pom.xml" }}
+
+  build_jdk11:
     machine:
       enabled: true
       image: circleci/classic:latest
@@ -72,50 +73,50 @@ jobs:
     working_directory: ~/drill
 
     steps:
-    - checkout
-    - restore_cache:
-        keys:
-        - m2-{{ checksum "pom.xml" }}
-        - m2- # used if checksum fails
-    - run:
-        name: Update packages list
-        command:
-          sudo apt-get update
-    - run:
-        name: Install java 9
-        command:
-          sudo apt-get -y install openjdk-9-jdk
-    - run:
-        name: Set default java 9
-        command:
-          sudo update-java-alternatives --set java-1.9.0-openjdk-amd64
-    - run:
-        name: Update maven version
-        command:
-          curl -fsSL https://git.io/vpDIf | bash -s -- 3.6.0
-    - run:
-        name: Install libaio1.so library for MySQL integration tests
-        command:
-          sudo apt-get install libaio1 libaio-dev
-    - run:
-        name: Drill project build
-        command: >
-          mvn install -Drat.skip=false -Dlicense.skip=false -DmemoryMb=2560 -DdirectMemoryMb=4608
-          -DexcludedGroups="org.apache.drill.categories.SlowTest,org.apache.drill.categories.UnlikelyTest" --batch-mode
-    - run:
-        name: Save test results
-        command: |
-          mkdir -p ~/test-results/junit/
-          find . -type f -regex ".*/target/surefire-reports/.*xml" -exec cp {} ~/test-results/junit/ \;
-        when: always
-    - store_test_results:
-        path: ~/test-results
-    - save_cache:
-        paths:
-          - ~/.m2
-        key: m2-{{ checksum "pom.xml" }}
+      - checkout
+      - restore_cache:
+          keys:
+            - m2-{{ checksum "pom.xml" }}
+            - m2- # used if checksum fails
+      - run:
+          name: Update packages list
+          command:
+            sudo apt-get update
+      - run:
+          name: Install java 11
+          command:
+            sudo apt-get -y install openjdk-11-jdk
+      - run:
+          name: Set default java 11
+          command:
+            sudo update-java-alternatives --set java-1.11.0-openjdk-amd64
+      - run:
+          name: Update maven version
+          command:
+            curl -fsSL https://git.io/vpDIf | bash -s -- 3.6.3
+      - run:
+          name: Install libaio1.so library for MySQL integration tests
+          command:
+            sudo apt-get install libaio1 libaio-dev
+      - run:
+          name: Drill project build
+          command: >
+            mvn install -Drat.skip=false -Dlicense.skip=false -DmemoryMb=2560 -DdirectMemoryMb=4608
+            -DexcludedGroups="org.apache.drill.categories.SlowTest,org.apache.drill.categories.UnlikelyTest" --batch-mode
+      - run:
+          name: Save test results
+          command: |
+            mkdir -p ~/test-results/junit/
+            find . -type f -regex ".*/target/surefire-reports/.*xml" -exec cp {} ~/test-results/junit/ \;
+          when: always
+      - store_test_results:
+          path: ~/test-results
+      - save_cache:
+          paths:
+            - ~/.m2
+          key: m2-{{ checksum "pom.xml" }}
 
-  build_jdk10:
+  build_jdk12:
     machine:
       enabled: true
       image: circleci/classic:latest
@@ -124,50 +125,50 @@ jobs:
     working_directory: ~/drill
 
     steps:
-    - checkout
-    - restore_cache:
-        keys:
-        - m2-{{ checksum "pom.xml" }}
-        - m2- # used if checksum fails
-    - run:
-        name: Update packages list
-        command:
-          sudo apt-get update
-    - run:
-        name: Install java 10
-        command:
-          sudo apt-get -y install openjdk-10-jdk
-    - run:
-        name: Set default java 10
-        command:
-          sudo update-java-alternatives --set java-1.10.0-openjdk-amd64
-    - run:
-        name: Update maven version
-        command:
-          curl -fsSL https://git.io/vpDIf | bash -s -- 3.6.0
-    - run:
-        name: Install libaio1.so library for MySQL integration tests
-        command:
-          sudo apt-get install libaio1 libaio-dev
-    - run:
-        name: Drill project build
-        command: >
-          mvn install -Drat.skip=false -Dlicense.skip=false -DmemoryMb=2560 -DdirectMemoryMb=4608
-          -DexcludedGroups="org.apache.drill.categories.SlowTest,org.apache.drill.categories.UnlikelyTest" --batch-mode
-    - run:
-        name: Save test results
-        command: |
-          mkdir -p ~/test-results/junit/
-          find . -type f -regex ".*/target/surefire-reports/.*xml" -exec cp {} ~/test-results/junit/ \;
-        when: always
-    - store_test_results:
-        path: ~/test-results
-    - save_cache:
-        paths:
-          - ~/.m2
-        key: m2-{{ checksum "pom.xml" }}
+      - checkout
+      - restore_cache:
+          keys:
+            - m2-{{ checksum "pom.xml" }}
+            - m2- # used if checksum fails
+      - run:
+          name: Update packages list
+          command:
+            sudo apt-get update
+      - run:
+          name: Install java 12
+          command:
+            sudo apt-get -y install openjdk-12-jdk
+      - run:
+          name: Set default java 12
+          command:
+            sudo update-java-alternatives --set java-1.12.0-openjdk-amd64
+      - run:
+          name: Update maven version
+          command:
+            curl -fsSL https://git.io/vpDIf | bash -s -- 3.6.3
+      - run:
+          name: Install libaio1.so library for MySQL integration tests
+          command:
+            sudo apt-get install libaio1 libaio-dev
+      - run:
+          name: Drill project build
+          command: >
+            mvn install -Drat.skip=false -Dlicense.skip=false -DmemoryMb=2560 -DdirectMemoryMb=4608
+            -DexcludedGroups="org.apache.drill.categories.SlowTest,org.apache.drill.categories.UnlikelyTest" --batch-mode
+      - run:
+          name: Save test results
+          command: |
+            mkdir -p ~/test-results/junit/
+            find . -type f -regex ".*/target/surefire-reports/.*xml" -exec cp {} ~/test-results/junit/ \;
+          when: always
+      - store_test_results:
+          path: ~/test-results
+      - save_cache:
+          paths:
+            - ~/.m2
+          key: m2-{{ checksum "pom.xml" }}
 
-  build_jdk11:
+  build_jdk13:
     machine:
       enabled: true
       image: circleci/classic:latest
@@ -176,56 +177,54 @@ jobs:
     working_directory: ~/drill
 
     steps:
-    - checkout
-    - restore_cache:
-        keys:
-        - m2-{{ checksum "pom.xml" }}
-        - m2- # used if checksum fails
-    - run:
-        name: Update packages list
-        command:
-          sudo apt-get update
-    - run:
-        name: Install java 11
-        command:
-          sudo apt-get -y install openjdk-11-jdk
-    - run:
-        name: Set default java 11
-        command:
-          sudo update-java-alternatives --set java-1.11.0-openjdk-amd64
-    - run:
-        name: Update maven version
-        command:
-          curl -fsSL https://git.io/vpDIf | bash -s -- 3.6.0
-    - run:
-        name: Install libaio1.so library for MySQL integration tests
-        command:
-          sudo apt-get install libaio1 libaio-dev
-    - run:
-        name: Drill project build
-        # Set forkCount to 1 since tests use more memory and memory limitations for CircleCI is reached
-        # for default value of forkCount.
-        command: >
-          mvn install -Drat.skip=false -Dlicense.skip=false -DmemoryMb=2560 -DdirectMemoryMb=4608 -DforkCount=1
-          -DexcludedGroups="org.apache.drill.categories.SlowTest,org.apache.drill.categories.UnlikelyTest" --batch-mode
-    - run:
-        name: Save test results
-        command: |
-          mkdir -p ~/test-results/junit/
-          find . -type f -regex ".*/target/surefire-reports/.*xml" -exec cp {} ~/test-results/junit/ \;
-        when: always
-    - store_test_results:
-        path: ~/test-results
-    - save_cache:
-        paths:
-          - ~/.m2
-        key: m2-{{ checksum "pom.xml" }}
+      - checkout
+      - restore_cache:
+          keys:
+            - m2-{{ checksum "pom.xml" }}
+            - m2- # used if checksum fails
+      - run:
+          name: Update packages list
+          command:
+            sudo apt-get update
+      - run:
+          name: Install java 13
+          command:
+            sudo apt-get -y install openjdk-13-jdk
+      - run:
+          name: Set default java 13
+          command:
+            sudo update-java-alternatives --set java-1.13.0-openjdk-amd64
+      - run:
+          name: Update maven version
+          command:
+            curl -fsSL https://git.io/vpDIf | bash -s -- 3.6.3
+      - run:
+          name: Install libaio1.so library for MySQL integration tests
+          command:
+            sudo apt-get install libaio1 libaio-dev
+      - run:
+          name: Drill project build
+          command: >
+            mvn install -Drat.skip=false -Dlicense.skip=false -DmemoryMb=2560 -DdirectMemoryMb=4608
+            -DexcludedGroups="org.apache.drill.categories.SlowTest,org.apache.drill.categories.UnlikelyTest" --batch-mode
+      - run:
+          name: Save test results
+          command: |
+            mkdir -p ~/test-results/junit/
+            find . -type f -regex ".*/target/surefire-reports/.*xml" -exec cp {} ~/test-results/junit/ \;
+          when: always
+      - store_test_results:
+          path: ~/test-results
+      - save_cache:
+          paths:
+            - ~/.m2
+          key: m2-{{ checksum "pom.xml" }}
 
 workflows:
   version: 2
   build_and_test:
     jobs:
-    - build_jdk8
-    - build_jdk9
-    - build_jdk10
-    - build_jdk11
+      - build_jdk8
+      - build_jdk11
+      - build_jdk12
+      - build_jdk13
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveClusterTest.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveClusterTest.java
new file mode 100644
index 0000000..3fd3a11
--- /dev/null
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveClusterTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.hive;
+
+import org.apache.drill.test.ClusterTest;
+import org.junit.BeforeClass;
+
+/**
+ * Base class for Hive cluster tests.
+ */
+public class HiveClusterTest extends ClusterTest {
+
+  @BeforeClass
+  public static void checkJavaVersion() {
+    HiveTestUtilities.assumeJavaVersion();
+  }
+}
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestBase.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestBase.java
index c3acdb0..05ce7ae 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestBase.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestBase.java
@@ -37,30 +37,35 @@ public class HiveTestBase extends PlanTestBase {
   public static final HiveTestFixture HIVE_TEST_FIXTURE;
 
   static {
-    // generate hive data common for all test classes using own dirWatcher
-    BaseDirTestWatcher generalDirWatcher = new BaseDirTestWatcher() {
-      {
+    if (HiveTestUtilities.supportedJavaVersion()) {
+      // generate hive data common for all test classes using own dirWatcher
+      BaseDirTestWatcher generalDirWatcher = new BaseDirTestWatcher() {
+        {
         /*
            Below protected method invoked to create directory DirWatcher.dir with path like:
            ./target/org.apache.drill.exec.hive.HiveTestBase123e4567-e89b-12d3-a456-556642440000.
            Then subdirectory with name 'root' will be used to hold metastore_db and other data shared between
            all derivatives of the class. Note that UUID suffix is necessary to avoid conflicts between forked JVMs.
         */
-        starting(Description.createSuiteDescription(HiveTestBase.class.getName().concat(UUID.randomUUID().toString())));
-      }
-    };
-    File baseDir = generalDirWatcher.getRootDir();
-    HIVE_TEST_FIXTURE = HiveTestFixture.builder(baseDir).build();
-    HiveTestDataGenerator dataGenerator = new HiveTestDataGenerator(generalDirWatcher, baseDir,
-        HIVE_TEST_FIXTURE.getWarehouseDir());
-    HIVE_TEST_FIXTURE.getDriverManager().runWithinSession(dataGenerator::generateData);
+          starting(Description.createSuiteDescription(HiveTestBase.class.getName().concat(UUID.randomUUID().toString())));
+        }
+      };
+      File baseDir = generalDirWatcher.getRootDir();
+      HIVE_TEST_FIXTURE = HiveTestFixture.builder(baseDir).build();
+      HiveTestDataGenerator dataGenerator = new HiveTestDataGenerator(generalDirWatcher, baseDir,
+          HIVE_TEST_FIXTURE.getWarehouseDir());
+      HIVE_TEST_FIXTURE.getDriverManager().runWithinSession(dataGenerator::generateData);
 
-    // set hook for clearing watcher's dir on JVM shutdown
-    Runtime.getRuntime().addShutdownHook(new Thread(() -> FileUtils.deleteQuietly(generalDirWatcher.getDir())));
+      // set hook for clearing watcher's dir on JVM shutdown
+      Runtime.getRuntime().addShutdownHook(new Thread(() -> FileUtils.deleteQuietly(generalDirWatcher.getDir())));
+    } else {
+      HIVE_TEST_FIXTURE = null;
+    }
   }
 
   @BeforeClass
   public static void setUp() {
+    HiveTestUtilities.assumeJavaVersion();
     HIVE_TEST_FIXTURE.getPluginManager().addHivePluginTo(bits);
   }
 
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestUtilities.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestUtilities.java
index ead1479..8518fca 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestUtilities.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/HiveTestUtilities.java
@@ -30,9 +30,12 @@ import org.apache.drill.test.TestTools;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.junit.AssumptionViolatedException;
 
 import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.startsWith;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assume.assumeThat;
 
 public class HiveTestUtilities {
 
@@ -128,4 +131,23 @@ public class HiveTestUtilities {
     assertThat(plan, containsString("HiveDrillNativeParquetScan"));
   }
 
+  /**
+   * Current Hive version doesn't support JDK 9+.
+   * Checks if current version is supported by Hive.
+   *
+   * @return {@code true} if current version is supported by Hive, {@code false} otherwise
+   */
+  public static boolean supportedJavaVersion() {
+    return System.getProperty("java.version").startsWith("1.8");
+  }
+
+  /**
+   * Checks if current version is supported by Hive.
+   *
+   * @throws AssumptionViolatedException if current version is not supported by Hive,
+   * so unit tests may be skipped.
+   */
+  public static void assumeJavaVersion() throws AssumptionViolatedException {
+    assumeThat("Skipping tests since Hive supports only JDK 8.", System.getProperty("java.version"), startsWith("1.8"));
+  }
 }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/complex_types/TestHiveArrays.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/complex_types/TestHiveArrays.java
index 955b800..9513cbb 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/complex_types/TestHiveArrays.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/complex_types/TestHiveArrays.java
@@ -28,12 +28,12 @@ import java.util.stream.Stream;
 import org.apache.drill.categories.HiveStorageTest;
 import org.apache.drill.categories.SlowTest;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.hive.HiveClusterTest;
 import org.apache.drill.exec.hive.HiveTestFixture;
 import org.apache.drill.exec.hive.HiveTestUtilities;
 import org.apache.drill.exec.util.StoragePluginTestUtils;
 import org.apache.drill.exec.util.Text;
 import org.apache.drill.test.ClusterFixture;
-import org.apache.drill.test.ClusterTest;
 import org.apache.drill.test.TestBuilder;
 import org.apache.hadoop.hive.ql.Driver;
 import org.junit.AfterClass;
@@ -50,7 +50,7 @@ import static org.apache.drill.test.TestBuilder.listOf;
 import static org.apache.drill.test.TestBuilder.mapOfObject;
 
 @Category({SlowTest.class, HiveStorageTest.class})
-public class TestHiveArrays extends ClusterTest {
+public class TestHiveArrays extends HiveClusterTest {
 
   private static HiveTestFixture hiveTestFixture;
 
@@ -67,7 +67,7 @@ public class TestHiveArrays extends ClusterTest {
   }
 
   @AfterClass
-  public static void tearDown() throws Exception {
+  public static void tearDown() {
     if (hiveTestFixture != null) {
       hiveTestFixture.getPluginManager().removeHivePluginFrom(cluster.drillbit());
     }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/complex_types/TestHiveMaps.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/complex_types/TestHiveMaps.java
index 8ee7eaa..4f2118e 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/complex_types/TestHiveMaps.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/complex_types/TestHiveMaps.java
@@ -24,11 +24,11 @@ import java.nio.file.Paths;
 import org.apache.drill.categories.HiveStorageTest;
 import org.apache.drill.categories.SlowTest;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.hive.HiveClusterTest;
 import org.apache.drill.exec.hive.HiveTestFixture;
 import org.apache.drill.exec.hive.HiveTestUtilities;
 import org.apache.drill.exec.util.StoragePluginTestUtils;
 import org.apache.drill.test.ClusterFixture;
-import org.apache.drill.test.ClusterTest;
 import org.apache.hadoop.hive.ql.Driver;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -45,7 +45,7 @@ import static org.apache.drill.test.TestBuilder.mapOf;
 import static org.apache.drill.test.TestBuilder.mapOfObject;
 
 @Category({SlowTest.class, HiveStorageTest.class})
-public class TestHiveMaps extends ClusterTest {
+public class TestHiveMaps extends HiveClusterTest {
 
   private static HiveTestFixture hiveTestFixture;
 
@@ -59,7 +59,7 @@ public class TestHiveMaps extends ClusterTest {
   }
 
   @AfterClass
-  public static void tearDown() throws Exception {
+  public static void tearDown() {
     if (hiveTestFixture != null) {
       hiveTestFixture.getPluginManager().removeHivePluginFrom(cluster.drillbit());
     }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/complex_types/TestHiveStructs.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/complex_types/TestHiveStructs.java
index 106ab17..4b0750d 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/complex_types/TestHiveStructs.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/complex_types/TestHiveStructs.java
@@ -23,13 +23,13 @@ import java.nio.file.Paths;
 import org.apache.drill.categories.HiveStorageTest;
 import org.apache.drill.categories.SlowTest;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.hive.HiveClusterTest;
 import org.apache.drill.exec.hive.HiveTestFixture;
 import org.apache.drill.exec.hive.HiveTestUtilities;
 import org.apache.drill.exec.util.JsonStringHashMap;
 import org.apache.drill.exec.util.StoragePluginTestUtils;
 import org.apache.drill.exec.util.Text;
 import org.apache.drill.test.ClusterFixture;
-import org.apache.drill.test.ClusterTest;
 import org.apache.hadoop.hive.ql.Driver;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -44,7 +44,7 @@ import static org.apache.drill.test.TestBuilder.mapOf;
 import static org.apache.drill.test.TestBuilder.mapOfObject;
 
 @Category({SlowTest.class, HiveStorageTest.class})
-public class TestHiveStructs extends ClusterTest {
+public class TestHiveStructs extends HiveClusterTest {
 
   private static final JsonStringHashMap<String, Object> STR_N0_ROW_1 = mapOf(
       "f_int", -3000, "f_string", new Text("AbbBBa"), "f_varchar", new Text("-c54g"), "f_char", new Text("Th"),
@@ -88,7 +88,7 @@ public class TestHiveStructs extends ClusterTest {
   }
 
   @AfterClass
-  public static void tearDown() throws Exception {
+  public static void tearDown() {
     if (hiveTestFixture != null) {
       hiveTestFixture.getPluginManager().removeHivePluginFrom(cluster.drillbit());
     }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/complex_types/TestHiveUnions.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/complex_types/TestHiveUnions.java
index 3dc09c9..3a02a1e 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/complex_types/TestHiveUnions.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/complex_types/TestHiveUnions.java
@@ -24,10 +24,10 @@ import org.apache.drill.categories.HiveStorageTest;
 import org.apache.drill.categories.SlowTest;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.expr.fn.impl.DateUtility;
+import org.apache.drill.exec.hive.HiveClusterTest;
 import org.apache.drill.exec.hive.HiveTestFixture;
 import org.apache.drill.exec.hive.HiveTestUtilities;
 import org.apache.drill.test.ClusterFixture;
-import org.apache.drill.test.ClusterTest;
 import org.apache.hadoop.hive.ql.Driver;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -39,7 +39,7 @@ import static org.apache.drill.test.TestBuilder.mapOf;
 import static org.apache.drill.test.TestBuilder.mapOfObject;
 
 @Category({SlowTest.class, HiveStorageTest.class})
-public class TestHiveUnions extends ClusterTest {
+public class TestHiveUnions extends HiveClusterTest {
 
   private static HiveTestFixture hiveTestFixture;
 
@@ -54,7 +54,7 @@ public class TestHiveUnions extends ClusterTest {
   }
 
   @AfterClass
-  public static void tearDown() throws Exception {
+  public static void tearDown() {
     if (hiveTestFixture != null) {
       hiveTestFixture.getPluginManager().removeHivePluginFrom(cluster.drillbit());
     }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/BaseTestHiveImpersonation.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/BaseTestHiveImpersonation.java
index 39f8655..422d44a 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/BaseTestHiveImpersonation.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/BaseTestHiveImpersonation.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.calcite.schema.Schema.TableType;
+import org.apache.drill.exec.hive.HiveTestUtilities;
 import org.apache.drill.exec.impersonation.BaseTestImpersonation;
 import org.apache.drill.exec.store.hive.HiveStoragePluginConfig;
 import org.apache.drill.test.TestBuilder;
@@ -34,6 +35,7 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.shims.ShimLoader;
+import org.junit.BeforeClass;
 
 import static org.apache.drill.exec.hive.HiveTestUtilities.createDirWithPosixPermissions;
 import static org.apache.drill.exec.hive.HiveTestUtilities.executeQuery;
@@ -62,6 +64,11 @@ public class BaseTestHiveImpersonation extends BaseTestImpersonation {
       "(rownum INT, name STRING, gpa FLOAT, studentnum BIGINT) " +
       "partitioned by (age INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' STORED AS TEXTFILE";
 
+  @BeforeClass
+  public static void setUp() {
+    HiveTestUtilities.assumeJavaVersion();
+  }
+
   protected static void prepHiveConfAndData() throws Exception {
     hiveConf = new HiveConf();
 
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java
index 5b53113..52ef567 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java
@@ -21,6 +21,7 @@ import java.util.Objects;
 
 import org.apache.drill.categories.HiveStorageTest;
 import org.apache.drill.categories.SlowTest;
+import org.apache.drill.exec.hive.HiveTestUtilities;
 import org.apache.drill.exec.sql.TestBaseViewSupport;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -34,14 +35,15 @@ import static org.apache.drill.exec.util.StoragePluginTestUtils.DFS_TMP_SCHEMA;
 public class TestViewSupportOnHiveTables extends TestBaseViewSupport {
 
   @BeforeClass
-  public static void setUp() throws Exception {
+  public static void setUp() {
+    HiveTestUtilities.assumeJavaVersion();
     Objects.requireNonNull(HIVE_TEST_FIXTURE, "Failed to configure Hive storage plugin, " +
         "because HiveTestBase.HIVE_TEST_FIXTURE isn't initialized!")
         .getPluginManager().addHivePluginTo(bits);
   }
 
   @AfterClass
-  public static void tearDown() throws Exception {
+  public static void tearDown() {
     if (HIVE_TEST_FIXTURE != null) {
       HIVE_TEST_FIXTURE.getPluginManager().removeHivePluginFrom(bits);
     }
diff --git a/contrib/storage-mongo/pom.xml b/contrib/storage-mongo/pom.xml
index a58ed57..cf5b98d 100644
--- a/contrib/storage-mongo/pom.xml
+++ b/contrib/storage-mongo/pom.xml
@@ -66,7 +66,7 @@
     <dependency>
       <groupId>de.flapdoodle.embed</groupId>
       <artifactId>de.flapdoodle.embed.mongo</artifactId>
-      <version>2.0.3</version>
+      <version>2.2.0</version>
       <scope>test</scope>
     </dependency>
   </dependencies>
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java
index f5cef38..c0d91a1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.compile;
 
 import java.lang.reflect.Modifier;
 import java.util.Collection;
-import java.util.Iterator;
+import java.util.HashSet;
 import java.util.Set;
 
 import org.apache.drill.exec.compile.ClassTransformer.ClassSet;
@@ -30,15 +30,15 @@ import org.objectweb.asm.ClassVisitor;
 import org.objectweb.asm.ClassWriter;
 import org.objectweb.asm.FieldVisitor;
 import org.objectweb.asm.MethodVisitor;
+import org.objectweb.asm.commons.ClassRemapper;
+import org.objectweb.asm.commons.MethodRemapper;
 import org.objectweb.asm.commons.Remapper;
-import org.objectweb.asm.commons.RemappingClassAdapter;
-import org.objectweb.asm.commons.RemappingMethodAdapter;
 import org.objectweb.asm.commons.SimpleRemapper;
 import org.objectweb.asm.tree.ClassNode;
-import org.objectweb.asm.tree.FieldNode;
 import org.objectweb.asm.tree.MethodNode;
 
-import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Serves two purposes. Renames all inner classes references to the outer class to the new name. Also adds all the
@@ -46,10 +46,10 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
  */
 @SuppressWarnings("unused")
 class MergeAdapter extends ClassVisitor {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MergeAdapter.class);
+  private static final Logger logger = LoggerFactory.getLogger(MergeAdapter.class);
   private final ClassNode classToMerge;
   private final ClassSet set;
-  private final Set<String> mergingNames = Sets.newHashSet();
+  private final Set<String> mergingNames = new HashSet<>();
   private final boolean hasInit;
   private String name;
 
@@ -62,8 +62,8 @@ class MergeAdapter extends ClassVisitor {
     this.set = set;
 
     boolean hasInit = false;
-    for (Object o  : classToMerge.methods) {
-      String name = ((MethodNode)o).name;
+    for (MethodNode methodNode : classToMerge.methods) {
+      String name = methodNode.name;
       if (name.equals("<init>")) {
         continue;
       }
@@ -78,18 +78,12 @@ class MergeAdapter extends ClassVisitor {
 
   @Override
   public void visitInnerClass(String name, String outerName, String innerName, int access) {
-    // logger.debug(String.format("[Inner Class] Name: %s, outerName: %s, innerName: %s, templateName: %s, newName: %s.",
-    // name, outerName, innerName, templateName, newName));
-
     if (name.startsWith(set.precompiled.slash)) {
-//      outerName = outerName.replace(precompiled.slash, generated.slash);
       name = name.replace(set.precompiled.slash, set.generated.slash);
       int i = name.lastIndexOf('$');
       outerName = name.substring(0, i);
-      super.visitInnerClass(name, outerName, innerName, access);
-    } else {
-      super.visitInnerClass(name, outerName, innerName, access);
     }
+    super.visitInnerClass(name, outerName, innerName, access);
   }
 
   // visit the class
@@ -97,13 +91,10 @@ class MergeAdapter extends ClassVisitor {
   public void visit(int version, int access, String name, String signature, String superName, String[] interfaces) {
     // use the access and names of the impl class.
     this.name = name;
-    if (name.contains("$")) {
-      super.visit(version, access, name, signature, superName, interfaces);
-    } else {
-      super.visit(version, access ^ Modifier.ABSTRACT | Modifier.FINAL, name, signature, superName, interfaces);
+    if (!name.contains("$")) {
+      access = access ^ Modifier.ABSTRACT | Modifier.FINAL;
     }
-
-//    this.cname = name;
+    super.visit(version, access, name, signature, superName, interfaces);
   }
 
   @Override
@@ -113,23 +104,15 @@ class MergeAdapter extends ClassVisitor {
 
     // skip all abstract methods as they should have implementations.
     if ((access & Modifier.ABSTRACT) != 0 || mergingNames.contains(name)) {
-
-//      logger.debug("Skipping copy of '{}()' since it is abstract or listed elsewhere.", arg1);
       return null;
     }
     if (signature != null) {
       signature = signature.replace(set.precompiled.slash, set.generated.slash);
     }
-    // if ((access & Modifier.PUBLIC) == 0) {
-    // access = access ^ Modifier.PUBLIC ^ Modifier.PROTECTED | Modifier.PRIVATE;
-    // }
+
     MethodVisitor mv = super.visitMethod(access, name, desc, signature, exceptions);
-    if (!name.equals("<init>")) {
-      access = access | Modifier.FINAL;
-    } else {
-      if (hasInit) {
-        return new DrillInitMethodVisitor(this.name, mv);
-      }
+    if (name.equals("<init>") && hasInit) {
+      return new DrillInitMethodVisitor(this.name, mv);
     }
     return mv;
   }
@@ -137,28 +120,21 @@ class MergeAdapter extends ClassVisitor {
   @Override
   public void visitEnd() {
     // add all the fields of the class we're going to merge.
-    for (Iterator<?> it = classToMerge.fields.iterator(); it.hasNext();) {
-
-      // Special handling for nested classes. Drill uses non-static nested
-      // "inner" classes in some templates. Prior versions of Drill would
-      // create the generated nested classes as static, then this line
-      // would copy the "this$0" field to convert the static nested class
-      // into a non-static inner class. However, that approach is not
-      // compatible with plain-old Java compilation. Now, Drill generates
-      // the nested classes as non-static inner classes. As a result, we
-      // do not want to copy the hidden fields; we'll end up with two if
-      // we do.
-
-      FieldNode field = (FieldNode) it.next();
-      if (! field.name.startsWith("this$")) {
-        field.accept(this);
-      }
-    }
+    // Special handling for nested classes. Drill uses non-static nested
+    // "inner" classes in some templates. Prior versions of Drill would
+    // create the generated nested classes as static, then this line
+    // would copy the "this$0" field to convert the static nested class
+    // into a non-static inner class. However, that approach is not
+    // compatible with plain-old Java compilation. Now, Drill generates
+    // the nested classes as non-static inner classes. As a result, we
+    // do not want to copy the hidden fields; we'll end up with two if
+    // we do.
+    classToMerge.fields.stream()
+        .filter(field -> !field.name.startsWith("this$"))
+        .forEach(field -> field.accept(this));
 
     // add all the methods that we to include.
-    for (Iterator<?> it = classToMerge.methods.iterator(); it.hasNext();) {
-      MethodNode mn = (MethodNode) it.next();
-
+    for (MethodNode mn : classToMerge.methods) {
       if (mn.name.equals("<init>")) {
         continue;
       }
@@ -178,7 +154,7 @@ class MergeAdapter extends ClassVisitor {
       while (top.parent != null) {
         top = top.parent;
       }
-      mn.accept(new RemappingMethodAdapter(mn.access, mn.desc, mv,
+      mn.accept(new MethodRemapper(mv,
           new SimpleRemapper(top.precompiled.slash, top.generated.slash)));
 
     }
@@ -250,7 +226,7 @@ class MergeAdapter extends ClassVisitor {
         writerVisitor = new DrillCheckClassAdapter(CompilationConfig.ASM_API_VERSION,
             new CheckClassVisitorFsm(CompilationConfig.ASM_API_VERSION, writerVisitor), true);
       }
-      ClassVisitor remappingAdapter = new RemappingClassAdapter(writerVisitor, re);
+      ClassVisitor remappingAdapter = new ClassRemapper(writerVisitor, re);
       if (verifyBytecode) {
         remappingAdapter = new DrillCheckClassAdapter(CompilationConfig.ASM_API_VERSION,
             new CheckClassVisitorFsm(CompilationConfig.ASM_API_VERSION, remappingAdapter), true);
@@ -285,11 +261,11 @@ class MergeAdapter extends ClassVisitor {
   }
 
   private static class RemapClasses extends Remapper {
-    final Set<String> innerClasses = Sets.newHashSet();
-    ClassSet top;
-    ClassSet current;
+    private final Set<String> innerClasses = new HashSet<>();
+    private final ClassSet top;
+    private final ClassSet current;
 
-    public RemapClasses(final ClassSet set) {
+    public RemapClasses(ClassSet set) {
       current = set;
       ClassSet top = set;
       while (top.parent != null) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataAggregateHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataAggregateHelper.java
index 1cca788..6f00dea 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataAggregateHelper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataAggregateHelper.java
@@ -182,7 +182,8 @@ public class MetadataAggregateHelper {
   private void addLastModifiedCall() {
     String lastModifiedColumn = columnNamesOptions.lastModifiedTime();
     LogicalExpression lastModifiedTime;
-    if (createNewAggregations()) {
+    // it is enough to call any_value(`lmt`) for file metadata level or more specific metadata
+    if (context.metadataLevel().includes(MetadataType.FILE)) {
       lastModifiedTime = new FunctionCall("any_value",
           Collections.singletonList(
               FieldReference.getWithQuotedRef(lastModifiedColumn)),
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java
index 2c657e6..5134f42 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/BatchValidator.java
@@ -176,6 +176,10 @@ public class BatchValidator {
   }
 
   public static boolean validate(RecordBatch batch) {
+    // This is a handy place to trace batches as they flow up
+    // the DAG. Works best for single-threaded runs with a few records.
+    // System.out.println(batch.getClass().getSimpleName());
+    // RowSetFormatter.print(batch);
     ErrorReporter reporter = errorReporter(batch);
     int rowCount = batch.getRecordCount();
     int valueCount = rowCount;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/spnego/TestDrillSpnegoAuthenticator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/spnego/TestDrillSpnegoAuthenticator.java
index efa1974..1059e46 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/spnego/TestDrillSpnegoAuthenticator.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/spnego/TestDrillSpnegoAuthenticator.java
@@ -133,7 +133,6 @@ public class TestDrillSpnegoAuthenticator extends BaseTest {
   /**
    * Test to verify response when request is sent for {@link WebServerConstants#SPENGO_LOGIN_RESOURCE_PATH} from
    * unauthenticated session. Expectation is client will receive response with Negotiate header.
-   * @throws Exception
    */
   @Test
   public void testNewSessionReqForSpnegoLogin() throws Exception {
@@ -154,7 +153,6 @@ public class TestDrillSpnegoAuthenticator extends BaseTest {
   /**
    * Test to verify response when request is sent for {@link WebServerConstants#SPENGO_LOGIN_RESOURCE_PATH} from
    * authenticated session. Expectation is server will find the authenticated UserIdentity.
-   * @throws Exception
    */
   @Test
   public void testAuthClientRequestForSpnegoLoginResource() throws Exception {
@@ -179,7 +177,6 @@ public class TestDrillSpnegoAuthenticator extends BaseTest {
    * Test to verify response when request is sent for any other resource other than
    * {@link WebServerConstants#SPENGO_LOGIN_RESOURCE_PATH} from authenticated session. Expectation is server will
    * find the authenticated UserIdentity and will not perform the authentication again for new resource.
-   * @throws Exception
    */
   @Test
   public void testAuthClientRequestForOtherPage() throws Exception {
@@ -203,8 +200,7 @@ public class TestDrillSpnegoAuthenticator extends BaseTest {
   /**
    * Test to verify that when request is sent for {@link WebServerConstants#LOGOUT_RESOURCE_PATH} then the UserIdentity
    * will be removed from the session and returned authentication will be null from
-   * {@link DrillSpnegoAuthenticator#validateRequest(ServletRequest, ServletResponse, boolean)}
-   * @throws Exception
+   * {@link DrillSpnegoAuthenticator#validateRequest(javax.servlet.ServletRequest, javax.servlet.ServletResponse, boolean)}
    */
   @Test
   public void testAuthClientRequestForLogOut() throws Exception {
@@ -228,7 +224,6 @@ public class TestDrillSpnegoAuthenticator extends BaseTest {
   /**
    * Test to verify authentication fails when client sends invalid SPNEGO token for the
    * {@link WebServerConstants#SPENGO_LOGIN_RESOURCE_PATH} resource.
-   * @throws Exception
    */
   @Test
   public void testSpnegoLoginInvalidToken() throws Exception {
@@ -242,28 +237,25 @@ public class TestDrillSpnegoAuthenticator extends BaseTest {
         spnegoHelper.clientKeytab.getAbsoluteFile());
 
     // Generate a SPNEGO token for the peer SERVER_PRINCIPAL from this CLIENT_PRINCIPAL
-    final String token = Subject.doAs(clientSubject, new PrivilegedExceptionAction<String>() {
-      @Override
-      public String run() throws Exception {
-
-        final GSSManager gssManager = GSSManager.getInstance();
-        GSSContext gssContext = null;
-        try {
-          final Oid oid = GSSUtil.GSS_SPNEGO_MECH_OID;
-          final GSSName serviceName = gssManager.createName(spnegoHelper.SERVER_PRINCIPAL, GSSName.NT_USER_NAME, oid);
-
-          gssContext = gssManager.createContext(serviceName, oid, null, GSSContext.DEFAULT_LIFETIME);
-          gssContext.requestCredDeleg(true);
-          gssContext.requestMutualAuth(true);
-
-          byte[] outToken = new byte[0];
-          outToken = gssContext.initSecContext(outToken, 0, outToken.length);
-          return Base64.encodeBase64String(outToken);
-
-        } finally {
-          if (gssContext != null) {
-            gssContext.dispose();
-          }
+    final String token = Subject.doAs(clientSubject, (PrivilegedExceptionAction<String>) () -> {
+
+      final GSSManager gssManager = GSSManager.getInstance();
+      GSSContext gssContext = null;
+      try {
+        final Oid oid = GSSUtil.GSS_SPNEGO_MECH_OID;
+        final GSSName serviceName = gssManager.createName(spnegoHelper.SERVER_PRINCIPAL, GSSName.NT_USER_NAME, oid);
+
+        gssContext = gssManager.createContext(serviceName, oid, null, GSSContext.DEFAULT_LIFETIME);
+        gssContext.requestCredDeleg(true);
+        gssContext.requestMutualAuth(true);
+
+        byte[] outToken = new byte[0];
+        outToken = gssContext.initSecContext(outToken, 0, outToken.length);
+        return Base64.encodeBase64String(outToken);
+
+      } finally {
+        if (gssContext != null) {
+          gssContext.dispose();
         }
       }
     });
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchemaWithMetastore.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchemaWithMetastore.java
index c89472b..28d6a6a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchemaWithMetastore.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchemaWithMetastore.java
@@ -395,7 +395,10 @@ public class TestInfoSchemaWithMetastore extends ClusterTest {
   }
 
   private ZonedDateTime currentUtcTime() {
-    ZonedDateTime currentTime = ZonedDateTime.of(LocalDateTime.now(), ZoneId.systemDefault());
+    // Java 9 and later returns LocalDateTime with nanoseconds precision,
+    // but Java 8 returns LocalDateTime with milliseconds precision
+    // and metastore stores last modified time in milliseconds
+    ZonedDateTime currentTime = ZonedDateTime.of(LocalDateTime.now().withNano(0), ZoneId.systemDefault());
     return currentTime.withZoneSameInstant(ZoneId.of("UTC"));
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestMetastoreCommands.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestMetastoreCommands.java
index b8bf889..ccee4ed 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestMetastoreCommands.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestMetastoreCommands.java
@@ -530,7 +530,8 @@ public class TestMetastoreCommands extends ClusterTest {
             .basicRequests()
             .tableMetadata(tableInfo);
 
-        assertEquals(expectedTableMetadata, actualTableMetadata);
+        assertEquals(String.format("Table metadata mismatch for [%s] metadata level", analyzeLevel),
+            expectedTableMetadata, actualTableMetadata);
       } finally {
         run("analyze table dfs.tmp.`%s` drop metadata if exists", tableName);
       }
diff --git a/exec/memory/base/src/test/java/org/apache/drill/exec/memory/BoundsCheckingTest.java b/exec/memory/base/src/test/java/org/apache/drill/exec/memory/BoundsCheckingTest.java
index 3b5c7ee..dff8555 100644
--- a/exec/memory/base/src/test/java/org/apache/drill/exec/memory/BoundsCheckingTest.java
+++ b/exec/memory/base/src/test/java/org/apache/drill/exec/memory/BoundsCheckingTest.java
@@ -17,9 +17,6 @@
  */
 package org.apache.drill.exec.memory;
 
-import java.lang.reflect.Field;
-import java.lang.reflect.Modifier;
-
 import org.apache.drill.test.BaseTest;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -29,55 +26,47 @@ import org.junit.Test;
 
 import io.netty.buffer.DrillBuf;
 import io.netty.util.IllegalReferenceCountException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 public class BoundsCheckingTest extends BaseTest {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BoundsCheckingTest.class);
+  private static final Logger logger = LoggerFactory.getLogger(BoundsCheckingTest.class);
 
   private static boolean old;
 
   private RootAllocator allocator;
 
-  private static boolean setBoundsChecking(boolean enabled) throws Exception
-  {
-    Field field = BoundsChecking.class.getDeclaredField("BOUNDS_CHECKING_ENABLED");
-    Field modifiersField = Field.class.getDeclaredField("modifiers");
-    modifiersField.setAccessible(true);
-    modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
-    boolean old = field.getBoolean(null);
-    field.setAccessible(true);
-    field.set(null, enabled);
-    return old;
+  private static boolean setBoundsChecking(boolean enabled) {
+    String oldValue = System.setProperty(BoundsChecking.ENABLE_UNSAFE_BOUNDS_CHECK_PROPERTY, String.valueOf(enabled));
+    return Boolean.parseBoolean(oldValue);
   }
 
   @BeforeClass
-  public static void setBoundsCheckingEnabled() throws Exception
-  {
+  public static void setBoundsCheckingEnabled() {
     old = setBoundsChecking(true);
   }
 
   @AfterClass
-  public static void restoreBoundsChecking() throws Exception
-  {
+  public static void restoreBoundsChecking() {
     setBoundsChecking(old);
   }
 
   @Before
-  public void setupAllocator()
-  {
+  public void setupAllocator() {
     allocator = new RootAllocator(Integer.MAX_VALUE);
   }
 
   @After
-  public void closeAllocator()
-  {
+  public void closeAllocator() {
     allocator.close();
   }
 
   @Test
-  public void testLengthCheck()
-  {
+  public void testLengthCheck() {
+    assertTrue(BoundsChecking.BOUNDS_CHECKING_ENABLED);
     try {
       BoundsChecking.lengthCheck(null, 0, 0);
       fail("expecting NullPointerException");
diff --git a/metastore/iceberg-metastore/pom.xml b/metastore/iceberg-metastore/pom.xml
index 04fc137..9ff2b6e 100644
--- a/metastore/iceberg-metastore/pom.xml
+++ b/metastore/iceberg-metastore/pom.xml
@@ -136,15 +136,9 @@
     </dependency>
 
     <dependency>
-      <groupId>com.sun.codemodel</groupId>
-      <artifactId>codemodel</artifactId>
-      <version>${codemodel.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>net.openhft</groupId>
-      <artifactId>compiler</artifactId>
-      <version>2.3.4</version>
+      <groupId>org.ow2.asm</groupId>
+      <artifactId>asm</artifactId>
+      <version>${asm.version}</version>
       <scope>test</scope>
     </dependency>
   </dependencies>
diff --git a/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/schema/TestIcebergTableSchema.java b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/schema/TestIcebergTableSchema.java
index c95b9a0..3c66ba1 100644
--- a/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/schema/TestIcebergTableSchema.java
+++ b/metastore/iceberg-metastore/src/test/java/org/apache/drill/metastore/iceberg/schema/TestIcebergTableSchema.java
@@ -17,35 +17,27 @@
  */
 package org.apache.drill.metastore.iceberg.schema;
 
-import com.sun.codemodel.CodeWriter;
-import com.sun.codemodel.JAnnotationArrayMember;
-import com.sun.codemodel.JAnnotationUse;
-import com.sun.codemodel.JClass;
-import com.sun.codemodel.JClassAlreadyExistsException;
-import com.sun.codemodel.JCodeModel;
-import com.sun.codemodel.JDefinedClass;
-import com.sun.codemodel.JFieldVar;
-import com.sun.codemodel.JMod;
-import com.sun.codemodel.JPackage;
-import net.openhft.compiler.CompilerUtils;
 import org.apache.drill.metastore.MetastoreFieldDefinition;
 import org.apache.drill.metastore.iceberg.IcebergBaseTest;
 import org.apache.drill.metastore.iceberg.exceptions.IcebergMetastoreException;
-import org.apache.drill.metastore.metadata.MetadataType;
 import org.apache.iceberg.PartitionSpec;
 import org.apache.iceberg.Schema;
 import org.apache.iceberg.types.Types;
 import org.junit.Test;
+import org.objectweb.asm.ClassWriter;
+import org.objectweb.asm.FieldVisitor;
+import org.objectweb.asm.Opcodes;
+import org.objectweb.asm.Type;
+import org.objectweb.asm.signature.SignatureVisitor;
+import org.objectweb.asm.signature.SignatureWriter;
 
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.stream.Stream;
 
+import static org.objectweb.asm.Opcodes.ACC_PUBLIC;
+import static org.objectweb.asm.Opcodes.V1_8;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
@@ -53,40 +45,36 @@ import static org.junit.Assert.assertNull;
 public class TestIcebergTableSchema extends IcebergBaseTest {
 
   @Test
-  public void testAllTypes() throws Exception {
+  public void testAllTypes() {
     Class<?> clazz = new ClassGenerator(getClass().getSimpleName() + "AllTypes") {
 
       @Override
-      void addFields(JDefinedClass jDefinedClass) {
-        JFieldVar stringField = jDefinedClass.field(DEFAULT_FIELD_MODE, String.class, "stringField");
+      void addFields(ClassWriter classWriter) {
+        FieldVisitor stringField = addField(classWriter, Opcodes.ACC_PRIVATE, "stringField", String.class);
         annotate(stringField);
 
-        JFieldVar intField = jDefinedClass.field(DEFAULT_FIELD_MODE, int.class, "intField");
+        FieldVisitor intField = addField(classWriter, Opcodes.ACC_PRIVATE, "intField", int.class);
         annotate(intField);
 
-        JFieldVar integerField = jDefinedClass.field(DEFAULT_FIELD_MODE, Integer.class, "integerField");
+        FieldVisitor integerField = addField(classWriter, Opcodes.ACC_PRIVATE, "integerField", Integer.class);
         annotate(integerField);
 
-        JFieldVar longField = jDefinedClass.field(DEFAULT_FIELD_MODE, Long.class, "longField");
+        FieldVisitor longField = addField(classWriter, Opcodes.ACC_PRIVATE, "longField", Long.class);
         annotate(longField);
 
-        JFieldVar floatField = jDefinedClass.field(DEFAULT_FIELD_MODE, Float.class, "floatField");
+        FieldVisitor floatField = addField(classWriter, Opcodes.ACC_PRIVATE, "floatField", Float.class);
         annotate(floatField);
 
-        JFieldVar doubleField = jDefinedClass.field(DEFAULT_FIELD_MODE, Double.class, "doubleField");
+        FieldVisitor doubleField = addField(classWriter, Opcodes.ACC_PRIVATE, "doubleField", Double.class);
         annotate(doubleField);
 
-        JFieldVar booleanField = jDefinedClass.field(DEFAULT_FIELD_MODE, Boolean.class, "booleanField");
+        FieldVisitor booleanField = addField(classWriter, Opcodes.ACC_PRIVATE, "booleanField", Boolean.class);
         annotate(booleanField);
 
-        JCodeModel jCodeModel = jDefinedClass.owner();
-
-        JClass listRef = jCodeModel.ref(List.class).narrow(String.class);
-        JFieldVar listField = jDefinedClass.field(DEFAULT_FIELD_MODE, listRef, "listField");
+        FieldVisitor listField = addField(classWriter, Opcodes.ACC_PRIVATE, "listField", List.class, String.class);
         annotate(listField);
 
-        JClass mapRef = jCodeModel.ref(Map.class).narrow(String.class, Float.class);
-        JFieldVar mapField = jDefinedClass.field(DEFAULT_FIELD_MODE, mapRef, "mapField");
+        FieldVisitor mapField = addField(classWriter, Opcodes.ACC_PRIVATE, "mapField", Map.class, String.class, Float.class);
         annotate(mapField);
       }
 
@@ -114,15 +102,15 @@ public class TestIcebergTableSchema extends IcebergBaseTest {
   }
 
   @Test
-  public void testIgnoreUnannotatedFields() throws Exception {
+  public void testIgnoreUnannotatedFields() {
     Class<?> clazz = new ClassGenerator(getClass().getSimpleName() + "IgnoreUnannotatedFields") {
 
       @Override
-      void addFields(JDefinedClass jDefinedClass) {
-        JFieldVar stringField = jDefinedClass.field(DEFAULT_FIELD_MODE, String.class, "stringField");
+      void addFields(ClassWriter classWriter) {
+        FieldVisitor stringField = addField(classWriter, Opcodes.ACC_PRIVATE, "stringField", String.class);
         annotate(stringField);
 
-        jDefinedClass.field(DEFAULT_FIELD_MODE, Integer.class, "integerField");
+        addField(classWriter, Opcodes.ACC_PRIVATE, "integerField", Integer.class);
       }
     }.generate();
 
@@ -132,16 +120,28 @@ public class TestIcebergTableSchema extends IcebergBaseTest {
   }
 
   @Test
-  public void testNestedComplexType() throws Exception {
+  public void testNestedComplexType() {
     Class<?> clazz = new ClassGenerator(getClass().getSimpleName() + "NestedComplexType") {
 
       @Override
-      void addFields(JDefinedClass jDefinedClass) {
-        JCodeModel jCodeModel = jDefinedClass.owner();
-
-        JClass nestedListRef = jCodeModel.ref(List.class).narrow(String.class);
-        JClass listRef = jCodeModel.ref(List.class).narrow(nestedListRef);
-        JFieldVar listField = jDefinedClass.field(DEFAULT_FIELD_MODE, listRef, "listField");
+      void addFields(ClassWriter classWriter) {
+        String descriptor = Type.getType(List.class).getDescriptor();
+
+        String signature = FieldSignatureBuilder.builder()
+            .declareType(List.class)
+            .startGeneric()
+                .declareType(List.class)
+                .startGeneric()
+                    .declareType(String.class)
+                    .endType()
+                .endGeneric()
+                .endType()
+            .endGeneric()
+            .endType()
+            .buildSignature();
+
+        FieldVisitor listField =
+            classWriter.visitField(Opcodes.ACC_PRIVATE, "stringField", descriptor, signature, null);
         annotate(listField);
       }
     }.generate();
@@ -152,12 +152,12 @@ public class TestIcebergTableSchema extends IcebergBaseTest {
   }
 
   @Test
-  public void testUnpartitionedPartitionSpec() throws Exception {
+  public void testUnpartitionedPartitionSpec() {
     Class<?> clazz = new ClassGenerator(getClass().getSimpleName() + "UnpartitionedPartitionSpec") {
 
       @Override
-      void addFields(JDefinedClass jDefinedClass) {
-        JFieldVar stringField = jDefinedClass.field(DEFAULT_FIELD_MODE, String.class, "stringField");
+      void addFields(ClassWriter classWriter) {
+        FieldVisitor stringField = addField(classWriter, Opcodes.ACC_PRIVATE, "stringField", String.class);
         annotate(stringField);
       }
     }.generate();
@@ -169,24 +169,24 @@ public class TestIcebergTableSchema extends IcebergBaseTest {
   }
 
   @Test
-  public void testPartitionedPartitionSpec() throws Exception {
+  public void testPartitionedPartitionSpec() {
     Class<?> clazz = new ClassGenerator(getClass().getSimpleName() + "PartitionedPartitionSpec") {
 
       @Override
-      void addFields(JDefinedClass jDefinedClass) {
-        JFieldVar partKey1 = jDefinedClass.field(DEFAULT_FIELD_MODE, String.class, "partKey1");
+      void addFields(ClassWriter classWriter) {
+        FieldVisitor partKey1 = addField(classWriter, Opcodes.ACC_PRIVATE, "partKey1", String.class);
         annotate(partKey1);
 
-        JFieldVar partKey2 = jDefinedClass.field(DEFAULT_FIELD_MODE, String.class, "partKey2");
+        FieldVisitor partKey2 = addField(classWriter, Opcodes.ACC_PRIVATE, "partKey2", String.class);
         annotate(partKey2);
 
-        JFieldVar partKey3 = jDefinedClass.field(DEFAULT_FIELD_MODE, String.class, "partKey3");
+        FieldVisitor partKey3 = addField(classWriter, Opcodes.ACC_PRIVATE, "partKey3", String.class);
         annotate(partKey3);
 
-        JFieldVar integerField = jDefinedClass.field(DEFAULT_FIELD_MODE, Integer.class, "integerField");
+        FieldVisitor integerField = addField(classWriter, Opcodes.ACC_PRIVATE, "integerField", Integer.class);
         annotate(integerField);
 
-        JFieldVar booleanField = jDefinedClass.field(DEFAULT_FIELD_MODE, Boolean.class, "booleanField");
+        FieldVisitor booleanField = addField(classWriter, Opcodes.ACC_PRIVATE, "booleanField", Boolean.class);
         annotate(booleanField);
       }
     }.generate();
@@ -216,15 +216,15 @@ public class TestIcebergTableSchema extends IcebergBaseTest {
   }
 
   @Test
-  public void testUnMatchingPartitionSpec() throws Exception {
+  public void testUnMatchingPartitionSpec() {
     Class<?> clazz = new ClassGenerator(getClass().getSimpleName() + "UnMatchingPartitionSpec") {
 
       @Override
-      void addFields(JDefinedClass jDefinedClass) {
-        JFieldVar partKey1 = jDefinedClass.field(DEFAULT_FIELD_MODE, String.class, "partKey1");
+      void addFields(ClassWriter classWriter) {
+        FieldVisitor partKey1 = addField(classWriter, Opcodes.ACC_PRIVATE, "partKey1", String.class);
         annotate(partKey1);
 
-        JFieldVar integerField = jDefinedClass.field(DEFAULT_FIELD_MODE, Integer.class, "integerField");
+        FieldVisitor integerField = addField(classWriter, Opcodes.ACC_PRIVATE, "integerField", Integer.class);
         annotate(integerField);
       }
     }.generate();
@@ -238,9 +238,7 @@ public class TestIcebergTableSchema extends IcebergBaseTest {
    * Generates and loads class at the runtime with specified fields.
    * Fields may or may not be annotated.
    */
-  private abstract class ClassGenerator {
-
-    final int DEFAULT_FIELD_MODE = JMod.PRIVATE;
+  private static abstract class ClassGenerator {
 
     private final String name;
 
@@ -248,53 +246,112 @@ public class TestIcebergTableSchema extends IcebergBaseTest {
       this.name = name;
     }
 
-    Class<?> generate() throws JClassAlreadyExistsException, IOException, ClassNotFoundException {
-      JCodeModel jCodeModel = prepareModel();
-      ByteArrayStreamCodeWriter codeWriter = new ByteArrayStreamCodeWriter();
-      jCodeModel.build(codeWriter);
+    Class<?> generate() {
+      ClassWriter classWriter = generateClass();
 
-      String sourceCode = codeWriter.sourceCode();
-      return CompilerUtils.CACHED_COMPILER.loadFromJava(name, sourceCode);
+      byte[] bytes = classWriter.toByteArray();
+      return new ClassLoader() {
+        public Class<?> injectClass(String name, byte[] classBytes) {
+          return defineClass(name, classBytes, 0, classBytes.length);
+        }
+      }.injectClass(name, bytes);
     }
 
-    private JCodeModel prepareModel() throws JClassAlreadyExistsException {
-      JCodeModel jCodeModel = new JCodeModel();
-      JPackage jPackage = jCodeModel._package("");
-      JDefinedClass jDefinedClass = jPackage._class(name);
-      addFields(jDefinedClass);
-      return jCodeModel;
+    public FieldVisitor addField(ClassWriter classWriter, int access, String fieldName, Class<?> clazz, Class<?>... genericTypes) {
+      String descriptor = Type.getType(clazz).getDescriptor();
+
+      String signature = null;
+
+      if (genericTypes.length > 0) {
+        FieldSignatureBuilder fieldSignatureBuilder = FieldSignatureBuilder.builder()
+            .declareType(clazz)
+            .startGeneric();
+        for (Class<?> genericType : genericTypes) {
+          fieldSignatureBuilder
+              .declareType(genericType)
+              .endType();
+        }
+        signature = fieldSignatureBuilder
+            .endGeneric()
+            .endType()
+            .buildSignature();
+      }
+
+      return classWriter.visitField(access, fieldName, descriptor, signature, null);
     }
 
-    void annotate(JFieldVar field) {
-      annotate(field, MetadataType.ALL);
+    void annotate(FieldVisitor field) {
+      field.visitAnnotation(Type.getType(MetastoreFieldDefinition.class).getDescriptor(), true);
     }
 
-    void annotate(JFieldVar field, MetadataType... scopes) {
-      JAnnotationUse annotate = field.annotate(MetastoreFieldDefinition.class);
-      assert scopes.length != 0;
-      JAnnotationArrayMember scopesParam = annotate.paramArray("scopes");
-      Stream.of(scopes).forEach(scopesParam::param);
+    private ClassWriter generateClass() {
+      ClassWriter classWriter = new ClassWriter(ClassWriter.COMPUTE_FRAMES);
+      classWriter.visit(V1_8, ACC_PUBLIC, name, null, Type.getInternalName(Object.class), null);
+      addFields(classWriter);
+      classWriter.visitEnd();
+
+      return classWriter;
     }
 
-    abstract void addFields(JDefinedClass jDefinedClass);
+    abstract void addFields(ClassWriter classWriter);
+  }
 
-    private class ByteArrayStreamCodeWriter extends CodeWriter {
+  /**
+   * Helper class for constructing field type signature string.
+   * <p>
+   * Example of usage:
+   * <p>
+   * Desired type: {@code List<Map<String, List<Integer>>>}
+   * <pre><code>
+   *         String signature = FieldSignatureBuilder.builder()
+   *           .declareType(List.class)
+   *           .startGeneric()
+   *               .declareType(Map.class)
+   *               .startGeneric()
+   *                   .declareType(String.class)
+   *                   .endType()
+   *                   .declareType(List.class)
+   *                   .startGeneric()
+   *                       .declareType(Integer.class)
+   *                       .endType()
+   *                   .endGeneric()
+   *                   .endType()
+   *               .endGeneric()
+   *               .endType()
+   *           .endGeneric()
+   *           .endType()
+   *           .buildSignature();
+   * </code></pre>
+   */
+  private static class FieldSignatureBuilder {
+    private final SignatureVisitor signatureVisitor = new SignatureWriter();
 
-      private final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+    public FieldSignatureBuilder declareType(Class<?> clazz) {
+      signatureVisitor.visitClassType(Type.getInternalName(clazz));
+      return this;
+    }
 
-      @Override
-      public OutputStream openBinary(JPackage pkg, String fileName) {
-        return outputStream;
-      }
+    public FieldSignatureBuilder startGeneric() {
+      signatureVisitor.visitTypeArgument('=');
+      return this;
+    }
 
-      @Override
-      public void close() {
-        // no need to close byte array stream
-      }
+    public FieldSignatureBuilder endGeneric() {
+      signatureVisitor.visitSuperclass();
+      return this;
+    }
 
-      String sourceCode() {
-        return new String(outputStream.toByteArray());
-      }
+    public FieldSignatureBuilder endType() {
+      signatureVisitor.visitEnd();
+      return this;
+    }
+
+    public String buildSignature() {
+      return signatureVisitor.toString();
+    }
+
+    public static FieldSignatureBuilder builder() {
+      return new FieldSignatureBuilder();
     }
   }
 }
diff --git a/pom.xml b/pom.xml
index 4815771..fb6735c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -86,14 +86,14 @@
     <hbase.version>2.2.2</hbase.version>
     <fmpp.version>1.0</fmpp.version>
     <freemarker.version>2.3.28</freemarker.version>
-    <javassist.version>3.25.0-GA</javassist.version>
+    <javassist.version>3.26.0-GA</javassist.version>
     <msgpack.version>0.6.6</msgpack.version>
     <reflections.version>0.9.10</reflections.version>
     <avro.version>1.9.0</avro.version>
     <metrics.version>4.0.2</metrics.version>
     <jetty.version>9.3.25.v20180904</jetty.version>
     <jersey.version>2.25.1</jersey.version>
-    <asm.version>7.0</asm.version>
+    <asm.version>7.2</asm.version>
     <excludedGroups />
     <memoryMb>4096</memoryMb>
     <directMemoryMb>4096</directMemoryMb>
@@ -109,6 +109,7 @@
     <codemodel.version>2.6</codemodel.version>
     <joda.version>2.10.5</joda.version>
     <javax.el.version>3.0.0</javax.el.version>
+    <surefire.version>3.0.0-M4</surefire.version>
   </properties>
 
   <scm>
@@ -553,7 +554,7 @@
                   <version>[{$lowestMavenVersion},4)</version>
                 </requireMavenVersion>
                 <requireJavaVersion>
-                  <version>[1.8,12)</version>
+                  <version>[1.8,14)</version>
                 </requireJavaVersion>
               </rules>
             </configuration>
@@ -760,7 +761,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-surefire-plugin</artifactId>
-          <version>3.0.0-M3</version>
+          <version>${surefire.version}</version>
           <executions>
             <execution>
               <id>default-test</id>
@@ -774,7 +775,7 @@
             <dependency>
               <groupId>org.apache.maven.surefire</groupId>
               <artifactId>surefire-junit47</artifactId>
-              <version>3.0.0-M3</version>
+              <version>${surefire.version}</version>
             </dependency>
           </dependencies>
           <configuration>
@@ -915,6 +916,14 @@
               </goals>
             </execution>
           </executions>
+          <dependencies>
+            <!-- Specifies asm version which supports JDK 12+ -->
+            <dependency>
+              <groupId>org.ow2.asm</groupId>
+              <artifactId>asm</artifactId>
+              <version>${asm.version}</version>
+            </dependency>
+          </dependencies>
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
diff --git a/tools/fmpp/pom.xml b/tools/fmpp/pom.xml
index ac12752..327542c 100644
--- a/tools/fmpp/pom.xml
+++ b/tools/fmpp/pom.xml
@@ -82,7 +82,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-plugin-plugin</artifactId>
-        <version>3.4</version>
+        <version>3.6.0</version>
         <configuration>
           <goalPrefix>drill-fmpp</goalPrefix>
         </configuration>
@@ -102,6 +102,14 @@
             <phase>process-classes</phase>
           </execution>
          </executions>
+        <!-- Specifies asm version which supports JDK 9+ -->
+        <dependencies>
+          <dependency>
+            <groupId>org.ow2.asm</groupId>
+            <artifactId>asm</artifactId>
+            <version>${asm.version}</version>
+          </dependency>
+        </dependencies>
        </plugin>
     </plugins>
   </build>


[drill] 09/12: DRILL-7482: Fix missing artifact and overlapping classes warnings in Drill build

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

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

commit 435b6cb1d8f3cc21444e658d39f84e97b3538030
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Wed Dec 11 13:38:09 2019 +0200

    DRILL-7482: Fix missing artifact and overlapping classes warnings in Drill build
    
    closes #1927
---
 contrib/storage-hive/hive-exec-shade/pom.xml |  5 ++++-
 exec/jdbc-all/pom.xml                        | 10 ++++++++++
 pom.xml                                      | 11 +++++++++++
 3 files changed, 25 insertions(+), 1 deletion(-)

diff --git a/contrib/storage-hive/hive-exec-shade/pom.xml b/contrib/storage-hive/hive-exec-shade/pom.xml
index 4e79444..57f87f2 100644
--- a/contrib/storage-hive/hive-exec-shade/pom.xml
+++ b/contrib/storage-hive/hive-exec-shade/pom.xml
@@ -92,6 +92,8 @@
           <artifactSet>
             <includes>
               <include>org.apache.hive:hive-exec</include>
+              <!-- Provides custom version of parquet-hadoop-bundle instead of the shaded one
+                to be able to use version with some critical fixes like PARQUET-363 -->
               <include>org.apache.parquet:parquet-hadoop-bundle</include>
               <include>commons-codec:commons-codec</include>
               <include>com.fasterxml.jackson.core:jackson-databind</include>
@@ -154,7 +156,8 @@
                 <!-- This exclusion can be removed once hive-exec uses parquet-hadoop-bundle 1.8.2 or higher.
                  It can be so, for example, after upgrading Hive to 3.0. To check if it's safe to remove the exclusion
                  you can use TestHiveStorage.readFromAlteredPartitionedTableWithEmptyGroupType() test case. -->
-                <exclude>org/apache/parquet/schema/*</exclude>
+                <exclude>org/apache/parquet/**</exclude>
+                <exclude>shaded/parquet/org/**</exclude>
               </excludes>
             </filter>
           </filters>
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index a722e33..ab6bf7b 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -343,6 +343,9 @@
               <exclude>commons-beanutils:commons-beanutils:jar:*</exclude>
               <exclude>io.netty:netty-tcnative:jar:*</exclude>
               <exclude>com.fasterxml.woodstox:woodstox-core:jar:*</exclude>
+              <exclude>com.google.code.findbugs:jsr305:*</exclude>
+              <exclude>com.esri.geometry:esri-geometry-api:*</exclude>
+              <exclude>fr.bmartel:pcapngdecoder:*</exclude>
               <exclude>dnsjava:dnsjava:jar:*</exclude>
             </excludes>
           </artifactSet>
@@ -506,6 +509,8 @@
              <artifact>org.apache.calcite.avatica:*</artifact>
              <excludes>
                <exclude>META-INF/services/java.sql.Driver</exclude>
+               <!-- Excludes shaded slf4j to avoid conflicts when they are put into the fat jar -->
+               <exclude>org/slf4j/**</exclude>
              </excludes>
            </filter>
          </filters>
@@ -670,6 +675,9 @@
                     <exclude>commons-io:commons-io</exclude>
                     <exclude>commons-beanutils:commons-beanutils-core:jar:*</exclude>
                     <exclude>commons-beanutils:commons-beanutils:jar:*</exclude>
+                    <exclude>com.google.code.findbugs:jsr305:*</exclude>
+                    <exclude>com.esri.geometry:esri-geometry-api:*</exclude>
+                    <exclude>fr.bmartel:pcapngdecoder:*</exclude>
                   </excludes>
                 </artifactSet>
                 <relocations>
@@ -817,6 +825,8 @@
                     <artifact>org.apache.calcite.avatica:*</artifact>
                     <excludes>
                       <exclude>META-INF/services/java.sql.Driver</exclude>
+                      <!-- Excludes shaded slf4j to avoid conflicts when they are put into the fat jar -->
+                      <exclude>org/slf4j/**</exclude>
                     </excludes>
                   </filter>
                 </filters>
diff --git a/pom.xml b/pom.xml
index 3b90251..4815771 100644
--- a/pom.xml
+++ b/pom.xml
@@ -108,6 +108,7 @@
     <protobuf.version>3.6.1</protobuf.version>
     <codemodel.version>2.6</codemodel.version>
     <joda.version>2.10.5</joda.version>
+    <javax.el.version>3.0.0</javax.el.version>
   </properties>
 
   <scm>
@@ -1806,6 +1807,16 @@
         <artifactId>joda-time</artifactId>
         <version>${joda.version}</version>
       </dependency>
+      <dependency>
+        <groupId>javax.el</groupId>
+        <artifactId>javax.el-api</artifactId>
+        <version>${javax.el.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.glassfish</groupId>
+        <artifactId>javax.el</artifactId>
+        <version>${javax.el.version}</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
 


[drill] 07/12: DRILL-7481: Fix raw type warnings in Iceberg Metastore and related classes

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

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

commit 2e8619c952449a10fc9aae3722a62346dcd154db
Author: Arina Ielchiieva <ar...@gmail.com>
AuthorDate: Wed Dec 11 12:39:29 2019 +0200

    DRILL-7481: Fix raw type warnings in Iceberg Metastore and related classes
    
    closes #1924
---
 .../apache/drill/exec/store/ischema/RecordCollector.java |  2 +-
 .../org/apache/drill/exec/store/ischema/Records.java     |  4 ++--
 .../metastore/iceberg/schema/IcebergTableSchema.java     |  2 +-
 .../iceberg/transform/FilterExpressionVisitor.java       | 16 ++++++++--------
 .../metastore/components/tables/BasicTablesRequests.java |  2 +-
 .../drill/metastore/expressions/FilterExpression.java    | 16 ++++++++--------
 6 files changed, 21 insertions(+), 21 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/RecordCollector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/RecordCollector.java
index 48c85bf..1d63738 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/RecordCollector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/RecordCollector.java
@@ -345,7 +345,7 @@ public interface RecordCollector {
 
           String tableName = table.getTableInfo().name();
           if (filterEvaluator.shouldVisitColumn(schemaPath, tableName, columnName)) {
-            ColumnStatistics columnStatistics =
+            ColumnStatistics<?> columnStatistics =
               table.getColumnStatistics(SchemaPath.parseFromString(columnName));
             records.add(new Records.Column(IS_CATALOG_NAME, schemaPath, tableName, columnName,
               column, columnStatistics, currentIndex, isNested));
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java
index 09f8cee..acb7cb7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java
@@ -577,7 +577,7 @@ public class Records {
     }
 
     public Column(String catalog, String schemaName, String tableName, String columnName,
-                  ColumnMetadata columnMetadata, ColumnStatistics columnStatistics, int index,
+                  ColumnMetadata columnMetadata, ColumnStatistics<?> columnStatistics, int index,
                   boolean isNested) {
       this.TABLE_CATALOG = catalog;
       this.TABLE_SCHEMA = schemaName;
@@ -813,4 +813,4 @@ public class Records {
           .toInstant());
     }
   }
-}
\ No newline at end of file
+}
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/schema/IcebergTableSchema.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/schema/IcebergTableSchema.java
index 04665a9..832998a 100644
--- a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/schema/IcebergTableSchema.java
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/schema/IcebergTableSchema.java
@@ -146,7 +146,7 @@ public class IcebergTableSchema {
     if (!(type instanceof Class)) {
       throw new IcebergMetastoreException("Unexpected generics type: " + type.getTypeName());
     }
-    Class typeArgument = (Class) type;
+    Class<?> typeArgument = (Class<?>) type;
     String typeSimpleName = typeArgument.getSimpleName().toLowerCase();
     org.apache.iceberg.types.Type icebergType = JAVA_TO_ICEBERG_TYPE_MAP.get(typeSimpleName);
     if (icebergType == null) {
diff --git a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/FilterExpressionVisitor.java b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/FilterExpressionVisitor.java
index cad02d5..b582b05 100644
--- a/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/FilterExpressionVisitor.java
+++ b/metastore/iceberg-metastore/src/main/java/org/apache/drill/metastore/iceberg/transform/FilterExpressionVisitor.java
@@ -40,42 +40,42 @@ public class FilterExpressionVisitor implements FilterExpression.Visitor<Express
   }
 
   @Override
-  public Expression visit(SimplePredicate.Equal expression) {
+  public Expression visit(SimplePredicate.Equal<?> expression) {
     return Expressions.equal(expression.reference(), expression.value());
   }
 
   @Override
-  public Expression visit(SimplePredicate.NotEqual expression) {
+  public Expression visit(SimplePredicate.NotEqual<?> expression) {
     return Expressions.notEqual(expression.reference(), expression.value());
   }
 
   @Override
-  public Expression visit(SimplePredicate.LessThan expression) {
+  public Expression visit(SimplePredicate.LessThan<?> expression) {
     return Expressions.lessThan(expression.reference(), expression.value());
   }
 
   @Override
-  public Expression visit(SimplePredicate.LessThanOrEqual expression) {
+  public Expression visit(SimplePredicate.LessThanOrEqual<?> expression) {
     return Expressions.lessThanOrEqual(expression.reference(), expression.value());
   }
 
   @Override
-  public Expression visit(SimplePredicate.GreaterThan expression) {
+  public Expression visit(SimplePredicate.GreaterThan<?> expression) {
     return Expressions.greaterThan(expression.reference(), expression.value());
   }
 
   @Override
-  public Expression visit(SimplePredicate.GreaterThanOrEqual expression) {
+  public Expression visit(SimplePredicate.GreaterThanOrEqual<?> expression) {
     return Expressions.greaterThanOrEqual(expression.reference(), expression.value());
   }
 
   @Override
-  public Expression visit(ListPredicate.In expression) {
+  public Expression visit(ListPredicate.In<?> expression) {
     return toInExpression(expression.reference(), expression.values());
   }
 
   @Override
-  public Expression visit(ListPredicate.NotIn expression) {
+  public Expression visit(ListPredicate.NotIn<?> expression) {
     Expression in = toInExpression(expression.reference(), expression.values());
     return Expressions.not(in);
   }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/BasicTablesRequests.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/BasicTablesRequests.java
index 84010e5..fbee2d8 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/BasicTablesRequests.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/BasicTablesRequests.java
@@ -819,7 +819,7 @@ public class BasicTablesRequests {
         }
 
         if (value instanceof List) {
-          List<?> list = (List) value;
+          List<?> list = (List<?>) value;
           if (list.isEmpty()) {
             return;
           }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/FilterExpression.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/FilterExpression.java
index 45b75db..784a892 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/FilterExpression.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/expressions/FilterExpression.java
@@ -58,14 +58,14 @@ public interface FilterExpression {
    */
   interface Visitor<T> {
 
-    T visit(SimplePredicate.Equal expression);
-    T visit(SimplePredicate.NotEqual expression);
-    T visit(SimplePredicate.LessThan expression);
-    T visit(SimplePredicate.LessThanOrEqual expression);
-    T visit(SimplePredicate.GreaterThan expression);
-    T visit(SimplePredicate.GreaterThanOrEqual expression);
-    T visit(ListPredicate.In expression);
-    T visit(ListPredicate.NotIn expression);
+    T visit(SimplePredicate.Equal<?> expression);
+    T visit(SimplePredicate.NotEqual<?> expression);
+    T visit(SimplePredicate.LessThan<?> expression);
+    T visit(SimplePredicate.LessThanOrEqual<?> expression);
+    T visit(SimplePredicate.GreaterThan<?> expression);
+    T visit(SimplePredicate.GreaterThanOrEqual<?> expression);
+    T visit(ListPredicate.In<?> expression);
+    T visit(ListPredicate.NotIn<?> expression);
     T visit(IsPredicate.IsNull expression);
     T visit(IsPredicate.IsNotNull expression);
     T visit(SingleExpressionPredicate.Not expression);


[drill] 01/12: DRILL-6332: Allow to provide two-component Kerberos principals

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

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

commit b2e3ba8b97848fadf9f215d3d2abcb2ea6d68165
Author: Stefan Hammer <st...@at.bosch.com>
AuthorDate: Fri Dec 13 13:38:35 2019 +0100

    DRILL-6332: Allow to provide two-component Kerberos principals
    
    closes #1931
---
 .../java/org/apache/drill/common/KerberosUtil.java | 27 ++++++++++++++--------
 .../apache/drill/exec/server/BootStrapContext.java |  7 +++---
 2 files changed, 22 insertions(+), 12 deletions(-)

diff --git a/common/src/main/java/org/apache/drill/common/KerberosUtil.java b/common/src/main/java/org/apache/drill/common/KerberosUtil.java
index 387f43d..368e493 100644
--- a/common/src/main/java/org/apache/drill/common/KerberosUtil.java
+++ b/common/src/main/java/org/apache/drill/common/KerberosUtil.java
@@ -38,29 +38,38 @@ public final class KerberosUtil {
    * Returns principal of format primary/instance@REALM.
    *
    * @param primary non-null primary component
-   * @param instance non-null instance component
+   * @param instance non-null instance component, can be empty string
    * @param realm non-null realm component
-   * @return principal of format primary/instance@REALM
+   * @return principal of format primary/instance@REALM or primary@REALM
    */
   public static String getPrincipalFromParts(final String primary, final String instance, final String realm) {
-    return checkNotNull(primary) + "/" +
-        checkNotNull(instance) + "@" +
-        checkNotNull(realm);
+    checkNotNull(primary);
+    checkNotNull(realm);
+
+    return primary +
+        ((instance != "") ? "/" + instance : "")
+        + "@" + realm;
   }
 
   /**
-   * Expects principal of the format primary/instance@REALM.
+   * Expects principal of the format primary/instance@REALM or primary@REALM.
    *
    * @param principal principal
    * @return components
    */
   public static String[] splitPrincipalIntoParts(final String principal) {
     final String[] components = principal.split("[/@]");
-    checkState(components.length == 3);
+    checkState(components.length < 4);
+    checkState(components.length > 1);
     checkNotNull(components[0]);
     checkNotNull(components[1]);
-    checkNotNull(components[2]);
-    return components;
+
+    if (components.length == 2) {
+      return new String[] { components[0], "", components[1] };
+    } else {
+      checkNotNull(components[2]);
+      return components;
+    }
   }
 
   public static String canonicalizeInstanceName(String instanceName, final String canonicalName) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
index 466dc14..74a5d96 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
@@ -135,10 +135,11 @@ public class BootStrapContext implements AutoCloseable {
         final String parts[] = KerberosUtil.splitPrincipalIntoParts(principal);
         if (parts.length != 3) {
           throw new DrillbitStartupException(
-              String.format("Invalid %s, Drill service principal must be of format: primary/instance@REALM",
-                ExecConstants.SERVICE_PRINCIPAL));
+            String.format("Invalid %s, Drill service principal must be of format 'primary/instance@REALM' or 'primary@REALM'",
+              ExecConstants.SERVICE_PRINCIPAL));
         }
-        parts[1] = KerberosUtil.canonicalizeInstanceName(parts[1], hostName);
+
+        parts[1] = (parts[1] == "") ? "" : KerberosUtil.canonicalizeInstanceName(parts[1], hostName);
 
         final String canonicalizedPrincipal = KerberosUtil.getPrincipalFromParts(parts[0], parts[1], parts[2]);
         final String keytab = config.getString(ExecConstants.SERVICE_KEYTAB_LOCATION);


[drill] 11/12: DRILL-7484: Malware found in the Drill test folder

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

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

commit 68dd10a58e59b784f08cd13a3fa5626282eedc2b
Author: Charles Givre <cg...@apache.org>
AuthorDate: Mon Dec 16 08:14:48 2019 -0500

    DRILL-7484: Malware found in the Drill test folder
    
    closes #1934
---
 .../drill/exec/store/pcap/TestSessionizePCAP.java  | 168 +++++++++++++++------
 .../test/resources/store/pcap/attack-trace.pcap    | Bin 189103 -> 0 bytes
 .../test/resources/store/pcap/dataFromRemote.txt   |   1 +
 .../src/test/resources/store/pcap/http.pcap        | Bin 0 -> 25803 bytes
 4 files changed, 121 insertions(+), 48 deletions(-)

diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestSessionizePCAP.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestSessionizePCAP.java
index 8c2818d..fe1c3d5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestSessionizePCAP.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcap/TestSessionizePCAP.java
@@ -18,12 +18,20 @@
 
 package org.apache.drill.exec.store.pcap;
 
-
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
+import org.apache.drill.exec.record.metadata.SchemaBuilder;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.test.ClusterFixture;
 import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
+import org.apache.drill.test.rowSet.RowSetComparison;
 import org.joda.time.Period;
+
+import java.io.IOException;
+import java.nio.file.Files;
 import java.nio.file.Paths;
-import java.time.LocalDateTime;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import java.time.format.DateTimeFormatter;
@@ -47,61 +55,125 @@ public class TestSessionizePCAP extends ClusterTest {
 
   @Test
   public void testSessionizedStarQuery() throws Exception {
-    String sql = "SELECT * FROM cp.`/store/pcap/attack-trace.pcap` WHERE src_port=1821 AND dst_port=445";
-
-    testBuilder()
-      .sqlQuery(sql)
-      .ordered()
-      .baselineColumns("session_start_time", "session_end_time", "session_duration", "total_packet_count", "connection_time", "src_ip", "dst_ip", "src_port", "dst_port",
-        "src_mac_address", "dst_mac_address", "tcp_session", "is_corrupt", "data_from_originator", "data_from_remote", "data_volume_from_origin",
-        "data_volume_from_remote", "packet_count_from_origin", "packet_count_from_remote")
-      .baselineValues(LocalDateTime.parse("2009-04-20T03:28:28.374", formatter),
-        LocalDateTime.parse("2009-04-20T03:28:28.508", formatter),
-        Period.parse("PT0.134S"), 4,
-        Period.parse("PT0.119S"),
-        "98.114.205.102",
-        "192.150.11.111",
-        1821, 445,
-        "00:08:E2:3B:56:01",
-        "00:30:48:62:4E:4A",
-        -8791568836279708938L,
-        false,
-        "........I....>...>..........Ib...<...<..........I....>...>", "", 62,0, 3, 1)
-      .go();
+    String sql = "SELECT * FROM cp.`/store/pcap/http.pcap`";
+    String dataFromRemote = readAFileIntoString(dirTestWatcher.getRootDir().getAbsolutePath() + "/store/pcap/dataFromRemote.txt");
+
+    QueryBuilder q = client.queryBuilder().sql(sql);
+    RowSet results = q.rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+      .addNullable("src_ip", TypeProtos.MinorType.VARCHAR)
+      .addNullable("dst_ip", TypeProtos.MinorType.VARCHAR)
+      .addNullable("src_port", TypeProtos.MinorType.INT)
+      .addNullable("dst_port", TypeProtos.MinorType.INT)
+      .addNullable("src_mac_address", TypeProtos.MinorType.VARCHAR)
+      .addNullable("dst_mac_address", TypeProtos.MinorType.VARCHAR)
+      .addNullable("session_start_time", TypeProtos.MinorType.TIMESTAMP)
+      .addNullable("session_end_time", TypeProtos.MinorType.TIMESTAMP)
+      .addNullable("session_duration", TypeProtos.MinorType.INTERVAL)
+      .addNullable("total_packet_count", TypeProtos.MinorType.INT)
+      .addNullable("data_volume_from_origin", TypeProtos.MinorType.INT)
+      .addNullable("data_volume_from_remote", TypeProtos.MinorType.INT)
+      .addNullable("packet_count_from_origin", TypeProtos.MinorType.INT)
+      .addNullable("packet_count_from_remote", TypeProtos.MinorType.INT)
+      .addNullable("connection_time", TypeProtos.MinorType.INTERVAL)
+      .addNullable("tcp_session", TypeProtos.MinorType.BIGINT)
+      .addNullable("is_corrupt", TypeProtos.MinorType.BIT)
+      .addNullable("data_from_originator", TypeProtos.MinorType.VARCHAR)
+      .addNullable("data_from_remote", TypeProtos.MinorType.VARCHAR)
+      .buildSchema();
+
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+      .addRow(
+        "145.254.160.237",
+        "65.208.228.223",
+        3372, 80,
+        "00:00:01:00:00:00",
+        "FE:FF:20:00:01:00",
+        1084443427311L,
+        1084443445216L,
+        Period.parse("PT17.905S"), 31,
+        437,18000,14, 17,
+        Period.parse("PT0.911S"),
+        -789689725566200012L, false,
+        "r-Agent: Mozilla/5.0 (Windows; U; Windows NT 5.1; en-US; rv:1.6) Gecko/20040113..Accept: text/xml,application/xml,application/xhtml+xml,text/html;q=0.9,text/plain;q=0.8,image/png,image/jpeg,image/gif;q=0.2,*/*;q=0.1..Accept-Language: en-us,en;q=0.5..Accept-Encoding: gzip,deflate..Accept-Charset: ISO-8859-1,utf-8;q=0.7,*;q=0.7..Keep-Alive: 300..Connection: keep-alive..Referer: http://www.ethereal.com/development.html....$K.@....6...6",
+        dataFromRemote
+        )
+      .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
   }
 
   @Test
   public void testSessionizedSpecificQuery() throws Exception {
-    String sql = "SELECT session_start_time, session_end_time,session_duration, total_packet_count, connection_time, src_ip, dst_ip, src_port, dst_port, src_mac_address, dst_mac_address, tcp_session, " +
-      "is_corrupt, data_from_originator, data_from_remote, data_volume_from_origin, data_volume_from_remote, packet_count_from_origin, packet_count_from_remote " +
-      "FROM cp.`/store/pcap/attack-trace.pcap` WHERE src_port=1821 AND dst_port=445";
-
-    testBuilder()
-      .sqlQuery(sql)
-      .ordered()
-      .baselineColumns("session_start_time", "session_end_time", "session_duration", "total_packet_count", "connection_time", "src_ip", "dst_ip", "src_port", "dst_port",
-        "src_mac_address", "dst_mac_address", "tcp_session", "is_corrupt", "data_from_originator", "data_from_remote", "data_volume_from_origin",
-        "data_volume_from_remote", "packet_count_from_origin", "packet_count_from_remote")
-      .baselineValues(LocalDateTime.parse("2009-04-20T03:28:28.374", formatter),
-        LocalDateTime.parse("2009-04-20T03:28:28.508", formatter),
-        Period.parse("PT0.134S"), 4,
-        Period.parse("PT0.119S"),
-        "98.114.205.102",
-        "192.150.11.111",
-        1821, 445,
-        "00:08:E2:3B:56:01",
-        "00:30:48:62:4E:4A",
-        -8791568836279708938L,
-        false,
-        "........I....>...>..........Ib...<...<..........I....>...>", "", 62,0, 3, 1)
-      .go();
+    String sql = "SELECT src_ip, dst_ip, src_port, dst_port, src_mac_address, dst_mac_address," +
+      "session_start_time, session_end_time, session_duration, total_packet_count, data_volume_from_origin, data_volume_from_remote," +
+      "packet_count_from_origin, packet_count_from_remote, connection_time, tcp_session, is_corrupt, data_from_originator, data_from_remote " +
+      "FROM cp.`/store/pcap/http.pcap`";
+
+    String dataFromRemote = readAFileIntoString(dirTestWatcher.getRootDir().getAbsolutePath() + "/store/pcap/dataFromRemote.txt");
+
+    QueryBuilder q = client.queryBuilder().sql(sql);
+    RowSet results = q.rowSet();
+
+    TupleMetadata expectedSchema = new SchemaBuilder()
+      .addNullable("src_ip", TypeProtos.MinorType.VARCHAR)
+      .addNullable("dst_ip", TypeProtos.MinorType.VARCHAR)
+      .addNullable("src_port", TypeProtos.MinorType.INT)
+      .addNullable("dst_port", TypeProtos.MinorType.INT)
+      .addNullable("src_mac_address", TypeProtos.MinorType.VARCHAR)
+      .addNullable("dst_mac_address", TypeProtos.MinorType.VARCHAR)
+      .addNullable("session_start_time", TypeProtos.MinorType.TIMESTAMP)
+      .addNullable("session_end_time", TypeProtos.MinorType.TIMESTAMP)
+      .addNullable("session_duration", TypeProtos.MinorType.INTERVAL)
+      .addNullable("total_packet_count", TypeProtos.MinorType.INT)
+      .addNullable("data_volume_from_origin", TypeProtos.MinorType.INT)
+      .addNullable("data_volume_from_remote", TypeProtos.MinorType.INT)
+      .addNullable("packet_count_from_origin", TypeProtos.MinorType.INT)
+      .addNullable("packet_count_from_remote", TypeProtos.MinorType.INT)
+      .addNullable("connection_time", TypeProtos.MinorType.INTERVAL)
+      .addNullable("tcp_session", TypeProtos.MinorType.BIGINT)
+      .addNullable("is_corrupt", TypeProtos.MinorType.BIT)
+      .addNullable("data_from_originator", TypeProtos.MinorType.VARCHAR)
+      .addNullable("data_from_remote", TypeProtos.MinorType.VARCHAR)
+      .buildSchema();
+
+    RowSet expected = new RowSetBuilder(client.allocator(), expectedSchema)
+      .addRow(
+        "145.254.160.237",
+        "65.208.228.223",
+        3372, 80,
+        "00:00:01:00:00:00",
+        "FE:FF:20:00:01:00",
+        1084443427311L,
+        1084443445216L,
+        Period.parse("PT17.905S"), 31,
+        437,18000,14, 17,
+        Period.parse("PT0.911S"),
+        -789689725566200012L, false,
+        "r-Agent: Mozilla/5.0 (Windows; U; Windows NT 5.1; en-US; rv:1.6) Gecko/20040113..Accept: text/xml,application/xml,application/xhtml+xml,text/html;q=0.9,text/plain;q=0.8,image/png,image/jpeg,image/gif;q=0.2,*/*;q=0.1..Accept-Language: en-us,en;q=0.5..Accept-Encoding: gzip,deflate..Accept-Charset: ISO-8859-1,utf-8;q=0.7,*;q=0.7..Keep-Alive: 300..Connection: keep-alive..Referer: http://www.ethereal.com/development.html....$K.@....6...6",
+        dataFromRemote
+      )
+      .build();
+
+    new RowSetComparison(expected).verifyAndClearAll(results);
   }
 
   @Test
   public void testSerDe() throws Exception {
-    String sql = "SELECT COUNT(*) FROM cp.`/store/pcap/attack-trace.pcap`";
+    String sql = "SELECT COUNT(*) FROM cp.`/store/pcap/http.pcap`";
     String plan = queryBuilder().sql(sql).explainJson();
     long cnt = queryBuilder().physical(plan).singletonLong();
-    assertEquals("Counts should match", 5L, cnt);
+    assertEquals("Counts should match", 1L, cnt);
+  }
+
+  /**
+   * Helper function to read a file into a String.
+   * @param filePath Input file which is to be read into a String
+   * @return String The text content of the file.
+   * @throws IOException If the file is unreachable or unreadable, throw IOException.
+   */
+  private static String readAFileIntoString(String filePath) throws IOException {
+    return new String(Files.readAllBytes(Paths.get(filePath)));
   }
 }
diff --git a/exec/java-exec/src/test/resources/store/pcap/attack-trace.pcap b/exec/java-exec/src/test/resources/store/pcap/attack-trace.pcap
deleted file mode 100644
index 68e1fff..0000000
Binary files a/exec/java-exec/src/test/resources/store/pcap/attack-trace.pcap and /dev/null differ
diff --git a/exec/java-exec/src/test/resources/store/pcap/dataFromRemote.txt b/exec/java-exec/src/test/resources/store/pcap/dataFromRemote.txt
new file mode 100644
index 0000000..cde5c69
--- /dev/null
+++ b/exec/java-exec/src/test/resources/store/pcap/dataFromRemote.txt
@@ -0,0 +1 @@
+10:17:12 GMT..Server: Apache..Last-Modified: Tue, 20 Apr 2004 13:17:00 GMT..ETag: "9a01a-4696-7e354b00"..Accept-Ranges: bytes..Content-Length: 18070..Keep-Alive: timeout=15, max=100..Connection: Keep-Alive..Content-Type: text/html; charset=ISO-8859-1....<?xml version="1.0" encoding="UTF-8"?>.<!DOCTYPE html.  PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN".  "DTD/xhtml1-strict.dtd">.<html xmlns="http://www.w3.org/1999/xhtml" xml:lang="en" lang="en">.  <head>.    <title>Ethereal: Download</title [...]
\ No newline at end of file
diff --git a/exec/java-exec/src/test/resources/store/pcap/http.pcap b/exec/java-exec/src/test/resources/store/pcap/http.pcap
new file mode 100644
index 0000000..54f6f29
Binary files /dev/null and b/exec/java-exec/src/test/resources/store/pcap/http.pcap differ


[drill] 12/12: DRILL-7486: Refactor row set reader builders

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

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

commit 5339fc23eb1b177bbc20ed74637e6b11e3ffa803
Author: Paul Rogers <pa...@yahoo.com>
AuthorDate: Thu Dec 12 17:37:53 2019 -0800

    DRILL-7486: Refactor row set reader builders
    
    Moves reader building code into a shared location, independent
    of the RowSet class. Allows readers to be built from a
    VectorContainer in addition to a row set.
    
    closes #1928
---
 .../physical/resultSet/impl/WriterIndexImpl.java   |  2 +-
 ...stractReaderBuilder.java => ReaderBuilder.java} | 29 ++++++++++++-
 ...eReaderBuilder.java => HyperReaderBuilder.java} | 49 +++++++++++----------
 .../resultSet/model/single/DirectRowIndex.java     |  5 ++-
 ...ReaderBuilder.java => SimpleReaderBuilder.java} | 50 ++++++++++++++++++++--
 .../exec/physical/rowSet/AbstractSingleRowSet.java | 19 ++------
 .../drill/exec/physical/rowSet/DirectRowSet.java   |  2 +-
 .../HyperRowIndex.java}                            | 27 +++++++-----
 .../exec/physical/rowSet/HyperRowSetImpl.java      | 15 ++-----
 .../IndirectRowIndex.java}                         | 24 +++++++----
 .../drill/exec/physical/rowSet/IndirectRowSet.java | 35 +++------------
 .../exec/physical/rowSet/RowSetFormatter.java      |  5 ---
 12 files changed, 148 insertions(+), 114 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/WriterIndexImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/WriterIndexImpl.java
index ce830d7..fe73cbb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/WriterIndexImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/impl/WriterIndexImpl.java
@@ -39,7 +39,7 @@ import org.apache.drill.exec.vector.accessor.ColumnWriterIndex;
 class WriterIndexImpl implements ColumnWriterIndex {
 
   private final ResultSetLoader rsLoader;
-  private int rowIndex = 0;
+  private int rowIndex;
 
   public WriterIndexImpl(ResultSetLoader rsLoader) {
     this.rsLoader = rsLoader;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/AbstractReaderBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/ReaderBuilder.java
similarity index 60%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/AbstractReaderBuilder.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/ReaderBuilder.java
index ea8e11b..ae217d3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/AbstractReaderBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/ReaderBuilder.java
@@ -18,6 +18,12 @@
 package org.apache.drill.exec.physical.resultSet.model;
 
 import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.physical.impl.protocol.BatchAccessor;
+import org.apache.drill.exec.physical.resultSet.model.hyper.HyperReaderBuilder;
+import org.apache.drill.exec.physical.resultSet.model.single.SimpleReaderBuilder;
+import org.apache.drill.exec.physical.rowSet.RowSetReaderImpl;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
 import org.apache.drill.exec.vector.accessor.reader.AbstractObjectReader;
 import org.apache.drill.exec.vector.accessor.reader.ArrayReaderImpl;
@@ -25,7 +31,27 @@ import org.apache.drill.exec.vector.accessor.reader.BaseScalarReader;
 import org.apache.drill.exec.vector.accessor.reader.ColumnReaderFactory;
 import org.apache.drill.exec.vector.accessor.reader.VectorAccessor;
 
-public abstract class AbstractReaderBuilder {
+public abstract class ReaderBuilder {
+
+  public static RowSetReaderImpl buildReader(BatchAccessor batch) {
+    if (batch.schema().getSelectionVectorMode() == SelectionVectorMode.FOUR_BYTE) {
+      try {
+        return HyperReaderBuilder.build(batch);
+      } catch (SchemaChangeException e) {
+
+        // The caller is responsible for ensuring that the hyper-batch
+        // has a consistent schema. If it is possible that the schema is
+        // inconsistent, then call the build() method directory and
+        // "do the right thing", which is pretty much to fail, as a
+        // hyper-batch is a very awkward place to discover an inconsistent
+        // schema.
+
+        throw new IllegalStateException("Hyper-batch contains an inconsistent schema", e);
+      }
+    } else {
+      return SimpleReaderBuilder.build(batch);
+    }
+  }
 
   protected AbstractObjectReader buildScalarReader(VectorAccessor va, ColumnMetadata schema) {
     BaseScalarReader scalarReader = ColumnReaderFactory.buildColumnReader(va);
@@ -41,5 +67,4 @@ public abstract class AbstractReaderBuilder {
       throw new UnsupportedOperationException(mode.toString());
     }
   }
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/hyper/BaseReaderBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/hyper/HyperReaderBuilder.java
similarity index 85%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/hyper/BaseReaderBuilder.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/hyper/HyperReaderBuilder.java
index c83f018..533d6c1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/hyper/BaseReaderBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/hyper/HyperReaderBuilder.java
@@ -23,8 +23,10 @@ import java.util.List;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.physical.resultSet.model.AbstractReaderBuilder;
-import org.apache.drill.exec.physical.resultSet.model.ReaderIndex;
+import org.apache.drill.exec.physical.impl.protocol.BatchAccessor;
+import org.apache.drill.exec.physical.resultSet.model.ReaderBuilder;
+import org.apache.drill.exec.physical.rowSet.HyperRowIndex;
+import org.apache.drill.exec.physical.rowSet.RowSetReaderImpl;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.metadata.ColumnMetadata;
@@ -33,7 +35,6 @@ import org.apache.drill.exec.record.metadata.VariantMetadata;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.accessor.ColumnReaderIndex;
-import org.apache.drill.exec.vector.accessor.impl.AccessorUtilities;
 import org.apache.drill.exec.vector.accessor.reader.AbstractObjectReader;
 import org.apache.drill.exec.vector.accessor.reader.ArrayReaderImpl;
 import org.apache.drill.exec.vector.accessor.reader.MapReader;
@@ -71,31 +72,33 @@ import org.apache.drill.exec.vector.accessor.reader.VectorAccessors.BaseHyperVec
  * the outer vector accessor.)
  */
 
-public abstract class BaseReaderBuilder extends AbstractReaderBuilder {
+public class HyperReaderBuilder extends ReaderBuilder {
 
-  /**
-   * Read-only row index into the hyper row set with batch and index
-   * values mapping via an SV4.
-   */
-
-  public static class HyperRowIndex extends ReaderIndex {
+  private static final HyperReaderBuilder INSTANCE = new HyperReaderBuilder();
 
-    private final SelectionVector4 sv4;
+  private HyperReaderBuilder() { }
 
-    public HyperRowIndex(SelectionVector4 sv4) {
-      super(sv4.getCount());
-      this.sv4 = sv4;
-    }
+  public static RowSetReaderImpl build(VectorContainer container, TupleMetadata schema, SelectionVector4 sv4) {
+    HyperRowIndex rowIndex = new HyperRowIndex(sv4);
+    return new RowSetReaderImpl(schema, rowIndex,
+        INSTANCE.buildContainerChildren(container, schema));
+  }
 
-    @Override
-    public int offset() {
-      return AccessorUtilities.sv4Index(sv4.get(position));
-    }
+  /**
+   * Build a hyper-batch reader given a batch accessor.
+   *
+   * @param batch wrapper which provides the container and SV4
+   * @return a row set reader for the hyper-batch
+   * @throws SchemaChangeException if the individual batches have
+   * inconsistent schemas (say, a column in batch 1 is an INT, but in
+   * batch 2 it is a VARCHAR)
+   */
 
-    @Override
-    public int hyperVectorIndex( ) {
-      return AccessorUtilities.sv4Batch(sv4.get(position));
-    }
+  public static RowSetReaderImpl build(BatchAccessor batch) throws SchemaChangeException {
+    VectorContainer container = batch.container();
+    return build(container,
+        new HyperSchemaInference().infer(container),
+        batch.selectionVector4());
   }
 
   /**
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/DirectRowIndex.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/DirectRowIndex.java
index ea64ac1..8fada11 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/DirectRowIndex.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/DirectRowIndex.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.physical.resultSet.model.single;
 
 import org.apache.drill.exec.physical.resultSet.model.ReaderIndex;
+import org.apache.drill.exec.record.VectorContainer;
 
 /**
  * Reader index that points directly to each row in the row set.
@@ -28,8 +29,8 @@ import org.apache.drill.exec.physical.resultSet.model.ReaderIndex;
 
 public class DirectRowIndex extends ReaderIndex {
 
-  public DirectRowIndex(int rowCount) {
-    super(rowCount);
+  public DirectRowIndex(VectorContainer container) {
+    super(container.getRecordCount());
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/BaseReaderBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/SimpleReaderBuilder.java
similarity index 81%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/BaseReaderBuilder.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/SimpleReaderBuilder.java
index 3bbe6ab..0d330d6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/BaseReaderBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/SimpleReaderBuilder.java
@@ -23,10 +23,18 @@ import java.util.List;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.exec.physical.resultSet.model.AbstractReaderBuilder;
+import org.apache.drill.exec.physical.impl.protocol.BatchAccessor;
 import org.apache.drill.exec.physical.resultSet.model.MetadataProvider;
+import org.apache.drill.exec.physical.resultSet.model.MetadataProvider.MetadataCreator;
+import org.apache.drill.exec.physical.resultSet.model.MetadataProvider.MetadataRetrieval;
 import org.apache.drill.exec.physical.resultSet.model.MetadataProvider.VectorDescrip;
+import org.apache.drill.exec.physical.resultSet.model.ReaderBuilder;
+import org.apache.drill.exec.physical.resultSet.model.ReaderIndex;
+import org.apache.drill.exec.physical.rowSet.IndirectRowIndex;
+import org.apache.drill.exec.physical.rowSet.RowSet;
+import org.apache.drill.exec.physical.rowSet.RowSetReaderImpl;
 import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.accessor.reader.AbstractObjectReader;
 import org.apache.drill.exec.vector.accessor.reader.AbstractScalarReader;
@@ -46,7 +54,7 @@ import org.apache.drill.exec.vector.complex.UnionVector;
  * <p>
  * Derived classes handle the details of the various kinds of readers.
  * Today there is a single subclass that builds (test-time)
- * {@link org.apache.drill.exec.physical.rowSet.RowSet} objects. The idea, however, is that we may eventually
+ * {@link RowSet} objects. The idea, however, is that we may eventually
  * want to create a "result set reader" for use in internal operators,
  * in parallel to the "result set loader". The result set reader would
  * handle a stream of incoming batches. The extant RowSet class handles
@@ -59,9 +67,43 @@ import org.apache.drill.exec.vector.complex.UnionVector;
  * quite complex.
  */
 
-public abstract class BaseReaderBuilder extends AbstractReaderBuilder {
+public class SimpleReaderBuilder extends ReaderBuilder {
 
-  protected List<AbstractObjectReader> buildContainerChildren(
+  private static final SimpleReaderBuilder INSTANCE = new SimpleReaderBuilder();
+
+  private SimpleReaderBuilder() { }
+
+  public static RowSetReaderImpl build(VectorContainer container,
+      TupleMetadata schema, ReaderIndex rowIndex) {
+    return new RowSetReaderImpl(schema, rowIndex,
+        INSTANCE.buildContainerChildren(container,
+           new MetadataRetrieval(schema)));
+  }
+
+  public static RowSetReaderImpl build(VectorContainer container, ReaderIndex rowIndex) {
+    MetadataCreator mdCreator = new MetadataCreator();
+    List<AbstractObjectReader> children = INSTANCE.buildContainerChildren(container,
+        mdCreator);
+    return new RowSetReaderImpl(mdCreator.tuple(), rowIndex, children);
+  }
+
+  public static RowSetReaderImpl build(BatchAccessor batch) {
+    return SimpleReaderBuilder.build(batch.container(),
+        readerIndex(batch));
+  }
+
+  public static ReaderIndex readerIndex(BatchAccessor batch) {
+    switch (batch.schema().getSelectionVectorMode()) {
+    case TWO_BYTE:
+      return new IndirectRowIndex(batch.selectionVector2());
+    case NONE:
+      return new DirectRowIndex(batch.container());
+    default:
+      throw new UnsupportedOperationException("Cannot use this method for a hyper-batch");
+    }
+  }
+
+  public List<AbstractObjectReader> buildContainerChildren(
       VectorContainer container, MetadataProvider mdProvider) {
     final List<AbstractObjectReader> readers = new ArrayList<>();
     for (int i = 0; i < container.getNumberOfColumns(); i++) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/AbstractSingleRowSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/AbstractSingleRowSet.java
index 398ed33..6d955c7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/AbstractSingleRowSet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/AbstractSingleRowSet.java
@@ -17,13 +17,12 @@
  */
 package org.apache.drill.exec.physical.rowSet;
 
-import org.apache.drill.exec.record.RecordBatchSizer;
 import org.apache.drill.exec.physical.resultSet.model.ReaderIndex;
-import org.apache.drill.exec.physical.resultSet.model.MetadataProvider.MetadataRetrieval;
-import org.apache.drill.exec.physical.resultSet.model.single.BaseReaderBuilder;
+import org.apache.drill.exec.physical.resultSet.model.single.SimpleReaderBuilder;
+import org.apache.drill.exec.physical.rowSet.RowSet.SingleRowSet;
+import org.apache.drill.exec.record.RecordBatchSizer;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.exec.physical.rowSet.RowSet.SingleRowSet;
 
 /**
  * Base class for row sets backed by a single record batch.
@@ -31,16 +30,6 @@ import org.apache.drill.exec.physical.rowSet.RowSet.SingleRowSet;
 
 public abstract class AbstractSingleRowSet extends AbstractRowSet implements SingleRowSet {
 
-  public static class RowSetReaderBuilder extends BaseReaderBuilder {
-
-    public RowSetReader buildReader(AbstractSingleRowSet rowSet, ReaderIndex rowIndex) {
-      TupleMetadata schema = rowSet.schema();
-      return new RowSetReaderImpl(schema, rowIndex,
-          buildContainerChildren(rowSet.container(),
-              new MetadataRetrieval(schema)));
-    }
-  }
-
   protected AbstractSingleRowSet(AbstractSingleRowSet rowSet) {
     super(rowSet.container, rowSet.schema);
   }
@@ -64,6 +53,6 @@ public abstract class AbstractSingleRowSet extends AbstractRowSet implements Sin
    * (non-map) vectors.
    */
   protected RowSetReader buildReader(ReaderIndex rowIndex) {
-    return new RowSetReaderBuilder().buildReader(this, rowIndex);
+    return SimpleReaderBuilder.build(container(), schema, rowIndex);
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/DirectRowSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/DirectRowSet.java
index 0164819..67dd7e5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/DirectRowSet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/DirectRowSet.java
@@ -122,7 +122,7 @@ public class DirectRowSet extends AbstractSingleRowSet implements ExtendableRowS
 
   @Override
   public RowSetReader reader() {
-    return buildReader(new DirectRowIndex(rowCount()));
+    return buildReader(new DirectRowIndex(container));
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/DirectRowIndex.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/HyperRowIndex.java
similarity index 58%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/DirectRowIndex.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/HyperRowIndex.java
index ea64ac1..1e1cd5b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/DirectRowIndex.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/HyperRowIndex.java
@@ -15,26 +15,33 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.resultSet.model.single;
+package org.apache.drill.exec.physical.rowSet;
 
 import org.apache.drill.exec.physical.resultSet.model.ReaderIndex;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.vector.accessor.impl.AccessorUtilities;
 
 /**
- * Reader index that points directly to each row in the row set.
- * This index starts with pointing to the -1st row, so that the
- * reader can require a <tt>next()</tt> for every row, including
- * the first. (This is the JDBC <tt>RecordSet</tt> convention.)
+ * Read-only row index into the hyper row set with batch and index
+ * values mapping via an SV4.
  */
 
-public class DirectRowIndex extends ReaderIndex {
+public class HyperRowIndex extends ReaderIndex {
 
-  public DirectRowIndex(int rowCount) {
-    super(rowCount);
+  private final SelectionVector4 sv4;
+
+  public HyperRowIndex(SelectionVector4 sv4) {
+    super(sv4.getCount());
+    this.sv4 = sv4;
   }
 
   @Override
-  public int offset() { return position; }
+  public int offset() {
+    return AccessorUtilities.sv4Index(sv4.get(position));
+  }
 
   @Override
-  public int hyperVectorIndex() { return 0; }
+  public int hyperVectorIndex( ) {
+    return AccessorUtilities.sv4Batch(sv4.get(position));
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/HyperRowSetImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/HyperRowSetImpl.java
index 31d12cd..0773d5a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/HyperRowSetImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/HyperRowSetImpl.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.physical.resultSet.model.hyper.BaseReaderBuilder;
+import org.apache.drill.exec.physical.resultSet.model.hyper.HyperReaderBuilder;
 import org.apache.drill.exec.physical.resultSet.model.hyper.HyperSchemaInference;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
@@ -44,16 +44,6 @@ import org.apache.drill.exec.physical.rowSet.RowSet.HyperRowSet;
 
 public class HyperRowSetImpl extends AbstractRowSet implements HyperRowSet {
 
-  public static class RowSetReaderBuilder extends BaseReaderBuilder {
-
-    public RowSetReader buildReader(HyperRowSet rowSet, SelectionVector4 sv4) {
-      TupleMetadata schema = rowSet.schema();
-      HyperRowIndex rowIndex = new HyperRowIndex(sv4);
-      return new RowSetReaderImpl(schema, rowIndex,
-          buildContainerChildren(rowSet.container(), schema));
-    }
-  }
-
   public static class HyperRowSetBuilderImpl implements HyperRowSetBuilder {
 
     private final BufferAllocator allocator;
@@ -95,6 +85,7 @@ public class HyperRowSetImpl extends AbstractRowSet implements HyperRowSet {
       for (VectorContainer container : batches) {
         hyperContainer.addBatch(container);
       }
+      hyperContainer.setRecordCount(totalRowCount);
 
       // TODO: This has a bug. If the hyperset has two batches with unions,
       // and the first union contains only VARCHAR, while the second contains
@@ -160,7 +151,7 @@ public class HyperRowSetImpl extends AbstractRowSet implements HyperRowSet {
 
   @Override
   public RowSetReader reader() {
-    return new RowSetReaderBuilder().buildReader(this, sv4);
+    return HyperReaderBuilder.build(container(), schema, sv4);
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/DirectRowIndex.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/IndirectRowIndex.java
similarity index 57%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/DirectRowIndex.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/IndirectRowIndex.java
index ea64ac1..8aa0770 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/resultSet/model/single/DirectRowIndex.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/IndirectRowIndex.java
@@ -15,25 +15,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.resultSet.model.single;
+package org.apache.drill.exec.physical.rowSet;
 
 import org.apache.drill.exec.physical.resultSet.model.ReaderIndex;
+import org.apache.drill.exec.record.selection.SelectionVector2;
 
 /**
- * Reader index that points directly to each row in the row set.
- * This index starts with pointing to the -1st row, so that the
- * reader can require a <tt>next()</tt> for every row, including
- * the first. (This is the JDBC <tt>RecordSet</tt> convention.)
+ * Reader index that points to each row indirectly through the
+ * selection vector. The {@link #offset()} method points to the
+ * actual data row, while the {@link #logicalIndex()} method gives
+ * the position relative to the indirection vector. That is,
+ * the position increases monotonically, but the index jumps
+ * around as specified by the indirection vector.
  */
 
-public class DirectRowIndex extends ReaderIndex {
+public class IndirectRowIndex extends ReaderIndex {
 
-  public DirectRowIndex(int rowCount) {
-    super(rowCount);
+  private final SelectionVector2 sv2;
+
+  public IndirectRowIndex(SelectionVector2 sv2) {
+    super(sv2.getCount());
+    this.sv2 = sv2;
   }
 
   @Override
-  public int offset() { return position; }
+  public int offset() { return sv2.getIndex(position); }
 
   @Override
   public int hyperVectorIndex() { return 0; }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/IndirectRowSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/IndirectRowSet.java
index cf7da3c..17bbf74 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/IndirectRowSet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/IndirectRowSet.java
@@ -17,18 +17,17 @@
  */
 package org.apache.drill.exec.physical.rowSet;
 
+import java.util.Collections;
+import java.util.Set;
+
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.physical.resultSet.model.ReaderIndex;
 import org.apache.drill.exec.physical.resultSet.model.single.SingleSchemaInference;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.RecordBatchSizer;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 
-import java.util.Collections;
-import java.util.Set;
-
 /**
  * Single row set coupled with an indirection (selection) vector,
  * specifically an SV2.
@@ -36,31 +35,6 @@ import java.util.Set;
 
 public class IndirectRowSet extends AbstractSingleRowSet {
 
-  /**
-   * Reader index that points to each row indirectly through the
-   * selection vector. The {@link #offset()} method points to the
-   * actual data row, while the {@link #logicalIndex()} method gives
-   * the position relative to the indirection vector. That is,
-   * the position increases monotonically, but the index jumps
-   * around as specified by the indirection vector.
-   */
-
-  private static class IndirectRowIndex extends ReaderIndex {
-
-    private final SelectionVector2 sv2;
-
-    public IndirectRowIndex(SelectionVector2 sv2) {
-      super(sv2.getCount());
-      this.sv2 = sv2;
-    }
-
-    @Override
-    public int offset() { return sv2.getIndex(position); }
-
-    @Override
-    public int hyperVectorIndex() { return 0; }
-  }
-
   private final SelectionVector2 sv2;
 
   private IndirectRowSet(VectorContainer container, SelectionVector2 sv2) {
@@ -117,7 +91,8 @@ public class IndirectRowSet extends AbstractSingleRowSet {
 
   @Override
   public RowSetReader reader() {
-    return buildReader(new IndirectRowIndex(getSv2()));
+    IndirectRowIndex index = new IndirectRowIndex(getSv2());
+    return buildReader(index);
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/RowSetFormatter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/RowSetFormatter.java
index 9bf9f44..eeb2236 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/RowSetFormatter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/RowSetFormatter.java
@@ -25,7 +25,6 @@ import org.apache.commons.io.output.StringBuilderWriter;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.exec.physical.impl.protocol.BatchAccessor;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 
@@ -61,10 +60,6 @@ public class RowSetFormatter {
     RowSets.wrap(batch).print();
   }
 
-  public static void print(RecordBatch batch) {
-    RowSets.wrap(batch).print();
-  }
-
   public static String toString(RowSet rowSet) {
     StringBuilderWriter out = new StringBuilderWriter();
     new RowSetFormatter(rowSet, out).write();


[drill] 08/12: DRILL-7479: Partial fixes for metadata parameterized type issues

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

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

commit 005f4d9ca4cec0646ce2dd15399603a57a86460d
Author: Paul Rogers <pa...@yahoo.com>
AuthorDate: Tue Dec 10 20:13:30 2019 -0800

    DRILL-7479: Partial fixes for metadata parameterized type issues
    
    See DRILL-7479 and DRILL-7480 for an explanation. Adds generic
    type parameters where needed to avoid the need to supporess
    warnings. However, type parameters are probably not needed
    at all and should be removed in the future for reasons explained
    in DRILL-7480.
    
    closes #1923
---
 .../drill/exec/expr/ComparisonPredicate.java       | 14 +++---
 .../org/apache/drill/exec/expr/IsPredicate.java    |  9 ++--
 .../apache/drill/exec/expr/StatisticsProvider.java | 18 ++++----
 .../MetastoreParquetTableMetadataProvider.java     |  8 ++--
 .../metastore/SimpleFileTableMetadataProvider.java |  8 ++--
 .../base/AbstractGroupScanWithMetadata.java        | 10 ++---
 .../impl/metadata/MetadataControllerBatch.java     | 50 ++++++++++------------
 .../impl/metadata/MetadataHandlerBatch.java        |  7 ++-
 .../drill/exec/planner/common/DrillStatsTable.java | 12 +++---
 .../planner/cost/DrillRelMdDistinctRowCount.java   |  2 +-
 .../exec/planner/cost/DrillRelMdSelectivity.java   |  9 ++--
 .../store/parquet/AbstractParquetGroupScan.java    |  4 +-
 .../parquet/AbstractParquetScanBatchCreator.java   |  6 +--
 .../store/parquet/BaseParquetMetadataProvider.java | 17 ++++----
 .../exec/store/parquet/FilterEvaluatorUtils.java   | 13 +++---
 .../drill/exec/store/parquet/ParquetGroupScan.java |  9 ++--
 .../store/parquet/ParquetGroupScanStatistics.java  | 13 +++---
 .../exec/store/parquet/ParquetPushDownFilter.java  |  8 ++--
 .../store/parquet/ParquetTableMetadataUtils.java   | 31 +++++++-------
 .../exec/sql/TestInfoSchemaWithMetastore.java      | 10 ++---
 .../drill/exec/sql/TestMetastoreCommands.java      | 44 ++++++++++---------
 .../exec/store/parquet/TestFileGenerator.java      | 29 -------------
 .../store/parquet/TestParquetFilterPushDown.java   | 24 +++++------
 .../components/tables/BasicTablesTransformer.java  |  3 ++
 .../components/tables/TableMetadataUnit.java       |  2 +
 .../metastore/exceptions/MetastoreException.java   |  6 ++-
 .../drill/metastore/metadata/BaseMetadata.java     | 28 ++++++------
 .../metastore/metadata/BaseTableMetadata.java      | 12 +++---
 .../apache/drill/metastore/metadata/Metadata.java  |  6 +--
 .../metadata/NonInterestingColumnsMetadata.java    | 13 +++---
 .../drill/metastore/metadata/TableMetadata.java    |  3 +-
 .../CollectableColumnStatisticsKind.java           |  2 +-
 .../metastore/statistics/ColumnStatistics.java     | 30 +++++++------
 .../metastore/statistics/ColumnStatisticsKind.java | 31 ++++++++------
 .../metastore/statistics/StatisticsHolder.java     | 12 +++---
 .../drill/metastore/util/TableMetadataUtils.java   | 44 ++++++++++++-------
 .../tables/TestTableMetadataUnitConversion.java    |  4 +-
 .../metastore/metadata/MetadataSerDeTest.java      | 16 +++----
 38 files changed, 287 insertions(+), 280 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ComparisonPredicate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ComparisonPredicate.java
index fa51780..3594a6d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ComparisonPredicate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ComparisonPredicate.java
@@ -88,11 +88,11 @@ public class ComparisonPredicate<C extends Comparable<C>> extends LogicalExpress
   @Override
   @SuppressWarnings("unchecked")
   public RowsMatch matches(StatisticsProvider<C> evaluator) {
-    ColumnStatistics leftStat = left.accept(evaluator, null);
+    ColumnStatistics<C> leftStat = (ColumnStatistics<C>) left.accept(evaluator, null);
     if (IsPredicate.isNullOrEmpty(leftStat)) {
       return RowsMatch.SOME;
     }
-    ColumnStatistics rightStat = right.accept(evaluator, null);
+    ColumnStatistics<C> rightStat = (ColumnStatistics<C>) right.accept(evaluator, null);
     if (IsPredicate.isNullOrEmpty(rightStat)) {
       return RowsMatch.SOME;
     }
@@ -112,9 +112,11 @@ public class ComparisonPredicate<C extends Comparable<C>> extends LogicalExpress
       int leftScale = left.getMajorType().getScale();
       int rightScale = right.getMajorType().getScale();
       if (leftScale > rightScale) {
-        rightStat = adjustDecimalStatistics(rightStat, leftScale - rightScale);
+        rightStat = (ColumnStatistics<C>) adjustDecimalStatistics(
+            (ColumnStatistics<BigInteger>) rightStat, leftScale - rightScale);
       } else if (leftScale < rightScale) {
-        leftStat = adjustDecimalStatistics(leftStat, rightScale - leftScale);
+        leftStat = (ColumnStatistics<C>) adjustDecimalStatistics(
+            (ColumnStatistics<BigInteger>) leftStat, rightScale - leftScale);
       }
     }
     return predicate.apply(leftStat, rightStat);
@@ -127,7 +129,7 @@ public class ComparisonPredicate<C extends Comparable<C>> extends LogicalExpress
    * @param scale adjustment scale
    * @return adjusted statistics
    */
-  private ColumnStatistics adjustDecimalStatistics(ColumnStatistics<BigInteger> statistics, int scale) {
+  private ColumnStatistics<BigInteger> adjustDecimalStatistics(ColumnStatistics<BigInteger> statistics, int scale) {
     BigInteger min = new BigDecimal(ColumnStatisticsKind.MIN_VALUE.getValueStatistic(statistics))
         .setScale(scale, RoundingMode.HALF_UP).unscaledValue();
     BigInteger max = new BigDecimal(ColumnStatisticsKind.MAX_VALUE.getValueStatistic(statistics))
@@ -139,7 +141,7 @@ public class ComparisonPredicate<C extends Comparable<C>> extends LogicalExpress
   /**
    * If one rowgroup contains some null values, change the RowsMatch.ALL into RowsMatch.SOME (null values should be discarded by filter)
    */
-  private static RowsMatch checkNull(ColumnStatistics leftStat, ColumnStatistics rightStat) {
+  private static RowsMatch checkNull(ColumnStatistics<?> leftStat, ColumnStatistics<?> rightStat) {
     return !IsPredicate.hasNoNulls(leftStat) || !IsPredicate.hasNoNulls(rightStat) ? RowsMatch.SOME : RowsMatch.ALL;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/IsPredicate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/IsPredicate.java
index aa0b9fd..def1d3c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/IsPredicate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/IsPredicate.java
@@ -63,7 +63,8 @@ public class IsPredicate<C extends Comparable<C>> extends LogicalExpressionBase
    */
   @Override
   public RowsMatch matches(StatisticsProvider<C> evaluator) {
-    ColumnStatistics<C> exprStat = expr.accept(evaluator, null);
+    @SuppressWarnings("unchecked")
+    ColumnStatistics<C> exprStat = (ColumnStatistics<C>) expr.accept(evaluator, null);
     return isNullOrEmpty(exprStat) ? RowsMatch.SOME : predicate.apply(exprStat, evaluator);
   }
 
@@ -71,7 +72,7 @@ public class IsPredicate<C extends Comparable<C>> extends LogicalExpressionBase
    * @param stat statistics object
    * @return <tt>true</tt> if the input stat object is null or has invalid statistics; false otherwise
    */
-  public static boolean isNullOrEmpty(ColumnStatistics stat) {
+  public static boolean isNullOrEmpty(ColumnStatistics<?> stat) {
     return stat == null
         || !stat.contains(ColumnStatisticsKind.MIN_VALUE)
         || !stat.contains(ColumnStatisticsKind.MAX_VALUE)
@@ -85,7 +86,7 @@ public class IsPredicate<C extends Comparable<C>> extends LogicalExpressionBase
    * If it contains some null values, then we change the RowsMatch.ALL into RowsMatch.SOME, which sya that maybe
    * some values (the null ones) should be disgarded.
    */
-  private static RowsMatch checkNull(ColumnStatistics exprStat) {
+  private static RowsMatch checkNull(ColumnStatistics<?> exprStat) {
     return hasNoNulls(exprStat) ? RowsMatch.ALL : RowsMatch.SOME;
   }
 
@@ -95,7 +96,7 @@ public class IsPredicate<C extends Comparable<C>> extends LogicalExpressionBase
    * @param stat column statistics
    * @return <tt>true</tt> if the statistics does not have nulls and <tt>false</tt> otherwise
    */
-  static boolean hasNoNulls(ColumnStatistics stat) {
+  static boolean hasNoNulls(ColumnStatistics<?> stat) {
     return ColumnStatisticsKind.NULLS_COUNT.getFrom(stat) == 0;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/StatisticsProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/StatisticsProvider.java
index 5b3cfc8..1ab7579 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/StatisticsProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/StatisticsProvider.java
@@ -48,12 +48,12 @@ import java.util.EnumSet;
 import java.util.Map;
 import java.util.Set;
 
-public class StatisticsProvider<T extends Comparable<T>> extends AbstractExprVisitor<ColumnStatistics, Void, RuntimeException> {
+public class StatisticsProvider<T extends Comparable<T>> extends AbstractExprVisitor<ColumnStatistics<?>, Void, RuntimeException> {
 
-  private final Map<SchemaPath, ColumnStatistics> columnStatMap;
+  private final Map<SchemaPath, ColumnStatistics<?>> columnStatMap;
   private final long rowCount;
 
-  public StatisticsProvider(Map<SchemaPath, ColumnStatistics> columnStatMap, long rowCount) {
+  public StatisticsProvider(Map<SchemaPath, ColumnStatistics<?>> columnStatMap, long rowCount) {
     this.columnStatMap = columnStatMap;
     this.rowCount = rowCount;
   }
@@ -63,14 +63,14 @@ public class StatisticsProvider<T extends Comparable<T>> extends AbstractExprVis
   }
 
   @Override
-  public ColumnStatistics visitUnknown(LogicalExpression e, Void value) {
+  public ColumnStatistics<?> visitUnknown(LogicalExpression e, Void value) {
     // do nothing for the unknown expression
     return null;
   }
 
   @Override
-  public ColumnStatistics visitTypedFieldExpr(TypedFieldExpr typedFieldExpr, Void value) {
-    ColumnStatistics columnStatistics = columnStatMap.get(typedFieldExpr.getPath().getUnIndexed());
+  public ColumnStatistics<?> visitTypedFieldExpr(TypedFieldExpr typedFieldExpr, Void value) {
+    ColumnStatistics<?> columnStatistics = columnStatMap.get(typedFieldExpr.getPath().getUnIndexed());
     if (columnStatistics != null) {
       return columnStatistics;
     } else if (typedFieldExpr.getMajorType().equals(Types.OPTIONAL_INT)) {
@@ -132,7 +132,7 @@ public class StatisticsProvider<T extends Comparable<T>> extends AbstractExprVis
 
   @Override
   @SuppressWarnings("unchecked")
-  public ColumnStatistics visitFunctionHolderExpression(FunctionHolderExpression holderExpr, Void value) {
+  public ColumnStatistics<?> visitFunctionHolderExpression(FunctionHolderExpression holderExpr, Void value) {
     FuncHolder funcHolder = holderExpr.getHolder();
 
     if (!(funcHolder instanceof DrillSimpleFuncHolder)) {
@@ -143,7 +143,7 @@ public class StatisticsProvider<T extends Comparable<T>> extends AbstractExprVis
     String funcName = ((DrillSimpleFuncHolder) funcHolder).getRegisteredNames()[0];
 
     if (FunctionReplacementUtils.isCastFunction(funcName)) {
-      ColumnStatistics<T> stat = holderExpr.args.get(0).accept(this, null);
+      ColumnStatistics<T> stat = (ColumnStatistics<T>) holderExpr.args.get(0).accept(this, null);
       if (!IsPredicate.isNullOrEmpty(stat)) {
         return evalCastFunc(holderExpr, stat);
       }
@@ -151,7 +151,7 @@ public class StatisticsProvider<T extends Comparable<T>> extends AbstractExprVis
     return null;
   }
 
-  private ColumnStatistics evalCastFunc(FunctionHolderExpression holderExpr, ColumnStatistics<T> input) {
+  private ColumnStatistics<?> evalCastFunc(FunctionHolderExpression holderExpr, ColumnStatistics<T> input) {
     try {
       DrillSimpleFuncHolder funcHolder = (DrillSimpleFuncHolder) holderExpr.getHolder();
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/MetastoreParquetTableMetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/MetastoreParquetTableMetadataProvider.java
index 24736bf..acad916 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/MetastoreParquetTableMetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/MetastoreParquetTableMetadataProvider.java
@@ -85,7 +85,7 @@ public class MetastoreParquetTableMetadataProvider implements ParquetTableMetada
   private Multimap<Path, RowGroupMetadata> rowGroups;
   private NonInterestingColumnsMetadata nonInterestingColumnsMetadata;
   // stores builder to provide lazy init for fallback ParquetTableMetadataProvider
-  private ParquetFileTableMetadataProviderBuilder fallbackBuilder;
+  private final ParquetFileTableMetadataProviderBuilder fallbackBuilder;
   private ParquetTableMetadataProvider fallback;
 
   private MetastoreParquetTableMetadataProvider(List<ReadEntryWithPath> entries,
@@ -259,12 +259,12 @@ public class MetastoreParquetTableMetadataProvider implements ParquetTableMetada
     if (nonInterestingColumnsMetadata == null) {
       TupleMetadata schema = getTableMetadata().getSchema();
 
-      List<StatisticsHolder> statistics = Collections.singletonList(new StatisticsHolder<>(Statistic.NO_COLUMN_STATS, ColumnStatisticsKind.NULLS_COUNT));
+      List<StatisticsHolder<?>> statistics = Collections.singletonList(new StatisticsHolder<>(Statistic.NO_COLUMN_STATS, ColumnStatisticsKind.NULLS_COUNT));
 
       List<SchemaPath> columnPaths = SchemaUtil.getSchemaPaths(schema);
       List<SchemaPath> interestingColumns = getInterestingColumns(columnPaths);
       // populates statistics for non-interesting columns and columns for which statistics wasn't collected
-      Map<SchemaPath, ColumnStatistics> columnsStatistics = columnPaths.stream()
+      Map<SchemaPath, ColumnStatistics<?>> columnsStatistics = columnPaths.stream()
           .filter(schemaPath -> !interestingColumns.contains(schemaPath)
               || SchemaPathUtils.getColumnMetadata(schemaPath, schema).isArray())
           .collect(Collectors.toMap(
@@ -315,7 +315,7 @@ public class MetastoreParquetTableMetadataProvider implements ParquetTableMetada
 
     // builder for fallback ParquetFileTableMetadataProvider
     // for the case when required metadata is absent in Metastore
-    private ParquetFileTableMetadataProviderBuilder fallback;
+    private final ParquetFileTableMetadataProviderBuilder fallback;
 
     public Builder(MetastoreMetadataProviderManager source) {
       this.metadataProviderManager = source;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/SimpleFileTableMetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/SimpleFileTableMetadataProvider.java
index b3a34658..1ae99e4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/SimpleFileTableMetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/metastore/SimpleFileTableMetadataProvider.java
@@ -49,7 +49,7 @@ import java.util.Map;
 public class SimpleFileTableMetadataProvider implements TableMetadataProvider {
   private static final Logger logger = LoggerFactory.getLogger(SimpleFileTableMetadataProvider.class);
 
-  private TableMetadata tableMetadata;
+  private final TableMetadata tableMetadata;
 
   private SimpleFileTableMetadataProvider(TableMetadata tableMetadata) {
     this.tableMetadata = tableMetadata;
@@ -111,7 +111,7 @@ public class SimpleFileTableMetadataProvider implements TableMetadataProvider {
     private long lastModifiedTime = -1L;
     private TupleMetadata schema;
 
-    private MetadataProviderManager metadataProviderManager;
+    private final MetadataProviderManager metadataProviderManager;
 
     public Builder(MetadataProviderManager source) {
       this.metadataProviderManager = source;
@@ -147,7 +147,7 @@ public class SimpleFileTableMetadataProvider implements TableMetadataProvider {
       TableMetadataProvider source = metadataProviderManager.getTableMetadataProvider();
       if (source == null) {
         DrillStatsTable statsProvider = metadataProviderManager.getStatsProvider();
-        Map<SchemaPath, ColumnStatistics> columnsStatistics = new HashMap<>();
+        Map<SchemaPath, ColumnStatistics<?>> columnsStatistics = new HashMap<>();
 
         if (statsProvider != null) {
           if (!statsProvider.isMaterialized()) {
@@ -156,7 +156,7 @@ public class SimpleFileTableMetadataProvider implements TableMetadataProvider {
           if (statsProvider.isMaterialized()) {
             for (SchemaPath column : statsProvider.getColumns()) {
               columnsStatistics.put(column,
-                  new ColumnStatistics(DrillStatsTable.getEstimatedColumnStats(statsProvider, column)));
+                  new ColumnStatistics<>(DrillStatsTable.getEstimatedColumnStats(statsProvider, column)));
             }
           }
         }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScanWithMetadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScanWithMetadata.java
index 2371c6d..40ab594 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScanWithMetadata.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScanWithMetadata.java
@@ -261,7 +261,7 @@ public abstract class AbstractGroupScanWithMetadata<P extends TableMetadataProvi
    * @return group scan with applied filter expression
    */
   @Override
-  public AbstractGroupScanWithMetadata applyFilter(LogicalExpression filterExpr, UdfUtilities udfUtilities,
+  public AbstractGroupScanWithMetadata<?> applyFilter(LogicalExpression filterExpr, UdfUtilities udfUtilities,
       FunctionImplementationRegistry functionImplementationRegistry, OptionManager optionManager) {
 
     // Builds filter for pruning. If filter cannot be built, null should be returned.
@@ -531,7 +531,6 @@ public abstract class AbstractGroupScanWithMetadata<P extends TableMetadataProvi
     return columnMetadata != null ? columnMetadata.majorType() : null;
   }
 
-  @SuppressWarnings("unchecked")
   @JsonIgnore
   public <T> T getPartitionValue(Path path, SchemaPath column, Class<T> clazz) {
     return getPartitionsMetadata().stream()
@@ -642,7 +641,7 @@ public abstract class AbstractGroupScanWithMetadata<P extends TableMetadataProvi
     // and files which belongs to that partitions may be returned
     protected MetadataType overflowLevel = MetadataType.NONE;
 
-    public GroupScanWithMetadataFilterer(AbstractGroupScanWithMetadata source) {
+    public GroupScanWithMetadataFilterer(AbstractGroupScanWithMetadata<?> source) {
       this.source = source;
     }
 
@@ -651,7 +650,7 @@ public abstract class AbstractGroupScanWithMetadata<P extends TableMetadataProvi
      *
      * @return implementation of {@link AbstractGroupScanWithMetadata} with filtered metadata
      */
-    public abstract AbstractGroupScanWithMetadata build();
+    public abstract AbstractGroupScanWithMetadata<?> build();
 
     public B table(TableMetadata tableMetadata) {
       this.tableMetadata = tableMetadata;
@@ -968,8 +967,7 @@ public abstract class AbstractGroupScanWithMetadata<P extends TableMetadataProvi
           filterPredicate = getFilterPredicate(filterExpression, udfUtilities,
               context, optionManager, true, true, schema);
         }
-        @SuppressWarnings("rawtypes")
-        Map<SchemaPath, ColumnStatistics> columnsStatistics = metadata.getColumnsStatistics();
+        Map<SchemaPath, ColumnStatistics<?>> columnsStatistics = metadata.getColumnsStatistics();
 
         // adds partition (dir) column statistics if it may be used during filter evaluation
         if (metadata instanceof LocationProvider && optionManager != null) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatch.java
index 7ef4741..73a55b0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataControllerBatch.java
@@ -363,7 +363,6 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
         .build();
   }
 
-  @SuppressWarnings("rawtypes")
   private List<TableMetadataUnit> getMetadataUnits(TupleReader reader, int nestingLevel) {
     List<TableMetadataUnit> metadataUnits = new ArrayList<>();
 
@@ -383,7 +382,7 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
       }
     }
 
-    List<StatisticsHolder> metadataStatistics = getMetadataStatistics(reader, columnMetadata);
+    List<StatisticsHolder<?>> metadataStatistics = getMetadataStatistics(reader, columnMetadata);
 
     Long rowCount = (Long) metadataStatistics.stream()
         .filter(statisticsHolder -> statisticsHolder.getStatisticsKind() == TableStatisticsKind.ROW_COUNT)
@@ -391,7 +390,7 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
         .map(StatisticsHolder::getStatisticsValue)
         .orElse(null);
 
-    Map<SchemaPath, ColumnStatistics> columnStatistics = getColumnStatistics(reader, columnMetadata, rowCount);
+    Map<SchemaPath, ColumnStatistics<?>> columnStatistics = getColumnStatistics(reader, columnMetadata, rowCount);
 
     MetadataType metadataType = MetadataType.valueOf(metadataColumnReader.scalar().getString());
 
@@ -426,9 +425,8 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
     return metadataUnits;
   }
 
-  @SuppressWarnings("rawtypes")
-  private PartitionMetadata getPartitionMetadata(TupleReader reader, List<StatisticsHolder> metadataStatistics,
-      Map<SchemaPath, ColumnStatistics> columnStatistics, int nestingLevel) {
+  private PartitionMetadata getPartitionMetadata(TupleReader reader, List<StatisticsHolder<?>> metadataStatistics,
+      Map<SchemaPath, ColumnStatistics<?>> columnStatistics, int nestingLevel) {
     List<String> segmentColumns = popConfig.getContext().segmentColumns();
 
     String segmentKey = segmentColumns.size() > 0
@@ -458,10 +456,9 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
         .build();
   }
 
-  @SuppressWarnings("rawtypes")
-  private BaseTableMetadata getTableMetadata(TupleReader reader, List<StatisticsHolder> metadataStatistics,
-      Map<SchemaPath, ColumnStatistics> columnStatistics) {
-    List<StatisticsHolder> updatedMetaStats = new ArrayList<>(metadataStatistics);
+  private BaseTableMetadata getTableMetadata(TupleReader reader, List<StatisticsHolder<?>> metadataStatistics,
+      Map<SchemaPath, ColumnStatistics<?>> columnStatistics) {
+    List<StatisticsHolder<?>> updatedMetaStats = new ArrayList<>(metadataStatistics);
     updatedMetaStats.add(new StatisticsHolder<>(popConfig.getContext().analyzeMetadataLevel(), TableStatisticsKind.ANALYZE_METADATA_LEVEL));
 
     MetadataInfo metadataInfo = MetadataInfo.builder()
@@ -483,7 +480,7 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
 
     if (context.getOptions().getOption(PlannerSettings.STATISTICS_USE)) {
       DrillStatsTable statistics = new DrillStatsTable(statisticsCollector.getStatistics());
-      Map<SchemaPath, ColumnStatistics> tableColumnStatistics =
+      Map<SchemaPath, ColumnStatistics<?>> tableColumnStatistics =
           ParquetTableMetadataUtils.getColumnStatistics(tableMetadata.getSchema(), statistics);
       tableMetadata = tableMetadata.cloneWithStats(tableColumnStatistics, DrillStatsTable.getEstimatedTableStats(statistics));
     }
@@ -491,9 +488,8 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
     return tableMetadata;
   }
 
-  @SuppressWarnings("rawtypes")
-  private SegmentMetadata getSegmentMetadata(TupleReader reader, List<StatisticsHolder> metadataStatistics,
-      Map<SchemaPath, ColumnStatistics> columnStatistics, int nestingLevel) {
+  private SegmentMetadata getSegmentMetadata(TupleReader reader, List<StatisticsHolder<?>> metadataStatistics,
+      Map<SchemaPath, ColumnStatistics<?>> columnStatistics, int nestingLevel) {
     List<String> segmentColumns = popConfig.getContext().segmentColumns();
 
     String segmentKey = segmentColumns.size() > 0
@@ -540,9 +536,8 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
         .build();
   }
 
-  @SuppressWarnings("rawtypes")
-  private FileMetadata getFileMetadata(TupleReader reader, List<StatisticsHolder> metadataStatistics,
-      Map<SchemaPath, ColumnStatistics> columnStatistics, int nestingLevel) {
+  private FileMetadata getFileMetadata(TupleReader reader, List<StatisticsHolder<?>> metadataStatistics,
+      Map<SchemaPath, ColumnStatistics<?>> columnStatistics, int nestingLevel) {
     List<String> segmentColumns = popConfig.getContext().segmentColumns();
 
     String segmentKey = segmentColumns.size() > 0
@@ -574,9 +569,8 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
         .build();
   }
 
-  @SuppressWarnings("rawtypes")
-  private RowGroupMetadata getRowGroupMetadata(TupleReader reader,List<StatisticsHolder> metadataStatistics,
-      Map<SchemaPath, ColumnStatistics> columnStatistics, int nestingLevel) {
+  private RowGroupMetadata getRowGroupMetadata(TupleReader reader,List<StatisticsHolder<?>> metadataStatistics,
+      Map<SchemaPath, ColumnStatistics<?>> columnStatistics, int nestingLevel) {
 
     List<String> segmentColumns = popConfig.getContext().segmentColumns();
     String segmentKey = segmentColumns.size() > 0
@@ -613,9 +607,8 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
         .build();
   }
 
-  @SuppressWarnings("rawtypes")
-  private Map<SchemaPath, ColumnStatistics> getColumnStatistics(TupleReader reader, TupleMetadata columnMetadata, Long rowCount) {
-    Multimap<String, StatisticsHolder> columnStatistics = ArrayListMultimap.create();
+  private Map<SchemaPath, ColumnStatistics<?>> getColumnStatistics(TupleReader reader, TupleMetadata columnMetadata, Long rowCount) {
+    Multimap<String, StatisticsHolder<?>> columnStatistics = ArrayListMultimap.create();
     Map<String, TypeProtos.MinorType> columnTypes = new HashMap<>();
     for (ColumnMetadata column : columnMetadata) {
       String fieldName = AnalyzeColumnUtils.getColumnName(column.name());
@@ -633,7 +626,7 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
 
     // adds NON_NULL_COUNT to use it during filter pushdown
     if (rowCount != null) {
-      Map<String, StatisticsHolder> nullsCountColumnStatistics = new HashMap<>();
+      Map<String, StatisticsHolder<?>> nullsCountColumnStatistics = new HashMap<>();
       columnStatistics.asMap().forEach((key, value) ->
           value.stream()
               .filter(statisticsHolder -> statisticsHolder.getStatisticsKind() == ColumnStatisticsKind.NON_NULL_VALUES_COUNT)
@@ -647,16 +640,15 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
       nullsCountColumnStatistics.forEach(columnStatistics::put);
     }
 
-    Map<SchemaPath, ColumnStatistics> resultingStats = new HashMap<>();
+    Map<SchemaPath, ColumnStatistics<?>> resultingStats = new HashMap<>();
 
     columnStatistics.asMap().forEach((fieldName, statisticsHolders) ->
         resultingStats.put(SchemaPath.parseFromString(fieldName), new ColumnStatistics<>(statisticsHolders, columnTypes.get(fieldName))));
     return resultingStats;
   }
 
-  @SuppressWarnings("rawtypes")
-  private List<StatisticsHolder> getMetadataStatistics(TupleReader reader, TupleMetadata columnMetadata) {
-    List<StatisticsHolder> metadataStatistics = new ArrayList<>();
+  private List<StatisticsHolder<?>> getMetadataStatistics(TupleReader reader, TupleMetadata columnMetadata) {
+    List<StatisticsHolder<?>> metadataStatistics = new ArrayList<>();
     String rgs = columnNamesOptions.rowGroupStart();
     String rgl = columnNamesOptions.rowGroupLength();
     for (ColumnMetadata column : columnMetadata) {
@@ -753,6 +745,8 @@ public class MetadataControllerBatch extends AbstractBinaryRecordBatch<MetadataC
       case FILE: {
         childLocations.add(new Path(reader.column(MetastoreAnalyzeConstants.LOCATION_FIELD).scalar().getString()));
       }
+      default:
+        break;
     }
 
     return childLocations;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatch.java
index 044d140..5c45150 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/metadata/MetadataHandlerBatch.java
@@ -170,6 +170,8 @@ public class MetadataHandlerBatch extends AbstractSingleRecordBatch<MetadataHand
                   new ArrayList<>(metadataToHandle.values()));
           return populateContainer(segments);
         }
+        default:
+          break;
       }
     }
     return outcome;
@@ -194,7 +196,6 @@ public class MetadataHandlerBatch extends AbstractSingleRecordBatch<MetadataHand
     }
   }
 
-  @SuppressWarnings("unchecked")
   private <T extends BaseMetadata & LocationProvider> VectorContainer writeMetadata(List<T> metadataList) {
     BaseMetadata firstElement = metadataList.iterator().next();
 
@@ -304,7 +305,6 @@ public class MetadataHandlerBatch extends AbstractSingleRecordBatch<MetadataHand
     return new ResultSetLoaderImpl(container.getAllocator(), options);
   }
 
-  @SuppressWarnings("unchecked")
   private <T extends BaseMetadata & LocationProvider> VectorContainer writeMetadataUsingBatchSchema(List<T> metadataList) {
     Preconditions.checkArgument(!metadataList.isEmpty(), "Metadata list shouldn't be empty.");
 
@@ -413,6 +413,7 @@ public class MetadataHandlerBatch extends AbstractSingleRecordBatch<MetadataHand
     container.setEmpty();
   }
 
+  @Override
   protected boolean setupNewSchema() {
     setupSchemaFromContainer(incoming.getContainer());
     return true;
@@ -473,6 +474,8 @@ public class MetadataHandlerBatch extends AbstractSingleRecordBatch<MetadataHand
           }
           break;
         }
+        default:
+          break;
       }
     }
   }
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
index 9e4e925..70f3e16 100644
--- 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
@@ -263,6 +263,7 @@ public class DrillStatsTable {
     @JsonProperty ("directories") List<DirectoryStatistics_v0> directoryStatistics;
     // Default constructor required for deserializer
     public Statistics_v0 () { }
+    @Override
     @JsonGetter ("directories")
     public List<DirectoryStatistics_v0> getDirectoryStatistics() {
       return directoryStatistics;
@@ -296,6 +297,7 @@ public class DrillStatsTable {
     List<DirectoryStatistics_v1> directoryStatistics;
     // Default constructor required for deserializer
     public Statistics_v1 () { }
+    @Override
     @JsonGetter ("directories")
     public List<DirectoryStatistics_v1> getDirectoryStatistics() {
       return directoryStatistics;
@@ -396,7 +398,7 @@ public class DrillStatsTable {
     }
     @JsonIgnore
     public void buildHistogram(byte[] tdigest_bytearray) {
-      int num_buckets = (int) Math.min(ndv, (long) DrillStatsTable.NUM_HISTOGRAM_BUCKETS);
+      int num_buckets = (int) Math.min(ndv, DrillStatsTable.NUM_HISTOGRAM_BUCKETS);
       this.histogram = HistogramUtils.buildHistogramFromTDigest(tdigest_bytearray, this.getType(),
               num_buckets, nonNullCount);
     }
@@ -479,9 +481,9 @@ public class DrillStatsTable {
    * @param statsProvider the source of statistics
    * @return list of {@link StatisticsKind} and statistics values
    */
-  public static List<StatisticsHolder> getEstimatedTableStats(DrillStatsTable statsProvider) {
+  public static List<StatisticsHolder<?>> getEstimatedTableStats(DrillStatsTable statsProvider) {
     if (statsProvider != null && statsProvider.isMaterialized()) {
-      List<StatisticsHolder> tableStatistics = Arrays.asList(
+      List<StatisticsHolder<?>> tableStatistics = Arrays.asList(
           new StatisticsHolder<>(statsProvider.getRowCount(), TableStatisticsKind.EST_ROW_COUNT),
           new StatisticsHolder<>(Boolean.TRUE, TableStatisticsKind.HAS_DESCRIPTIVE_STATISTICS));
       return tableStatistics;
@@ -496,9 +498,9 @@ public class DrillStatsTable {
    * @param fieldName     name of the columns whose statistics should be obtained
    * @return list of {@link StatisticsKind} and statistics values
    */
-  public static List<StatisticsHolder> getEstimatedColumnStats(DrillStatsTable statsProvider, SchemaPath fieldName) {
+  public static List<StatisticsHolder<?>> getEstimatedColumnStats(DrillStatsTable statsProvider, SchemaPath fieldName) {
     if (statsProvider != null && statsProvider.isMaterialized()) {
-      List<StatisticsHolder> statisticsValues = new ArrayList<>();
+      List<StatisticsHolder<?>> statisticsValues = new ArrayList<>();
       Double ndv = statsProvider.getNdv(fieldName);
       if (ndv != null) {
         statisticsValues.add(new StatisticsHolder<>(ndv, ColumnStatisticsKind.NDV));
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 3053297..9f2901b 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
@@ -163,7 +163,7 @@ public class DrillRelMdDistinctRowCount extends RelMdDistinctRowCount{
       if (!groupKey.get(i)) {
         continue;
       }
-      ColumnStatistics columnStatistics = tableMetadata != null ?
+      ColumnStatistics<?> columnStatistics = tableMetadata != null ?
           tableMetadata.getColumnStatistics(SchemaPath.getSimplePath(colName)) : null;
       Double ndv = columnStatistics != null ? ColumnStatisticsKind.NDV.getFrom(columnStatistics) : null;
       // Skip NDV, if not available
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java
index 49d2129..9f3beca 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdSelectivity.java
@@ -187,6 +187,7 @@ public class DrillRelMdSelectivity extends RelMdSelectivity {
             try {
               RexVisitor<Void> visitor =
                       new RexVisitorImpl<Void>(true) {
+                        @Override
                         public Void visitCall(RexCall call) {
                           if (call.getKind() != SqlKind.EQUALS) {
                             throw new Util.FoundOne(call);
@@ -294,7 +295,7 @@ public class DrillRelMdSelectivity extends RelMdSelectivity {
   private double computeEqualsSelectivity(TableMetadata tableMetadata, RexNode orPred, List<SchemaPath> fieldNames) {
     SchemaPath col = getColumn(orPred, fieldNames);
     if (col != null) {
-      ColumnStatistics columnStatistics = tableMetadata != null ? tableMetadata.getColumnStatistics(col) : null;
+      ColumnStatistics<?> columnStatistics = tableMetadata != null ? tableMetadata.getColumnStatistics(col) : null;
       Double ndv = columnStatistics != null ? ColumnStatisticsKind.NDV.getFrom(columnStatistics) : null;
       if (ndv != null) {
         return 1.00 / ndv;
@@ -307,7 +308,7 @@ public class DrillRelMdSelectivity extends RelMdSelectivity {
   private double computeRangeSelectivity(TableMetadata tableMetadata, RexNode orPred, List<SchemaPath> fieldNames) {
     SchemaPath col = getColumn(orPred, fieldNames);
     if (col != null) {
-      ColumnStatistics columnStatistics = tableMetadata != null ? tableMetadata.getColumnStatistics(col) : null;
+      ColumnStatistics<?> columnStatistics = tableMetadata != null ? tableMetadata.getColumnStatistics(col) : null;
       Histogram histogram = columnStatistics != null ? ColumnStatisticsKind.HISTOGRAM.getFrom(columnStatistics) : null;
       if (histogram != null) {
         Double totalCount = ColumnStatisticsKind.ROWCOUNT.getFrom(columnStatistics);
@@ -324,7 +325,7 @@ public class DrillRelMdSelectivity extends RelMdSelectivity {
   private double computeIsNotNullSelectivity(TableMetadata tableMetadata, RexNode orPred, List<SchemaPath> fieldNames) {
     SchemaPath col = getColumn(orPred, fieldNames);
     if (col != null) {
-      ColumnStatistics columnStatistics = tableMetadata != null ? tableMetadata.getColumnStatistics(col) : null;
+      ColumnStatistics<?> columnStatistics = tableMetadata != null ? tableMetadata.getColumnStatistics(col) : null;
       Double nonNullCount = columnStatistics != null ? ColumnStatisticsKind.NON_NULL_COUNT.getFrom(columnStatistics) : null;
       if (nonNullCount != null) {
         // Cap selectivity below Calcite Guess
@@ -423,6 +424,7 @@ public class DrillRelMdSelectivity extends RelMdSelectivity {
     try {
       RexVisitor<Void> visitor =
           new RexVisitorImpl<Void>(true) {
+            @Override
             public Void visitCall(RexCall call) {
               for (RexNode child : call.getOperands()) {
                 child.accept(this);
@@ -430,6 +432,7 @@ public class DrillRelMdSelectivity extends RelMdSelectivity {
               return super.visitCall(call);
             }
 
+            @Override
             public Void visitInputRef(RexInputRef inputRef) {
               throw new Util.FoundOne(inputRef);
             }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
index a9bff98..ac2c3ff 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
@@ -229,7 +229,7 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
    * @return group scan with applied filter expression
    */
   @Override
-  public AbstractGroupScanWithMetadata applyFilter(LogicalExpression filterExpr, UdfUtilities udfUtilities,
+  public AbstractGroupScanWithMetadata<?> applyFilter(LogicalExpression filterExpr, UdfUtilities udfUtilities,
       FunctionImplementationRegistry functionImplementationRegistry, OptionManager optionManager) {
     // Builds filter for pruning. If filter cannot be built, null should be returned.
     FilterPredicate<?> filterPredicate = getFilterPredicate(filterExpr, udfUtilities, functionImplementationRegistry, optionManager, true);
@@ -481,7 +481,7 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
   protected abstract static class RowGroupScanFilterer<B extends RowGroupScanFilterer<B>> extends GroupScanWithMetadataFilterer<B> {
     protected Multimap<Path, RowGroupMetadata> rowGroups = LinkedListMultimap.create();
 
-    public RowGroupScanFilterer(AbstractGroupScanWithMetadata source) {
+    public RowGroupScanFilterer(AbstractGroupScanWithMetadata<?> source) {
       super(source);
     }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
index de555c2..74b4c17 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
@@ -102,7 +102,7 @@ public abstract class AbstractParquetScanBatchCreator {
       Path prevRowGroupPath = null;
       Metadata_V4.ParquetTableMetadata_v4 tableMetadataV4 = null;
       Metadata_V4.ParquetFileAndRowCountMetadata fileMetadataV4 = null;
-      FilterPredicate filterPredicate = null;
+      FilterPredicate<?> filterPredicate = null;
       Set<SchemaPath> schemaPathsInExpr = null;
       Set<SchemaPath> columnsInExpr = null;
       // for debug/info logging
@@ -135,7 +135,7 @@ public abstract class AbstractParquetScanBatchCreator {
         Here we could store a map from file names to footers, to prevent re-reading the footer for each row group in a file
         TODO - to prevent reading the footer again in the parquet record reader (it is read earlier in the ParquetStorageEngine)
         we should add more information to the RowGroupInfo that will be populated upon the first read to
-        provide the reader with all of th file meta-data it needs
+        provide the reader with all of the file meta-data it needs
         These fields will be added to the constructor below
         */
 
@@ -190,7 +190,7 @@ public abstract class AbstractParquetScanBatchCreator {
 
             MetadataBase.RowGroupMetadata rowGroupMetadata = fileMetadataV4.getFileMetadata().getRowGroups().get(rowGroup.getRowGroupIndex());
 
-            Map<SchemaPath, ColumnStatistics> columnsStatistics = ParquetTableMetadataUtils.getRowGroupColumnStatistics(tableMetadataV4, rowGroupMetadata);
+            Map<SchemaPath, ColumnStatistics<?>> columnsStatistics = ParquetTableMetadataUtils.getRowGroupColumnStatistics(tableMetadataV4, rowGroupMetadata);
 
             try {
               Map<SchemaPath, TypeProtos.MajorType> intermediateColumns =
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BaseParquetMetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BaseParquetMetadataProvider.java
index ccf5c15..b535e0f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BaseParquetMetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/BaseParquetMetadataProvider.java
@@ -207,7 +207,7 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
   @Override
   public TableMetadata getTableMetadata() {
     if (tableMetadata == null) {
-      List<StatisticsHolder> tableStatistics = new ArrayList<>(DrillStatsTable.getEstimatedTableStats(statsTable));
+      List<StatisticsHolder<?>> tableStatistics = new ArrayList<>(DrillStatsTable.getEstimatedTableStats(statsTable));
       Map<SchemaPath, TypeProtos.MajorType> fields = ParquetTableMetadataUtils.resolveFields(parquetTableMetadata);
       Map<SchemaPath, TypeProtos.MajorType> intermediateFields = ParquetTableMetadataUtils.resolveIntermediateFields(parquetTableMetadata);
 
@@ -223,7 +223,7 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
         });
       }
 
-      Map<SchemaPath, ColumnStatistics> columnsStatistics;
+      Map<SchemaPath, ColumnStatistics<?>> columnsStatistics;
       if (collectMetadata) {
         Collection<? extends BaseMetadata> metadata = getFilesMetadataMap().values();
         if (metadata.isEmpty()) {
@@ -243,18 +243,18 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
         fields.forEach((columnPath, value) -> {
           long columnValueCount = getParquetGroupScanStatistics().getColumnValueCount(columnPath);
           // Adds statistics values itself if statistics is available
-          List<StatisticsHolder> stats = new ArrayList<>(DrillStatsTable.getEstimatedColumnStats(statsTable, columnPath));
+          List<StatisticsHolder<?>> stats = new ArrayList<>(DrillStatsTable.getEstimatedColumnStats(statsTable, columnPath));
           unhandledColumns.remove(columnPath);
 
           // adds statistics for partition columns
           stats.add(new StatisticsHolder<>(columnValueCount, TableStatisticsKind.ROW_COUNT));
           stats.add(new StatisticsHolder<>(getParquetGroupScanStatistics().getRowCount() - columnValueCount, ColumnStatisticsKind.NULLS_COUNT));
-          columnsStatistics.put(columnPath, new ColumnStatistics(stats, value.getMinorType()));
+          columnsStatistics.put(columnPath, new ColumnStatistics<>(stats, value.getMinorType()));
         });
 
         for (SchemaPath column : unhandledColumns) {
           columnsStatistics.put(column,
-              new ColumnStatistics(DrillStatsTable.getEstimatedColumnStats(statsTable, column)));
+              new ColumnStatistics<>(DrillStatsTable.getEstimatedColumnStats(statsTable, column)));
         }
       }
       MetadataInfo metadataInfo = MetadataInfo.builder().type(MetadataType.TABLE).build();
@@ -327,9 +327,9 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
           partitionPaths.forEach((path, value) -> partitionsForValue.put(value, path));
 
           partitionsForValue.asMap().forEach((partitionKey, value) -> {
-            Map<SchemaPath, ColumnStatistics> columnsStatistics = new HashMap<>();
+            Map<SchemaPath, ColumnStatistics<?>> columnsStatistics = new HashMap<>();
 
-            List<StatisticsHolder> statistics = new ArrayList<>();
+            List<StatisticsHolder<?>> statistics = new ArrayList<>();
             partitionKey = partitionKey == NULL_VALUE ? null : partitionKey;
             statistics.add(new StatisticsHolder<>(partitionKey, ColumnStatisticsKind.MIN_VALUE));
             statistics.add(new StatisticsHolder<>(partitionKey, ColumnStatisticsKind.MAX_VALUE));
@@ -381,6 +381,7 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
         .collect(Collectors.toList());
   }
 
+  @SuppressWarnings("unused")
   @Override
   public Map<Path, SegmentMetadata> getSegmentsMetadataMap() {
     if (segments == null) {
@@ -456,7 +457,7 @@ public abstract class BaseParquetMetadataProvider implements ParquetMetadataProv
    */
   private static <T extends BaseMetadata & LocationProvider> SegmentMetadata combineToSegmentMetadata(Collection<T> metadataList,
       SchemaPath column, Set<Path> metadataLocations) {
-    List<StatisticsHolder> segmentStatistics =
+    List<StatisticsHolder<?>> segmentStatistics =
         Collections.singletonList(
             new StatisticsHolder<>(
                 TableStatisticsKind.ROW_COUNT.mergeStatistics(metadataList),
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FilterEvaluatorUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FilterEvaluatorUtils.java
index 022975e..ffde9c3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FilterEvaluatorUtils.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/FilterEvaluatorUtils.java
@@ -66,7 +66,7 @@ public class FilterEvaluatorUtils {
 
     RowGroupMetadata rowGroupMetadata = new ArrayList<>(ParquetTableMetadataUtils.getRowGroupsMetadata(footer).values()).get(rowGroupIndex);
     NonInterestingColumnsMetadata nonInterestingColumnsMetadata = ParquetTableMetadataUtils.getNonInterestingColumnsMeta(footer);
-    Map<SchemaPath, ColumnStatistics> columnsStatistics = rowGroupMetadata.getColumnsStatistics();
+    Map<SchemaPath, ColumnStatistics<?>> columnsStatistics = rowGroupMetadata.getColumnsStatistics();
 
     // Add column statistics of non-interesting columns if there are any
     columnsStatistics.putAll(nonInterestingColumnsMetadata.getColumnsStatistics());
@@ -78,7 +78,7 @@ public class FilterEvaluatorUtils {
         fragmentContext, fragmentContext.getFunctionRegistry(), new HashSet<>(schemaPathsInExpr));
   }
 
-  public static RowsMatch matches(LogicalExpression expr, Map<SchemaPath, ColumnStatistics> columnsStatistics, TupleMetadata schema,
+  public static RowsMatch matches(LogicalExpression expr, Map<SchemaPath, ColumnStatistics<?>> columnsStatistics, TupleMetadata schema,
                                   long rowCount, UdfUtilities udfUtilities, FunctionLookupContext functionImplementationRegistry,
                                   Set<SchemaPath> schemaPathsInExpr) {
     ErrorCollector errorCollector = new ErrorCollectorImpl();
@@ -95,21 +95,22 @@ public class FilterEvaluatorUtils {
     }
 
     Set<LogicalExpression> constantBoundaries = ConstantExpressionIdentifier.getConstantExpressionSet(materializedFilter);
-    FilterPredicate parquetPredicate = FilterBuilder.buildFilterPredicate(
+    FilterPredicate<?> parquetPredicate = FilterBuilder.buildFilterPredicate(
         materializedFilter, constantBoundaries, udfUtilities, true);
 
     return matches(parquetPredicate, columnsStatistics, rowCount, schema, schemaPathsInExpr);
   }
 
   @SuppressWarnings("unchecked")
-  public static RowsMatch matches(FilterPredicate parquetPredicate,
-                                  Map<SchemaPath, ColumnStatistics> columnsStatistics,
+  public static <T extends Comparable<T>> RowsMatch matches(FilterPredicate<T> parquetPredicate,
+                                  Map<SchemaPath, ColumnStatistics<?>> columnsStatistics,
                                   long rowCount,
                                   TupleMetadata fileMetadata,
                                   Set<SchemaPath> schemaPathsInExpr) {
     RowsMatch rowsMatch = RowsMatch.SOME;
     if (parquetPredicate != null) {
-      StatisticsProvider rangeExprEvaluator = new StatisticsProvider(columnsStatistics, rowCount);
+      @SuppressWarnings("rawtypes")
+      StatisticsProvider<T> rangeExprEvaluator = new StatisticsProvider(columnsStatistics, rowCount);
       rowsMatch = parquetPredicate.matches(rangeExprEvaluator);
     }
     return rowsMatch == RowsMatch.ALL && isRepeated(schemaPathsInExpr, fileMetadata) ? RowsMatch.SOME : rowsMatch;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
index 051b436..5c0e2e4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
@@ -65,10 +65,10 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
   private final ParquetFormatPlugin formatPlugin;
   private final ParquetFormatConfig formatConfig;
 
-  private boolean usedMetadataCache; // false by default
+  private final boolean usedMetadataCache; // false by default
   // may change when filter push down / partition pruning is applied
-  private Path selectionRoot;
-  private Path cacheFileRoot;
+  private final Path selectionRoot;
+  private final Path cacheFileRoot;
 
   @SuppressWarnings("unused")
   @JsonCreator
@@ -198,6 +198,7 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
     return formatPlugin.getStorageConfig();
   }
 
+  @Override
   @JsonProperty
   public Path getSelectionRoot() {
     return selectionRoot;
@@ -288,7 +289,7 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
   }
 
   @Override
-  protected RowGroupScanFilterer getFilterer() {
+  protected RowGroupScanFilterer<?> getFilterer() {
     return new ParquetGroupScanFilterer(this);
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScanStatistics.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScanStatistics.java
index 17a7c55..f15409d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScanStatistics.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScanStatistics.java
@@ -59,8 +59,7 @@ public class ParquetGroupScanStatistics<T extends BaseMetadata & LocationProvide
     collect(rowGroupInfos);
   }
 
-  @SuppressWarnings("unchecked")
-  public ParquetGroupScanStatistics(ParquetGroupScanStatistics that) {
+  public ParquetGroupScanStatistics(ParquetGroupScanStatistics<T> that) {
     this.partitionValueMap = HashBasedTable.create(that.partitionValueMap);
     this.partitionColTypeMap = new HashMap<>(that.partitionColTypeMap);
     this.columnValueCounts = new HashMap<>(that.columnValueCounts);
@@ -101,9 +100,9 @@ public class ParquetGroupScanStatistics<T extends BaseMetadata & LocationProvide
     boolean first = true;
     for (T metadata : metadataList) {
       long localRowCount = TableStatisticsKind.ROW_COUNT.getValue(metadata);
-      for (Map.Entry<SchemaPath, ColumnStatistics> columnsStatistics : metadata.getColumnsStatistics().entrySet()) {
+      for (Map.Entry<SchemaPath, ColumnStatistics<?>> columnsStatistics : metadata.getColumnsStatistics().entrySet()) {
         SchemaPath schemaPath = columnsStatistics.getKey();
-        ColumnStatistics statistics = columnsStatistics.getValue();
+        ColumnStatistics<?> statistics = columnsStatistics.getValue();
         MutableLong emptyCount = new MutableLong();
         MutableLong previousCount = columnValueCounts.putIfAbsent(schemaPath, emptyCount);
         if (previousCount == null) {
@@ -164,7 +163,7 @@ public class ParquetGroupScanStatistics<T extends BaseMetadata & LocationProvide
    * @param rowCount         row count
    * @return whether column is a potential partition column
    */
-  private boolean checkForPartitionColumn(ColumnStatistics columnStatistics,
+  private boolean checkForPartitionColumn(ColumnStatistics<?> columnStatistics,
                                           boolean first,
                                           long rowCount,
                                           TypeProtos.MajorType type,
@@ -202,11 +201,11 @@ public class ParquetGroupScanStatistics<T extends BaseMetadata & LocationProvide
    * @param rowCount         rows count in column chunk
    * @return true if column has single value
    */
-  private boolean hasSingleValue(ColumnStatistics columnStatistics, long rowCount) {
+  private boolean hasSingleValue(ColumnStatistics<?> columnStatistics, long rowCount) {
     return columnStatistics != null && isSingleVal(columnStatistics, rowCount);
   }
 
-  private boolean isSingleVal(ColumnStatistics columnStatistics, long rowCount) {
+  private boolean isSingleVal(ColumnStatistics<?> columnStatistics, long rowCount) {
     Long numNulls = ColumnStatisticsKind.NULLS_COUNT.getFrom(columnStatistics);
     if (numNulls != null && numNulls != Statistic.NO_COLUMN_STATS) {
       Object min = columnStatistics.get(ColumnStatisticsKind.MIN_VALUE);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
index bd2e119..219359b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
@@ -138,7 +138,7 @@ public abstract class ParquetPushDownFilter extends StoragePluginOptimizerRule {
 
     final List<RexNode> qualifiedPredList = new ArrayList<>();
 
-    // list of predicates which cannot be converted to parquet filter predicate
+    // list of predicates which cannot be converted to Parquet filter predicate
     List<RexNode> nonConvertedPredList = new ArrayList<>();
 
     for (RexNode pred : predList) {
@@ -147,7 +147,7 @@ public abstract class ParquetPushDownFilter extends StoragePluginOptimizerRule {
             new DrillParseContext(PrelUtil.getPlannerSettings(call.getPlanner())), scan, pred);
 
         // checks whether predicate may be used for filter pushdown
-        FilterPredicate parquetFilterPredicate =
+        FilterPredicate<?> parquetFilterPredicate =
             groupScan.getFilterPredicate(drillPredicate,
                 optimizerContext,
                 optimizerContext.getFunctionRegistry(), optimizerContext.getPlannerSettings().getOptions(), false);
@@ -171,11 +171,11 @@ public abstract class ParquetPushDownFilter extends StoragePluginOptimizerRule {
     LogicalExpression conditionExp = DrillOptiq.toDrill(
         new DrillParseContext(PrelUtil.getPlannerSettings(call.getPlanner())), scan, qualifiedPred);
 
-    // Default - pass the original filter expr to (potentialy) be used at run-time
+    // Default - pass the original filter expr to (potentially) be used at run-time
     groupScan.setFilterForRuntime(conditionExp, optimizerContext); // later may remove or set to another filter (see below)
 
     Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null;
-    AbstractGroupScanWithMetadata newGroupScan = groupScan.applyFilter(conditionExp, optimizerContext,
+    AbstractGroupScanWithMetadata<?> newGroupScan = groupScan.applyFilter(conditionExp, optimizerContext,
         optimizerContext.getFunctionRegistry(), optimizerContext.getPlannerSettings().getOptions());
     if (timer != null) {
       logger.debug("Took {} ms to apply filter on parquet row groups. ", timer.elapsed(TimeUnit.MILLISECONDS));
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetTableMetadataUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetTableMetadataUtils.java
index 718f3c0..68254d3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetTableMetadataUtils.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetTableMetadataUtils.java
@@ -80,7 +80,7 @@ import java.util.stream.Collectors;
 @SuppressWarnings("WeakerAccess")
 public class ParquetTableMetadataUtils {
 
-  static final List<CollectableColumnStatisticsKind> PARQUET_COLUMN_STATISTICS =
+  static final List<CollectableColumnStatisticsKind<?>> PARQUET_COLUMN_STATISTICS =
           ImmutableList.of(
               ColumnStatisticsKind.MAX_VALUE,
               ColumnStatisticsKind.MIN_VALUE,
@@ -102,8 +102,8 @@ public class ParquetTableMetadataUtils {
    * @param supportsFileImplicitColumns whether implicit columns are supported
    * @return map with added statistics for implicit and partition (dir) columns
    */
-  public static Map<SchemaPath, ColumnStatistics> addImplicitColumnsStatistics(
-      Map<SchemaPath, ColumnStatistics> columnsStatistics, List<SchemaPath> columns,
+  public static Map<SchemaPath, ColumnStatistics<?>> addImplicitColumnsStatistics(
+      Map<SchemaPath, ColumnStatistics<?>> columnsStatistics, List<SchemaPath> columns,
       List<String> partitionValues, OptionManager optionManager, Path location, boolean supportsFileImplicitColumns) {
     ColumnExplorer columnExplorer = new ColumnExplorer(optionManager, columns);
 
@@ -157,11 +157,11 @@ public class ParquetTableMetadataUtils {
    */
   public static RowGroupMetadata getRowGroupMetadata(MetadataBase.ParquetTableMetadataBase tableMetadata,
       MetadataBase.RowGroupMetadata rowGroupMetadata, int rgIndexInFile, Path location) {
-    Map<SchemaPath, ColumnStatistics> columnsStatistics = getRowGroupColumnStatistics(tableMetadata, rowGroupMetadata);
-    List<StatisticsHolder> rowGroupStatistics = new ArrayList<>();
+    Map<SchemaPath, ColumnStatistics<?>> columnsStatistics = getRowGroupColumnStatistics(tableMetadata, rowGroupMetadata);
+    List<StatisticsHolder<?>> rowGroupStatistics = new ArrayList<>();
     rowGroupStatistics.add(new StatisticsHolder<>(rowGroupMetadata.getRowCount(), TableStatisticsKind.ROW_COUNT));
-    rowGroupStatistics.add(new StatisticsHolder<>(rowGroupMetadata.getStart(), new BaseStatisticsKind(ExactStatisticsConstants.START, true)));
-    rowGroupStatistics.add(new StatisticsHolder<>(rowGroupMetadata.getLength(), new BaseStatisticsKind(ExactStatisticsConstants.LENGTH, true)));
+    rowGroupStatistics.add(new StatisticsHolder<>(rowGroupMetadata.getStart(), new BaseStatisticsKind<>(ExactStatisticsConstants.START, true)));
+    rowGroupStatistics.add(new StatisticsHolder<>(rowGroupMetadata.getLength(), new BaseStatisticsKind<>(ExactStatisticsConstants.LENGTH, true)));
 
     Map<SchemaPath, TypeProtos.MajorType> columns = getRowGroupFields(tableMetadata, rowGroupMetadata);
     Map<SchemaPath, TypeProtos.MajorType> intermediateColumns = getIntermediateFields(tableMetadata, rowGroupMetadata);
@@ -195,7 +195,7 @@ public class ParquetTableMetadataUtils {
     if (rowGroups.isEmpty()) {
       return null;
     }
-    List<StatisticsHolder> fileStatistics = new ArrayList<>();
+    List<StatisticsHolder<?>> fileStatistics = new ArrayList<>();
     fileStatistics.add(new StatisticsHolder<>(TableStatisticsKind.ROW_COUNT.mergeStatistics(rowGroups), TableStatisticsKind.ROW_COUNT));
 
     RowGroupMetadata rowGroupMetadata = rowGroups.iterator().next();
@@ -255,10 +255,10 @@ public class ParquetTableMetadataUtils {
    * @param rowGroupMetadata metadata to convert
    * @return map with converted row group metadata
    */
-  public static Map<SchemaPath, ColumnStatistics> getRowGroupColumnStatistics(
+  public static Map<SchemaPath, ColumnStatistics<?>> getRowGroupColumnStatistics(
       MetadataBase.ParquetTableMetadataBase tableMetadata, MetadataBase.RowGroupMetadata rowGroupMetadata) {
 
-    Map<SchemaPath, ColumnStatistics> columnsStatistics = new HashMap<>();
+    Map<SchemaPath, ColumnStatistics<?>> columnsStatistics = new HashMap<>();
 
     for (MetadataBase.ColumnMetadata column : rowGroupMetadata.getColumns()) {
       SchemaPath colPath = SchemaPath.getCompoundPath(column.getName());
@@ -271,7 +271,7 @@ public class ParquetTableMetadataUtils {
       OriginalType originalType = getOriginalType(tableMetadata, column);
       TypeProtos.MinorType type = ParquetReaderUtility.getMinorType(primitiveType, originalType);
 
-      List<StatisticsHolder> statistics = new ArrayList<>();
+      List<StatisticsHolder<?>> statistics = new ArrayList<>();
       statistics.add(new StatisticsHolder<>(getValue(column.getMinValue(), primitiveType, originalType), ColumnStatisticsKind.MIN_VALUE));
       statistics.add(new StatisticsHolder<>(getValue(column.getMaxValue(), primitiveType, originalType), ColumnStatisticsKind.MAX_VALUE));
       statistics.add(new StatisticsHolder<>(nulls, ColumnStatisticsKind.NULLS_COUNT));
@@ -286,7 +286,7 @@ public class ParquetTableMetadataUtils {
    * @return returns non-interesting columns metadata
    */
   public static NonInterestingColumnsMetadata getNonInterestingColumnsMeta(MetadataBase.ParquetTableMetadataBase parquetTableMetadata) {
-    Map<SchemaPath, ColumnStatistics> columnsStatistics = new HashMap<>();
+    Map<SchemaPath, ColumnStatistics<?>> columnsStatistics = new HashMap<>();
     if (parquetTableMetadata instanceof Metadata_V4.ParquetTableMetadata_v4) {
       Map<Metadata_V4.ColumnTypeMetadata_v4.Key, Metadata_V4.ColumnTypeMetadata_v4> columnTypeInfoMap =
               ((Metadata_V4.ParquetTableMetadata_v4) parquetTableMetadata).getColumnTypeInfoMap();
@@ -298,7 +298,7 @@ public class ParquetTableMetadataUtils {
       for (Metadata_V4.ColumnTypeMetadata_v4 columnTypeMetadata : columnTypeInfoMap.values()) {
         if (!columnTypeMetadata.isInteresting) {
           SchemaPath schemaPath = SchemaPath.getCompoundPath(columnTypeMetadata.name);
-          List<StatisticsHolder> statistics = new ArrayList<>();
+          List<StatisticsHolder<?>> statistics = new ArrayList<>();
           statistics.add(new StatisticsHolder<>(Statistic.NO_COLUMN_STATS, ColumnStatisticsKind.NULLS_COUNT));
           PrimitiveType.PrimitiveTypeName primitiveType = columnTypeMetadata.primitiveType;
           OriginalType originalType = columnTypeMetadata.originalType;
@@ -371,7 +371,8 @@ public class ParquetTableMetadataUtils {
     } else if (value instanceof String) { // value is obtained from metadata cache v2+
       return ((String) value).getBytes();
     } else if (value instanceof Map) { // value is obtained from metadata cache v1
-      String bytesString = (String) ((Map) value).get("bytes");
+      @SuppressWarnings("unchecked")
+      String bytesString = ((Map<String,String>) value).get("bytes");
       if (bytesString != null) {
         return bytesString.getBytes();
       }
@@ -647,7 +648,7 @@ public class ParquetTableMetadataUtils {
    * @param statistics source of column statistics
    * @return map with schema path and {@link ColumnStatistics}
    */
-  public static Map<SchemaPath, ColumnStatistics> getColumnStatistics(TupleMetadata schema, DrillStatsTable statistics) {
+  public static Map<SchemaPath, ColumnStatistics<?>> getColumnStatistics(TupleMetadata schema, DrillStatsTable statistics) {
     List<SchemaPath> schemaPaths = SchemaUtil.getSchemaPaths(schema);
 
     return schemaPaths.stream()
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchemaWithMetastore.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchemaWithMetastore.java
index 6ce32e4..c89472b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchemaWithMetastore.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchemaWithMetastore.java
@@ -209,21 +209,21 @@ public class TestInfoSchemaWithMetastore extends ClusterTest {
     schema.addColumn(varcharCol);
     schema.addColumn(timestampColumn);
 
-    Map<SchemaPath, ColumnStatistics> columnsStatistics = new HashMap<>();
+    Map<SchemaPath, ColumnStatistics<?>> columnsStatistics = new HashMap<>();
     columnsStatistics.put(SchemaPath.parseFromString("varchar_col"),
-      new ColumnStatistics(Arrays.asList(
+      new ColumnStatistics<>(Arrays.asList(
         new StatisticsHolder<>("aaa", ColumnStatisticsKind.MIN_VALUE),
         new StatisticsHolder<>("zzz", ColumnStatisticsKind.MAX_VALUE))));
     columnsStatistics.put(SchemaPath.parseFromString("struct_col.nested_struct.nested_struct_varchar"),
-      new ColumnStatistics(Arrays.asList(
+      new ColumnStatistics<>(Arrays.asList(
         new StatisticsHolder<>("bbb", ColumnStatisticsKind.MIN_VALUE),
         new StatisticsHolder<>("ccc", ColumnStatisticsKind.MAX_VALUE))));
     columnsStatistics.put(SchemaPath.parseFromString("bigint_col"),
-      new ColumnStatistics(Arrays.asList(
+      new ColumnStatistics<>(Arrays.asList(
         new StatisticsHolder<>(100L, ColumnStatisticsKind.NULLS_COUNT),
         new StatisticsHolder<>(10.5D, ColumnStatisticsKind.NDV))));
     columnsStatistics.put(SchemaPath.parseFromString("struct_col.struct_bigint"),
-      new ColumnStatistics(Collections.singletonList(
+      new ColumnStatistics<>(Collections.singletonList(
         new StatisticsHolder<>(10.5D, ColumnStatisticsKind.NON_NULL_COUNT))));
 
     ZonedDateTime currentTime = currentUtcTime();
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestMetastoreCommands.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestMetastoreCommands.java
index ff2108b..b8bf889 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestMetastoreCommands.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestMetastoreCommands.java
@@ -81,7 +81,6 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 @Category({SlowTest.class, MetastoreTest.class})
-@SuppressWarnings({"rawtypes", "unchecked"})
 public class TestMetastoreCommands extends ClusterTest {
 
   private static final TupleMetadata SCHEMA = new SchemaBuilder()
@@ -98,7 +97,8 @@ public class TestMetastoreCommands extends ClusterTest {
       .add("o_comment", TypeProtos.MinorType.VARCHAR)
       .build();
 
-  private static final Map<SchemaPath, ColumnStatistics> TABLE_COLUMN_STATISTICS = ImmutableMap.<SchemaPath, ColumnStatistics>builder()
+  private static final Map<SchemaPath, ColumnStatistics<?>> TABLE_COLUMN_STATISTICS =
+      ImmutableMap.<SchemaPath, ColumnStatistics<?>>builder()
       .put(SchemaPath.getSimplePath("o_shippriority"),
           getColumnStatistics(0, 0, 120L, TypeProtos.MinorType.INT))
       .put(SchemaPath.getSimplePath("o_orderstatus"),
@@ -124,7 +124,8 @@ public class TestMetastoreCommands extends ClusterTest {
           getColumnStatistics(757382400000L, 850953600000L, 120L, TypeProtos.MinorType.DATE))
       .build();
 
-  private static final Map<SchemaPath, ColumnStatistics> DIR0_1994_SEGMENT_COLUMN_STATISTICS = ImmutableMap.<SchemaPath, ColumnStatistics>builder()
+  private static final Map<SchemaPath, ColumnStatistics<?>> DIR0_1994_SEGMENT_COLUMN_STATISTICS =
+      ImmutableMap.<SchemaPath, ColumnStatistics<?>>builder()
       .put(SchemaPath.getSimplePath("o_shippriority"),
           getColumnStatistics(0, 0, 40L, TypeProtos.MinorType.INT))
       .put(SchemaPath.getSimplePath("o_orderstatus"),
@@ -150,7 +151,8 @@ public class TestMetastoreCommands extends ClusterTest {
           getColumnStatistics(757382400000L, 788140800000L, 40L, TypeProtos.MinorType.DATE))
       .build();
 
-  private static final Map<SchemaPath, ColumnStatistics> DIR0_1994_Q1_SEGMENT_COLUMN_STATISTICS = ImmutableMap.<SchemaPath, ColumnStatistics>builder()
+  private static final Map<SchemaPath, ColumnStatistics<?>> DIR0_1994_Q1_SEGMENT_COLUMN_STATISTICS =
+      ImmutableMap.<SchemaPath, ColumnStatistics<?>>builder()
       .put(SchemaPath.getSimplePath("o_shippriority"),
           getColumnStatistics(0, 0, 10L, TypeProtos.MinorType.INT))
       .put(SchemaPath.getSimplePath("o_orderstatus"),
@@ -594,7 +596,7 @@ public class TestMetastoreCommands extends ClusterTest {
 
     TableInfo tableInfo = getTableInfo(tableName, "tmp");
 
-    Map<SchemaPath, ColumnStatistics> updatedTableColumnStatistics = new HashMap<>();
+    Map<SchemaPath, ColumnStatistics<?>> updatedTableColumnStatistics = new HashMap<>();
 
     SchemaPath orderStatusPath = SchemaPath.getSimplePath("o_orderstatus");
     SchemaPath dir0Path = SchemaPath.getSimplePath("dir0");
@@ -646,7 +648,7 @@ public class TestMetastoreCommands extends ClusterTest {
 
     TableInfo tableInfo = getTableInfo(tableName, "tmp");
 
-    Map<SchemaPath, ColumnStatistics> updatedTableColumnStatistics = new HashMap<>();
+    Map<SchemaPath, ColumnStatistics<?>> updatedTableColumnStatistics = new HashMap<>();
 
     SchemaPath dir0Path = SchemaPath.getSimplePath("dir0");
     SchemaPath dir1Path = SchemaPath.getSimplePath("dir1");
@@ -696,7 +698,7 @@ public class TestMetastoreCommands extends ClusterTest {
 
     TableInfo tableInfo = getTableInfo(tableName, "tmp");
 
-    Map<SchemaPath, ColumnStatistics> updatedTableColumnStatistics = new HashMap<>();
+    Map<SchemaPath, ColumnStatistics<?>> updatedTableColumnStatistics = new HashMap<>();
 
     SchemaPath orderStatusPath = SchemaPath.getSimplePath("o_orderstatus");
     SchemaPath orderDatePath = SchemaPath.getSimplePath("o_orderdate");
@@ -767,7 +769,7 @@ public class TestMetastoreCommands extends ClusterTest {
 
     TableInfo tableInfo = getTableInfo(tableName, "tmp");
 
-    Map<SchemaPath, ColumnStatistics> updatedTableColumnStatistics = new HashMap<>();
+    Map<SchemaPath, ColumnStatistics<?>> updatedTableColumnStatistics = new HashMap<>();
 
     SchemaPath orderStatusPath = SchemaPath.getSimplePath("o_orderstatus");
     SchemaPath orderDatePath = SchemaPath.getSimplePath("o_orderdate");
@@ -844,7 +846,7 @@ public class TestMetastoreCommands extends ClusterTest {
 
     TableInfo tableInfo = getTableInfo(tableName, "tmp");
 
-    Map<SchemaPath, ColumnStatistics> updatedTableColumnStatistics = new HashMap<>();
+    Map<SchemaPath, ColumnStatistics<?>> updatedTableColumnStatistics = new HashMap<>();
 
     SchemaPath orderStatusPath = SchemaPath.getSimplePath("o_orderstatus");
     SchemaPath orderDatePath = SchemaPath.getSimplePath("o_orderdate");
@@ -977,7 +979,7 @@ public class TestMetastoreCommands extends ClusterTest {
     TableInfo tableInfo = getTableInfo(tableName, "tmp");
 
     // updates statistics values due to new segment
-    Map<SchemaPath, ColumnStatistics> updatedStatistics = new HashMap<>(TABLE_COLUMN_STATISTICS);
+    Map<SchemaPath, ColumnStatistics<?>> updatedStatistics = new HashMap<>(TABLE_COLUMN_STATISTICS);
     updatedStatistics.replaceAll((logicalExpressions, columnStatistics) ->
         columnStatistics.cloneWith(new ColumnStatistics<>(
             Arrays.asList(
@@ -1068,15 +1070,15 @@ public class TestMetastoreCommands extends ClusterTest {
     TableInfo tableInfo = getTableInfo(tableName, "tmp");
 
     // updates statistics values due to new segment
-    Map<SchemaPath, ColumnStatistics> updatedStatistics = new HashMap<>(TABLE_COLUMN_STATISTICS);
+    Map<SchemaPath, ColumnStatistics<?>> updatedStatistics = new HashMap<>(TABLE_COLUMN_STATISTICS);
     updatedStatistics.replaceAll((logicalExpressions, columnStatistics) ->
-        columnStatistics.cloneWith(new ColumnStatistics(
+        columnStatistics.cloneWith(new ColumnStatistics<>(
             Arrays.asList(
                 new StatisticsHolder<>(130L, TableStatisticsKind.ROW_COUNT),
                 new StatisticsHolder<>(130L, ColumnStatisticsKind.NON_NULL_VALUES_COUNT)))));
 
     updatedStatistics.computeIfPresent(SchemaPath.getSimplePath("dir1"), (logicalExpressions, columnStatistics) ->
-        columnStatistics.cloneWith(new ColumnStatistics(
+        columnStatistics.cloneWith(new ColumnStatistics<>(
             Collections.singletonList(new StatisticsHolder<>("Q5", ColumnStatisticsKind.MAX_VALUE)))));
 
     BaseTableMetadata expectedTableMetadata = BaseTableMetadata.builder()
@@ -1152,9 +1154,9 @@ public class TestMetastoreCommands extends ClusterTest {
     TableInfo tableInfo = getTableInfo(tableName, "tmp");
 
     // updates statistics values due to new segment
-    Map<SchemaPath, ColumnStatistics> updatedStatistics = new HashMap<>(TABLE_COLUMN_STATISTICS);
+    Map<SchemaPath, ColumnStatistics<?>> updatedStatistics = new HashMap<>(TABLE_COLUMN_STATISTICS);
     updatedStatistics.replaceAll((logicalExpressions, columnStatistics) ->
-        columnStatistics.cloneWith(new ColumnStatistics(
+        columnStatistics.cloneWith(new ColumnStatistics<>(
             Arrays.asList(
                 new StatisticsHolder<>(130L, TableStatisticsKind.ROW_COUNT),
                 new StatisticsHolder<>(130L, ColumnStatisticsKind.NON_NULL_VALUES_COUNT)))));
@@ -1625,15 +1627,15 @@ public class TestMetastoreCommands extends ClusterTest {
           .basicRequests()
           .tableMetadata(tableInfo);
 
-      Map<SchemaPath, ColumnStatistics> tableColumnStatistics = new HashMap<>(TABLE_COLUMN_STATISTICS);
+      Map<SchemaPath, ColumnStatistics<?>> tableColumnStatistics = new HashMap<>(TABLE_COLUMN_STATISTICS);
       tableColumnStatistics.computeIfPresent(SchemaPath.getSimplePath("o_clerk"),
           (logicalExpressions, columnStatistics) ->
-              columnStatistics.cloneWith(new ColumnStatistics(
+              columnStatistics.cloneWith(new ColumnStatistics<>(
                   Collections.singletonList(new StatisticsHolder<>("Clerk#000000006", ColumnStatisticsKind.MIN_VALUE)))));
 
       tableColumnStatistics.computeIfPresent(SchemaPath.getSimplePath("o_totalprice"),
           (logicalExpressions, columnStatistics) ->
-              columnStatistics.cloneWith(new ColumnStatistics(
+              columnStatistics.cloneWith(new ColumnStatistics<>(
                   Collections.singletonList(new StatisticsHolder<>(328207.15, ColumnStatisticsKind.MAX_VALUE)))));
 
       BaseTableMetadata expectedTableMetadata = BaseTableMetadata.builder()
@@ -1779,7 +1781,7 @@ public class TestMetastoreCommands extends ClusterTest {
 
     TableInfo tableInfo = getTableInfo(tableName, "tmp");
 
-    Map<SchemaPath, ColumnStatistics> tableColumnStatistics = new HashMap<>(TABLE_COLUMN_STATISTICS);
+    Map<SchemaPath, ColumnStatistics<?>> tableColumnStatistics = new HashMap<>(TABLE_COLUMN_STATISTICS);
     tableColumnStatistics.remove(SchemaPath.getSimplePath("dir0"));
     tableColumnStatistics.remove(SchemaPath.getSimplePath("dir1"));
 
@@ -1800,7 +1802,7 @@ public class TestMetastoreCommands extends ClusterTest {
             getColumnStatistics(757382400000L, 764640000000L, 120L, TypeProtos.MinorType.DATE));
 
     tableColumnStatistics.replaceAll((logicalExpressions, columnStatistics) ->
-        columnStatistics.cloneWith(new ColumnStatistics(
+        columnStatistics.cloneWith(new ColumnStatistics<>(
             Arrays.asList(
                 new StatisticsHolder<>(10L, TableStatisticsKind.ROW_COUNT),
                 new StatisticsHolder<>(10L, ColumnStatisticsKind.NON_NULL_VALUES_COUNT)))));
@@ -1905,7 +1907,7 @@ public class TestMetastoreCommands extends ClusterTest {
             .resumeSchema()
         .build();
 
-    Map<SchemaPath, ColumnStatistics> columnStatistics = ImmutableMap.<SchemaPath, ColumnStatistics>builder()
+    Map<SchemaPath, ColumnStatistics<?>> columnStatistics = ImmutableMap.<SchemaPath, ColumnStatistics<?>>builder()
         .put(SchemaPath.getCompoundPath("user_info", "state"),
             getColumnStatistics("ct", "nj", 5L, TypeProtos.MinorType.VARCHAR))
         .put(SchemaPath.getSimplePath("date"),
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java
index f198d3c..c44ef23 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java
@@ -105,35 +105,6 @@ public class TestFileGenerator {
     props.fields.put("S_COMMENT", new FieldInfo("binary", "bin2", -1, bin2Vals, TypeProtos.MinorType.VARBINARY, props));
   }
 
-  private static abstract class ValueProducer {
-
-    public abstract void reset();
-    public abstract Object getValue();
-  }
-
-  private static class ValueRepeaterProducer extends ValueProducer {
-
-    WrapAroundCounter position;
-    Object[] values;
-
-    public ValueRepeaterProducer(Object[] values) {
-      this.values = values;
-      position = new WrapAroundCounter(values.length);
-    }
-
-    @Override
-    public void reset() {
-      position.reset();
-    }
-
-    @Override
-    public Object getValue() {
-      Object ret = values[position.val];
-      position.increment();
-      return ret;
-    }
-  }
-
   public static void generateParquetFile(String filename, ParquetTestProperties props) throws Exception {
 
     int currentBooleanByte = 0;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
index 86a7bf9..e51e311 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
@@ -608,13 +608,13 @@ public class TestParquetFilterPushDown extends PlanTestBase {
     Mockito.when(booleanStatistics.get(ColumnStatisticsKind.MIN_VALUE)).thenReturn(false); // min false
     Mockito.when(booleanStatistics.get(ColumnStatisticsKind.MAX_VALUE)).thenReturn(true); // max true
     Mockito.when(booleanStatistics.getValueComparator()).thenReturn(Comparator.nullsFirst(Comparator.naturalOrder())); // comparator
-    IsPredicate isTrue = (IsPredicate) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_TRUE, le);
+    IsPredicate<Boolean> isTrue = (IsPredicate<Boolean>) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_TRUE, le);
     assertEquals(RowsMatch.SOME, isTrue.matches(re));
-    IsPredicate isFalse = (IsPredicate)  IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_FALSE, le);
+    IsPredicate<Boolean> isFalse = (IsPredicate<Boolean>) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_FALSE, le);
     assertEquals(RowsMatch.SOME, isFalse.matches(re));
-    IsPredicate isNotTrue = (IsPredicate)  IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_NOT_TRUE, le);
+    IsPredicate<Boolean> isNotTrue = (IsPredicate<Boolean>) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_NOT_TRUE, le);
     assertEquals(RowsMatch.SOME, isNotTrue.matches(re));
-    IsPredicate isNotFalse = (IsPredicate)  IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_NOT_FALSE, le);
+    IsPredicate<Boolean> isNotFalse = (IsPredicate<Boolean>) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_NOT_FALSE, le);
     assertEquals(RowsMatch.SOME, isNotFalse.matches(re));
   }
 
@@ -631,13 +631,13 @@ public class TestParquetFilterPushDown extends PlanTestBase {
     Mockito.when(booleanStatistics.get(ColumnStatisticsKind.MIN_VALUE)).thenReturn(false); // min false
     Mockito.when(booleanStatistics.get(ColumnStatisticsKind.MAX_VALUE)).thenReturn(false); // max false
     Mockito.when(booleanStatistics.getValueComparator()).thenReturn(Comparator.nullsFirst(Comparator.naturalOrder())); // comparator
-    IsPredicate isTrue = (IsPredicate) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_TRUE, le);
+    IsPredicate<Boolean> isTrue = (IsPredicate<Boolean>) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_TRUE, le);
     assertEquals(RowsMatch.NONE, isTrue.matches(re));
-    IsPredicate isFalse = (IsPredicate)  IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_FALSE, le);
+    IsPredicate<Boolean> isFalse = (IsPredicate<Boolean>) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_FALSE, le);
     assertEquals(RowsMatch.ALL, isFalse.matches(re));
-    IsPredicate isNotTrue = (IsPredicate)  IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_NOT_TRUE, le);
+    IsPredicate<Boolean> isNotTrue = (IsPredicate<Boolean>) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_NOT_TRUE, le);
     assertEquals(RowsMatch.ALL, isNotTrue.matches(re));
-    IsPredicate isNotFalse = (IsPredicate)  IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_NOT_FALSE, le);
+    IsPredicate<Boolean> isNotFalse = (IsPredicate<Boolean>) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_NOT_FALSE, le);
     assertEquals(RowsMatch.NONE, isNotFalse.matches(re));
   }
 
@@ -653,13 +653,13 @@ public class TestParquetFilterPushDown extends PlanTestBase {
     Mockito.when(booleanStatistics.get(ColumnStatisticsKind.NULLS_COUNT)).thenReturn(0L); // no nulls
     Mockito.when(booleanStatistics.get(ColumnStatisticsKind.MIN_VALUE)).thenReturn(true); // min false
     Mockito.when(booleanStatistics.get(ColumnStatisticsKind.MAX_VALUE)).thenReturn(true); // max false
-    IsPredicate isTrue = (IsPredicate) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_TRUE, le);
+    IsPredicate<Boolean> isTrue = (IsPredicate<Boolean>) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_TRUE, le);
     assertEquals(RowsMatch.ALL, isTrue.matches(re));
-    IsPredicate isFalse = (IsPredicate)  IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_FALSE, le);
+    IsPredicate<Boolean> isFalse = (IsPredicate<Boolean>) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_FALSE, le);
     assertEquals(RowsMatch.NONE, isFalse.matches(re));
-    IsPredicate isNotTrue = (IsPredicate)  IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_NOT_TRUE, le);
+    IsPredicate<Boolean> isNotTrue = (IsPredicate<Boolean>) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_NOT_TRUE, le);
     assertEquals(RowsMatch.NONE, isNotTrue.matches(re));
-    IsPredicate isNotFalse = (IsPredicate)  IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_NOT_FALSE, le);
+    IsPredicate<Boolean> isNotFalse = (IsPredicate<Boolean>) IsPredicate.createIsPredicate(FunctionGenerationHelper.IS_NOT_FALSE, le);
     assertEquals(RowsMatch.ALL, isNotFalse.matches(re));
   }
 
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/BasicTablesTransformer.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/BasicTablesTransformer.java
index 863ba9d..d25fd2e 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/BasicTablesTransformer.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/BasicTablesTransformer.java
@@ -99,6 +99,9 @@ public class BasicTablesTransformer {
         case PARTITION:
           partitions.add(PartitionMetadata.builder().metadataUnit(unit).build());
           break;
+        default:
+          // Ignore unsupported type
+          break;
       }
     }
     return new MetadataHolder(tables, segments, files, rowGroups, partitions);
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/TableMetadataUnit.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/TableMetadataUnit.java
index 7863a4d..1d6ed84 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/TableMetadataUnit.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/components/tables/TableMetadataUnit.java
@@ -511,6 +511,8 @@ public class TableMetadataUnit {
               rowGroupColumns.add(name);
               partitionColumns.add(name);
               break;
+            default:
+              throw new IllegalStateException(scope.name());
           }
         }
 
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/exceptions/MetastoreException.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/exceptions/MetastoreException.java
index 96516df..f94d128 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/exceptions/MetastoreException.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/exceptions/MetastoreException.java
@@ -18,9 +18,11 @@
 package org.apache.drill.metastore.exceptions;
 
 /**
- * Drill Metastore runtime exception to indicate that exception was caused by Drill Metastore.
- * Drill Metastore implementations can use or extend it to throw Metastore specific exceptions.
+ * Drill Metastore runtime exception to indicate that exception was caused by
+ * Drill Metastore. Drill Metastore implementations can use or extend it to
+ * throw Metastore specific exceptions.
  */
+@SuppressWarnings("serial")
 public class MetastoreException extends RuntimeException {
 
   public MetastoreException(String message, Throwable cause) {
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseMetadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseMetadata.java
index ae6c547..c24b54f 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseMetadata.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseMetadata.java
@@ -42,8 +42,8 @@ public abstract class BaseMetadata implements Metadata {
   protected final TableInfo tableInfo;
   protected final MetadataInfo metadataInfo;
   protected final TupleMetadata schema;
-  protected final Map<SchemaPath, ColumnStatistics> columnsStatistics;
-  protected final Map<String, StatisticsHolder> metadataStatistics;
+  protected final Map<SchemaPath, ColumnStatistics<?>> columnsStatistics;
+  protected final Map<String, StatisticsHolder<?>> metadataStatistics;
   protected final long lastModifiedTime;
 
   protected <T extends BaseMetadataBuilder<T>> BaseMetadata(BaseMetadataBuilder<T> builder) {
@@ -60,12 +60,12 @@ public abstract class BaseMetadata implements Metadata {
   }
 
   @Override
-  public Map<SchemaPath, ColumnStatistics> getColumnsStatistics() {
+  public Map<SchemaPath, ColumnStatistics<?>> getColumnsStatistics() {
     return columnsStatistics;
   }
 
   @Override
-  public ColumnStatistics getColumnStatistics(SchemaPath columnName) {
+  public ColumnStatistics<?> getColumnStatistics(SchemaPath columnName) {
     return columnsStatistics.get(columnName);
   }
 
@@ -77,20 +77,20 @@ public abstract class BaseMetadata implements Metadata {
   @Override
   @SuppressWarnings("unchecked")
   public <V> V getStatistic(StatisticsKind<V> statisticsKind) {
-    StatisticsHolder<V> statisticsHolder = metadataStatistics.get(statisticsKind.getName());
+    StatisticsHolder<V> statisticsHolder = (StatisticsHolder<V>)
+        metadataStatistics.get(statisticsKind.getName());
     return statisticsHolder != null ? statisticsHolder.getStatisticsValue() : null;
   }
 
   @Override
-  public boolean containsExactStatistics(StatisticsKind statisticsKind) {
-    StatisticsHolder statisticsHolder = metadataStatistics.get(statisticsKind.getName());
+  public boolean containsExactStatistics(StatisticsKind<?> statisticsKind) {
+    StatisticsHolder<?> statisticsHolder = metadataStatistics.get(statisticsKind.getName());
     return statisticsHolder != null && statisticsHolder.getStatisticsKind().isExact();
   }
 
   @Override
-  @SuppressWarnings("unchecked")
   public <V> V getStatisticsForColumn(SchemaPath columnName, StatisticsKind<V> statisticsKind) {
-    return (V) columnsStatistics.get(columnName).get(statisticsKind);
+    return columnsStatistics.get(columnName).get(statisticsKind);
   }
 
   @Override
@@ -172,14 +172,14 @@ public abstract class BaseMetadata implements Metadata {
 
   protected abstract void toMetadataUnitBuilder(TableMetadataUnit.Builder builder);
 
-  protected abstract BaseMetadataBuilder toBuilder();
+  protected abstract BaseMetadataBuilder<?> toBuilder();
 
   public static abstract class BaseMetadataBuilder<T extends BaseMetadataBuilder<T>> {
     protected TableInfo tableInfo;
     protected MetadataInfo metadataInfo;
     protected TupleMetadata schema;
-    protected Map<SchemaPath, ColumnStatistics> columnsStatistics;
-    protected Collection<StatisticsHolder> metadataStatistics;
+    protected Map<SchemaPath, ColumnStatistics<?>> columnsStatistics;
+    protected Collection<StatisticsHolder<?>> metadataStatistics;
     protected long lastModifiedTime = UNDEFINED_TIME;
 
     public T tableInfo(TableInfo tableInfo) {
@@ -197,12 +197,12 @@ public abstract class BaseMetadata implements Metadata {
       return self();
     }
 
-    public T columnsStatistics(Map<SchemaPath, ColumnStatistics> columnsStatistics) {
+    public T columnsStatistics(Map<SchemaPath, ColumnStatistics<?>> columnsStatistics) {
       this.columnsStatistics = columnsStatistics;
       return self();
     }
 
-    public T metadataStatistics(Collection<StatisticsHolder> metadataStatistics) {
+    public T metadataStatistics(Collection<StatisticsHolder<?>> metadataStatistics) {
       this.metadataStatistics = metadataStatistics;
       return self();
     }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseTableMetadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseTableMetadata.java
index 3b6922d..839cd51 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseTableMetadata.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/BaseTableMetadata.java
@@ -102,9 +102,8 @@ public class BaseTableMetadata extends BaseMetadata implements TableMetadata {
   }
 
   @Override
-  @SuppressWarnings("unchecked")
-  public BaseTableMetadata cloneWithStats(Map<SchemaPath, ColumnStatistics> columnStatistics, List<StatisticsHolder> tableStatistics) {
-    Map<String, StatisticsHolder> mergedTableStatistics = new HashMap<>(this.metadataStatistics);
+  public BaseTableMetadata cloneWithStats(Map<SchemaPath, ColumnStatistics<?>> columnStatistics, List<StatisticsHolder<?>> tableStatistics) {
+    Map<String, StatisticsHolder<?>> mergedTableStatistics = new HashMap<>(this.metadataStatistics);
 
     // overrides statistics value for the case when new statistics is exact or existing one was estimated
     tableStatistics.stream()
@@ -113,12 +112,12 @@ public class BaseTableMetadata extends BaseMetadata implements TableMetadata {
               || !this.metadataStatistics.get(statisticsHolder.getStatisticsKind().getName()).getStatisticsKind().isExact())
         .forEach(statisticsHolder -> mergedTableStatistics.put(statisticsHolder.getStatisticsKind().getName(), statisticsHolder));
 
-    Map<SchemaPath, ColumnStatistics> newColumnsStatistics = new HashMap<>(this.columnsStatistics);
+    Map<SchemaPath, ColumnStatistics<?>> newColumnsStatistics = new HashMap<>(this.columnsStatistics);
     this.columnsStatistics.forEach(
         (columnName, value) -> {
-          ColumnStatistics sourceStatistics = columnStatistics.get(columnName);
+          ColumnStatistics<?> sourceStatistics = columnStatistics.get(columnName);
           if (sourceStatistics != null) {
-            newColumnsStatistics.put(columnName, value.cloneWith(sourceStatistics));
+            newColumnsStatistics.put(columnName, value.genericClone(sourceStatistics));
           }
         });
 
@@ -148,6 +147,7 @@ public class BaseTableMetadata extends BaseMetadata implements TableMetadata {
     }
   }
 
+  @Override
   public BaseTableMetadataBuilder toBuilder() {
     return builder()
         .tableInfo(tableInfo)
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/Metadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/Metadata.java
index 6b40d95..3d99ae7 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/Metadata.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/Metadata.java
@@ -37,7 +37,7 @@ public interface Metadata {
    *
    * @return statistics stored in current metadata
    */
-  Map<SchemaPath, ColumnStatistics> getColumnsStatistics();
+  Map<SchemaPath, ColumnStatistics<?>> getColumnsStatistics();
 
   /**
    * Returns statistics for specified column stored in current metadata.
@@ -45,7 +45,7 @@ public interface Metadata {
    * @param columnName column whose statistics should be returned
    * @return statistics for specified column
    */
-  ColumnStatistics getColumnStatistics(SchemaPath columnName);
+  ColumnStatistics<?> getColumnStatistics(SchemaPath columnName);
 
   /**
    * Returns schema stored in current metadata represented as
@@ -70,7 +70,7 @@ public interface Metadata {
    * @param statisticsKind statistics kind to check
    * @return true if value which corresponds to the specified statistics kind is exact
    */
-  boolean containsExactStatistics(StatisticsKind statisticsKind);
+  boolean containsExactStatistics(StatisticsKind<?> statisticsKind);
 
   /**
    * Returns value of column statistics which corresponds to specified {@link StatisticsKind}
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/NonInterestingColumnsMetadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/NonInterestingColumnsMetadata.java
index 6944ab0..26cf665 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/NonInterestingColumnsMetadata.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/NonInterestingColumnsMetadata.java
@@ -34,19 +34,19 @@ import java.util.Map;
  * to NonInterestingColumnsMetadata.
  */
 public class NonInterestingColumnsMetadata implements Metadata {
-  private final Map<SchemaPath, ColumnStatistics> columnsStatistics;
+  private final Map<SchemaPath, ColumnStatistics<?>> columnsStatistics;
 
-  public NonInterestingColumnsMetadata(Map<SchemaPath, ColumnStatistics> columnsStatistics) {
+  public NonInterestingColumnsMetadata(Map<SchemaPath, ColumnStatistics<?>> columnsStatistics) {
     this.columnsStatistics = columnsStatistics;
   }
 
   @Override
-  public Map<SchemaPath, ColumnStatistics> getColumnsStatistics() {
+  public Map<SchemaPath, ColumnStatistics<?>> getColumnsStatistics() {
     return columnsStatistics;
   }
 
   @Override
-  public ColumnStatistics getColumnStatistics(SchemaPath columnName) {
+  public ColumnStatistics<?> getColumnStatistics(SchemaPath columnName) {
     return columnsStatistics.get(columnName);
   }
 
@@ -61,14 +61,13 @@ public class NonInterestingColumnsMetadata implements Metadata {
   }
 
   @Override
-  public boolean containsExactStatistics(StatisticsKind statisticsKind) {
+  public boolean containsExactStatistics(StatisticsKind<?> statisticsKind) {
     return false;
   }
 
   @Override
-  @SuppressWarnings("unchecked")
   public <V> V getStatisticsForColumn(SchemaPath columnName, StatisticsKind<V> statisticsKind) {
-    return (V) columnsStatistics.get(columnName).get(statisticsKind);
+    return columnsStatistics.get(columnName).get(statisticsKind);
   }
 
   @Override
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableMetadata.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableMetadata.java
index 517d232..d4704fc 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableMetadata.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/metadata/TableMetadata.java
@@ -32,6 +32,7 @@ public interface TableMetadata extends Metadata {
 
   Path getLocation();
   long getLastModifiedTime();
-  TableMetadata cloneWithStats(Map<SchemaPath, ColumnStatistics> columnStatistics, List<StatisticsHolder> tableStatistics);
+  TableMetadata cloneWithStats(Map<SchemaPath, ColumnStatistics<?>> columnStatistics,
+      List<StatisticsHolder<?>> tableStatistics);
   List<SchemaPath> getInterestingColumns();
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/CollectableColumnStatisticsKind.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/CollectableColumnStatisticsKind.java
index 00b1f1d..8c4a2a8 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/CollectableColumnStatisticsKind.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/CollectableColumnStatisticsKind.java
@@ -32,5 +32,5 @@ public interface CollectableColumnStatisticsKind<V> extends StatisticsKind<V> {
    * @param statistics list of {@link ColumnStatistics} instances to be collected
    * @return column statistics value received by collecting specified {@link ColumnStatistics}
    */
-  Object mergeStatistics(List<? extends ColumnStatistics> statistics);
+  Object mergeStatistics(List<? extends ColumnStatistics<?>> statistics);
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatistics.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatistics.java
index 0cb33db..5fbb85a 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatistics.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatistics.java
@@ -74,13 +74,13 @@ public class ColumnStatistics<T> {
       .registerModule(new JodaModule())
       .readerFor(ColumnStatistics.class);
 
-  private final Map<String, StatisticsHolder> statistics;
+  private final Map<String, StatisticsHolder<?>> statistics;
   private final Comparator<T> valueComparator;
   private final TypeProtos.MinorType type;
 
   @JsonCreator
   @SuppressWarnings("unchecked")
-  public ColumnStatistics(@JsonProperty("statistics") Collection<StatisticsHolder> statistics,
+  public ColumnStatistics(@JsonProperty("statistics") Collection<StatisticsHolder<?>> statistics,
                           @JsonProperty("type") TypeProtos.MinorType type) {
     this.type = type;
     this.valueComparator = type != null
@@ -93,7 +93,7 @@ public class ColumnStatistics<T> {
             (a, b) -> a.getStatisticsKind().isExact() ? a : b));
   }
 
-  public ColumnStatistics(Collection<StatisticsHolder> statistics) {
+  public ColumnStatistics(Collection<StatisticsHolder<?>> statistics) {
     this(statistics, TypeProtos.MinorType.INT);
   }
 
@@ -105,7 +105,8 @@ public class ColumnStatistics<T> {
    */
   @SuppressWarnings("unchecked")
   public <V> V get(StatisticsKind<V> statisticsKind) {
-    StatisticsHolder<V> statisticsHolder = statistics.get(statisticsKind.getName());
+    StatisticsHolder<V> statisticsHolder = (StatisticsHolder<V>)
+        statistics.get(statisticsKind.getName());
     if (statisticsHolder != null) {
       return statisticsHolder.getStatisticsValue();
     }
@@ -118,7 +119,7 @@ public class ColumnStatistics<T> {
    * @param statisticsKind statistics kind to check
    * @return true if specified statistics kind is set
    */
-  public boolean contains(StatisticsKind statisticsKind) {
+  public boolean contains(StatisticsKind<?> statisticsKind) {
     return statistics.containsKey(statisticsKind.getName());
   }
 
@@ -129,8 +130,8 @@ public class ColumnStatistics<T> {
    * @param statisticsKind statistics kind to check
    * @return true if value which corresponds to the specified statistics kind is exact
    */
-  public boolean containsExact(StatisticsKind statisticsKind) {
-    StatisticsHolder statisticsHolder = statistics.get(statisticsKind.getName());
+  public boolean containsExact(StatisticsKind<?> statisticsKind) {
+    StatisticsHolder<?> statisticsHolder = statistics.get(statisticsKind.getName());
     if (statisticsHolder != null) {
       return statisticsHolder.getStatisticsKind().isExact();
     }
@@ -153,10 +154,10 @@ public class ColumnStatistics<T> {
    * @return new {@link ColumnStatistics} instance with overridden statistics
    */
   public ColumnStatistics<T> cloneWith(ColumnStatistics<T> sourceStatistics) {
-    Map<String, StatisticsHolder> newStats = new HashMap<>(this.statistics);
+    Map<String, StatisticsHolder<?>> newStats = new HashMap<>(this.statistics);
     sourceStatistics.statistics.values().forEach(statisticsHolder -> {
-      StatisticsKind statisticsKindToMerge = statisticsHolder.getStatisticsKind();
-      StatisticsHolder oldStatistics = statistics.get(statisticsKindToMerge.getName());
+      StatisticsKind<?> statisticsKindToMerge = statisticsHolder.getStatisticsKind();
+      StatisticsHolder<?> oldStatistics = statistics.get(statisticsKindToMerge.getName());
       if (oldStatistics == null
           || !oldStatistics.getStatisticsKind().isExact()
           || statisticsKindToMerge.isExact()) {
@@ -167,9 +168,14 @@ public class ColumnStatistics<T> {
     return new ColumnStatistics<>(newStats.values(), type);
   }
 
+  @SuppressWarnings("unchecked")
+  public ColumnStatistics<T> genericClone(ColumnStatistics<?> sourceStatistics) {
+    return cloneWith((ColumnStatistics<T>) sourceStatistics);
+  }
+
   @JsonProperty("statistics")
   @SuppressWarnings("unused") // used for serialization
-  private Collection<StatisticsHolder> getAll() {
+  private Collection<StatisticsHolder<?>> getAll() {
     return statistics.values();
   }
 
@@ -212,7 +218,7 @@ public class ColumnStatistics<T> {
         .toString();
   }
 
-  public static ColumnStatistics of(String columnStatistics) {
+  public static ColumnStatistics<?> of(String columnStatistics) {
     try {
       return OBJECT_READER.readValue(columnStatistics);
     } catch (IOException e) {
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatisticsKind.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatisticsKind.java
index 613b602..5cfed03 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatisticsKind.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatisticsKind.java
@@ -19,6 +19,7 @@ package org.apache.drill.metastore.statistics;
 
 import org.apache.drill.metastore.metadata.BaseMetadata;
 
+import java.util.Comparator;
 import java.util.List;
 
 /**
@@ -33,7 +34,7 @@ public class ColumnStatisticsKind<T> extends BaseStatisticsKind<T> implements Co
   public static final ColumnStatisticsKind<Long> NULLS_COUNT =
       new ColumnStatisticsKind<Long>(ExactStatisticsConstants.NULLS_COUNT, true) {
         @Override
-        public Long mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
+        public Long mergeStatistics(List<? extends ColumnStatistics<?>> statisticsList) {
           long nullsCount = 0;
           for (ColumnStatistics<?> statistics : statisticsList) {
             Long statNullsCount = statistics.get(this);
@@ -47,7 +48,7 @@ public class ColumnStatisticsKind<T> extends BaseStatisticsKind<T> implements Co
         }
 
         @Override
-        public Long getFrom(ColumnStatistics metadata) {
+        public Long getFrom(ColumnStatistics<?> metadata) {
           Long rowCount = super.getFrom(metadata);
           return rowCount != null ? rowCount : Statistic.NO_COLUMN_STATS;
         }
@@ -60,11 +61,12 @@ public class ColumnStatisticsKind<T> extends BaseStatisticsKind<T> implements Co
       new ColumnStatisticsKind<Object>(ExactStatisticsConstants.MIN_VALUE, true) {
         @Override
         @SuppressWarnings("unchecked")
-        public Object mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
+        public Object mergeStatistics(List<? extends ColumnStatistics<?>> statisticsList) {
           Object minValue = null;
-          for (ColumnStatistics statistics : statisticsList) {
+          for (ColumnStatistics<?> statistics : statisticsList) {
             Object statMinValue = getValueStatistic(statistics);
-            if (statMinValue != null && (statistics.getValueComparator().compare(minValue, statMinValue) > 0 || minValue == null)) {
+            Comparator<Object> comp = (Comparator<Object>) statistics.getValueComparator();
+            if (statMinValue != null && (comp.compare(minValue, statMinValue) > 0 || minValue == null)) {
               minValue = statMinValue;
             }
           }
@@ -79,11 +81,12 @@ public class ColumnStatisticsKind<T> extends BaseStatisticsKind<T> implements Co
       new ColumnStatisticsKind<Object>(ExactStatisticsConstants.MAX_VALUE, true) {
         @Override
         @SuppressWarnings("unchecked")
-        public Object mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
+        public Object mergeStatistics(List<? extends ColumnStatistics<?>> statisticsList) {
           Object maxValue = null;
-          for (ColumnStatistics statistics : statisticsList) {
+          for (ColumnStatistics<?> statistics : statisticsList) {
             Object statMaxValue = getValueStatistic(statistics);
-            if (statMaxValue != null && statistics.getValueComparator().compare(maxValue, statMaxValue) < 0) {
+            Comparator<Object> comp = (Comparator<Object>) statistics.getValueComparator();
+            if (statMaxValue != null && comp.compare(maxValue, statMaxValue) < 0) {
               maxValue = statMaxValue;
             }
           }
@@ -97,7 +100,7 @@ public class ColumnStatisticsKind<T> extends BaseStatisticsKind<T> implements Co
   public static final ColumnStatisticsKind<Long> NON_NULL_VALUES_COUNT =
       new ColumnStatisticsKind<Long>(ExactStatisticsConstants.NON_NULL_VALUES_COUNT, true) {
         @Override
-        public Long mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
+        public Long mergeStatistics(List<? extends ColumnStatistics<?>> statisticsList) {
           long nonNullRowCount = 0;
           for (ColumnStatistics<?> statistics : statisticsList) {
             Long nnRowCount = statistics.get(this);
@@ -115,7 +118,7 @@ public class ColumnStatisticsKind<T> extends BaseStatisticsKind<T> implements Co
   public static final ColumnStatisticsKind<Double> NON_NULL_COUNT =
       new ColumnStatisticsKind<Double>(Statistic.NNROWCOUNT, false) {
         @Override
-        public Double mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
+        public Double mergeStatistics(List<? extends ColumnStatistics<?>> statisticsList) {
           double nonNullRowCount = 0;
           for (ColumnStatistics<?> statistics : statisticsList) {
             Double nnRowCount = statistics.get(this);
@@ -133,7 +136,7 @@ public class ColumnStatisticsKind<T> extends BaseStatisticsKind<T> implements Co
   public static final ColumnStatisticsKind<Double> ROWCOUNT =
       new ColumnStatisticsKind<Double>(Statistic.ROWCOUNT, false) {
         @Override
-        public Double mergeStatistics(List<? extends ColumnStatistics> statisticsList) {
+        public Double mergeStatistics(List<? extends ColumnStatistics<?>> statisticsList) {
           double rowCount = 0;
           for (ColumnStatistics<?> statistics : statisticsList) {
             Double count = getFrom(statistics);
@@ -154,8 +157,8 @@ public class ColumnStatisticsKind<T> extends BaseStatisticsKind<T> implements Co
   /**
    * Column statistics kind which is the width of the specific column.
    */
-  public static final ColumnStatisticsKind AVG_WIDTH =
-      new ColumnStatisticsKind(Statistic.AVG_WIDTH, false);
+  public static final ColumnStatisticsKind<?> AVG_WIDTH =
+      new ColumnStatisticsKind<>(Statistic.AVG_WIDTH, false);
 
   /**
    * Column statistics kind which is the histogram of the specific column.
@@ -184,7 +187,7 @@ public class ColumnStatisticsKind<T> extends BaseStatisticsKind<T> implements Co
   }
 
   @Override
-  public T mergeStatistics(List<? extends ColumnStatistics> statistics) {
+  public T mergeStatistics(List<? extends ColumnStatistics<?>> statistics) {
     throw new UnsupportedOperationException("Cannot merge statistics for " + statisticKey);
   }
 }
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/StatisticsHolder.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/StatisticsHolder.java
index 94e8b10..ffbe3ac 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/StatisticsHolder.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/StatisticsHolder.java
@@ -42,19 +42,19 @@ public class StatisticsHolder<T> {
   private static final ObjectReader OBJECT_READER = new ObjectMapper().readerFor(StatisticsHolder.class);
 
   private final T statisticsValue;
-  private final BaseStatisticsKind statisticsKind;
+  private final BaseStatisticsKind<?> statisticsKind;
 
   @JsonCreator
   public StatisticsHolder(@JsonProperty("statisticsValue") T statisticsValue,
-                          @JsonProperty("statisticsKind") BaseStatisticsKind statisticsKind) {
+                          @JsonProperty("statisticsKind") BaseStatisticsKind<?> statisticsKind) {
     this.statisticsValue = statisticsValue;
     this.statisticsKind = statisticsKind;
   }
 
   public StatisticsHolder(T statisticsValue,
-                          StatisticsKind statisticsKind) {
+                          StatisticsKind<?> statisticsKind) {
     this.statisticsValue = statisticsValue;
-    this.statisticsKind = (BaseStatisticsKind) statisticsKind;
+    this.statisticsKind = (BaseStatisticsKind<?>) statisticsKind;
   }
 
   @JsonTypeInfo(use = JsonTypeInfo.Id.CLASS,
@@ -63,7 +63,7 @@ public class StatisticsHolder<T> {
     return statisticsValue;
   }
 
-  public StatisticsKind getStatisticsKind() {
+  public StatisticsKind<?> getStatisticsKind() {
     return statisticsKind;
   }
 
@@ -101,7 +101,7 @@ public class StatisticsHolder<T> {
         .toString();
   }
 
-  public static StatisticsHolder of(String serialized) {
+  public static StatisticsHolder<?> of(String serialized) {
     try {
       return OBJECT_READER.readValue(serialized);
     } catch (IOException e) {
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/util/TableMetadataUtils.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/util/TableMetadataUtils.java
index 1aac570..2443663 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/util/TableMetadataUtils.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/util/TableMetadataUtils.java
@@ -46,21 +46,32 @@ public class TableMetadataUtils {
    * @param type type of the column
    * @return {@link Comparator} instance
    */
-  public static Comparator getComparator(TypeProtos.MinorType type) {
+  @SuppressWarnings("unchecked")
+  public static <T> Comparator<T> getComparator(TypeProtos.MinorType type) {
     switch (type) {
       case INTERVALDAY:
       case INTERVAL:
       case INTERVALYEAR:
-        return Comparator.nullsFirst(UnsignedBytes.lexicographicalComparator());
+        // This odd cast is needed because this method is poorly designed.
+        // The method is statically typed to type T. But, the type
+        // is selected dynamically at runtime via the type parameter.
+        // As a result, we are casting a comparator to the WRONG type
+        // in some cases. We have to remove the byte[] type, then force
+        // the type to T. This works because we should only use this
+        // case if T is byte[]. But, this is a horrible hack and should
+        // be fixed.
+        return (Comparator<T>) (Comparator<?>)
+            Comparator.nullsFirst(UnsignedBytes.lexicographicalComparator());
       case UINT1:
-        return Comparator.nullsFirst(UnsignedBytes::compare);
+        return (Comparator<T>)
+            Comparator.nullsFirst(UnsignedBytes::compare);
       case UINT2:
       case UINT4:
-        return Comparator.nullsFirst(Integer::compareUnsigned);
+        return (Comparator<T>) Comparator.nullsFirst(Integer::compareUnsigned);
       case UINT8:
-        return Comparator.nullsFirst(Long::compareUnsigned);
+        return (Comparator<T>) Comparator.nullsFirst(Long::compareUnsigned);
       default:
-        return getNaturalNullsFirstComparator();
+        return (Comparator<T>) getNaturalNullsFirstComparator();
     }
   }
 
@@ -83,14 +94,15 @@ public class TableMetadataUtils {
    * @param statisticsToCollect kinds of statistics that should be collected
    * @return list of merged metadata
    */
-  public static <T extends BaseMetadata> Map<SchemaPath, ColumnStatistics> mergeColumnsStatistics(
-      Collection<T> metadataList, Set<SchemaPath> columns, List<CollectableColumnStatisticsKind> statisticsToCollect) {
-    Map<SchemaPath, ColumnStatistics> columnsStatistics = new HashMap<>();
+  @SuppressWarnings({ "unchecked", "rawtypes" })
+  public static <T extends BaseMetadata> Map<SchemaPath, ColumnStatistics<?>> mergeColumnsStatistics(
+      Collection<T> metadataList, Set<SchemaPath> columns, List<CollectableColumnStatisticsKind<?>> statisticsToCollect) {
+    Map<SchemaPath, ColumnStatistics<?>> columnsStatistics = new HashMap<>();
 
     for (SchemaPath column : columns) {
-      List<ColumnStatistics> statisticsList = new ArrayList<>();
+      List<ColumnStatistics<?>> statisticsList = new ArrayList<>();
       for (T metadata : metadataList) {
-        ColumnStatistics statistics = metadata.getColumnsStatistics().get(column);
+        ColumnStatistics<?> statistics = metadata.getColumnsStatistics().get(column);
         if (statistics == null) {
           // schema change happened, set statistics which represents all nulls
           statistics = new ColumnStatistics(
@@ -99,12 +111,12 @@ public class TableMetadataUtils {
         }
         statisticsList.add(statistics);
       }
-      List<StatisticsHolder> statisticsHolders = new ArrayList<>();
-      for (CollectableColumnStatisticsKind statisticsKind : statisticsToCollect) {
+      List<StatisticsHolder<?>> statisticsHolders = new ArrayList<>();
+      for (CollectableColumnStatisticsKind<?> statisticsKind : statisticsToCollect) {
         Object mergedStatistic = statisticsKind.mergeStatistics(statisticsList);
         statisticsHolders.add(new StatisticsHolder<>(mergedStatistic, statisticsKind));
       }
-      Iterator<ColumnStatistics> iterator = statisticsList.iterator();
+      Iterator<ColumnStatistics<?>> iterator = statisticsList.iterator();
       // Use INT if statistics wasn't provided
       TypeProtos.MinorType comparatorType = iterator.hasNext() ? iterator.next().getComparatorType() : TypeProtos.MinorType.INT;
       columnsStatistics.put(column, new ColumnStatistics<>(statisticsHolders, comparatorType));
@@ -120,13 +132,13 @@ public class TableMetadataUtils {
    * @return new {@link TableMetadata} instance with updated statistics
    */
   public static TableMetadata updateRowCount(TableMetadata tableMetadata, Collection<? extends BaseMetadata> statistics) {
-    List<StatisticsHolder> newStats = new ArrayList<>();
+    List<StatisticsHolder<?>> newStats = new ArrayList<>();
 
     newStats.add(new StatisticsHolder<>(TableStatisticsKind.ROW_COUNT.mergeStatistics(statistics), TableStatisticsKind.ROW_COUNT));
 
     Set<SchemaPath> columns = tableMetadata.getColumnsStatistics().keySet();
 
-    Map<SchemaPath, ColumnStatistics> columnsStatistics =
+    Map<SchemaPath, ColumnStatistics<?>> columnsStatistics =
         mergeColumnsStatistics(statistics, columns,
             Collections.singletonList(ColumnStatisticsKind.NULLS_COUNT));
 
diff --git a/metastore/metastore-api/src/test/java/org/apache/drill/metastore/components/tables/TestTableMetadataUnitConversion.java b/metastore/metastore-api/src/test/java/org/apache/drill/metastore/components/tables/TestTableMetadataUnitConversion.java
index c31c254..7a6212c 100644
--- a/metastore/metastore-api/src/test/java/org/apache/drill/metastore/components/tables/TestTableMetadataUnitConversion.java
+++ b/metastore/metastore-api/src/test/java/org/apache/drill/metastore/components/tables/TestTableMetadataUnitConversion.java
@@ -387,9 +387,9 @@ public class TestTableMetadataUnitConversion extends BaseTest {
 
     private final TableInfo fullTableInfo;
     private final TableInfo basicTableInfo;
-    private final Map<SchemaPath, ColumnStatistics> columnsStatistics;
+    private final Map<SchemaPath, ColumnStatistics<?>> columnsStatistics;
     private final Map<String, String> unitColumnsStatistics;
-    private final Collection<StatisticsHolder> metadataStatistics;
+    private final Collection<StatisticsHolder<?>> metadataStatistics;
     private final List<String> unitMetadataStatistics;
     private final TupleMetadata schema;
     private final String unitSchema;
diff --git a/metastore/metastore-api/src/test/java/org/apache/drill/metastore/metadata/MetadataSerDeTest.java b/metastore/metastore-api/src/test/java/org/apache/drill/metastore/metadata/MetadataSerDeTest.java
index 520e59e..bc6fad2 100644
--- a/metastore/metastore-api/src/test/java/org/apache/drill/metastore/metadata/MetadataSerDeTest.java
+++ b/metastore/metastore-api/src/test/java/org/apache/drill/metastore/metadata/MetadataSerDeTest.java
@@ -70,7 +70,7 @@ public class MetadataSerDeTest extends BaseTest {
 
   @Test
   public void testColumnStatisticsSerialization() {
-    List<StatisticsHolder> statistics = Arrays.asList(
+    List<StatisticsHolder<?>> statistics = Arrays.asList(
         new StatisticsHolder<>("aaa", ColumnStatisticsKind.MIN_VALUE),
         new StatisticsHolder<>("zzz", ColumnStatisticsKind.MAX_VALUE),
         new StatisticsHolder<>(3, ColumnStatisticsKind.NULLS_COUNT),
@@ -95,7 +95,7 @@ public class MetadataSerDeTest extends BaseTest {
 
   @Test
   public void testColumnStatisticsDeserialization() {
-    List<StatisticsHolder> statistics = Arrays.asList(
+    List<StatisticsHolder<?>> statistics = Arrays.asList(
         new StatisticsHolder<>("aaa", ColumnStatisticsKind.MIN_VALUE),
         new StatisticsHolder<>("zzz", ColumnStatisticsKind.MAX_VALUE),
         new StatisticsHolder<>(3, ColumnStatisticsKind.NULLS_COUNT),
@@ -103,13 +103,13 @@ public class MetadataSerDeTest extends BaseTest {
     ColumnStatistics<String> columnStatistics = new ColumnStatistics<>(statistics, TypeProtos.MinorType.VARCHAR);
     String serializedColumnStatistics = columnStatistics.jsonString();
 
-    ColumnStatistics deserialized = ColumnStatistics.of(serializedColumnStatistics);
+    ColumnStatistics<?> deserialized = ColumnStatistics.of(serializedColumnStatistics);
 
     assertEquals("Type was incorrectly deserialized",
         columnStatistics.getComparatorType(),
         deserialized.getComparatorType());
 
-    for (StatisticsHolder statistic : statistics) {
+    for (StatisticsHolder<?> statistic : statistics) {
       assertEquals("Statistics kind was incorrectly deserialized",
           statistic.getStatisticsKind().isExact(),
           deserialized.containsExact(statistic.getStatisticsKind()));
@@ -120,7 +120,7 @@ public class MetadataSerDeTest extends BaseTest {
   }
 
   private <T> void checkStatisticsHolderSerialization(T statisticsValue,
-      BaseStatisticsKind statisticsKind, String expectedString) {
+      BaseStatisticsKind<?> statisticsKind, String expectedString) {
     StatisticsHolder<T> statisticsHolder =
         new StatisticsHolder<>(statisticsValue, statisticsKind);
     String serializedStatisticsHolder = statisticsHolder.jsonString();
@@ -131,10 +131,10 @@ public class MetadataSerDeTest extends BaseTest {
   }
 
   private <T> void checkStatisticsHolderDeserialization(T statisticsValue,
-      BaseStatisticsKind statisticsKind) {
+      BaseStatisticsKind<?> statisticsKind) {
     StatisticsHolder<T> rowCount =
         new StatisticsHolder<>(statisticsValue, statisticsKind);
-    StatisticsHolder deserializedRowCount = StatisticsHolder.of(rowCount.jsonString());
+    StatisticsHolder<?> deserializedRowCount = StatisticsHolder.of(rowCount.jsonString());
 
     assertTrue("Statistics value was incorrectly deserialized",
         Objects.deepEquals(rowCount.getStatisticsValue(), deserializedRowCount.getStatisticsValue()));
@@ -142,7 +142,7 @@ public class MetadataSerDeTest extends BaseTest {
     assertStatisticsKindsEquals(rowCount, deserializedRowCount);
   }
 
-  private <T> void assertStatisticsKindsEquals(StatisticsHolder<T> expected, StatisticsHolder actual) {
+  private <T> void assertStatisticsKindsEquals(StatisticsHolder<T> expected, StatisticsHolder<?> actual) {
     assertEquals("isExact statistics kind was incorrectly deserialized",
         expected.getStatisticsKind().isExact(),
         actual.getStatisticsKind().isExact());