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

[drill] branch master updated: DRILL-7314: Use TupleMetadata instead of concrete implementation

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

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


The following commit(s) were added to refs/heads/master by this push:
     new e843588  DRILL-7314: Use TupleMetadata instead of concrete implementation
e843588 is described below

commit e843588178dd25408c447217aac25c0970924ebc
Author: Arina Ielchiieva <ar...@gmail.com>
AuthorDate: Fri Jul 5 16:43:37 2019 +0300

    DRILL-7314: Use TupleMetadata instead of concrete implementation
    
    1. Add ser / de implementation for TupleMetadata interface based on types.
    2. Replace TupleSchema usage where possible.
    3. Move patcher classes into commons.
    4. Upgrade some dependencies and general refactoring.
---
 common/pom.xml                                     |  2 +-
 .../apache/drill/common}/util/GuavaPatcher.java    | 14 +++--
 .../apache/drill/common}/util/ProtobufPatcher.java |  8 ++-
 .../drill/exec/store/mapr/db/MapRDBSubScan.java    |  4 +-
 .../store/mapr/db/RestrictedMapRDBSubScan.java     | 19 +++---
 .../store/mapr/db/binary/BinaryTableGroupScan.java | 10 ++-
 .../store/mapr/db/json/JsonTableGroupScan.java     | 27 ++++----
 .../mapr/db/json/RestrictedJsonTableGroupScan.java |  5 +-
 .../drill/maprdb/tests/index/TableIndexCmd.java    |  2 +-
 .../org/apache/drill/hbase/HBaseTestsSuite.java    |  4 +-
 .../hive/HiveDrillNativeParquetRowGroupScan.java   | 30 +++++----
 .../store/hive/HiveDrillNativeParquetScan.java     |  3 +-
 exec/java-exec/pom.xml                             |  2 +-
 .../drill/exec/coord/ClusterCoordinator.java       | 16 ++---
 .../base/AbstractGroupScanWithMetadata.java        | 12 ++--
 .../exec/physical/rowSet/impl/TupleState.java      |  2 +-
 .../exec/physical/rowSet/model/BaseTupleModel.java |  6 +-
 .../rowSet/model/hyper/HyperSchemaInference.java   |  2 +-
 .../record/metadata/schema/SchemaContainer.java    |  3 +-
 .../org/apache/drill/exec/server/Drillbit.java     | 18 +++---
 .../exec/store/StoragePluginRegistryImpl.java      |  2 +-
 .../drill/exec/store/dfs/easy/EasyGroupScan.java   |  4 +-
 .../drill/exec/store/dfs/easy/EasySubScan.java     |  3 +-
 .../store/parquet/AbstractParquetGroupScan.java    |  1 -
 .../store/parquet/AbstractParquetRowGroupScan.java | 20 +++---
 .../parquet/AbstractParquetScanBatchCreator.java   | 20 +++---
 .../drill/exec/store/parquet/ParquetGroupScan.java | 10 +--
 .../exec/store/parquet/ParquetRowGroupScan.java    | 15 +++--
 .../org/apache/drill/exec/work/WorkManager.java    | 71 +++++++++-------------
 .../test/java/org/apache/drill/exec/ExecTest.java  |  2 +-
 .../physical/impl/join/TestHashJoinOutcome.java    | 20 +++---
 .../exec/record/metadata/TestTupleSchema.java      | 47 ++++++++++++++
 .../record/metadata/schema/TestSchemaProvider.java |  1 +
 .../drill/exec/store/pcapng/TestPcapngHeaders.java | 13 ++--
 exec/jdbc-all/pom.xml                              |  2 +-
 .../drill/exec/record/metadata/TupleMetadata.java  | 51 +++++++++++++++-
 .../drill/exec/record/metadata/TupleSchema.java    |  2 +
 logical/pom.xml                                    |  2 +-
 pom.xml                                            | 30 ++++-----
 39 files changed, 289 insertions(+), 216 deletions(-)

diff --git a/common/pom.xml b/common/pom.xml
index bc3a9aa..47b60fc 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -42,7 +42,7 @@
       <!-- add as provided scope so that we can compile TestTools.  Should only be ever used in a test scenario where someone else is bringing JUnit in. -->
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
-      <version>${dep.junit.version}</version>
+      <version>${junit.version}</version>
     </dependency>
 
     <dependency>
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/GuavaPatcher.java b/common/src/main/java/org/apache/drill/common/util/GuavaPatcher.java
similarity index 89%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/util/GuavaPatcher.java
rename to common/src/main/java/org/apache/drill/common/util/GuavaPatcher.java
index 3269aa8..22a4cb5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/GuavaPatcher.java
+++ b/common/src/main/java/org/apache/drill/common/util/GuavaPatcher.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.util;
+package org.apache.drill.common.util;
 
 import java.lang.reflect.Modifier;
 
@@ -24,9 +24,11 @@ import javassist.CtClass;
 import javassist.CtConstructor;
 import javassist.CtMethod;
 import javassist.CtNewMethod;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class GuavaPatcher {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(GuavaPatcher.class);
+  private static final Logger logger = LoggerFactory.getLogger(GuavaPatcher.class);
 
   private static boolean patched;
 
@@ -58,9 +60,9 @@ public class GuavaPatcher {
     }
 
     // Add back the Stopwatch.elapsedMillis() method for old consumers.
-    CtMethod newmethod = CtNewMethod.make(
+    CtMethod newMethod = CtNewMethod.make(
         "public long elapsedMillis() { return elapsed(java.util.concurrent.TimeUnit.MILLISECONDS); }", cc);
-    cc.addMethod(newmethod);
+    cc.addMethod(newMethod);
 
     // Load the modified class instead of the original.
     cc.toClass();
@@ -75,10 +77,10 @@ public class GuavaPatcher {
 
 
     // Add back the Closeables.closeQuietly() method for old consumers.
-    CtMethod newmethod = CtNewMethod.make(
+    CtMethod newMethod = CtNewMethod.make(
         "public static void closeQuietly(java.io.Closeable closeable) { try{closeable.close();}catch(Exception e){} }",
         cc);
-    cc.addMethod(newmethod);
+    cc.addMethod(newMethod);
 
     // Load the modified class instead of the original.
     cc.toClass();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/ProtobufPatcher.java b/common/src/main/java/org/apache/drill/common/util/ProtobufPatcher.java
similarity index 97%
rename from exec/java-exec/src/main/java/org/apache/drill/exec/util/ProtobufPatcher.java
rename to common/src/main/java/org/apache/drill/common/util/ProtobufPatcher.java
index 77144ae..b0c11a5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/ProtobufPatcher.java
+++ b/common/src/main/java/org/apache/drill/common/util/ProtobufPatcher.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.drill.exec.util;
+package org.apache.drill.common.util;
 
 import javassist.CannotCompileException;
 import javassist.ClassPool;
@@ -26,12 +26,14 @@ import javassist.CtNewConstructor;
 import javassist.CtNewMethod;
 import javassist.Modifier;
 import javassist.NotFoundException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ProtobufPatcher {
 
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProtobufPatcher.class);
+  private static final Logger logger = LoggerFactory.getLogger(ProtobufPatcher.class);
 
-  private static volatile boolean patchingAttempted = false;
+  private static boolean patchingAttempted = false;
 
   /**
    * Makes protobuf version 3.6+ compatible to libraries that still use protobuf 2.5.0.
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java
index 9b41243..d01a63b 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java
@@ -30,7 +30,6 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 
 import com.fasterxml.jackson.annotation.JacksonInject;
@@ -60,8 +59,7 @@ public class MapRDBSubScan extends AbstractDbSubScan {
                        @JsonProperty("columns") List<SchemaPath> columns,
                        @JsonProperty("maxRecordsToRead") int maxRecordsToRead,
                        @JsonProperty("tableType") String tableType,
-                       // TODO: DRILL-7314 - replace TupleSchema with TupleMetadata
-                       @JsonProperty("schema") TupleSchema schema) throws ExecutionSetupException {
+                       @JsonProperty("schema") TupleMetadata schema) throws ExecutionSetupException {
     this(userName,
         (MapRDBFormatPlugin) engineRegistry.getFormatPlugin(storageConfig, formatPluginConfig),
         regionScanSpecList,
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScan.java
index 8b696c7..c1369a3 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScan.java
@@ -30,7 +30,6 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.impl.join.RowKeyJoin;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 
 /**
@@ -39,19 +38,17 @@ import org.apache.drill.exec.store.StoragePluginRegistry;
 */
 @JsonTypeName("maprdb-restricted-subscan")
 public class RestrictedMapRDBSubScan extends MapRDBSubScan {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RestrictedMapRDBSubScan.class);
 
   @JsonCreator
   public RestrictedMapRDBSubScan(@JacksonInject StoragePluginRegistry engineRegistry,
-                       @JsonProperty("userName") String userName,
-                       @JsonProperty("formatPluginConfig") MapRDBFormatPluginConfig formatPluginConfig,
-                       @JsonProperty("storageConfig") StoragePluginConfig storageConfig,
-                       @JsonProperty("regionScanSpecList") List<RestrictedMapRDBSubScanSpec> regionScanSpecList,
-                       @JsonProperty("columns") List<SchemaPath> columns,
-                       @JsonProperty("maxRecordsToRead") int maxRecordsToRead,
-                       @JsonProperty("tableType") String tableType,
-                       // TODO: DRILL-7314 - replace TupleSchema with TupleMetadata
-                       @JsonProperty("schema") TupleSchema schema) throws ExecutionSetupException {
+                                 @JsonProperty("userName") String userName,
+                                 @JsonProperty("formatPluginConfig") MapRDBFormatPluginConfig formatPluginConfig,
+                                 @JsonProperty("storageConfig") StoragePluginConfig storageConfig,
+                                 @JsonProperty("regionScanSpecList") List<RestrictedMapRDBSubScanSpec> regionScanSpecList,
+                                 @JsonProperty("columns") List<SchemaPath> columns,
+                                 @JsonProperty("maxRecordsToRead") int maxRecordsToRead,
+                                 @JsonProperty("tableType") String tableType,
+                                 @JsonProperty("schema") TupleMetadata schema) throws ExecutionSetupException {
     this(userName,
         (MapRDBFormatPlugin) engineRegistry.getFormatPlugin(storageConfig, formatPluginConfig),
         regionScanSpecList, columns, maxRecordsToRead, tableType, schema);
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
index 96f96ca..b2873a4 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/BinaryTableGroupScan.java
@@ -32,7 +32,7 @@ import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.ScanStats;
 import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
-import org.apache.drill.exec.record.metadata.TupleSchema;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.planner.index.Statistics;
 import org.apache.drill.exec.store.StoragePluginRegistry;
@@ -85,8 +85,7 @@ public class BinaryTableGroupScan extends MapRDBGroupScan implements DrillHBaseC
                               @JsonProperty("storage") FileSystemConfig storagePluginConfig,
                               @JsonProperty("format") MapRDBFormatPluginConfig formatPluginConfig,
                               @JsonProperty("columns") List<SchemaPath> columns,
-                              // TODO: DRILL-7314 - replace TupleSchema with TupleMetadata
-                              @JsonProperty("schema") TupleSchema schema,
+                              @JsonProperty("schema") TupleMetadata schema,
                               @JacksonInject StoragePluginRegistry pluginRegistry) throws ExecutionSetupException, IOException {
     this(userName, (AbstractStoragePlugin) pluginRegistry.getPlugin(storagePluginConfig),
         (MapRDBFormatPlugin) pluginRegistry.getFormatPlugin(storagePluginConfig, formatPluginConfig),
@@ -140,7 +139,7 @@ public class BinaryTableGroupScan extends MapRDBGroupScan implements DrillHBaseC
         tableStats = new MapRDBTableStats(getHBaseConf(), hbaseScanSpec.getTableName());
       }
       boolean foundStartRegion = false;
-      final TreeMap<TabletFragmentInfo, String> regionsToScan = new TreeMap<TabletFragmentInfo, String>();
+      final TreeMap<TabletFragmentInfo, String> regionsToScan = new TreeMap<>();
       List<HRegionLocation> regionLocations = locator.getAllRegionLocations();
       for (HRegionLocation regionLocation : regionLocations) {
         HRegionInfo regionInfo = regionLocation.getRegionInfo();
@@ -163,7 +162,7 @@ public class BinaryTableGroupScan extends MapRDBGroupScan implements DrillHBaseC
 
   protected MapRDBSubScanSpec getSubScanSpec(TabletFragmentInfo tfi) {
     HBaseScanSpec spec = hbaseScanSpec;
-    MapRDBSubScanSpec subScanSpec = new MapRDBSubScanSpec(
+    return new MapRDBSubScanSpec(
         spec.getTableName(),
         null /* indexFid */,
         getRegionsToScan().get(tfi),
@@ -172,7 +171,6 @@ public class BinaryTableGroupScan extends MapRDBGroupScan implements DrillHBaseC
         spec.getSerializedFilter(),
         null,
         getUserName());
-    return subScanSpec;
   }
 
   @Override
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
index 414c7f3..0047d29 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
@@ -43,7 +43,7 @@ import org.apache.drill.exec.physical.base.IndexGroupScan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.ScanStats;
 import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
-import org.apache.drill.exec.record.metadata.TupleSchema;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.planner.index.IndexDescriptor;
 import org.apache.drill.exec.planner.index.MapRDBIndexDescriptor;
@@ -123,8 +123,7 @@ public class JsonTableGroupScan extends MapRDBGroupScan implements IndexGroupSca
                             @JsonProperty("storage") FileSystemConfig storagePluginConfig,
                             @JsonProperty("format") MapRDBFormatPluginConfig formatPluginConfig,
                             @JsonProperty("columns") List<SchemaPath> columns,
-                            // TODO: DRILL-7314 - replace TupleSchema with TupleMetadata
-                            @JsonProperty("schema") TupleSchema schema,
+                            @JsonProperty("schema") TupleMetadata schema,
                             @JacksonInject StoragePluginRegistry pluginRegistry) throws ExecutionSetupException, IOException {
     this(userName, (AbstractStoragePlugin) pluginRegistry.getPlugin(storagePluginConfig),
         (MapRDBFormatPlugin) pluginRegistry.getFormatPlugin(storagePluginConfig, formatPluginConfig),
@@ -476,13 +475,12 @@ public class JsonTableGroupScan extends MapRDBGroupScan implements IndexGroupSca
   public RestrictedJsonTableGroupScan getRestrictedScan(List<SchemaPath> columns) {
     try {
       RestrictedJsonTableGroupScan newScan = new RestrictedJsonTableGroupScan(this.getUserName(),
-            (FileSystemPlugin) this.getStoragePlugin(),
-            this.getFormatPlugin(),
-            this.getScanSpec(),
-            this.getColumns(),
-            this.getStatistics(),
-            // TODO: DRILL-7314 - replace TupleSchema with TupleMetadata
-            (TupleSchema) this.getSchema());
+        (FileSystemPlugin) this.getStoragePlugin(),
+        this.getFormatPlugin(),
+        this.getScanSpec(),
+        this.getColumns(),
+        this.getStatistics(),
+        this.getSchema());
 
       newScan.columns = columns;
       return newScan;
@@ -670,7 +668,7 @@ public class JsonTableGroupScan extends MapRDBGroupScan implements IndexGroupSca
     double fullTableSize;
 
     if (useNumRegions) {
-      return getMaxParallelizationWidth() > 1 ? true: false;
+      return getMaxParallelizationWidth() > 1;
     }
 
     // This function gets called multiple times during planning. To avoid performance
@@ -695,7 +693,7 @@ public class JsonTableGroupScan extends MapRDBGroupScan implements IndexGroupSca
       fullTableSize = rowCount * rowSize;
     }
 
-    return (long) fullTableSize / scanRangeSize > 1 ? true : false;
+    return (long) fullTableSize / scanRangeSize > 1;
   }
 
   @Override
@@ -738,8 +736,9 @@ public class JsonTableGroupScan extends MapRDBGroupScan implements IndexGroupSca
   }
 
   /**
-   * Json table reader support limit
-   * @return
+   * Checks if Json table reader supports limit push down.
+   *
+   * @return true if limit push down is supported, false otherwise
    */
   @Override
   public boolean supportsLimitPushdown() {
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java
index e1285eb..398fcc1 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/RestrictedJsonTableGroupScan.java
@@ -25,7 +25,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
-import org.apache.drill.exec.record.metadata.TupleSchema;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.metastore.FileSystemMetadataProviderManager;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
@@ -61,8 +61,7 @@ public class RestrictedJsonTableGroupScan extends JsonTableGroupScan {
                                       @JsonProperty("scanSpec") JsonScanSpec scanSpec, /* scan spec of the original table */
                                       @JsonProperty("columns") List<SchemaPath> columns,
                                       @JsonProperty("") MapRDBStatistics statistics,
-                                      // TODO: DRILL-7314 - replace TupleSchema with TupleMetadata
-                                      @JsonProperty("schema") TupleSchema schema) throws IOException {
+                                      @JsonProperty("schema") TupleMetadata schema) throws IOException {
     super(userName, storagePlugin, formatPlugin, scanSpec, columns,
         statistics, FileSystemMetadataProviderManager.getMetadataProviderForSchema(schema));
   }
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/TableIndexCmd.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/TableIndexCmd.java
index bf50195..f766f7f 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/TableIndexCmd.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/TableIndexCmd.java
@@ -20,7 +20,7 @@ package com.mapr.drill.maprdb.tests.index;
 
 import com.mapr.db.Admin;
 import com.mapr.db.MapRDB;
-import org.apache.drill.exec.util.GuavaPatcher;
+import org.apache.drill.common.util.GuavaPatcher;
 
 import java.util.HashMap;
 import java.util.Map;
diff --git a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/HBaseTestsSuite.java b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/HBaseTestsSuite.java
index 5cd4d3d..8460647 100644
--- a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/HBaseTestsSuite.java
+++ b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/HBaseTestsSuite.java
@@ -22,8 +22,8 @@ import java.lang.management.ManagementFactory;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.drill.exec.ZookeeperTestUtil;
-import org.apache.drill.exec.util.GuavaPatcher;
-import org.apache.drill.exec.util.ProtobufPatcher;
+import org.apache.drill.common.util.GuavaPatcher;
+import org.apache.drill.common.util.ProtobufPatcher;
 import org.apache.drill.hbase.test.Drill2130StorageHBaseHamcrestConfigurationTest;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetRowGroupScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetRowGroupScan.java
index 09533bb..d5436eb 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetRowGroupScan.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetRowGroupScan.java
@@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
-import org.apache.drill.exec.record.metadata.TupleSchema;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.store.parquet.ParquetReaderConfig;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -49,7 +49,6 @@ public class HiveDrillNativeParquetRowGroupScan extends AbstractParquetRowGroupS
   private final HiveStoragePluginConfig hiveStoragePluginConfig;
   private final HivePartitionHolder hivePartitionHolder;
   private final Map<String, String> confProperties;
-  private final TupleSchema tupleSchema;
 
   @JsonCreator
   public HiveDrillNativeParquetRowGroupScan(@JacksonInject StoragePluginRegistry registry,
@@ -61,16 +60,16 @@ public class HiveDrillNativeParquetRowGroupScan extends AbstractParquetRowGroupS
                                             @JsonProperty("confProperties") Map<String, String> confProperties,
                                             @JsonProperty("readerConfig") ParquetReaderConfig readerConfig,
                                             @JsonProperty("filter") LogicalExpression filter,
-                                            @JsonProperty("tupleScema") TupleSchema tupleSchema) throws ExecutionSetupException {
+                                            @JsonProperty("schema") TupleMetadata schema) throws ExecutionSetupException {
     this(userName,
-        (HiveStoragePlugin) registry.getPlugin(hiveStoragePluginConfig),
-        rowGroupReadEntries,
-        columns,
-        hivePartitionHolder,
-        confProperties,
-        readerConfig,
-        filter,
-        tupleSchema);
+      (HiveStoragePlugin) registry.getPlugin(hiveStoragePluginConfig),
+      rowGroupReadEntries,
+      columns,
+      hivePartitionHolder,
+      confProperties,
+      readerConfig,
+      filter,
+      schema);
   }
 
   public HiveDrillNativeParquetRowGroupScan(String userName,
@@ -81,13 +80,12 @@ public class HiveDrillNativeParquetRowGroupScan extends AbstractParquetRowGroupS
                                             Map<String, String> confProperties,
                                             ParquetReaderConfig readerConfig,
                                             LogicalExpression filter,
-                                            TupleSchema tupleSchema) {
-    super(userName, rowGroupReadEntries, columns, readerConfig, filter,null, tupleSchema);
+                                            TupleMetadata schema) {
+    super(userName, rowGroupReadEntries, columns, readerConfig, filter,null, schema);
     this.hiveStoragePlugin = Preconditions.checkNotNull(hiveStoragePlugin, "Could not find format config for the given configuration");
     this.hiveStoragePluginConfig = hiveStoragePlugin.getConfig();
     this.hivePartitionHolder = hivePartitionHolder;
     this.confProperties = confProperties;
-    this.tupleSchema = tupleSchema;
   }
 
   @JsonProperty
@@ -114,7 +112,7 @@ public class HiveDrillNativeParquetRowGroupScan extends AbstractParquetRowGroupS
   public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
     Preconditions.checkArgument(children.isEmpty());
     return new HiveDrillNativeParquetRowGroupScan(getUserName(), hiveStoragePlugin, rowGroupReadEntries, columns, hivePartitionHolder,
-      confProperties, readerConfig, filter, tupleSchema);
+      confProperties, readerConfig, filter, schema);
   }
 
   @Override
@@ -125,7 +123,7 @@ public class HiveDrillNativeParquetRowGroupScan extends AbstractParquetRowGroupS
   @Override
   public AbstractParquetRowGroupScan copy(List<SchemaPath> columns) {
     return new HiveDrillNativeParquetRowGroupScan(getUserName(), hiveStoragePlugin, rowGroupReadEntries, columns, hivePartitionHolder,
-      confProperties, readerConfig, filter, tupleSchema);
+      confProperties, readerConfig, filter, schema);
   }
 
   @Override
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java
index 710ab7f..8b89488 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeParquetScan.java
@@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JacksonInject;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
-import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.store.parquet.ParquetReaderConfig;
 import org.apache.drill.metastore.metadata.LocationProvider;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
@@ -144,7 +143,7 @@ public class HiveDrillNativeParquetScan extends AbstractParquetGroupScan {
       subPartitionHolder.add(readEntry.getPath(), values);
     }
     return new HiveDrillNativeParquetRowGroupScan(getUserName(), hiveStoragePlugin, readEntries, columns, subPartitionHolder,
-      confProperties, readerConfig, filter, (TupleSchema) getTableMetadata().getSchema());
+      confProperties, readerConfig, filter, getTableMetadata().getSchema());
   }
 
   @Override
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 4d19395..a80bc48 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -121,7 +121,7 @@
     <dependency>
       <groupId>com.sun.codemodel</groupId>
       <artifactId>codemodel</artifactId>
-      <version>2.6</version>
+      <version>${codemodel.version}</version>
     </dependency>
     <dependency>
       <groupId>org.eclipse.jetty</groupId>
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
index aa7f66a..c9bc3b4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
@@ -32,7 +32,6 @@ import org.apache.drill.exec.work.foreman.DrillbitStatusListener;
  * as well as understand other node's existence and capabilities.
  **/
 public abstract class ClusterCoordinator implements AutoCloseable {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClusterCoordinator.class);
 
   protected ConcurrentHashMap<DrillbitStatusListener, DrillbitStatusListener> listeners = new ConcurrentHashMap<>(
       16, 0.75f, 16);
@@ -44,7 +43,7 @@ public abstract class ClusterCoordinator implements AutoCloseable {
    *          The maximum time to wait before throwing an exception if the
    *          cluster coordination service has not successfully started. Use 0
    *          to wait indefinitely.
-   * @throws Exception
+   * @throws Exception in case when unable to start coordinator
    */
   public abstract void start(long millisToWait) throws Exception;
 
@@ -77,9 +76,9 @@ public abstract class ClusterCoordinator implements AutoCloseable {
      * Get the drillbit endpoint associated with the registration handle
      * @return drillbit endpoint
      */
-    public abstract DrillbitEndpoint getEndPoint();
+    DrillbitEndpoint getEndPoint();
 
-    public abstract void setEndPoint(DrillbitEndpoint endpoint);
+    void setEndPoint(DrillbitEndpoint endpoint);
   }
 
   public abstract DistributedSemaphore getSemaphore(String name, int maximumLeases);
@@ -96,7 +95,8 @@ public abstract class ClusterCoordinator implements AutoCloseable {
 
   /**
    * Actions to take when there are a set of new de-active drillbits.
-   * @param unregisteredBits
+   *
+   * @param unregisteredBits set of drillbits to unregister
    */
   protected void drillbitUnregistered(Set<DrillbitEndpoint> unregisteredBits) {
     for (DrillbitStatusListener listener : listeners.keySet()) {
@@ -113,7 +113,8 @@ public abstract class ClusterCoordinator implements AutoCloseable {
   /**
    * Register a DrillbitStatusListener.
    * Note : the listeners are not guaranteed to be called in the order in which they call this method, since all the listeners are in a ConcurrentHashMap.
-   * @param listener
+   *
+   * @param listener status listener
    */
   public void addDrillbitStatusListener(DrillbitStatusListener listener) {
     listeners.putIfAbsent(listener, listener);
@@ -121,7 +122,8 @@ public abstract class ClusterCoordinator implements AutoCloseable {
 
   /**
    * Unregister a DrillbitStatusListener.
-   * @param listener
+   *
+   * @param listener status listener
    */
   public void removeDrillbitStatusListener(DrillbitStatusListener listener) {
     listeners.remove(listener);
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 98646b4..eecf992 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
@@ -348,7 +348,7 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
                                                    OptionManager optionManager,
                                                    boolean omitUnsupportedExprs) {
     return getFilterPredicate(filterExpr, udfUtilities, functionImplementationRegistry, optionManager,
-            omitUnsupportedExprs, supportsFileImplicitColumns(), (TupleSchema) getTableMetadata().getSchema());
+            omitUnsupportedExprs, supportsFileImplicitColumns(), getTableMetadata().getSchema());
   }
 
   /**
@@ -361,7 +361,7 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
    * @param omitUnsupportedExprs           whether expressions which cannot be converted
    *                                       may be omitted from the resulting expression
    * @param supportsFileImplicitColumns    whether implicit columns are supported
-   * @param tupleSchema                    schema
+   * @param schema                         schema
    * @return parquet filter predicate
    */
   public static FilterPredicate getFilterPredicate(LogicalExpression filterExpr,
@@ -370,8 +370,8 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
                                             OptionManager optionManager,
                                             boolean omitUnsupportedExprs,
                                             boolean supportsFileImplicitColumns,
-                                            TupleSchema tupleSchema) {
-    TupleMetadata types = tupleSchema.copy();
+                                            TupleMetadata schema) {
+    TupleMetadata types = schema.copy();
 
     Set<SchemaPath> schemaPathsInExpr = filterExpr.accept(new FilterEvaluatorUtils.FieldReferenceFinder(), null);
 
@@ -402,9 +402,9 @@ public abstract class AbstractGroupScanWithMetadata extends AbstractFileGroupSca
   }
 
   @JsonIgnore
-  public TupleSchema getSchema() {
+  public TupleMetadata getSchema() {
     // creates a copy of TupleMetadata from tableMetadata
-    TupleSchema tuple = new TupleSchema();
+    TupleMetadata tuple = new TupleSchema();
     for (ColumnMetadata md : getTableMetadata().getSchema()) {
       tuple.addColumn(md.copy());
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java
index d0b44a4..3711a2d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/impl/TupleState.java
@@ -437,7 +437,7 @@ public abstract class TupleState extends ContainerState
    * Internal writer schema that matches the column list.
    */
 
-  protected final TupleSchema schema = new TupleSchema();
+  protected final TupleMetadata schema = new TupleSchema();
 
   /**
    * Metadata description of the output container (for the row) or map
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/BaseTupleModel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/BaseTupleModel.java
index 9320d11..fd4a11f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/BaseTupleModel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/BaseTupleModel.java
@@ -62,7 +62,7 @@ public abstract class BaseTupleModel implements TupleModel {
    * {@link org.apache.drill.exec.record.VectorContainer}, this abstraction keeps the schema in sync
    * with vectors as columns are added.
    */
-  protected final TupleSchema schema;
+  protected final TupleMetadata schema;
 
   public BaseTupleModel() {
 
@@ -74,7 +74,7 @@ public abstract class BaseTupleModel implements TupleModel {
     columns = new ArrayList<>();
   }
 
-  public BaseTupleModel(TupleSchema schema, List<ColumnModel> columns) {
+  public BaseTupleModel(TupleMetadata schema, List<ColumnModel> columns) {
     this.schema = schema;
     this.columns = columns;
     assert schema.size() == columns.size();
@@ -105,7 +105,7 @@ public abstract class BaseTupleModel implements TupleModel {
    * @param column column implementation to add
    */
   protected void addBaseColumn(BaseColumnModel column) {
-    schema.add(column.schema());
+    schema.addColumn(column.schema());
     columns.add(column);
     assert columns.size() == schema.size();
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/hyper/HyperSchemaInference.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/hyper/HyperSchemaInference.java
index 3e14866..e4e8e28 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/hyper/HyperSchemaInference.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/rowSet/model/hyper/HyperSchemaInference.java
@@ -40,7 +40,7 @@ import org.apache.drill.exec.vector.ValueVector;
 public class HyperSchemaInference {
 
   public TupleMetadata infer(VectorContainer container) throws SchemaChangeException {
-    TupleSchema schema = new TupleSchema();
+    TupleMetadata schema = new TupleSchema();
     for (int i = 0; i < container.getNumberOfColumns(); i++) {
       VectorWrapper<?> vw = container.getValueVector(i);
       schema.addColumn(buildColumn(vw));
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/schema/SchemaContainer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/schema/SchemaContainer.java
index cdd56ce..74e058d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/schema/SchemaContainer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/schema/SchemaContainer.java
@@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonInclude;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.record.metadata.schema.parser.SchemaExprParser;
 import java.io.IOException;
 import java.util.Map;
@@ -39,7 +38,7 @@ public class SchemaContainer {
 
   @JsonCreator
   public SchemaContainer(@JsonProperty("table") String table,
-                         @JsonProperty("schema") TupleSchema schema,
+                         @JsonProperty("schema") TupleMetadata schema,
                          @JsonProperty("version") Integer version) {
     this.table = table;
     this.schema = schema;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
index 215fbb0..10f6849 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
@@ -46,8 +46,8 @@ import org.apache.drill.exec.store.sys.PersistentStoreRegistry;
 import org.apache.drill.exec.store.sys.store.provider.CachingPersistentStoreProvider;
 import org.apache.drill.exec.store.sys.store.provider.InMemoryStoreProvider;
 import org.apache.drill.exec.store.sys.store.provider.LocalPersistentStoreProvider;
-import org.apache.drill.exec.util.GuavaPatcher;
-import org.apache.drill.exec.util.ProtobufPatcher;
+import org.apache.drill.common.util.GuavaPatcher;
+import org.apache.drill.common.util.ProtobufPatcher;
 import org.apache.drill.exec.work.WorkManager;
 import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
 import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
@@ -93,8 +93,6 @@ public class Drillbit implements AutoCloseable {
 
   public final static String SYSTEM_OPTIONS_NAME = "org.apache.drill.exec.server.Drillbit.system_options";
 
-  private boolean isClosed = false;
-
   private final ClusterCoordinator coord;
   private final ServiceEngine engine;
   private final PersistentStoreProvider storeProvider;
@@ -179,7 +177,7 @@ public class Drillbit implements AutoCloseable {
       String drillClusterPath = "/" + zkRoot + "/" +  clusterId;
       ACLProvider aclProvider = ZKACLProviderFactory.getACLProvider(config, drillClusterPath, context);
       coord = new ZKClusterCoordinator(config, aclProvider);
-      storeProvider = new PersistentStoreRegistry<ClusterCoordinator>(this.coord, config).newPStoreProvider();
+      storeProvider = new PersistentStoreRegistry<>(this.coord, config).newPStoreProvider();
     }
 
     //Check if InMemory Profile Store, else use Default Store Provider
@@ -289,7 +287,7 @@ public class Drillbit implements AutoCloseable {
     //safe to exit
     updateState(State.OFFLINE);
     stateManager.setState(DrillbitState.OFFLINE);
-    if(quiescentMode == true) {
+    if(quiescentMode) {
       return;
     }
     if (coord != null && registrationHandle != null) {
@@ -342,9 +340,9 @@ public class Drillbit implements AutoCloseable {
     final SystemOptionManager optionManager = getContext().getOptionManager();
 
     // parse out the properties, validate, and then set them
-    final String systemProps[] = allSystemProps.split(",");
+    final String[] systemProps = allSystemProps.split(",");
     for (final String systemProp : systemProps) {
-      final String keyValue[] = systemProp.split("=");
+      final String[] keyValue = systemProp.split("=");
       if (keyValue.length != 2) {
         throwInvalidSystemOption(systemProp, "does not contain a key=value assignment");
       }
@@ -379,7 +377,7 @@ public class Drillbit implements AutoCloseable {
 
     private final Drillbit drillbit;
     private final StackTrace stackTrace;
-    public GracefulShutdownThread(final Drillbit drillbit, final StackTrace stackTrace) {
+    GracefulShutdownThread(final Drillbit drillbit, final StackTrace stackTrace) {
       this.drillbit = drillbit;
       this.stackTrace = stackTrace;
     }
@@ -453,7 +451,7 @@ public class Drillbit implements AutoCloseable {
      * @param stackTrace the stack trace from where the Drillbit was started;
      *   use new StackTrace() to generate this
      */
-    public ShutdownThread(final Drillbit drillbit, final StackTrace stackTrace) {
+    ShutdownThread(final Drillbit drillbit, final StackTrace stackTrace) {
       this.drillbit = drillbit;
       this.stackTrace = stackTrace;
       /*
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java
index 23029ee..0fbfe4f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java
@@ -473,7 +473,7 @@ public class StoragePluginRegistryImpl implements StoragePluginRegistry {
       }
       if (i == 0) {
         logger.debug("Skipping registration of StoragePlugin {} as it doesn't have a constructor with the parameters "
-            + "of (StorangePluginConfig, Config)", plugin.getCanonicalName());
+            + "of (StoragePluginConfig, Config)", plugin.getCanonicalName());
       }
     }
     return availablePlugins;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
index ac83a71..197b5a3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
@@ -36,7 +36,6 @@ import org.apache.drill.metastore.metadata.TableMetadataProvider;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.store.ColumnExplorer;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
@@ -90,8 +89,7 @@ public class EasyGroupScan extends AbstractFileGroupScan {
       @JacksonInject StoragePluginRegistry engineRegistry,
       @JsonProperty("columns") List<SchemaPath> columns,
       @JsonProperty("selectionRoot") Path selectionRoot,
-      // TODO: DRILL-7314 - replace TupleSchema with TupleMetadata
-      @JsonProperty("schema") TupleSchema schema
+      @JsonProperty("schema") TupleMetadata schema
       ) throws IOException, ExecutionSetupException {
     super(ImpersonationUtil.resolveUserName(userName));
     this.selection = FileSelection.create(null, files, selectionRoot);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
index e307660..82e5cac 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
@@ -25,7 +25,6 @@ import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.AbstractSubScan;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
-import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.schedule.CompleteFileWork.FileWorkImpl;
 
@@ -57,7 +56,7 @@ public class EasySubScan extends AbstractSubScan {
     @JsonProperty("columns") List<SchemaPath> columns,
     @JsonProperty("selectionRoot") Path selectionRoot,
     @JsonProperty("partitionDepth") int partitionDepth,
-    @JsonProperty("schema") TupleSchema schema
+    @JsonProperty("schema") TupleMetadata schema
     ) throws ExecutionSetupException {
     super(userName);
     this.formatPlugin = (EasyFormatPlugin<?>) engineRegistry.getFormatPlugin(storageConfig, formatConfig);
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 e9dc8a2..9b253f5 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
@@ -618,7 +618,6 @@ public abstract class AbstractParquetGroupScan extends AbstractGroupScanWithMeta
           }
         }
       }
-
       return prunedFiles;
     }
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetRowGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetRowGroupScan.java
index f05175e..460fa6c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetRowGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetRowGroupScan.java
@@ -27,7 +27,7 @@ import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.SubScan;
-import org.apache.drill.exec.record.metadata.TupleSchema;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 
@@ -43,22 +43,22 @@ public abstract class AbstractParquetRowGroupScan extends AbstractBase implement
   protected final ParquetReaderConfig readerConfig;
   protected final LogicalExpression filter;
   protected final Path selectionRoot;
-  protected final TupleSchema tupleSchema;
+  protected final TupleMetadata schema;
 
   protected AbstractParquetRowGroupScan(String userName,
-                                     List<RowGroupReadEntry> rowGroupReadEntries,
-                                     List<SchemaPath> columns,
-                                     ParquetReaderConfig readerConfig,
-                                     LogicalExpression filter,
-                                     Path selectionRoot,
-                                     TupleSchema tupleSchema) {
+                                        List<RowGroupReadEntry> rowGroupReadEntries,
+                                        List<SchemaPath> columns,
+                                        ParquetReaderConfig readerConfig,
+                                        LogicalExpression filter,
+                                        Path selectionRoot,
+                                        TupleMetadata schema) {
     super(userName);
     this.rowGroupReadEntries = rowGroupReadEntries;
     this.columns = columns == null ? GroupScan.ALL_COLUMNS : columns;
     this.readerConfig = readerConfig == null ? ParquetReaderConfig.getDefaultInstance() : readerConfig;
     this.filter = filter;
     this.selectionRoot = selectionRoot;
-    this.tupleSchema = tupleSchema;
+    this.schema = schema;
   }
 
   @JsonProperty
@@ -109,7 +109,7 @@ public abstract class AbstractParquetRowGroupScan extends AbstractBase implement
   }
 
   @JsonProperty
-  public TupleSchema getTupleSchema() { return tupleSchema; }
+  public TupleMetadata getSchema() { return schema; }
 
   public abstract AbstractParquetRowGroupScan copy(List<SchemaPath> columns);
   @JsonIgnore
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 586e094..923efac 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
@@ -24,7 +24,7 @@ import org.apache.drill.exec.expr.FilterPredicate;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.expr.stat.RowsMatch;
 import org.apache.drill.exec.physical.base.AbstractGroupScanWithMetadata;
-import org.apache.drill.exec.record.metadata.TupleSchema;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.store.CommonParquetRecordReader;
 import org.apache.drill.exec.store.parquet.metadata.Metadata;
 import org.apache.drill.exec.store.parquet.metadata.MetadataBase;
@@ -86,8 +86,8 @@ public abstract class AbstractParquetScanBatchCreator {
     ParquetReaderConfig readerConfig = rowGroupScan.getReaderConfig();
     RowGroupReadEntry firstRowGroup = null; // to be scanned in case ALL row groups are pruned out
     ParquetMetadata firstFooter = null;
-    long rowgroupsPruned = 0; // for stats
-    TupleSchema tupleSchema = rowGroupScan.getTupleSchema();
+    long rowGroupsPruned = 0; // for stats
+    TupleMetadata schema = rowGroupScan.getSchema();
 
     try {
 
@@ -105,7 +105,7 @@ public abstract class AbstractParquetScanBatchCreator {
       Set<String> columnsInExpr = null;
       // for debug/info logging
       long totalPruneTime = 0;
-      long totalRowgroups = rowGroupScan.getRowGroupReadEntries().size();
+      long totalRowGroups = rowGroupScan.getRowGroupReadEntries().size();
       Stopwatch pruneTimer = Stopwatch.createUnstarted();
       int countMatchClassCastExceptions = 0; // in case match() hits CCE, count and report these
       String matchCastErrorMessage = ""; // report the error too (Java insists on initializing this ....)
@@ -115,7 +115,7 @@ public abstract class AbstractParquetScanBatchCreator {
         filterPredicate = AbstractGroupScanWithMetadata.getFilterPredicate(filterExpr, context,
           (FunctionImplementationRegistry) context.getFunctionRegistry(), context.getOptions(), true,
           true /* supports file implicit columns */,
-          tupleSchema);
+          schema);
         // Extract only the relevant columns from the filter (sans implicit columns, if any)
         schemaPathsInExpr = filterExpr.accept(new FilterEvaluatorUtils.FieldReferenceFinder(), null);
         columnsInExpr = new HashSet<>();
@@ -207,7 +207,7 @@ public abstract class AbstractParquetScanBatchCreator {
 
           // If this rowgroup failed the match - skip it (i.e., no reader for this rowgroup)
           if (matchResult == RowsMatch.NONE) {
-            rowgroupsPruned++; // one more RG was pruned
+            rowGroupsPruned++; // one more RG was pruned
             if (firstRowGroup == null) {  // keep the first RG, to be used in case all row groups are pruned
               firstRowGroup = rowGroup;
               firstFooter = footer;
@@ -227,16 +227,16 @@ public abstract class AbstractParquetScanBatchCreator {
       }
       // do some logging, if relevant
       if ( totalPruneTime > 0 ) {
-        logger.info("Finished parquet_runtime_pruning in {} usec. Out of given {} rowgroups, {} were pruned. {}", totalPruneTime, totalRowgroups, rowgroupsPruned,
-          totalRowgroups == rowgroupsPruned ? "ALL_PRUNED !!" : "");
+        logger.info("Finished parquet_runtime_pruning in {} usec. Out of given {} rowgroups, {} were pruned. {}", totalPruneTime, totalRowGroups, rowGroupsPruned,
+          totalRowGroups == rowGroupsPruned ? "ALL_PRUNED !!" : "");
       }
       if ( countMatchClassCastExceptions > 0 ) {
-        logger.info("Run-time pruning skipped for {} out of {} rowgroups due to: {}",countMatchClassCastExceptions, totalRowgroups, matchCastErrorMessage);
+        logger.info("Run-time pruning skipped for {} out of {} rowgroups due to: {}",countMatchClassCastExceptions, totalRowGroups, matchCastErrorMessage);
       }
 
       // Update stats (same in every reader - the others would just overwrite the stats)
       for (CommonParquetRecordReader rr : readers ) {
-          rr.updateRowgroupsStats(totalRowgroups, rowgroupsPruned);
+          rr.updateRowgroupsStats(totalRowGroups, rowGroupsPruned);
       }
 
     } catch (IOException|InterruptedException e) {
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 41ed1f5..54d3af3 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
@@ -22,10 +22,10 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.metastore.FileSystemMetadataProviderManager;
 import org.apache.drill.metastore.MetadataProviderManager;
 import org.apache.drill.metastore.ParquetTableMetadataProvider;
-import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.metastore.metadata.LocationProvider;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -73,7 +73,7 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
                           @JsonProperty("cacheFileRoot") Path cacheFileRoot,
                           @JsonProperty("readerConfig") ParquetReaderConfig readerConfig,
                           @JsonProperty("filter") LogicalExpression filter,
-                          @JsonProperty("schema") TupleSchema schema) throws IOException, ExecutionSetupException {
+                          @JsonProperty("schema") TupleMetadata schema) throws IOException, ExecutionSetupException {
     super(ImpersonationUtil.resolveUserName(userName), columns, entries, readerConfig, filter);
     Preconditions.checkNotNull(storageConfig);
     Preconditions.checkNotNull(formatConfig);
@@ -199,7 +199,7 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
   @JsonProperty
   @JsonIgnore(value = false)
   @Override
-  public TupleSchema getSchema() {
+  public TupleMetadata getSchema() {
     return super.getSchema();
   }
   // getters for serialization / deserialization end
@@ -207,7 +207,7 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
   @Override
   public ParquetRowGroupScan getSpecificScan(int minorFragmentId) {
     return new ParquetRowGroupScan(getUserName(), formatPlugin, getReadEntries(minorFragmentId), columns, readerConfig, selectionRoot, filter,
-      tableMetadata == null ? null : (TupleSchema) tableMetadata.getSchema());
+      tableMetadata == null ? null : tableMetadata.getSchema());
   }
 
   @Override
@@ -302,7 +302,7 @@ public class ParquetGroupScan extends AbstractParquetGroupScan {
    */
   private class ParquetGroupScanFilterer extends RowGroupScanFilterer<ParquetGroupScanFilterer> {
 
-    public ParquetGroupScanFilterer(ParquetGroupScan source) {
+    ParquetGroupScanFilterer(ParquetGroupScan source) {
       super(source);
     }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
index fcbf307..0bdc266 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
@@ -27,7 +27,7 @@ import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
-import org.apache.drill.exec.record.metadata.TupleSchema;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.store.ColumnExplorer;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 
@@ -43,7 +43,6 @@ import org.apache.hadoop.fs.Path;
 // Class containing information for reading a single parquet row group from HDFS
 @JsonTypeName("parquet-row-group-scan")
 public class ParquetRowGroupScan extends AbstractParquetRowGroupScan {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetRowGroupScan.class);
 
   private final ParquetFormatPlugin formatPlugin;
   private final ParquetFormatConfig formatConfig;
@@ -58,7 +57,7 @@ public class ParquetRowGroupScan extends AbstractParquetRowGroupScan {
                              @JsonProperty("readerConfig") ParquetReaderConfig readerConfig,
                              @JsonProperty("selectionRoot") Path selectionRoot,
                              @JsonProperty("filter") LogicalExpression filter,
-                             @JsonProperty("tupleSchema") TupleSchema tupleSchema) throws ExecutionSetupException {
+                             @JsonProperty("schema") TupleMetadata schema) throws ExecutionSetupException {
     this(userName,
         (ParquetFormatPlugin) registry.getFormatPlugin(Preconditions.checkNotNull(storageConfig), Preconditions.checkNotNull(formatConfig)),
         rowGroupReadEntries,
@@ -66,7 +65,7 @@ public class ParquetRowGroupScan extends AbstractParquetRowGroupScan {
         readerConfig,
         selectionRoot,
         filter,
-        tupleSchema);
+        schema);
   }
 
   public ParquetRowGroupScan(String userName,
@@ -76,8 +75,8 @@ public class ParquetRowGroupScan extends AbstractParquetRowGroupScan {
                              ParquetReaderConfig readerConfig,
                              Path selectionRoot,
                              LogicalExpression filter,
-                             TupleSchema tupleSchema) {
-    super(userName, rowGroupReadEntries, columns, readerConfig, filter, selectionRoot, tupleSchema);
+                             TupleMetadata schema) {
+    super(userName, rowGroupReadEntries, columns, readerConfig, filter, selectionRoot, schema);
     this.formatPlugin = Preconditions.checkNotNull(formatPlugin, "Could not find format config for the given configuration");
     this.formatConfig = formatPlugin.getConfig();
   }
@@ -100,7 +99,7 @@ public class ParquetRowGroupScan extends AbstractParquetRowGroupScan {
   @Override
   public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
     Preconditions.checkArgument(children.isEmpty());
-    return new ParquetRowGroupScan(getUserName(), formatPlugin, rowGroupReadEntries, columns, readerConfig, selectionRoot, filter, tupleSchema);
+    return new ParquetRowGroupScan(getUserName(), formatPlugin, rowGroupReadEntries, columns, readerConfig, selectionRoot, filter, schema);
   }
 
   @Override
@@ -110,7 +109,7 @@ public class ParquetRowGroupScan extends AbstractParquetRowGroupScan {
 
   @Override
   public AbstractParquetRowGroupScan copy(List<SchemaPath> columns) {
-    return new ParquetRowGroupScan(getUserName(), formatPlugin, rowGroupReadEntries, columns, readerConfig, selectionRoot, filter, tupleSchema);
+    return new ParquetRowGroupScan(getUserName(), formatPlugin, rowGroupReadEntries, columns, readerConfig, selectionRoot, filter, schema);
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
index 7915843..a739c5c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
@@ -116,13 +116,7 @@ public class WorkManager implements AutoCloseable {
     dContext = new DrillbitContext(endpoint, bContext, coord, controller, data, workBus, provider, profilesProvider);
     statusThread.start();
 
-    DrillMetrics.register("drill.fragments.running",
-        new Gauge<Integer>() {
-          @Override
-          public Integer getValue() {
-            return runningFragments.size();
-          }
-        });
+    DrillMetrics.register("drill.fragments.running", (Gauge<Integer>) runningFragments::size);
   }
 
   public Executor getExecutor() {
@@ -239,7 +233,7 @@ public class WorkManager implements AutoCloseable {
    *  shutdown request is triggered.
    */
   public synchronized Map<String, Integer> getRemainingQueries() {
-        Map<String, Integer> queriesInfo = new HashMap<String, Integer>();
+        Map<String, Integer> queriesInfo = new HashMap<>();
         queriesInfo.put("queriesCount", queries.size());
         queriesInfo.put("fragmentsCount", runningFragments.size());
         return queriesInfo;
@@ -258,7 +252,8 @@ public class WorkManager implements AutoCloseable {
 
     /**
      * Add a self contained runnable work to executor service.
-     * @param runnable
+     *
+     * @param runnable runnable to execute
      */
     public void addNewWork(final Runnable runnable) {
       executor.execute(runnable);
@@ -280,22 +275,17 @@ public class WorkManager implements AutoCloseable {
             .build(logger);
       }
 
-      executor.execute(new Runnable()
-      {
-        @Override
-        public void run()
-        {
-          final Thread currentThread = Thread.currentThread();
-          final String originalName = currentThread.getName();
-          try {
-            currentThread.setName(queryIdString + ":foreman:cancel");
-            logger.debug("Canceling foreman");
-            foreman.cancel();
-          } catch (Throwable t) {
-            logger.warn("Exception while canceling foreman", t);
-          } finally {
-            currentThread.setName(originalName);
-          }
+      executor.execute(() -> {
+        final Thread currentThread = Thread.currentThread();
+        final String originalName = currentThread.getName();
+        try {
+          currentThread.setName(queryIdString + ":foreman:cancel");
+          logger.debug("Canceling foreman");
+          foreman.cancel();
+        } catch (Throwable t) {
+          logger.warn("Exception while canceling foreman", t);
+        } finally {
+          currentThread.setName(originalName);
         }
       });
       return true;
@@ -381,7 +371,7 @@ public class WorkManager implements AutoCloseable {
 
     /**
      * receive the RuntimeFilter thorough the wire
-     * @param runtimeFilter
+     * @param runtimeFilter runtime filter
      */
     public void receiveRuntimeFilter(final RuntimeFilterWritable runtimeFilter) {
       BitData.RuntimeFilterBDef runtimeFilterDef = runtimeFilter.getRuntimeFilterBDef();
@@ -393,20 +383,17 @@ public class WorkManager implements AutoCloseable {
       if (toForeman) {
         Foreman foreman = queries.get(queryId);
         if (foreman != null) {
-          executor.execute(new Runnable() {
-            @Override
-            public void run() {
-              final Thread currentThread = Thread.currentThread();
-              final String originalName = currentThread.getName();
-              currentThread.setName(queryIdStr + ":foreman:routeRuntimeFilter");
-              try {
-                foreman.getRuntimeFilterRouter().register(runtimeFilter);
-              } catch (Exception e) {
-                logger.warn("Exception while registering the RuntimeFilter", e);
-              } finally {
-                currentThread.setName(originalName);
-                runtimeFilter.close();
-              }
+          executor.execute(() -> {
+            final Thread currentThread = Thread.currentThread();
+            final String originalName = currentThread.getName();
+            currentThread.setName(queryIdStr + ":foreman:routeRuntimeFilter");
+            try {
+              foreman.getRuntimeFilterRouter().register(runtimeFilter);
+            } catch (Exception e) {
+              logger.warn("Exception while registering the RuntimeFilter", e);
+            } finally {
+              currentThread.setName(originalName);
+              runtimeFilter.close();
             }
           });
         }
@@ -426,7 +413,7 @@ public class WorkManager implements AutoCloseable {
   }
 
   /**
-   * Periodically gather current statistics. {@link QueryManager} uses a FragmentStatusListener to
+   * Periodically gather current statistics. {@link org.apache.drill.exec.work.foreman.QueryManager} uses a FragmentStatusListener to
    * maintain changes to state, and should be current. However, we want to collect current statistics
    * about RUNNING queries, such as current memory consumption, number of rows processed, and so on.
    * The FragmentStatusListener only tracks changes to state, so the statistics kept there will be
@@ -436,7 +423,7 @@ public class WorkManager implements AutoCloseable {
    * Tunnel, whereas for remote Foreman it is sent over the Control Tunnel.
    */
   private class StatusThread extends Thread {
-    public StatusThread() {
+    StatusThread() {
       // assume this thread is created by a non-daemon thread
       setName("WorkManager.StatusThread");
     }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
index 6725bb9..1ccd9b4 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
@@ -32,7 +32,7 @@ import org.apache.drill.exec.planner.PhysicalPlanReaderTestFactory;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.options.SystemOptionManager;
 import org.apache.drill.exec.store.sys.store.provider.LocalPersistentStoreProvider;
-import org.apache.drill.exec.util.GuavaPatcher;
+import org.apache.drill.common.util.GuavaPatcher;
 import org.apache.drill.test.BaseDirTestWatcher;
 import org.apache.drill.test.DrillTest;
 import org.apache.hadoop.conf.Configuration;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinOutcome.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinOutcome.java
index 57dbfd8..6d914db 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinOutcome.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinOutcome.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.join;
 
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.sql.SqlKind;
@@ -33,7 +34,6 @@ import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
-import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.exec.store.mock.MockStorePOP;
 import org.apache.drill.test.rowSet.RowSet;
 import org.junit.After;
@@ -45,7 +45,7 @@ import org.junit.experimental.categories.Category;
 import java.util.ArrayList;
 import java.util.List;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertSame;
 
 /**
  *  Unit tests of the Hash Join getting various outcomes as input
@@ -56,8 +56,8 @@ public class TestHashJoinOutcome extends PhysicalOpUnitTestBase {
   // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestHashJoinOutcome.class);
 
   // input batch schemas
-  private static TupleSchema inputSchemaRight;
-  private static TupleSchema inputSchemaLeft;
+  private static TupleMetadata inputSchemaRight;
+  private static TupleMetadata inputSchemaLeft;
   private static BatchSchema batchSchemaRight;
   private static BatchSchema batchSchemaLeft;
 
@@ -82,11 +82,11 @@ public class TestHashJoinOutcome extends PhysicalOpUnitTestBase {
 
   @BeforeClass
   public static void setUpBeforeClass() {
-    inputSchemaRight = (TupleSchema) new SchemaBuilder()
+    inputSchemaRight = new SchemaBuilder()
       .add("rightcol", TypeProtos.MinorType.INT)
       .buildSchema();
     batchSchemaRight = new BatchSchema(BatchSchema.SelectionVectorMode.NONE, inputSchemaRight.toFieldList());
-    inputSchemaLeft = (TupleSchema) new SchemaBuilder()
+    inputSchemaLeft = new SchemaBuilder()
       .add("leftcol", TypeProtos.MinorType.INT)
       .buildSchema();
     batchSchemaLeft = new BatchSchema(BatchSchema.SelectionVectorMode.NONE, inputSchemaLeft.toFieldList());
@@ -184,10 +184,10 @@ public class TestHashJoinOutcome extends PhysicalOpUnitTestBase {
     HashJoinBatch hjBatch = new HashJoinBatch(hjConf,operatorFixture.getFragmentContext(), mockInputBatchLeft, mockInputBatchRight );
 
     RecordBatch.IterOutcome gotOutcome = hjBatch.next();
-    assertTrue(gotOutcome == RecordBatch.IterOutcome.OK_NEW_SCHEMA );
+    assertSame(gotOutcome, RecordBatch.IterOutcome.OK_NEW_SCHEMA);
 
     gotOutcome = hjBatch.next();
-    assertTrue(gotOutcome == expectedOutcome); // verify returned outcome
+    assertSame(gotOutcome, expectedOutcome); // verify returned outcome
   }
 
   @Test
@@ -243,10 +243,10 @@ public class TestHashJoinOutcome extends PhysicalOpUnitTestBase {
     HashJoinBatch hjBatch = new HashJoinBatch(hjConf, operatorFixture.getFragmentContext(), mockInputBatchLeft, mockInputBatchRight);
 
     RecordBatch.IterOutcome gotOutcome = hjBatch.next();
-    assertTrue(gotOutcome == RecordBatch.IterOutcome.OK_NEW_SCHEMA);
+    assertSame(gotOutcome, RecordBatch.IterOutcome.OK_NEW_SCHEMA);
 
     gotOutcome = hjBatch.next();
-    assertTrue(gotOutcome == RecordBatch.IterOutcome.NONE);
+    assertSame(gotOutcome, RecordBatch.IterOutcome.NONE);
 
     secondInputRowSetRight.clear();
     thirdInputRowSetRight.clear();
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/metadata/TestTupleSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/metadata/TestTupleSchema.java
index 24adf67..47dffac 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/metadata/TestTupleSchema.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/metadata/TestTupleSchema.java
@@ -822,4 +822,51 @@ public class TestTupleSchema extends SubOperatorTest {
       // Expected
     }
   }
+
+  @Test
+  public void testJsonString() {
+    TupleMetadata schema = new SchemaBuilder()
+      .add("col_int", MinorType.INT)
+      .buildSchema();
+    schema.setProperty("prop", "val");
+
+    String expected = "{\"type\":\"tuple_schema\","
+      + "\"columns\":[{\"name\":\"col_int\",\"type\":\"INT\",\"mode\":\"REQUIRED\"}],"
+      + "\"properties\":{\"prop\":\"val\"}}";
+
+    assertEquals(expected, schema.jsonString());
+  }
+
+  @Test
+  public void testFromJsonTyped() {
+    String jsonString = "{\"type\":\"tuple_schema\","
+      + "\"columns\":[{\"name\":\"col_int\",\"type\":\"INT\",\"mode\":\"REQUIRED\"}],"
+      + "\"properties\":{\"prop\":\"val\"}}";
+
+    TupleMetadata result = TupleMetadata.of(jsonString);
+    assertTrue(result instanceof TupleSchema);
+    assertEquals(1, result.size());
+    ColumnMetadata column = result.metadata("col_int");
+    assertEquals(MinorType.INT, column.type());
+    assertEquals("val", result.property("prop"));
+  }
+
+  @Test
+  public void testFromJsonUntyped() {
+    String untyped = "{\"columns\":[{\"name\":\"col_int\",\"type\":\"INT\",\"mode\":\"REQUIRED\"}],"
+      + "\"properties\":{\"prop\":\"val\"}}";
+    TupleMetadata result = TupleMetadata.of(untyped);
+    assertTrue(result instanceof TupleSchema);
+    assertEquals(1, result.size());
+    ColumnMetadata column = result.metadata("col_int");
+    assertEquals(MinorType.INT, column.type());
+    assertEquals("val", result.property("prop"));
+  }
+
+  @Test
+  public void testNullOrEmptyJsonString() {
+    assertNull(TupleMetadata.of(null));
+    assertNull(TupleMetadata.of(""));
+    assertNull(TupleMetadata.of("   "));
+  }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/metadata/schema/TestSchemaProvider.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/metadata/schema/TestSchemaProvider.java
index ec5d48f..03dcc3c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/metadata/schema/TestSchemaProvider.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/metadata/schema/TestSchemaProvider.java
@@ -147,6 +147,7 @@ public class TestSchemaProvider {
 
     String expectedContent = "{\n"
       + "  \"schema\" : {\n"
+      + "    \"type\" : \"tuple_schema\",\n"
       + "    \"columns\" : [\n"
       + "      {\n"
       + "        \"name\" : \"i\",\n"
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcapng/TestPcapngHeaders.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcapng/TestPcapngHeaders.java
index 5dcffa9..60bbeaf 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcapng/TestPcapngHeaders.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/pcapng/TestPcapngHeaders.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.store.pcapng;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.record.metadata.TupleSchema;
 import org.apache.drill.test.ClusterFixture;
 import org.apache.drill.test.ClusterTest;
@@ -44,7 +45,7 @@ public class TestPcapngHeaders extends ClusterTest {
     String query = "select * from dfs.`store/pcapng/sniff.pcapng`";
     RowSet actual = client.queryBuilder().sql(query).rowSet();
 
-    TupleSchema expectedSchema = new TupleSchema();
+    TupleMetadata expectedSchema = new TupleSchema();
 
     expectedSchema.add(MaterializedField.create("tcp_flags_ece_ecn_capable", Types.optional(TypeProtos.MinorType.INT)));
     expectedSchema.add(MaterializedField.create("tcp_flags_ece_congestion_experienced", Types.optional(TypeProtos.MinorType.INT)));
@@ -83,7 +84,7 @@ public class TestPcapngHeaders extends ClusterTest {
     String query = "select sRc_ip, dst_IP, dst_mAc_address, src_Port, tcp_session, `Timestamp`  from dfs.`store/pcapng/sniff.pcapng`";
     RowSet actual = client.queryBuilder().sql(query).rowSet();
 
-    TupleSchema expectedSchema = new TupleSchema();
+    TupleMetadata expectedSchema = new TupleSchema();
 
     expectedSchema.add(MaterializedField.create("sRc_ip", Types.optional(TypeProtos.MinorType.VARCHAR)));
     expectedSchema.add(MaterializedField.create("dst_IP", Types.optional(TypeProtos.MinorType.VARCHAR)));
@@ -103,7 +104,7 @@ public class TestPcapngHeaders extends ClusterTest {
     String query = "select `timestamp`, `name`, `color` from dfs.`store/pcapng/sniff.pcapng`";
     RowSet actual = client.queryBuilder().sql(query).rowSet();
 
-    TupleSchema expectedSchema = new TupleSchema();
+    TupleMetadata expectedSchema = new TupleSchema();
 
     expectedSchema.add(MaterializedField.create("timestamp", Types.required(TypeProtos.MinorType.TIMESTAMP)));
     expectedSchema.add(MaterializedField.create("name", Types.optional(TypeProtos.MinorType.INT)));
@@ -120,7 +121,7 @@ public class TestPcapngHeaders extends ClusterTest {
     String query = "select src_ip, dst_ip, dst_mac_address, src_port, tcp_session, `timestamp`  from dfs.`store/pcapng/sniff.pcapng`";
     RowSet actual = client.queryBuilder().sql(query).rowSet();
 
-    TupleSchema expectedSchema = new TupleSchema();
+    TupleMetadata expectedSchema = new TupleSchema();
 
     expectedSchema.add(MaterializedField.create("sRc_ip", Types.optional(TypeProtos.MinorType.VARCHAR)));
     expectedSchema.add(MaterializedField.create("dst_IP", Types.optional(TypeProtos.MinorType.VARCHAR)));
@@ -158,7 +159,7 @@ public class TestPcapngHeaders extends ClusterTest {
     String query = "select arr[3] as arr from dfs.`store/pcapng/sniff.pcapng`";
     RowSet actual = client.queryBuilder().sql(query).rowSet();
 
-    TupleSchema expectedSchema = new TupleSchema();
+    TupleMetadata expectedSchema = new TupleSchema();
 
     expectedSchema.add(MaterializedField.create("arr", Types.optional(TypeProtos.MinorType.INT)));
 
@@ -187,7 +188,7 @@ public class TestPcapngHeaders extends ClusterTest {
     String query = "select top['nested'] as nested from dfs.`store/pcapng/sniff.pcapng`";
     RowSet actual = client.queryBuilder().sql(query).rowSet();
 
-    TupleSchema expectedSchema = new TupleSchema();
+    TupleMetadata expectedSchema = new TupleSchema();
 
     expectedSchema.add(MaterializedField.create("nested", Types.optional(TypeProtos.MinorType.INT)));
 
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index a48dabb..2dab193 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -40,7 +40,7 @@
     <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
-      <version>${dep.slf4j.version}</version>
+      <version>${slf4j.version}</version>
     </dependency>
 
     <dependency>
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleMetadata.java b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleMetadata.java
index 40a3e8a..1ff6160 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleMetadata.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleMetadata.java
@@ -17,8 +17,15 @@
  */
 package org.apache.drill.exec.record.metadata;
 
+import java.io.IOException;
 import java.util.List;
 
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.ObjectWriter;
 import org.apache.drill.exec.record.MaterializedField;
 
 /**
@@ -43,9 +50,16 @@ import org.apache.drill.exec.record.MaterializedField;
  * In the future, this structure will also gather metadata useful
  * for vector processing such as expected widths and so on.
  */
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = TupleSchema.class)
+@JsonSubTypes({
+  @JsonSubTypes.Type(value = TupleSchema.class, name = TupleSchema.TYPE)
+})
 public interface TupleMetadata extends Propertied, Iterable<ColumnMetadata> {
 
-  public static final String IS_STRICT_SCHEMA_PROP = DRILL_PROP_PREFIX + "strict";
+  ObjectWriter WRITER = new ObjectMapper().writerFor(TupleMetadata.class);
+  ObjectReader READER = new ObjectMapper().readerFor(TupleMetadata.class);
+
+  String IS_STRICT_SCHEMA_PROP = DRILL_PROP_PREFIX + "strict";
 
   /**
    * Add a new column to the schema.
@@ -95,4 +109,39 @@ public interface TupleMetadata extends Propertied, Iterable<ColumnMetadata> {
 
   String fullName(ColumnMetadata column);
   String fullName(int index);
+  TupleMetadata copy();
+
+  /**
+   * Converts current {@link TupleMetadata} implementation into JSON string representation.
+   *
+   * @return tuple metadata in JSON string representation
+   * @throws IllegalStateException if unable to convert current instance into JSON string
+   */
+  default String jsonString() {
+    try {
+      return WRITER.writeValueAsString(this);
+    } catch (JsonProcessingException e) {
+      throw new IllegalStateException("Unable to convert tuple metadata into JSON string: " + toString(), e);
+    }
+  }
+
+  /**
+   * Converts given JSON string into {@link TupleMetadata} instance.
+   * {@link TupleMetadata} implementation is determined by present type property. For example: "type":"tuple_schema".
+   * If given JSON string is untyped, used default implementation: {@link TupleSchema}.
+   *
+   * @param jsonString tuple metadata in JSON string representation
+   * @return {@link TupleMetadata} instance, null if given JSON string is null or empty
+   * @throws IllegalArgumentException if unable to deserialize given JSON string
+   */
+  static TupleMetadata of(String jsonString) {
+    if (jsonString == null || jsonString.trim().isEmpty()) {
+      return null;
+    }
+    try {
+      return READER.readValue(jsonString);
+    } catch (IOException e) {
+      throw new IllegalArgumentException("Unable to deserialize given JSON string into tuple metadata: " + jsonString, e);
+    }
+  }
 }
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleSchema.java b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleSchema.java
index 080e8fc..b7c1208 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleSchema.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleSchema.java
@@ -46,6 +46,8 @@ import java.util.stream.Collectors;
 @JsonPropertyOrder({"columns", "properties"})
 public class TupleSchema extends AbstractPropertied implements TupleMetadata {
 
+  public static final String TYPE = "tuple_schema";
+
   private MapColumnMetadata parentMap;
   private final TupleNameSpace<ColumnMetadata> nameSpace = new TupleNameSpace<>();
 
diff --git a/logical/pom.xml b/logical/pom.xml
index cd212aa..b6d995b 100644
--- a/logical/pom.xml
+++ b/logical/pom.xml
@@ -55,7 +55,7 @@
       <!-- add as provided scope so that we can compile TestTools.  Should only be ever used in a test scenario where someone else is bringing JUnit in. -->
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
-      <version>${dep.junit.version}</version>
+      <version>${junit.version}</version>
     </dependency>
 
     <dependency>
diff --git a/pom.xml b/pom.xml
index ed39611..d4b3095 100644
--- a/pom.xml
+++ b/pom.xml
@@ -44,8 +44,8 @@
 
     <target.gen.source.path>${project.basedir}/target/generated-sources</target.gen.source.path>
     <proto.cas.path>${project.basedir}/src/main/protobuf/</proto.cas.path>
-    <dep.junit.version>4.12</dep.junit.version>
-    <dep.slf4j.version>1.7.25</dep.slf4j.version>
+    <junit.version>4.12</junit.version>
+    <slf4j.version>1.7.26</slf4j.version>
     <shaded.guava.version>23.0</shaded.guava.version>
     <guava.version>19.0</guava.version>
     <forkCount>2</forkCount>
@@ -61,8 +61,8 @@
     <avatica.version>1.15.0</avatica.version>
     <janino.version>3.0.11</janino.version>
     <sqlline.version>1.8.0</sqlline.version>
-    <jackson.version>2.9.5</jackson.version>
-    <jackson.databind.version>2.9.5</jackson.databind.version>
+    <jackson.version>2.9.9</jackson.version>
+    <jackson.databind.version>2.9.9</jackson.databind.version>
     <zookeeper.version>3.4.12</zookeeper.version>
     <mapr.release.version>6.1.0-mapr</mapr.release.version>
     <ojai.version>3.0-mapr-1808</ojai.version>
@@ -70,14 +70,13 @@
     <findbugs.version>3.0.0</findbugs.version>
     <netty.tcnative.classifier />
     <commons.io.version>2.4</commons.io.version>
-    <junit.version>4.11</junit.version>
     <hamcrest.core.version>1.3</hamcrest.core.version>
     <maven.embedder.version>3.5.3</maven.embedder.version>
     <curator.version>2.7.1</curator.version>
-    <wiremock.standalone.version>2.5.1</wiremock.standalone.version>
-    <jmockit.version>1.43</jmockit.version>
-    <logback.version>1.0.13</logback.version>
-    <mockito.version>2.18.3</mockito.version>
+    <wiremock.standalone.version>2.23.2</wiremock.standalone.version>
+    <jmockit.version>1.47</jmockit.version>
+    <logback.version>1.2.3</logback.version>
+    <mockito.version>2.28.2</mockito.version>
     <!--
       Currently Hive storage plugin only supports Apache Hive 2.3.2 or vendor specific variants of the
       Apache Hive 2.3.2. If the version is changed, make sure the jars and their dependencies are updated.
@@ -101,12 +100,13 @@
     <rat.skip>true</rat.skip>
     <license.skip>true</license.skip>
     <docker.repository>drill/apache-drill</docker.repository>
-    <antlr.version>4.7.1</antlr.version>
+    <antlr.version>4.7.2</antlr.version>
     <lowestMavenVersion>3.3.3</lowestMavenVersion>
     <commons.net.version>3.6</commons.net.version>
     <commons.validator.version>1.6</commons.validator.version>
     <commons.text.version>1.6</commons.text.version>
     <protobuf.version>3.6.1</protobuf.version>
+    <codemodel.version>2.6</codemodel.version>
   </properties>
 
   <scm>
@@ -945,25 +945,25 @@
     <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
-      <version>${dep.slf4j.version}</version>
+      <version>${slf4j.version}</version>
     </dependency>
 
     <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>jul-to-slf4j</artifactId>
-      <version>${dep.slf4j.version}</version>
+      <version>${slf4j.version}</version>
     </dependency>
 
     <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>jcl-over-slf4j</artifactId>
-      <version>${dep.slf4j.version}</version>
+      <version>${slf4j.version}</version>
     </dependency>
 
     <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>log4j-over-slf4j</artifactId>
-      <version>${dep.slf4j.version}</version>
+      <version>${slf4j.version}</version>
     </dependency>
 
     <dependency>
@@ -983,7 +983,7 @@
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
-      <version>${dep.junit.version}</version>
+      <version>${junit.version}</version>
       <scope>test</scope>
     </dependency>
     <dependency>