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 2018/08/27 08:25:31 UTC

[drill] branch master updated (ddb35ce -> d8f9fb6)

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

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


    from ddb35ce  DRILL-6688 Data batches for Project operator exceed the maximum specified (#1442)
     new 8bcb103  DRILL-6492: Ensure schema / workspace case insensitivity in Drill
     new 9896182  DRILL-6647: Update Calcite version to 1.17.0
     new 260a164  DRILL-6693: When a query is started from Drill Web Console, the UI becomes inaccessible until the query finishes
     new a1f3f9a  DRILL-6703: Query with complex expressions in lateral and unnest fails with CannotPlanException
     new e9ffb5b  DRILL-6644: Don't reserve space for incoming probe batches unnecessarily during the build phase.
     new d8f9fb6  DRILL-6461: Added basic data correctness tests for hash agg, and improved operator unit testing framework.

The 6 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:
 .../drill/common/map/CaseInsensitiveMap.java       |   25 +-
 .../drill/common/scanner/ClassPathScanner.java     |   22 +-
 .../drill/common/map/TestCaseInsensitiveMap.java   |   15 +
 .../drill/exec/store/hbase/HBaseSchemaFactory.java |   26 +-
 .../exec/store/hive/schema/HiveDatabaseSchema.java |   11 +-
 .../exec/store/hive/schema/HiveSchemaFactory.java  |   29 +-
 .../apache/drill/exec/hive/TestHiveStorage.java    |    5 +
 .../exec/hive/TestInfoSchemaOnHiveStorage.java     |    2 +-
 .../hive/BaseTestHiveImpersonation.java            |   11 -
 .../hive/TestSqlStdBasedAuthorization.java         |   19 +-
 .../hive/TestStorageBasedHiveAuthorization.java    |   30 +-
 .../drill/exec/store/jdbc/JdbcStoragePlugin.java   |    2 +-
 .../store/kafka/schema/KafkaSchemaFactory.java     |   14 +-
 .../drill/exec/store/kudu/KuduSchemaFactory.java   |   22 +-
 .../store/mongo/schema/MongoSchemaFactory.java     |   16 +-
 contrib/storage-opentsdb/README.md                 |   16 +-
 .../exec/store/openTSDB/OpenTSDBStoragePlugin.java |    5 +-
 .../openTSDB/schema/OpenTSDBSchemaFactory.java     |   19 +-
 exec/java-exec/src/main/codegen/data/Parser.tdd    |    8 +-
 .../java/org/apache/drill/exec/ExecConstants.java  |    1 +
 .../drill/exec/coord/zk/ZookeeperClient.java       |    2 +-
 .../expr/fn/registry/LocalFunctionRegistry.java    |   11 +-
 .../drill/exec/physical/impl/TopN/TopNBatch.java   |   12 -
 .../physical/impl/common/HashTableTemplate.java    |   12 +-
 .../exec/physical/impl/join/HashJoinBatch.java     |   23 +-
 .../join/HashJoinMechanicalMemoryCalculator.java   |    1 -
 .../impl/join/HashJoinMemoryCalculator.java        |    3 +-
 .../impl/join/HashJoinMemoryCalculatorImpl.java    |  106 +-
 .../physical/impl/project/ProjectRecordBatch.java  |    2 +
 .../physical/impl/svremover/AbstractCopier.java    |   26 +-
 .../physical/impl/svremover/AbstractSV2Copier.java |    4 +-
 .../physical/impl/svremover/AbstractSV4Copier.java |    4 +-
 .../drill/exec/physical/impl/svremover/Copier.java |    4 +-
 .../physical/impl/svremover/GenericCopier.java     |    7 +-
 .../physical/impl/svremover/StraightCopier.java    |    3 +-
 .../apache/drill/exec/planner/PlannerPhase.java    |    3 -
 .../logical/DirPrunedEnumerableTableScan.java      |   12 +-
 .../exec/planner/logical/DrillConditions.java      |   20 +-
 ...rojectComplexRexNodeCorrelateTransposeRule.java |  154 --
 .../drill/exec/planner/logical/StoragePlugins.java |    7 +-
 .../exec/planner/sql/DrillConvertletTable.java     |   35 +-
 .../drill/exec/planner/sql/SchemaUtilites.java     |    3 +-
 .../drill/exec/planner/sql/SqlConverter.java       |    2 +-
 .../planner/sql/handlers/ComplexUnnestVisitor.java |  199 +++
 .../planner/sql/handlers/DefaultSqlHandler.java    |   33 +-
 .../sql/handlers/DescribeSchemaHandler.java        |    6 +-
 .../planner/sql/handlers/DescribeTableHandler.java |   69 +-
 .../planner/sql/handlers/ShowSchemasHandler.java   |   27 +-
 .../planner/sql/handlers/ShowTablesHandler.java    |   74 +-
 .../apache/drill/exec/record/RecordBatchSizer.java |   17 +
 .../apache/drill/exec/server/rest/WebServer.java   |    5 +-
 .../apache/drill/exec/store/AbstractSchema.java    |   50 +-
 .../AbstractSchemaFactory.java}                    |   19 +-
 .../drill/exec/store/AbstractStoragePlugin.java    |   14 +-
 .../drill/exec/store/PartitionExplorerImpl.java    |    2 +-
 .../org/apache/drill/exec/store/SchemaFactory.java |    5 +-
 .../org/apache/drill/exec/store/StoragePlugin.java |    2 +
 .../apache/drill/exec/store/StoragePluginMap.java  |    6 +-
 .../drill/exec/store/StoragePluginRegistry.java    |    6 +-
 .../exec/store/StoragePluginRegistryImpl.java      |  468 +++---
 .../SystemPlugin.java}                             |   10 +-
 .../drill/exec/store/dfs/FileSystemConfig.java     |    6 +-
 .../drill/exec/store/dfs/FileSystemPlugin.java     |    2 +
 .../exec/store/dfs/FileSystemSchemaFactory.java    |   20 +-
 .../drill/exec/store/ischema/InfoSchemaConfig.java |    3 +-
 .../exec/store/ischema/InfoSchemaConstants.java    |    2 +-
 .../store/ischema/InfoSchemaStoragePlugin.java     |   62 +-
 .../store/sys/CaseInsensitivePersistentStore.java  |   79 ++
 .../drill/exec/store/sys/SystemTablePlugin.java    |   54 +-
 .../exec/store/sys/SystemTablePluginConfig.java    |    1 -
 .../drill/exec/work/metadata/MetadataProvider.java |   24 +-
 .../java-exec/src/main/resources/drill-module.conf |   28 +-
 .../drill/common/scanner/TestClassPathScanner.java |   43 +-
 .../drill/exec/coord/zk/TestZookeeperClient.java   |   10 +-
 .../exec/impersonation/BaseTestImpersonation.java  |   19 +-
 .../TestImpersonationDisabledWithMiniDFS.java      |   12 +-
 .../impersonation/TestImpersonationMetadata.java   |  112 +-
 .../impersonation/TestImpersonationQueries.java    |   17 +-
 .../physical/impl/BaseTestOpBatchEmitOutcome.java  |    2 +-
 .../drill/exec/physical/impl/MockRecordBatch.java  |  196 ++-
 .../physical/impl/agg/TestAggWithAnyValue.java     |    6 +-
 .../exec/physical/impl/agg/TestHashAggBatch.java   |  212 +++
 .../physical/impl/common/HashPartitionTest.java    |   70 +-
 .../impl/join/TestBuildSidePartitioningImpl.java   |  104 +-
 .../exec/physical/impl/join/TestHashJoinJPPD.java  |    6 +-
 .../exec/physical/impl/join/TestHashJoinSpill.java |    8 +-
 .../impl/join/TestPostBuildCalculationsImpl.java   |  127 +-
 .../impl/lateraljoin/TestE2EUnnestAndLateral.java  |   39 +
 .../physical/impl/limit/TestLimitOperator.java     |   16 +-
 .../impl/svremover/AbstractGenericCopierTest.java  |   90 +-
 .../physical/impl/svremover/GenericCopierTest.java |    4 +-
 .../impl/svremover/GenericSV2BatchCopierTest.java  |    4 +-
 .../impl/svremover/GenericSV2CopierTest.java       |    4 +-
 .../impl/svremover/GenericSV4CopierTest.java       |    9 +-
 .../physical/unit/BasicPhysicalOpUnitTest.java     |   30 +-
 .../exec/physical/unit/MiniPlanUnitTestBase.java   |    3 +-
 .../exec/physical/unit/TestNullInputMiniPlan.java  |    4 +-
 .../exec/physical/unit/TestOutputBatchSize.java    |   80 +-
 .../exec/planner/TestDirectoryExplorerUDFs.java    |    2 +-
 .../org/apache/drill/exec/sql/TestInfoSchema.java  |   64 +-
 .../exec/sql/TestSchemaCaseInsensitivity.java      |   92 ++
 .../columnreaders/TestBatchSizingMemoryUtil.java   |    2 +-
 .../exec/work/metadata/TestMetadataProvider.java   |   38 +-
 .../drill/test/LegacyOperatorTestBuilder.java      |  178 +++
 .../org/apache/drill/test/OperatorTestBuilder.java |  314 ++++
 .../apache/drill/test/OperatorTestBuilderTest.java |  157 ++
 .../unit => test}/PhysicalOpUnitTestBase.java      |  169 +--
 .../org/apache/drill/test/rowSet/RowSetBatch.java  |  111 --
 .../apache/drill/test/rowSet/RowSetComparison.java |  151 +-
 .../drill/test/rowSet/TestRowSetComparison.java    |  211 +++
 .../drill/jdbc/impl/DrillConnectionImpl.java       |  400 ++----
 .../drill/jdbc/impl/DrillDatabaseMetaDataImpl.java |  354 +++--
 .../apache/drill/jdbc/impl/DrillJdbc41Factory.java |  120 +-
 .../jdbc/impl/DrillPreparedStatementImpl.java      |  463 +-----
 .../apache/drill/jdbc/impl/DrillResultSetImpl.java | 1494 ++++++--------------
 .../jdbc/impl/DrillResultSetMetaDataImpl.java      |   10 +-
 .../apache/drill/jdbc/impl/DrillStatementImpl.java |  327 +----
 .../java/org/apache/drill/jdbc/JdbcTestBase.java   |  129 +-
 ...rill2489CallsAfterCloseThrowExceptionsTest.java |  107 +-
 .../main/codegen/templates/FixedValueVectors.java  |    2 +-
 .../codegen/templates/NullableValueVectors.java    |    2 +-
 pom.xml                                            |    4 +-
 122 files changed, 4087 insertions(+), 4088 deletions(-)
 delete mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ProjectComplexRexNodeCorrelateTransposeRule.java
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ComplexUnnestVisitor.java
 copy exec/java-exec/src/main/java/org/apache/drill/exec/{compile/sig/CodeGeneratorArgument.java => store/AbstractSchemaFactory.java} (72%)
 copy exec/java-exec/src/main/java/org/apache/drill/exec/{physical/impl/PhysicalConfig.java => store/SystemPlugin.java} (83%)
 create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/CaseInsensitivePersistentStore.java
 create mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestHashAggBatch.java
 create mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSchemaCaseInsensitivity.java
 create mode 100644 exec/java-exec/src/test/java/org/apache/drill/test/LegacyOperatorTestBuilder.java
 create mode 100644 exec/java-exec/src/test/java/org/apache/drill/test/OperatorTestBuilder.java
 create mode 100644 exec/java-exec/src/test/java/org/apache/drill/test/OperatorTestBuilderTest.java
 rename exec/java-exec/src/test/java/org/apache/drill/{exec/physical/unit => test}/PhysicalOpUnitTestBase.java (71%)
 delete mode 100644 exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBatch.java
 create mode 100644 exec/java-exec/src/test/java/org/apache/drill/test/rowSet/TestRowSetComparison.java


[drill] 01/06: DRILL-6492: Ensure schema / workspace case insensitivity in Drill

Posted by ar...@apache.org.
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

commit 8bcb103a0e3bcc5f85a03cbed3c6c0cea254ec4e
Author: Arina Ielchiieva <ar...@gmail.com>
AuthorDate: Fri Aug 17 17:04:30 2018 +0300

    DRILL-6492: Ensure schema / workspace case insensitivity in Drill
    
    1. StoragePluginsRegistryImpl was updated:
    a. for backward compatibility at init to convert all existing storage plugins names to lower case, in case of duplicates, to log warning and skip the duplicate.
    b. to wrap persistent plugins registry into case insensitive store wrapper (CaseInsensitivePersistentStore) to ensure all given keys are converted into lower case when performing insert, update, delete, search operations.
    c. to load system storage plugins dynamically by @SystemStorage annotation.
    2. StoragePlugins class was updated to stored storage plugins configs by name in case insensitive map.
    3. SchemaUtilities.searchSchemaTree method was updated to convert all schema names into lower case to ensure that are they are matched case insensitively (all schemas are stored in Drill in lower case).
    4. FileSystemConfig was updated to store workspaces by name in case insensitive hash map.
    5. All plugins schema factories are now extend AbstractSchemaFactory to ensure that given schema name is converted to lower case.
    6. New method areTableNamesAreCaseInsensitive was added to AbstractSchema to indicate if schema tables names are case insensitive. By default, false. Schema implementation is responsible for table names case insensitive search in case it supports one. Currently, information_schema, sys and hive do so.
    7. System storage plugins (information_schema, sys) were refactored to ensure their schema, table names are case insensitive, also the annotation @SystemPlugin and additional constructor were added to allow dynamically load system plugins at storage plugin registry during init phase.
    8. MetadataProvider was updated to concert all schema filter conditions into lower case to ensure schema would be matched case insensitively.
    9. ShowSchemasHandler, ShowTablesHandler, DescribeTableHandler were updated to ensure schema / tables names (this depends if schema supports case insensitive table names) would be found case insensitively.
    
    git closes #1439
---
 .../drill/common/map/CaseInsensitiveMap.java       |  25 +-
 .../drill/common/scanner/ClassPathScanner.java     |  22 +-
 .../drill/common/map/TestCaseInsensitiveMap.java   |  15 +
 .../drill/exec/store/hbase/HBaseSchemaFactory.java |  26 +-
 .../exec/store/hive/schema/HiveDatabaseSchema.java |  11 +-
 .../exec/store/hive/schema/HiveSchemaFactory.java  |  29 +-
 .../apache/drill/exec/hive/TestHiveStorage.java    |   5 +
 .../exec/hive/TestInfoSchemaOnHiveStorage.java     |   2 +-
 .../hive/BaseTestHiveImpersonation.java            |  11 -
 .../hive/TestSqlStdBasedAuthorization.java         |  19 +-
 .../hive/TestStorageBasedHiveAuthorization.java    |  30 +-
 .../store/kafka/schema/KafkaSchemaFactory.java     |  14 +-
 .../drill/exec/store/kudu/KuduSchemaFactory.java   |  22 +-
 .../store/mongo/schema/MongoSchemaFactory.java     |  16 +-
 contrib/storage-opentsdb/README.md                 |  16 +-
 .../exec/store/openTSDB/OpenTSDBStoragePlugin.java |   5 +-
 .../openTSDB/schema/OpenTSDBSchemaFactory.java     |  19 +-
 .../drill/exec/coord/zk/ZookeeperClient.java       |   2 +-
 .../expr/fn/registry/LocalFunctionRegistry.java    |  11 +-
 .../drill/exec/planner/logical/StoragePlugins.java |   7 +-
 .../drill/exec/planner/sql/SchemaUtilites.java     |   3 +-
 .../sql/handlers/DescribeSchemaHandler.java        |   6 +-
 .../planner/sql/handlers/DescribeTableHandler.java |  69 +--
 .../planner/sql/handlers/ShowSchemasHandler.java   |  27 +-
 .../planner/sql/handlers/ShowTablesHandler.java    |  74 ++--
 .../apache/drill/exec/store/AbstractSchema.java    |  23 +-
 ...chemaConfig.java => AbstractSchemaFactory.java} |  22 +-
 .../drill/exec/store/AbstractStoragePlugin.java    |  14 +-
 .../drill/exec/store/PartitionExplorerImpl.java    |   2 +-
 .../org/apache/drill/exec/store/SchemaFactory.java |   5 +-
 .../org/apache/drill/exec/store/StoragePlugin.java |   2 +
 .../apache/drill/exec/store/StoragePluginMap.java  |   6 +-
 .../drill/exec/store/StoragePluginRegistry.java    |   6 +-
 .../exec/store/StoragePluginRegistryImpl.java      | 468 ++++++++++++---------
 .../InfoSchemaConfig.java => SystemPlugin.java}    |  28 +-
 .../drill/exec/store/dfs/FileSystemConfig.java     |   6 +-
 .../drill/exec/store/dfs/FileSystemPlugin.java     |   2 +
 .../exec/store/dfs/FileSystemSchemaFactory.java    |  20 +-
 .../drill/exec/store/ischema/InfoSchemaConfig.java |   3 +-
 .../exec/store/ischema/InfoSchemaConstants.java    |   2 +-
 .../store/ischema/InfoSchemaStoragePlugin.java     |  62 ++-
 .../store/sys/CaseInsensitivePersistentStore.java  |  79 ++++
 .../drill/exec/store/sys/SystemTablePlugin.java    |  54 ++-
 .../exec/store/sys/SystemTablePluginConfig.java    |   1 -
 .../drill/exec/work/metadata/MetadataProvider.java |  24 +-
 .../java-exec/src/main/resources/drill-module.conf |  25 +-
 .../drill/common/scanner/TestClassPathScanner.java |  43 +-
 .../drill/exec/coord/zk/TestZookeeperClient.java   |  10 +-
 .../exec/impersonation/BaseTestImpersonation.java  |  19 +-
 .../TestImpersonationDisabledWithMiniDFS.java      |  12 +-
 .../impersonation/TestImpersonationMetadata.java   | 112 ++---
 .../impersonation/TestImpersonationQueries.java    |  17 +-
 .../exec/planner/TestDirectoryExplorerUDFs.java    |   2 +-
 .../org/apache/drill/exec/sql/TestInfoSchema.java  |  64 ++-
 .../exec/sql/TestSchemaCaseInsensitivity.java      |  92 ++++
 .../exec/work/metadata/TestMetadataProvider.java   |  38 +-
 56 files changed, 1061 insertions(+), 688 deletions(-)

diff --git a/common/src/main/java/org/apache/drill/common/map/CaseInsensitiveMap.java b/common/src/main/java/org/apache/drill/common/map/CaseInsensitiveMap.java
index 20e46dd..a9f18d3 100644
--- a/common/src/main/java/org/apache/drill/common/map/CaseInsensitiveMap.java
+++ b/common/src/main/java/org/apache/drill/common/map/CaseInsensitiveMap.java
@@ -22,6 +22,7 @@ import com.google.common.collect.Maps;
 
 import java.util.Collection;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 
 /**
@@ -41,7 +42,7 @@ public class CaseInsensitiveMap<VALUE> implements Map<String, VALUE> {
    * @return key case-insensitive concurrent map
    */
   public static <VALUE> CaseInsensitiveMap<VALUE> newConcurrentMap() {
-    return new CaseInsensitiveMap<>(Maps.<String, VALUE>newConcurrentMap());
+    return new CaseInsensitiveMap<>(Maps.newConcurrentMap());
   }
 
   /**
@@ -51,7 +52,7 @@ public class CaseInsensitiveMap<VALUE> implements Map<String, VALUE> {
    * @return key case-insensitive hash map
    */
   public static <VALUE> CaseInsensitiveMap<VALUE> newHashMap() {
-    return new CaseInsensitiveMap<>(Maps.<String, VALUE>newHashMap());
+    return new CaseInsensitiveMap<>(Maps.newHashMap());
   }
 
   /**
@@ -63,7 +64,7 @@ public class CaseInsensitiveMap<VALUE> implements Map<String, VALUE> {
    * @return key case-insensitive hash map
    */
   public static <VALUE> CaseInsensitiveMap<VALUE> newHashMapWithExpectedSize(final int expectedSize) {
-    return new CaseInsensitiveMap<>(Maps.<String, VALUE>newHashMapWithExpectedSize(expectedSize));
+    return new CaseInsensitiveMap<>(Maps.newHashMapWithExpectedSize(expectedSize));
   }
 
   /**
@@ -154,4 +155,22 @@ public class CaseInsensitiveMap<VALUE> implements Map<String, VALUE> {
   public Set<Entry<String, VALUE>> entrySet() {
     return underlyingMap.entrySet();
   }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(underlyingMap);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof CaseInsensitiveMap)) {
+      return false;
+    }
+    CaseInsensitiveMap<?> that = (CaseInsensitiveMap<?>) o;
+    return Objects.equals(underlyingMap, that.underlyingMap);
+  }
+
 }
diff --git a/common/src/main/java/org/apache/drill/common/scanner/ClassPathScanner.java b/common/src/main/java/org/apache/drill/common/scanner/ClassPathScanner.java
index 909e811..552c073 100644
--- a/common/src/main/java/org/apache/drill/common/scanner/ClassPathScanner.java
+++ b/common/src/main/java/org/apache/drill/common/scanner/ClassPathScanner.java
@@ -133,12 +133,9 @@ public final class ClassPathScanner {
      * @return the class names of all children direct or indirect
      */
     public Set<ChildClassDescriptor> getChildrenOf(String name) {
-      Set<ChildClassDescriptor> result = new HashSet<>();
       Collection<ChildClassDescriptor> scannedChildren = children.get(name);
       // add all scanned children
-      for (ChildClassDescriptor child : scannedChildren) {
-        result.add(child);
-      }
+      Set<ChildClassDescriptor> result = new HashSet<>(scannedChildren);
       // recursively add children's children
       Collection<ChildClassDescriptor> allChildren = new ArrayList<>();
       allChildren.addAll(scannedChildren);
@@ -272,7 +269,7 @@ public final class ClassPathScanner {
           List<FieldDescriptor> fieldDescriptors = new ArrayList<>(classFields.size());
           for (FieldInfo field : classFields) {
             String fieldName = field.getName();
-            AnnotationsAttribute fieldAnnotations = ((AnnotationsAttribute)field.getAttribute(AnnotationsAttribute.visibleTag));
+            AnnotationsAttribute fieldAnnotations = ((AnnotationsAttribute) field.getAttribute(AnnotationsAttribute.visibleTag));
             fieldDescriptors.add(new FieldDescriptor(fieldName, field.getDescriptor(), getAnnotationDescriptors(fieldAnnotations)));
           }
           functions.add(new AnnotatedClassDescriptor(classFile.getName(), classAnnotations, fieldDescriptors));
@@ -281,6 +278,9 @@ public final class ClassPathScanner {
     }
 
     private List<AnnotationDescriptor> getAnnotationDescriptors(AnnotationsAttribute annotationsAttr) {
+      if (annotationsAttr == null) {
+        return Collections.emptyList();
+      }
       List<AnnotationDescriptor> annotationDescriptors = new ArrayList<>(annotationsAttr.numAnnotations());
       for (javassist.bytecode.annotation.Annotation annotation : annotationsAttr.getAnnotations()) {
         // Sigh: javassist uses raw collections (is this 2002?)
@@ -319,7 +319,7 @@ public final class ClassPathScanner {
    *           to scan for (relative to specified class loaders' classpath roots)
    * @param  returnRootPathname  whether to collect classpath root portion of
    *           URL for each resource instead of full URL of each resource
-   * @returns  ...; empty set if none
+   * @return  empty set if none
    */
   public static Set<URL> forResource(final String resourcePathname, final boolean returnRootPathname) {
     logger.debug("Scanning classpath for resources with pathname \"{}\".",
@@ -437,11 +437,11 @@ public final class ClassPathScanner {
 
   static ScanResult emptyResult() {
     return new ScanResult(
-        Collections.<String>emptyList(),
-        Collections.<String>emptyList(),
-        Collections.<String>emptyList(),
-        Collections.<AnnotatedClassDescriptor>emptyList(),
-        Collections.<ParentClassDescriptor>emptyList());
+        Collections.emptyList(),
+        Collections.emptyList(),
+        Collections.emptyList(),
+        Collections.emptyList(),
+        Collections.emptyList());
   }
 
   public static ScanResult fromPrescan(DrillConfig config) {
diff --git a/common/src/test/java/org/apache/drill/common/map/TestCaseInsensitiveMap.java b/common/src/test/java/org/apache/drill/common/map/TestCaseInsensitiveMap.java
index 8901d53..19b2b93 100644
--- a/common/src/test/java/org/apache/drill/common/map/TestCaseInsensitiveMap.java
+++ b/common/src/test/java/org/apache/drill/common/map/TestCaseInsensitiveMap.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import java.util.Set;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
@@ -66,4 +67,18 @@ public class TestCaseInsensitiveMap {
     final Set<Map.Entry<String, Integer>> entrySet = map.entrySet();
     assertEquals(2, entrySet.size());
   }
+
+  @Test
+  public void checkEquals() {
+    Map<String, String> map1 = CaseInsensitiveMap.newHashMap();
+    map1.put("key_1", "value_1");
+
+    Map<String, String> map2 = CaseInsensitiveMap.newHashMap();
+    map2.put("KEY_1", "value_1");
+    assertEquals(map1, map2);
+
+    map2.put("key_2", "value_2");
+    assertNotEquals(map1, map2);
+  }
+
 }
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
index b8e825b..46e0444 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
@@ -24,36 +24,34 @@ import java.util.Set;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
 import org.apache.drill.exec.store.AbstractSchema;
+import org.apache.drill.exec.store.AbstractSchemaFactory;
 import org.apache.drill.exec.store.SchemaConfig;
-import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.Admin;
 
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Sets;
 
-public class HBaseSchemaFactory implements SchemaFactory {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HBaseSchemaFactory.class);
+public class HBaseSchemaFactory extends AbstractSchemaFactory {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HBaseSchemaFactory.class);
 
-  final String schemaName;
-  final HBaseStoragePlugin plugin;
+  private final HBaseStoragePlugin plugin;
 
   public HBaseSchemaFactory(HBaseStoragePlugin plugin, String name) throws IOException {
+    super(name);
     this.plugin = plugin;
-    this.schemaName = name;
   }
 
   @Override
   public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException {
-    HBaseSchema schema = new HBaseSchema(schemaName);
-    SchemaPlus hPlus = parent.add(schemaName, schema);
+    HBaseSchema schema = new HBaseSchema(getName());
+    SchemaPlus hPlus = parent.add(getName(), schema);
     schema.setHolder(hPlus);
   }
 
   class HBaseSchema extends AbstractSchema {
 
-    public HBaseSchema(String name) {
-      super(ImmutableList.<String>of(), name);
+    HBaseSchema(String name) {
+      super(Collections.emptyList(), name);
     }
 
     public void setHolder(SchemaPlus plusOfThis) {
@@ -73,13 +71,13 @@ public class HBaseSchemaFactory implements SchemaFactory {
     public Table getTable(String name) {
       HBaseScanSpec scanSpec = new HBaseScanSpec(name);
       try {
-        return new DrillHBaseTable(schemaName, plugin, scanSpec);
+        return new DrillHBaseTable(getName(), plugin, scanSpec);
       } catch (Exception e) {
         // Calcite firstly looks for a table in the default schema, if the table was not found,
         // it looks in the root schema.
         // If the table does not exist, a query will fail at validation stage,
         // so the error should not be thrown here.
-        logger.warn("Failure while loading table '{}' for database '{}'.", name, schemaName, e.getCause());
+        logger.warn("Failure while loading table '{}' for database '{}'.", name, getName(), e.getCause());
         return null;
       }
     }
@@ -94,7 +92,7 @@ public class HBaseSchemaFactory implements SchemaFactory {
         }
         return tableNames;
       } catch (Exception e) {
-        logger.warn("Failure while loading table names for database '{}'.", schemaName, e.getCause());
+        logger.warn("Failure while loading table names for database '{}'.", getName(), e.getCause());
         return Collections.emptySet();
       }
     }
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveDatabaseSchema.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveDatabaseSchema.java
index ec1d0c6..23f346f 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveDatabaseSchema.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveDatabaseSchema.java
@@ -38,7 +38,8 @@ import org.apache.thrift.TException;
 import java.util.List;
 import java.util.Set;
 
-public class HiveDatabaseSchema extends AbstractSchema{
+public class HiveDatabaseSchema extends AbstractSchema {
+
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveDatabaseSchema.class);
 
   private final HiveSchema hiveSchema;
@@ -105,10 +106,16 @@ public class HiveDatabaseSchema extends AbstractSchema{
     return tableNameToTable;
   }
 
+  @Override
+  public boolean areTableNamesCaseSensitive() {
+    return false;
+  }
+
   private static class HiveTableWithoutStatisticAndRowType implements Table {
+
     private final TableType tableType;
 
-    public HiveTableWithoutStatisticAndRowType(final TableType tableType) {
+    HiveTableWithoutStatisticAndRowType(final TableType tableType) {
       this.tableType = tableType;
     }
 
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
index e3cb3a2..53f6c60 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
@@ -33,8 +33,8 @@ import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.store.AbstractSchema;
+import org.apache.drill.exec.store.AbstractSchemaFactory;
 import org.apache.drill.exec.store.SchemaConfig;
-import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.drill.exec.store.hive.DrillHiveMetaStoreClient;
 import org.apache.drill.exec.store.hive.HiveReadEntry;
 import org.apache.drill.exec.store.hive.HiveStoragePlugin;
@@ -48,8 +48,8 @@ import org.apache.thrift.TException;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Sets;
 
-public class HiveSchemaFactory implements SchemaFactory {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveSchemaFactory.class);
+public class HiveSchemaFactory extends AbstractSchemaFactory {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveSchemaFactory.class);
 
   // MetaStoreClient created using process user credentials
   private final DrillHiveMetaStoreClient processUserMetastoreClient;
@@ -57,13 +57,12 @@ public class HiveSchemaFactory implements SchemaFactory {
   private final LoadingCache<String, DrillHiveMetaStoreClient> metaStoreClientLoadingCache;
 
   private final HiveStoragePlugin plugin;
-  private final String schemaName;
   private final HiveConf hiveConf;
   private final boolean isDrillImpersonationEnabled;
   private final boolean isHS2DoAsSet;
 
   public HiveSchemaFactory(final HiveStoragePlugin plugin, final String name, final HiveConf hiveConf) throws ExecutionSetupException {
-    this.schemaName = name;
+    super(name);
     this.plugin = plugin;
 
     this.hiveConf = hiveConf;
@@ -126,8 +125,8 @@ public class HiveSchemaFactory implements SchemaFactory {
         throw new IOException("Failure setting up Hive metastore client.", e);
       }
     }
-    HiveSchema schema = new HiveSchema(schemaConfig, mClientForSchemaTree, schemaName);
-    SchemaPlus hPlus = parent.add(schemaName, schema);
+    HiveSchema schema = new HiveSchema(schemaConfig, mClientForSchemaTree, getName());
+    SchemaPlus hPlus = parent.add(getName(), schema);
     schema.setHolder(hPlus);
   }
 
@@ -137,7 +136,7 @@ public class HiveSchemaFactory implements SchemaFactory {
     private final DrillHiveMetaStoreClient mClient;
     private HiveDatabaseSchema defaultSchema;
 
-    public HiveSchema(final SchemaConfig schemaConfig, final DrillHiveMetaStoreClient mClient, final String name) {
+    HiveSchema(final SchemaConfig schemaConfig, final DrillHiveMetaStoreClient mClient, final String name) {
       super(ImmutableList.<String>of(), name);
       this.schemaConfig = schemaConfig;
       this.mClient = mClient;
@@ -149,7 +148,7 @@ public class HiveSchemaFactory implements SchemaFactory {
       try {
         List<String> dbs = mClient.getDatabases(schemaConfig.getIgnoreAuthErrors());
         if (!dbs.contains(name)) {
-          logger.debug("Database '{}' doesn't exists in Hive storage '{}'", name, schemaName);
+          logger.debug("Database '{}' doesn't exists in Hive storage '{}'", name, getName());
           return null;
         }
         HiveDatabaseSchema schema = getSubSchemaKnownExists(name);
@@ -164,8 +163,7 @@ public class HiveSchemaFactory implements SchemaFactory {
 
     /** Help method to get subschema when we know it exists (already checks the existence) */
     private HiveDatabaseSchema getSubSchemaKnownExists(String name) {
-      HiveDatabaseSchema schema = new HiveDatabaseSchema(this, name, mClient, schemaConfig);
-      return schema;
+      return new HiveDatabaseSchema(this, name, mClient, schemaConfig);
     }
 
     void setHolder(SchemaPlus plusOfThis) {
@@ -206,6 +204,11 @@ public class HiveSchemaFactory implements SchemaFactory {
       return defaultSchema.getTableNames();
     }
 
+    @Override
+    public boolean areTableNamesCaseSensitive() {
+      return false;
+    }
+
     DrillTable getDrillTable(String dbName, String t) {
       HiveReadEntry entry = getSelectionBaseOnName(dbName, t);
       if (entry == null) {
@@ -216,9 +219,9 @@ public class HiveSchemaFactory implements SchemaFactory {
           ImpersonationUtil.getProcessUserName();
 
       if (entry.getJdbcTableType() == TableType.VIEW) {
-        return new DrillHiveViewTable(schemaName, plugin, userToImpersonate, entry);
+        return new DrillHiveViewTable(getName(), plugin, userToImpersonate, entry);
       } else {
-        return new DrillHiveTable(schemaName, plugin, userToImpersonate, entry);
+        return new DrillHiveTable(getName(), plugin, userToImpersonate, entry);
       }
     }
 
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java
index 94f39b8..2410010 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveStorage.java
@@ -435,6 +435,11 @@ public class TestHiveStorage extends HiveTestBase {
     test(query);
   }
 
+  @Test
+  public void testSchemaCaseInsensitive() throws Exception {
+    test("select * from Hive.`Default`.Kv");
+  }
+
   private void verifyColumnsMetadata(List<UserProtos.ResultColumnMetadata> columnsList, Map<String, Integer> expectedResult) {
     for (UserProtos.ResultColumnMetadata columnMetadata : columnsList) {
       assertTrue("Column should be present in result set", expectedResult.containsKey(columnMetadata.getColumnName()));
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java
index c5c0d48..37b8ea0 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java
@@ -90,7 +90,7 @@ public class TestInfoSchemaOnHiveStorage extends HiveTestBase {
         .baselineValues("dfs.tmp")
         .baselineValues("sys")
         .baselineValues("cp.default")
-        .baselineValues("INFORMATION_SCHEMA")
+        .baselineValues("information_schema")
         .go();
   }
 
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 3c6e2c2..e361c66 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
@@ -19,8 +19,6 @@ package org.apache.drill.exec.impersonation.hive;
 
 import org.apache.calcite.schema.Schema.TableType;
 import org.apache.drill.test.TestBuilder;
-import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.dotdrill.DotDrillType;
 import org.apache.drill.exec.impersonation.BaseTestImpersonation;
 import org.apache.drill.exec.store.hive.HiveStoragePluginConfig;
 import org.apache.hadoop.fs.FileSystem;
@@ -154,15 +152,6 @@ public class BaseTestHiveImpersonation extends BaseTestImpersonation {
     testBuilder.go();
   }
 
-  protected static void createView(final String viewOwner, final String viewGroup, final String viewName,
-                                 final String viewDef) throws Exception {
-    updateClient(viewOwner);
-    test(String.format("ALTER SESSION SET `%s`='%o';", ExecConstants.NEW_VIEW_DEFAULT_PERMS_KEY, (short) 0750));
-    test("CREATE VIEW %s.%s.%s AS %s", MINIDFS_STORAGE_PLUGIN_NAME, "tmp", viewName, viewDef);
-    final Path viewFilePath = new Path("/tmp/", viewName + DotDrillType.VIEW.getEnding());
-    fs.setOwner(viewFilePath, viewOwner, viewGroup);
-  }
-
   public static void stopHiveMetaStore() throws Exception {
     // Unfortunately Hive metastore doesn't provide an API to shut it down. It will be exited as part of the test JVM
     // exit. As each metastore server instance is using its own resources and not sharing it with other metastore
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestSqlStdBasedAuthorization.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestSqlStdBasedAuthorization.java
index 30b7430..7bc98a3 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestSqlStdBasedAuthorization.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestSqlStdBasedAuthorization.java
@@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Maps;
 import org.apache.drill.categories.HiveStorageTest;
 import org.apache.drill.categories.SlowTest;
-import org.apache.drill.exec.store.dfs.WorkspaceConfig;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
@@ -34,6 +33,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import java.util.HashMap;
 import java.util.Map;
 
 import static org.apache.drill.exec.hive.HiveTestUtilities.executeQuery;
@@ -66,10 +66,10 @@ public class TestSqlStdBasedAuthorization extends BaseTestHiveImpersonation {
   private static final String v_student_u1g1_750 = "v_student_u1g1_750";
 
   private static final String query_v_student_u0g0_750 = String.format(
-      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINIDFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u0g0_750);
+      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u0g0_750);
 
   private static final String query_v_student_u1g1_750 = String.format(
-      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINIDFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u1g1_750);
+      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u1g1_750);
 
   // Role for testing purpose
   private static final String test_role0 = "role0";
@@ -82,7 +82,7 @@ public class TestSqlStdBasedAuthorization extends BaseTestHiveImpersonation {
     startHiveMetaStore();
     startDrillCluster(true);
     addHiveStoragePlugin(getHivePluginConfig());
-    addMiniDfsBasedStorage(Maps.<String, WorkspaceConfig>newHashMap());
+    addMiniDfsBasedStorage(new HashMap<>());
     generateTestData();
   }
 
@@ -134,8 +134,7 @@ public class TestSqlStdBasedAuthorization extends BaseTestHiveImpersonation {
             hivePluginName, db_general, g_student_user0));
 
     createView(org1Users[1], org1Groups[1], v_student_u1g1_750,
-        String.format("SELECT rownum, name, age FROM %s.%s.%s",
-            MINIDFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u0g0_750));
+        String.format("SELECT rownum, name, age FROM %s.%s.%s", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u0g0_750));
   }
 
   private static void createTbl(final Driver driver, final String db, final String tbl, final String tblDef,
@@ -277,15 +276,15 @@ public class TestSqlStdBasedAuthorization extends BaseTestHiveImpersonation {
   @Test
   public void selectUser2_v_student_u0g0_750() throws Exception {
     updateClient(org1Users[2]);
-    errorMsgTestHelper(query_v_student_u0g0_750,
-        "Not authorized to read view [v_student_u0g0_750] in schema [miniDfsPlugin.tmp]");
+    errorMsgTestHelper(query_v_student_u0g0_750, String.format(
+        "Not authorized to read view [v_student_u0g0_750] in schema [%s.tmp]", MINI_DFS_STORAGE_PLUGIN_NAME));
   }
 
   @Test
   public void selectUser0_v_student_u1g1_750() throws Exception {
     updateClient(org1Users[0]);
-    errorMsgTestHelper(query_v_student_u1g1_750,
-        "Not authorized to read view [v_student_u1g1_750] in schema [miniDfsPlugin.tmp]");
+    errorMsgTestHelper(query_v_student_u1g1_750, String.format(
+        "Not authorized to read view [v_student_u1g1_750] in schema [%s.tmp]", MINI_DFS_STORAGE_PLUGIN_NAME));
   }
 
   @Test
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestStorageBasedHiveAuthorization.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestStorageBasedHiveAuthorization.java
index 5a3e373..d54c4e0 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestStorageBasedHiveAuthorization.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestStorageBasedHiveAuthorization.java
@@ -92,10 +92,10 @@ public class TestStorageBasedHiveAuthorization extends BaseTestHiveImpersonation
   private static final String v_student_u1g1_750 = "v_student_u1g1_750";
 
   private static final String query_v_student_u0g0_750 = String.format(
-      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINIDFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u0g0_750);
+      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u0g0_750);
 
   private static final String query_v_student_u1g1_750 = String.format(
-      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINIDFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u1g1_750);
+      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u1g1_750);
 
   // Create a view on "partitioned_student_u0_700". View is owned by user0:group0 and has permissions 750
   private static final String v_partitioned_student_u0g0_750 = "v_partitioned_student_u0g0_750";
@@ -104,11 +104,11 @@ public class TestStorageBasedHiveAuthorization extends BaseTestHiveImpersonation
   private static final String v_partitioned_student_u1g1_750 = "v_partitioned_student_u1g1_750";
 
   private static final String query_v_partitioned_student_u0g0_750 = String.format(
-      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINIDFS_STORAGE_PLUGIN_NAME, "tmp",
+      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp",
       v_partitioned_student_u0g0_750);
 
   private static final String query_v_partitioned_student_u1g1_750 = String.format(
-      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINIDFS_STORAGE_PLUGIN_NAME, "tmp",
+      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp",
       v_partitioned_student_u1g1_750);
 
   @BeforeClass
@@ -199,16 +199,14 @@ public class TestStorageBasedHiveAuthorization extends BaseTestHiveImpersonation
             hivePluginName, db_general, g_student_u0_700));
 
     createView(org1Users[1], org1Groups[1], v_student_u1g1_750,
-        String.format("SELECT rownum, name, age FROM %s.%s.%s",
-            MINIDFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u0g0_750));
+        String.format("SELECT rownum, name, age FROM %s.%s.%s", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u0g0_750));
 
     createView(org1Users[0], org1Groups[0], v_partitioned_student_u0g0_750,
         String.format("SELECT rownum, name, age, studentnum FROM %s.%s.%s",
             hivePluginName, db_general, g_partitioned_student_u0_700));
 
     createView(org1Users[1], org1Groups[1], v_partitioned_student_u1g1_750,
-        String.format("SELECT rownum, name, age FROM %s.%s.%s",
-            MINIDFS_STORAGE_PLUGIN_NAME, "tmp", v_partitioned_student_u0g0_750));
+        String.format("SELECT rownum, name, age FROM %s.%s.%s", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", v_partitioned_student_u0g0_750));
   }
 
   private static void createPartitionedTable(final Driver hiveDriver, final String db, final String tbl,
@@ -521,15 +519,15 @@ public class TestStorageBasedHiveAuthorization extends BaseTestHiveImpersonation
   @Test
   public void selectUser2_v_student_u0g0_750() throws Exception {
     updateClient(org1Users[2]);
-    errorMsgTestHelper(query_v_student_u0g0_750,
-        "Not authorized to read view [v_student_u0g0_750] in schema [miniDfsPlugin.tmp]");
+    errorMsgTestHelper(query_v_student_u0g0_750, String.format(
+        "Not authorized to read view [v_student_u0g0_750] in schema [%s.tmp]", MINI_DFS_STORAGE_PLUGIN_NAME));
   }
 
   @Test
   public void selectUser0_v_student_u1g1_750() throws Exception {
     updateClient(org1Users[0]);
-    errorMsgTestHelper(query_v_student_u1g1_750,
-        "Not authorized to read view [v_student_u1g1_750] in schema [miniDfsPlugin.tmp]");
+    errorMsgTestHelper(query_v_student_u1g1_750, String.format(
+        "Not authorized to read view [v_student_u1g1_750] in schema [%s.tmp]", MINI_DFS_STORAGE_PLUGIN_NAME));
   }
 
   @Test
@@ -555,15 +553,15 @@ public class TestStorageBasedHiveAuthorization extends BaseTestHiveImpersonation
   @Test
   public void selectUser2_v_partitioned_student_u0g0_750() throws Exception {
     updateClient(org1Users[2]);
-    errorMsgTestHelper(query_v_partitioned_student_u0g0_750,
-        "Not authorized to read view [v_partitioned_student_u0g0_750] in schema [miniDfsPlugin.tmp]");
+    errorMsgTestHelper(query_v_partitioned_student_u0g0_750, String.format(
+        "Not authorized to read view [v_partitioned_student_u0g0_750] in schema [%s.tmp]", MINI_DFS_STORAGE_PLUGIN_NAME));
   }
 
   @Test
   public void selectUser0_v_partitioned_student_u1g1_750() throws Exception {
     updateClient(org1Users[0]);
-    errorMsgTestHelper(query_v_partitioned_student_u1g1_750,
-        "Not authorized to read view [v_partitioned_student_u1g1_750] in schema [miniDfsPlugin.tmp]");
+    errorMsgTestHelper(query_v_partitioned_student_u1g1_750, String.format(
+        "Not authorized to read view [v_partitioned_student_u1g1_750] in schema [%s.tmp]", MINI_DFS_STORAGE_PLUGIN_NAME));
   }
 
   @Test
diff --git a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/schema/KafkaSchemaFactory.java b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/schema/KafkaSchemaFactory.java
index 8f44a93..86ef095 100644
--- a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/schema/KafkaSchemaFactory.java
+++ b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/schema/KafkaSchemaFactory.java
@@ -20,25 +20,23 @@ package org.apache.drill.exec.store.kafka.schema;
 import java.io.IOException;
 
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.drill.exec.store.AbstractSchemaFactory;
 import org.apache.drill.exec.store.SchemaConfig;
-import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.drill.exec.store.kafka.KafkaStoragePlugin;
 
-public class KafkaSchemaFactory implements SchemaFactory {
+public class KafkaSchemaFactory extends AbstractSchemaFactory {
 
-  private final String schemaName;
   private final KafkaStoragePlugin plugin;
 
   public KafkaSchemaFactory(KafkaStoragePlugin plugin, String schemaName) {
+    super(schemaName);
     this.plugin = plugin;
-    this.schemaName = schemaName;
   }
 
   @Override
-  public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent)
-      throws IOException {
-    KafkaMessageSchema schema = new KafkaMessageSchema(this.plugin, this.schemaName);
-    SchemaPlus hPlus = parent.add(schemaName, schema);
+  public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException {
+    KafkaMessageSchema schema = new KafkaMessageSchema(plugin, getName());
+    SchemaPlus hPlus = parent.add(getName(), schema);
     schema.setHolder(hPlus);
   }
 
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java
index fe2def3..c1eb1e5 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduSchemaFactory.java
@@ -29,37 +29,35 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.Writer;
 import org.apache.drill.exec.planner.logical.CreateTableEntry;
 import org.apache.drill.exec.store.AbstractSchema;
+import org.apache.drill.exec.store.AbstractSchemaFactory;
 import org.apache.drill.exec.store.SchemaConfig;
-import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.kudu.Schema;
 import org.apache.kudu.client.KuduTable;
 import org.apache.kudu.client.ListTablesResponse;
 
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Sets;
 
-public class KuduSchemaFactory implements SchemaFactory {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KuduSchemaFactory.class);
+public class KuduSchemaFactory extends AbstractSchemaFactory {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KuduSchemaFactory.class);
 
-  final String schemaName;
-  final KuduStoragePlugin plugin;
+  private final KuduStoragePlugin plugin;
 
   public KuduSchemaFactory(KuduStoragePlugin plugin, String name) throws IOException {
+    super(name);
     this.plugin = plugin;
-    this.schemaName = name;
   }
 
   @Override
   public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException {
-    KuduTables schema = new KuduTables(schemaName);
-    SchemaPlus hPlus = parent.add(schemaName, schema);
+    KuduTables schema = new KuduTables(getName());
+    SchemaPlus hPlus = parent.add(getName(), schema);
     schema.setHolder(hPlus);
   }
 
   class KuduTables extends AbstractSchema {
 
-    public KuduTables(String name) {
-      super(ImmutableList.<String>of(), name);
+    KuduTables(String name) {
+      super(Collections.emptyList(), name);
     }
 
     public void setHolder(SchemaPlus plusOfThis) {
@@ -81,7 +79,7 @@ public class KuduSchemaFactory implements SchemaFactory {
       try {
         KuduTable table = plugin.getClient().openTable(name);
         Schema schema = table.getSchema();
-        return new DrillKuduTable(schemaName, plugin, schema, scanSpec);
+        return new DrillKuduTable(getName(), plugin, schema, scanSpec);
       } catch (Exception e) {
         logger.warn("Failure while retrieving kudu table {}", name, e);
         return null;
diff --git a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/schema/MongoSchemaFactory.java b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/schema/MongoSchemaFactory.java
index 366f129..3437420 100644
--- a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/schema/MongoSchemaFactory.java
+++ b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/schema/MongoSchemaFactory.java
@@ -32,8 +32,8 @@ import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.planner.logical.DynamicDrillTable;
 import org.apache.drill.exec.store.AbstractSchema;
+import org.apache.drill.exec.store.AbstractSchemaFactory;
 import org.apache.drill.exec.store.SchemaConfig;
-import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.drill.exec.store.mongo.MongoScanSpec;
 import org.apache.drill.exec.store.mongo.MongoStoragePlugin;
 import org.apache.drill.exec.store.mongo.MongoStoragePluginConfig;
@@ -49,21 +49,19 @@ import com.google.common.collect.Sets;
 import com.mongodb.MongoException;
 import com.mongodb.client.MongoDatabase;
 
-public class MongoSchemaFactory implements SchemaFactory {
+public class MongoSchemaFactory extends AbstractSchemaFactory {
 
-  static final Logger logger = LoggerFactory
-      .getLogger(MongoSchemaFactory.class);
+  private static final Logger logger = LoggerFactory.getLogger(MongoSchemaFactory.class);
 
   private static final String DATABASES = "databases";
 
   private LoadingCache<String, List<String>> databases;
   private LoadingCache<String, List<String>> tableNameLoader;
-  private final String schemaName;
   private final MongoStoragePlugin plugin;
 
   public MongoSchemaFactory(MongoStoragePlugin plugin, String schemaName) throws ExecutionSetupException {
+    super(schemaName);
     this.plugin = plugin;
-    this.schemaName = schemaName;
 
     databases = CacheBuilder //
         .newBuilder() //
@@ -119,8 +117,8 @@ public class MongoSchemaFactory implements SchemaFactory {
 
   @Override
   public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException {
-    MongoSchema schema = new MongoSchema(schemaName);
-    SchemaPlus hPlus = parent.add(schemaName, schema);
+    MongoSchema schema = new MongoSchema(getName());
+    SchemaPlus hPlus = parent.add(getName(), schema);
     schema.setHolder(hPlus);
   }
 
@@ -186,7 +184,7 @@ public class MongoSchemaFactory implements SchemaFactory {
 
     DrillTable getDrillTable(String dbName, String collectionName) {
       MongoScanSpec mongoScanSpec = new MongoScanSpec(dbName, collectionName);
-      return new DynamicDrillTable(plugin, schemaName, null, mongoScanSpec);
+      return new DynamicDrillTable(plugin, getName(), null, mongoScanSpec);
     }
 
     @Override
diff --git a/contrib/storage-opentsdb/README.md b/contrib/storage-opentsdb/README.md
index 0c616b5..04ac1a6 100644
--- a/contrib/storage-opentsdb/README.md
+++ b/contrib/storage-opentsdb/README.md
@@ -1,4 +1,4 @@
-# drill-storage-openTSDB
+# drill--opentsdb-storage
 
 Implementation of TSDB storage plugin. Plugin uses REST API to work with TSDB. 
 
@@ -30,12 +30,12 @@ List of supported time
 
 Params must be specified in FROM clause of the query separated by commas. For example
 
-`openTSDB.(metric=metric_name, start=4d-ago, aggregator=sum)`
+`opentsdb.(metric=metric_name, start=4d-ago, aggregator=sum)`
 
 Supported queries for now are listed below:
 
 ```
-USE openTSDB
+USE opentsdb
 ```
 
 ```
@@ -44,26 +44,26 @@ SHOW tables
 Will print available metrics. Max number of the printed results is a Integer.MAX value
 
 ```
-SELECT * FROM openTSDB. `(metric=warp.speed.test, start=47y-ago, aggregator=sum)` 
+SELECT * FROM opentsdb. `(metric=warp.speed.test, start=47y-ago, aggregator=sum)` 
 ```
 Return aggregated elements from `warp.speed.test` table since 47y-ago 
 
 ```
-SELECT * FROM openTSDB.`(metric=warp.speed.test, aggregator=avg, start=47y-ago)`
+SELECT * FROM opentsdb.`(metric=warp.speed.test, aggregator=avg, start=47y-ago)`
 ```
 Return aggregated elements from `warp.speed.test` table
 
 ```
-SELECT `timestamp`, sum(`aggregated value`) FROM openTSDB.`(metric=warp.speed.test, aggregator=avg, start=47y-ago)` GROUP BY `timestamp`
+SELECT `timestamp`, sum(`aggregated value`) FROM opentsdb.`(metric=warp.speed.test, aggregator=avg, start=47y-ago)` GROUP BY `timestamp`
 ```
 Return aggregated and grouped value by standard drill functions from `warp.speed.test table`, but with the custom aggregator
 
 ```
-SELECT * FROM openTSDB.`(metric=warp.speed.test, aggregator=avg, start=47y-ago, downsample=5m-avg)`
+SELECT * FROM opentsdb.`(metric=warp.speed.test, aggregator=avg, start=47y-ago, downsample=5m-avg)`
 ```
 Return aggregated data limited by downsample
 
 ```
-SELECT * FROM openTSDB.`(metric=warp.speed.test, aggregator=avg, start=47y-ago, end=1407165403000)`
+SELECT * FROM opentsdb.`(metric=warp.speed.test, aggregator=avg, start=47y-ago, end=1407165403000)`
 ```
 Return aggregated data limited by end time
\ No newline at end of file
diff --git a/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBStoragePlugin.java b/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBStoragePlugin.java
index 2de763b..539442a 100644
--- a/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBStoragePlugin.java
+++ b/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBStoragePlugin.java
@@ -36,10 +36,9 @@ public class OpenTSDBStoragePlugin extends AbstractStoragePlugin {
 
   private final ServiceImpl db;
 
-  public OpenTSDBStoragePlugin(OpenTSDBStoragePluginConfig configuration, DrillbitContext context, String name)
-      throws IOException {
+  public OpenTSDBStoragePlugin(OpenTSDBStoragePluginConfig configuration, DrillbitContext context, String name) throws IOException {
     super(context, name);
-    this.schemaFactory = new OpenTSDBSchemaFactory(this, name);
+    this.schemaFactory = new OpenTSDBSchemaFactory(this, getName());
     this.engineConfig = configuration;
     this.db = new ServiceImpl(configuration.getConnection());
   }
diff --git a/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/schema/OpenTSDBSchemaFactory.java b/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/schema/OpenTSDBSchemaFactory.java
index 3b86a95..f7ae4f3 100644
--- a/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/schema/OpenTSDBSchemaFactory.java
+++ b/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/schema/OpenTSDBSchemaFactory.java
@@ -20,8 +20,8 @@ package org.apache.drill.exec.store.openTSDB.schema;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
 import org.apache.drill.exec.store.AbstractSchema;
+import org.apache.drill.exec.store.AbstractSchemaFactory;
 import org.apache.drill.exec.store.SchemaConfig;
-import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.drill.exec.store.openTSDB.DrillOpenTSDBTable;
 import org.apache.drill.exec.store.openTSDB.OpenTSDBScanSpec;
 import org.apache.drill.exec.store.openTSDB.OpenTSDBStoragePlugin;
@@ -34,41 +34,40 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.Set;
 
-public class OpenTSDBSchemaFactory implements SchemaFactory {
+public class OpenTSDBSchemaFactory extends AbstractSchemaFactory {
 
-  private static final Logger log = LoggerFactory.getLogger(OpenTSDBSchemaFactory.class);
+  private static final Logger logger = LoggerFactory.getLogger(OpenTSDBSchemaFactory.class);
 
-  private final String schemaName;
   private final OpenTSDBStoragePlugin plugin;
 
   public OpenTSDBSchemaFactory(OpenTSDBStoragePlugin plugin, String schemaName) {
+    super(schemaName);
     this.plugin = plugin;
-    this.schemaName = schemaName;
   }
 
   @Override
   public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException {
-    OpenTSDBSchema schema = new OpenTSDBSchema(schemaName);
-    parent.add(schemaName, schema);
+    OpenTSDBSchema schema = new OpenTSDBSchema(getName());
+    parent.add(getName(), schema);
   }
 
   class OpenTSDBSchema extends AbstractSchema {
 
     OpenTSDBSchema(String name) {
-      super(Collections.<String>emptyList(), name);
+      super(Collections.emptyList(), name);
     }
 
     @Override
     public Table getTable(String name) {
       OpenTSDBScanSpec scanSpec = new OpenTSDBScanSpec(name);
       try {
-        return new DrillOpenTSDBTable(schemaName, plugin, new Schema(plugin.getClient(), name), scanSpec);
+        return new DrillOpenTSDBTable(getName(), plugin, new Schema(plugin.getClient(), name), scanSpec);
       } catch (Exception e) {
         // Calcite firstly looks for a table in the default schema, if the table was not found,
         // it looks in the root schema.
         // If the table does not exist, a query will fail at validation stage,
         // so the error should not be thrown here.
-        logger.warn("Failure while loading table '{}' for database '{}'.", name, schemaName, e.getCause());
+        logger.warn("Failure while loading table '{}' for database '{}'.", name, getName(), e.getCause());
         return null;
       }
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZookeeperClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZookeeperClient.java
index 0839898..97dd2d2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZookeeperClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/coord/zk/ZookeeperClient.java
@@ -68,7 +68,7 @@ public class ZookeeperClient implements AutoCloseable {
    */
   public void start() throws Exception {
     curator.newNamespaceAwareEnsurePath(root).ensure(curator.getZookeeperClient()); // ensure root is created
-    getCache().start();
+    getCache().start(PathChildrenCache.StartMode.BUILD_INITIAL_CACHE); //build cache at start up, to ensure we get correct results right away
   }
 
   public PathChildrenCache getCache() {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/LocalFunctionRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/LocalFunctionRegistry.java
index 1318f72..9b1a34a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/LocalFunctionRegistry.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/LocalFunctionRegistry.java
@@ -17,7 +17,9 @@
  */
 package org.apache.drill.exec.expr.fn.registry;
 
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -35,6 +37,7 @@ import org.apache.drill.common.scanner.persistence.AnnotatedClassDescriptor;
 import org.apache.drill.common.scanner.persistence.ScanResult;
 import org.apache.drill.exec.exception.FunctionValidationException;
 import org.apache.drill.exec.exception.JarValidationException;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
 import org.apache.drill.exec.expr.fn.DrillFuncHolder;
 import org.apache.drill.exec.expr.fn.FunctionConverter;
 import org.apache.drill.exec.planner.logical.DrillConstExecutor;
@@ -112,7 +115,7 @@ public class LocalFunctionRegistry {
   public List<String> validate(String jarName, ScanResult scanResult) {
     List<String> functions = Lists.newArrayList();
     FunctionConverter converter = new FunctionConverter();
-    List<AnnotatedClassDescriptor> providerClasses = scanResult.getAnnotatedClasses();
+    List<AnnotatedClassDescriptor> providerClasses = scanResult.getAnnotatedClasses(FunctionTemplate.class.getName());
 
     if (registryHolder.containsJar(jarName)) {
       throw new JarValidationException(String.format("Jar with %s name has been already registered", jarName));
@@ -158,11 +161,11 @@ public class LocalFunctionRegistry {
    * @param version remote function registry version number with which local function registry is synced
    */
   public void register(List<JarScan> jars, long version) {
-    Map<String, List<FunctionHolder>> newJars = Maps.newHashMap();
+    Map<String, List<FunctionHolder>> newJars = new HashMap<>();
     for (JarScan jarScan : jars) {
       FunctionConverter converter = new FunctionConverter();
-      List<AnnotatedClassDescriptor> providerClasses = jarScan.getScanResult().getAnnotatedClasses();
-      List<FunctionHolder> functions = Lists.newArrayList();
+      List<AnnotatedClassDescriptor> providerClasses = jarScan.getScanResult().getAnnotatedClasses(FunctionTemplate.class.getName());
+      List<FunctionHolder> functions = new ArrayList<>();
       newJars.put(jarScan.getJarName(), functions);
       for (AnnotatedClassDescriptor func : providerClasses) {
         DrillFuncHolder holder = converter.getHolder(func, jarScan.getClassLoader());
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/StoragePlugins.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/StoragePlugins.java
index 1493a92..ad0c2cc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/StoragePlugins.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/StoragePlugins.java
@@ -28,6 +28,7 @@ import java.util.Optional;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.config.LogicalPlanPersistence;
 import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.map.CaseInsensitiveMap;
 import org.apache.drill.common.scanner.ClassPathScanner;
 import org.apache.drill.common.scanner.persistence.ScanResult;
 
@@ -38,11 +39,13 @@ import com.google.common.io.Resources;
 
 public class StoragePlugins implements Iterable<Map.Entry<String, StoragePluginConfig>> {
 
-  private Map<String, StoragePluginConfig> storage;
+  private final Map<String, StoragePluginConfig> storage;
 
   @JsonCreator
   public StoragePlugins(@JsonProperty("storage") Map<String, StoragePluginConfig> storage) {
-    this.storage = storage;
+    Map<String, StoragePluginConfig> caseInsensitiveStorage = CaseInsensitiveMap.newHashMap();
+    Optional.ofNullable(storage).ifPresent(caseInsensitiveStorage::putAll);
+    this.storage = caseInsensitiveStorage;
   }
 
   public static void main(String[] args) throws Exception{
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java
index a262363..b47ab32 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java
@@ -104,7 +104,8 @@ public class SchemaUtilites {
   /** Utility method to search for schema path starting from the given <i>schema</i> reference */
   private static SchemaPlus searchSchemaTree(SchemaPlus schema, final List<String> schemaPath) {
     for (String schemaName : schemaPath) {
-      schema = schema.getSubSchema(schemaName);
+      // schemas in Drill are case insensitive and stored in lower case
+      schema = schema.getSubSchema(schemaName.toLowerCase());
       if (schema == null) {
         return null;
       }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java
index bb51ef0..3f11fd1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java
@@ -80,9 +80,9 @@ public class DescribeSchemaHandler extends DefaultSqlHandler {
         .build(logger);
     }
 
+    AbstractSchema drillSchema = SchemaUtilites.unwrapAsDrillSchemaInstance(schemaPlus);
     StoragePlugin storagePlugin;
     try {
-      AbstractSchema drillSchema = SchemaUtilites.unwrapAsDrillSchemaInstance(schemaPlus);
       storagePlugin = context.getStorage().getPlugin(drillSchema.getSchemaPath().get(0));
       if (storagePlugin == null) {
         throw new DrillRuntimeException(String.format("Unable to find storage plugin with the following name [%s].",
@@ -95,10 +95,10 @@ public class DescribeSchemaHandler extends DefaultSqlHandler {
     try {
       Map configMap = mapper.convertValue(storagePlugin.getConfig(), Map.class);
       if (storagePlugin instanceof FileSystemPlugin) {
-        transformWorkspaces(schema.names, configMap);
+        transformWorkspaces(drillSchema.getSchemaPath(), configMap);
       }
       String properties = mapper.writeValueAsString(configMap);
-      return DirectPlan.createDirectPlan(context, new DescribeSchemaResult(Joiner.on(".").join(schema.names), properties));
+      return DirectPlan.createDirectPlan(context, new DescribeSchemaResult(drillSchema.getFullSchemaName(), properties));
     } catch (JsonProcessingException e) {
       throw new DrillRuntimeException("Error while trying to convert storage config to json string", e);
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeTableHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeTableHandler.java
index 32768f8..70dd5a8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeTableHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeTableHandler.java
@@ -24,10 +24,12 @@ import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.IS_SCHEMA_
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SHRD_COL_TABLE_NAME;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SHRD_COL_TABLE_SCHEMA;
 
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlCharStringLiteral;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
@@ -37,6 +39,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.tools.RelConversionException;
 import org.apache.calcite.tools.ValidationException;
+import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.drill.common.exceptions.UserException;
@@ -44,11 +47,10 @@ import org.apache.drill.exec.planner.sql.SchemaUtilites;
 import org.apache.drill.exec.planner.sql.SqlConverter;
 import org.apache.drill.exec.planner.sql.parser.DrillParserUtil;
 import org.apache.drill.exec.planner.sql.parser.DrillSqlDescribeTable;
+import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.ischema.InfoSchemaTableType;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 
-import com.google.common.collect.ImmutableList;
-
 public class DescribeTableHandler extends DefaultSqlHandler {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DescribeTableHandler.class);
 
@@ -60,23 +62,19 @@ public class DescribeTableHandler extends DefaultSqlHandler {
     DrillSqlDescribeTable node = unwrap(sqlNode, DrillSqlDescribeTable.class);
 
     try {
-      List<SqlNode> selectList =
-          ImmutableList.of(new SqlIdentifier(COLS_COL_COLUMN_NAME, SqlParserPos.ZERO),
-                           new SqlIdentifier(COLS_COL_DATA_TYPE, SqlParserPos.ZERO),
-                           new SqlIdentifier(COLS_COL_IS_NULLABLE, SqlParserPos.ZERO));
+      List<SqlNode> selectList = Arrays.asList(
+          new SqlIdentifier(COLS_COL_COLUMN_NAME, SqlParserPos.ZERO),
+          new SqlIdentifier(COLS_COL_DATA_TYPE, SqlParserPos.ZERO),
+          new SqlIdentifier(COLS_COL_IS_NULLABLE, SqlParserPos.ZERO));
 
-      SqlNode fromClause = new SqlIdentifier(
-          ImmutableList.of(IS_SCHEMA_NAME, InfoSchemaTableType.COLUMNS.name()), null, SqlParserPos.ZERO, null);
+      SqlNode fromClause = new SqlIdentifier(Arrays.asList(IS_SCHEMA_NAME, InfoSchemaTableType.COLUMNS.name()), SqlParserPos.ZERO);
 
-      final SqlIdentifier table = node.getTable();
-      final SchemaPlus defaultSchema = config.getConverter().getDefaultSchema();
-      final List<String> schemaPathGivenInCmd = Util.skipLast(table.names);
-      final SchemaPlus schema = SchemaUtilites.findSchema(defaultSchema, schemaPathGivenInCmd);
-      final String charset = Util.getDefaultCharset().name();
+      SchemaPlus defaultSchema = config.getConverter().getDefaultSchema();
+      List<String> schemaPathGivenInCmd = Util.skipLast(node.getTable().names);
+      SchemaPlus schema = SchemaUtilites.findSchema(defaultSchema, schemaPathGivenInCmd);
 
       if (schema == null) {
-        SchemaUtilites.throwSchemaNotFoundException(defaultSchema,
-            SchemaUtilites.SCHEMA_PATH_JOINER.join(schemaPathGivenInCmd));
+        SchemaUtilites.throwSchemaNotFoundException(defaultSchema, SchemaUtilites.getSchemaPath(schemaPathGivenInCmd));
       }
 
       if (SchemaUtilites.isRootSchema(schema)) {
@@ -85,10 +83,11 @@ public class DescribeTableHandler extends DefaultSqlHandler {
             .build(logger);
       }
 
-      final String tableName = Util.last(table.names);
-
       // find resolved schema path
-      final String schemaPath = SchemaUtilites.unwrapAsDrillSchemaInstance(schema).getFullSchemaName();
+      AbstractSchema drillSchema = SchemaUtilites.unwrapAsDrillSchemaInstance(schema);
+      String schemaPath = drillSchema.getFullSchemaName();
+
+      String tableName = Util.last(node.getTable().names);
 
       if (schema.getTable(tableName) == null) {
         throw UserException.validationError()
@@ -101,14 +100,21 @@ public class DescribeTableHandler extends DefaultSqlHandler {
         schemaCondition = DrillParserUtil.createCondition(
             new SqlIdentifier(SHRD_COL_TABLE_SCHEMA, SqlParserPos.ZERO),
             SqlStdOperatorTable.EQUALS,
-            SqlLiteral.createCharString(schemaPath, charset, SqlParserPos.ZERO)
+            SqlLiteral.createCharString(schemaPath, Util.getDefaultCharset().name(), SqlParserPos.ZERO)
         );
       }
 
-      SqlNode where = DrillParserUtil.createCondition(
-          new SqlIdentifier(SHRD_COL_TABLE_NAME, SqlParserPos.ZERO),
+      SqlNode tableNameColumn = new SqlIdentifier(SHRD_COL_TABLE_NAME, SqlParserPos.ZERO);
+
+      // if table names are case insensitive, wrap column values and condition in lower function
+      if (!drillSchema.areTableNamesCaseSensitive()) {
+        tableNameColumn = SqlStdOperatorTable.LOWER.createCall(SqlParserPos.ZERO, tableNameColumn);
+        tableName = tableName.toLowerCase();
+      }
+
+      SqlNode where = DrillParserUtil.createCondition(tableNameColumn,
           SqlStdOperatorTable.EQUALS,
-          SqlLiteral.createCharString(tableName, charset, SqlParserPos.ZERO));
+          SqlLiteral.createCharString(tableName, Util.getDefaultCharset().name(), SqlParserPos.ZERO));
 
       where = DrillParserUtil.createCondition(schemaCondition, SqlStdOperatorTable.AND, where);
 
@@ -116,14 +122,21 @@ public class DescribeTableHandler extends DefaultSqlHandler {
       if (node.getColumn() != null) {
         columnFilter =
             DrillParserUtil.createCondition(
-                new SqlIdentifier(COLS_COL_COLUMN_NAME, SqlParserPos.ZERO),
+                SqlStdOperatorTable.LOWER.createCall(SqlParserPos.ZERO, new SqlIdentifier(COLS_COL_COLUMN_NAME, SqlParserPos.ZERO)),
                 SqlStdOperatorTable.EQUALS,
-                SqlLiteral.createCharString(node.getColumn().toString(), charset, SqlParserPos.ZERO));
+                SqlLiteral.createCharString(node.getColumn().toString().toLowerCase(), Util.getDefaultCharset().name(), SqlParserPos.ZERO));
       } else if (node.getColumnQualifier() != null) {
-        columnFilter =
-            DrillParserUtil.createCondition(
-                new SqlIdentifier(COLS_COL_COLUMN_NAME, SqlParserPos.ZERO),
-                SqlStdOperatorTable.LIKE, node.getColumnQualifier());
+        SqlNode columnQualifier = node.getColumnQualifier();
+        SqlNode column = new SqlIdentifier(COLS_COL_COLUMN_NAME, SqlParserPos.ZERO);
+        if (columnQualifier instanceof SqlCharStringLiteral) {
+          NlsString conditionString = ((SqlCharStringLiteral) columnQualifier).getNlsString();
+          columnQualifier = SqlCharStringLiteral.createCharString(
+              conditionString.getValue().toLowerCase(),
+              conditionString.getCharsetName(),
+              columnQualifier.getParserPosition());
+          column = SqlStdOperatorTable.LOWER.createCall(SqlParserPos.ZERO, column);
+        }
+        columnFilter = DrillParserUtil.createCondition(column, SqlStdOperatorTable.LIKE, columnQualifier);
       }
 
       where = DrillParserUtil.createCondition(where, SqlStdOperatorTable.AND, columnFilter);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowSchemasHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowSchemasHandler.java
index ab460ad..2c07d2c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowSchemasHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowSchemasHandler.java
@@ -17,8 +17,12 @@
  */
 package org.apache.drill.exec.planner.sql.handlers;
 
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
+import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.util.NlsString;
 import org.apache.drill.exec.planner.sql.parser.DrillParserUtil;
 import org.apache.drill.exec.planner.sql.parser.SqlShowSchemas;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.IS_SCHEMA_NAME;
@@ -33,8 +37,6 @@ import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
 
-import com.google.common.collect.ImmutableList;
-
 public class ShowSchemasHandler extends DefaultSqlHandler {
 
   public ShowSchemasHandler(SqlHandlerConfig config) { super(config); }
@@ -43,17 +45,24 @@ public class ShowSchemasHandler extends DefaultSqlHandler {
   @Override
   public SqlNode rewrite(SqlNode sqlNode) throws ForemanSetupException {
     SqlShowSchemas node = unwrap(sqlNode, SqlShowSchemas.class);
-    List<SqlNode> selectList =
-        ImmutableList.of(new SqlIdentifier(SCHS_COL_SCHEMA_NAME, SqlParserPos.ZERO));
+    List<SqlNode> selectList = Collections.singletonList(new SqlIdentifier(SCHS_COL_SCHEMA_NAME, SqlParserPos.ZERO));
 
-    SqlNode fromClause = new SqlIdentifier(
-        ImmutableList.of(IS_SCHEMA_NAME, InfoSchemaTableType.SCHEMATA.name()), null, SqlParserPos.ZERO, null);
+    SqlNode fromClause = new SqlIdentifier(Arrays.asList(IS_SCHEMA_NAME, InfoSchemaTableType.SCHEMATA.name()), SqlParserPos.ZERO);
 
     SqlNode where = null;
-    final SqlNode likePattern = node.getLikePattern();
+    SqlNode likePattern = node.getLikePattern();
     if (likePattern != null) {
-      where = DrillParserUtil.createCondition(new SqlIdentifier(SCHS_COL_SCHEMA_NAME, SqlParserPos.ZERO),
-                                              SqlStdOperatorTable.LIKE, likePattern);
+      SqlNode column = new SqlIdentifier(SCHS_COL_SCHEMA_NAME, SqlParserPos.ZERO);
+      // schema names are case insensitive, wrap column in lower function, pattern to lower case
+      if (likePattern instanceof SqlCharStringLiteral) {
+        NlsString conditionString = ((SqlCharStringLiteral) likePattern).getNlsString();
+        likePattern = SqlCharStringLiteral.createCharString(
+            conditionString.getValue().toLowerCase(),
+            conditionString.getCharsetName(),
+            likePattern.getParserPosition());
+        column = SqlStdOperatorTable.LOWER.createCall(SqlParserPos.ZERO, column);
+      }
+      where = DrillParserUtil.createCondition(column, SqlStdOperatorTable.LIKE, likePattern);
     } else if (node.getWhereClause() != null) {
       where = node.getWhereClause();
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowTablesHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowTablesHandler.java
index e73e829..a910f9a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowTablesHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowTablesHandler.java
@@ -21,10 +21,12 @@ import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.IS_SCHEMA_
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SHRD_COL_TABLE_NAME;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SHRD_COL_TABLE_SCHEMA;
 
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlCharStringLiteral;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
@@ -34,6 +36,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.tools.RelConversionException;
 import org.apache.calcite.tools.ValidationException;
+import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.drill.common.exceptions.UserException;
@@ -45,9 +48,6 @@ import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.ischema.InfoSchemaTableType;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-
 public class ShowTablesHandler extends DefaultSqlHandler {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ShowTablesHandler.class);
 
@@ -57,48 +57,56 @@ public class ShowTablesHandler extends DefaultSqlHandler {
   @Override
   public SqlNode rewrite(SqlNode sqlNode) throws ForemanSetupException {
     SqlShowTables node = unwrap(sqlNode, SqlShowTables.class);
-    List<SqlNode> selectList = Lists.newArrayList();
-    SqlNode fromClause;
-    SqlNode where;
-
-    // create select columns
-    selectList.add(new SqlIdentifier(SHRD_COL_TABLE_SCHEMA, SqlParserPos.ZERO));
-    selectList.add(new SqlIdentifier(SHRD_COL_TABLE_NAME, SqlParserPos.ZERO));
+    List<SqlNode> selectList = Arrays.asList(
+        new SqlIdentifier(SHRD_COL_TABLE_SCHEMA, SqlParserPos.ZERO),
+        new SqlIdentifier(SHRD_COL_TABLE_NAME, SqlParserPos.ZERO));
 
-    fromClause = new SqlIdentifier(ImmutableList.of(IS_SCHEMA_NAME, InfoSchemaTableType.TABLES.name()), SqlParserPos.ZERO);
+    SqlNode fromClause = new SqlIdentifier(Arrays.asList(IS_SCHEMA_NAME, InfoSchemaTableType.TABLES.name()), SqlParserPos.ZERO);
 
-    final SqlIdentifier db = node.getDb();
-    String tableSchema;
-    if (db != null) {
-      tableSchema = db.toString();
-    } else {
-      // If no schema is given in SHOW TABLES command, list tables from current schema
-      SchemaPlus schema = config.getConverter().getDefaultSchema();
+    SchemaPlus schemaPlus;
+    if (node.getDb() != null) {
+      List<String> schemaNames = node.getDb().names;
+      schemaPlus = SchemaUtilites.findSchema(config.getConverter().getDefaultSchema(), schemaNames);
 
-      if (SchemaUtilites.isRootSchema(schema)) {
-        // If the default schema is a root schema, throw an error to select a default schema
+      if (schemaPlus == null) {
         throw UserException.validationError()
-            .message("No default schema selected. Select a schema using 'USE schema' command")
+            .message(String.format("Invalid schema name [%s]", SchemaUtilites.getSchemaPath(schemaNames)))
             .build(logger);
       }
 
-      final AbstractSchema drillSchema = SchemaUtilites.unwrapAsDrillSchemaInstance(schema);
-      tableSchema = drillSchema.getFullSchemaName();
+    } else {
+      // If no schema is given in SHOW TABLES command, list tables from current schema
+      schemaPlus = config.getConverter().getDefaultSchema();
+    }
+
+    if (SchemaUtilites.isRootSchema(schemaPlus)) {
+      // If the default schema is a root schema, throw an error to select a default schema
+      throw UserException.validationError()
+          .message("No default schema selected. Select a schema using 'USE schema' command")
+          .build(logger);
     }
 
-    final String charset = Util.getDefaultCharset().name();
-    where = DrillParserUtil.createCondition(
+    AbstractSchema drillSchema = SchemaUtilites.unwrapAsDrillSchemaInstance(schemaPlus);
+
+    SqlNode where = DrillParserUtil.createCondition(
         new SqlIdentifier(SHRD_COL_TABLE_SCHEMA, SqlParserPos.ZERO),
         SqlStdOperatorTable.EQUALS,
-        SqlLiteral.createCharString(tableSchema, charset, SqlParserPos.ZERO));
+        SqlLiteral.createCharString(drillSchema.getFullSchemaName(), Util.getDefaultCharset().name(), SqlParserPos.ZERO));
 
     SqlNode filter = null;
-    final SqlNode likePattern = node.getLikePattern();
-    if (likePattern != null) {
-      filter = DrillParserUtil.createCondition(
-          new SqlIdentifier(SHRD_COL_TABLE_NAME, SqlParserPos.ZERO),
-          SqlStdOperatorTable.LIKE,
-          likePattern);
+    if (node.getLikePattern() != null) {
+      SqlNode likePattern = node.getLikePattern();
+      SqlNode column = new SqlIdentifier(SHRD_COL_TABLE_NAME, SqlParserPos.ZERO);
+      // wrap columns name values and condition in lower function if case insensitive
+      if (!drillSchema.areTableNamesCaseSensitive() && likePattern instanceof SqlCharStringLiteral) {
+        NlsString conditionString = ((SqlCharStringLiteral) likePattern).getNlsString();
+        likePattern = SqlCharStringLiteral.createCharString(
+            conditionString.getValue().toLowerCase(),
+            conditionString.getCharsetName(),
+            likePattern.getParserPosition());
+        column = SqlStdOperatorTable.LOWER.createCall(SqlParserPos.ZERO, column);
+      }
+      filter = DrillParserUtil.createCondition(column, SqlStdOperatorTable.LIKE, likePattern);
     } else if (node.getWhereClause() != null) {
       filter = node.getWhereClause();
     }
@@ -119,4 +127,4 @@ public class ShowTablesHandler extends DefaultSqlHandler {
     converter.useRootSchemaAsDefault(false);
     return sqlNodeRelDataTypePair;
   }
-}
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
index fb9bc6b..2b70c3b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.store;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -46,7 +47,8 @@ public abstract class AbstractSchema implements Schema, SchemaPartitionExplorer,
   private static final Expression EXPRESSION = new DefaultExpression(Object.class);
 
   public AbstractSchema(List<String> parentSchemaPath, String name) {
-    schemaPath = Lists.newArrayList();
+    name = name == null ? null : name.toLowerCase();
+    schemaPath = new ArrayList<>();
     schemaPath.addAll(parentSchemaPath);
     schemaPath.add(name);
     this.name = name;
@@ -96,7 +98,7 @@ public abstract class AbstractSchema implements Schema, SchemaPartitionExplorer,
    * Create a new view given definition.
    * @param view View info including name, definition etc.
    * @return Returns true if an existing view is replaced with the given view. False otherwise.
-   * @throws IOException
+   * @throws IOException in case of error creating a view
    */
   public boolean createView(View view) throws IOException {
     throw UserException.unsupportedError()
@@ -107,8 +109,8 @@ public abstract class AbstractSchema implements Schema, SchemaPartitionExplorer,
   /**
    * Drop the view with given name.
    *
-   * @param viewName
-   * @throws IOException
+   * @param viewName view name
+   * @throws IOException in case of error dropping the view
    */
   public void dropView(String viewName) throws IOException {
     throw UserException.unsupportedError()
@@ -217,7 +219,7 @@ public abstract class AbstractSchema implements Schema, SchemaPartitionExplorer,
    * plugin supports).
    * It is not guaranteed that the retrieved tables would have RowType and Statistic being fully populated.
    *
-   * Specifically, calling {@link Table#getRowType(RelDataTypeFactory)} or {@link Table#getStatistic()} might incur
+   * Specifically, calling {@link Table#getRowType(org.apache.calcite.rel.type.RelDataTypeFactory)} or {@link Table#getStatistic()} might incur
    * {@link UnsupportedOperationException} being thrown.
    *
    * @param  tableNames the requested tables, specified by the table names
@@ -263,4 +265,15 @@ public abstract class AbstractSchema implements Schema, SchemaPartitionExplorer,
     return tableNamesAndTypes;
   }
 
+  /**
+   * Indicates if table names in schema are case sensitive. By default they are.
+   * If schema implementation claims its table names are case insensitive,
+   * it is responsible for making case insensitive look up by table name.
+   *
+   * @return true if table names are case sensitive
+   */
+  public boolean areTableNamesCaseSensitive() {
+    return true;
+  }
+
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchemaFactory.java
similarity index 64%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConfig.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchemaFactory.java
index adf15a2..da5af1d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchemaFactory.java
@@ -15,23 +15,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.store.ischema;
+package org.apache.drill.exec.store;
 
-import org.apache.drill.common.logical.StoragePluginConfig;
-
-public class InfoSchemaConfig extends StoragePluginConfig {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(InfoSchemaConfig.class);
+/**
+ * Abstract implementation of {@link SchemaFactory}, ensures that given schema name is always converted is lower case.
+ */
+public abstract class AbstractSchemaFactory implements SchemaFactory {
 
-  public static final String NAME = "ischema";
+  private final String name;
 
-  @Override
-  public int hashCode(){
-    return 1;
+  protected AbstractSchemaFactory(String name) {
+    this.name = name == null ? null : name.toLowerCase();
   }
 
-  @Override
-  public boolean equals(Object o){
-    return o instanceof InfoSchemaConfig;
+  public String getName() {
+    return name;
   }
 
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java
index 9818ff3..b90fd63 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java
@@ -34,7 +34,8 @@ import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.options.SessionOptionManager;
 import org.apache.drill.exec.store.dfs.FormatPlugin;
 
-/** Abstract class for StorePlugin implementations.
+/**
+ * Abstract class for StorePlugin implementations.
  * See StoragePlugin for description of the interface intent and its methods.
  */
 public abstract class AbstractStoragePlugin implements StoragePlugin {
@@ -44,7 +45,7 @@ public abstract class AbstractStoragePlugin implements StoragePlugin {
 
   protected AbstractStoragePlugin(DrillbitContext inContext, String inName) {
     this.context = inContext;
-    this.name = inName;
+    this.name = inName == null ? null : inName.toLowerCase();
   }
 
   @Override
@@ -137,12 +138,13 @@ public abstract class AbstractStoragePlugin implements StoragePlugin {
     throw new UnsupportedOperationException(String.format("%s doesn't support format plugins", getClass().getName()));
   }
 
-  public DrillbitContext getContext() {
-    return context;
-  }
-
+  @Override
   public String getName() {
     return name;
   }
 
+  public DrillbitContext getContext() {
+    return context;
+  }
+
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/PartitionExplorerImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/PartitionExplorerImpl.java
index c87d8f6..a63d015 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/PartitionExplorerImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/PartitionExplorerImpl.java
@@ -36,7 +36,7 @@ public class PartitionExplorerImpl implements PartitionExplorer {
                                            List<String> partitionValues
                                            ) throws PartitionNotFoundException {
 
-    AbstractSchema subSchema = rootSchema.getSubSchema(schema).unwrap(AbstractSchema.class);
+    AbstractSchema subSchema = rootSchema.getSubSchema(schema.toLowerCase()).unwrap(AbstractSchema.class);
     return subSchema.getSubPartitions(table, partitionColumns, partitionValues);
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaFactory.java
index 2027527..4545169 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaFactory.java
@@ -25,14 +25,13 @@ import java.io.IOException;
  * StoragePlugins implements this interface to register the schemas they provide.
  */
 public interface SchemaFactory {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SchemaFactory.class);
 
   /**
    * Register the schemas provided by this SchemaFactory implementation under the given parent schema.
    *
    * @param schemaConfig Configuration for schema objects.
    * @param parent Reference to parent schema.
-   * @throws IOException
+   * @throws IOException in case of error during schema registration
    */
-  public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException;
+  void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException;
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java
index 7bd7eaf..4e6a7c2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java
@@ -112,4 +112,6 @@ public interface StoragePlugin extends SchemaFactory, AutoCloseable {
    * @throws UnsupportedOperationException, if storage plugin doesn't support format plugins.
    */
   FormatPlugin getFormatPlugin(FormatPluginConfig config);
+
+  String getName();
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginMap.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginMap.java
index 582791e..62fd031 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginMap.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginMap.java
@@ -22,8 +22,6 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.logical.StoragePluginConfig;
@@ -31,17 +29,19 @@ import org.apache.drill.common.logical.StoragePluginConfig;
 import com.google.common.collect.LinkedListMultimap;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Multimaps;
+import org.apache.drill.common.map.CaseInsensitiveMap;
 
 /**
  * Holds maps to storage plugins. Supports name => plugin and config => plugin mappings.
  *
  * This is inspired by ConcurrentMap but provides a secondary key mapping that allows an alternative lookup mechanism.
  * The class is responsible for internally managing consistency between the two maps. This class is threadsafe.
+ * Name map is case insensitive.
  */
 class StoragePluginMap implements Iterable<Entry<String, StoragePlugin>>, AutoCloseable {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StoragePluginMap.class);
 
-  private final ConcurrentMap<String, StoragePlugin> nameMap = new ConcurrentHashMap<>();
+  private final Map<String, StoragePlugin> nameMap = CaseInsensitiveMap.newConcurrentMap();
 
   @SuppressWarnings("unchecked")
   private final Multimap<StoragePluginConfig, StoragePlugin> configMap =
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java
index 313d3b9..036187a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java
@@ -27,8 +27,7 @@ import org.apache.drill.exec.store.dfs.FormatPlugin;
 import org.apache.drill.exec.store.sys.PersistentStore;
 
 public interface StoragePluginRegistry extends Iterable<Map.Entry<String, StoragePlugin>>, AutoCloseable {
-  String SYS_PLUGIN = "sys";
-  String INFORMATION_SCHEMA_PLUGIN = "INFORMATION_SCHEMA";
+
   String STORAGE_PLUGIN_REGISTRY_IMPL = "drill.exec.storage.registry";
   String ACTION_ON_STORAGE_PLUGINS_OVERRIDE_FILE = "drill.exec.storage.action_on_plugins_override_file";
   String PSTORE_NAME = "sys.storage_plugins";
@@ -92,8 +91,7 @@ public interface StoragePluginRegistry extends Iterable<Map.Entry<String, Storag
    * @return A FormatPlugin instance
    * @throws ExecutionSetupException if plugin cannot be obtained
    */
-  FormatPlugin getFormatPlugin(StoragePluginConfig storageConfig, FormatPluginConfig formatConfig)
-      throws ExecutionSetupException;
+  FormatPlugin getFormatPlugin(StoragePluginConfig storageConfig, FormatPluginConfig formatConfig) throws ExecutionSetupException;
 
   /**
    * Get the PStore for this StoragePluginRegistry. (Used in the management layer.)
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 03ce5a9..bd5a93d 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
@@ -43,19 +43,18 @@ import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.map.CaseInsensitiveMap;
 import org.apache.drill.common.scanner.ClassPathScanner;
+import org.apache.drill.common.scanner.persistence.AnnotatedClassDescriptor;
 import org.apache.drill.common.scanner.persistence.ScanResult;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.StoreException;
 import org.apache.drill.exec.planner.logical.StoragePlugins;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.dfs.FormatPlugin;
-import org.apache.drill.exec.store.ischema.InfoSchemaConfig;
-import org.apache.drill.exec.store.ischema.InfoSchemaStoragePlugin;
+import org.apache.drill.exec.store.sys.CaseInsensitivePersistentStore;
 import org.apache.drill.exec.store.sys.PersistentStore;
 import org.apache.drill.exec.store.sys.PersistentStoreConfig;
-import org.apache.drill.exec.store.sys.SystemTablePlugin;
-import org.apache.drill.exec.store.sys.SystemTablePluginConfig;
 
 import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
@@ -67,35 +66,28 @@ import com.google.common.cache.RemovalListener;
 import com.google.common.io.Resources;
 
 public class StoragePluginRegistryImpl implements StoragePluginRegistry {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StoragePluginRegistryImpl.class);
 
-  private Map<Object, Constructor<? extends StoragePlugin>> availablePlugins = Collections.emptyMap();
-  private final StoragePluginMap enabledPlugins = new StoragePluginMap();
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StoragePluginRegistryImpl.class);
 
-  private DrillbitContext context;
-  private final DrillSchemaFactory schemaFactory = new DrillSchemaFactory();
-  private final PersistentStore<StoragePluginConfig> pluginSystemTable;
+  private final StoragePluginMap enabledPlugins;
+  private final DrillSchemaFactory schemaFactory;
+  private final DrillbitContext context;
   private final LogicalPlanPersistence lpPersistence;
   private final ScanResult classpathScan;
+  private final PersistentStore<StoragePluginConfig> pluginSystemTable;
   private final LoadingCache<StoragePluginConfig, StoragePlugin> ephemeralPlugins;
 
+  private Map<Object, Constructor<? extends StoragePlugin>> availablePlugins = Collections.emptyMap();
+  private Map<String, StoragePlugin> systemPlugins = Collections.emptyMap();
+
   public StoragePluginRegistryImpl(DrillbitContext context) {
+    this.enabledPlugins = new StoragePluginMap();
+    this.schemaFactory = new DrillSchemaFactory(null);
     this.context = checkNotNull(context);
     this.lpPersistence = checkNotNull(context.getLpPersistence());
     this.classpathScan = checkNotNull(context.getClasspathScan());
-    try {
-      this.pluginSystemTable = context
-          .getStoreProvider()
-          .getOrCreateStore(PersistentStoreConfig
-              .newJacksonBuilder(lpPersistence.getMapper(), StoragePluginConfig.class)
-              .name(PSTORE_NAME)
-              .build());
-    } catch (StoreException | RuntimeException e) {
-      logger.error("Failure while loading storage plugin registry.", e);
-      throw new RuntimeException("Failure while reading and loading storage plugin configuration.", e);
-    }
-
-    ephemeralPlugins = CacheBuilder.newBuilder()
+    this.pluginSystemTable = initPluginsSystemTable(context, lpPersistence);
+    this.ephemeralPlugins = CacheBuilder.newBuilder()
         .expireAfterAccess(24, TimeUnit.HOURS)
         .maximumSize(250)
         .removalListener(
@@ -109,15 +101,11 @@ public class StoragePluginRegistryImpl implements StoragePluginRegistry {
   }
 
   @Override
-  public PersistentStore<StoragePluginConfig> getStore() {
-    return pluginSystemTable;
-  }
-
-  @Override
   public void init() {
     availablePlugins = findAvailablePlugins(classpathScan);
+    systemPlugins = initSystemPlugins(classpathScan, context);
     try {
-      StoragePlugins bootstrapPlugins = pluginSystemTable.getAll().hasNext() ? null : loadBootstrapPlugins();
+      StoragePlugins bootstrapPlugins = pluginSystemTable.getAll().hasNext() ? null : loadBootstrapPlugins(lpPersistence);
 
       StoragePluginsHandler storagePluginsHandler = new StoragePluginsHandlerService(context);
       storagePluginsHandler.loadPlugins(pluginSystemTable, bootstrapPlugins);
@@ -129,115 +117,18 @@ public class StoragePluginRegistryImpl implements StoragePluginRegistry {
     }
   }
 
-  /**
-   * Read bootstrap storage plugins {@link ExecConstants#BOOTSTRAP_STORAGE_PLUGINS_FILE} files for the first fresh
-   * instantiating of Drill
-   *
-   * @return bootstrap storage plugins
-   * @throws IOException if a read error occurs
-   */
-  private StoragePlugins loadBootstrapPlugins() throws IOException {
-    // bootstrap load the config since no plugins are stored.
-    logger.info("No storage plugin instances configured in persistent store, loading bootstrap configuration.");
-    Set<URL> urls = ClassPathScanner.forResource(ExecConstants.BOOTSTRAP_STORAGE_PLUGINS_FILE, false);
-    if (urls != null && !urls.isEmpty()) {
-      logger.info("Loading the storage plugin configs from URLs {}.", urls);
-      StoragePlugins bootstrapPlugins = new StoragePlugins(new HashMap<>());
-      Map<String, URL> pluginURLMap = new HashMap<>();
-      for (URL url : urls) {
-        String pluginsData = Resources.toString(url, Charsets.UTF_8);
-        StoragePlugins plugins = lpPersistence.getMapper().readValue(pluginsData, StoragePlugins.class);
-        for (Entry<String, StoragePluginConfig> plugin : plugins) {
-          StoragePluginConfig oldPluginConfig = bootstrapPlugins.putIfAbsent(plugin.getKey(), plugin.getValue());
-          if (oldPluginConfig != null) {
-            logger.warn("Duplicate plugin instance '{}' defined in [{}, {}], ignoring the later one.",
-                plugin.getKey(), pluginURLMap.get(plugin.getKey()), url);
-          } else {
-            pluginURLMap.put(plugin.getKey(), url);
-          }
-        }
-      }
-      return bootstrapPlugins;
-    } else {
-      throw new IOException("Failure finding " + ExecConstants.BOOTSTRAP_STORAGE_PLUGINS_FILE);
-    }
-  }
-
-  /**
-   * It initializes {@link #enabledPlugins} with currently enabled plugins
-   */
-  private void defineEnabledPlugins() {
-    Map<String, StoragePlugin> activePlugins = new HashMap<>();
-    Iterator<Entry<String, StoragePluginConfig>> allPlugins = pluginSystemTable.getAll();
-    while (allPlugins.hasNext()) {
-      Entry<String, StoragePluginConfig> plugin = allPlugins.next();
-      String name = plugin.getKey();
-      StoragePluginConfig config = plugin.getValue();
-      if (config.isEnabled()) {
-        try {
-          StoragePlugin storagePlugin = create(name, config);
-          activePlugins.put(name, storagePlugin);
-        } catch (ExecutionSetupException e) {
-          logger.error("Failure while setting up StoragePlugin with name: '{}', disabling.", name, e);
-          config.setEnabled(false);
-          pluginSystemTable.put(name, config);
-        }
-      }
-    }
-
-    activePlugins.put(INFORMATION_SCHEMA_PLUGIN, new InfoSchemaStoragePlugin(new InfoSchemaConfig(), context,
-        INFORMATION_SCHEMA_PLUGIN));
-    activePlugins.put(SYS_PLUGIN, new SystemTablePlugin(SystemTablePluginConfig.INSTANCE, context, SYS_PLUGIN));
-
-    enabledPlugins.putAll(activePlugins);
-  }
-
-  /**
-   * Add a plugin and configuration. Assumes neither exists. Primarily
-   * for testing.
-   *
-   * @param name plugin name
-   * @param config plugin config
-   * @param plugin plugin implementation
-   */
-  @VisibleForTesting
-  public void addPluginToPersistentStoreIfAbsent(String name, StoragePluginConfig config, StoragePlugin plugin) {
-    addEnabledPlugin(name, plugin);
-    pluginSystemTable.putIfAbsent(name, config);
-  }
-
-  @Override
-  public void addEnabledPlugin(String name, StoragePlugin plugin) {
-    enabledPlugins.put(name, plugin);
-  }
-
   @Override
   public void deletePlugin(String name) {
-    @SuppressWarnings("resource")
     StoragePlugin plugin = enabledPlugins.remove(name);
     closePlugin(plugin);
     pluginSystemTable.delete(name);
   }
 
-  private void closePlugin(StoragePlugin plugin) {
-    if (plugin == null) {
-      return;
-    }
-
-    try {
-      plugin.close();
-    } catch (Exception e) {
-      logger.warn("Exception while shutting down storage plugin.");
-    }
-  }
-
-  @SuppressWarnings("resource")
   @Override
-  public StoragePlugin createOrUpdate(String name, StoragePluginConfig config, boolean persist)
-      throws ExecutionSetupException {
+  public StoragePlugin createOrUpdate(String name, StoragePluginConfig config, boolean persist) throws ExecutionSetupException {
     for (;;) {
-      final StoragePlugin oldPlugin = enabledPlugins.get(name);
-      final StoragePlugin newPlugin = create(name, config);
+      StoragePlugin oldPlugin = enabledPlugins.get(name);
+      StoragePlugin newPlugin = create(name, config);
       boolean done = false;
       try {
         if (oldPlugin != null) {
@@ -273,12 +164,12 @@ public class StoragePluginRegistryImpl implements StoragePluginRegistry {
   @Override
   public StoragePlugin getPlugin(String name) throws ExecutionSetupException {
     StoragePlugin plugin = enabledPlugins.get(name);
-    if (name.equals(SYS_PLUGIN) || name.equals(INFORMATION_SCHEMA_PLUGIN)) {
+    if (systemPlugins.get(name) != null) {
       return plugin;
     }
 
     // since we lazily manage the list of plugins per server, we need to update this once we know that it is time.
-    StoragePluginConfig config = this.pluginSystemTable.get(name);
+    StoragePluginConfig config = pluginSystemTable.get(name);
     if (config == null) {
       if (plugin != null) {
         enabledPlugins.remove(name);
@@ -294,7 +185,6 @@ public class StoragePluginRegistryImpl implements StoragePluginRegistry {
     }
   }
 
-
   @Override
   public StoragePlugin getPlugin(StoragePluginConfig config) throws ExecutionSetupException {
     if (config instanceof NamedStoragePluginConfig) {
@@ -322,14 +212,246 @@ public class StoragePluginRegistryImpl implements StoragePluginRegistry {
     }
   }
 
-  @SuppressWarnings("resource")
   @Override
-  public FormatPlugin getFormatPlugin(StoragePluginConfig storageConfig, FormatPluginConfig formatConfig)
-      throws ExecutionSetupException {
+  public void addEnabledPlugin(String name, StoragePlugin plugin) {
+    enabledPlugins.put(name, plugin);
+  }
+
+  @Override
+  public FormatPlugin getFormatPlugin(StoragePluginConfig storageConfig, FormatPluginConfig formatConfig) throws ExecutionSetupException {
     StoragePlugin storagePlugin = getPlugin(storageConfig);
     return storagePlugin.getFormatPlugin(formatConfig);
   }
 
+  @Override
+  public PersistentStore<StoragePluginConfig> getStore() {
+    return pluginSystemTable;
+  }
+
+  @Override
+  public SchemaFactory getSchemaFactory() {
+    return schemaFactory;
+  }
+
+  @Override
+  public Iterator<Entry<String, StoragePlugin>> iterator() {
+    return enabledPlugins.iterator();
+  }
+
+  @Override
+  public synchronized void close() throws Exception {
+    ephemeralPlugins.invalidateAll();
+    enabledPlugins.close();
+    pluginSystemTable.close();
+  }
+
+  /**
+   * Add a plugin and configuration. Assumes neither exists. Primarily for testing.
+   *
+   * @param config plugin config
+   * @param plugin plugin implementation
+   */
+  @VisibleForTesting
+  public void addPluginToPersistentStoreIfAbsent(String name, StoragePluginConfig config, StoragePlugin plugin) {
+    addEnabledPlugin(name, plugin);
+    pluginSystemTable.putIfAbsent(name, config);
+  }
+
+  /**
+   * <ol>
+   *   <li>Initializes persistent store for storage plugins.</li>
+   *   <li>Since storage plugins names are case-insensitive in Drill, to ensure backward compatibility,
+   *   re-writes those not stored in lower case with lower case names, for duplicates issues warning. </li>
+   *   <li>Wraps plugin system table into case insensitive wrapper.</li>
+   * </ol>
+   *
+   * @param context drillbit context
+   * @param lpPersistence deserialization mapper provider
+   * @return persistent store for storage plugins
+   */
+  private PersistentStore<StoragePluginConfig> initPluginsSystemTable(DrillbitContext context, LogicalPlanPersistence lpPersistence) {
+
+    try {
+      PersistentStore<StoragePluginConfig> pluginSystemTable = context
+          .getStoreProvider()
+          .getOrCreateStore(PersistentStoreConfig
+              .newJacksonBuilder(lpPersistence.getMapper(), StoragePluginConfig.class)
+              .name(PSTORE_NAME)
+              .build());
+
+      Iterator<Entry<String, StoragePluginConfig>> storedPlugins = pluginSystemTable.getAll();
+      while (storedPlugins.hasNext()) {
+        Entry<String, StoragePluginConfig> entry = storedPlugins.next();
+        String pluginName = entry.getKey();
+        if (!pluginName.equals(pluginName.toLowerCase())) {
+          logger.debug("Replacing plugin name {} with its lower case equivalent.", pluginName);
+          pluginSystemTable.delete(pluginName);
+          if (!pluginSystemTable.putIfAbsent(pluginName.toLowerCase(), entry.getValue())) {
+            logger.warn("Duplicated storage plugin name [{}] is found. Duplicate is deleted from persistent storage.", pluginName);
+          }
+        }
+      }
+
+      return new CaseInsensitivePersistentStore<>(pluginSystemTable);
+    } catch (StoreException e) {
+      logger.error("Failure while loading storage plugin registry.", e);
+      throw new DrillRuntimeException("Failure while reading and loading storage plugin configuration.", e);
+    }
+  }
+
+  /**
+   * Read bootstrap storage plugins {@link ExecConstants#BOOTSTRAP_STORAGE_PLUGINS_FILE} files for the first fresh
+   * instantiating of Drill
+   *
+   * @param lpPersistence deserialization mapper provider
+   * @return bootstrap storage plugins
+   * @throws IOException if a read error occurs
+   */
+  private StoragePlugins loadBootstrapPlugins(LogicalPlanPersistence lpPersistence) throws IOException {
+    // bootstrap load the config since no plugins are stored.
+    logger.info("No storage plugin instances configured in persistent store, loading bootstrap configuration.");
+    Set<URL> urls = ClassPathScanner.forResource(ExecConstants.BOOTSTRAP_STORAGE_PLUGINS_FILE, false);
+    if (urls != null && !urls.isEmpty()) {
+      logger.info("Loading the storage plugin configs from URLs {}.", urls);
+      StoragePlugins bootstrapPlugins = new StoragePlugins(new HashMap<>());
+      Map<String, URL> pluginURLMap = new HashMap<>();
+      for (URL url : urls) {
+        String pluginsData = Resources.toString(url, Charsets.UTF_8);
+        StoragePlugins plugins = lpPersistence.getMapper().readValue(pluginsData, StoragePlugins.class);
+        for (Entry<String, StoragePluginConfig> plugin : plugins) {
+          StoragePluginConfig oldPluginConfig = bootstrapPlugins.putIfAbsent(plugin.getKey(), plugin.getValue());
+          if (oldPluginConfig != null) {
+            logger.warn("Duplicate plugin instance '{}' defined in [{}, {}], ignoring the later one.",
+                plugin.getKey(), pluginURLMap.get(plugin.getKey()), url);
+          } else {
+            pluginURLMap.put(plugin.getKey(), url);
+          }
+        }
+      }
+      return bootstrapPlugins;
+    } else {
+      throw new IOException("Failure finding " + ExecConstants.BOOTSTRAP_STORAGE_PLUGINS_FILE);
+    }
+  }
+
+  /**
+   * Dynamically loads system plugins annotated with {@link SystemPlugin}.
+   * Will skip plugin initialization if no matching constructor, incorrect class implementation, name absence are detected.
+   *
+   * @param classpathScan classpath scan result
+   * @param context drillbit context
+   * @return map with system plugins stored by name
+   */
+  private Map<String, StoragePlugin> initSystemPlugins(ScanResult classpathScan, DrillbitContext context) {
+    Map<String, StoragePlugin> plugins = CaseInsensitiveMap.newHashMap();
+    List<AnnotatedClassDescriptor> annotatedClasses = classpathScan.getAnnotatedClasses(SystemPlugin.class.getName());
+    logger.trace("Found {} annotated classes with SystemPlugin annotation: {}.", annotatedClasses.size(), annotatedClasses);
+
+    for (AnnotatedClassDescriptor annotatedClass : annotatedClasses) {
+      try {
+        Class<?> aClass = Class.forName(annotatedClass.getClassName());
+        boolean isPluginInitialized = false;
+
+        for (Constructor<?> constructor : aClass.getConstructors()) {
+          Class<?>[] parameterTypes = constructor.getParameterTypes();
+
+          if (parameterTypes.length != 1 || parameterTypes[0] != DrillbitContext.class) {
+            logger.trace("Not matching constructor for {}. Expecting constructor with one parameter for DrillbitContext class.",
+                annotatedClass.getClassName());
+            continue;
+          }
+
+          Object instance = constructor.newInstance(context);
+          if (!(instance instanceof StoragePlugin)) {
+            logger.debug("Created instance of {} does not implement StoragePlugin interface.", annotatedClass.getClassName());
+            continue;
+          }
+
+          StoragePlugin storagePlugin = (StoragePlugin) instance;
+          String name = storagePlugin.getName();
+          if (name == null) {
+            logger.debug("Storage plugin name {} is not defined. Skipping plugin initialization.", annotatedClass.getClassName());
+            continue;
+          }
+          plugins.put(name, storagePlugin);
+          isPluginInitialized = true;
+
+        }
+        if (!isPluginInitialized) {
+          logger.debug("Skipping plugin registration, did not find matching constructor or initialized object of wrong type.");
+        }
+      } catch (ReflectiveOperationException e) {
+        logger.warn("Error during system plugin {} initialization. Plugin initialization will be skipped.", annotatedClass.getClassName(), e);
+      }
+    }
+    logger.trace("The following system plugins have been initialized: {}.", plugins.keySet());
+    return plugins;
+  }
+
+  /**
+   * Get a list of all available storage plugin class constructors.
+   * @param classpathScan A classpath scan to use.
+   * @return A Map of StoragePluginConfig => StoragePlugin.<init>() constructors.
+   */
+  @SuppressWarnings("unchecked")
+  private Map<Object, Constructor<? extends StoragePlugin>> findAvailablePlugins(final ScanResult classpathScan) {
+    Map<Object, Constructor<? extends StoragePlugin>> availablePlugins = new HashMap<>();
+    final Collection<Class<? extends StoragePlugin>> pluginClasses =
+        classpathScan.getImplementations(StoragePlugin.class);
+    final String lineBrokenList =
+        pluginClasses.size() == 0
+            ? "" : "\n\t- " + Joiner.on("\n\t- ").join(pluginClasses);
+    logger.debug("Found {} storage plugin configuration classes: {}.",
+        pluginClasses.size(), lineBrokenList);
+    for (Class<? extends StoragePlugin> plugin : pluginClasses) {
+      int i = 0;
+      for (Constructor<?> c : plugin.getConstructors()) {
+        Class<?>[] params = c.getParameterTypes();
+        if (params.length != 3
+            || params[1] != DrillbitContext.class
+            || !StoragePluginConfig.class.isAssignableFrom(params[0])
+            || params[2] != String.class) {
+          logger.debug("Skipping StoragePlugin constructor {} for plugin class {} since it doesn't implement a "
+              + "[constructor(StoragePluginConfig, DrillbitContext, String)]", c, plugin);
+          continue;
+        }
+        availablePlugins.put(params[0], (Constructor<? extends StoragePlugin>) c);
+        i++;
+      }
+      if (i == 0) {
+        logger.debug("Skipping registration of StoragePlugin {} as it doesn't have a constructor with the parameters "
+            + "of (StorangePluginConfig, Config)", plugin.getCanonicalName());
+      }
+    }
+    return availablePlugins;
+  }
+
+  /**
+   * It initializes {@link #enabledPlugins} with currently enabled plugins
+   */
+  private void defineEnabledPlugins() {
+    Map<String, StoragePlugin> activePlugins = new HashMap<>();
+    Iterator<Entry<String, StoragePluginConfig>> allPlugins = pluginSystemTable.getAll();
+    while (allPlugins.hasNext()) {
+      Entry<String, StoragePluginConfig> plugin = allPlugins.next();
+      String name = plugin.getKey();
+      StoragePluginConfig config = plugin.getValue();
+      if (config.isEnabled()) {
+        try {
+          StoragePlugin storagePlugin = create(name, config);
+          activePlugins.put(name, storagePlugin);
+        } catch (ExecutionSetupException e) {
+          logger.error("Failure while setting up StoragePlugin with name: '{}', disabling.", name, e);
+          config.setEnabled(false);
+          pluginSystemTable.put(name, config);
+        }
+      }
+    }
+
+    activePlugins.putAll(systemPlugins);
+    enabledPlugins.putAll(activePlugins);
+  }
+
   private StoragePlugin create(String name, StoragePluginConfig pluginConfig) throws ExecutionSetupException {
     // TODO: DRILL-6412: clients for storage plugins shouldn't be created, if storage plugin is disabled
     // Creating of the StoragePlugin leads to instantiating storage clients
@@ -343,30 +465,33 @@ public class StoragePluginRegistryImpl implements StoragePluginRegistry {
       plugin = c.newInstance(pluginConfig, context, name);
       plugin.start();
       return plugin;
-    } catch (InstantiationException | IllegalAccessException | IllegalArgumentException | InvocationTargetException
-        | IOException e) {
+    } catch (ReflectiveOperationException | IOException e) {
       Throwable t = e instanceof InvocationTargetException ? ((InvocationTargetException) e).getTargetException() : e;
       if (t instanceof ExecutionSetupException) {
         throw ((ExecutionSetupException) t);
       }
-      throw new ExecutionSetupException(String.format(
-          "Failure setting up new storage plugin configuration for config %s", pluginConfig), t);
+      throw new ExecutionSetupException(String.format("Failure setting up new storage plugin configuration for config %s", pluginConfig), t);
     }
   }
 
-  @Override
-  public Iterator<Entry<String, StoragePlugin>> iterator() {
-    return enabledPlugins.iterator();
-  }
+  private void closePlugin(StoragePlugin plugin) {
+    if (plugin == null) {
+      return;
+    }
 
-  @Override
-  public SchemaFactory getSchemaFactory() {
-    return schemaFactory;
+    try {
+      plugin.close();
+    } catch (Exception e) {
+      logger.warn("Exception while shutting down storage plugin.");
+    }
   }
 
-  public class DrillSchemaFactory implements SchemaFactory {
+  public class DrillSchemaFactory extends AbstractSchemaFactory {
+
+    public DrillSchemaFactory(String name) {
+      super(name);
+    }
 
-    @SuppressWarnings("resource")
     @Override
     public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException {
       Stopwatch watch = Stopwatch.createStarted();
@@ -385,7 +510,7 @@ public class StoragePluginRegistryImpl implements StoragePluginRegistry {
         }
         // remove those which are no longer in the registry
         for (String pluginName : currentPluginNames) {
-          if (pluginName.equals(SYS_PLUGIN) || pluginName.equals(INFORMATION_SCHEMA_PLUGIN)) {
+          if (systemPlugins.get(pluginName) != null) {
             continue;
           }
           enabledPlugins.remove(pluginName);
@@ -448,49 +573,4 @@ public class StoragePluginRegistryImpl implements StoragePluginRegistry {
 
   }
 
-  @Override
-  public synchronized void close() throws Exception {
-    ephemeralPlugins.invalidateAll();
-    enabledPlugins.close();
-    pluginSystemTable.close();
-  }
-
-  /**
-   * Get a list of all available storage plugin class constructors.
-   * @param classpathScan A classpath scan to use.
-   * @return A Map of StoragePluginConfig => StoragePlugin.<init>() constructors.
-   */
-  @SuppressWarnings("unchecked")
-  public static Map<Object, Constructor<? extends StoragePlugin>> findAvailablePlugins(final ScanResult classpathScan) {
-    Map<Object, Constructor<? extends StoragePlugin>> availablePlugins = new HashMap<>();
-    final Collection<Class<? extends StoragePlugin>> pluginClasses =
-        classpathScan.getImplementations(StoragePlugin.class);
-    final String lineBrokenList =
-        pluginClasses.size() == 0
-            ? "" : "\n\t- " + Joiner.on("\n\t- ").join(pluginClasses);
-    logger.debug("Found {} storage plugin configuration classes: {}.",
-        pluginClasses.size(), lineBrokenList);
-    for (Class<? extends StoragePlugin> plugin : pluginClasses) {
-      int i = 0;
-      for (Constructor<?> c : plugin.getConstructors()) {
-        Class<?>[] params = c.getParameterTypes();
-        if (params.length != 3
-            || params[1] != DrillbitContext.class
-            || !StoragePluginConfig.class.isAssignableFrom(params[0])
-            || params[2] != String.class) {
-          logger.info("Skipping StoragePlugin constructor {} for plugin class {} since it doesn't implement a "
-              + "[constructor(StoragePluginConfig, DrillbitContext, String)]", c, plugin);
-          continue;
-        }
-        availablePlugins.put(params[0], (Constructor<? extends StoragePlugin>) c);
-        i++;
-      }
-      if (i == 0) {
-        logger.debug("Skipping registration of StoragePlugin {} as it doesn't have a constructor with the parameters "
-            + "of (StorangePluginConfig, Config)", plugin.getCanonicalName());
-      }
-    }
-    return availablePlugins;
-  }
-
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SystemPlugin.java
similarity index 63%
copy from exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConfig.java
copy to exec/java-exec/src/main/java/org/apache/drill/exec/store/SystemPlugin.java
index adf15a2..85236af 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SystemPlugin.java
@@ -15,23 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.store.ischema;
+package org.apache.drill.exec.store;
 
-import org.apache.drill.common.logical.StoragePluginConfig;
-
-public class InfoSchemaConfig extends StoragePluginConfig {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(InfoSchemaConfig.class);
-
-  public static final String NAME = "ischema";
-
-  @Override
-  public int hashCode(){
-    return 1;
-  }
-
-  @Override
-  public boolean equals(Object o){
-    return o instanceof InfoSchemaConfig;
-  }
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
 
+/**
+ * Is used to indicated system plugins which will be dynamically initialized during storage plugin registry init stage.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.TYPE})
+public @interface SystemPlugin {
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemConfig.java
index 4eda955..be944a0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemConfig.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.store.dfs;
 
 import java.util.Map;
+import java.util.Optional;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -25,6 +26,7 @@ import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
 
 import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.drill.common.map.CaseInsensitiveMap;
 
 @JsonTypeName(FileSystemConfig.NAME)
 public class FileSystemConfig extends StoragePluginConfig {
@@ -43,7 +45,9 @@ public class FileSystemConfig extends StoragePluginConfig {
                           @JsonProperty("formats") Map<String, FormatPluginConfig> formats) {
     this.connection = connection;
     this.config = config;
-    this.workspaces = workspaces;
+    Map<String, WorkspaceConfig> caseInsensitiveWorkspaces = CaseInsensitiveMap.newHashMap();
+    Optional.ofNullable(workspaces).ifPresent(caseInsensitiveWorkspaces::putAll);
+    this.workspaces = caseInsensitiveWorkspaces;
     this.formats = formats;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
index cb66913..ed66366 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
@@ -57,6 +57,8 @@ import com.google.common.collect.ImmutableSet.Builder;
  */
 public class FileSystemPlugin extends AbstractStoragePlugin {
 
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FileSystemPlugin.class);
+
   private final FileSystemSchemaFactory schemaFactory;
   private final FormatCreator formatCreator;
   private final Map<FormatPluginConfig, FormatPlugin> formatPluginsByConfig;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
index 0d7dce4..1a97e60 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
@@ -19,6 +19,8 @@ package org.apache.drill.exec.store.dfs;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -27,54 +29,50 @@ import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
 
+import org.apache.drill.exec.store.AbstractSchemaFactory;
 import org.apache.drill.exec.store.StorageStrategy;
 import org.apache.drill.exec.planner.logical.CreateTableEntry;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.PartitionNotFoundException;
 import org.apache.drill.exec.store.SchemaConfig;
-import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.drill.exec.store.dfs.WorkspaceSchemaFactory.WorkspaceSchema;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Maps;
 import org.apache.drill.exec.util.DrillFileSystemUtil;
 import org.apache.drill.exec.util.ImpersonationUtil;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 
-
 /**
  * This is the top level schema that responds to root level path requests. Also supports
  */
-public class FileSystemSchemaFactory implements SchemaFactory{
+public class FileSystemSchemaFactory extends AbstractSchemaFactory {
 
   public static final String DEFAULT_WS_NAME = "default";
 
   public static final String LOCAL_FS_SCHEME = "file";
 
   private List<WorkspaceSchemaFactory> factories;
-  private String schemaName;
   protected FileSystemPlugin plugin;
 
   public FileSystemSchemaFactory(String schemaName, List<WorkspaceSchemaFactory> factories) {
+    super(schemaName);
     // when the correspondent FileSystemPlugin is not passed in, we dig into ANY workspace factory to get it.
     if (factories.size() > 0) {
       this.plugin = factories.get(0).getPlugin();
     }
-    this.schemaName = schemaName;
     this.factories = factories;
   }
 
   public FileSystemSchemaFactory(FileSystemPlugin plugin, String schemaName, List<WorkspaceSchemaFactory> factories) {
+    super(schemaName);
     this.plugin = plugin;
-    this.schemaName = schemaName;
     this.factories = factories;
   }
 
   @Override
   public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException {
     @SuppressWarnings("resource")
-    FileSystemSchema schema = new FileSystemSchema(schemaName, schemaConfig);
+    FileSystemSchema schema = new FileSystemSchema(getName(), schemaConfig);
     SchemaPlus plusOfThis = parent.add(schema.getName(), schema);
     schema.setPlus(plusOfThis);
   }
@@ -82,10 +80,10 @@ public class FileSystemSchemaFactory implements SchemaFactory{
   public class FileSystemSchema extends AbstractSchema {
 
     private final WorkspaceSchema defaultSchema;
-    private final Map<String, WorkspaceSchema> schemaMap = Maps.newHashMap();
+    private final Map<String, WorkspaceSchema> schemaMap = new HashMap<>();
 
     public FileSystemSchema(String name, SchemaConfig schemaConfig) throws IOException {
-      super(ImmutableList.<String>of(), name);
+      super(Collections.emptyList(), name);
       final DrillFileSystem fs = ImpersonationUtil.createFileSystem(schemaConfig.getUserName(), plugin.getFsConf());
       for(WorkspaceSchemaFactory f :  factories){
         WorkspaceSchema s = f.createSchema(getSchemaPath(), schemaConfig, fs);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConfig.java
index adf15a2..317e608 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConfig.java
@@ -20,10 +20,11 @@ package org.apache.drill.exec.store.ischema;
 import org.apache.drill.common.logical.StoragePluginConfig;
 
 public class InfoSchemaConfig extends StoragePluginConfig {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(InfoSchemaConfig.class);
 
   public static final String NAME = "ischema";
 
+  public static final InfoSchemaConfig INSTANCE = new InfoSchemaConfig();
+
   @Override
   public int hashCode(){
     return 1;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConstants.java
index 15bdcd9..63e19d3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaConstants.java
@@ -29,7 +29,7 @@ public interface InfoSchemaConstants {
    String IS_CATALOG_CONNECT = "";
 
   /** Name of information schema. */
-   String IS_SCHEMA_NAME = "INFORMATION_SCHEMA";
+   String IS_SCHEMA_NAME = "information_schema";
 
   // CATALOGS column names:
    String CATS_COL_CATALOG_CONNECT = "CATALOG_CONNECT";
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
index fc1f01a..4005a40 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
@@ -17,35 +17,41 @@
  */
 package org.apache.drill.exec.store.ischema;
 
-import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-
 import com.google.common.collect.ImmutableSet;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
 
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.IS_SCHEMA_NAME;
+
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.map.CaseInsensitiveMap;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Maps;
 import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.drill.exec.store.StoragePluginOptimizerRule;
+import org.apache.drill.exec.store.SystemPlugin;
 
+@SystemPlugin
 public class InfoSchemaStoragePlugin extends AbstractStoragePlugin {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(InfoSchemaStoragePlugin.class);
 
   private final InfoSchemaConfig config;
 
+  @SuppressWarnings("unused") // used in StoragePluginRegistryImpl to dynamically init system plugins
+  public InfoSchemaStoragePlugin(DrillbitContext context) {
+    this(InfoSchemaConfig.INSTANCE, context, InfoSchemaConstants.IS_SCHEMA_NAME);
+  }
+
   public InfoSchemaStoragePlugin(InfoSchemaConfig config, DrillbitContext context, String name){
     super(context, name);
     this.config = config;
@@ -57,8 +63,7 @@ public class InfoSchemaStoragePlugin extends AbstractStoragePlugin {
   }
 
   @Override
-  public InfoSchemaGroupScan getPhysicalScan(String userName, JSONOptions selection, List<SchemaPath> columns)
-      throws IOException {
+  public InfoSchemaGroupScan getPhysicalScan(String userName, JSONOptions selection, List<SchemaPath> columns) {
     InfoSchemaTableType table = selection.getWith(getContext().getLpPersistence(),  InfoSchemaTableType.class);
     return new InfoSchemaGroupScan(table);
   }
@@ -69,23 +74,35 @@ public class InfoSchemaStoragePlugin extends AbstractStoragePlugin {
   }
 
   @Override
-  public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException {
-    ISchema s = new ISchema(parent, this);
+  public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) {
+    ISchema s = new ISchema(this);
     parent.add(s.getName(), s);
   }
 
   /**
    * Representation of the INFORMATION_SCHEMA schema.
    */
-  private class ISchema extends AbstractSchema{
-    private Map<String, InfoSchemaDrillTable> tables;
-    public ISchema(SchemaPlus parent, InfoSchemaStoragePlugin plugin){
-      super(ImmutableList.<String>of(), IS_SCHEMA_NAME);
-      Map<String, InfoSchemaDrillTable> tbls = Maps.newHashMap();
-      for(InfoSchemaTableType tbl : InfoSchemaTableType.values()){
-        tbls.put(tbl.name(), new InfoSchemaDrillTable(plugin, IS_SCHEMA_NAME, tbl, config));
-      }
-      this.tables = ImmutableMap.copyOf(tbls);
+  private class ISchema extends AbstractSchema {
+
+    private final Map<String, InfoSchemaDrillTable> tables;
+    // for backward compatibility keep IS schema table names in upper case
+    // the way they used to appear in INFORMATION_SCHEMA.TABLES table
+    // though user can query them in any case
+    private final Set<String> originalTableNames;
+
+    ISchema(InfoSchemaStoragePlugin plugin) {
+
+      super(Collections.emptyList(), IS_SCHEMA_NAME);
+
+      this.tables = CaseInsensitiveMap.newHashMap();
+      this.originalTableNames = new HashSet<>();
+
+      Arrays.stream(InfoSchemaTableType.values()).forEach(
+          table -> {
+            tables.put(table.name(), new InfoSchemaDrillTable(plugin, getName(), table, config));
+            originalTableNames.add(table.name());
+          }
+      );
     }
 
     @Override
@@ -95,13 +112,18 @@ public class InfoSchemaStoragePlugin extends AbstractStoragePlugin {
 
     @Override
     public Set<String> getTableNames() {
-      return tables.keySet();
+      return originalTableNames;
     }
 
     @Override
     public String getTypeName() {
       return InfoSchemaConfig.NAME;
     }
+
+    @Override
+    public boolean areTableNamesCaseSensitive() {
+      return false;
+    }
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/CaseInsensitivePersistentStore.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/CaseInsensitivePersistentStore.java
new file mode 100644
index 0000000..38bd529
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/CaseInsensitivePersistentStore.java
@@ -0,0 +1,79 @@
+/*
+ * 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.store.sys;
+
+import java.util.Iterator;
+import java.util.Map;
+
+/**
+ * Wrapper around {@link PersistentStore} to ensure all passed keys are converted to lower case and stored this way.
+ * This will ensure case-insensitivity during insert, update, deletion or search.
+ */
+public class CaseInsensitivePersistentStore<V> implements PersistentStore<V> {
+
+  private final PersistentStore<V> underlyingStore;
+
+  public CaseInsensitivePersistentStore(PersistentStore<V> underlyingStore) {
+    this.underlyingStore = underlyingStore;
+  }
+
+  @Override
+  public boolean contains(String key) {
+    return underlyingStore.contains(key.toLowerCase());
+  }
+
+  @Override
+  public V get(String key) {
+    return underlyingStore.get(key.toLowerCase());
+  }
+
+  @Override
+  public void put(String key, V value) {
+    underlyingStore.put(key.toLowerCase(), value);
+  }
+
+  @Override
+  public Iterator<Map.Entry<String, V>> getAll() {
+    return underlyingStore.getAll();
+  }
+
+  @Override
+  public PersistentStoreMode getMode() {
+    return underlyingStore.getMode();
+  }
+
+  @Override
+  public void delete(String key) {
+    underlyingStore.delete(key.toLowerCase());
+  }
+
+  @Override
+  public boolean putIfAbsent(String key, V value) {
+    return underlyingStore.putIfAbsent(key.toLowerCase(), value);
+  }
+
+  @Override
+  public Iterator<Map.Entry<String, V>> getRange(int skip, int take) {
+    return underlyingStore.getRange(skip, take);
+  }
+
+  @Override
+  public void close() throws Exception {
+    underlyingStore.close();
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTablePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTablePlugin.java
index 10e082f..d282017 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTablePlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTablePlugin.java
@@ -17,39 +17,47 @@
  */
 package org.apache.drill.exec.store.sys;
 
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.map.CaseInsensitiveMap;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.SchemaConfig;
+import org.apache.drill.exec.store.SystemPlugin;
 import org.apache.drill.exec.store.pojo.PojoDataType;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
-
 /**
  * A "storage" plugin for system tables.
  */
+@SystemPlugin
 public class SystemTablePlugin extends AbstractStoragePlugin {
-  // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SystemTablePlugin.class);
 
   public static final String SYS_SCHEMA_NAME = "sys";
 
   private final SystemTablePluginConfig config;
-  private final SystemSchema schema = new SystemSchema();
+  private final SystemSchema schema;
+
+  @SuppressWarnings("unused") // used in StoragePluginRegistryImpl to dynamically init system plugins
+  public SystemTablePlugin(DrillbitContext context) {
+    this(SystemTablePluginConfig.INSTANCE, context, SYS_SCHEMA_NAME);
+  }
 
   public SystemTablePlugin(SystemTablePluginConfig config, DrillbitContext context, String name) {
     super(context, name);
     this.config = config;
+    this.schema = new SystemSchema(this);
   }
 
   @Override
@@ -73,31 +81,29 @@ public class SystemTablePlugin extends AbstractStoragePlugin {
    */
   private class SystemSchema extends AbstractSchema {
 
-    private final Set<String> tableNames;
+    private final Map<String, StaticDrillTable> tables;
+
+    SystemSchema(SystemTablePlugin plugin) {
 
-    public SystemSchema() {
-      super(ImmutableList.of(), SYS_SCHEMA_NAME);
-      Set<String> names = Sets.newHashSet();
-      for (SystemTable t : SystemTable.values()) {
-        names.add(t.getTableName());
-      }
-      this.tableNames = ImmutableSet.copyOf(names);
+      super(Collections.emptyList(), SYS_SCHEMA_NAME);
+
+      this.tables = Arrays.stream(SystemTable.values())
+          .collect(
+              Collectors.toMap(
+                  SystemTable::getTableName,
+                  table -> new StaticDrillTable(getName(), plugin, TableType.SYSTEM_TABLE, table, new PojoDataType(table.getPojoClass())),
+                  (o, n) -> n,
+                  CaseInsensitiveMap::newHashMap));
     }
 
     @Override
     public Set<String> getTableNames() {
-      return tableNames;
+      return tables.keySet();
     }
 
     @Override
     public DrillTable getTable(String name) {
-      for (SystemTable table : SystemTable.values()) {
-        if (table.getTableName().equalsIgnoreCase(name)) {
-          return new StaticDrillTable(getName(), SystemTablePlugin.this, TableType.SYSTEM_TABLE,
-            table, new PojoDataType(table.getPojoClass()));
-        }
-      }
-      return null;
+      return tables.get(name);
     }
 
     @Override
@@ -105,5 +111,9 @@ public class SystemTablePlugin extends AbstractStoragePlugin {
       return SystemTablePluginConfig.NAME;
     }
 
+    @Override
+    public boolean areTableNamesCaseSensitive() {
+      return false;
+    }
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTablePluginConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTablePluginConfig.java
index 360182e..914fcf0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTablePluginConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTablePluginConfig.java
@@ -23,7 +23,6 @@ import org.apache.drill.common.logical.StoragePluginConfig;
  * A namesake plugin configuration for system tables.
  */
 public class SystemTablePluginConfig extends StoragePluginConfig {
-//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SystemTablePluginConfig.class);
 
   public static final String NAME = "system-tables";
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java
index 0f71775..7a9c9a6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java
@@ -434,18 +434,36 @@ public class MetadataProvider {
 
   /**
    * Helper method to create a {@link InfoSchemaFilter} that combines the given filters with an AND.
+   *
    * @param catalogNameFilter Optional filter on <code>catalog name</code>
    * @param schemaNameFilter Optional filter on <code>schema name</code>
    * @param tableNameFilter Optional filter on <code>table name</code>
    * @param tableTypeFilter Optional filter on <code>table type</code>
    * @param columnNameFilter Optional filter on <code>column name</code>
-   * @return
+   * @return information schema filter
    */
-  private static InfoSchemaFilter createInfoSchemaFilter(final LikeFilter catalogNameFilter,
-      final LikeFilter schemaNameFilter, final LikeFilter tableNameFilter, List<String> tableTypeFilter, final LikeFilter columnNameFilter) {
+  private static InfoSchemaFilter createInfoSchemaFilter(LikeFilter catalogNameFilter,
+                                                         LikeFilter schemaNameFilter,
+                                                         LikeFilter tableNameFilter,
+                                                         List<String> tableTypeFilter,
+                                                         LikeFilter columnNameFilter) {
 
     FunctionExprNode exprNode = createLikeFunctionExprNode(CATS_COL_CATALOG_NAME,  catalogNameFilter);
 
+    // schema names are case insensitive in Drill and stored in lower case
+    // convert like filter condition elements to lower case
+    if (schemaNameFilter != null) {
+      LikeFilter.Builder builder = LikeFilter.newBuilder();
+      if (schemaNameFilter.hasPattern()) {
+        builder.setPattern(schemaNameFilter.getPattern().toLowerCase());
+      }
+
+      if (schemaNameFilter.hasEscape()) {
+        builder.setEscape(schemaNameFilter.getEscape().toLowerCase());
+      }
+      schemaNameFilter = builder.build();
+    }
+
     exprNode = combineFunctions(AND_FUNCTION,
         exprNode,
         combineFunctions(OR_FUNCTION,
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index e3b0604..eca59b9 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -32,16 +32,23 @@ drill {
       org.apache.drill.exec.store.StoragePlugin
     ],
 
-    annotations += org.apache.drill.exec.expr.annotations.FunctionTemplate
+    annotations: ${?drill.classpath.scanning.annotations} [
+      org.apache.drill.exec.expr.annotations.FunctionTemplate,
+      org.apache.drill.exec.store.SystemPlugin
+    ],
 
     packages: ${?drill.classpath.scanning.packages} [
-          org.apache.drill.exec.expr,
-          org.apache.drill.exec.physical,
-          org.apache.drill.exec.store,
-          org.apache.drill.exec.rpc.user.security,
-          org.apache.drill.exec.rpc.security,
-          org.apache.drill.exec.server.rest.auth
-    ]
+      org.apache.drill.exec.expr,
+      org.apache.drill.exec.physical,
+      org.apache.drill.exec.store,
+      org.apache.drill.exec.rpc.user.security,
+      org.apache.drill.exec.rpc.security,
+      org.apache.drill.exec.server.rest.auth
+    ],
+
+    // caches scanned result during build time
+    // set to false to avoid the need for a full Drill build during development
+    cache.enabled: true
   }
 }
 
@@ -231,7 +238,7 @@ drill.exec: {
   },
   debug: {
     // If true, inserts the iterator validator atop each operator.
-    // Primrily used for testing.
+    // Primarily used for testing.
     validate_iterators: false,
     // If iterator validation is enabled, also validates the vectors
     // in each batch. Primarily used for testing. To enable from
diff --git a/exec/java-exec/src/test/java/org/apache/drill/common/scanner/TestClassPathScanner.java b/exec/java-exec/src/test/java/org/apache/drill/common/scanner/TestClassPathScanner.java
index f4dc837..01b1b3b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/common/scanner/TestClassPathScanner.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/common/scanner/TestClassPathScanner.java
@@ -17,21 +17,19 @@
  */
 package org.apache.drill.common.scanner;
 
-import static java.util.Arrays.asList;
-import static java.util.Collections.sort;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.lang.annotation.Annotation;
 import java.lang.reflect.Field;
 import java.lang.reflect.Method;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.stream.Collectors;
 
 import org.apache.drill.categories.SlowTest;
 import org.apache.drill.common.config.DrillConfig;
@@ -43,26 +41,27 @@ import org.apache.drill.exec.expr.DrillFunc;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate;
 import org.apache.drill.exec.fn.impl.testing.GeneratorFunctions.RandomBigIntGauss;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.store.SystemPlugin;
+import org.apache.drill.exec.store.ischema.InfoSchemaStoragePlugin;
+import org.apache.drill.exec.store.sys.SystemTablePlugin;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({SlowTest.class})
 public class TestClassPathScanner {
 
-  @SafeVarargs
-  final private <T extends Comparable<? super T>> void assertListEqualsUnordered(Collection<T> list, T... expected) {
-    List<T> expectedList = asList(expected);
-    sort(expectedList);
-    List<T> gotList = new ArrayList<>(list);
-    sort(gotList);
-    assertEquals(expectedList.toString(), gotList.toString());
+  private static ScanResult result;
+
+  @BeforeClass
+  public static void init() {
+    result = ClassPathScanner.fromPrescan(DrillConfig.create());
   }
 
   @Test
-  public void test() throws Exception {
-    ScanResult result = ClassPathScanner.fromPrescan(DrillConfig.create());
-    List<AnnotatedClassDescriptor> functions = result.getAnnotatedClasses();
+  public void testFunctionTemplates() throws Exception {
+    List<AnnotatedClassDescriptor> functions = result.getAnnotatedClasses(FunctionTemplate.class.getName());
     Set<String> scanned = new HashSet<>();
     AnnotatedClassDescriptor functionRandomBigIntGauss = null;
     for (AnnotatedClassDescriptor function : functions) {
@@ -94,6 +93,7 @@ public class TestClassPathScanner {
       List<AnnotationDescriptor> scannedAnnotations = function.getAnnotations();
       verifyAnnotations(annotations, scannedAnnotations);
       FunctionTemplate bytecodeAnnotation = function.getAnnotationProxy(FunctionTemplate.class);
+      assertNotNull(bytecodeAnnotation);
       FunctionTemplate reflectionAnnotation = c.getAnnotation(FunctionTemplate.class);
       assertEquals(reflectionAnnotation.name(), bytecodeAnnotation.name());
       assertArrayEquals(reflectionAnnotation.names(), bytecodeAnnotation.names());
@@ -110,6 +110,17 @@ public class TestClassPathScanner {
     assertTrue(result.getImplementations(DrillFunc.class).size() > 0);
   }
 
+  @Test
+  public void testSystemPlugins() {
+    List<AnnotatedClassDescriptor> annotatedClasses = result.getAnnotatedClasses(SystemPlugin.class.getName());
+    List<AnnotatedClassDescriptor> foundPlugins =
+        annotatedClasses.stream()
+            .filter(a -> InfoSchemaStoragePlugin.class.getName().equals(a.getClassName())
+                || SystemTablePlugin.class.getName().equals(a.getClassName()))
+            .collect(Collectors.toList());
+    assertEquals(2, foundPlugins.size());
+  }
+
   private <T> void validateType(ScanResult result, String baseType) throws ClassNotFoundException {
     if (baseType.startsWith("org.apache.hadoop.hive")) {
       return;
@@ -132,8 +143,8 @@ public class TestClassPathScanner {
       Class<? extends Annotation> annotationType = annotation.annotationType();
       assertEquals(annotationType.getName(), scannedAnnotation.getAnnotationType());
       if (annotation instanceof FunctionTemplate) {
-        FunctionTemplate ft = (FunctionTemplate)annotation;
-        if (ft.name() != null && !ft.name().equals("")) {
+        FunctionTemplate ft = (FunctionTemplate) annotation;
+        if (!"".equals(ft.name())) {
           assertEquals(ft.name(), scannedAnnotation.getSingleValue("name"));
         }
       }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/coord/zk/TestZookeeperClient.java b/exec/java-exec/src/test/java/org/apache/drill/exec/coord/zk/TestZookeeperClient.java
index e0e6c79..aaa5ee9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/coord/zk/TestZookeeperClient.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/coord/zk/TestZookeeperClient.java
@@ -57,10 +57,10 @@ public class TestZookeeperClient {
   private CuratorFramework curator;
   private ZookeeperClient client;
 
-  static class ClientWithMockCache extends ZookeeperClient {
+  private static class ClientWithMockCache extends ZookeeperClient {
     private final PathChildrenCache cacheMock = Mockito.mock(PathChildrenCache.class);
 
-    public ClientWithMockCache(final CuratorFramework curator, final String root, final CreateMode mode) {
+    ClientWithMockCache(final CuratorFramework curator, final String root, final CreateMode mode) {
       super(curator, root, mode);
     }
 
@@ -97,7 +97,7 @@ public class TestZookeeperClient {
 
     Mockito
         .verify(client.getCache())
-        .start();
+        .start(PathChildrenCache.StartMode.BUILD_INITIAL_CACHE);
   }
 
   @Test
@@ -163,7 +163,7 @@ public class TestZookeeperClient {
         .when(client.getCache().getCurrentData(abspath))
         .thenReturn(null);
 
-    assertEquals("get should return null", null, client.get(path));
+    assertNull("get should return null", client.get(path));
 
     Mockito
         .when(client.getCache().getCurrentData(abspath))
@@ -198,7 +198,7 @@ public class TestZookeeperClient {
 
 
   @Test
-  public void testEntriesReturnsRelativePaths() throws Exception {
+  public void testEntriesReturnsRelativePaths() {
     final ChildData child = Mockito.mock(ChildData.class);
     Mockito
         .when(child.getPath())
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/BaseTestImpersonation.java b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/BaseTestImpersonation.java
index 3b25ddb..17756a8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/BaseTestImpersonation.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/BaseTestImpersonation.java
@@ -44,7 +44,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 public class BaseTestImpersonation extends PlanTestBase {
-  protected static final String MINIDFS_STORAGE_PLUGIN_NAME = "miniDfsPlugin";
+  protected static final String MINI_DFS_STORAGE_PLUGIN_NAME = "mini_dfs_plugin";
   protected static final String processUser = System.getProperty("user.name");
 
   protected static MiniDFSCluster dfsCluster;
@@ -74,8 +74,8 @@ public class BaseTestImpersonation extends PlanTestBase {
 
   /**
    * Start a MiniDFS cluster backed Drillbit cluster with impersonation enabled.
-   * @param testClass
-   * @throws Exception
+   * @param testClass test class
+   * @throws Exception in case of errors during start up
    */
   protected static void startMiniDfsCluster(String testClass) throws Exception {
     startMiniDfsCluster(testClass, true);
@@ -83,12 +83,11 @@ public class BaseTestImpersonation extends PlanTestBase {
 
   /**
    * Start a MiniDFS cluster backed Drillbit cluster
-   * @param testClass
+   * @param testClass test class
    * @param isImpersonationEnabled Enable impersonation in the cluster?
-   * @throws Exception
+   * @throws Exception in case of errors during start up
    */
-  protected static void startMiniDfsCluster(
-      final String testClass, final boolean isImpersonationEnabled) throws Exception {
+  protected static void startMiniDfsCluster(String testClass, boolean isImpersonationEnabled) throws Exception {
     Preconditions.checkArgument(!Strings.isNullOrEmpty(testClass), "Expected a non-null and non-empty test class name");
     dfsConf = new Configuration();
 
@@ -133,7 +132,7 @@ public class BaseTestImpersonation extends PlanTestBase {
 
     FileSystemConfig miniDfsPluginConfig = new FileSystemConfig(connection, null, workspaces, lfsPluginConfig.getFormats());
     miniDfsPluginConfig.setEnabled(true);
-    pluginRegistry.createOrUpdate(MINIDFS_STORAGE_PLUGIN_NAME, miniDfsPluginConfig, true);
+    pluginRegistry.createOrUpdate(MINI_DFS_STORAGE_PLUGIN_NAME, miniDfsPluginConfig, true);
   }
 
   protected static void createAndAddWorkspace(String name, String path, short permissions, String owner,
@@ -168,7 +167,7 @@ public class BaseTestImpersonation extends PlanTestBase {
 
   // Return the user workspace for given user.
   protected static String getWSSchema(String user) {
-    return MINIDFS_STORAGE_PLUGIN_NAME + "." + user;
+    return MINI_DFS_STORAGE_PLUGIN_NAME + "." + user;
   }
 
   protected static String getUserHome(String user) {
@@ -194,7 +193,7 @@ public class BaseTestImpersonation extends PlanTestBase {
                                  final String viewDef) throws Exception {
     updateClient(viewOwner);
     test(String.format("ALTER SESSION SET `%s`='%o';", ExecConstants.NEW_VIEW_DEFAULT_PERMS_KEY, (short) 0750));
-    test("CREATE VIEW %s.%s.%s AS %s", MINIDFS_STORAGE_PLUGIN_NAME, "tmp", viewName, viewDef);
+    test("CREATE VIEW %s.%s.%s AS %s", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", viewName, viewDef);
     final Path viewFilePath = new Path("/tmp/", viewName + DotDrillType.VIEW.getEnding());
     fs.setOwner(viewFilePath, viewOwner, viewGroup);
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationDisabledWithMiniDFS.java b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationDisabledWithMiniDFS.java
index cebf649..700d820 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationDisabledWithMiniDFS.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationDisabledWithMiniDFS.java
@@ -45,8 +45,7 @@ public class TestImpersonationDisabledWithMiniDFS extends BaseTestImpersonation
 
   private static void createTestData() throws Exception {
     // Create test table in minidfs.tmp schema for use in test queries
-    test(String.format("CREATE TABLE %s.tmp.dfsRegion AS SELECT * FROM cp.`region.json`",
-        MINIDFS_STORAGE_PLUGIN_NAME));
+    test(String.format("CREATE TABLE %s.tmp.dfsRegion AS SELECT * FROM cp.`region.json`", MINI_DFS_STORAGE_PLUGIN_NAME));
 
     // generate a large enough file that the DFS will not fulfill requests to read a
     // page of data all at once, see notes above testReadLargeParquetFileFromDFS()
@@ -59,8 +58,7 @@ public class TestImpersonationDisabledWithMiniDFS extends BaseTestImpersonation
             "UNION ALL (SELECT employee_id, full_name FROM cp.`employee.json`)" +
             "UNION ALL (SELECT employee_id, full_name FROM cp.`employee.json`)" +
             "UNION ALL (SELECT employee_id, full_name FROM cp.`employee.json`)" +
-        "UNION ALL (SELECT employee_id, full_name FROM cp.`employee.json`)",
-        MINIDFS_STORAGE_PLUGIN_NAME));
+        "UNION ALL (SELECT employee_id, full_name FROM cp.`employee.json`)", MINI_DFS_STORAGE_PLUGIN_NAME));
   }
 
   /**
@@ -77,7 +75,7 @@ public class TestImpersonationDisabledWithMiniDFS extends BaseTestImpersonation
    */
   @Test
   public void testReadLargeParquetFileFromDFS() throws Exception {
-    test(String.format("USE %s", MINIDFS_STORAGE_PLUGIN_NAME));
+    test(String.format("USE %s", MINI_DFS_STORAGE_PLUGIN_NAME));
     test("SELECT * FROM tmp.`large_employee`");
   }
 
@@ -87,7 +85,7 @@ public class TestImpersonationDisabledWithMiniDFS extends BaseTestImpersonation
         String.format("SELECT sales_city, sales_country FROM tmp.dfsRegion ORDER BY region_id DESC LIMIT 2");
 
     testBuilder()
-        .optionSettingQueriesForTestQuery(String.format("USE %s", MINIDFS_STORAGE_PLUGIN_NAME))
+        .optionSettingQueriesForTestQuery(String.format("USE %s", MINI_DFS_STORAGE_PLUGIN_NAME))
         .sqlQuery(query)
         .unOrdered()
         .baselineColumns("sales_city", "sales_country")
@@ -98,7 +96,7 @@ public class TestImpersonationDisabledWithMiniDFS extends BaseTestImpersonation
 
   @AfterClass
   public static void removeMiniDfsBasedStorage() throws Exception {
-    getDrillbitContext().getStorage().deletePlugin(MINIDFS_STORAGE_PLUGIN_NAME);
+    getDrillbitContext().getStorage().deletePlugin(MINI_DFS_STORAGE_PLUGIN_NAME);
     stopMiniDfsCluster();
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationMetadata.java b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationMetadata.java
index 08c09d1..d14c7ad 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationMetadata.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationMetadata.java
@@ -52,8 +52,8 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
   private static final String user1 = "drillTestUser1";
   private static final String user2 = "drillTestUser2";
 
-  private static final String group0 = "drillTestGrp0";
-  private static final String group1 = "drillTestGrp1";
+  private static final String group0 = "drill_test_grp_0";
+  private static final String group1 = "drill_test_grp_1";
 
   static {
     UserGroupInformation.createUserForTesting(user1, new String[]{ group1, group0 });
@@ -75,23 +75,23 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
 
     Map<String, WorkspaceConfig> workspaces = Maps.newHashMap();
 
-    // Create /drillTestGrp0_700 directory with permissions 700 (owned by user running the tests)
-    createAndAddWorkspace("drillTestGrp0_700", "/drillTestGrp0_700", (short)0700, processUser, group0, workspaces);
+    // Create /drill_test_grp_0_700 directory with permissions 700 (owned by user running the tests)
+    createAndAddWorkspace("drill_test_grp_0_700", "/drill_test_grp_0_700", (short)0700, processUser, group0, workspaces);
 
-    // Create /drillTestGrp0_750 directory with permissions 750 (owned by user running the tests)
-    createAndAddWorkspace("drillTestGrp0_750", "/drillTestGrp0_750", (short)0750, processUser, group0, workspaces);
+    // Create /drill_test_grp_0_750 directory with permissions 750 (owned by user running the tests)
+    createAndAddWorkspace("drill_test_grp_0_750", "/drill_test_grp_0_750", (short)0750, processUser, group0, workspaces);
 
-    // Create /drillTestGrp0_755 directory with permissions 755 (owned by user running the tests)
-    createAndAddWorkspace("drillTestGrp0_755", "/drillTestGrp0_755", (short)0755, processUser, group0, workspaces);
+    // Create /drill_test_grp_0_755 directory with permissions 755 (owned by user running the tests)
+    createAndAddWorkspace("drill_test_grp_0_755", "/drill_test_grp_0_755", (short)0755, processUser, group0, workspaces);
 
-    // Create /drillTestGrp0_770 directory with permissions 770 (owned by user running the tests)
-    createAndAddWorkspace("drillTestGrp0_770", "/drillTestGrp0_770", (short)0770, processUser, group0, workspaces);
+    // Create /drill_test_grp_0_770 directory with permissions 770 (owned by user running the tests)
+    createAndAddWorkspace("drill_test_grp_0_770", "/drill_test_grp_0_770", (short)0770, processUser, group0, workspaces);
 
-    // Create /drillTestGrp0_777 directory with permissions 777 (owned by user running the tests)
-    createAndAddWorkspace("drillTestGrp0_777", "/drillTestGrp0_777", (short)0777, processUser, group0, workspaces);
+    // Create /drill_test_grp_0_777 directory with permissions 777 (owned by user running the tests)
+    createAndAddWorkspace("drill_test_grp_0_777", "/drill_test_grp_0_777", (short)0777, processUser, group0, workspaces);
 
-    // Create /drillTestGrp1_700 directory with permissions 700 (owned by user1)
-    createAndAddWorkspace("drillTestGrp1_700", "/drillTestGrp1_700", (short)0700, user1, group1, workspaces);
+    // Create /drill_test_grp_1_700 directory with permissions 700 (owned by user1)
+    createAndAddWorkspace("drill_test_grp_1_700", "/drill_test_grp_1_700", (short)0700, user1, group1, workspaces);
 
     // create /user2_workspace1 with 775 permissions (owner by user1)
     createAndAddWorkspace("user2_workspace1", "/user2_workspace1", (short)0775, user2, group1, workspaces);
@@ -107,17 +107,17 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
 
     // create tables as user2
     updateClient(user2);
-    test("use `%s.user2_workspace1`", MINIDFS_STORAGE_PLUGIN_NAME);
+    test("use `%s.user2_workspace1`", MINI_DFS_STORAGE_PLUGIN_NAME);
     // create a table that can be dropped by another user in a different group
     test("create table parquet_table_775 as select * from cp.`employee.json`");
 
     // create a table that cannot be dropped by another user
-    test("use `%s.user2_workspace2`", MINIDFS_STORAGE_PLUGIN_NAME);
+    test("use `%s.user2_workspace2`", MINI_DFS_STORAGE_PLUGIN_NAME);
     test("create table parquet_table_700 as select * from cp.`employee.json`");
 
     // Drop tables as user1
     updateClient(user1);
-    test("use `%s.user2_workspace1`", MINIDFS_STORAGE_PLUGIN_NAME);
+    test("use `%s.user2_workspace1`", MINI_DFS_STORAGE_PLUGIN_NAME);
     testBuilder()
         .sqlQuery("drop table parquet_table_775")
         .unOrdered()
@@ -125,7 +125,7 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
         .baselineValues(true, String.format("Table [%s] dropped", "parquet_table_775"))
         .go();
 
-    test("use `%s.user2_workspace2`", MINIDFS_STORAGE_PLUGIN_NAME);
+    test("use `%s.user2_workspace2`", MINI_DFS_STORAGE_PLUGIN_NAME);
     boolean dropFailed = false;
     try {
       test("drop table parquet_table_700");
@@ -142,7 +142,7 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
     updateClient(processUser);
 
     // Process user start the mini dfs, he has read/write permissions by default
-    final String viewName = String.format("%s.drillTestGrp0_700.testView", MINIDFS_STORAGE_PLUGIN_NAME);
+    final String viewName = String.format("%s.drill_test_grp_0_700.testView", MINI_DFS_STORAGE_PLUGIN_NAME);
     try {
       test("CREATE VIEW " + viewName + " AS SELECT * FROM cp.`region.json`");
       test("SELECT * FROM " + viewName + " LIMIT 2");
@@ -155,20 +155,20 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
   public void testShowFilesInWSWithUserAndGroupPermissionsForQueryUser() throws Exception {
     updateClient(user1);
 
-    // Try show tables in schema "drillTestGrp1_700" which is owned by "user1"
-    int count = testSql(String.format("SHOW FILES IN %s.drillTestGrp1_700", MINIDFS_STORAGE_PLUGIN_NAME));
+    // Try show tables in schema "drill_test_grp_1_700" which is owned by "user1"
+    int count = testSql(String.format("SHOW FILES IN %s.drill_test_grp_1_700", MINI_DFS_STORAGE_PLUGIN_NAME));
     assertTrue(count > 0);
 
-    // Try show tables in schema "drillTestGrp0_750" which is owned by "processUser" and has group permissions for "user1"
-    count = testSql(String.format("SHOW FILES IN %s.drillTestGrp0_750", MINIDFS_STORAGE_PLUGIN_NAME));
+    // Try show tables in schema "drill_test_grp_0_750" which is owned by "processUser" and has group permissions for "user1"
+    count = testSql(String.format("SHOW FILES IN %s.drill_test_grp_0_750", MINI_DFS_STORAGE_PLUGIN_NAME));
     assertTrue(count > 0);
   }
 
   @Test
   public void testShowFilesInWSWithOtherPermissionsForQueryUser() throws Exception {
     updateClient(user2);
-    // Try show tables in schema "drillTestGrp0_755" which is owned by "processUser" and group0. "user2" is not part of the "group0"
-    int count = testSql(String.format("SHOW FILES IN %s.drillTestGrp0_755", MINIDFS_STORAGE_PLUGIN_NAME));
+    // Try show tables in schema "drill_test_grp_0_755" which is owned by "processUser" and group0. "user2" is not part of the "group0"
+    int count = testSql(String.format("SHOW FILES IN %s.drill_test_grp_0_755", MINI_DFS_STORAGE_PLUGIN_NAME));
     assertTrue(count > 0);
   }
 
@@ -178,8 +178,8 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
 
     try {
       setSessionOption(ExecConstants.LIST_FILES_RECURSIVELY, true);
-      // Try show tables in schema "drillTestGrp1_700" which is owned by "user1"
-      int count = testSql(String.format("SHOW FILES IN %s.drillTestGrp1_700", MINIDFS_STORAGE_PLUGIN_NAME));
+      // Try show tables in schema "drill_test_grp_1_700" which is owned by "user1"
+      int count = testSql(String.format("SHOW FILES IN %s.drill_test_grp_1_700", MINI_DFS_STORAGE_PLUGIN_NAME));
       assertEquals("Counts should match", 0, count);
     } finally {
       resetSessionOption(ExecConstants.LIST_FILES_RECURSIVELY);
@@ -189,53 +189,53 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
   @Test
   public void testShowSchemasAsUser1() throws Exception {
     // "user1" is part of "group0" and has access to following workspaces
-    // drillTestGrp1_700 (through ownership)
-    // drillTestGrp0_750, drillTestGrp0_770 (through "group" category permissions)
-    // drillTestGrp0_755, drillTestGrp0_777 (through "others" category permissions)
+    // drill_test_grp_1_700 (through ownership)
+    // drill_test_grp_0_750, drill_test_grp_0_770 (through "group" category permissions)
+    // drill_test_grp_0_755, drill_test_grp_0_777 (through "others" category permissions)
     updateClient(user1);
     testBuilder()
-        .sqlQuery("SHOW SCHEMAS LIKE '%drillTest%'")
+        .sqlQuery("SHOW SCHEMAS LIKE '%drill_test%'")
         .unOrdered()
         .baselineColumns("SCHEMA_NAME")
-        .baselineValues(String.format("%s.drillTestGrp0_750", MINIDFS_STORAGE_PLUGIN_NAME))
-        .baselineValues(String.format("%s.drillTestGrp0_755", MINIDFS_STORAGE_PLUGIN_NAME))
-        .baselineValues(String.format("%s.drillTestGrp0_770", MINIDFS_STORAGE_PLUGIN_NAME))
-        .baselineValues(String.format("%s.drillTestGrp0_777", MINIDFS_STORAGE_PLUGIN_NAME))
-        .baselineValues(String.format("%s.drillTestGrp1_700", MINIDFS_STORAGE_PLUGIN_NAME))
+        .baselineValues(String.format("%s.drill_test_grp_0_750", MINI_DFS_STORAGE_PLUGIN_NAME))
+        .baselineValues(String.format("%s.drill_test_grp_0_755", MINI_DFS_STORAGE_PLUGIN_NAME))
+        .baselineValues(String.format("%s.drill_test_grp_0_770", MINI_DFS_STORAGE_PLUGIN_NAME))
+        .baselineValues(String.format("%s.drill_test_grp_0_777", MINI_DFS_STORAGE_PLUGIN_NAME))
+        .baselineValues(String.format("%s.drill_test_grp_1_700", MINI_DFS_STORAGE_PLUGIN_NAME))
         .go();
   }
 
   @Test
   public void testShowSchemasAsUser2() throws Exception {
     // "user2" is part of "group0", but part of "group1" and has access to following workspaces
-    // drillTestGrp0_755, drillTestGrp0_777 (through "others" category permissions)
+    // drill_test_grp_0_755, drill_test_grp_0_777 (through "others" category permissions)
     updateClient(user2);
     testBuilder()
-        .sqlQuery("SHOW SCHEMAS LIKE '%drillTest%'")
+        .sqlQuery("SHOW SCHEMAS LIKE '%drill_test%'")
         .unOrdered()
         .baselineColumns("SCHEMA_NAME")
-        .baselineValues(String.format("%s.drillTestGrp0_755", MINIDFS_STORAGE_PLUGIN_NAME))
-        .baselineValues(String.format("%s.drillTestGrp0_777", MINIDFS_STORAGE_PLUGIN_NAME))
+        .baselineValues(String.format("%s.drill_test_grp_0_755", MINI_DFS_STORAGE_PLUGIN_NAME))
+        .baselineValues(String.format("%s.drill_test_grp_0_777", MINI_DFS_STORAGE_PLUGIN_NAME))
         .go();
   }
 
   @Test
   public void testCreateViewInDirWithUserPermissionsForQueryUser() throws Exception {
-    final String viewSchema = MINIDFS_STORAGE_PLUGIN_NAME + ".drillTestGrp1_700"; // Workspace dir owned by "user1"
+    final String viewSchema = MINI_DFS_STORAGE_PLUGIN_NAME + ".drill_test_grp_1_700"; // Workspace dir owned by "user1"
     testCreateViewTestHelper(user1, viewSchema, "view1");
   }
 
   @Test
   public void testCreateViewInDirWithGroupPermissionsForQueryUser() throws Exception {
     // Workspace dir owned by "processUser", workspace group is "group0" and "user1" is part of "group0"
-    final String viewSchema = MINIDFS_STORAGE_PLUGIN_NAME + ".drillTestGrp0_770";
+    final String viewSchema = MINI_DFS_STORAGE_PLUGIN_NAME + ".drill_test_grp_0_770";
     testCreateViewTestHelper(user1, viewSchema, "view1");
   }
 
   @Test
   public void testCreateViewInDirWithOtherPermissionsForQueryUser() throws Exception {
     // Workspace dir owned by "processUser", workspace group is "group0" and "user2" is not part of "group0"
-    final String viewSchema = MINIDFS_STORAGE_PLUGIN_NAME + ".drillTestGrp0_777";
+    final String viewSchema = MINI_DFS_STORAGE_PLUGIN_NAME + ".drill_test_grp_0_777";
     testCreateViewTestHelper(user2, viewSchema, "view1");
   }
 
@@ -273,7 +273,7 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
   @Test
   public void testCreateViewInWSWithNoPermissionsForQueryUser() throws Exception {
     // Workspace dir owned by "processUser", workspace group is "group0" and "user2" is not part of "group0"
-    final String viewSchema = MINIDFS_STORAGE_PLUGIN_NAME + ".drillTestGrp0_755";
+    final String viewSchema = MINI_DFS_STORAGE_PLUGIN_NAME + ".drill_test_grp_0_755";
     final String viewName = "view1";
 
     updateClient(user2);
@@ -282,7 +282,7 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
 
     final String query = "CREATE VIEW " + viewName + " AS SELECT " +
         "c_custkey, c_nationkey FROM cp.`tpch/customer.parquet` ORDER BY c_custkey;";
-    final String expErrorMsg = "PERMISSION ERROR: Permission denied: user=drillTestUser2, access=WRITE, inode=\"/drillTestGrp0_755/";
+    final String expErrorMsg = "PERMISSION ERROR: Permission denied: user=drillTestUser2, access=WRITE, inode=\"/drill_test_grp_0_755/";
     errorMsgTestHelper(query, expErrorMsg);
 
     // SHOW TABLES is expected to return no records as view creation fails above.
@@ -296,21 +296,21 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
 
   @Test
   public void testCreateTableInDirWithUserPermissionsForQueryUser() throws Exception {
-    final String tableWS = "drillTestGrp1_700"; // Workspace dir owned by "user1"
+    final String tableWS = "drill_test_grp_1_700"; // Workspace dir owned by "user1"
     testCreateTableTestHelper(user1, tableWS, "table1");
   }
 
   @Test
   public void testCreateTableInDirWithGroupPermissionsForQueryUser() throws Exception {
     // Workspace dir owned by "processUser", workspace group is "group0" and "user1" is part of "group0"
-    final String tableWS = "drillTestGrp0_770";
+    final String tableWS = "drill_test_grp_0_770";
     testCreateTableTestHelper(user1, tableWS, "table1");
   }
 
   @Test
   public void testCreateTableInDirWithOtherPermissionsForQueryUser() throws Exception {
     // Workspace dir owned by "processUser", workspace group is "group0" and "user2" is not part of "group0"
-    final String tableWS = "drillTestGrp0_777";
+    final String tableWS = "drill_test_grp_0_777";
     testCreateTableTestHelper(user2, tableWS, "table1");
   }
 
@@ -319,7 +319,7 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
     try {
       updateClient(user);
 
-      test("USE " + Joiner.on(".").join(MINIDFS_STORAGE_PLUGIN_NAME, tableWS));
+      test("USE " + Joiner.on(".").join(MINI_DFS_STORAGE_PLUGIN_NAME, tableWS));
 
       test("CREATE TABLE " + tableName + " AS SELECT " +
           "c_custkey, c_nationkey FROM cp.`tpch/customer.parquet` ORDER BY c_custkey;");
@@ -345,7 +345,7 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
   @Test
   public void testCreateTableInWSWithNoPermissionsForQueryUser() throws Exception {
     // Workspace dir owned by "processUser", workspace group is "group0" and "user2" is not part of "group0"
-    final String tableWS = "drillTestGrp0_755";
+    final String tableWS = "drill_test_grp_0_755";
     final String tableName = "table1";
 
     UserRemoteException ex = null;
@@ -353,7 +353,7 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
     try {
       updateClient(user2);
 
-      test("USE " + Joiner.on(".").join(MINIDFS_STORAGE_PLUGIN_NAME, tableWS));
+      test("USE " + Joiner.on(".").join(MINI_DFS_STORAGE_PLUGIN_NAME, tableWS));
 
       test("CREATE TABLE " + tableName + " AS SELECT " +
           "c_custkey, c_nationkey FROM cp.`tpch/customer.parquet` ORDER BY c_custkey;");
@@ -363,16 +363,16 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
 
     assertNotNull("UserRemoteException is expected", ex);
     assertThat(ex.getMessage(),
-        containsString("SYSTEM ERROR: RemoteException: Permission denied: user=drillTestUser2, access=WRITE, inode=\"/drillTestGrp0_755/"));
+        containsString("SYSTEM ERROR: RemoteException: Permission denied: user=drillTestUser2, access=WRITE, inode=\"/drill_test_grp_0_755/"));
   }
 
   @Test
   public void testRefreshMetadata() throws Exception {
     final String tableName = "nation1";
-    final String tableWS = "drillTestGrp1_700";
+    final String tableWS = "drill_test_grp_1_700";
 
     updateClient(user1);
-    test("USE " + Joiner.on(".").join(MINIDFS_STORAGE_PLUGIN_NAME, tableWS));
+    test("USE " + Joiner.on(".").join(MINI_DFS_STORAGE_PLUGIN_NAME, tableWS));
 
     test("CREATE TABLE " + tableName + " partition by (n_regionkey) AS SELECT * " +
               "FROM cp.`tpch/nation.parquet`;");
@@ -390,8 +390,8 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
   }
 
   @AfterClass
-  public static void removeMiniDfsBasedStorage() throws Exception {
-    getDrillbitContext().getStorage().deletePlugin(MINIDFS_STORAGE_PLUGIN_NAME);
+  public static void removeMiniDfsBasedStorage() {
+    getDrillbitContext().getStorage().deletePlugin(MINI_DFS_STORAGE_PLUGIN_NAME);
     stopMiniDfsCluster();
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationQueries.java b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationQueries.java
index c3c1fc5..ec9d75a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationQueries.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestImpersonationQueries.java
@@ -205,8 +205,7 @@ public class TestImpersonationQueries extends BaseTestImpersonation {
 
     assertNotNull("UserRemoteException is expected", ex);
     assertThat(ex.getMessage(), containsString("PERMISSION ERROR: " +
-      String.format("Not authorized to read table [lineitem] in schema [%s.user0_1]",
-        MINIDFS_STORAGE_PLUGIN_NAME)));
+      String.format("Not authorized to read table [lineitem] in schema [%s.user0_1]", MINI_DFS_STORAGE_PLUGIN_NAME)));
   }
 
   @Test
@@ -261,19 +260,19 @@ public class TestImpersonationQueries extends BaseTestImpersonation {
   public void sequenceFileChainedImpersonationWithView() throws Exception {
     // create a view named "simple_seq_view" on "simple.seq". View is owned by user0:group0 and has permissions 750
     createView(org1Users[0], org1Groups[0], "simple_seq_view",
-      String.format("SELECT convert_from(t.binary_key, 'UTF8') as k FROM %s.`%s` t", MINIDFS_STORAGE_PLUGIN_NAME,
+      String.format("SELECT convert_from(t.binary_key, 'UTF8') as k FROM %s.`%s` t", MINI_DFS_STORAGE_PLUGIN_NAME,
         new Path(getUserHome(org1Users[0]), "simple.seq")));
     try {
       updateClient(org1Users[1]);
-      test("SELECT k FROM %s.%s.%s", MINIDFS_STORAGE_PLUGIN_NAME, "tmp", "simple_seq_view");
+      test("SELECT k FROM %s.%s.%s", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", "simple_seq_view");
     } catch (UserRemoteException e) {
       assertNull("This test should pass.", e);
     }
     createView(org1Users[1], org1Groups[1], "simple_seq_view_2",
-      String.format("SELECT k FROM %s.%s.%s", MINIDFS_STORAGE_PLUGIN_NAME, "tmp", "simple_seq_view"));
+      String.format("SELECT k FROM %s.%s.%s", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", "simple_seq_view"));
     try {
       updateClient(org1Users[2]);
-      test("SELECT k FROM %s.%s.%s", MINIDFS_STORAGE_PLUGIN_NAME, "tmp", "simple_seq_view_2");
+      test("SELECT k FROM %s.%s.%s", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", "simple_seq_view_2");
     } catch (UserRemoteException e) {
       assertNull("This test should pass.", e);
     }
@@ -282,11 +281,11 @@ public class TestImpersonationQueries extends BaseTestImpersonation {
   @Test
   public void avroChainedImpersonationWithView() throws Exception {
     createView(org1Users[0], org1Groups[0], "simple_avro_view",
-      String.format("SELECT h_boolean, e_double FROM %s.`%s` t", MINIDFS_STORAGE_PLUGIN_NAME,
+      String.format("SELECT h_boolean, e_double FROM %s.`%s` t", MINI_DFS_STORAGE_PLUGIN_NAME,
         new Path(getUserHome(org1Users[0]), "simple.avro")));
     try {
       updateClient(org1Users[1]);
-      test("SELECT h_boolean FROM %s.%s.%s", MINIDFS_STORAGE_PLUGIN_NAME, "tmp", "simple_avro_view");
+      test("SELECT h_boolean FROM %s.%s.%s", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", "simple_avro_view");
     } catch (UserRemoteException e) {
       assertNull("This test should pass.", e);
     }
@@ -294,7 +293,7 @@ public class TestImpersonationQueries extends BaseTestImpersonation {
 
   @AfterClass
   public static void removeMiniDfsBasedStorage() throws Exception {
-    getDrillbitContext().getStorage().deletePlugin(MINIDFS_STORAGE_PLUGIN_NAME);
+    getDrillbitContext().getStorage().deletePlugin(MINI_DFS_STORAGE_PLUGIN_NAME);
     stopMiniDfsCluster();
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/TestDirectoryExplorerUDFs.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/TestDirectoryExplorerUDFs.java
index dea4c59..f7c329c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/TestDirectoryExplorerUDFs.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/TestDirectoryExplorerUDFs.java
@@ -96,7 +96,7 @@ public class TestDirectoryExplorerUDFs extends PlanTestBase {
         .add("BIGFILE_2")
         .build();
 
-    String query = "select * from dfs.`%s/*/*.csv` where dir0 = %s('dfs.root','%s')";
+    String query = "select * from dfs.`%s/*/*.csv` where dir0 = %s('dFs.RoOt','%s')";
     for (ConstantFoldingTestConfig config : tests) {
       // make all of the other folders unexpected patterns, except for the one expected in this case
       List<String> excludedPatterns = Lists.newArrayList();
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 6e7d054..e295eee 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
@@ -17,33 +17,33 @@
  */
 package org.apache.drill.exec.sql;
 
-import static com.fasterxml.jackson.databind.SerializationFeature.INDENT_OUTPUT;
-import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.CATS_COL_CATALOG_CONNECT;
-import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.CATS_COL_CATALOG_DESCRIPTION;
-import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.CATS_COL_CATALOG_NAME;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.collect.ImmutableList;
-import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.SqlTest;
-import org.apache.drill.test.TestBuilder;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.store.dfs.FileSystemConfig;
 import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.test.BaseTestQuery;
+import org.apache.drill.test.TestBuilder;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import java.nio.file.Paths;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
+import static com.fasterxml.jackson.databind.SerializationFeature.INDENT_OUTPUT;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.CATS_COL_CATALOG_CONNECT;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.CATS_COL_CATALOG_DESCRIPTION;
+import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.CATS_COL_CATALOG_NAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 /**
  * Contains tests for
  * -- InformationSchema
@@ -84,15 +84,13 @@ public class TestInfoSchema extends BaseTestQuery {
 
   @Test
   public void showTablesFromDb() throws Exception{
-    final List<String[]> expected =
-        ImmutableList.of(
-            new String[] { "INFORMATION_SCHEMA", "VIEWS" },
-            new String[] { "INFORMATION_SCHEMA", "COLUMNS" },
-            new String[] { "INFORMATION_SCHEMA", "TABLES" },
-            new String[] { "INFORMATION_SCHEMA", "CATALOGS" },
-            new String[] { "INFORMATION_SCHEMA", "SCHEMATA" },
-            new String[] { "INFORMATION_SCHEMA", "FILES" }
-        );
+    final List<String[]> expected = Arrays.asList(
+        new String[]{"information_schema", "VIEWS"},
+        new String[]{"information_schema", "COLUMNS"},
+        new String[]{"information_schema", "TABLES"},
+        new String[]{"information_schema", "CATALOGS"},
+        new String[]{"information_schema", "SCHEMATA"},
+        new String[]{"information_schema", "FILES"});
 
     final TestBuilder t1 = testBuilder()
         .sqlQuery("SHOW TABLES FROM INFORMATION_SCHEMA")
@@ -119,7 +117,7 @@ public class TestInfoSchema extends BaseTestQuery {
         .sqlQuery("SHOW TABLES FROM INFORMATION_SCHEMA WHERE TABLE_NAME='VIEWS'")
         .unOrdered()
         .baselineColumns("TABLE_SCHEMA", "TABLE_NAME")
-        .baselineValues("INFORMATION_SCHEMA", "VIEWS")
+        .baselineValues("information_schema", "VIEWS")
         .go();
   }
 
@@ -130,38 +128,26 @@ public class TestInfoSchema extends BaseTestQuery {
         .unOrdered()
         .optionSettingQueriesForTestQuery("USE INFORMATION_SCHEMA")
         .baselineColumns("TABLE_SCHEMA", "TABLE_NAME")
-        .baselineValues("INFORMATION_SCHEMA", "SCHEMATA")
+        .baselineValues("information_schema", "SCHEMATA")
         .go();
   }
 
   @Test
   public void showDatabases() throws Exception{
-    final List<String[]> expected =
-        ImmutableList.of(
-            new String[] { "dfs.default" },
-            new String[] { "dfs.root" },
-            new String[] { "dfs.tmp" },
-            new String[] { "cp.default" },
-            new String[] { "sys" },
-            new String[] { "INFORMATION_SCHEMA" }
-        );
+    List<String> expected = Arrays.asList("dfs.default", "dfs.root", "dfs.tmp", "cp.default", "sys", "information_schema");
 
-    final TestBuilder t1 = testBuilder()
+    TestBuilder t1 = testBuilder()
         .sqlQuery("SHOW DATABASES")
         .unOrdered()
         .baselineColumns("SCHEMA_NAME");
-    for(String[] expectedRow : expected) {
-      t1.baselineValues(expectedRow);
-    }
+    expected.forEach(t1::baselineValues);
     t1.go();
 
-    final TestBuilder t2 = testBuilder()
+    TestBuilder t2 = testBuilder()
         .sqlQuery("SHOW SCHEMAS")
         .unOrdered()
         .baselineColumns("SCHEMA_NAME");
-    for(String[] expectedRow : expected) {
-      t2.baselineValues(expectedRow);
-    }
+    expected.forEach(t2::baselineValues);
     t2.go();
   }
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSchemaCaseInsensitivity.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSchemaCaseInsensitivity.java
new file mode 100644
index 0000000..c274e0b
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSchemaCaseInsensitivity.java
@@ -0,0 +1,92 @@
+/*
+ * 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.sql;
+
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterFixtureBuilder;
+import org.apache.drill.test.ClusterTest;
+import org.apache.drill.test.QueryBuilder;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestSchemaCaseInsensitivity extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    ClusterFixtureBuilder builder = ClusterFixture.builder(dirTestWatcher);
+    startCluster(builder);
+  }
+
+  @Test
+  public void testUseCommand() throws Exception {
+    queryBuilder().sql("use Information_Schema").run();
+    queryBuilder().sql("use Sys").run();
+    queryBuilder().sql("use Dfs").run();
+    queryBuilder().sql("use Dfs.Tmp").run();
+  }
+
+  @Test
+  public void testDescribeSchema() throws Exception {
+    checkRecordCount(1, "describe schema SyS");
+    checkRecordCount(1, "describe schema Information_Schema");
+
+    client.testBuilder()
+        .sqlQuery("describe schema DfS.tMp")
+        .unOrdered()
+        .sqlBaselineQuery("describe schema dfs.tmp")
+        .go();
+  }
+
+  @Test
+  public void testDescribeTable() throws Exception {
+    checkRecordCount(4, "describe Information_Schema.`Tables`");
+    checkRecordCount(1, "describe Information_Schema.`Tables` Table_Catalog");
+    checkRecordCount(1, "describe Information_Schema.`Tables` '%Catalog'");
+    checkRecordCount(6, "describe SyS.Version");
+  }
+
+
+  @Test
+  public void testShowSchemas() throws Exception {
+    checkRecordCount(1, "show schemas like '%Y%'");
+    checkRecordCount(1, "show schemas like 'Info%'");
+    checkRecordCount(1, "show schemas like 'D%Tmp'");
+  }
+
+  @Test
+  public void testShowTables() throws Exception {
+    checkRecordCount(1, "show tables in Information_Schema like 'SC%'");
+    checkRecordCount(1, "show tables in Sys like '%ION'");
+  }
+
+  @Test
+  public void testSelectStatement() throws Exception {
+    checkRecordCount(1, "select * from Information_Schema.Schemata where Schema_Name = 'dfs.tmp'");
+    checkRecordCount(1, "select * from Sys.Version");
+  }
+
+  private void checkRecordCount(long recordCount, String sqlQuery) throws Exception {
+    QueryBuilder.QuerySummary summary = queryBuilder().sql(sqlQuery).run();
+    assertTrue(summary.succeeded());
+    assertEquals(recordCount, summary.recordCount());
+  }
+
+}
\ No newline at end of file
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/work/metadata/TestMetadataProvider.java b/exec/java-exec/src/test/java/org/apache/drill/exec/work/metadata/TestMetadataProvider.java
index 8d34ebc..c5b607b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/work/metadata/TestMetadataProvider.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/work/metadata/TestMetadataProvider.java
@@ -24,7 +24,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.drill.test.BaseTestQuery;
@@ -105,7 +105,7 @@ public class TestMetadataProvider extends BaseTestQuery {
     List<SchemaMetadata> schemas = resp.getSchemasList();
     assertEquals(6, schemas.size());
 
-    verifySchema("INFORMATION_SCHEMA", schemas);
+    verifySchema("information_schema", schemas);
     verifySchema("cp.default", schemas);
     verifySchema("dfs.default", schemas);
     verifySchema("dfs.root", schemas);
@@ -115,15 +115,15 @@ public class TestMetadataProvider extends BaseTestQuery {
 
   @Test
   public void schemasWithSchemaNameFilter() throws Exception {
-    // test("SELECT * FROM INFORMATION_SCHEMA.SCHEMATA WHERE SCHEMA_NAME LIKE '%y%'"); // SQL equivalent
+    // test("SELECT * FROM INFORMATION_SCHEMA.SCHEMATA WHERE SCHEMA_NAME LIKE '%tion_sc%'"); // SQL equivalent
 
-    GetSchemasResp resp = client.getSchemas(null, LikeFilter.newBuilder().setPattern("%y%").build()).get();
+    GetSchemasResp resp = client.getSchemas(null, LikeFilter.newBuilder().setPattern("%TiOn_Sc%").build()).get();
 
     assertEquals(RequestStatus.OK, resp.getStatus());
     List<SchemaMetadata> schemas = resp.getSchemasList();
     assertEquals(1, schemas.size());
 
-    verifySchema("sys", schemas);
+    verifySchema("information_schema", schemas);
   }
 
   @Test
@@ -151,12 +151,12 @@ public class TestMetadataProvider extends BaseTestQuery {
     List<TableMetadata> tables = resp.getTablesList();
     assertEquals(18, tables.size());
 
-    verifyTable("INFORMATION_SCHEMA", "CATALOGS", tables);
-    verifyTable("INFORMATION_SCHEMA", "COLUMNS", tables);
-    verifyTable("INFORMATION_SCHEMA", "SCHEMATA", tables);
-    verifyTable("INFORMATION_SCHEMA", "TABLES", tables);
-    verifyTable("INFORMATION_SCHEMA", "VIEWS", tables);
-    verifyTable("INFORMATION_SCHEMA", "FILES", tables);
+    verifyTable("information_schema", "CATALOGS", tables);
+    verifyTable("information_schema", "COLUMNS", tables);
+    verifyTable("information_schema", "SCHEMATA", tables);
+    verifyTable("information_schema", "TABLES", tables);
+    verifyTable("information_schema", "VIEWS", tables);
+    verifyTable("information_schema", "FILES", tables);
     verifyTable("sys", "boot", tables);
     verifyTable("sys", "drillbits", tables);
     verifyTable("sys", "memory", tables);
@@ -172,7 +172,7 @@ public class TestMetadataProvider extends BaseTestQuery {
   public void tablesWithTableFilter() throws Exception {
     // test("SELECT * FROM INFORMATION_SCHEMA.`TABLES` WHERE TABLE_TYPE IN ('TABLE')"); // SQL equivalent
 
-    GetTablesResp resp = client.getTables(null, null, null, Arrays.asList("TABLE")).get();
+    GetTablesResp resp = client.getTables(null, null, null, Collections.singletonList("TABLE")).get();
 
     assertEquals(RequestStatus.OK, resp.getStatus());
     List<TableMetadata> tables = resp.getTablesList();
@@ -183,18 +183,18 @@ public class TestMetadataProvider extends BaseTestQuery {
   public void tablesWithSystemTableFilter() throws Exception {
     // test("SELECT * FROM INFORMATION_SCHEMA.`TABLES` WHERE TABLE_TYPE IN ('SYSTEM_TABLE')"); // SQL equivalent
 
-    GetTablesResp resp = client.getTables(null, null, null, Arrays.asList("SYSTEM_TABLE")).get();
+    GetTablesResp resp = client.getTables(null, null, null, Collections.singletonList("SYSTEM_TABLE")).get();
 
     assertEquals(RequestStatus.OK, resp.getStatus());
     List<TableMetadata> tables = resp.getTablesList();
     assertEquals(18, tables.size());
 
-    verifyTable("INFORMATION_SCHEMA", "CATALOGS", tables);
-    verifyTable("INFORMATION_SCHEMA", "COLUMNS", tables);
-    verifyTable("INFORMATION_SCHEMA", "SCHEMATA", tables);
-    verifyTable("INFORMATION_SCHEMA", "TABLES", tables);
-    verifyTable("INFORMATION_SCHEMA", "VIEWS", tables);
-    verifyTable("INFORMATION_SCHEMA", "FILES", tables);
+    verifyTable("information_schema", "CATALOGS", tables);
+    verifyTable("information_schema", "COLUMNS", tables);
+    verifyTable("information_schema", "SCHEMATA", tables);
+    verifyTable("information_schema", "TABLES", tables);
+    verifyTable("information_schema", "VIEWS", tables);
+    verifyTable("information_schema", "FILES", tables);
     verifyTable("sys", "boot", tables);
     verifyTable("sys", "drillbits", tables);
     verifyTable("sys", "memory", tables);


[drill] 03/06: DRILL-6693: When a query is started from Drill Web Console, the UI becomes inaccessible until the query finishes

Posted by ar...@apache.org.
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

commit 260a164f1884f99208811ec151078fa019534243
Author: Vlad Rozov <vr...@apache.org>
AuthorDate: Wed Aug 22 18:42:11 2018 -0700

    DRILL-6693: When a query is started from Drill Web Console, the UI becomes inaccessible until the query finishes
    
    git closes #1440
---
 .../java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java | 1 +
 .../src/main/java/org/apache/drill/exec/server/rest/WebServer.java   | 5 +++--
 exec/java-exec/src/main/resources/drill-module.conf                  | 3 ++-
 3 files changed, 6 insertions(+), 3 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 43ee886..3817971 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -182,6 +182,7 @@ public final class ExecConstants {
   public static final String HTTP_PORT_HUNT = "drill.exec.http.porthunt";
   public static final String HTTP_JETTY_SERVER_ACCEPTORS = "drill.exec.http.jetty.server.acceptors";
   public static final String HTTP_JETTY_SERVER_SELECTORS = "drill.exec.http.jetty.server.selectors";
+  public static final String HTTP_JETTY_SERVER_HANDLERS = "drill.exec.http.jetty.server.handlers";
   public static final String HTTP_ENABLE_SSL = "drill.exec.http.ssl_enabled";
   public static final String HTTP_CORS_ENABLED = "drill.exec.http.cors.enabled";
   public static final String HTTP_CORS_ALLOWED_ORIGINS = "drill.exec.http.cors.allowedOrigins";
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
index 098845e..8f88e21 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebServer.java
@@ -136,7 +136,8 @@ public class WebServer implements AutoCloseable {
     final boolean portHunt = config.getBoolean(ExecConstants.HTTP_PORT_HUNT);
     final int acceptors = config.getInt(ExecConstants.HTTP_JETTY_SERVER_ACCEPTORS);
     final int selectors = config.getInt(ExecConstants.HTTP_JETTY_SERVER_SELECTORS);
-    final QueuedThreadPool threadPool = new QueuedThreadPool(2, 2, 60000);
+    final int handlers = config.getInt(ExecConstants.HTTP_JETTY_SERVER_HANDLERS);
+    final QueuedThreadPool threadPool = new QueuedThreadPool(2, 2);
     embeddedJetty = new Server(threadPool);
     ServletContextHandler webServerContext = createServletContextHandler(authEnabled);
     //Allow for Other Drillbits to make REST calls
@@ -147,7 +148,7 @@ public class WebServer implements AutoCloseable {
     embeddedJetty.setHandler(webServerContext);
 
     ServerConnector connector = createConnector(port, acceptors, selectors);
-    threadPool.setMaxThreads(1 + connector.getAcceptors() + connector.getSelectorManager().getSelectorCount());
+    threadPool.setMaxThreads(handlers + connector.getAcceptors() + connector.getSelectorManager().getSelectorCount());
     embeddedJetty.addConnector(connector);
     for (int retry = 0; retry < PORT_HUNT_TRIES; retry++) {
       connector.setPort(port);
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index eca59b9..d4cda92 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -133,7 +133,8 @@ drill.exec: {
     jetty : {
       server : {
         acceptors : 1,
-        selectors : 2
+        selectors : 1,
+        handlers : 5
       }
     }
     max_profiles: 100,


[drill] 04/06: DRILL-6703: Query with complex expressions in lateral and unnest fails with CannotPlanException

Posted by ar...@apache.org.
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

commit a1f3f9a7ee06d14e1932374e928853480e86dd90
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Wed Aug 22 13:02:33 2018 +0300

    DRILL-6703: Query with complex expressions in lateral and unnest fails with CannotPlanException
    
    git closes #1441
---
 .../apache/drill/exec/planner/PlannerPhase.java    |   3 -
 ...rojectComplexRexNodeCorrelateTransposeRule.java | 154 ----------------
 .../planner/sql/handlers/ComplexUnnestVisitor.java | 199 +++++++++++++++++++++
 .../planner/sql/handlers/DefaultSqlHandler.java    |  33 ++--
 .../impl/lateraljoin/TestE2EUnnestAndLateral.java  |  39 ++++
 5 files changed, 252 insertions(+), 176 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
index 2d02011..168ff9e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
@@ -37,7 +37,6 @@ import org.apache.drill.exec.planner.logical.DrillJoinRel;
 import org.apache.drill.exec.planner.logical.DrillJoinRule;
 import org.apache.drill.exec.planner.logical.DrillLimitRule;
 import org.apache.drill.exec.planner.logical.DrillMergeProjectRule;
-import org.apache.drill.exec.planner.logical.ProjectComplexRexNodeCorrelateTransposeRule;
 import org.apache.drill.exec.planner.logical.DrillProjectLateralJoinTransposeRule;
 import org.apache.drill.exec.planner.logical.DrillProjectPushIntoLateralJoinRule;
 import org.apache.drill.exec.planner.logical.DrillProjectRule;
@@ -312,8 +311,6 @@ public enum PlannerPhase {
       RuleInstance.PROJECT_WINDOW_TRANSPOSE_RULE,
       DrillPushProjectIntoScanRule.INSTANCE,
 
-      ProjectComplexRexNodeCorrelateTransposeRule.INSTANCE,
-
       /*
        Convert from Calcite Logical to Drill Logical Rules.
        */
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ProjectComplexRexNodeCorrelateTransposeRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ProjectComplexRexNodeCorrelateTransposeRule.java
deleted file mode 100644
index a979d5b..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ProjectComplexRexNodeCorrelateTransposeRule.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.planner.logical;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Correlate;
-import org.apache.calcite.rel.core.CorrelationId;
-import org.apache.calcite.rel.core.Uncollect;
-import org.apache.calcite.rel.logical.LogicalCorrelate;
-import org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.rex.RexCorrelVariable;
-import org.apache.calcite.rex.RexFieldAccess;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexShuttle;
-import org.apache.calcite.tools.RelBuilder;
-import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.calcite.util.trace.CalciteTrace;
-import org.apache.drill.common.exceptions.UserException;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Rule that moves non-{@link RexFieldAccess} rex node from project below {@link Uncollect}
- * to the left side of the {@link Correlate}.
- */
-public class ProjectComplexRexNodeCorrelateTransposeRule extends RelOptRule {
-
-  public static final RelOptRule INSTANCE = new ProjectComplexRexNodeCorrelateTransposeRule();
-
-  public ProjectComplexRexNodeCorrelateTransposeRule() {
-    super(operand(LogicalCorrelate.class,
-        operand(RelNode.class, any()),
-        operand(Uncollect.class, operand(LogicalProject.class, any()))),
-        DrillRelFactories.LOGICAL_BUILDER,
-        "ProjectComplexRexNodeCorrelateTransposeRule");
-  }
-
-  @Override
-  public void onMatch(RelOptRuleCall call) {
-    final Correlate correlate = call.rel(0);
-    final Uncollect uncollect = call.rel(2);
-    final LogicalProject project = call.rel(3);
-
-    // uncollect requires project with single expression
-    RexNode projectedNode = project.getProjects().iterator().next();
-
-    // check that the expression is complex call
-    if (!(projectedNode instanceof RexFieldAccess)) {
-      RelBuilder builder = call.builder();
-      RexBuilder rexBuilder = builder.getRexBuilder();
-
-      builder.push(correlate.getLeft());
-
-      // creates project with complex expr on top of the left side
-      List<RexNode> leftProjExprs = new ArrayList<>();
-
-      String complexFieldName = correlate.getRowType().getFieldNames()
-            .get(correlate.getRowType().getFieldNames().size() - 1);
-
-      List<String> fieldNames = new ArrayList<>();
-      for (RelDataTypeField field : correlate.getLeft().getRowType().getFieldList()) {
-        leftProjExprs.add(rexBuilder.makeInputRef(correlate.getLeft(), field.getIndex()));
-        fieldNames.add(field.getName());
-      }
-      fieldNames.add(complexFieldName);
-      List<RexNode> topProjectExpressions = new ArrayList<>(leftProjExprs);
-
-      // adds complex expression with replaced correlation
-      // to the projected list from the left
-      leftProjExprs.add(projectedNode.accept(new RexFieldAccessReplacer(builder)));
-
-      RelNode leftProject = builder.project(leftProjExprs, fieldNames)
-          .build();
-
-      CorrelationId correlationId = correlate.getCluster().createCorrel();
-      RexCorrelVariable rexCorrel =
-          (RexCorrelVariable) rexBuilder.makeCorrel(
-              leftProject.getRowType(),
-              correlationId);
-      builder.push(project.getInput());
-      RelNode rightProject = builder.project(
-              ImmutableList.of(rexBuilder.makeFieldAccess(rexCorrel, leftProjExprs.size() - 1)),
-              ImmutableList.of(complexFieldName))
-          .build();
-
-      int requiredColumnsCount = correlate.getRequiredColumns().cardinality();
-      if (requiredColumnsCount != 1) {
-        throw UserException.planError()
-            .message("Required columns count for Correlate operator " +
-                "differs from the expected value:\n" +
-                "Expected columns count is %s, but actual is %s",
-                1, requiredColumnsCount)
-            .build(CalciteTrace.getPlannerTracer());
-      }
-
-      RelNode newUncollect = uncollect.copy(uncollect.getTraitSet(), rightProject);
-      Correlate newCorrelate = correlate.copy(uncollect.getTraitSet(), leftProject, newUncollect,
-          correlationId, ImmutableBitSet.of(leftProjExprs.size() - 1), correlate.getJoinType());
-      builder.push(newCorrelate);
-
-      switch(correlate.getJoinType()) {
-        case LEFT:
-        case INNER:
-          // adds field from the right input of correlate to the top project
-          topProjectExpressions.add(
-              rexBuilder.makeInputRef(newCorrelate, topProjectExpressions.size() + 1));
-          // fall through
-        case ANTI:
-        case SEMI:
-          builder.project(topProjectExpressions, correlate.getRowType().getFieldNames());
-      }
-
-      call.transformTo(builder.build());
-    }
-  }
-
-  /**
-   * Visitor for RexNode which replaces {@link RexFieldAccess}
-   * with a reference to the field used in {@link RexFieldAccess}.
-   */
-  private static class RexFieldAccessReplacer extends RexShuttle {
-    private final RelBuilder builder;
-
-    public RexFieldAccessReplacer(RelBuilder builder) {
-      this.builder = builder;
-    }
-
-    @Override
-    public RexNode visitFieldAccess(RexFieldAccess fieldAccess) {
-      return builder.field(fieldAccess.getField().getName());
-    }
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ComplexUnnestVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ComplexUnnestVisitor.java
new file mode 100644
index 0000000..0134377
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ComplexUnnestVisitor.java
@@ -0,0 +1,199 @@
+/*
+ * 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.planner.sql.handlers;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.core.Correlate;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Uncollect;
+import org.apache.calcite.rel.logical.LogicalCorrelate;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.drill.exec.planner.logical.DrillRelFactories;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Visitor that moves non-{@link RexFieldAccess} rex node from project below {@link Uncollect}
+ * to the left side of the {@link Correlate}.
+ */
+public class ComplexUnnestVisitor extends RelShuttleImpl {
+  private static final String COMPLEX_FIELD_NAME = "$COMPLEX_FIELD_NAME";
+
+  private final Map<CorrelationId, RelNode> leftInputs = new HashMap<>();
+  private final Map<CorrelationId, CorrelationId> updatedCorrelationIds = new HashMap<>();
+
+  private ComplexUnnestVisitor() {
+  }
+
+  @Override
+  public RelNode visit(LogicalCorrelate correlate) {
+    RelNode left = correlate.getLeft().accept(this);
+    leftInputs.put(correlate.getCorrelationId(), left);
+
+    RelNode right = correlate.getRight().accept(this);
+    // if right input wasn't changed or left input wasn't changed
+    // after rewriting right input, no need to create Correlate with new CorrelationId
+    if (correlate.getRight() == right
+        || left == leftInputs.get(correlate.getCorrelationId())) {
+      if (correlate.getLeft() == left) {
+        return correlate;
+      }
+      // changed only inputs, but CorrelationId left the same
+      return correlate.copy(correlate.getTraitSet(), Arrays.asList(left, right));
+    }
+
+    Correlate newCorrelate = correlate.copy(correlate.getTraitSet(),
+        leftInputs.get(correlate.getCorrelationId()), right,
+        updatedCorrelationIds.get(correlate.getCorrelationId()),
+        ImmutableBitSet.of(left.getRowType().getFieldCount()), correlate.getJoinType());
+
+    RelBuilder builder = DrillRelFactories.LOGICAL_BUILDER.create(correlate.getCluster(), null);
+    builder.push(newCorrelate);
+
+    List<RexNode> topProjectExpressions = left.getRowType().getFieldList().stream()
+        .map(field -> builder.getRexBuilder().makeInputRef(left, field.getIndex()))
+        .collect(Collectors.toList());
+
+    switch (correlate.getJoinType()) {
+      case LEFT:
+      case INNER:
+        // adds field from the right input of correlate to the top project
+        topProjectExpressions.add(
+            builder.getRexBuilder().makeInputRef(newCorrelate, topProjectExpressions.size() + 1));
+        // fall through
+      case ANTI:
+      case SEMI:
+        builder.project(topProjectExpressions, correlate.getRowType().getFieldNames());
+    }
+    return builder.build();
+  }
+
+  @Override
+  public RelNode visit(RelNode other) {
+    if (other instanceof Uncollect) {
+      return visit((Uncollect) other);
+    }
+    return super.visit(other);
+  }
+
+  public RelNode visit(Uncollect uncollect) {
+    RelBuilder builder = DrillRelFactories.LOGICAL_BUILDER.create(uncollect.getCluster(), null);
+    RexBuilder rexBuilder = builder.getRexBuilder();
+
+    assert uncollect.getInput() instanceof Project : "Uncollect should have Project input";
+
+    Project project = (Project) uncollect.getInput();
+    // If project below uncollect contains only field references, no need to rewrite it
+    List<RexNode> projectChildExps = project.getChildExps();
+    assert projectChildExps.size() == 1 : "Uncollect does not support multiple expressions";
+
+    RexNode projectExpr = projectChildExps.iterator().next();
+    if (projectExpr.getKind() == SqlKind.FIELD_ACCESS) {
+      return uncollect;
+    }
+
+    // Collects CorrelationId instances used in current rel node
+    RelOptUtil.VariableUsedVisitor variableUsedVisitor = new RelOptUtil.VariableUsedVisitor(null);
+    project.accept(variableUsedVisitor);
+
+    assert variableUsedVisitor.variables.size() == 1 : "Uncollect supports only single correlated reference";
+
+    CorrelationId oldCorrId = variableUsedVisitor.variables.iterator().next();
+    RelNode left = leftInputs.get(oldCorrId);
+
+    // Creates new project to be placed on top of the left input of correlate
+    List<RexNode> leftProjExprs = new ArrayList<>();
+
+    List<String> fieldNames = new ArrayList<>();
+    for (RelDataTypeField field : left.getRowType().getFieldList()) {
+      leftProjExprs.add(rexBuilder.makeInputRef(left, field.getIndex()));
+      fieldNames.add(field.getName());
+    }
+    fieldNames.add(COMPLEX_FIELD_NAME);
+
+    builder.push(left);
+
+    // Adds complex expression with replaced correlation
+    // to the projected list from the left
+    leftProjExprs.add(new RexFieldAccessReplacer(builder).apply(projectExpr));
+
+    RelNode leftProject =
+        builder.project(leftProjExprs, fieldNames)
+            .build();
+    leftInputs.put(oldCorrId, leftProject);
+
+    builder.push(project.getInput());
+
+    CorrelationId newCorrId = uncollect.getCluster().createCorrel();
+    // stores new CorrelationId to be used during the creation of new Correlate
+    updatedCorrelationIds.put(oldCorrId, newCorrId);
+
+    RexNode rexCorrel = rexBuilder.makeCorrel(leftProject.getRowType(), newCorrId);
+
+    // constructs Project below Uncollect with updated RexCorrelVariable
+    builder.project(
+        ImmutableList.of(rexBuilder.makeFieldAccess(rexCorrel, leftProjExprs.size() - 1)),
+        ImmutableList.of(COMPLEX_FIELD_NAME));
+    return uncollect.copy(uncollect.getTraitSet(), builder.build());
+  }
+
+  /**
+   * Rewrites rel node tree and moves non-{@link RexFieldAccess} rex node from the project
+   * below {@link Uncollect} to the left side of the {@link Correlate}.
+   *
+   * @param relNode tree to be rewritten
+   * @return rewritten rel node tree
+   */
+  public static RelNode rewriteUnnestWithComplexExprs(RelNode relNode) {
+    ComplexUnnestVisitor visitor = new ComplexUnnestVisitor();
+    return relNode.accept(visitor);
+  }
+
+  /**
+   * Visitor for RexNode which replaces {@link RexFieldAccess}
+   * with a reference to the field used in {@link RexFieldAccess}.
+   */
+  private static class RexFieldAccessReplacer extends RexShuttle {
+    private final RelBuilder builder;
+
+    public RexFieldAccessReplacer(RelBuilder builder) {
+      this.builder = builder;
+    }
+
+    @Override
+    public RexNode visitFieldAccess(RexFieldAccess fieldAccess) {
+      return builder.field(fieldAccess.getField().getName());
+    }
+  }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
index 0d9793a..5ba5640 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
@@ -29,8 +29,6 @@ import com.fasterxml.jackson.databind.ser.impl.SimpleFilterProvider;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Sets;
 import org.apache.calcite.plan.RelOptCostImpl;
-import org.apache.calcite.plan.RelOptLattice;
-import org.apache.calcite.plan.RelOptMaterialization;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptUtil;
@@ -111,7 +109,6 @@ import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
 import org.apache.drill.exec.work.foreman.UnsupportedRelOperatorException;
 import org.slf4j.Logger;
 
-import com.fasterxml.jackson.core.JsonProcessingException;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Stopwatch;
 import com.google.common.collect.Lists;
@@ -164,7 +161,7 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
     }
   }
 
-  protected void log(final String name, final PhysicalPlan plan, final Logger logger) throws JsonProcessingException {
+  protected void log(final String name, final PhysicalPlan plan, final Logger logger) {
     if (logger.isDebugEnabled()) {
       PropertyFilter filter = new SimpleBeanPropertyFilter.SerializeExceptFilter(Sets.newHashSet("password"));
       String planText = plan.unparse(context.getLpPersistence().getMapper()
@@ -193,9 +190,8 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
    * Rewrite the parse tree. Used before validating the parse tree. Useful if a particular statement needs to converted
    * into another statement.
    *
-   * @param node
+   * @param node sql parse tree to be rewritten
    * @return Rewritten sql parse tree
-   * @throws RelConversionException
    */
   protected SqlNode rewrite(SqlNode node) throws RelConversionException, ForemanSetupException {
     return node;
@@ -217,10 +213,9 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
    *
    * @param relNode relational node
    * @return Drill Logical RelNode tree
-   * @throws SqlUnsupportedException
-   * @throws RelConversionException
+   * @throws SqlUnsupportedException if query cannot be planned
    */
-  protected DrillRel convertToRawDrel(final RelNode relNode) throws SqlUnsupportedException, RelConversionException {
+  protected DrillRel convertToRawDrel(final RelNode relNode) throws SqlUnsupportedException {
     if (context.getOptions().getOption(ExecConstants.EARLY_LIMIT0_OPT) &&
         context.getPlannerSettings().isTypeInferenceEnabled() &&
         FindLimit0Visitor.containsLimit0(relNode)) {
@@ -299,7 +294,7 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
     } catch (RelOptPlanner.CannotPlanException ex) {
       logger.error(ex.getMessage());
 
-      if(JoinUtils.checkCartesianJoin(relNode, new ArrayList<Integer>(), new ArrayList<Integer>(), new ArrayList<Boolean>())) {
+      if (JoinUtils.checkCartesianJoin(relNode, new ArrayList<>(), new ArrayList<>(), new ArrayList<>())) {
         throw new UnsupportedRelOperatorException("This query cannot be planned possibly due to either a cartesian join or an inequality join");
       } else {
         throw ex;
@@ -313,10 +308,9 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
    *
    * @param relNode root RelNode corresponds to Calcite Logical RelNode.
    * @return Drill Logical RelNode tree
-   * @throws RelConversionException
-   * @throws SqlUnsupportedException
+   * @throws SqlUnsupportedException if query cannot be planned
    */
-  protected DrillRel convertToDrel(RelNode relNode) throws RelConversionException, SqlUnsupportedException {
+  protected DrillRel convertToDrel(RelNode relNode) throws SqlUnsupportedException {
     final DrillRel convertedRelNode = convertToRawDrel(relNode);
 
     return new DrillScreenRel(convertedRelNode.getCluster(), convertedRelNode.getTraitSet(),
@@ -429,7 +423,7 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
           "Cluster is expected to be constructed using VolcanoPlanner. Was actually of type %s.", planner.getClass()
               .getName());
       output = program.run(planner, input, toTraits,
-          ImmutableList.<RelOptMaterialization>of(), ImmutableList.<RelOptLattice>of());
+          ImmutableList.of(), ImmutableList.of());
 
       break;
     }
@@ -465,7 +459,7 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
     } catch (RelOptPlanner.CannotPlanException ex) {
       logger.error(ex.getMessage());
 
-      if(JoinUtils.checkCartesianJoin(drel, new ArrayList<Integer>(), new ArrayList<Integer>(), new ArrayList<Boolean>())) {
+      if (JoinUtils.checkCartesianJoin(drel, new ArrayList<>(), new ArrayList<>(), new ArrayList<>())) {
         throw new UnsupportedRelOperatorException("This query cannot be planned possibly due to either a cartesian join or an inequality join");
       } else {
         throw ex;
@@ -488,7 +482,7 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
       } catch (RelOptPlanner.CannotPlanException ex) {
         logger.error(ex.getMessage());
 
-        if(JoinUtils.checkCartesianJoin(drel, new ArrayList<Integer>(), new ArrayList<Integer>(), new ArrayList<Boolean>())) {
+        if (JoinUtils.checkCartesianJoin(drel, new ArrayList<>(), new ArrayList<>(), new ArrayList<>())) {
           throw new UnsupportedRelOperatorException("This query cannot be planned possibly due to either a cartesian join or an inequality join");
         } else {
           throw ex;
@@ -526,8 +520,8 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
      * We want to have smaller dataset on the right side, since hash table builds on right side.
      */
     if (context.getPlannerSettings().isHashJoinSwapEnabled()) {
-      phyRelNode = SwapHashJoinVisitor.swapHashJoin(phyRelNode, Double.valueOf(context.getPlannerSettings()
-          .getHashJoinSwapMarginFactor()));
+      phyRelNode = SwapHashJoinVisitor.swapHashJoin(phyRelNode,
+          context.getPlannerSettings().getHashJoinSwapMarginFactor());
     }
 
     /* Parquet row group filter pushdown in planning time */
@@ -710,7 +704,8 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
       throw ex;
     }
 
-    return rel;
+    // moves complex expressions below Uncollect to the right side of Correlate
+    return ComplexUnnestVisitor.rewriteUnnestWithComplexExprs(rel);
   }
 
   protected DrillRel addRenamedProject(DrillRel rel, RelDataType validatedRowType) {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/lateraljoin/TestE2EUnnestAndLateral.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/lateraljoin/TestE2EUnnestAndLateral.java
index 0283ade..6578d0e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/lateraljoin/TestE2EUnnestAndLateral.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/lateraljoin/TestE2EUnnestAndLateral.java
@@ -656,4 +656,43 @@ public class TestE2EUnnestAndLateral extends ClusterTest {
         .go();
   }
 
+  @Test
+  public void testLateralWithComplexProject() throws Exception {
+    String sql = "select l.name from cp.`lateraljoin/nested-customer.parquet` c,\n" +
+        "lateral (select u.item.i_name as name from unnest(c.orders[0].items) as u(item)) l limit 1";
+
+    testBuilder()
+        .sqlQuery(sql)
+        .unOrdered()
+        .baselineColumns("name")
+        .baselineValues("paper towel")
+        .go();
+  }
+
+  @Test
+  public void testLateralWithAgg() throws Exception {
+    String sql = "select l.name from cp.`lateraljoin/nested-customer.parquet` c,\n" +
+        "lateral (select max(u.item.i_name) as name from unnest(c.orders[0].items) as u(item)) l limit 1";
+
+    testBuilder()
+        .sqlQuery(sql)
+        .unOrdered()
+        .baselineColumns("name")
+        .baselineValues("paper towel")
+        .go();
+  }
+
+  @Test
+  public void testMultiLateralWithComplexProject() throws Exception {
+    String sql = "select l1.name, l2.name as name2 from cp.`lateraljoin/nested-customer.parquet` c,\n" +
+      "lateral (select u.item.i_name as name from unnest(c.orders[0].items) as u(item)) l1," +
+      "lateral (select u.item.i_name as name from unnest(c.orders[0].items) as u(item)) l2 limit 1";
+
+    testBuilder()
+      .sqlQuery(sql)
+      .unOrdered()
+      .baselineColumns("name", "name2")
+      .baselineValues("paper towel", "paper towel")
+      .go();
+  }
 }


[drill] 06/06: DRILL-6461: Added basic data correctness tests for hash agg, and improved operator unit testing framework.

Posted by ar...@apache.org.
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

commit d8f9fb6a5cf22a01fa3f48bd40e7dbeb3cb6e4e4
Author: Timothy Farkas <ti...@apache.org>
AuthorDate: Mon Jun 4 10:45:12 2018 -0700

    DRILL-6461: Added basic data correctness tests for hash agg, and improved operator unit testing framework.
    
    git closes #1344
---
 .../drill/exec/physical/impl/TopN/TopNBatch.java   |  12 -
 .../physical/impl/common/HashTableTemplate.java    |  12 +-
 .../physical/impl/project/ProjectRecordBatch.java  |   2 +
 .../physical/impl/svremover/AbstractCopier.java    |  26 +-
 .../physical/impl/svremover/AbstractSV2Copier.java |   4 +-
 .../physical/impl/svremover/AbstractSV4Copier.java |   4 +-
 .../drill/exec/physical/impl/svremover/Copier.java |   4 +-
 .../physical/impl/svremover/GenericCopier.java     |   7 +-
 .../physical/impl/svremover/StraightCopier.java    |   3 +-
 .../apache/drill/exec/record/RecordBatchSizer.java |  17 ++
 .../physical/impl/BaseTestOpBatchEmitOutcome.java  |   2 +-
 .../drill/exec/physical/impl/MockRecordBatch.java  | 196 ++++++++++---
 .../physical/impl/agg/TestAggWithAnyValue.java     |   6 +-
 .../exec/physical/impl/agg/TestHashAggBatch.java   | 212 ++++++++++++++
 .../physical/impl/common/HashPartitionTest.java    |  70 +++--
 .../exec/physical/impl/join/TestHashJoinJPPD.java  |   6 +-
 .../exec/physical/impl/join/TestHashJoinSpill.java |   8 +-
 .../physical/impl/limit/TestLimitOperator.java     |  16 +-
 .../impl/svremover/AbstractGenericCopierTest.java  |  90 +++---
 .../physical/impl/svremover/GenericCopierTest.java |   4 +-
 .../impl/svremover/GenericSV2BatchCopierTest.java  |   4 +-
 .../impl/svremover/GenericSV2CopierTest.java       |   4 +-
 .../impl/svremover/GenericSV4CopierTest.java       |   9 +-
 .../physical/unit/BasicPhysicalOpUnitTest.java     |  30 +-
 .../exec/physical/unit/MiniPlanUnitTestBase.java   |   3 +-
 .../exec/physical/unit/TestOutputBatchSize.java    |  80 +++---
 .../columnreaders/TestBatchSizingMemoryUtil.java   |   2 +-
 .../drill/test/LegacyOperatorTestBuilder.java      | 178 ++++++++++++
 .../org/apache/drill/test/OperatorTestBuilder.java | 314 +++++++++++++++++++++
 .../apache/drill/test/OperatorTestBuilderTest.java | 157 +++++++++++
 .../unit => test}/PhysicalOpUnitTestBase.java      | 169 ++---------
 .../org/apache/drill/test/rowSet/RowSetBatch.java  | 111 --------
 .../apache/drill/test/rowSet/RowSetComparison.java | 151 +++++++---
 .../drill/test/rowSet/TestRowSetComparison.java    | 211 ++++++++++++++
 .../main/codegen/templates/FixedValueVectors.java  |   2 +-
 .../codegen/templates/NullableValueVectors.java    |   2 +-
 36 files changed, 1616 insertions(+), 512 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
index 2763f59..2e343b6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
@@ -137,23 +137,11 @@ public class TopNBatch extends AbstractRecordBatch<TopN> {
 
   @Override
   public void buildSchema() throws SchemaChangeException {
-    VectorContainer c = new VectorContainer(oContext);
     IterOutcome outcome = next(incoming);
     switch (outcome) {
       case OK:
       case OK_NEW_SCHEMA:
         for (VectorWrapper<?> w : incoming) {
-          // TODO: Not sure why the special handling for AbstractContainerVector is needed since creation of child
-          // vectors is taken care correctly if the field is retrieved from incoming vector and passed to it rather than
-          // creating a new Field instance just based on name and type.
-          @SuppressWarnings("resource")
-          ValueVector v = c.addOrGet(w.getField());
-          if (v instanceof AbstractContainerVector) {
-            w.getValueVector().makeTransferPair(v);
-            v.clear();
-          }
-        }
-        for (VectorWrapper<?> w : c) {
           @SuppressWarnings("resource")
           ValueVector v = container.addOrGet(w.getField());
           if (v instanceof AbstractContainerVector) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
index 3c418b9..958a0b6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
@@ -545,6 +545,15 @@ public abstract class HashTableTemplate implements HashTable {
 
   @Override
   public void clear() {
+    clear(true);
+  }
+
+  private void clear(boolean close) {
+    if (close) {
+      // If we are closing, we need to clear the htContainerOrig as well.
+      htContainerOrig.clear();
+    }
+
     if (batchHolders != null) {
       for (BatchHolder bh : batchHolders) {
         bh.clear();
@@ -842,7 +851,7 @@ public abstract class HashTableTemplate implements HashTable {
    *
    */
   public void reset() {
-    this.clear(); // Clear all current batch holders and hash table (i.e. free their memory)
+    this.clear(false); // Clear all current batch holders and hash table (i.e. free their memory)
 
     freeIndex = 0; // all batch holders are gone
     // reallocate batch holders, and the hash table to the original size
@@ -852,6 +861,7 @@ public abstract class HashTableTemplate implements HashTable {
     totalIndexSize = 0;
     startIndices = allocMetadataVector(originalTableSize, EMPTY_SLOT);
   }
+
   public void updateIncoming(VectorContainer newIncoming, RecordBatch newIncomingProbe) {
     incomingBuild = newIncoming;
     incomingProbe = newIncomingProbe;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
index dd93325..b459e1c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
@@ -329,6 +329,8 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project> {
       m.setValueCount(count);
     }
 
+    container.setRecordCount(count);
+
     if (complexWriters == null) {
       return;
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractCopier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractCopier.java
index 47ec1cb..a463519 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractCopier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractCopier.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.physical.impl.svremover;
 
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
-import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.vector.AllocationHelper;
@@ -30,7 +30,7 @@ public abstract class AbstractCopier implements Copier {
   protected VectorContainer outgoing;
 
   @Override
-  public void setup(RecordBatch incoming, VectorContainer outgoing) {
+  public void setup(VectorAccessible incoming, VectorContainer outgoing) {
     this.outgoing = outgoing;
 
     final int count = outgoing.getNumberOfColumns();
@@ -43,15 +43,7 @@ public abstract class AbstractCopier implements Copier {
 
   @Override
   public int copyRecords(int index, int recordCount) {
-    for(VectorWrapper<?> out : outgoing){
-      TypeProtos.MajorType type = out.getField().getType();
-      if (!Types.isFixedWidthType(type) || Types.isRepeated(type)) {
-        out.getValueVector().allocateNew();
-      } else {
-        AllocationHelper.allocate(out.getValueVector(), recordCount, 1);
-      }
-    }
-
+    allocateOutgoing(outgoing, recordCount);
     return insertRecords(0, index, recordCount);
   }
 
@@ -91,4 +83,16 @@ public abstract class AbstractCopier implements Copier {
   public abstract void copyEntryIndirect(int inIndex, int outIndex);
 
   public abstract void copyEntry(int inIndex, int outIndex);
+
+  public static void allocateOutgoing(VectorContainer outgoing, int recordCount) {
+    for(VectorWrapper<?> out : outgoing) {
+      TypeProtos.MajorType type = out.getField().getType();
+
+      if (!Types.isFixedWidthType(type) || Types.isRepeated(type)) {
+        out.getValueVector().allocateNew();
+      } else {
+        AllocationHelper.allocate(out.getValueVector(), recordCount, 1);
+      }
+    }
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractSV2Copier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractSV2Copier.java
index 68a0889..d273fd3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractSV2Copier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractSV2Copier.java
@@ -17,8 +17,8 @@
  */
 package org.apache.drill.exec.physical.impl.svremover;
 
-import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.selection.SelectionVector2;
@@ -33,7 +33,7 @@ public abstract class AbstractSV2Copier extends AbstractCopier {
   protected List<TransferPair> transferPairs = new ArrayList<>();
 
   @Override
-  public void setup(RecordBatch incoming, VectorContainer outgoing) {
+  public void setup(VectorAccessible incoming, VectorContainer outgoing) {
     super.setup(incoming, outgoing);
     this.sv2 = incoming.getSelectionVector2();
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractSV4Copier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractSV4Copier.java
index 56e2586..970f970 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractSV4Copier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/AbstractSV4Copier.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.svremover;
 
-import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.selection.SelectionVector4;
@@ -30,7 +30,7 @@ public abstract class AbstractSV4Copier extends AbstractCopier {
   private SelectionVector4 sv4;
 
   @Override
-  public void setup(RecordBatch incoming, VectorContainer outgoing) {
+  public void setup(VectorAccessible incoming, VectorContainer outgoing) {
     super.setup(incoming, outgoing);
     this.sv4 = incoming.getSelectionVector4();
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/Copier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/Copier.java
index 92dea70..f8934e5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/Copier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/Copier.java
@@ -17,11 +17,11 @@
  */
 package org.apache.drill.exec.physical.impl.svremover;
 
-import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorAccessible;
 
 public interface Copier {
-  void setup(RecordBatch incoming, VectorContainer outgoing);
+  void setup(VectorAccessible incoming, VectorContainer outgoing);
   int copyRecords(int index, int recordCount);
   int appendRecord(int index);
   int appendRecords(int index, int recordCount);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/GenericCopier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/GenericCopier.java
index 72516e0..f64a11e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/GenericCopier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/GenericCopier.java
@@ -17,11 +17,13 @@
  */
 package org.apache.drill.exec.physical.impl.svremover;
 
-import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.vector.ValueVector;
 
+import static org.apache.drill.exec.physical.impl.svremover.AbstractCopier.allocateOutgoing;
+
 public class GenericCopier implements Copier {
   private ValueVector[] vvOut;
   private ValueVector[] vvIn;
@@ -29,7 +31,7 @@ public class GenericCopier implements Copier {
   private VectorContainer outgoing;
 
   @Override
-  public void setup(RecordBatch incoming, VectorContainer outgoing) {
+  public void setup(VectorAccessible incoming, VectorContainer outgoing) {
     this.outgoing = outgoing;
 
     final int count = outgoing.getNumberOfColumns();
@@ -53,6 +55,7 @@ public class GenericCopier implements Copier {
 
   @Override
   public int copyRecords(int index, int recordCount) {
+    allocateOutgoing(outgoing, recordCount);
     return insertRecords(0, index, recordCount);
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/StraightCopier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/StraightCopier.java
index 33f2a96..cecfc5a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/StraightCopier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/StraightCopier.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.physical.impl.svremover;
 import com.google.common.collect.Lists;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.TransferPair;
+import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.vector.SchemaChangeCallBack;
@@ -39,7 +40,7 @@ public class StraightCopier implements Copier {
     }
 
     @Override
-    public void setup(RecordBatch incoming, VectorContainer outgoing) {
+    public void setup(VectorAccessible incoming, VectorContainer outgoing) {
       for(VectorWrapper<?> vv : incoming){
         TransferPair tp = vv.getValueVector().makeTransferPair(outputContainer.addOrGet(vv.getField(), callBack));
         pairs.add(tp);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java
index 83287ee..dac80a5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchSizer.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.exec.record;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Set;
 import java.util.Map;
 
@@ -634,6 +636,11 @@ public class RecordBatchSizer {
   private Map<String, ColumnSize> columnSizes = CaseInsensitiveMap.newHashMap();
 
   /**
+   * This field is used by the convenience method {@link #columnsList()}.
+   */
+  private List<ColumnSize> columnSizesList = new ArrayList<>();
+
+  /**
    * Number of records (rows) in the batch.
    */
   private int rowCount;
@@ -715,6 +722,8 @@ public class RecordBatchSizer {
     for (VectorWrapper<?> vw : va) {
       ColumnSize colSize = measureColumn(vw.getValueVector(), "");
       columnSizes.put(vw.getField().getName(), colSize);
+      columnSizesList.add(colSize);
+      stdRowWidth += colSize.getStdDataSizePerEntry();
       netBatchSize += colSize.getTotalNetSize();
       maxSize = Math.max(maxSize, colSize.getTotalDataSize());
       if (colSize.metadata.isNullable()) {
@@ -885,6 +894,14 @@ public class RecordBatchSizer {
   public Map<String, ColumnSize> columns() { return columnSizes; }
 
   /**
+   * This is a convenience method to get the sizes of columns in the same order that the corresponding value vectors
+   * are stored within a {@link org.apache.drill.exec.record.VectorAccessible}.
+   * @return The sizes of columns in the same order that the corresponding value vectors are stored within a
+   * {@link org.apache.drill.exec.record.VectorAccessible}.
+   */
+  public List<ColumnSize> columnsList() { return columnSizesList; }
+
+  /**
    * Compute the "real" width of the row, taking into account each varchar column size
    * (historically capped at 50, and rounded up to power of 2 to match drill buf allocation)
    * and null marking columns.
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/BaseTestOpBatchEmitOutcome.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/BaseTestOpBatchEmitOutcome.java
index 4eaca2b..620a61c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/BaseTestOpBatchEmitOutcome.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/BaseTestOpBatchEmitOutcome.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.physical.impl;
 
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.physical.unit.PhysicalOpUnitTestBase;
+import org.apache.drill.test.PhysicalOpUnitTestBase;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/MockRecordBatch.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/MockRecordBatch.java
index ed7af4c..f3ec7b0 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/MockRecordBatch.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/MockRecordBatch.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl;
 
+import com.google.common.base.Preconditions;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
@@ -29,9 +30,17 @@ import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.test.rowSet.DirectRowSet;
+import org.apache.drill.test.rowSet.IndirectRowSet;
+import org.apache.drill.test.rowSet.RowSet;
 
+import javax.annotation.Nullable;
+import javax.validation.constraints.NotNull;
+import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
 
 public class MockRecordBatch implements CloseableRecordBatch {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordBatch.class);
@@ -39,41 +48,83 @@ public class MockRecordBatch implements CloseableRecordBatch {
   // These resources are owned by this RecordBatch
   protected VectorContainer container;
   protected SelectionVector2 sv2;
+  protected SelectionVector4 sv4;
   private int currentContainerIndex;
   private int currentOutcomeIndex;
   private boolean isDone;
   private boolean limitWithUnnest;
 
   // All the below resources are owned by caller
-  private final List<VectorContainer> allTestContainers;
-  private List<SelectionVector2> allTestContainersSv2;
+  private final List<RowSet> rowSets;
   private final List<IterOutcome> allOutcomes;
   private final FragmentContext context;
   protected final OperatorContext oContext;
   protected final BufferAllocator allocator;
 
-  public MockRecordBatch(FragmentContext context, OperatorContext oContext,
-                         List<VectorContainer> testContainers, List<IterOutcome> iterOutcomes,
-                         BatchSchema schema) {
+  private MockRecordBatch(@NotNull final FragmentContext context,
+                          @Nullable final OperatorContext oContext,
+                          @NotNull final List<RowSet> testRowSets,
+                          @NotNull final List<IterOutcome> iterOutcomes,
+                          @NotNull final BatchSchema schema,
+                          final boolean dummy) {
+    Preconditions.checkNotNull(testRowSets);
+    Preconditions.checkNotNull(iterOutcomes);
+    Preconditions.checkNotNull(schema);
+
     this.context = context;
     this.oContext = oContext;
-    this.allocator = oContext.getAllocator();
-    this.allTestContainers = testContainers;
+    this.rowSets = testRowSets;
+    this.allocator = context.getAllocator();
     this.container = new VectorContainer(allocator, schema);
     this.allOutcomes = iterOutcomes;
     this.currentContainerIndex = 0;
     this.currentOutcomeIndex = 0;
     this.isDone = false;
-    this.allTestContainersSv2 = null;
-    this.sv2 = null;
   }
 
-  public MockRecordBatch(FragmentContext context, OperatorContext oContext,
-                         List<VectorContainer> testContainers, List<IterOutcome> iterOutcomes,
-                         List<SelectionVector2> testContainersSv2, BatchSchema schema) {
-    this(context, oContext, testContainers, iterOutcomes, schema);
-    allTestContainersSv2 = testContainersSv2;
-    sv2 = (allTestContainersSv2 != null && allTestContainersSv2.size() > 0) ? new SelectionVector2(allocator) : null;
+  @Deprecated
+  public MockRecordBatch(@Nullable final FragmentContext context,
+                         @Nullable final OperatorContext oContext,
+                         @NotNull final List<VectorContainer> testContainers,
+                         @NotNull final List<IterOutcome> iterOutcomes,
+                         final BatchSchema schema) {
+    this(context,
+         oContext,
+         testContainers.stream().
+           map(container -> DirectRowSet.fromContainer(container)).
+           collect(Collectors.toList()),
+         iterOutcomes,
+         schema,
+         true);
+  }
+
+  @Deprecated
+  public MockRecordBatch(@Nullable final FragmentContext context,
+                         @Nullable final OperatorContext oContext,
+                         @NotNull final List<VectorContainer> testContainers,
+                         @NotNull final List<IterOutcome> iterOutcomes,
+                         @NotNull final List<SelectionVector2> selectionVector2s,
+                         final BatchSchema schema) {
+    this(context,
+      oContext,
+      new Supplier<List<RowSet>>() {
+        @Override
+        public List<RowSet> get() {
+          List<RowSet> rowSets = new ArrayList<>();
+
+          for (int index = 0; index < testContainers.size(); index++) {
+            if (index >= selectionVector2s.size()) {
+              rowSets.add(IndirectRowSet.fromContainer(testContainers.get(index)));
+            } else {
+              rowSets.add(IndirectRowSet.fromSv2(testContainers.get(index), selectionVector2s.get(index)));
+            }
+          }
+          return rowSets;
+        }
+      }.get(),
+      iterOutcomes,
+      schema,
+      true);
   }
 
   @Override
@@ -94,7 +145,7 @@ public class MockRecordBatch implements CloseableRecordBatch {
 
   @Override
   public SelectionVector4 getSelectionVector4() {
-    return null;
+    return sv4;
   }
 
   @Override
@@ -146,10 +197,11 @@ public class MockRecordBatch implements CloseableRecordBatch {
       return IterOutcome.NONE;
     }
 
-    IterOutcome currentOutcome = IterOutcome.OK;
+    IterOutcome currentOutcome;
 
-    if (currentContainerIndex < allTestContainers.size()) {
-      final VectorContainer input = allTestContainers.get(currentContainerIndex);
+    if (currentContainerIndex < rowSets.size()) {
+      final RowSet rowSet = rowSets.get(currentContainerIndex);
+      final VectorContainer input = rowSet.container();
       final int recordCount = input.getRecordCount();
       // We need to do this since the downstream operator expects vector reference to be same
       // after first next call in cases when schema is not changed
@@ -158,19 +210,34 @@ public class MockRecordBatch implements CloseableRecordBatch {
         container.clear();
         container = new VectorContainer(allocator, inputSchema);
       }
-      container.transferIn(input);
-      container.setRecordCount(recordCount);
-
-      // Transfer the sv2 as well
-      final SelectionVector2 inputSv2 =
-        (allTestContainersSv2 != null && allTestContainersSv2.size() > 0)
-          ? allTestContainersSv2.get(currentContainerIndex) : null;
-      if (inputSv2 != null) {
-        sv2.allocateNewSafe(inputSv2.getCount());
-        for (int i=0; i<inputSv2.getCount(); ++i) {
-          sv2.setIndex(i, inputSv2.getIndex(i));
-        }
-        sv2.setRecordCount(inputSv2.getCount());
+
+      switch (rowSet.indirectionType()) {
+        case NONE:
+        case TWO_BYTE:
+          container.transferIn(input);
+          container.setRecordCount(recordCount);
+          final SelectionVector2 inputSv2 = ((RowSet.SingleRowSet) rowSet).getSv2();
+
+          if (sv2 != null) {
+            // Operators assume that new values for an Sv2 are transferred in.
+            sv2.allocateNewSafe(inputSv2.getCount());
+            for (int i=0; i<inputSv2.getCount(); ++i) {
+              sv2.setIndex(i, inputSv2.getIndex(i));
+            }
+            sv2.setRecordCount(inputSv2.getCount());
+          } else {
+            sv2 = inputSv2;
+          }
+
+          break;
+        case FOUR_BYTE:
+          // TODO find a clean way to transfer in for this case.
+          container.clear();
+          container = input;
+          sv4 = ((RowSet.HyperRowSet) rowSet).getSv4();
+          break;
+        default:
+          throw new UnsupportedOperationException();
       }
     }
 
@@ -222,4 +289,69 @@ public class MockRecordBatch implements CloseableRecordBatch {
   public void useUnnestKillHandlingForLimit(boolean limitWithUnnest) {
     this.limitWithUnnest = limitWithUnnest;
   }
+
+  public static class Builder {
+    private final List<RowSet> rowSets = new ArrayList<>();
+    private final List<IterOutcome> iterOutcomes = new ArrayList<>();
+
+    private BatchSchema batchSchema;
+    private OperatorContext oContext;
+
+    public Builder() {
+    }
+
+    private Builder sendData(final RowSet rowSet, final IterOutcome outcome) {
+      Preconditions.checkState(batchSchema == null);
+      rowSets.add(rowSet);
+      iterOutcomes.add(outcome);
+      return this;
+    }
+
+    public Builder sendData(final RowSet rowSet) {
+      final IterOutcome outcome = rowSets.isEmpty()? IterOutcome.OK_NEW_SCHEMA: IterOutcome.OK;
+      return sendData(rowSet, outcome);
+    }
+
+    public Builder sendDataWithNewSchema(final RowSet rowSet) {
+      return sendData(rowSet, IterOutcome.OK_NEW_SCHEMA);
+    }
+
+    public Builder sendDataAndEmit(final RowSet rowSet) {
+      return sendData(rowSet, IterOutcome.EMIT);
+    }
+
+    public Builder terminateWithError(IterOutcome errorOutcome) {
+      Preconditions.checkArgument(errorOutcome != IterOutcome.STOP);
+      Preconditions.checkArgument(errorOutcome != IterOutcome.OUT_OF_MEMORY);
+
+      iterOutcomes.add(errorOutcome);
+      return this;
+    }
+
+    public Builder setSchema(final BatchSchema batchSchema) {
+      Preconditions.checkState(!rowSets.isEmpty());
+      this.batchSchema = Preconditions.checkNotNull(batchSchema);
+      return this;
+    }
+
+    public Builder withOperatorContext(final OperatorContext oContext) {
+      this.oContext = Preconditions.checkNotNull(oContext);
+      return this;
+    }
+
+    public MockRecordBatch build(final FragmentContext context) {
+      BatchSchema tempSchema = batchSchema;
+
+      if (tempSchema == null && !rowSets.isEmpty()) {
+        tempSchema = rowSets.get(0).batchSchema();
+      }
+
+      return new MockRecordBatch(context,
+        oContext,
+        rowSets,
+        iterOutcomes,
+        tempSchema,
+        true);
+    }
+  }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAggWithAnyValue.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAggWithAnyValue.java
index 37c0b52..9909bca 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAggWithAnyValue.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAggWithAnyValue.java
@@ -20,7 +20,7 @@ package org.apache.drill.exec.physical.impl.agg;
 
 import com.google.common.collect.Lists;
 import org.apache.drill.exec.physical.config.StreamingAggregate;
-import org.apache.drill.exec.physical.unit.PhysicalOpUnitTestBase;
+import org.apache.drill.test.PhysicalOpUnitTestBase;
 import org.apache.drill.exec.util.JsonStringArrayList;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.OperatorTest;
@@ -59,7 +59,7 @@ public class TestAggWithAnyValue {
               "{ \"age\": {\"min\":43, \"max\":80}, \"city\": \"Palo Alto\", \"de\": \"987654321987654321987654321.13987654321\"," +
               " \"a\": [{\"b\":70, \"c\":85}, {\"b\":90, \"c\":145}], \"m\": [{\"n\": [7, 8, 9]}], \"f\": [{\"g\": {\"h\": [{\"k\": 50}, {\"k\": 60}]}}]," +
               "\"p\": {\"q\": [33, 34, 35]}" + "}]");
-      opTestBuilder()
+      legacyOpTestBuilder()
           .physicalOperator(aggConf)
           .inputDataStreamJson(inputJsonBatches)
           .baselineColumns("age", "any_a")
@@ -146,4 +146,4 @@ public class TestAggWithAnyValue {
           .go();
     }
   }
-}
\ No newline at end of file
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestHashAggBatch.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestHashAggBatch.java
new file mode 100644
index 0000000..2c6976c
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestHashAggBatch.java
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl.agg;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.config.HashAggregate;
+import org.apache.drill.exec.physical.impl.MockRecordBatch;
+import org.apache.drill.exec.planner.physical.AggPrelBase;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.PhysicalOpUnitTestBase;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSetBuilder;
+import org.apache.drill.test.rowSet.schema.SchemaBuilder;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.drill.exec.ExecConstants.HASHAGG_NUM_PARTITIONS_KEY;
+
+public class TestHashAggBatch extends PhysicalOpUnitTestBase {
+  public static final String FIRST_NAME_COL = "firstname";
+  public static final String LAST_NAME_COL = "lastname";
+  public static final String STUFF_COL = "stuff";
+  public static final String TOTAL_STUFF_COL = "totalstuff";
+
+  public static final List<String> FIRST_NAMES = ImmutableList.of(
+    "Strawberry",
+    "Banana",
+    "Mango",
+    "Grape");
+
+  public static final List<String> LAST_NAMES = ImmutableList.of(
+    "Red",
+    "Green",
+    "Blue",
+    "Purple");
+
+  public static final TupleMetadata INT_OUTPUT_SCHEMA = new SchemaBuilder()
+    .add(FIRST_NAME_COL, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.REQUIRED)
+    .add(LAST_NAME_COL, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.REQUIRED)
+    .add(TOTAL_STUFF_COL, TypeProtos.MinorType.BIGINT, TypeProtos.DataMode.OPTIONAL)
+    .buildSchema();
+
+  // TODO remove this in order to test multiple partitions
+  @Before
+  public void setupSimpleSingleBatchSumTestPhase1of2() {
+    operatorFixture.getOptionManager().setLocalOption(HASHAGG_NUM_PARTITIONS_KEY, 1);
+  }
+
+  @Test
+  public void simpleSingleBatchSumTestPhase1of2() throws Exception {
+    batchSumTest(100, Integer.MAX_VALUE, AggPrelBase.OperatorPhase.PHASE_1of2);
+  }
+
+  @Test
+  public void simpleMultiBatchSumTestPhase1of2() throws Exception {
+    batchSumTest(100, 100, AggPrelBase.OperatorPhase.PHASE_1of2);
+  }
+
+  @Test
+  public void simpleSingleBatchSumTestPhase1of1() throws Exception {
+    batchSumTest(100, Integer.MAX_VALUE, AggPrelBase.OperatorPhase.PHASE_1of1);
+  }
+
+  @Test
+  public void simpleMultiBatchSumTestPhase1of1() throws Exception {
+    batchSumTest(100, 100, AggPrelBase.OperatorPhase.PHASE_1of1);
+  }
+
+  @Test
+  public void simpleSingleBatchSumTestPhase2of2() throws Exception {
+    batchSumTest(100, Integer.MAX_VALUE, AggPrelBase.OperatorPhase.PHASE_2of2);
+  }
+
+  @Test
+  public void simpleMultiBatchSumTestPhase2of2() throws Exception {
+    batchSumTest(100, 100, AggPrelBase.OperatorPhase.PHASE_2of2);
+  }
+
+  private void batchSumTest(int totalCount, int maxInputBatchSize, AggPrelBase.OperatorPhase phase) throws Exception {
+    final HashAggregate hashAggregate = createHashAggPhysicalOperator(phase);
+    final List<RowSet> inputRowSets = buildInputRowSets(TypeProtos.MinorType.INT, TypeProtos.DataMode.REQUIRED,
+      totalCount, maxInputBatchSize);
+
+    final MockRecordBatch.Builder rowSetBatchBuilder = new MockRecordBatch.Builder();
+    inputRowSets.forEach(rowSet -> rowSetBatchBuilder.sendData(rowSet));
+    final MockRecordBatch inputRowSetBatch = rowSetBatchBuilder.build(fragContext);
+
+    final RowSet expectedRowSet = buildIntExpectedRowSet(totalCount);
+
+    opTestBuilder()
+      .physicalOperator(hashAggregate)
+      .combineOutputBatches()
+      .unordered()
+      .addUpstreamBatch(inputRowSetBatch)
+      .addExpectedResult(expectedRowSet)
+      .go();
+  }
+
+  private HashAggregate createHashAggPhysicalOperator(AggPrelBase.OperatorPhase phase) {
+    final List<NamedExpression> keyExpressions = Lists.newArrayList(
+      new NamedExpression(SchemaPath.getSimplePath(FIRST_NAME_COL), new FieldReference(FIRST_NAME_COL)),
+      new NamedExpression(SchemaPath.getSimplePath(LAST_NAME_COL), new FieldReference(LAST_NAME_COL)));
+
+    final List<NamedExpression> aggExpressions = Lists.newArrayList(
+      new NamedExpression(
+        new FunctionCall("sum", ImmutableList.of(SchemaPath.getSimplePath(STUFF_COL)),
+          new ExpressionPosition(null, 0)),
+        new FieldReference(TOTAL_STUFF_COL)));
+
+    return new HashAggregate(
+      null,
+      phase,
+      keyExpressions,
+      aggExpressions,
+      0.0f);
+  }
+
+  private TupleMetadata buildInputSchema(TypeProtos.MinorType minorType, TypeProtos.DataMode dataMode) {
+    return new SchemaBuilder()
+      .add(FIRST_NAME_COL, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.REQUIRED)
+      .add(LAST_NAME_COL, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.REQUIRED)
+      .add(STUFF_COL, minorType, dataMode)
+      .buildSchema();
+  }
+
+  private List<RowSet> buildInputRowSets(final TypeProtos.MinorType minorType,
+                                  final TypeProtos.DataMode dataMode,
+                                  final int dataCount,
+                                  final int maxBatchSize) {
+    Preconditions.checkArgument(dataCount > 0);
+    Preconditions.checkArgument(maxBatchSize > 0);
+
+    List<RowSet> inputRowSets = new ArrayList<>();
+    int currentBatchSize = 0;
+    RowSetBuilder inputRowSetBuilder = null;
+
+    for (int multiplier = 1, firstNameIndex = 0; firstNameIndex < FIRST_NAMES.size(); firstNameIndex++) {
+      final String firstName = FIRST_NAMES.get(firstNameIndex);
+
+      for (int lastNameIndex = 0; lastNameIndex < LAST_NAMES.size(); lastNameIndex++, multiplier++) {
+        final String lastName = LAST_NAMES.get(lastNameIndex);
+
+        for (int index = 1; index <= dataCount; index++) {
+          final int num = index * multiplier;
+
+          if (currentBatchSize == 0) {
+            final TupleMetadata inputSchema = buildInputSchema(minorType, dataMode);
+            inputRowSetBuilder = new RowSetBuilder(operatorFixture.allocator(), inputSchema);
+          }
+
+          inputRowSetBuilder.addRow(firstName, lastName, num);
+          currentBatchSize++;
+
+          if (currentBatchSize == maxBatchSize) {
+            final RowSet rowSet = inputRowSetBuilder.build();
+            inputRowSets.add(rowSet);
+            currentBatchSize = 0;
+          }
+        }
+      }
+    }
+
+    if (currentBatchSize != 0) {
+      inputRowSets.add(inputRowSetBuilder.build());
+    }
+
+    return inputRowSets;
+  }
+
+  private RowSet buildIntExpectedRowSet(final int dataCount) {
+    final RowSetBuilder expectedRowSetBuilder = new RowSetBuilder(operatorFixture.allocator(), INT_OUTPUT_SCHEMA);
+
+    for (int multiplier = 1, firstNameIndex = 0; firstNameIndex < FIRST_NAMES.size(); firstNameIndex++) {
+      final String firstName = FIRST_NAMES.get(firstNameIndex);
+
+      for (int lastNameIndex = 0; lastNameIndex < LAST_NAMES.size(); lastNameIndex++, multiplier++) {
+        final String lastName = LAST_NAMES.get(lastNameIndex);
+        final long total = ((dataCount * (dataCount + 1)) / 2) * multiplier;
+
+        expectedRowSetBuilder.addRow(firstName, lastName, total);
+      }
+    }
+
+    return expectedRowSetBuilder.build();
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/common/HashPartitionTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/common/HashPartitionTest.java
index 48fd856..bbe57fa 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/common/HashPartitionTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/common/HashPartitionTest.java
@@ -32,6 +32,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.config.HashJoinPOP;
+import org.apache.drill.exec.physical.impl.MockRecordBatch;
 import org.apache.drill.exec.physical.impl.aggregate.SpilledRecordbatch;
 import org.apache.drill.exec.physical.impl.join.HashJoinMemoryCalculator;
 import org.apache.drill.exec.physical.impl.join.HashJoinMemoryCalculatorImpl;
@@ -41,13 +42,13 @@ import org.apache.drill.exec.planner.logical.DrillJoinRel;
 import org.apache.drill.exec.proto.ExecProtos;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.CloseableRecordBatch;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.test.BaseDirTestWatcher;
 import org.apache.drill.test.OperatorFixture;
 import org.apache.drill.test.rowSet.DirectRowSet;
 import org.apache.drill.test.rowSet.RowSet;
-import org.apache.drill.test.rowSet.RowSetBatch;
 import org.apache.drill.test.rowSet.RowSetBuilder;
 import org.apache.drill.test.rowSet.RowSetComparison;
 import org.junit.Assert;
@@ -68,27 +69,31 @@ public class HashPartitionTest {
       private RowSet probeRowSet;
 
       @Override
-      public RecordBatch createBuildBatch(BatchSchema schema, BufferAllocator allocator) {
-        buildRowSet = new RowSetBuilder(allocator, schema)
+      public CloseableRecordBatch createBuildBatch(BatchSchema schema, FragmentContext context) {
+        buildRowSet = new RowSetBuilder(context.getAllocator(), schema)
           .addRow(1, "green")
           .addRow(3, "red")
           .addRow(2, "blue")
           .build();
-        return new RowSetBatch(buildRowSet);
+        return new MockRecordBatch.Builder().
+          sendData(buildRowSet).
+          build(context);
       }
 
       @Override
-      public void createResultBuildBatch(BatchSchema schema, BufferAllocator allocator) {
+      public void createResultBuildBatch(BatchSchema schema, FragmentContext context) {
       }
 
       @Override
-      public RecordBatch createProbeBatch(BatchSchema schema, BufferAllocator allocator) {
-        probeRowSet = new RowSetBuilder(allocator, schema)
+      public CloseableRecordBatch createProbeBatch(BatchSchema schema, FragmentContext context) {
+        probeRowSet = new RowSetBuilder(context.getAllocator(), schema)
           .addRow(.5, "yellow")
           .addRow(1.5, "blue")
           .addRow(2.5, "black")
           .build();
-        return new RowSetBatch(probeRowSet);
+        return new MockRecordBatch.Builder().
+          sendData(probeRowSet).
+          build(context);
       }
 
       @Override
@@ -114,9 +119,9 @@ public class HashPartitionTest {
 
         final HashJoinMemoryCalculator.BuildSidePartitioning noopCalc = new HashJoinMemoryCalculatorImpl.NoopBuildSidePartitioningImpl();
 
-        hashPartition.appendInnerRow(buildRowSet.container(), 0, 10, noopCalc);
-        hashPartition.appendInnerRow(buildRowSet.container(), 1, 11, noopCalc);
-        hashPartition.appendInnerRow(buildRowSet.container(), 2, 12, noopCalc);
+        hashPartition.appendInnerRow(buildBatch.getContainer(), 0, 10, noopCalc);
+        hashPartition.appendInnerRow(buildBatch.getContainer(), 1, 11, noopCalc);
+        hashPartition.appendInnerRow(buildBatch.getContainer(), 2, 12, noopCalc);
         hashPartition.completeAnInnerBatch(false, false);
         hashPartition.buildContainersHashTableAndHelper();
 
@@ -155,22 +160,24 @@ public class HashPartitionTest {
       private RowSet actualBuildRowSet;
 
       @Override
-      public RecordBatch createBuildBatch(BatchSchema schema, BufferAllocator allocator) {
-        buildRowSet = new RowSetBuilder(allocator, schema)
+      public CloseableRecordBatch createBuildBatch(BatchSchema schema, FragmentContext context) {
+        buildRowSet = new RowSetBuilder(context.getAllocator(), schema)
           .addRow(1, "green")
           .addRow(3, "red")
           .addRow(2, "blue")
           .build();
-        return new RowSetBatch(buildRowSet);
+        return new MockRecordBatch.Builder().
+          sendData(buildRowSet).
+          build(context);
       }
 
       @Override
-      public void createResultBuildBatch(BatchSchema schema, BufferAllocator allocator) {
+      public void createResultBuildBatch(BatchSchema schema, FragmentContext context) {
         final BatchSchema newSchema = BatchSchema.newBuilder()
           .addFields(schema)
           .addField(MaterializedField.create(HashPartition.HASH_VALUE_COLUMN_NAME, HashPartition.HVtype))
           .build();
-        actualBuildRowSet = new RowSetBuilder(allocator, newSchema)
+        actualBuildRowSet = new RowSetBuilder(context.getAllocator(), newSchema)
           .addRow(1, "green", 10)
           .addRow(3, "red", 11)
           .addRow(2, "blue", 12)
@@ -178,13 +185,15 @@ public class HashPartitionTest {
       }
 
       @Override
-      public RecordBatch createProbeBatch(BatchSchema schema, BufferAllocator allocator) {
-        probeRowSet = new RowSetBuilder(allocator, schema)
+      public CloseableRecordBatch createProbeBatch(BatchSchema schema, FragmentContext context) {
+        probeRowSet = new RowSetBuilder(context.getAllocator(), schema)
           .addRow(.5, "yellow")
           .addRow(1.5, "blue")
           .addRow(2.5, "black")
           .build();
-        return new RowSetBatch(probeRowSet);
+        return new MockRecordBatch.Builder().
+          sendData(probeRowSet).
+          build(context);
       }
 
       @Override
@@ -210,9 +219,9 @@ public class HashPartitionTest {
 
         final HashJoinMemoryCalculator.BuildSidePartitioning noopCalc = new HashJoinMemoryCalculatorImpl.NoopBuildSidePartitioningImpl();
 
-        hashPartition.appendInnerRow(buildRowSet.container(), 0, 10, noopCalc);
-        hashPartition.appendInnerRow(buildRowSet.container(), 1, 11, noopCalc);
-        hashPartition.appendInnerRow(buildRowSet.container(), 2, 12, noopCalc);
+        hashPartition.appendInnerRow(buildBatch.getContainer(), 0, 10, noopCalc);
+        hashPartition.appendInnerRow(buildBatch.getContainer(), 1, 11, noopCalc);
+        hashPartition.appendInnerRow(buildBatch.getContainer(), 2, 12, noopCalc);
         hashPartition.completeAnInnerBatch(false, false);
         hashPartition.spillThisPartition();
         final String spillFile = hashPartition.getSpillFile();
@@ -260,15 +269,17 @@ public class HashPartitionTest {
         MaterializedField buildColB = MaterializedField.create("buildColB", Types.required(TypeProtos.MinorType.VARCHAR));
         List<MaterializedField> buildCols = Lists.newArrayList(buildColA, buildColB);
         final BatchSchema buildSchema = new BatchSchema(BatchSchema.SelectionVectorMode.NONE, buildCols);
-        final RecordBatch buildBatch = testCase.createBuildBatch(buildSchema, allocator);
-        testCase.createResultBuildBatch(buildSchema, allocator);
+        final CloseableRecordBatch buildBatch = testCase.createBuildBatch(buildSchema, operatorContext.getFragmentContext());
+        buildBatch.next();
+        testCase.createResultBuildBatch(buildSchema, operatorContext.getFragmentContext());
 
         // Create probe batch
         MaterializedField probeColA = MaterializedField.create("probeColA", Types.required(TypeProtos.MinorType.FLOAT4));
         MaterializedField probeColB = MaterializedField.create("probeColB", Types.required(TypeProtos.MinorType.VARCHAR));
         List<MaterializedField> probeCols = Lists.newArrayList(probeColA, probeColB);
         final BatchSchema probeSchema = new BatchSchema(BatchSchema.SelectionVectorMode.NONE, probeCols);
-        final RecordBatch probeBatch = testCase.createProbeBatch(probeSchema, allocator);
+        final CloseableRecordBatch probeBatch = testCase.createProbeBatch(probeSchema, operatorContext.getFragmentContext());
+        probeBatch.next();
 
         final LogicalExpression buildColExpression = SchemaPath.getSimplePath(buildColB.getName());
         final LogicalExpression probeColExpression = SchemaPath.getSimplePath(probeColB.getName());
@@ -285,14 +296,17 @@ public class HashPartitionTest {
         baseHashTable.updateIncoming(buildBatch, probeBatch);
 
         testCase.run(spillSet, buildSchema, probeSchema, buildBatch, probeBatch, baseHashTable, context, operatorContext);
+
+        buildBatch.close();
+        probeBatch.close();
       }
     }
   }
 
   interface HashPartitionTestCase {
-    RecordBatch createBuildBatch(BatchSchema schema, BufferAllocator allocator);
-    void createResultBuildBatch(BatchSchema schema, BufferAllocator allocator);
-    RecordBatch createProbeBatch(BatchSchema schema, BufferAllocator allocator);
+    CloseableRecordBatch createBuildBatch(BatchSchema schema, FragmentContext context);
+    void createResultBuildBatch(BatchSchema schema, FragmentContext context);
+    CloseableRecordBatch createProbeBatch(BatchSchema schema, FragmentContext context);
 
     void run(SpillSet spillSet,
              BatchSchema buildSchema,
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinJPPD.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinJPPD.java
index aae566b..d3d1487 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinJPPD.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinJPPD.java
@@ -22,10 +22,10 @@ import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.drill.categories.OperatorTest;
 import org.apache.drill.categories.SlowTest;
 import org.apache.drill.exec.physical.config.HashJoinPOP;
-import org.apache.drill.exec.physical.unit.PhysicalOpUnitTestBase;
 import org.apache.drill.exec.work.filter.BloomFilter;
 import org.apache.drill.exec.work.filter.BloomFilterDef;
 import org.apache.drill.exec.work.filter.RuntimeFilterDef;
+import org.apache.drill.test.PhysicalOpUnitTestBase;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -58,7 +58,7 @@ public class TestHashJoinJPPD extends PhysicalOpUnitTestBase {
     for ( int cnt = 1; cnt <= numRows; cnt++ ) {
       leftTable.add("[{\"lft\": " + cnt + ", \"a\" : \"a string\"}]");
     }
-    opTestBuilder()
+    legacyOpTestBuilder()
       .physicalOperator(joinConf)
       .inputDataStreamsJson(Lists.newArrayList(leftTable,rightTable))
       .baselineColumns("lft", "a", "b", "rgt")
@@ -91,7 +91,7 @@ public class TestHashJoinJPPD extends PhysicalOpUnitTestBase {
     for ( int cnt = 1; cnt <= numRows; cnt++ ) {
       leftTable.add("[{\"lft\": " + cnt + ", \"a\" : \"a string\"}]");
     }
-    opTestBuilder()
+    legacyOpTestBuilder()
       .physicalOperator(joinConf)
       .inputDataStreamsJson(Lists.newArrayList(leftTable,rightTable))
       .baselineColumns("lft", "a", "b", "rgt")
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinSpill.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinSpill.java
index 0c08611..67d214f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinSpill.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoinSpill.java
@@ -23,7 +23,7 @@ import org.apache.drill.categories.OperatorTest;
 import org.apache.drill.categories.SlowTest;
 
 import org.apache.drill.exec.physical.config.HashJoinPOP;
-import org.apache.drill.exec.physical.unit.PhysicalOpUnitTestBase;
+import org.apache.drill.test.PhysicalOpUnitTestBase;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -52,7 +52,7 @@ public class TestHashJoinSpill extends PhysicalOpUnitTestBase {
       rightTable.add("[{\"rgt\": " + cnt + ", \"b\" : \"a string\"}]");
     }
 
-    opTestBuilder()
+    legacyOpTestBuilder()
       .physicalOperator(joinConf)
       .inputDataStreamsJson(Lists.newArrayList(leftTable,rightTable))
       .baselineColumns("lft", "a", "b", "rgt")
@@ -79,7 +79,7 @@ public class TestHashJoinSpill extends PhysicalOpUnitTestBase {
       rightTable.add("[{\"rgt\": " + cnt + ", \"b\" : \"a string\"}]");
     }
 
-    opTestBuilder()
+    legacyOpTestBuilder()
       .physicalOperator(joinConf)
       .inputDataStreamsJson(Lists.newArrayList(leftTable,rightTable))
       .baselineColumns("lft", "a", "b", "rgt")
@@ -110,7 +110,7 @@ public class TestHashJoinSpill extends PhysicalOpUnitTestBase {
       // rightTable.add("[{\"rgt\": " + cnt + ", \"b\" : \"a string\"}]");
     }
 
-    opTestBuilder()
+    legacyOpTestBuilder()
       .physicalOperator(joinConf)
       .inputDataStreamsJson(Lists.newArrayList(leftTable,rightTable))
       .baselineColumns("lft", "a", "b", "rgt")
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestLimitOperator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestLimitOperator.java
index 7225edc..3dbd1cc 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestLimitOperator.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/limit/TestLimitOperator.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.physical.impl.limit;
 
 import com.google.common.collect.Lists;
 import org.apache.drill.exec.physical.config.Limit;
-import org.apache.drill.exec.physical.unit.PhysicalOpUnitTestBase;
+import org.apache.drill.test.PhysicalOpUnitTestBase;
 import org.apache.drill.test.BaseDirTestWatcher;
 import org.apache.drill.test.ClientFixture;
 import org.apache.drill.test.ClusterFixture;
@@ -55,7 +55,7 @@ public class TestLimitOperator extends PhysicalOpUnitTestBase {
     List<String> inputJsonBatches = Lists.newArrayList(
       "[{\"a\": 5, \"b\" : 1 }]",
       "[{\"a\": 5, \"b\" : 5},{\"a\": 3, \"b\" : 8}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
       .physicalOperator(limitConf)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b")
@@ -71,7 +71,7 @@ public class TestLimitOperator extends PhysicalOpUnitTestBase {
     List<String> inputJsonBatches = Lists.newArrayList(
       "[{\"a\": 5, \"b\" : 1 }]",
       "[{\"a\": 5, \"b\" : 5},{\"a\": 3, \"b\" : 8}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
       .physicalOperator(limitConf)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b")
@@ -85,7 +85,7 @@ public class TestLimitOperator extends PhysicalOpUnitTestBase {
     List<String> inputJsonBatches = Lists.newArrayList(
       "[{\"a\": 5, \"b\" : 1 }]",
       "[{\"a\": 5, \"b\" : 5},{\"a\": 3, \"b\" : 8}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
       .physicalOperator(limitConf)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b")
@@ -99,7 +99,7 @@ public class TestLimitOperator extends PhysicalOpUnitTestBase {
     List<String> inputJsonBatches = Lists.newArrayList(
       "[{\"a\": 5, \"b\" : 1 }]",
       "[{\"a\": 5, \"b\" : 5},{\"a\": 3, \"b\" : 8}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
       .physicalOperator(limitConf)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b")
@@ -114,7 +114,7 @@ public class TestLimitOperator extends PhysicalOpUnitTestBase {
     List<String> inputJsonBatches = Lists.newArrayList(
       "[{\"a\": 5, \"b\" : 1 }]",
       "[{\"a\": 5, \"b\" : 5},{\"a\": 3, \"b\" : 8}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
       .physicalOperator(limitConf)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b")
@@ -130,7 +130,7 @@ public class TestLimitOperator extends PhysicalOpUnitTestBase {
     List<String> inputJsonBatches = Lists.newArrayList(
       "[{\"a\": 5, \"b\" : 1 }]",
       "[{\"a\": 5, \"b\" : 5},{\"a\": 3, \"b\" : 8}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
       .physicalOperator(limitConf)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b")
@@ -144,7 +144,7 @@ public class TestLimitOperator extends PhysicalOpUnitTestBase {
     List<String> inputJsonBatches = Lists.newArrayList(
       "[{\"a\": 5, \"b\" : 1 }]",
       "[{\"a\": 5, \"b\" : 5},{\"a\": 3, \"b\" : 8}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
       .physicalOperator(limitConf)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b")
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/AbstractGenericCopierTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/AbstractGenericCopierTest.java
index 7d444b4..292af20 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/AbstractGenericCopierTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/AbstractGenericCopierTest.java
@@ -20,77 +20,99 @@ package org.apache.drill.exec.physical.impl.svremover;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.memory.RootAllocator;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.impl.MockRecordBatch;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.VectorContainer;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.vector.SchemaChangeCallBack;
+import org.apache.drill.test.BaseDirTestWatcher;
+import org.apache.drill.test.OperatorFixture;
+import org.apache.drill.test.rowSet.DirectRowSet;
 import org.apache.drill.test.rowSet.RowSet;
-import org.apache.drill.test.rowSet.RowSetBatch;
 import org.apache.drill.test.rowSet.RowSetBuilder;
 import org.apache.drill.test.rowSet.RowSetComparison;
 import org.apache.drill.test.rowSet.schema.SchemaBuilder;
+import org.junit.Rule;
 import org.junit.Test;
 
 public abstract class AbstractGenericCopierTest {
+  @Rule
+  public final BaseDirTestWatcher baseDirTestWatcher = new BaseDirTestWatcher();
+
   @Test
-  public void testCopyRecords() throws SchemaChangeException {
-    try (RootAllocator allocator = new RootAllocator(10_000_000)) {
-      final TupleMetadata batchSchema = createTestSchema(BatchSchema.SelectionVectorMode.NONE);
+  public void testCopyRecords() throws Exception {
+    try (OperatorFixture operatorFixture = new OperatorFixture.Builder(baseDirTestWatcher).build()) {
+      final BufferAllocator allocator = operatorFixture.allocator();
+      final BatchSchema batchSchema = createTestSchema(BatchSchema.SelectionVectorMode.NONE);
       final RowSet srcRowSet = createSrcRowSet(allocator);
-      final RowSet destRowSet = new RowSetBuilder(allocator, batchSchema).build();
-      final VectorContainer destContainer = destRowSet.container();
-      final Copier copier = createCopier(new RowSetBatch(srcRowSet), destContainer, null);
+      final VectorContainer destContainer = new VectorContainer(allocator, batchSchema);
+
+      destContainer.setRecordCount(0);
       final RowSet expectedRowSet = createExpectedRowset(allocator);
 
-      copier.copyRecords(0, 3);
+      MockRecordBatch mockRecordBatch = null;
 
       try {
-        new RowSetComparison(expectedRowSet).verify(destRowSet);
+        mockRecordBatch = new MockRecordBatch.Builder().
+          sendData(srcRowSet).
+          build(operatorFixture.getFragmentContext());
+        mockRecordBatch.next();
+        final Copier copier = createCopier(mockRecordBatch, destContainer, null);
+        copier.copyRecords(0, 3);
+
+        new RowSetComparison(expectedRowSet).verify(DirectRowSet.fromContainer(destContainer));
       } finally {
-        srcRowSet.clear();
-
-        if (srcRowSet instanceof RowSet.HyperRowSet) {
-          ((RowSet.HyperRowSet)srcRowSet).getSv4().clear();
+        if (mockRecordBatch != null) {
+          mockRecordBatch.close();
         }
 
-        destRowSet.clear();
+        srcRowSet.clear();
+        destContainer.clear();
         expectedRowSet.clear();
       }
     }
   }
 
   @Test
-  public void testAppendRecords() throws SchemaChangeException {
-    try (RootAllocator allocator = new RootAllocator(10_000_000)) {
-      final TupleMetadata batchSchema = createTestSchema(BatchSchema.SelectionVectorMode.NONE);
+  public void testAppendRecords() throws Exception {
+    try (OperatorFixture operatorFixture = new OperatorFixture.Builder(baseDirTestWatcher).build()) {
+      final BufferAllocator allocator = operatorFixture.allocator();
+      final BatchSchema batchSchema = createTestSchema(BatchSchema.SelectionVectorMode.NONE);
       final RowSet srcRowSet = createSrcRowSet(allocator);
-      final RowSet destRowSet = new RowSetBuilder(allocator, batchSchema).build();
-      final VectorContainer destContainer = destRowSet.container();
-      final Copier copier = createCopier(new RowSetBatch(srcRowSet), destContainer, null);
+      final VectorContainer destContainer = new VectorContainer(allocator, batchSchema);
+
+      AbstractCopier.allocateOutgoing(destContainer, 3);
+
+      destContainer.setRecordCount(0);
       final RowSet expectedRowSet = createExpectedRowset(allocator);
 
-      copier.appendRecord(0);
-      copier.appendRecords(1, 2);
+      MockRecordBatch mockRecordBatch = null;
 
       try {
-        new RowSetComparison(expectedRowSet).verify(destRowSet);
+        mockRecordBatch = new MockRecordBatch.Builder().
+          sendData(srcRowSet).
+          build(operatorFixture.getFragmentContext());
+        mockRecordBatch.next();
+        final Copier copier = createCopier(mockRecordBatch, destContainer, null);
+        copier.appendRecord(0);
+        copier.appendRecords(1, 2);
+
+        new RowSetComparison(expectedRowSet).verify(DirectRowSet.fromContainer(destContainer));
       } finally {
-        srcRowSet.clear();
-
-        if (srcRowSet instanceof RowSet.HyperRowSet) {
-          ((RowSet.HyperRowSet)srcRowSet).getSv4().clear();
+        if (mockRecordBatch != null) {
+          mockRecordBatch.close();
         }
 
-        destRowSet.clear();
+        srcRowSet.clear();
+        destContainer.clear();
         expectedRowSet.clear();
       }
     }
   }
 
-  public abstract RowSet createSrcRowSet(RootAllocator allocator) throws SchemaChangeException;
+  public abstract RowSet createSrcRowSet(BufferAllocator allocator) throws SchemaChangeException;
 
   public Copier createCopier(RecordBatch incoming, VectorContainer outputContainer,
                                       SchemaChangeCallBack callback) {
@@ -117,7 +139,7 @@ public abstract class AbstractGenericCopierTest {
     return new Object[]{106, "black", new float[]{.75f}, new String[]{"4a"}};
   }
 
-  private RowSet createExpectedRowset(RootAllocator allocator) {
+  public RowSet createExpectedRowset(BufferAllocator allocator) {
     return new RowSetBuilder(allocator, createTestSchema(BatchSchema.SelectionVectorMode.NONE))
       .addRow(row1())
       .addRow(row2())
@@ -125,7 +147,7 @@ public abstract class AbstractGenericCopierTest {
       .build();
   }
 
-  protected TupleMetadata createTestSchema(BatchSchema.SelectionVectorMode mode) {
+  protected BatchSchema createTestSchema(BatchSchema.SelectionVectorMode mode) {
     MaterializedField colA = MaterializedField.create("colA", Types.required(TypeProtos.MinorType.INT));
     MaterializedField colB = MaterializedField.create("colB", Types.required(TypeProtos.MinorType.VARCHAR));
     MaterializedField colC = MaterializedField.create("colC", Types.repeated(TypeProtos.MinorType.FLOAT4));
@@ -136,6 +158,6 @@ public abstract class AbstractGenericCopierTest {
       .add(colC)
       .add(colD)
       .withSVMode(mode)
-      .buildSchema();
+      .build();
   }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/GenericCopierTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/GenericCopierTest.java
index d6c38e7..2636490 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/GenericCopierTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/GenericCopierTest.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.svremover;
 
-import org.apache.drill.exec.memory.RootAllocator;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.VectorContainer;
@@ -27,7 +27,7 @@ import org.apache.drill.test.rowSet.RowSetBuilder;
 
 public class GenericCopierTest extends AbstractGenericCopierTest {
   @Override
-  public RowSet createSrcRowSet(RootAllocator allocator) {
+  public RowSet createSrcRowSet(BufferAllocator allocator) {
     return new RowSetBuilder(allocator, createTestSchema(BatchSchema.SelectionVectorMode.NONE))
       .addRow(row1())
       .addRow(row2())
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/GenericSV2BatchCopierTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/GenericSV2BatchCopierTest.java
index 748e0d0..2fec0e5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/GenericSV2BatchCopierTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/GenericSV2BatchCopierTest.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.svremover;
 
-import org.apache.drill.exec.memory.RootAllocator;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.test.rowSet.RowSet;
 import org.apache.drill.test.rowSet.RowSetBuilder;
@@ -29,7 +29,7 @@ import org.apache.drill.test.rowSet.RowSetBuilder;
 public class GenericSV2BatchCopierTest extends AbstractGenericCopierTest {
 
   @Override
-  public RowSet createSrcRowSet(RootAllocator allocator) {
+  public RowSet createSrcRowSet(BufferAllocator allocator) {
     return new RowSetBuilder(allocator, createTestSchema(BatchSchema.SelectionVectorMode.TWO_BYTE))
       .addSelection(true, row1())
       .addRow(row2())
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/GenericSV2CopierTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/GenericSV2CopierTest.java
index b2f0e51..42182e9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/GenericSV2CopierTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/GenericSV2CopierTest.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.svremover;
 
-import org.apache.drill.exec.memory.RootAllocator;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.test.rowSet.RowSet;
 import org.apache.drill.test.rowSet.RowSetBuilder;
@@ -25,7 +25,7 @@ import org.apache.drill.test.rowSet.RowSetBuilder;
 public class GenericSV2CopierTest extends AbstractGenericCopierTest {
 
   @Override
-  public RowSet createSrcRowSet(RootAllocator allocator) {
+  public RowSet createSrcRowSet(BufferAllocator allocator) {
     return new RowSetBuilder(allocator, createTestSchema(BatchSchema.SelectionVectorMode.TWO_BYTE))
       .addRow(row1())
       .addSelection(false, row4())
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/GenericSV4CopierTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/GenericSV4CopierTest.java
index a5f5bb7..46edab7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/GenericSV4CopierTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/svremover/GenericSV4CopierTest.java
@@ -19,11 +19,10 @@ package org.apache.drill.exec.physical.impl.svremover;
 
 import io.netty.buffer.DrillBuf;
 import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.memory.RootAllocator;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.ExpandableHyperContainer;
 import org.apache.drill.exec.record.VectorContainer;
-import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.test.rowSet.HyperRowSetImpl;
 import org.apache.drill.test.rowSet.RowSet;
@@ -32,8 +31,8 @@ import org.apache.drill.test.rowSet.RowSetBuilder;
 public class GenericSV4CopierTest extends AbstractGenericCopierTest {
 
   @Override
-  public RowSet createSrcRowSet(RootAllocator allocator) throws SchemaChangeException {
-    final TupleMetadata batchSchema = createTestSchema(BatchSchema.SelectionVectorMode.NONE);
+  public RowSet createSrcRowSet(BufferAllocator allocator) throws SchemaChangeException {
+    final BatchSchema batchSchema = createTestSchema(BatchSchema.SelectionVectorMode.NONE);
     final DrillBuf drillBuf = allocator.buffer(4 * 3);
     final SelectionVector4 sv4 = new SelectionVector4(drillBuf, 3, Character.MAX_VALUE);
 
@@ -52,10 +51,12 @@ public class GenericSV4CopierTest extends AbstractGenericCopierTest {
 
     final ExpandableHyperContainer hyperContainer = new ExpandableHyperContainer(batch1);
     hyperContainer.addBatch(batch2);
+    hyperContainer.setRecordCount(5);
 
     sv4.set(0, 0, 0);
     sv4.set(1, 1, 0);
     sv4.set(2, 1, 2);
+    sv4.setCount(3);
 
     return new HyperRowSetImpl(hyperContainer, sv4);
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/BasicPhysicalOpUnitTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/BasicPhysicalOpUnitTest.java
index 0a8cd88..363a939 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/BasicPhysicalOpUnitTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/BasicPhysicalOpUnitTest.java
@@ -38,6 +38,8 @@ import org.apache.drill.exec.physical.config.StreamingAggregate;
 import org.apache.drill.exec.physical.config.TopN;
 import org.apache.drill.exec.physical.config.FlattenPOP;
 import org.apache.drill.exec.planner.physical.AggPrelBase;
+import org.apache.drill.test.LegacyOperatorTestBuilder;
+import org.apache.drill.test.PhysicalOpUnitTestBase;
 import org.junit.Ignore;
 import org.junit.Test;
 
@@ -51,7 +53,7 @@ public class BasicPhysicalOpUnitTest extends PhysicalOpUnitTestBase {
     List<String> jsonBatches = Lists.newArrayList(
         "[{\"x\": 5 },{\"x\": 10 }]",
         "[{\"x\": 20 },{\"x\": 30 },{\"x\": 40 }]");
-    opTestBuilder()
+    legacyOpTestBuilder()
         .physicalOperator(projectConf)
         .inputDataStreamJson(jsonBatches)
         .baselineColumns("x")
@@ -69,7 +71,7 @@ public class BasicPhysicalOpUnitTest extends PhysicalOpUnitTestBase {
     List<String> jsonBatches = Lists.newArrayList(
         "[{\"json_col\": \"{ \\\"a\\\" : 1 }\"}]",
         "[{\"json_col\": \"{ \\\"a\\\" : 5 }\"}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
         .physicalOperator(projectConf)
         .inputDataStreamJson(jsonBatches)
         .baselineColumns("complex_col")
@@ -91,7 +93,7 @@ public class BasicPhysicalOpUnitTest extends PhysicalOpUnitTestBase {
     List<String> rightJsonBatches = Lists.newArrayList(
         "[{\"x1\": 5, \"a2\" : \"asdf\"}]",
         "[{\"x1\": 6, \"a2\" : \"qwerty\"},{\"x1\": 5, \"a2\" : \"12345\"}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
         .physicalOperator(joinConf)
         .inputDataStreamsJson(Lists.newArrayList(leftJsonBatches, rightJsonBatches))
         .baselineColumns("x", "a", "a2", "x1")
@@ -116,7 +118,7 @@ public class BasicPhysicalOpUnitTest extends PhysicalOpUnitTestBase {
     List<String> rightJsonBatches = Lists.newArrayList(
         "[{\"x1\": 5, \"a2\" : \"asdf\"}]",
         "[{\"x1\": 5, \"a2\" : \"12345\"}, {\"x1\": 6, \"a2\" : \"qwerty\"}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
         .physicalOperator(joinConf)
         .inputDataStreamsJson(Lists.newArrayList(leftJsonBatches, rightJsonBatches))
         .baselineColumns("x", "a", "a2", "x1")
@@ -135,7 +137,7 @@ public class BasicPhysicalOpUnitTest extends PhysicalOpUnitTestBase {
     List<String> inputJsonBatches = Lists.newArrayList(
         "[{\"a\": 5, \"b\" : 1 }]",
         "[{\"a\": 5, \"b\" : 5},{\"a\": 3, \"b\" : 8}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
         .physicalOperator(aggConf)
         .inputDataStreamJson(inputJsonBatches)
         .baselineColumns("b_sum", "a")
@@ -150,7 +152,7 @@ public class BasicPhysicalOpUnitTest extends PhysicalOpUnitTestBase {
     List<String> inputJsonBatches = Lists.newArrayList(
         "[{\"a\": 5, \"b\" : 1 }]",
         "[{\"a\": 5, \"b\" : 5},{\"a\": 3, \"b\" : 8}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
         .physicalOperator(aggConf)
         .inputDataStreamJson(inputJsonBatches)
         .baselineColumns("b_sum", "a")
@@ -165,7 +167,7 @@ public class BasicPhysicalOpUnitTest extends PhysicalOpUnitTestBase {
     List<String> inputJsonBatches = Lists.newArrayList(
         "[{\"a\": {\"b\" : 1 }}]",
         "[{\"a\": {\"b\" : 5}},{\"a\": {\"b\" : 8}}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
         .physicalOperator(complexToJson)
         .inputDataStreamJson(inputJsonBatches)
         .baselineColumns("a")
@@ -182,7 +184,7 @@ public class BasicPhysicalOpUnitTest extends PhysicalOpUnitTestBase {
         "[{\"a\": 5, \"b\" : 1 }]",
         "[{\"a\": 5, \"b\" : 5},{\"a\": 3, \"b\" : 8}]",
         "[{\"a\": 40, \"b\" : 3},{\"a\": 13, \"b\" : 100}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
         .physicalOperator(filterConf)
         .inputDataStreamJson(inputJsonBatches)
         .baselineColumns("a", "b")
@@ -206,7 +208,7 @@ public class BasicPhysicalOpUnitTest extends PhysicalOpUnitTestBase {
       inputJsonBatches.add(batchString.toString());
     }
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
             .physicalOperator(flatten)
             .inputDataStreamJson(inputJsonBatches)
             .baselineColumns("a", "b")
@@ -225,7 +227,7 @@ public class BasicPhysicalOpUnitTest extends PhysicalOpUnitTestBase {
         "[{\"a\": 5, \"b\" : 1 }]",
         "[{\"a\": 5, \"b\" : 5},{\"a\": 3, \"b\" : 8}]",
         "[{\"a\": 40, \"b\" : 3},{\"a\": 13, \"b\" : 100}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
         .physicalOperator(sortConf)
         .maxAllocation(15_000_000L)
         .inputDataStreamJson(inputJsonBatches)
@@ -253,8 +255,8 @@ public class BasicPhysicalOpUnitTest extends PhysicalOpUnitTestBase {
       inputJsonBatches.add(batchString.toString());
     }
 
-    OperatorTestBuilder opTestBuilder =
-        opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder =
+        legacyOpTestBuilder()
             .initReservation(initReservation)
             .maxAllocation(maxAllocation)
             .physicalOperator(sortConf)
@@ -312,7 +314,7 @@ public class BasicPhysicalOpUnitTest extends PhysicalOpUnitTestBase {
         "[{\"a\": 5, \"b\" : 1 }]",
         "[{\"a\": 5, \"b\" : 5},{\"a\": 3, \"b\" : 8}]",
         "[{\"a\": 40, \"b\" : 3},{\"a\": 13, \"b\" : 100}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
         .physicalOperator(sortConf)
         .inputDataStreamJson(inputJsonBatches)
         .baselineColumns("a", "b")
@@ -336,7 +338,7 @@ public class BasicPhysicalOpUnitTest extends PhysicalOpUnitTestBase {
     List<String> rightJsonBatches = Lists.newArrayList(
         "[{\"x\": 5, \"a\" : \"asdf\"}]",
         "[{\"x\": 5, \"a\" : \"12345\"}, {\"x\": 6, \"a\" : \"qwerty\"}]");
-    opTestBuilder()
+    legacyOpTestBuilder()
         .physicalOperator(mergeConf)
         .inputDataStreamsJson(Lists.newArrayList(leftJsonBatches, rightJsonBatches))
         .baselineColumns("x", "a")
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java
index 79f260f..2f05bba 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/MiniPlanUnitTestBase.java
@@ -32,6 +32,7 @@ import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.parquet.ParquetDirectByteBufferAllocator;
 import org.apache.drill.exec.store.parquet.ParquetReaderUtility;
 import org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader;
+import org.apache.drill.test.PhysicalOpUnitTestBase;
 import org.apache.hadoop.fs.Path;
 import org.apache.parquet.hadoop.CodecFactory;
 import org.apache.parquet.hadoop.ParquetFileReader;
@@ -171,7 +172,7 @@ public class MiniPlanUnitTestBase extends PhysicalOpUnitTestBase {
   }
 
   /**
-   * Similar to {@link OperatorTestBuilder}, build a physical operator (RecordBatch) and specify its input record batches.
+   * Similar to {@link LegacyOperatorTestBuilder}, build a physical operator (RecordBatch) and specify its input record batches.
    * The input record batch could be a non-scan operator by calling {@link PopBuilder#addInputAsChild},
    * or a scan operator by calling {@link PopBuilder#addJsonScanAsChild()} if it's SCAN operator.
    *
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestOutputBatchSize.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestOutputBatchSize.java
index 84a4fbc..f632455 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestOutputBatchSize.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestOutputBatchSize.java
@@ -49,6 +49,8 @@ import org.apache.drill.exec.vector.UInt4Vector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.RepeatedListVector;
 import org.apache.drill.exec.vector.complex.RepeatedValueVector;
+import org.apache.drill.test.LegacyOperatorTestBuilder;
+import org.apache.drill.test.PhysicalOpUnitTestBase;
 import org.junit.Test;
 
 import java.util.ArrayList;
@@ -121,7 +123,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
 
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
             .physicalOperator(projectConf)
             .inputDataStreamJson(inputJsonBatches)
             .baselineColumns(baselineColumns)
@@ -197,7 +199,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
 
       fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", memoryLimit);
 
-      OperatorTestBuilder opTestBuilder = opTestBuilder()
+      LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
               .physicalOperator(projectConf)
               .inputDataStreamJson(inputJsonBatches)
               .baselineColumns(baselineColumns)
@@ -276,7 +278,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
 
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
             .physicalOperator(projectConf)
             .inputDataStreamJson(inputJsonBatches)
             .baselineColumns(baselineColumns)
@@ -347,7 +349,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
 
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
             .physicalOperator(projectConf)
             .inputDataStreamJson(inputJsonBatches)
             .baselineColumns(baselineColumns)
@@ -418,7 +420,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
 
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
             .physicalOperator(projectConf)
             .inputDataStreamJson(inputJsonBatches)
             .baselineColumns(baselineColumns)
@@ -481,7 +483,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately get 2 batches and max of 4.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(flatten)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b", "c")
@@ -549,7 +551,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately get 2 batches and max of 4.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(flatten)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b", "c")
@@ -611,7 +613,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately get 2 batches and max of 4.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(flatten)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b", "c")
@@ -672,7 +674,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately get 2 batches and max of 4.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(flatten)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b", "c")
@@ -760,7 +762,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately get 2 batches and max of 4.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(flatten)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b", "c")
@@ -869,7 +871,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately get 2 batches and max of 4.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(flatten)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b", "c")
@@ -967,7 +969,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately get 2 batches and max of 4.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(flatten)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b", "c")
@@ -1065,7 +1067,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
 
     // Here we expect 16 batches because each batch will be limited by upper limit of 65535 records.
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(flatten)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "c")
@@ -1125,7 +1127,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // Here we expect 10 batches because each batch will be bounded by lower limit of at least 1 record.
     // do not check the output batch size as it will be more than configured value of 1024, so we get
     // at least one record out.
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(flatten)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "c")
@@ -1172,7 +1174,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     batchString.append("]");
     inputJsonBatches.add(batchString.toString());
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(flatten)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b")
@@ -1233,7 +1235,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately get 2 batches and max of 4.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(flatten)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b", "c")
@@ -1302,7 +1304,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately 4 batches because of fragmentation factor of 2 accounted for in merge join.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize/2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(mergeJoin)
       .baselineColumns("a1", "b1", "c1", "a2", "b2", "c2")
       .expectedNumBatches(4)  // verify number of batches
@@ -1372,7 +1374,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We should get 1 batch.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize*2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(mergeJoin)
       .baselineColumns("a1", "b1", "c1", "a2", "b2", "c2")
       .expectedNumBatches(1)  // verify number of batches
@@ -1426,7 +1428,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // "a1" : 5,  "c1" : 3, "a2":6,  "c2": 3
 
     // expect two batches, batch limited by 65535 records
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(mergeJoin)
       .baselineColumns("a1", "c1", "a2", "c2")
       .expectedNumBatches(2)  // verify number of batches
@@ -1481,7 +1483,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // set very low value of output batch size so we can do only one row per batch.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", 128);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(mergeJoin)
       .baselineColumns("a1", "b1", "c1", "a2", "b2", "c2")
       .expectedNumBatches(10)  // verify number of batches
@@ -1549,7 +1551,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We should get 2 batches, one for the left and one for the right.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize*2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(unionAll)
       .baselineColumns("a1", "b1", "c1")
       .expectedNumBatches(2)  // verify number of batches
@@ -1620,7 +1622,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We should get 4 batches, 2 for the left and 2 for the right.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize/2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(unionAll)
       .baselineColumns("a1", "b1", "c1")
       .expectedNumBatches(4)  // verify number of batches
@@ -1692,7 +1694,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We should get 22 batches for 22 rows.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", 128);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(unionAll)
       .baselineColumns("a1","b1", "c1")
       .expectedNumBatches(22)  // verify number of batches
@@ -1764,7 +1766,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately 4 batches because of fragmentation factor of 2 accounted for in merge join.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize/2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(hashJoin)
       .baselineColumns("a1", "b1", "c1", "a2", "b2", "c2")
       .expectedNumBatches(4)  // verify number of batches
@@ -1834,7 +1836,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We should get 1 batch.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize*2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(hashJoin)
       .baselineColumns("a1", "b1", "c1", "a2", "b2", "c2")
       .expectedNumBatches(1)  // verify number of batches
@@ -1888,7 +1890,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // "a1" : 5,  "c1" : 3, "a2":6,  "c2": 3
 
     // expect two batches, batch limited by 65535 records
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(hashJoin)
       .baselineColumns("a1", "c1", "a2", "c2")
       .expectedNumBatches(2)  // verify number of batches
@@ -1943,7 +1945,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // set very low value of output batch size so we can do only one row per batch.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", 128);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(hashJoin)
       .baselineColumns("a1", "b1", "c1", "a2", "b2", "c2")
       .expectedNumBatches(10)  // verify number of batches
@@ -2011,7 +2013,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately 4 batches because of fragmentation factor of 2 accounted for in merge join.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize/2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(hashJoin)
       .baselineColumns("a1", "b1", "c1", "a2", "b2", "c2")
       .expectedNumBatches(4)  // verify number of batches
@@ -2080,7 +2082,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately 4 batches because of fragmentation factor of 2 accounted for in merge join.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize/2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(hashJoin)
       .baselineColumns("a1", "b1", "c1", "a2", "b2", "c2")
       .expectedNumBatches(4)  // verify number of batches
@@ -2102,7 +2104,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
        "[{\"a\": 5, \"b\" : 1 }]",
          "[{\"a\": 5, \"b\" : 5},{\"a\": 3, \"b\" : 8}]");
 
-    opTestBuilder()
+    legacyOpTestBuilder()
       .physicalOperator(aggConf)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("b_sum", "a")
@@ -2157,7 +2159,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately get 2 batches and max of 4.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(hashAgg)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b_sum")
@@ -2218,7 +2220,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately get 2 batches and max of 4.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(hashAgg)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b_avg")
@@ -2278,7 +2280,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately get 2 batches and max of 4.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize / 2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
       .physicalOperator(hashAgg)
       .inputDataStreamJson(inputJsonBatches)
       .baselineColumns("a", "b_max")
@@ -2350,7 +2352,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately 4 batches.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize/2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
             .physicalOperator(nestedLoopJoin)
             .baselineColumns("a1", "b1", "c1", "a2", "b2", "c2")
             .expectedNumBatches(4)  // verify number of batches
@@ -2424,7 +2426,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We should get 1 batch.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize*2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
             .physicalOperator(nestedLoopJoin)
             .baselineColumns("a1", "b1", "c1", "a2", "b2", "c2")
             .expectedNumBatches(1)  // verify number of batches
@@ -2482,7 +2484,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
 
     // we expect n(n+1)/2 number of records i.e. (500 * 501)/2 = 125250
     // expect two batches, batch limited by 65535 records
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
             .physicalOperator(nestedLoopJoin)
             .baselineColumns("a1", "c1", "a2", "c2")
             .expectedNumBatches(2)  // verify number of batches
@@ -2542,7 +2544,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // set very low value of output batch size so we can do only one row per batch.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", 128);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
             .physicalOperator(nestedLoopJoin)
             .baselineColumns("a1", "b1", "c1", "a2", "b2", "c2")
             .expectedNumBatches(10)  // verify number of batches
@@ -2612,7 +2614,7 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     // We will get approximately 4 batches.
     fragContext.getOptions().setLocalOption("drill.exec.memory.operator.output_batch_size", totalSize/2);
 
-    OperatorTestBuilder opTestBuilder = opTestBuilder()
+    LegacyOperatorTestBuilder opTestBuilder = legacyOpTestBuilder()
             .physicalOperator(nestedLoopJoin)
             .baselineColumns("a1", "b1", "c1", "a2", "b2", "c2")
             .expectedNumBatches(4)  // verify number of batches
@@ -2949,4 +2951,4 @@ public class TestOutputBatchSize extends PhysicalOpUnitTestBase {
     }
   }
 
-}
\ No newline at end of file
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/columnreaders/TestBatchSizingMemoryUtil.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/columnreaders/TestBatchSizingMemoryUtil.java
index 9f4e026..4e0a739 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/columnreaders/TestBatchSizingMemoryUtil.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/columnreaders/TestBatchSizingMemoryUtil.java
@@ -19,12 +19,12 @@ package org.apache.drill.exec.store.parquet.columnreaders;
 
 import java.math.BigDecimal;
 import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.exec.physical.unit.PhysicalOpUnitTestBase;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
 import org.apache.drill.exec.store.parquet.columnreaders.batchsizing.BatchSizingMemoryUtil;
 import org.apache.drill.exec.store.parquet.columnreaders.batchsizing.BatchSizingMemoryUtil.ColumnMemoryUsageInfo;
 import org.apache.drill.exec.store.parquet.columnreaders.batchsizing.RecordBatchSizerManager.ColumnMemoryQuota;
+import org.apache.drill.test.PhysicalOpUnitTestBase;
 import org.apache.drill.test.rowSet.RowSet;
 import org.apache.drill.test.rowSet.RowSetBuilder;
 import org.apache.drill.test.rowSet.schema.SchemaBuilder;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/LegacyOperatorTestBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/LegacyOperatorTestBuilder.java
new file mode 100644
index 0000000..dacb061
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/LegacyOperatorTestBuilder.java
@@ -0,0 +1,178 @@
+/*
+ * 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.test;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.AbstractBase;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.physical.impl.ScanBatch;
+import org.apache.drill.exec.record.RecordBatch;
+
+import java.io.UnsupportedEncodingException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * @deprecated Use {@link OperatorTestBuilder} instead.
+ */
+@Deprecated
+public class LegacyOperatorTestBuilder {
+
+  private PhysicalOpUnitTestBase physicalOpUnitTestBase;
+
+  private PhysicalOperator popConfig;
+  private String[] baselineColumns;
+  private List<Map<String, Object>> baselineRecords;
+  private List<List<String>> inputStreamsJSON;
+  private long initReservation = AbstractBase.INIT_ALLOCATION;
+  private long maxAllocation = AbstractBase.MAX_ALLOCATION;
+  private boolean expectNoRows;
+  private Long expectedBatchSize;
+  private Integer expectedNumBatches;
+  private Integer expectedTotalRows;
+
+  public LegacyOperatorTestBuilder(PhysicalOpUnitTestBase physicalOpUnitTestBase) {
+    this.physicalOpUnitTestBase = physicalOpUnitTestBase;
+  }
+
+  @SuppressWarnings({"unchecked", "resource"})
+  public void go() {
+    BatchCreator<PhysicalOperator> opCreator;
+    RecordBatch testOperator;
+    try {
+      physicalOpUnitTestBase.mockOpContext(popConfig, initReservation, maxAllocation);
+
+      opCreator = (BatchCreator<PhysicalOperator>) physicalOpUnitTestBase.opCreatorReg.getOperatorCreator(popConfig.getClass());
+      List<RecordBatch> incomingStreams = Lists.newArrayList();
+      if (inputStreamsJSON != null) {
+        for (List<String> batchesJson : inputStreamsJSON) {
+          incomingStreams.add(new ScanBatch(popConfig, physicalOpUnitTestBase.fragContext,
+              physicalOpUnitTestBase.getReaderListForJsonBatches(batchesJson, physicalOpUnitTestBase.fragContext)));
+        }
+      }
+
+      testOperator = opCreator.getBatch(physicalOpUnitTestBase.fragContext, popConfig, incomingStreams);
+
+      Map<String, List<Object>> actualSuperVectors = DrillTestWrapper.addToCombinedVectorResults(new PhysicalOpUnitTestBase.BatchIterator(testOperator), expectedBatchSize, expectedNumBatches, expectedTotalRows);
+      if ( expectedTotalRows != null ) { return; } // when checking total rows, don't compare actual results
+
+      Map<String, List<Object>> expectedSuperVectors;
+
+      if (expectNoRows) {
+        expectedSuperVectors = new TreeMap<>();
+        for (String column : baselineColumns) {
+          expectedSuperVectors.put(column, new ArrayList<>());
+        }
+      } else {
+        expectedSuperVectors = DrillTestWrapper.translateRecordListToHeapVectors(baselineRecords);
+      }
+
+      DrillTestWrapper.compareMergedVectors(expectedSuperVectors, actualSuperVectors);
+
+    } catch (ExecutionSetupException e) {
+      throw new RuntimeException(e);
+    } catch (UnsupportedEncodingException e) {
+      throw new RuntimeException(e);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public LegacyOperatorTestBuilder physicalOperator(PhysicalOperator batch) {
+    this.popConfig = batch;
+    return this;
+  }
+
+  public LegacyOperatorTestBuilder initReservation(long initReservation) {
+    this.initReservation = initReservation;
+    return this;
+  }
+
+  public LegacyOperatorTestBuilder maxAllocation(long maxAllocation) {
+    this.maxAllocation = maxAllocation;
+    return this;
+  }
+
+  public LegacyOperatorTestBuilder inputDataStreamJson(List<String> jsonBatches) {
+    this.inputStreamsJSON = new ArrayList<>();
+    this.inputStreamsJSON.add(jsonBatches);
+    return this;
+  }
+
+  public LegacyOperatorTestBuilder inputDataStreamsJson(List<List<String>> childStreams) {
+    this.inputStreamsJSON = childStreams;
+    return this;
+  }
+
+  public LegacyOperatorTestBuilder baselineColumns(String... columns) {
+    for (int i = 0; i < columns.length; i++) {
+      LogicalExpression ex = physicalOpUnitTestBase.parseExpr(columns[i]);
+      if (ex instanceof SchemaPath) {
+        columns[i] = ((SchemaPath)ex).toExpr();
+      } else {
+        throw new IllegalStateException("Schema path is not a valid format.");
+      }
+    }
+    this.baselineColumns = columns;
+    return this;
+  }
+
+  public LegacyOperatorTestBuilder baselineValues(Object... baselineValues) {
+    if (baselineRecords == null) {
+      baselineRecords = new ArrayList<>();
+    }
+    Map<String, Object> ret = new HashMap<>();
+    int i = 0;
+    Preconditions.checkArgument(baselineValues.length == baselineColumns.length,
+        "Must supply the same number of baseline values as columns.");
+    for (String s : baselineColumns) {
+      ret.put(s, baselineValues[i]);
+      i++;
+    }
+    this.baselineRecords.add(ret);
+    return this;
+  }
+
+  public LegacyOperatorTestBuilder expectZeroRows() {
+    this.expectNoRows = true;
+    return this;
+  }
+
+  public LegacyOperatorTestBuilder expectedNumBatches(Integer expectedNumBatches) {
+    this.expectedNumBatches = expectedNumBatches;
+    return this;
+  }
+
+  public LegacyOperatorTestBuilder expectedBatchSize(Long batchSize) {
+    this.expectedBatchSize = batchSize;
+    return this;
+  }
+
+  public LegacyOperatorTestBuilder expectedTotalRows(Integer expectedTotalRows) {
+    this.expectedTotalRows = expectedTotalRows;
+    return this;
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/OperatorTestBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/OperatorTestBuilder.java
new file mode 100644
index 0000000..746422c
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/OperatorTestBuilder.java
@@ -0,0 +1,314 @@
+/*
+ * 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.test;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang3.mutable.MutableInt;
+import org.apache.drill.exec.physical.base.AbstractBase;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.physical.impl.MockRecordBatch;
+import org.apache.drill.exec.physical.impl.svremover.Copier;
+import org.apache.drill.exec.physical.impl.svremover.GenericCopier;
+import org.apache.drill.exec.record.CloseableRecordBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatchSizer;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.vector.FixedWidthVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VariableWidthVector;
+import org.apache.drill.test.rowSet.DirectRowSet;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSetComparison;
+import org.junit.Assert;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+public class OperatorTestBuilder {
+
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorTestBuilder.class);
+
+  private final List<RowSet> expectedResults = new ArrayList<>();
+  private final List<MockRecordBatch> upstreamBatches = new ArrayList<>();
+  private PhysicalOpUnitTestBase physicalOpUnitTestBase;
+
+  private PhysicalOperator physicalOperator;
+  private long initReservation = AbstractBase.INIT_ALLOCATION;
+  private long maxAllocation = AbstractBase.MAX_ALLOCATION;
+  private Optional<Integer> expectedNumBatchesOpt = Optional.empty();
+  private Optional<Integer> expectedTotalRowsOpt = Optional.empty();
+  private boolean combineOutputBatches;
+  private boolean unordered;
+
+  public OperatorTestBuilder(PhysicalOpUnitTestBase physicalOpUnitTestBase) {
+    this.physicalOpUnitTestBase = physicalOpUnitTestBase;
+  }
+
+  @SuppressWarnings({"unchecked", "resource"})
+  public void go() throws Exception {
+    final List<RowSet> actualResults = new ArrayList<>();
+    CloseableRecordBatch testOperator = null;
+
+    try {
+      validate();
+      int expectedNumBatches = expectedNumBatchesOpt.orElse(expectedResults.size());
+      physicalOpUnitTestBase.mockOpContext(physicalOperator, initReservation, maxAllocation);
+
+      final BatchCreator<PhysicalOperator> opCreator = (BatchCreator<PhysicalOperator>) physicalOpUnitTestBase.opCreatorReg.getOperatorCreator(physicalOperator.getClass());
+      testOperator = opCreator.getBatch(physicalOpUnitTestBase.fragContext, physicalOperator, (List)upstreamBatches);
+
+      batchIterator: for (int batchIndex = 0;; batchIndex++) {
+        final RecordBatch.IterOutcome outcome = testOperator.next();
+
+        switch (outcome) {
+          case NONE:
+            if (!combineOutputBatches) {
+              Assert.assertEquals(expectedNumBatches, batchIndex);
+            }
+            // We are done iterating over batches. Now we need to compare them.
+            break batchIterator;
+          case OK_NEW_SCHEMA:
+            boolean skip = true;
+
+            try {
+              skip = testOperator.getContainer().getRecordCount() == 0;
+            } catch (IllegalStateException e) {
+              // We should skip this batch in this case. It means no data was included with the okay schema
+            } finally {
+              if (skip) {
+                batchIndex--;
+                break;
+              }
+            }
+          case OK:
+            if (!combineOutputBatches && batchIndex >= expectedNumBatches) {
+              testOperator.getContainer().clear();
+              Assert.fail("More batches received than expected.");
+            } else {
+              final boolean hasSelectionVector = testOperator.getSchema().getSelectionVectorMode().hasSelectionVector;
+              final VectorContainer container = testOperator.getContainer();
+
+              if (hasSelectionVector) {
+                throw new UnsupportedOperationException("Implement DRILL-6698");
+              } else {
+                actualResults.add(DirectRowSet.fromContainer(container));
+              }
+              break;
+            }
+          default:
+            throw new UnsupportedOperationException("Can't handle this yet");
+        }
+      }
+
+      int actualTotalRows = actualResults.stream()
+        .mapToInt(RowSet::rowCount)
+        .reduce(Integer::sum)
+        .orElse(0);
+
+      if (expectedResults.isEmpty()) {
+        Assert.assertEquals((int) expectedTotalRowsOpt.orElse(0), actualTotalRows);
+        // We are done, we don't have any expected result to compare
+        return;
+      }
+
+      if (combineOutputBatches) {
+        final RowSet expectedBatch = expectedResults.get(0);
+        final RowSet actualBatch = DirectRowSet.fromSchema(
+          physicalOpUnitTestBase.operatorFixture.allocator, actualResults.get(0).container().getSchema());
+        final VectorContainer actualBatchContainer = actualBatch.container();
+        actualBatchContainer.setRecordCount(0);
+
+        final int numColumns = expectedBatch.schema().size();
+        List<MutableInt> totalBytesPerColumn = new ArrayList<>();
+
+        for (int columnIndex = 0; columnIndex < numColumns; columnIndex++) {
+          totalBytesPerColumn.add(new MutableInt());
+        }
+
+        // Get column sizes for each result batch
+
+        final List<List<RecordBatchSizer.ColumnSize>> columnSizesPerBatch = actualResults.stream().map(rowSet -> {
+          switch (rowSet.indirectionType()) {
+            case NONE:
+              return new RecordBatchSizer(rowSet.container()).columnsList();
+            default:
+              throw new UnsupportedOperationException("Implement DRILL-6698");
+          }
+        }).collect(Collectors.toList());
+
+        // Get total bytes per column
+
+        for (List<RecordBatchSizer.ColumnSize> columnSizes: columnSizesPerBatch) {
+          for (int columnIndex = 0; columnIndex < numColumns; columnIndex++) {
+            final MutableInt totalBytes = totalBytesPerColumn.get(columnIndex);
+            final RecordBatchSizer.ColumnSize columnSize = columnSizes.get(columnIndex);
+            totalBytes.add(columnSize.getTotalDataSize());
+          }
+        }
+
+        for (int columnIndex = 0; columnIndex < numColumns; columnIndex++) {
+          final ValueVector valueVector = actualBatchContainer
+            .getValueVector(columnIndex)
+            .getValueVector();
+
+          if (valueVector instanceof FixedWidthVector) {
+            ((FixedWidthVector) valueVector).allocateNew(actualTotalRows);
+          } else if (valueVector instanceof VariableWidthVector) {
+            final MutableInt totalBytes = totalBytesPerColumn.get(columnIndex);
+            ((VariableWidthVector) valueVector).allocateNew(totalBytes.getValue(), actualTotalRows);
+          } else {
+            throw new UnsupportedOperationException();
+          }
+        }
+
+        try {
+          int currentIndex = 0;
+
+          for (RowSet actualRowSet: actualResults) {
+            final Copier copier;
+            final VectorContainer rowSetContainer = actualRowSet.container();
+            rowSetContainer.setRecordCount(actualRowSet.rowCount());
+
+            switch (actualRowSet.indirectionType()) {
+              case NONE:
+                copier = new GenericCopier();
+                break;
+              default:
+                throw new UnsupportedOperationException("Implement DRILL-6698");
+            }
+
+            copier.setup(rowSetContainer, actualBatchContainer);
+            copier.appendRecords(currentIndex, actualRowSet.rowCount());
+            currentIndex += actualRowSet.rowCount();
+
+            verify(expectedBatch, actualBatch);
+          }
+        } finally {
+          actualBatch.clear();
+        }
+      } else {
+        // Compare expected and actual results
+        for (int batchIndex = 0; batchIndex < expectedNumBatches; batchIndex++) {
+          final RowSet expectedBatch = expectedResults.get(batchIndex);
+          final RowSet actualBatch = actualResults.get(batchIndex);
+
+          verify(expectedBatch, actualBatch);
+        }
+      }
+    } finally {
+      // free resources
+
+      if (testOperator != null) {
+        testOperator.close();
+      }
+
+      actualResults.forEach(rowSet -> rowSet.clear());
+
+      if (expectedResults != null) {
+        expectedResults.forEach(rowSet -> rowSet.clear());
+      }
+
+      upstreamBatches.forEach(rowSetBatch -> {
+        try {
+          rowSetBatch.close();
+        } catch (Exception e) {
+          logger.error("Error while closing RowSetBatch", e);
+        }
+      });
+    }
+  }
+
+  private void verify(final RowSet expectedBatch, final RowSet actualBatch) {
+    if (unordered) {
+      new RowSetComparison(expectedBatch).unorderedVerify(actualBatch);
+    } else {
+      new RowSetComparison(expectedBatch).verify(actualBatch);
+    }
+  }
+
+  /**
+   * Make sure the inputs are valid.
+   */
+  private void validate() {
+    if (combineOutputBatches) {
+      Preconditions.checkArgument(expectedResults.isEmpty() || expectedResults.size() == 1,
+        "The number of expected result batches needs to be zero or one when combining output batches");
+      Preconditions.checkArgument((expectedResults.isEmpty() && (!expectedNumBatchesOpt.isPresent() && expectedTotalRowsOpt.isPresent())) ||
+          (!expectedResults.isEmpty() && (!expectedNumBatchesOpt.isPresent() && !expectedTotalRowsOpt.isPresent())),
+        "When definig expectedResults, you cannot define expectedNumBatch or expectedTotalRows and vice versa");
+    } else {
+      Preconditions.checkArgument((expectedResults.isEmpty() && (expectedNumBatchesOpt.isPresent() || expectedTotalRowsOpt.isPresent())) ||
+          (!expectedResults.isEmpty() && (!expectedNumBatchesOpt.isPresent() && !expectedTotalRowsOpt.isPresent())),
+        "When definig expectedResults, you cannot define expectedNumBatch or expectedTotalRows and vice versa");
+    }
+  }
+
+  public OperatorTestBuilder physicalOperator(PhysicalOperator batch) {
+    this.physicalOperator = batch;
+    return this;
+  }
+
+  public OperatorTestBuilder initReservation(long initReservation) {
+    this.initReservation = initReservation;
+    return this;
+  }
+
+  public OperatorTestBuilder maxAllocation(long maxAllocation) {
+    this.maxAllocation = maxAllocation;
+    return this;
+  }
+
+  public OperatorTestBuilder expectedNumBatches(int expectedNumBatches) {
+    this.expectedNumBatchesOpt = Optional.of(expectedNumBatches);
+    return this;
+  }
+
+  public OperatorTestBuilder expectedTotalRows(int expectedTotalRows) {
+    this.expectedTotalRowsOpt = Optional.of(expectedTotalRows);
+    return this;
+  }
+
+  /**
+   * Combines all the batches output by the operator into a single batch for comparison.
+   * @return This {@link OperatorTestBuilder}.
+   */
+  public OperatorTestBuilder combineOutputBatches() {
+    combineOutputBatches = true;
+    return this;
+  }
+
+  public OperatorTestBuilder unordered() {
+    unordered = true;
+    return this;
+  }
+
+  public OperatorTestBuilder addUpstreamBatch(final MockRecordBatch mockRecordBatch) {
+    Preconditions.checkNotNull(mockRecordBatch);
+    upstreamBatches.add(mockRecordBatch);
+    return this;
+  }
+
+  public OperatorTestBuilder addExpectedResult(final RowSet rowSet) {
+    Preconditions.checkNotNull(rowSet);
+    expectedResults.add(rowSet);
+    return this;
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/OperatorTestBuilderTest.java b/exec/java-exec/src/test/java/org/apache/drill/test/OperatorTestBuilderTest.java
new file mode 100644
index 0000000..be3cdc2
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/OperatorTestBuilderTest.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * 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.test;
+
+import com.google.common.collect.Lists;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.physical.base.AbstractBase;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.config.Project;
+import org.apache.drill.exec.physical.impl.MockRecordBatch;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.rowSet.RowSet;
+import org.apache.drill.test.rowSet.RowSetBuilder;
+import org.apache.drill.test.rowSet.schema.SchemaBuilder;
+import org.junit.ComparisonFailure;
+import org.junit.Test;
+
+import java.util.Iterator;
+import java.util.List;
+
+public class OperatorTestBuilderTest extends PhysicalOpUnitTestBase {
+  public static final String FIRST_NAME_COL = "firstname";
+  public static final String LAST_NAME_COL = "lastname";
+
+  @Test
+  public void noCombineUnorderedTestPass() throws Exception {
+    executeTest(buildInputData(), true, false);
+  }
+
+  @Test(expected = AssertionError.class)
+  public void noCombineUnorderedTestFail() throws Exception {
+    executeTest(buildIncorrectData(), true, false);
+  }
+
+  @Test
+  public void noCombineOrderedTestPass() throws Exception {
+    executeTest(buildInputData(), false, false);
+  }
+
+  @Test(expected = ComparisonFailure.class)
+  public void noCombineOrderedTestFail() throws Exception {
+    executeTest(buildIncorrectData(), false, false);
+  }
+
+  @Test
+  public void combineUnorderedTestPass() throws Exception {
+    executeTest(buildInputData(), true, true);
+  }
+
+  @Test(expected = AssertionError.class)
+  public void combineUnorderedTestFail() throws Exception {
+    executeTest(buildIncorrectData(), true, true);
+  }
+
+  @Test
+  public void combineOrderedTestPass() throws Exception {
+    executeTest(buildInputData(), false, true);
+  }
+
+  @Test(expected = ComparisonFailure.class)
+  public void combineOrderedTestFail() throws Exception {
+    executeTest(buildIncorrectData(), false, true);
+  }
+
+  private Project createProjectPhysicalOperator() {
+    final List<NamedExpression> exprs = Lists.newArrayList(
+      new NamedExpression(SchemaPath.getSimplePath(FIRST_NAME_COL), new FieldReference(FIRST_NAME_COL)),
+      new NamedExpression(SchemaPath.getSimplePath(LAST_NAME_COL), new FieldReference(LAST_NAME_COL)));
+
+    return new Project(exprs, new MockPhysicalOperator(), true);
+  }
+
+  private static TupleMetadata buildSchema() {
+    return new SchemaBuilder()
+      .add(FIRST_NAME_COL, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.REQUIRED)
+      .add(LAST_NAME_COL, TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.REQUIRED)
+      .buildSchema();
+  }
+
+  private RowSet buildInputData() {
+    return new RowSetBuilder(operatorFixture.allocator(), buildSchema()).
+      addRow("billy", "bob").
+      addRow("bobby", "fillet").
+      build();
+  }
+
+  private RowSet buildIncorrectData() {
+    return new RowSetBuilder(operatorFixture.allocator(), buildSchema()).
+      addRow("billy", "bob").
+      addRow("bambam", "fofof").
+      build();
+  }
+
+  private void executeTest(final RowSet expectedRowSet, boolean unordered, boolean combine) throws Exception {
+    final MockRecordBatch inputRowSetBatch = new MockRecordBatch.Builder().
+      sendData(buildInputData()).
+      build(fragContext);
+
+    final OperatorTestBuilder testBuilder = opTestBuilder()
+      .physicalOperator(createProjectPhysicalOperator());
+
+    if (combine) {
+      testBuilder.combineOutputBatches();
+    }
+
+    if (unordered) {
+      testBuilder.unordered();
+    }
+
+    testBuilder
+      .addUpstreamBatch(inputRowSetBatch)
+      .addExpectedResult(expectedRowSet)
+      .go();
+  }
+
+  public static class MockPhysicalOperator extends AbstractBase {
+    @Override
+    public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+      return null;
+    }
+
+    @Override
+    public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) throws ExecutionSetupException {
+      return null;
+    }
+
+    @Override
+    public int getOperatorType() {
+      return 0;
+    }
+
+    @Override
+    public Iterator<PhysicalOperator> iterator() {
+      return null;
+    }
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
similarity index 71%
rename from exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java
rename to exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
index b2412e5..1c4779c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/PhysicalOpUnitTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
@@ -15,12 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.drill.exec.physical.unit;
+package org.apache.drill.test;
 
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.memory.BufferAllocator;
@@ -40,9 +38,7 @@ import org.apache.drill.exec.server.QueryProfileStoreContext;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.store.easy.json.JSONRecordReader;
 import org.apache.drill.exec.work.batch.IncomingBuffers;
-import org.apache.drill.test.BaseDirTestWatcher;
 import org.apache.drill.exec.work.filter.RuntimeFilterWritable;
-import org.apache.drill.test.DrillTestWrapper;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.LogicalExpression;
@@ -55,27 +51,22 @@ import org.apache.drill.common.scanner.persistence.ScanResult;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.physical.base.AbstractBase;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.physical.impl.BatchCreator;
 import org.apache.drill.exec.physical.impl.OperatorCreatorRegistry;
-import org.apache.drill.exec.physical.impl.ScanBatch;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.RecordReader;
 import org.apache.drill.exec.rpc.NamedThreadFactory;
-import org.apache.drill.test.OperatorFixture;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.mockito.Mockito;
 
 import java.io.IOException;
-import java.io.UnsupportedEncodingException;
 
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
@@ -83,7 +74,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
-import java.util.TreeMap;
 
 public class PhysicalOpUnitTestBase extends ExecTest {
   protected MockExecutorFragmentContext fragContext;
@@ -96,9 +86,9 @@ public class PhysicalOpUnitTestBase extends ExecTest {
   @Rule
   public final BaseDirTestWatcher dirTestWatcher = new BaseDirTestWatcher();
 
-  private final DrillConfig drillConf = DrillConfig.create();
-  private final ScanResult classpathScan = ClassPathScanner.fromPrescan(drillConf);
-  private final OperatorCreatorRegistry opCreatorReg = new OperatorCreatorRegistry(classpathScan);
+  protected final DrillConfig drillConf = DrillConfig.create();
+  protected final ScanResult classpathScan = ClassPathScanner.fromPrescan(drillConf);
+  protected final OperatorCreatorRegistry opCreatorReg = new OperatorCreatorRegistry(classpathScan);
 
   @Before
   public void setup() throws Exception {
@@ -186,141 +176,22 @@ public class PhysicalOpUnitTestBase extends ExecTest {
     }
   }
 
-  protected OperatorTestBuilder opTestBuilder() {
-    return new OperatorTestBuilder();
+  /**
+   * Gets a {@link LegacyOperatorTestBuilder}.
+   * @deprecated Use {@link #opTestBuilder()} instead.
+   * @return A {@link LegacyOperatorTestBuilder}.
+   */
+  @Deprecated
+  protected LegacyOperatorTestBuilder legacyOpTestBuilder() {
+    return new LegacyOperatorTestBuilder(this);
   }
 
-  protected class OperatorTestBuilder {
-
-    private PhysicalOperator popConfig;
-    private String[] baselineColumns;
-    private List<Map<String, Object>> baselineRecords;
-    private List<List<String>> inputStreamsJSON;
-    private long initReservation = AbstractBase.INIT_ALLOCATION;
-    private long maxAllocation = AbstractBase.MAX_ALLOCATION;
-    private boolean checkBatchMemory;
-    private boolean expectNoRows;
-    private Long expectedBatchSize;
-    private Integer expectedNumBatches;
-    private Integer expectedTotalRows;
-
-    @SuppressWarnings({"unchecked", "resource"})
-    public void go() {
-      BatchCreator<PhysicalOperator> opCreator;
-      RecordBatch testOperator;
-      try {
-        mockOpContext(popConfig, initReservation, maxAllocation);
-
-        opCreator = (BatchCreator<PhysicalOperator>) opCreatorReg.getOperatorCreator(popConfig.getClass());
-        List<RecordBatch> incomingStreams = Lists.newArrayList();
-        if (inputStreamsJSON != null) {
-          for (List<String> batchesJson : inputStreamsJSON) {
-            incomingStreams.add(new ScanBatch(popConfig, fragContext,
-                getReaderListForJsonBatches(batchesJson, fragContext)));
-          }
-        }
-
-        testOperator = opCreator.getBatch(fragContext, popConfig, incomingStreams);
-
-        Map<String, List<Object>> actualSuperVectors = DrillTestWrapper.addToCombinedVectorResults(new BatchIterator(testOperator), expectedBatchSize, expectedNumBatches, expectedTotalRows);
-        if ( expectedTotalRows != null ) { return; } // when checking total rows, don't compare actual results
-
-        Map<String, List<Object>> expectedSuperVectors;
-
-        if (expectNoRows) {
-          expectedSuperVectors = new TreeMap<>();
-          for (String column : baselineColumns) {
-            expectedSuperVectors.put(column, new ArrayList<>());
-          }
-        } else {
-          expectedSuperVectors = DrillTestWrapper.translateRecordListToHeapVectors(baselineRecords);
-        }
-
-        DrillTestWrapper.compareMergedVectors(expectedSuperVectors, actualSuperVectors);
-
-      } catch (ExecutionSetupException e) {
-        throw new RuntimeException(e);
-      } catch (UnsupportedEncodingException e) {
-        throw new RuntimeException(e);
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    public OperatorTestBuilder physicalOperator(PhysicalOperator batch) {
-      this.popConfig = batch;
-      return this;
-    }
-
-    public OperatorTestBuilder initReservation(long initReservation) {
-      this.initReservation = initReservation;
-      return this;
-    }
-
-    public OperatorTestBuilder maxAllocation(long maxAllocation) {
-      this.maxAllocation = maxAllocation;
-      return this;
-    }
-
-    public OperatorTestBuilder inputDataStreamJson(List<String> jsonBatches) {
-      this.inputStreamsJSON = new ArrayList<>();
-      this.inputStreamsJSON.add(jsonBatches);
-      return this;
-    }
-
-    public OperatorTestBuilder inputDataStreamsJson(List<List<String>> childStreams) {
-      this.inputStreamsJSON = childStreams;
-      return this;
-    }
-
-    public OperatorTestBuilder baselineColumns(String... columns) {
-      for (int i = 0; i < columns.length; i++) {
-        LogicalExpression ex = parseExpr(columns[i]);
-        if (ex instanceof SchemaPath) {
-          columns[i] = ((SchemaPath)ex).toExpr();
-        } else {
-          throw new IllegalStateException("Schema path is not a valid format.");
-        }
-      }
-      this.baselineColumns = columns;
-      return this;
-    }
-
-    public OperatorTestBuilder baselineValues(Object... baselineValues) {
-      if (baselineRecords == null) {
-        baselineRecords = new ArrayList<>();
-      }
-      Map<String, Object> ret = new HashMap<>();
-      int i = 0;
-      Preconditions.checkArgument(baselineValues.length == baselineColumns.length,
-          "Must supply the same number of baseline values as columns.");
-      for (String s : baselineColumns) {
-        ret.put(s, baselineValues[i]);
-        i++;
-      }
-      this.baselineRecords.add(ret);
-      return this;
-    }
-
-    public OperatorTestBuilder expectZeroRows() {
-      this.expectNoRows = true;
-      return this;
-    }
-
-    public OperatorTestBuilder expectedNumBatches(Integer expectedNumBatches) {
-      this.expectedNumBatches = expectedNumBatches;
-      return this;
-    }
-
-    public OperatorTestBuilder expectedBatchSize(Long batchSize) {
-      this.expectedBatchSize = batchSize;
-      return this;
-    }
-
-    public OperatorTestBuilder expectedTotalRows(Integer expectedTotalRows) {
-      this.expectedTotalRows = expectedTotalRows;
-      return this;
-    }
+  /**
+   * Gets an {@link OperatorTestBuilder}.
+   * @return An {@link OperatorTestBuilder}.
+   */
+  protected OperatorTestBuilder opTestBuilder() {
+    return new OperatorTestBuilder(this);
   }
 
   /**
@@ -446,11 +317,11 @@ public class PhysicalOpUnitTestBase extends ExecTest {
   /**
    * <h2>Note</h2>
    * <p>
-   *   The {@link MockPhysicalOperator} should only be used in {@link PhysicalOpUnitTestBase} because {@link PhysicalOpUnitTestBase}
+   *   The {@link MockPhysicalOperator} should only be used in {@link PhysicalOpUnitTestBase} and its ancestors because {@link PhysicalOpUnitTestBase}
    *   needs a dummy {@link MockPhysicalOperator} to be passed to Scanners.
    * </p>
    */
-  protected static class MockPhysicalOperator extends AbstractBase
+  public static class MockPhysicalOperator extends AbstractBase
   {
     @Override
     public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBatch.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBatch.java
deleted file mode 100644
index a3cd918..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetBatch.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.test.rowSet;
-
-import com.google.common.base.Preconditions;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.TypedFieldId;
-import org.apache.drill.exec.record.VectorContainer;
-import org.apache.drill.exec.record.VectorWrapper;
-import org.apache.drill.exec.record.WritableBatch;
-import org.apache.drill.exec.record.selection.SelectionVector2;
-import org.apache.drill.exec.record.selection.SelectionVector4;
-
-import java.util.Iterator;
-
-public class RowSetBatch implements RecordBatch {
-  private final RowSet rowSet;
-
-  public RowSetBatch(final RowSet rowSet) {
-    this.rowSet = Preconditions.checkNotNull(rowSet);
-  }
-
-  @Override
-  public FragmentContext getContext() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public BatchSchema getSchema() {
-    return rowSet.batchSchema();
-  }
-
-  @Override
-  public int getRecordCount() {
-    return rowSet.container().getRecordCount();
-  }
-
-  @Override
-  public SelectionVector2 getSelectionVector2() {
-    if (rowSet instanceof IndirectRowSet) {
-      return ((IndirectRowSet)rowSet).getSv2();
-    }
-
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public SelectionVector4 getSelectionVector4() {
-    if (rowSet instanceof RowSet.HyperRowSet) {
-      return ((RowSet.HyperRowSet)rowSet).getSv4();
-    }
-
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void kill(boolean sendUpstream) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public VectorContainer getOutgoingContainer() {
-    return rowSet.container();
-  }
-
-  @Override
-  public TypedFieldId getValueVectorId(SchemaPath path) {
-    return rowSet.container().getValueVectorId(path);
-  }
-
-  @Override
-  public VectorWrapper<?> getValueAccessorById(Class<?> clazz, int... ids) {
-    return rowSet.container().getValueAccessorById(clazz, ids);
-  }
-
-  @Override
-  public IterOutcome next() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public WritableBatch getWritableBatch() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public Iterator<VectorWrapper<?>> iterator() {
-    return rowSet.container().iterator();
-  }
-
-  @Override
-  public VectorContainer getContainer() { return rowSet.container(); }
-}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetComparison.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetComparison.java
index e098e33..b1cb058 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetComparison.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/RowSetComparison.java
@@ -20,13 +20,22 @@ package org.apache.drill.test.rowSet;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Comparator;
+import java.util.List;
 
+import com.google.common.base.Optional;
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Multiset;
 import org.apache.drill.exec.vector.accessor.ArrayReader;
 import org.apache.drill.exec.vector.accessor.ObjectReader;
 import org.apache.drill.exec.vector.accessor.ScalarReader;
 import org.apache.drill.exec.vector.accessor.TupleReader;
 import org.bouncycastle.util.Arrays;
+import org.junit.Assert;
 
 /**
  * For testing, compare the contents of two row sets (record batches)
@@ -58,10 +67,10 @@ public class RowSetComparison {
    */
   private boolean mask[];
   /**
-   * Floats and doubles do not compare exactly. This delta is used
-   * by JUnit for such comparisons.
+   * Floats and doubles do not compare exactly. This MathContext is used
+   * to construct BigDecimals of the desired precision.
    */
-  private double delta = 0.001;
+  private MathContext scale = new MathContext(3);
   /**
    * Tests can skip the first n rows.
    */
@@ -106,14 +115,15 @@ public class RowSetComparison {
   }
 
   /**
-   * Specify the delta value to use when comparing float or
+   * Specify the precision to use when comparing float or
    * double values.
    *
-   * @param delta the delta to use in float and double comparisons
+   * @param scale the precision to use for comparing floats and doubles. See {@link BigDecimal#scale()} for
+   *              a definition scale.
    * @return this builder
    */
-  public RowSetComparison withDelta(double delta) {
-    this.delta = delta;
+  public RowSetComparison withScale(int scale) {
+    this.scale = new MathContext(scale);
     return this;
   }
 
@@ -142,28 +152,93 @@ public class RowSetComparison {
     return this;
   }
 
+  private void compareSchemasAndCounts(RowSet actual) {
+    assertTrue("Schemas don't match.\n" +
+      "Expected: " + expected.schema().toString() +
+      "\nActual: " + actual.schema(), expected.schema().isEquivalent(actual.schema()));
+    int testLength = getTestLength();
+    int dataLength = offset + testLength;
+    assertTrue("Missing expected rows", expected.rowCount() >= dataLength);
+    assertTrue("Missing actual rows", actual.rowCount() >= dataLength);
+  }
+
+  private int getTestLength() {
+    return span > -1 ? span : expected.rowCount() - offset;
+  }
+
+  public void unorderedVerify(RowSet actual) {
+    compareSchemasAndCounts(actual);
+
+    int testLength = getTestLength();
+    RowSetReader er = expected.reader();
+    RowSetReader ar = actual.reader();
+
+    for (int i = 0; i < offset; i++) {
+      er.next();
+      ar.next();
+    }
+
+    final Multiset<List<Object>> expectedSet = HashMultiset.create();
+    final Multiset<List<Object>> actualSet = HashMultiset.create();
+
+    for (int rowCounter = 0; rowCounter < testLength; rowCounter++) {
+      er.next();
+      ar.next();
+
+      expectedSet.add(buildRow(er));
+      actualSet.add(buildRow(ar));
+    }
+
+    Assert.assertEquals(expectedSet, actualSet);
+  }
+
+  /**
+   * Convenience method to verify the actual results, then free memory
+   * for both the expected and actual result sets.
+   * @param actual the actual results to verify
+   */
+  public void unorderedVerifyAndClearAll(RowSet actual) {
+    try {
+      unorderedVerify(actual);
+    } finally {
+      expected.clear();
+      actual.clear();
+    }
+  }
+
+  private List<Object> buildRow(RowSetReader reader) {
+    final List<Object> row = new ArrayList<>();
+
+    for (int i = 0; i < mask.length; i++) {
+      if (!mask[i]) {
+        continue;
+      }
+
+      final ScalarReader scalarReader = reader.column(i).scalar();
+      final Object value = getScalar(scalarReader);
+      row.add(value);
+    }
+
+    return row;
+  }
+
   /**
    * Verify the actual rows using the rules defined in this builder
    * @param actual the actual results to verify
    */
 
   public void verify(RowSet actual) {
-    assertTrue("Schemas don't match.\n" +
-        "Expected: " + expected.schema().toString() +
-        "\nActual: " + actual.schema(), expected.schema().isEquivalent(actual.schema()));
-    int testLength = expected.rowCount() - offset;
-    if (span > -1) {
-      testLength = span;
-    }
-    int dataLength = offset + testLength;
-    assertTrue("Missing expected rows", expected.rowCount() >= dataLength);
-    assertTrue("Missing actual rows", actual.rowCount() >= dataLength);
+    compareSchemasAndCounts(actual);
+    int testLength = getTestLength();
+
     RowSetReader er = expected.reader();
     RowSetReader ar = actual.reader();
+
     for (int i = 0; i < offset; i++) {
       er.next();
       ar.next();
     }
+
     for (int i = 0; i < testLength; i++) {
       er.next();
       ar.next();
@@ -245,34 +320,32 @@ public class RowSetComparison {
     if (! ec.isNull()) {
       assertTrue(label + " - column is null", ! ac.isNull());
     }
+
     switch (ec.valueType()) {
-    case BYTES: {
+      case BYTES:
         byte expected[] = ac.getBytes();
         byte actual[] = ac.getBytes();
         assertEquals(label + " - byte lengths differ", expected.length, actual.length);
         assertTrue(label, Arrays.areEqual(expected, actual));
         break;
-     }
-     case DOUBLE:
-       assertEquals(label, ec.getDouble(), ac.getDouble(), delta);
-       break;
-     case INTEGER:
-       assertEquals(label, ec.getInt(), ac.getInt());
-       break;
-     case LONG:
-       assertEquals(label, ec.getLong(), ac.getLong());
-       break;
-     case STRING:
-       assertEquals(label, ec.getString(), ac.getString());
-        break;
-     case DECIMAL:
-       assertEquals(label, ec.getDecimal(), ac.getDecimal());
-       break;
-     case PERIOD:
-       assertEquals(label, ec.getPeriod(), ac.getPeriod());
-       break;
-     default:
-        throw new IllegalStateException( "Unexpected type: " + ec.valueType());
+      default:
+        assertEquals(label, getScalar(ec), getScalar(ac));
+    }
+  }
+
+  private Object getScalar(final ScalarReader scalarReader) {
+    if (scalarReader.isNull()) {
+      return Optional.absent();
+    }
+
+    switch (scalarReader.valueType()) {
+      case BYTES:
+        return ByteBuffer.wrap(scalarReader.getBytes());
+      case DOUBLE: {
+        return new BigDecimal(scalarReader.getDouble(), this.scale).stripTrailingZeros();
+      }
+      default:
+        return scalarReader.getObject();
     }
   }
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/TestRowSetComparison.java b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/TestRowSetComparison.java
new file mode 100644
index 0000000..b8bdb12
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/rowSet/TestRowSetComparison.java
@@ -0,0 +1,211 @@
+/*
+ * 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.test.rowSet;
+
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.RootAllocator;
+import org.apache.drill.exec.record.metadata.TupleMetadata;
+import org.apache.drill.test.rowSet.schema.SchemaBuilder;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestRowSetComparison {
+  private BufferAllocator allocator;
+
+  @Before
+  public void setup() {
+    allocator = new RootAllocator(Long.MAX_VALUE);
+  }
+
+  @Test
+  public void simpleUnorderedComparisonMatchTest() {
+    final TupleMetadata schema = new SchemaBuilder()
+      .add("a", TypeProtos.MinorType.INT, TypeProtos.DataMode.REQUIRED)
+      .add("b", TypeProtos.MinorType.INT, TypeProtos.DataMode.REQUIRED)
+      .buildSchema();
+
+    final RowSet expected = new RowSetBuilder(allocator, schema)
+      .addRow(1, 1)
+      .addRow(1, 1)
+      .addRow(1, 2)
+      .addRow(2, 1)
+      .build();
+
+    final RowSet actual = new RowSetBuilder(allocator, schema)
+      .addRow(1, 1)
+      .addRow(1, 2)
+      .addRow(2, 1)
+      .addRow(1, 1)
+      .build();
+
+    try {
+      new RowSetComparison(expected).unorderedVerify(actual);
+    } finally {
+      expected.clear();
+      actual.clear();
+    }
+  }
+
+  @Test
+  public void simpleDoubleUnorderedComparisonMatchTest() {
+    final TupleMetadata schema = new SchemaBuilder()
+      .add("a", TypeProtos.MinorType.FLOAT4, TypeProtos.DataMode.REQUIRED)
+      .add("b", TypeProtos.MinorType.FLOAT8, TypeProtos.DataMode.REQUIRED)
+      .buildSchema();
+
+    final RowSet expected = new RowSetBuilder(allocator, schema)
+      .addRow(1.0f, 1.0)
+      .addRow(1.0f, 1.0)
+      .addRow(1.0f, 1.01)
+      .addRow(1.01f, 1.0)
+      .build();
+
+    final RowSet actual = new RowSetBuilder(allocator, schema)
+      .addRow(1.004f, .9996)
+      .addRow(1.0f, 1.008)
+      .addRow(1.008f, 1.0)
+      .addRow(.9996f, 1.004)
+      .build();
+
+    try {
+      new RowSetComparison(expected).unorderedVerify(actual);
+    } finally {
+      expected.clear();
+      actual.clear();
+    }
+  }
+
+  @Test
+  public void simpleVarcharUnorderedComparisonMatchTest() {
+    final TupleMetadata schema = new SchemaBuilder()
+      .add("a", TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.REQUIRED)
+      .buildSchema();
+
+    final RowSet expected = new RowSetBuilder(allocator, schema)
+      .addRow("aaa")
+      .addRow("bbb")
+      .addRow("ccc")
+      .addRow("bbb")
+      .build();
+
+    final RowSet actual = new RowSetBuilder(allocator, schema)
+      .addRow("ccc")
+      .addRow("aaa")
+      .addRow("bbb")
+      .addRow("bbb")
+      .build();
+
+    try {
+      new RowSetComparison(expected).unorderedVerify(actual);
+    } finally {
+      expected.clear();
+      actual.clear();
+    }
+  }
+
+  @Test(expected = AssertionError.class)
+  public void simpleUnorderedComparisonNoMatchTest() {
+    final TupleMetadata schema = new SchemaBuilder()
+      .add("a", TypeProtos.MinorType.INT, TypeProtos.DataMode.REQUIRED)
+      .add("b", TypeProtos.MinorType.INT, TypeProtos.DataMode.REQUIRED)
+      .buildSchema();
+
+    final RowSet expected = new RowSetBuilder(allocator, schema)
+      .addRow(1, 1)
+      .addRow(3, 2)
+      .addRow(2, 4)
+      .build();
+
+    final RowSet actual = new RowSetBuilder(allocator, schema)
+      .addRow(1, 1)
+      .addRow(2, 1)
+      .addRow(1, 1)
+      .build();
+
+    try {
+      new RowSetComparison(expected).unorderedVerify(actual);
+    } finally {
+      expected.clear();
+      actual.clear();
+    }
+  }
+
+  @Test(expected = AssertionError.class)
+  public void simpleDoubleUnorderedComparisonNoMatchTest() {
+    final TupleMetadata schema = new SchemaBuilder()
+      .add("a", TypeProtos.MinorType.FLOAT4, TypeProtos.DataMode.REQUIRED)
+      .add("b", TypeProtos.MinorType.FLOAT8, TypeProtos.DataMode.REQUIRED)
+      .buildSchema();
+
+    final RowSet expected = new RowSetBuilder(allocator, schema)
+      .addRow(1.0f, 1.0)
+      .addRow(1.0f, 1.0)
+      .addRow(1.0f, 1.01)
+      .addRow(1.01f, 1.0)
+      .build();
+
+    final RowSet actual = new RowSetBuilder(allocator, schema)
+      .addRow(1.009f, .9996)
+      .addRow(1.0f, 1.004)
+      .addRow(1.008f, 1.0)
+      .addRow(.9994f, 1.004)
+      .build();
+
+    try {
+      new RowSetComparison(expected).unorderedVerify(actual);
+    } finally {
+      expected.clear();
+      actual.clear();
+    }
+  }
+
+  @Test(expected = AssertionError.class)
+  public void simpleVarcharUnorderedComparisonNoMatchTest() {
+    final TupleMetadata schema = new SchemaBuilder()
+      .add("a", TypeProtos.MinorType.VARCHAR, TypeProtos.DataMode.REQUIRED)
+      .buildSchema();
+
+    final RowSet expected = new RowSetBuilder(allocator, schema)
+      .addRow("red")
+      .addRow("bbb")
+      .addRow("ccc")
+      .addRow("bbb")
+      .build();
+
+    final RowSet actual = new RowSetBuilder(allocator, schema)
+      .addRow("ccc")
+      .addRow("aaa")
+      .addRow("blue")
+      .addRow("bbb")
+      .build();
+
+    try {
+      new RowSetComparison(expected).unorderedVerify(actual);
+    } finally {
+      expected.clear();
+      actual.clear();
+    }
+  }
+
+  @After
+  public void teardown() {
+    allocator.close();
+  }
+}
diff --git a/exec/vector/src/main/codegen/templates/FixedValueVectors.java b/exec/vector/src/main/codegen/templates/FixedValueVectors.java
index eb413b0..a98aa66 100644
--- a/exec/vector/src/main/codegen/templates/FixedValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/FixedValueVectors.java
@@ -358,7 +358,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
 
   @Override
   public void copyEntry(int toIndex, ValueVector from, int fromIndex) {
-    ((${minor.class}Vector) from).data.getBytes(fromIndex * VALUE_WIDTH, data, toIndex * VALUE_WIDTH, VALUE_WIDTH);
+    copyFromSafe(fromIndex, toIndex, (${minor.class}Vector) from);
   }
 
   public void decrementAllocationMonitor() {
diff --git a/exec/vector/src/main/codegen/templates/NullableValueVectors.java b/exec/vector/src/main/codegen/templates/NullableValueVectors.java
index ff066fb..f82c718 100644
--- a/exec/vector/src/main/codegen/templates/NullableValueVectors.java
+++ b/exec/vector/src/main/codegen/templates/NullableValueVectors.java
@@ -433,7 +433,7 @@ public final class ${className} extends BaseDataValueVector implements <#if type
 
     // Handle the case of not-nullable copied into a nullable
     if (from instanceof ${minor.class}Vector) {
-      bits.getMutator().set(toIndex,1);
+      bits.getMutator().setSafe(toIndex,1);
       values.copyFromSafe(fromIndex,toIndex,(${minor.class}Vector)from);
       return;
     }


[drill] 05/06: DRILL-6644: Don't reserve space for incoming probe batches unnecessarily during the build phase.

Posted by ar...@apache.org.
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

commit e9ffb5bde37efad9f0b646773311f23ea2dbda5d
Author: Timothy Farkas <ti...@apache.org>
AuthorDate: Mon Jul 30 11:21:02 2018 -0700

    DRILL-6644: Don't reserve space for incoming probe batches unnecessarily during the build phase.
    
    git closes #1409
---
 .../exec/physical/impl/join/HashJoinBatch.java     |  23 ++--
 .../join/HashJoinMechanicalMemoryCalculator.java   |   1 -
 .../impl/join/HashJoinMemoryCalculator.java        |   3 +-
 .../impl/join/HashJoinMemoryCalculatorImpl.java    | 106 ++++++++---------
 .../impl/join/TestBuildSidePartitioningImpl.java   | 104 +++++++++++++----
 .../impl/join/TestPostBuildCalculationsImpl.java   | 127 +++++++++++++++++----
 .../exec/physical/unit/TestNullInputMiniPlan.java  |   4 +-
 7 files changed, 243 insertions(+), 125 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
index a8339f8..dd9b0d6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
@@ -287,7 +287,7 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
       prefetchedBuild,
       buildSideIsEmpty,
       RIGHT_INDEX,
-      right,
+      buildBatch,
       () -> {
         batchMemoryManager.update(RIGHT_INDEX, 0, true);
         logger.debug("BATCH_STATS, incoming right: {}", batchMemoryManager.getRecordBatchSizer(RIGHT_INDEX));
@@ -302,7 +302,7 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
       prefetchedProbe,
       probeSideIsEmpty,
       LEFT_INDEX,
-      left,
+      probeBatch,
       () -> {
         batchMemoryManager.update(LEFT_INDEX, 0);
         logger.debug("BATCH_STATS, incoming left: {}", batchMemoryManager.getRecordBatchSizer(LEFT_INDEX));
@@ -348,7 +348,10 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
       state = BatchState.STOP;
     } else {
       // Got our first batch(es)
-      memoryManagerUpdate.run();
+      if (cycleNum == 0) {
+        // Only collect stats for the first cylce
+        memoryManagerUpdate.run();
+      }
       state = BatchState.FIRST;
     }
 
@@ -558,6 +561,9 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
 
           state = BatchState.FIRST;  // TODO need to determine if this is still necessary since prefetchFirstBatchFromBothSides sets this
 
+          prefetchedBuild.setValue(false);
+          prefetchedProbe.setValue(false);
+
           return innerNext(); // start processing the next spilled partition "recursively"
         }
 
@@ -762,7 +768,6 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
         buildJoinColumns,
         leftUpstream == IterOutcome.NONE, // probeEmpty
         allocator.getLimit(),
-        maxIncomingBatchSize,
         numPartitions,
         RECORDS_PER_BATCH,
         RECORDS_PER_BATCH,
@@ -821,22 +826,19 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
     {
       // Initializing build calculator
       // Limit scope of these variables to this block
-      int maxBatchSize = firstCycle? RecordBatch.MAX_BATCH_ROW_COUNT : RECORDS_PER_BATCH;
-      boolean hasProbeData = leftUpstream != IterOutcome.NONE;
-      boolean doMemoryCalculation = canSpill && hasProbeData;
+      int maxBatchSize = firstCycle? RecordBatch.MAX_BATCH_ROW_COUNT: RECORDS_PER_BATCH;
+      boolean doMemoryCalculation = canSpill && !probeSideIsEmpty.booleanValue();
       HashJoinMemoryCalculator calc = getCalculatorImpl();
 
       calc.initialize(doMemoryCalculation);
       buildCalc = calc.next();
 
-      // We've sniffed first non empty build and probe batches so we have enough information to create a calculator
       buildCalc.initialize(firstCycle, true, // TODO Fix after growing hash values bug fixed
         buildBatch,
         probeBatch,
         buildJoinColumns,
-        leftUpstream == IterOutcome.NONE, // probeEmpty
+        probeSideIsEmpty.booleanValue(),
         allocator.getLimit(),
-        maxIncomingBatchSize,
         numPartitions,
         RECORDS_PER_BATCH,
         RECORDS_PER_BATCH,
@@ -1093,7 +1095,6 @@ public class HashJoinBatch extends AbstractBinaryRecordBatch<HashJoinPOP> {
 
     this.allocator = oContext.getAllocator();
 
-    maxIncomingBatchSize = context.getOptions().getLong(ExecConstants.OUTPUT_BATCH_SIZE);
     numPartitions = (int)context.getOptions().getOption(ExecConstants.HASHJOIN_NUM_PARTITIONS_VALIDATOR);
     if ( numPartitions == 1 ) { //
       disableSpilling("Spilling is disabled due to configuration setting of num_partitions to 1");
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMechanicalMemoryCalculator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMechanicalMemoryCalculator.java
index af6be8b..59496d9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMechanicalMemoryCalculator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMechanicalMemoryCalculator.java
@@ -73,7 +73,6 @@ public class HashJoinMechanicalMemoryCalculator implements HashJoinMemoryCalcula
                            Set<String> joinColumns,
                            boolean probeEmpty,
                            long memoryAvailable,
-                           long maxIncomingBatchSize,
                            int initialPartitions,
                            int recordsPerPartitionBatchBuild,
                            int recordsPerPartitionBatchProbe,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculator.java
index 0ccd912..38c2a35 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculator.java
@@ -96,14 +96,13 @@ public interface HashJoinMemoryCalculator extends HashJoinStateCalculator<HashJo
    * </ul>
    */
   interface BuildSidePartitioning extends HashJoinStateCalculator<PostBuildCalculations> {
-    void initialize(boolean autoTune,
+    void initialize(boolean firstCycle,
                     boolean reserveHash,
                     RecordBatch buildSideBatch,
                     RecordBatch probeSideBatch,
                     Set<String> joinColumns,
                     boolean probeEmpty,
                     long memoryAvailable,
-                    long maxIncomingBatchSize,
                     int initialPartitions,
                     int recordsPerPartitionBatchBuild,
                     int recordsPerPartitionBatchProbe,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java
index 2ab42e5..35ff7ec 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinMemoryCalculatorImpl.java
@@ -93,14 +93,13 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
     private int recordsPerPartitionBatchProbe;
 
     @Override
-    public void initialize(boolean autoTune,
+    public void initialize(boolean firstCycle,
                            boolean reserveHash,
                            RecordBatch buildSideBatch,
                            RecordBatch probeSideBatch,
                            Set<String> joinColumns,
                            boolean probeEmpty,
                            long memoryAvailable,
-                           long maxIncomingBatchSize,
                            int initialPartitions,
                            int recordsPerPartitionBatchBuild,
                            int recordsPerPartitionBatchProbe,
@@ -169,7 +168,7 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
    * <h1>Life Cycle</h1>
    * <p>
    *   <ul>
-   *     <li><b>Step 0:</b> Call {@link #initialize(boolean, boolean, RecordBatch, RecordBatch, Set, boolean, long, long, int, int, int, int, int, int, double)}.
+   *     <li><b>Step 0:</b> Call {@link #initialize(boolean, boolean, RecordBatch, RecordBatch, Set, boolean, long, int, int, int, int, int, int, double)}.
    *     This will initialize the StateCalculate with the additional information it needs.</li>
    *     <li><b>Step 1:</b> Call {@link #getNumPartitions()} to see the number of partitions that fit in memory.</li>
    *     <li><b>Step 2:</b> Call {@link #shouldSpill()} To determine if spilling needs to occurr.</li>
@@ -190,7 +189,6 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
     private int maxBatchNumRecordsProbe;
     private long memoryAvailable;
     private boolean probeEmpty;
-    private long maxIncomingBatchSize;
     private long maxBuildBatchSize;
     private long maxProbeBatchSize;
     private long maxOutputBatchSize;
@@ -200,7 +198,7 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
     private int recordsPerPartitionBatchProbe;
     private int outputBatchSize;
     private Map<String, Long> keySizes;
-    private boolean autoTune;
+    private boolean firstCycle;
     private boolean reserveHash;
     private double loadFactor;
 
@@ -228,14 +226,13 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
     }
 
     @Override
-    public void initialize(boolean autoTune,
+    public void initialize(boolean firstCycle,
                            boolean reserveHash,
                            RecordBatch buildBatch,
                            RecordBatch probeBatch,
                            Set<String> joinColumns,
                            boolean probeEmpty,
                            long memoryAvailable,
-                           long maxIncomingBatchSize,
                            int initialPartitions,
                            int recordsPerPartitionBatchBuild,
                            int recordsPerPartitionBatchProbe,
@@ -264,11 +261,10 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
         keySizes.put(joinColumn, (long)columnSize.getStdNetOrNetSizePerEntry());
       }
 
-      initialize(autoTune,
+      initialize(firstCycle,
         reserveHash,
         keySizes,
         memoryAvailable,
-        maxIncomingBatchSize,
         initialPartitions,
         probeEmpty,
         buildSizePredictor,
@@ -282,11 +278,10 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
     }
 
     @VisibleForTesting
-    protected void initialize(boolean autoTune,
+    protected void initialize(boolean firstCycle,
                               boolean reserveHash,
                               CaseInsensitiveMap<Long> keySizes,
                               long memoryAvailable,
-                              long maxIncomingBatchSize,
                               int initialPartitions,
                               boolean probeEmpty,
                               BatchSizePredictor buildSizePredictor,
@@ -305,12 +300,11 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
       firstInitialized = true;
 
       this.loadFactor = loadFactor;
-      this.autoTune = autoTune;
+      this.firstCycle = firstCycle;
       this.reserveHash = reserveHash;
       this.keySizes = Preconditions.checkNotNull(keySizes);
       this.memoryAvailable = memoryAvailable;
       this.probeEmpty = probeEmpty;
-      this.maxIncomingBatchSize = maxIncomingBatchSize;
       this.buildSizePredictor = buildSizePredictor;
       this.probeSizePredictor = probeSizePredictor;
       this.initialPartitions = initialPartitions;
@@ -358,22 +352,6 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
     {
       // Adjust based on number of records
       maxBuildBatchSize = buildSizePredictor.predictBatchSize(maxBatchNumRecordsBuild, false);
-
-      if (probeSizePredictor.hadDataLastTime()) {
-        // We have probe data and we can compute the max incoming size.
-        maxProbeBatchSize = probeSizePredictor.predictBatchSize(maxBatchNumRecordsProbe, false);
-      } else {
-        // We don't have probe data
-        if (probeEmpty) {
-          // We know the probe has no data, so we don't need to reserve any space for the incoming probe
-          maxProbeBatchSize = 0;
-        } else {
-          // The probe side may have data, so assume it is the max incoming batch size. This assumption
-          // can fail in some cases since the batch sizing project is incomplete.
-          maxProbeBatchSize = maxIncomingBatchSize;
-        }
-      }
-
       partitionBuildBatchSize = buildSizePredictor.predictBatchSize(recordsPerPartitionBatchBuild, reserveHash);
 
       if (probeSizePredictor.hadDataLastTime()) {
@@ -389,15 +367,18 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
         long incompletePartitionsBatchSizes = ((long) partitions) * partitionBuildBatchSize;
         // We need to reserve all the space for incomplete batches, and the incoming batch as well as the
         // probe batch we sniffed.
-        // TODO when batch sizing project is complete we won't have to sniff probe batches since
-        // they will have a well defined size.
-        reservedMemory = incompletePartitionsBatchSizes + maxBuildBatchSize + maxProbeBatchSize;
+        reservedMemory = incompletePartitionsBatchSizes + maxBuildBatchSize;
+
+        if (!firstCycle) {
+          // If this is NOT the first cycle the HashJoin operator owns the probe batch and we need to reserve space for it.
+          reservedMemory += probeSizePredictor.getBatchSize();
+        }
 
         if (probeSizePredictor.hadDataLastTime()) {
           // If we have probe data, use it in our memory reservation calculations.
           probeReservedMemory = PostBuildCalculationsImpl.calculateReservedMemory(
             partitions,
-            maxProbeBatchSize,
+            probeSizePredictor.getBatchSize(),
             maxOutputBatchSize,
             partitionProbeBatchSize);
 
@@ -407,7 +388,7 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
           maxReservedMemory = reservedMemory;
         }
 
-        if (!autoTune || maxReservedMemory <= memoryAvailable) {
+        if (!firstCycle || maxReservedMemory <= memoryAvailable) {
           // Stop the tuning loop if we are not doing auto tuning, or if we are living within our memory limit
           break;
         }
@@ -476,6 +457,7 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
       Preconditions.checkState(initialized);
 
       return new PostBuildCalculationsImpl(
+        firstCycle,
         probeSizePredictor,
         memoryAvailable,
         maxOutputBatchSize,
@@ -580,10 +562,10 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
 
     public static final int MIN_RECORDS_PER_PARTITION_BATCH_PROBE = 10;
 
+    private final boolean firstCycle;
     private final BatchSizePredictor probeSizePredictor;
     private final long memoryAvailable;
     private final long maxOutputBatchSize;
-    private final int maxBatchNumRecordsProbe;
     private final int recordsPerPartitionBatchProbe;
     private final PartitionStatSet buildPartitionStatSet;
     private final Map<String, Long> keySizes;
@@ -597,24 +579,25 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
     private boolean initialized;
     private long consumedMemory;
     private boolean probeEmpty;
-    private long maxProbeBatchSize;
     private long partitionProbeBatchSize;
     private int computedProbeRecordsPerBatch;
 
     @VisibleForTesting
-    public PostBuildCalculationsImpl(final BatchSizePredictor probeSizePredictor,
-                                      final long memoryAvailable,
-                                      final long maxOutputBatchSize,
-                                      final int maxBatchNumRecordsProbe,
-                                      final int recordsPerPartitionBatchProbe,
-                                      final PartitionStatSet buildPartitionStatSet,
-                                      final Map<String, Long> keySizes,
-                                      final HashTableSizeCalculator hashTableSizeCalculator,
-                                      final HashJoinHelperSizeCalculator hashJoinHelperSizeCalculator,
-                                      final double fragmentationFactor,
-                                      final double safetyFactor,
-                                      final double loadFactor,
-                                      final boolean reserveHash) {
+    public PostBuildCalculationsImpl(final boolean firstCycle,
+                                     final BatchSizePredictor probeSizePredictor,
+                                     final long memoryAvailable,
+                                     final long maxOutputBatchSize,
+                                     final int maxBatchNumRecordsProbe,
+                                     final int recordsPerPartitionBatchProbe,
+                                     final PartitionStatSet buildPartitionStatSet,
+                                     final Map<String, Long> keySizes,
+                                     final HashTableSizeCalculator hashTableSizeCalculator,
+                                     final HashJoinHelperSizeCalculator hashJoinHelperSizeCalculator,
+                                     final double fragmentationFactor,
+                                     final double safetyFactor,
+                                     final double loadFactor,
+                                     final boolean reserveHash) {
+      this.firstCycle = firstCycle;
       this.probeSizePredictor = Preconditions.checkNotNull(probeSizePredictor);
       this.memoryAvailable = memoryAvailable;
       this.maxOutputBatchSize = maxOutputBatchSize;
@@ -626,7 +609,6 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
       this.safetyFactor = safetyFactor;
       this.loadFactor = loadFactor;
       this.reserveHash = reserveHash;
-      this.maxBatchNumRecordsProbe = maxBatchNumRecordsProbe;
       this.recordsPerPartitionBatchProbe = recordsPerPartitionBatchProbe;
       this.computedProbeRecordsPerBatch = recordsPerPartitionBatchProbe;
     }
@@ -636,7 +618,7 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
       Preconditions.checkState(!initialized);
       // If we had probe data before there should still be probe data now.
       // If we didn't have probe data before we could get some new data now.
-      Preconditions.checkState(probeSizePredictor.hadDataLastTime() && !probeEmpty || !probeSizePredictor.hadDataLastTime());
+      Preconditions.checkState(!(probeEmpty && probeSizePredictor.hadDataLastTime()));
       initialized = true;
       this.probeEmpty = probeEmpty;
 
@@ -650,12 +632,11 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
         probeSizePredictor.updateStats();
       }
 
-      maxProbeBatchSize = probeSizePredictor.predictBatchSize(maxBatchNumRecordsProbe, false);
       partitionProbeBatchSize = probeSizePredictor.predictBatchSize(recordsPerPartitionBatchProbe, reserveHash);
 
       long worstCaseProbeMemory = calculateReservedMemory(
         buildPartitionStatSet.getSize(),
-        maxProbeBatchSize,
+        getIncomingProbeBatchReservedSpace(),
         maxOutputBatchSize,
         partitionProbeBatchSize);
 
@@ -667,7 +648,7 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
           buildPartitionStatSet.getSize(),
           recordsPerPartitionBatchProbe,
           MIN_RECORDS_PER_PARTITION_BATCH_PROBE,
-          maxProbeBatchSize,
+          getIncomingProbeBatchReservedSpace(),
           maxOutputBatchSize,
           partitionProbeBatchSize);
 
@@ -681,9 +662,14 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
       return computedProbeRecordsPerBatch;
     }
 
-    @VisibleForTesting
-    public long getMaxProbeBatchSize() {
-      return maxProbeBatchSize;
+    public long getIncomingProbeBatchReservedSpace() {
+      Preconditions.checkState(initialized);
+
+      if (firstCycle) {
+        return 0;
+      } else {
+        return probeSizePredictor.getBatchSize();
+      }
     }
 
     @VisibleForTesting
@@ -744,7 +730,7 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
 
       long reservedMemory = calculateReservedMemory(
         buildPartitionStatSet.getNumSpilledPartitions(),
-        maxProbeBatchSize,
+        getIncomingProbeBatchReservedSpace(),
         maxOutputBatchSize,
         partitionProbeBatchSize);
 
@@ -804,11 +790,11 @@ public class HashJoinMemoryCalculatorImpl implements HashJoinMemoryCalculator {
         "Mem calc stats:\n" +
         "memoryLimit = %s\n" +
         "consumedMemory = %s\n" +
-        "maxProbeBatchSize = %s\n" +
+        "maxIncomingProbeBatchReservedSpace = %s\n" +
         "maxOutputBatchSize = %s\n",
         PartitionStatSet.prettyPrintBytes(memoryAvailable),
         PartitionStatSet.prettyPrintBytes(consumedMemory),
-        PartitionStatSet.prettyPrintBytes(maxProbeBatchSize),
+        PartitionStatSet.prettyPrintBytes(getIncomingProbeBatchReservedSpace()),
         PartitionStatSet.prettyPrintBytes(maxOutputBatchSize));
 
       StringBuilder hashJoinHelperSb = new StringBuilder("Partition Hash Join Helpers\n");
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestBuildSidePartitioningImpl.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestBuildSidePartitioningImpl.java
index af943ec..665bc48 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestBuildSidePartitioningImpl.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestBuildSidePartitioningImpl.java
@@ -25,7 +25,16 @@ import org.junit.Test;
 
 public class TestBuildSidePartitioningImpl {
   @Test
-  public void testSimpleReserveMemoryCalculationNoHash() {
+  public void testSimpleReserveMemoryCalculationNoHashFirstCycle() {
+    testSimpleReserveMemoryCalculationNoHashHelper(true);
+  }
+
+  @Test
+  public void testSimpleReserveMemoryCalculationNoHashNotFirstCycle() {
+    testSimpleReserveMemoryCalculationNoHashHelper(false);
+  }
+
+  private void testSimpleReserveMemoryCalculationNoHashHelper(final boolean firstCycle) {
     final int maxBatchNumRecords = 20;
     final double fragmentationFactor = 2.0;
     final double safetyFactor = 1.5;
@@ -39,12 +48,12 @@ public class TestBuildSidePartitioningImpl {
         safetyFactor);
 
     final CaseInsensitiveMap<Long> keySizes = CaseInsensitiveMap.newHashMap();
+    final long accountedProbeBatchSize = firstCycle? 0: 10;
 
-    calc.initialize(true,
+    calc.initialize(firstCycle,
       false,
       keySizes,
-      200,
-      100,
+      190 + accountedProbeBatchSize,
       2,
       false,
       new MockBatchSizePredictor(20, 20, fragmentationFactor, safetyFactor),
@@ -62,7 +71,7 @@ public class TestBuildSidePartitioningImpl {
 
     long expectedReservedMemory = 60 // Max incoming batch size
       + 2 * 30 // build side batch for each spilled partition
-      + 60; // Max incoming probe batch size
+      + accountedProbeBatchSize; // Max incoming probe batch size
     long actualReservedMemory = calc.getBuildReservedMemory();
 
     Assert.assertEquals(expectedReservedMemory, actualReservedMemory);
@@ -89,7 +98,6 @@ public class TestBuildSidePartitioningImpl {
       true,
       keySizes,
       350,
-      100, // Ignored for test
       2,
       false,
       new MockBatchSizePredictor(20, 20, fragmentationFactor, safetyFactor),
@@ -107,7 +115,7 @@ public class TestBuildSidePartitioningImpl {
 
     long expectedReservedMemory = 60 // Max incoming batch size
       + 2 * (/* data size for batch */ 30 + /* Space reserved for hash value vector */ 10 * 4 * 2) // build side batch for each spilled partition
-      + 60; // Max incoming probe batch size
+      + 10; // Max incoming probe batch size
     long actualReservedMemory = calc.getBuildReservedMemory();
 
     Assert.assertEquals(expectedReservedMemory, actualReservedMemory);
@@ -135,7 +143,6 @@ public class TestBuildSidePartitioningImpl {
       false,
       keySizes,
       200,
-      100, // Ignored for test
       4,
       false,
       new MockBatchSizePredictor(20, 20, fragmentationFactor, safetyFactor),
@@ -153,17 +160,61 @@ public class TestBuildSidePartitioningImpl {
     calc.setPartitionStatSet(partitionStatSet);
 
     long expectedReservedMemory = 60 // Max incoming batch size
-      + 2 * 30 // build side batch for each spilled partition
-      + 60; // Max incoming probe batch size
+      + 2 * 30; // build side batch for each spilled partition
     long actualReservedMemory = calc.getBuildReservedMemory();
 
     Assert.assertEquals(expectedReservedMemory, actualReservedMemory);
     Assert.assertEquals(2, calc.getNumPartitions());
   }
 
+  @Test
+  public void testDontAdjustInitialPartitions() {
+    final int maxBatchNumRecords = 20;
+    final double fragmentationFactor = 2.0;
+    final double safetyFactor = 1.5;
+
+    final HashJoinMemoryCalculatorImpl.BuildSidePartitioningImpl calc =
+      new HashJoinMemoryCalculatorImpl.BuildSidePartitioningImpl(
+        BatchSizePredictorImpl.Factory.INSTANCE,
+        new HashTableSizeCalculatorConservativeImpl(RecordBatch.MAX_BATCH_ROW_COUNT, HashTableSizeCalculatorConservativeImpl.HASHTABLE_DOUBLING_FACTOR),
+        HashJoinHelperSizeCalculatorImpl.INSTANCE,
+        fragmentationFactor,
+        safetyFactor);
+
+    final CaseInsensitiveMap<Long> keySizes = CaseInsensitiveMap.newHashMap();
+
+    calc.initialize(
+      false,
+      false,
+      keySizes,
+      200,
+      4,
+      false,
+      new MockBatchSizePredictor(20, 20, fragmentationFactor, safetyFactor),
+      new MockBatchSizePredictor(10, 10, fragmentationFactor, safetyFactor),
+      10,
+      5,
+      maxBatchNumRecords,
+      maxBatchNumRecords,
+      16000,
+      .75);
+
+    final HashJoinMemoryCalculator.PartitionStatSet partitionStatSet =
+      new HashJoinMemoryCalculator.PartitionStatSet(new PartitionStatImpl(), new PartitionStatImpl(),
+        new PartitionStatImpl(), new PartitionStatImpl());
+    calc.setPartitionStatSet(partitionStatSet);
+
+    long expectedReservedMemory = 60 // Max incoming batch size
+      + 4 * 30 // build side batch for each spilled partition
+      + 10; // Max incoming probe batch size
+    long actualReservedMemory = calc.getBuildReservedMemory();
+
+    Assert.assertEquals(expectedReservedMemory, actualReservedMemory);
+    Assert.assertEquals(4, calc.getNumPartitions());
+  }
+
   @Test(expected = IllegalStateException.class)
   public void testHasDataProbeEmpty() {
-    final int maxIncomingBatchSize = 100;
     final int maxBatchNumRecords = 20;
     final double fragmentationFactor = 2.0;
     final double safetyFactor = 1.5;
@@ -183,7 +234,6 @@ public class TestBuildSidePartitioningImpl {
       false,
       keySizes,
       240,
-      maxIncomingBatchSize,
       4,
       true,
       new MockBatchSizePredictor(20, 20, fragmentationFactor, safetyFactor),
@@ -198,7 +248,6 @@ public class TestBuildSidePartitioningImpl {
 
   @Test
   public void testNoProbeDataForStats() {
-    final int maxIncomingBatchSize = 100;
     final int maxBatchNumRecords = 20;
     final double fragmentationFactor = 2.0;
     final double safetyFactor = 1.5;
@@ -218,7 +267,6 @@ public class TestBuildSidePartitioningImpl {
       false,
       keySizes,
       240,
-      maxIncomingBatchSize,
       4,
       false,
       new MockBatchSizePredictor(20, 20, fragmentationFactor, safetyFactor),
@@ -235,12 +283,11 @@ public class TestBuildSidePartitioningImpl {
     calc.setPartitionStatSet(partitionStatSet);
 
     long expectedReservedMemory = 60 // Max incoming batch size
-      + 2 * 30 // build side batch for each spilled partition
-      + maxIncomingBatchSize;
+      + 4 * 30; // build side batch for each spilled partition
     long actualReservedMemory = calc.getBuildReservedMemory();
 
     Assert.assertEquals(expectedReservedMemory, actualReservedMemory);
-    Assert.assertEquals(2, calc.getNumPartitions());
+    Assert.assertEquals(4, calc.getNumPartitions());
   }
 
   @Test
@@ -264,7 +311,6 @@ public class TestBuildSidePartitioningImpl {
       false,
       keySizes,
       200,
-      100, // Ignored for test
       4,
       true,
       new MockBatchSizePredictor(20, 20, fragmentationFactor, safetyFactor),
@@ -290,10 +336,20 @@ public class TestBuildSidePartitioningImpl {
   }
 
   @Test
-  public void testNoRoomInMemoryForBatch1() {
+  public void testNoRoomInMemoryForBatch1FirstCycle() {
+    testNoRoomInMemoryForBatch1Helper(true);
+  }
+
+  @Test
+  public void testNoRoomInMemoryForBatch1NotFirstCycle() {
+    testNoRoomInMemoryForBatch1Helper(false);
+  }
+
+  private void testNoRoomInMemoryForBatch1Helper(final boolean firstCycle) {
     final int maxBatchNumRecords = 20;
     final double fragmentationFactor = 2.0;
     final double safetyFactor = 1.5;
+    final long accountedProbeBatchSize = firstCycle? 0: 10;
 
     final HashJoinMemoryCalculatorImpl.BuildSidePartitioningImpl calc =
       new HashJoinMemoryCalculatorImpl.BuildSidePartitioningImpl(
@@ -306,11 +362,10 @@ public class TestBuildSidePartitioningImpl {
     final CaseInsensitiveMap<Long> keySizes = CaseInsensitiveMap.newHashMap();
 
     calc.initialize(
-      true,
+      firstCycle,
       false,
       keySizes,
-      180,
-      100, // Ignored for test
+      120 + accountedProbeBatchSize,
       2,
       false,
       new MockBatchSizePredictor(20, 20, fragmentationFactor, safetyFactor),
@@ -330,7 +385,7 @@ public class TestBuildSidePartitioningImpl {
 
     long expectedReservedMemory = 60 // Max incoming batch size
       + 2 * 30 // build side batch for each spilled partition
-      + 60; // Max incoming probe batch size
+      + accountedProbeBatchSize; // Max incoming probe batch size
     long actualReservedMemory = calc.getBuildReservedMemory();
 
     Assert.assertEquals(expectedReservedMemory, actualReservedMemory);
@@ -360,8 +415,7 @@ public class TestBuildSidePartitioningImpl {
       true,
       false,
       keySizes,
-      210,
-      100, // Ignored for test
+      160,
       2,
       false,
       new MockBatchSizePredictor(20, 20, fragmentationFactor, safetyFactor),
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestPostBuildCalculationsImpl.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestPostBuildCalculationsImpl.java
index aa7a435..fa3fdfb 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestPostBuildCalculationsImpl.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestPostBuildCalculationsImpl.java
@@ -121,6 +121,7 @@ public class TestPostBuildCalculationsImpl {
 
     final HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl calc =
       new HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl(
+        true,
         new ConditionalMockBatchSizePredictor(
           Lists.newArrayList(maxBatchNumRecordsProbe, recordsPerPartitionBatchProbe),
           Lists.newArrayList(maxProbeBatchSize, partitionProbeBatchSize),
@@ -167,6 +168,7 @@ public class TestPostBuildCalculationsImpl {
 
     final HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl calc =
       new HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl(
+        true,
         new ConditionalMockBatchSizePredictor(),
         50,
         1000,
@@ -188,7 +190,16 @@ public class TestPostBuildCalculationsImpl {
   }
 
   @Test
-  public void testHasNoProbeDataButProbeNonEmpty() {
+  public void testHasNoProbeDataButProbeNonEmptyFirstCycle() {
+    testHasNoProbeDataButProbeNonEmptyHelper(true);
+  }
+
+  @Test
+  public void testHasNoProbeDataButProbeNonEmptyNotFirstCycle() {
+    testHasNoProbeDataButProbeNonEmptyHelper(false);
+  }
+
+  private void testHasNoProbeDataButProbeNonEmptyHelper(final boolean firstCycle) {
     final Map<String, Long> keySizes = org.apache.drill.common.map.CaseInsensitiveMap.newHashMap();
 
     final PartitionStatImpl partition1 = new PartitionStatImpl();
@@ -210,14 +221,16 @@ public class TestPostBuildCalculationsImpl {
     final int recordsPerPartitionBatchProbe = 5;
     final long partitionProbeBatchSize = 15;
     final long maxProbeBatchSize = 60;
+    final long accountedProbeBatchSize = (firstCycle? 0: maxProbeBatchSize);
 
     final HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl calc =
       new HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl(
+        firstCycle,
         new ConditionalMockBatchSizePredictor(
           Lists.newArrayList(maxBatchNumRecordsProbe, recordsPerPartitionBatchProbe),
           Lists.newArrayList(maxProbeBatchSize, partitionProbeBatchSize),
           false),
-        290,
+        230 + accountedProbeBatchSize,
         20,
         maxBatchNumRecordsProbe,
         recordsPerPartitionBatchProbe,
@@ -232,7 +245,7 @@ public class TestPostBuildCalculationsImpl {
 
     calc.initialize(false);
 
-    long expected = 60 // maxProbeBatchSize
+    long expected = accountedProbeBatchSize
       + 160 // in memory partitions
       + 20 // max output batch size
       + 2 * 10 // Hash Table
@@ -243,7 +256,16 @@ public class TestPostBuildCalculationsImpl {
   }
 
   @Test
-  public void testProbingAndPartitioningBuildAllInMemoryNoSpill() {
+  public void testProbingAndPartitioningBuildAllInMemoryNoSpillFirstCycle() {
+    testProbingAndPartitioningBuildAllInMemoryNoSpillHelper(true);
+  }
+
+  @Test
+  public void testProbingAndPartitioningBuildAllInMemoryNoSpillNotFirstCycle() {
+    testProbingAndPartitioningBuildAllInMemoryNoSpillHelper(false);
+  }
+
+  private void testProbingAndPartitioningBuildAllInMemoryNoSpillHelper(final boolean firstCycle) {
     final Map<String, Long> keySizes = org.apache.drill.common.map.CaseInsensitiveMap.newHashMap();
 
     final PartitionStatImpl partition1 = new PartitionStatImpl();
@@ -265,14 +287,16 @@ public class TestPostBuildCalculationsImpl {
     final int recordsPerPartitionBatchProbe = 5;
     final long partitionProbeBatchSize = 15;
     final long maxProbeBatchSize = 60;
+    final long accountedProbeBatchSize = (firstCycle? 0: maxProbeBatchSize);
 
     final HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl calc =
       new HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl(
+        firstCycle,
         new ConditionalMockBatchSizePredictor(
           Lists.newArrayList(maxBatchNumRecordsProbe, recordsPerPartitionBatchProbe),
           Lists.newArrayList(maxProbeBatchSize, partitionProbeBatchSize),
           true),
-        290,
+        230 + accountedProbeBatchSize,
         20,
         maxBatchNumRecordsProbe,
         recordsPerPartitionBatchProbe,
@@ -287,7 +311,7 @@ public class TestPostBuildCalculationsImpl {
 
     calc.initialize(false);
 
-    long expected = 60 // maxProbeBatchSize
+    long expected = accountedProbeBatchSize
       + 160 // in memory partitions
       + 20 // max output batch size
       + 2 * 10 // Hash Table
@@ -298,7 +322,16 @@ public class TestPostBuildCalculationsImpl {
   }
 
   @Test
-  public void testProbingAndPartitioningBuildAllInMemorySpill() {
+  public void testProbingAndPartitioningBuildAllInMemorySpillFirstCycle() {
+    testProbingAndPartitioningBuildAllInMemorySpillHelper(true);
+  }
+
+  @Test
+  public void testProbingAndPartitioningBuildAllInMemorySpillNotFirstCycle() {
+    testProbingAndPartitioningBuildAllInMemorySpillHelper(false);
+  }
+
+  private void testProbingAndPartitioningBuildAllInMemorySpillHelper(final boolean firstCycle) {
     final Map<String, Long> keySizes = org.apache.drill.common.map.CaseInsensitiveMap.newHashMap();
 
     final PartitionStatImpl partition1 = new PartitionStatImpl();
@@ -320,16 +353,18 @@ public class TestPostBuildCalculationsImpl {
     final int recordsPerPartitionBatchProbe = 5;
     final long partitionProbeBatchSize = 15;
     final long maxProbeBatchSize = 60;
+    final long accountedProbeBatchSize = (firstCycle? 0: maxProbeBatchSize);
 
     HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl calc =
       new HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl(
+        firstCycle,
         new ConditionalMockBatchSizePredictor(
           Lists.newArrayList(maxBatchNumRecordsProbe, recordsPerPartitionBatchProbe),
           Lists.newArrayList(maxProbeBatchSize, partitionProbeBatchSize),
           true),
-        270,
+        210 + accountedProbeBatchSize,
         20,
-         maxBatchNumRecordsProbe,
+        maxBatchNumRecordsProbe,
         recordsPerPartitionBatchProbe,
         buildPartitionStatSet,
         keySizes,
@@ -342,7 +377,7 @@ public class TestPostBuildCalculationsImpl {
 
     calc.initialize(false);
 
-    long expected = 60 // maxProbeBatchSize
+    long expected = accountedProbeBatchSize
       + 160 // in memory partitions
       + 20 // max output batch size
       + 2 * 10 // Hash Table
@@ -351,7 +386,7 @@ public class TestPostBuildCalculationsImpl {
     Assert.assertEquals(expected, calc.getConsumedMemory());
     partition1.spill();
 
-    expected = 60 // maxProbeBatchSize
+    expected = accountedProbeBatchSize
       + 80 // in memory partitions
       + 20 // max output batch size
       + 10 // Hash Table
@@ -363,7 +398,16 @@ public class TestPostBuildCalculationsImpl {
   }
 
   @Test
-  public void testProbingAndPartitioningBuildAllInMemoryNoSpillWithHash() {
+  public void testProbingAndPartitioningBuildAllInMemoryNoSpillWithHashFirstCycle() {
+    testProbingAndPartitioningBuildAllInMemoryNoSpillWithHashHelper(true);
+  }
+
+  @Test
+  public void testProbingAndPartitioningBuildAllInMemoryNoSpillWithHashNotFirstCycle() {
+    testProbingAndPartitioningBuildAllInMemoryNoSpillWithHashHelper(false);
+  }
+
+  private void testProbingAndPartitioningBuildAllInMemoryNoSpillWithHashHelper(final boolean firstCycle) {
     final Map<String, Long> keySizes = org.apache.drill.common.map.CaseInsensitiveMap.newHashMap();
 
     final PartitionStatImpl partition1 = new PartitionStatImpl();
@@ -381,14 +425,16 @@ public class TestPostBuildCalculationsImpl {
     final int recordsPerPartitionBatchProbe = 5;
     final long partitionProbeBatchSize = 15;
     final long maxProbeBatchSize = 60;
+    final long accountedProbeBatchSize = (firstCycle? 0: maxProbeBatchSize);
 
     HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl calc =
       new HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl(
+        firstCycle,
         new ConditionalMockBatchSizePredictor(
           Lists.newArrayList(maxBatchNumRecordsProbe, recordsPerPartitionBatchProbe),
           Lists.newArrayList(maxProbeBatchSize, partitionProbeBatchSize),
           true),
-        180,
+        120 + accountedProbeBatchSize,
         20,
         maxBatchNumRecordsProbe,
         recordsPerPartitionBatchProbe,
@@ -403,7 +449,7 @@ public class TestPostBuildCalculationsImpl {
 
     calc.initialize(false);
 
-    long expected = 60 // maxProbeBatchSize
+    long expected = accountedProbeBatchSize // probe batch
       + 2 * 5 * 3 // partition batches
       + 20; // max output batch size
     Assert.assertFalse(calc.shouldSpill());
@@ -412,7 +458,16 @@ public class TestPostBuildCalculationsImpl {
   }
 
   @Test
-  public void testProbingAndPartitioningBuildAllInMemoryWithSpill() {
+  public void testProbingAndPartitioningBuildAllInMemoryWithSpillFirstCycle() {
+    testProbingAndPartitioningBuildAllInMemoryWithSpillHelper(true);
+  }
+
+  @Test
+  public void testProbingAndPartitioningBuildAllInMemoryWithSpillNotFirstCycle() {
+    testProbingAndPartitioningBuildAllInMemoryWithSpillHelper(false);
+  }
+
+  private void testProbingAndPartitioningBuildAllInMemoryWithSpillHelper(final boolean firstCycle) {
     final Map<String, Long> keySizes = org.apache.drill.common.map.CaseInsensitiveMap.newHashMap();
 
     final PartitionStatImpl partition1 = new PartitionStatImpl();
@@ -434,14 +489,16 @@ public class TestPostBuildCalculationsImpl {
     final int recordsPerPartitionBatchProbe = 5;
     final long partitionProbeBatchSize = 15;
     final long maxProbeBatchSize = 60;
+    final long accountedProbeBatchSize = (firstCycle? 0: maxProbeBatchSize);
 
     HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl calc =
       new HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl(
+        firstCycle,
         new ConditionalMockBatchSizePredictor(
           Lists.newArrayList(maxBatchNumRecordsProbe, recordsPerPartitionBatchProbe),
           Lists.newArrayList(maxProbeBatchSize, partitionProbeBatchSize),
           true),
-        200,
+        140 + accountedProbeBatchSize,
         20,
         maxBatchNumRecordsProbe,
         recordsPerPartitionBatchProbe,
@@ -456,7 +513,7 @@ public class TestPostBuildCalculationsImpl {
 
     calc.initialize(false);
 
-    long expected = 60 // maxProbeBatchSize
+    long expected = accountedProbeBatchSize
       + 80 // in memory partition
       + 10 // hash table size
       + 10 // hash join helper size
@@ -471,7 +528,16 @@ public class TestPostBuildCalculationsImpl {
   }
 
   @Test
-  public void testProbingAndPartitioningBuildSomeInMemory() {
+  public void testProbingAndPartitioningBuildSomeInMemoryFirstCycle() {
+    testProbingAndPartitioningBuildSomeInMemoryHelper(true);
+  }
+
+  @Test
+  public void testProbingAndPartitioningBuildSomeInMemoryNotFirstCycle() {
+    testProbingAndPartitioningBuildSomeInMemoryHelper(false);
+  }
+
+  private void testProbingAndPartitioningBuildSomeInMemoryHelper(final boolean firstCycle) {
     final Map<String, Long> keySizes = org.apache.drill.common.map.CaseInsensitiveMap.newHashMap();
 
     final PartitionStatImpl partition1 = new PartitionStatImpl();
@@ -497,14 +563,16 @@ public class TestPostBuildCalculationsImpl {
     final int recordsPerPartitionBatchProbe = 5;
     final long partitionProbeBatchSize = 15;
     final long maxProbeBatchSize = 60;
+    final long accountedProbeBatchSize = (firstCycle? 0: maxProbeBatchSize);
 
     HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl calc =
       new HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl(
+        firstCycle,
         new ConditionalMockBatchSizePredictor(
           Lists.newArrayList(maxBatchNumRecordsProbe, recordsPerPartitionBatchProbe),
           Lists.newArrayList(maxProbeBatchSize, partitionProbeBatchSize),
           true),
-        230,
+        170 + accountedProbeBatchSize,
         20,
         maxBatchNumRecordsProbe,
         recordsPerPartitionBatchProbe,
@@ -519,7 +587,7 @@ public class TestPostBuildCalculationsImpl {
 
     calc.initialize(false);
 
-    long expected = 60 // maxProbeBatchSize
+    long expected = accountedProbeBatchSize
       + 80 // in memory partition
       + 10 // hash table size
       + 10 // hash join helper size
@@ -533,8 +601,16 @@ public class TestPostBuildCalculationsImpl {
   }
 
   @Test
-  public void testProbingAndPartitioningBuildNoneInMemory() {
+  public void testProbingAndPartitioningBuildNoneInMemoryFirstCycle() {
+    testProbingAndPartitioningBuildNoneInMemoryHelper(true);
+  }
 
+  @Test
+  public void testProbingAndPartitioningBuildNoneInMemoryNotFirstCycle() {
+    testProbingAndPartitioningBuildNoneInMemoryHelper(false);
+  }
+
+  private void testProbingAndPartitioningBuildNoneInMemoryHelper(final boolean firstCycle) {
     final Map<String, Long> keySizes = org.apache.drill.common.map.CaseInsensitiveMap.newHashMap();
 
     final PartitionStatImpl partition1 = new PartitionStatImpl();
@@ -554,14 +630,16 @@ public class TestPostBuildCalculationsImpl {
     final int recordsPerPartitionBatchProbe = 5;
     final long partitionProbeBatchSize = 15;
     final long maxProbeBatchSize = 60;
+    final long accountedProbeBatchSize = (firstCycle? 0: maxProbeBatchSize);
 
     HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl calc =
       new HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl(
+        firstCycle,
         new ConditionalMockBatchSizePredictor(
           Lists.newArrayList(maxBatchNumRecordsProbe, recordsPerPartitionBatchProbe),
           Lists.newArrayList(maxProbeBatchSize, partitionProbeBatchSize),
           true),
-        110,
+        110 + accountedProbeBatchSize,
         20,
         maxBatchNumRecordsProbe,
         recordsPerPartitionBatchProbe,
@@ -576,7 +654,7 @@ public class TestPostBuildCalculationsImpl {
 
     calc.initialize(false);
     Assert.assertFalse(calc.shouldSpill());
-    Assert.assertEquals(110, calc.getConsumedMemory());
+    Assert.assertEquals(50 + accountedProbeBatchSize, calc.getConsumedMemory());
     Assert.assertNotNull(calc.next());
   }
 
@@ -611,6 +689,7 @@ public class TestPostBuildCalculationsImpl {
 
     HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl calc =
       new HashJoinMemoryCalculatorImpl.PostBuildCalculationsImpl(
+        true,
         new ConditionalMockBatchSizePredictor(
           Lists.newArrayList(maxBatchNumRecordsProbe, recordsPerPartitionBatchProbe),
           Lists.newArrayList(maxProbeBatchSize, partitionProbeBatchSize),
@@ -708,7 +787,7 @@ public class TestPostBuildCalculationsImpl {
 
     @Override
     public long getBatchSize() {
-      return 0;
+      return batchSize.get(0);
     }
 
     @Override
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestNullInputMiniPlan.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestNullInputMiniPlan.java
index 9ad7a9b..e1609a5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestNullInputMiniPlan.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/unit/TestNullInputMiniPlan.java
@@ -316,7 +316,7 @@ public class TestNullInputMiniPlan extends MiniPlanUnitTestBase{
         .build();
 
     RecordBatch joinBatch = new PopBuilder()
-        .physicalOperator(new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "a2")), JoinRelType.INNER, null))
+        .physicalOperator(new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a2", "EQUALS", "a")), JoinRelType.INNER, null))
         .addInput(left)
         .addInput(rightScan)
         .build();
@@ -379,7 +379,7 @@ public class TestNullInputMiniPlan extends MiniPlanUnitTestBase{
         .build();
 
     RecordBatch joinBatch = new PopBuilder()
-        .physicalOperator(new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a", "EQUALS", "a2")), JoinRelType.LEFT, null))
+        .physicalOperator(new HashJoinPOP(null, null, Lists.newArrayList(joinCond("a2", "EQUALS", "a")), JoinRelType.LEFT, null))
         .addInput(left)
         .addInput(rightScan)
         .build();


[drill] 02/06: DRILL-6647: Update Calcite version to 1.17.0

Posted by ar...@apache.org.
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

commit 989618287559de5e87349c637cb91150ea6a5135
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Mon Jul 2 17:59:42 2018 +0300

    DRILL-6647: Update Calcite version to 1.17.0
    
    closes #1425
---
 .../drill/exec/store/jdbc/JdbcStoragePlugin.java   |    2 +-
 exec/java-exec/src/main/codegen/data/Parser.tdd    |    8 +-
 .../logical/DirPrunedEnumerableTableScan.java      |   12 +-
 .../exec/planner/logical/DrillConditions.java      |   20 +-
 .../exec/planner/sql/DrillConvertletTable.java     |   35 +-
 .../drill/exec/planner/sql/SqlConverter.java       |    2 +-
 .../apache/drill/exec/store/AbstractSchema.java    |   27 +
 .../drill/jdbc/impl/DrillConnectionImpl.java       |  400 ++----
 .../drill/jdbc/impl/DrillDatabaseMetaDataImpl.java |  354 +++--
 .../apache/drill/jdbc/impl/DrillJdbc41Factory.java |  120 +-
 .../jdbc/impl/DrillPreparedStatementImpl.java      |  463 +-----
 .../apache/drill/jdbc/impl/DrillResultSetImpl.java | 1494 ++++++--------------
 .../jdbc/impl/DrillResultSetMetaDataImpl.java      |   10 +-
 .../apache/drill/jdbc/impl/DrillStatementImpl.java |  327 +----
 .../java/org/apache/drill/jdbc/JdbcTestBase.java   |  129 +-
 ...rill2489CallsAfterCloseThrowExceptionsTest.java |  107 +-
 pom.xml                                            |    4 +-
 17 files changed, 917 insertions(+), 2597 deletions(-)

diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStoragePlugin.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStoragePlugin.java
index efb5a5c..4de3cf8 100755
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStoragePlugin.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStoragePlugin.java
@@ -46,7 +46,7 @@ import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlDialectFactoryImpl;
-import org.apache.commons.dbcp.BasicDataSource;
+import org.apache.commons.dbcp2.BasicDataSource;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
diff --git a/exec/java-exec/src/main/codegen/data/Parser.tdd b/exec/java-exec/src/main/codegen/data/Parser.tdd
index 507d692..5891fb7 100644
--- a/exec/java-exec/src/main/codegen/data/Parser.tdd
+++ b/exec/java-exec/src/main/codegen/data/Parser.tdd
@@ -85,13 +85,17 @@
 
   # List of methods for parsing extensions to "DROP" calls.
   # Each must accept arguments "(SqlParserPos pos)".
-    dropStatementParserMethods: [
+  dropStatementParserMethods: [
   ]
 
   # List of keywords from "keywords" section that are not reserved.
-    nonReservedKeywords: [
+  nonReservedKeywords: [
   ]
 
+  # List of additional join types. Each is a method with no arguments.
+  # Example: LeftSemiJoin()
+  joinTypes: [
+  ]
   
   includeCompoundIdentifier: false,
   includeBraces: true,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DirPrunedEnumerableTableScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DirPrunedEnumerableTableScan.java
index 2202986..7fd348f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DirPrunedEnumerableTableScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DirPrunedEnumerableTableScan.java
@@ -17,14 +17,12 @@
  */
 package org.apache.drill.exec.planner.logical;
 
-import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
 import org.apache.calcite.adapter.enumerable.EnumerableConvention;
 import org.apache.calcite.adapter.enumerable.EnumerableTableScan;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
@@ -62,13 +60,11 @@ public class DirPrunedEnumerableTableScan extends EnumerableTableScan {
     final RelTraitSet traitSet =
         cluster.traitSetOf(EnumerableConvention.INSTANCE)
             .replaceIfs(RelCollationTraitDef.INSTANCE,
-                new Supplier<List<RelCollation>>() {
-                  public List<RelCollation> get() {
-                    if (table != null) {
-                      return table.getStatistic().getCollations();
-                    }
-                    return ImmutableList.of();
+                () -> {
+                  if (table != null) {
+                    return table.getStatistic().getCollations();
                   }
+                  return ImmutableList.of();
                 });
     return new DirPrunedEnumerableTableScan(cluster, traitSet, relOptTable, elementType, digestFromSelection);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConditions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConditions.java
index 266ce64..1341be1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConditions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConditions.java
@@ -19,24 +19,14 @@ package org.apache.drill.exec.planner.logical;
 
 import org.apache.calcite.rel.rules.PushProjector;
 import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexNode;
 
 public final class DrillConditions {
 
-  public static PushProjector.ExprCondition PRESERVE_ITEM = new PushProjector.ExprCondition() {
-
-    @Override
-    public boolean apply(RexNode rexNode) {
-      return false;
-    }
-
-    @Override
-    public boolean test(RexNode expr) {
-      if (expr instanceof RexCall) {
-        RexCall call = (RexCall)expr;
-        return "item".equals(call.getOperator().getName().toLowerCase());
-      }
-      return false;
+  public static final PushProjector.ExprCondition PRESERVE_ITEM = expr -> {
+    if (expr instanceof RexCall) {
+      RexCall call = (RexCall) expr;
+      return "item".equals(call.getOperator().getName().toLowerCase());
     }
+    return false;
   };
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillConvertletTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillConvertletTable.java
index 34141e5..5f3b95e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillConvertletTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillConvertletTable.java
@@ -17,7 +17,9 @@
  */
 package org.apache.drill.exec.planner.sql;
 
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlBasicCall;
@@ -25,7 +27,6 @@ import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql2rel.SqlRexContext;
 import org.apache.calcite.sql2rel.SqlRexConvertlet;
 import org.apache.calcite.sql2rel.SqlRexConvertletTable;
 import org.apache.calcite.sql2rel.StandardConvertletTable;
@@ -36,19 +37,37 @@ public class DrillConvertletTable implements SqlRexConvertletTable{
   public static HashMap<SqlOperator, SqlRexConvertlet> map = new HashMap<>();
 
   public static SqlRexConvertletTable INSTANCE = new DrillConvertletTable();
-  private static SqlRexConvertlet sqrtConvertlet = new SqlRexConvertlet() {
-    public RexNode convertCall(SqlRexContext cx, SqlCall call) {
-      RexNode operand = cx.convertExpression(call.operand(0));
-      return cx.getRexBuilder().makeCall(SqlStdOperatorTable.SQRT, operand);
+
+  private static final SqlRexConvertlet SQRT_CONVERTLET = (cx, call) -> {
+    RexNode operand = cx.convertExpression(call.operand(0));
+    return cx.getRexBuilder().makeCall(SqlStdOperatorTable.SQRT, operand);
+  };
+
+  // Rewrites COALESCE function into CASE WHEN IS NOT NULL operand1 THEN operand1...
+  private static final SqlRexConvertlet COALESCE_CONVERTLET = (cx, call) -> {
+    int operandsCount = call.operandCount();
+    if (operandsCount == 1) {
+      return cx.convertExpression(call.operand(0));
+    } else {
+      List<RexNode> caseOperands = new ArrayList<>();
+      for (int i = 0; i < operandsCount - 1; i++) {
+        RexNode caseOperand = cx.convertExpression(call.operand(i));
+        caseOperands.add(cx.getRexBuilder().makeCall(
+            SqlStdOperatorTable.IS_NOT_NULL, caseOperand));
+        caseOperands.add(caseOperand);
+      }
+      caseOperands.add(cx.convertExpression(call.operand(operandsCount - 1)));
+      return cx.getRexBuilder().makeCall(SqlStdOperatorTable.CASE, caseOperands);
     }
   };
 
   static {
-    // Use custom convertlet for extract function
+    // Use custom convertlet for EXTRACT function
     map.put(SqlStdOperatorTable.EXTRACT, DrillExtractConvertlet.INSTANCE);
-    // sqrt needs it's own convertlet because calcite overrides it to power(x,0.5)
+    // SQRT needs it's own convertlet because calcite overrides it to POWER(x, 0.5)
     // which is not suitable for Infinity value case
-    map.put(SqlStdOperatorTable.SQRT, sqrtConvertlet);
+    map.put(SqlStdOperatorTable.SQRT, SQRT_CONVERTLET);
+    map.put(SqlStdOperatorTable.COALESCE, COALESCE_CONVERTLET);
     map.put(SqlStdOperatorTable.AVG, new DrillAvgVarianceConvertlet(SqlKind.AVG));
     map.put(SqlStdOperatorTable.STDDEV_POP, new DrillAvgVarianceConvertlet(SqlKind.STDDEV_POP));
     map.put(SqlStdOperatorTable.STDDEV_SAMP, new DrillAvgVarianceConvertlet(SqlKind.STDDEV_SAMP));
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
index d4da23f..3ab92ab 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
@@ -377,7 +377,7 @@ public class SqlConverter {
      * during creating new projects since it may cause changing data mode
      * which causes to assertion errors during type validation
      */
-    Hook.REL_BUILDER_SIMPLIFY.add(Hook.property(false));
+    Hook.REL_BUILDER_SIMPLIFY.add(Hook.propertyJ(false));
 
     //To avoid unexpected column errors set a value of top to false
     final RelRoot rel = sqlToRelConverter.convertQuery(validatedNode, false, false);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
index 2b70c3b..18e3963 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
@@ -22,10 +22,13 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.calcite.linq4j.tree.DefaultExpression;
 import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaPlus;
@@ -163,6 +166,30 @@ public abstract class AbstractSchema implements Schema, SchemaPartitionExplorer,
     return Collections.emptyList();
   }
 
+  /**
+   * Returns a map of types in this schema by name.
+   *
+   * <p>The implementations of {@link #getTypeNames()}
+   * and {@link #getType(String)} depend on this map.
+   * The default implementation of this method returns the empty map.
+   * Override this method to change their behavior.</p>
+   *
+   * @return Map of types in this schema by name
+   */
+  protected Map<String, RelProtoDataType> getTypeMap() {
+    return ImmutableMap.of();
+  }
+
+  @Override
+  public Set<String> getTypeNames() {
+    return getTypeMap().keySet();
+  }
+
+  @Override
+  public RelProtoDataType getType(String name) {
+    return getTypeMap().get(name);
+  }
+
   @Override
   public Set<String> getFunctionNames() {
     return Collections.emptySet();
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java
index 2bac4ad..e005175 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java
@@ -18,11 +18,9 @@
 package org.apache.drill.jdbc.impl;
 
 import java.io.File;
-import java.sql.Array;
 import java.sql.Blob;
 import java.sql.CallableStatement;
 import java.sql.Clob;
-import java.sql.DatabaseMetaData;
 import java.sql.NClob;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
@@ -30,10 +28,8 @@ import java.sql.SQLClientInfoException;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 import java.sql.SQLNonTransientConnectionException;
-import java.sql.SQLWarning;
 import java.sql.SQLXML;
 import java.sql.Savepoint;
-import java.sql.Statement;
 import java.sql.Struct;
 import java.util.Map;
 import java.util.Properties;
@@ -80,7 +76,7 @@ import static org.apache.drill.exec.util.StoragePluginTestUtils.UNIT_TEST_PROP_P
 import static org.apache.drill.exec.util.StoragePluginTestUtils.updateSchemaLocation;
 
 /**
- * Drill's implementation of {@link Connection}.
+ * Drill's implementation of {@link java.sql.Connection}.
  */
 // (Was abstract to avoid errors _here_ if newer versions of JDBC added
 // interface methods, but now newer versions would probably use Java 8's default
@@ -208,9 +204,10 @@ class DrillConnectionImpl extends AvaticaConnection
    *
    * @throws  AlreadyClosedSqlException  if Connection is closed
    */
-  private void throwIfClosed() throws AlreadyClosedSqlException {
-    if ( isClosed() ) {
-      throw new AlreadyClosedSqlException( "Connection is already closed." );
+  @Override
+  protected void checkOpen() throws AlreadyClosedSqlException {
+    if (isClosed()) {
+      throw new AlreadyClosedSqlException("Connection is already closed.");
     }
   }
 
@@ -229,9 +226,9 @@ class DrillConnectionImpl extends AvaticaConnection
   }
 
   @Override
-  public void setAutoCommit( boolean autoCommit ) throws SQLException {
-    throwIfClosed();
-    if ( ! autoCommit ) {
+  public void setAutoCommit(boolean autoCommit) throws SQLException {
+    checkOpen();
+    if (!autoCommit) {
       throw new SQLFeatureNotSupportedException(
           "Can't turn off auto-committing; transactions are not supported.  "
           + "(Drill is not transactional.)" );
@@ -241,27 +238,25 @@ class DrillConnectionImpl extends AvaticaConnection
 
   @Override
   public void commit() throws SQLException {
-    throwIfClosed();
-    if ( getAutoCommit() ) {
-      throw new JdbcApiSqlException( "Can't call commit() in auto-commit mode." );
-    }
-    else {
+    checkOpen();
+    if (getAutoCommit()) {
+      throw new JdbcApiSqlException("Can't call commit() in auto-commit mode.");
+    } else {
       // (Currently not reachable.)
       throw new SQLFeatureNotSupportedException(
-          "Connection.commit() is not supported.  (Drill is not transactional.)" );
+          "Connection.commit() is not supported.  (Drill is not transactional.)");
     }
   }
 
   @Override
   public void rollback() throws SQLException {
-    throwIfClosed();
-    if ( getAutoCommit()  ) {
-      throw new JdbcApiSqlException( "Can't call rollback() in auto-commit mode." );
-    }
-    else {
+    checkOpen();
+    if (getAutoCommit()) {
+      throw new JdbcApiSqlException("Can't call rollback() in auto-commit mode.");
+    } else {
       // (Currently not reachable.)
       throw new SQLFeatureNotSupportedException(
-          "Connection.rollback() is not supported.  (Drill is not transactional.)" );
+          "Connection.rollback() is not supported.  (Drill is not transactional.)");
     }
   }
 
@@ -270,13 +265,12 @@ class DrillConnectionImpl extends AvaticaConnection
   public boolean isClosed() {
     try {
       return super.isClosed();
-    }
-    catch ( SQLException e ) {
+    } catch (SQLException e) {
       // Currently can't happen, since AvaticaConnection.isClosed() never throws
       // SQLException.
       throw new DrillRuntimeException(
           "Unexpected exception from " + getClass().getSuperclass()
-          + ".isClosed(): " + e,
+            + ".isClosed(): " + e,
           e );
     }
   }
@@ -284,40 +278,45 @@ class DrillConnectionImpl extends AvaticaConnection
 
   @Override
   public Savepoint setSavepoint() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     throw new SQLFeatureNotSupportedException(
-        "Savepoints are not supported.  (Drill is not transactional.)" );
+        "Savepoints are not supported. (Drill is not transactional.)" );
   }
 
   @Override
   public Savepoint setSavepoint(String name) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     throw new SQLFeatureNotSupportedException(
-        "Savepoints are not supported.  (Drill is not transactional.)" );
+        "Savepoints are not supported. (Drill is not transactional.)" );
   }
 
   @Override
     public void rollback(Savepoint savepoint) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     throw new SQLFeatureNotSupportedException(
-        "Savepoints are not supported.  (Drill is not transactional.)" );
+        "Savepoints are not supported. (Drill is not transactional.)" );
   }
 
   @Override
   public void releaseSavepoint(Savepoint savepoint) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     throw new SQLFeatureNotSupportedException(
-        "Savepoints are not supported.  (Drill is not transactional.)" );
+        "Savepoints are not supported. (Drill is not transactional.)" );
   }
 
 
-  private String isolationValueToString( final int level ) {
-    switch ( level ) {
-      case TRANSACTION_NONE:             return "TRANSACTION_NONE";
-      case TRANSACTION_READ_UNCOMMITTED: return "TRANSACTION_READ_UNCOMMITTED";
-      case TRANSACTION_READ_COMMITTED:   return "TRANSACTION_READ_COMMITTED";
-      case TRANSACTION_REPEATABLE_READ:  return "TRANSACTION_REPEATABLE_READ";
-      case TRANSACTION_SERIALIZABLE:     return "TRANSACTION_SERIALIZABLE";
+  private String isolationValueToString(int level) {
+    switch (level) {
+      case TRANSACTION_NONE:
+        return "TRANSACTION_NONE";
+      case TRANSACTION_READ_UNCOMMITTED:
+        return "TRANSACTION_READ_UNCOMMITTED";
+      case TRANSACTION_READ_COMMITTED:
+        return "TRANSACTION_READ_COMMITTED";
+      case TRANSACTION_REPEATABLE_READ:
+        return "TRANSACTION_REPEATABLE_READ";
+      case TRANSACTION_SERIALIZABLE:
+        return "TRANSACTION_SERIALIZABLE";
       default:
         return "<Unknown transaction isolation level value " + level + ">";
     }
@@ -325,8 +324,8 @@ class DrillConnectionImpl extends AvaticaConnection
 
   @Override
   public void setTransactionIsolation(int level) throws SQLException {
-    throwIfClosed();
-    switch ( level ) {
+    checkOpen();
+    switch (level) {
       case TRANSACTION_NONE:
         // No-op.  (Is already set in constructor, and we disallow changing it.)
         break;
@@ -334,73 +333,50 @@ class DrillConnectionImpl extends AvaticaConnection
       case TRANSACTION_READ_COMMITTED:
       case TRANSACTION_REPEATABLE_READ:
       case TRANSACTION_SERIALIZABLE:
-          throw new SQLFeatureNotSupportedException(
-              "Can't change transaction isolation level to Connection."
-              + isolationValueToString( level ) + " (from Connection."
-              + isolationValueToString( getTransactionIsolation() ) + ")."
-              + "  (Drill is not transactional.)" );
+        throw new SQLFeatureNotSupportedException(
+            "Can't change transaction isolation level to Connection."
+                + isolationValueToString(level) + " (from Connection."
+                + isolationValueToString(getTransactionIsolation()) + "). "
+                + "(Drill is not transactional.)");
       default:
         // Invalid value (or new one unknown to code).
         throw new JdbcApiSqlException(
-            "Invalid transaction isolation level value " + level );
+            "Invalid transaction isolation level value " + level);
         //break;
     }
   }
 
   @Override
-  public void setNetworkTimeout( Executor executor, int milliseconds )
-      throws AlreadyClosedSqlException,
-             JdbcApiSqlException,
-             SQLFeatureNotSupportedException {
-    throwIfClosed();
-    if ( null == executor ) {
+  public void setNetworkTimeout(Executor executor, int milliseconds)
+      throws JdbcApiSqlException, SQLFeatureNotSupportedException {
+    checkOpen();
+    if (null == executor) {
       throw new InvalidParameterSqlException(
-          "Invalid (null) \"executor\" parameter to setNetworkTimeout(...)" );
-    }
-    else if ( milliseconds < 0 ) {
+          "Invalid (null) \"executor\" parameter to setNetworkTimeout(...)");
+    } else if (milliseconds < 0) {
       throw new InvalidParameterSqlException(
           "Invalid (negative) \"milliseconds\" parameter to"
-          + " setNetworkTimeout(...) (" + milliseconds + ")" );
-    }
-    else {
-      if ( 0 != milliseconds ) {
+          + " setNetworkTimeout(...) (" + milliseconds + ")");
+    } else {
+      if (0 != milliseconds) {
         throw new SQLFeatureNotSupportedException(
-            "Setting network timeout is not supported." );
+            "Setting network timeout is not supported.");
       }
     }
   }
 
   @Override
-  public int getNetworkTimeout() throws AlreadyClosedSqlException
-  {
-    throwIfClosed();
+  public int getNetworkTimeout() throws AlreadyClosedSqlException {
+    checkOpen();
     return 0;  // (No timeout.)
   }
 
-
   @Override
   public DrillStatementImpl createStatement(int resultSetType,
                                             int resultSetConcurrency,
                                             int resultSetHoldability) throws SQLException {
-    throwIfClosed();
-    DrillStatementImpl statement =
-        (DrillStatementImpl) super.createStatement(resultSetType,
-                                                   resultSetConcurrency,
-                                                   resultSetHoldability);
-    return statement;
-  }
-
-  @Override
-  public PreparedStatement prepareStatement(String sql, int resultSetType,
-                                            int resultSetConcurrency,
-                                            int resultSetHoldability) throws SQLException {
-    throwIfClosed();
-    DrillPreparedStatementImpl statement =
-        (DrillPreparedStatementImpl) super.prepareStatement(sql,
-                                                            resultSetType,
-                                                            resultSetConcurrency,
-                                                            resultSetHoldability);
-    return statement;
+    return (DrillStatementImpl) super.createStatement(resultSetType,
+        resultSetConcurrency, resultSetHoldability);
   }
 
   @Override
@@ -420,172 +396,67 @@ class DrillConnectionImpl extends AvaticaConnection
   // No unwrap(Class<T>) (it doesn't throw SQLException if already closed).
 
   @Override
-  public AvaticaStatement createStatement() throws SQLException {
-    throwIfClosed();
-    return super.createStatement();
-  }
-
-  @Override
-  public PreparedStatement prepareStatement(String sql) throws SQLException {
-    throwIfClosed();
-    return super.prepareStatement(sql);
-  }
-
-  @Override
   public CallableStatement prepareCall(String sql) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.prepareCall(sql);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public String nativeSQL(String sql) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.nativeSQL(sql);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
-
-  @Override
-  public boolean getAutoCommit() throws SQLException {
-    throwIfClosed();
-    return super.getAutoCommit();
-  }
-
   // No close() (it doesn't throw SQLException if already closed).
 
   @Override
-  public DatabaseMetaData getMetaData() throws SQLException {
-    throwIfClosed();
-    return super.getMetaData();
-  }
-
-  @Override
-  public void setReadOnly(boolean readOnly) throws SQLException {
-    throwIfClosed();
-    super.setReadOnly(readOnly);
-  }
-
-  @Override
-  public boolean isReadOnly() throws SQLException {
-    throwIfClosed();
-    return super.isReadOnly();
-  }
-
-  @Override
-  public void setCatalog(String catalog) throws SQLException {
-    throwIfClosed();
-    super.setCatalog(catalog);
-  }
-
-  @Override
-  public String getCatalog() {
-    // Can't throw any SQLException because AvaticaConnection's getCatalog() is
-    // missing "throws SQLException".
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getCatalog();
-  }
-
-  @Override
-  public int getTransactionIsolation() throws SQLException {
-    throwIfClosed();
-    return super.getTransactionIsolation();
-  }
-
-  @Override
-  public SQLWarning getWarnings() throws SQLException {
-    throwIfClosed();
-    return super.getWarnings();
-  }
-
-  @Override
-  public void clearWarnings() throws SQLException {
-    throwIfClosed();
-    super.clearWarnings();
-  }
-
-  @Override
-  public Statement createStatement(int resultSetType,
-                                   int resultSetConcurrency) throws SQLException {
-    throwIfClosed();
-    return super.createStatement(resultSetType, resultSetConcurrency);
-  }
-
-  @Override
-  public PreparedStatement prepareStatement(String sql, int resultSetType,
-                                            int resultSetConcurrency) throws SQLException {
-    throwIfClosed();
-    return super.prepareStatement(sql, resultSetType, resultSetConcurrency);
-  }
-
-  @Override
   public CallableStatement prepareCall(String sql, int resultSetType,
                                        int resultSetConcurrency) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.prepareCall(sql, resultSetType, resultSetConcurrency);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public Map<String,Class<?>> getTypeMap() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.getTypeMap();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void setTypeMap(Map<String,Class<?>> map) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.setTypeMap(map);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void setHoldability(int holdability) throws SQLException {
-    throwIfClosed();
-    super.setHoldability(holdability);
-  }
-
-  @Override
-  public int getHoldability() throws SQLException {
-    throwIfClosed();
-    return super.getHoldability();
-  }
-
-  @Override
   public CallableStatement prepareCall(String sql, int resultSetType,
                                        int resultSetConcurrency,
                                        int resultSetHoldability) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.prepareCall(sql, resultSetType, resultSetConcurrency,
-                               resultSetHoldability);
-    }
-    catch (UnsupportedOperationException e) {
+        resultSetHoldability);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
@@ -593,11 +464,10 @@ class DrillConnectionImpl extends AvaticaConnection
   @Override
   public PreparedStatement prepareStatement(String sql,
                                             int autoGeneratedKeys) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.prepareStatement(sql, autoGeneratedKeys);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
@@ -605,11 +475,10 @@ class DrillConnectionImpl extends AvaticaConnection
   @Override
   public PreparedStatement prepareStatement(String sql,
                                             int columnIndexes[]) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.prepareStatement(sql, columnIndexes);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
@@ -617,81 +486,70 @@ class DrillConnectionImpl extends AvaticaConnection
   @Override
   public PreparedStatement prepareStatement(String sql,
                                             String columnNames[]) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.prepareStatement(sql, columnNames);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public Clob createClob() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.createClob();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public Blob createBlob() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.createBlob();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public NClob createNClob() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.createNClob();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public SQLXML createSQLXML() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.createSQLXML();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean isValid(int timeout) throws SQLException {
-    throwIfClosed();
-    try {
-      return super.isValid(timeout);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
+    checkOpen();
+    return super.isValid(timeout);
   }
 
   @Override
   public void setClientInfo(String name, String value) throws SQLClientInfoException {
     try {
-      throwIfClosed();
+      checkOpen();
     } catch (AlreadyClosedSqlException e) {
       throw new SQLClientInfoException(e.getMessage(), null, e);
     }
     try {
-      super.setClientInfo(name,  value);
-    }
-    catch (UnsupportedOperationException e) {
+      super.setClientInfo(name, value);
+    } catch (UnsupportedOperationException e) {
       SQLFeatureNotSupportedException intended =
           new SQLFeatureNotSupportedException(e.getMessage(), e);
       throw new SQLClientInfoException(e.getMessage(), null, intended);
@@ -701,14 +559,13 @@ class DrillConnectionImpl extends AvaticaConnection
   @Override
   public void setClientInfo(Properties properties) throws SQLClientInfoException {
     try {
-      throwIfClosed();
+      checkOpen();
     } catch (AlreadyClosedSqlException e) {
       throw new SQLClientInfoException(e.getMessage(), null, e);
     }
     try {
       super.setClientInfo(properties);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       SQLFeatureNotSupportedException intended =
           new SQLFeatureNotSupportedException(e.getMessage(), e);
       throw new SQLClientInfoException(e.getMessage(), null, intended);
@@ -716,80 +573,25 @@ class DrillConnectionImpl extends AvaticaConnection
   }
 
   @Override
-  public String getClientInfo(String name) throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getClientInfo(name);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public Properties getClientInfo() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getClientInfo();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public Array createArrayOf(String typeName, Object[] elements) throws SQLException {
-    throwIfClosed();
-    try {
-      return super.createArrayOf(typeName, elements);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
   public Struct createStruct(String typeName, Object[] attributes) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.createStruct(typeName, attributes);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void setSchema(String schema) throws SQLException {
-    throwIfClosed();
-    super.setSchema(schema);
-  }
-
-  @Override
-  public String getSchema() {
-    // Can't throw any SQLException because AvaticaConnection's getCatalog() is
-    // missing "throws SQLException".
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getSchema();
-  }
-
-  @Override
   public void abort(Executor executor) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.abort(executor);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
-
-
   // do not make public
   UnregisteredDriver getDriver() {
     return driver;
@@ -800,7 +602,7 @@ class DrillConnectionImpl extends AvaticaConnection
     return factory;
   }
 
-  private static void closeOrWarn(final AutoCloseable autoCloseable, final String message, final Logger logger) {
+  private static void closeOrWarn(AutoCloseable autoCloseable, String message, Logger logger) {
     if (autoCloseable == null) {
       return;
     }
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
index ccdc124..b7617b2 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
@@ -53,9 +53,8 @@ import com.google.common.base.Joiner;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableSet;
 
-
 /**
- * Drill's implementation of {@link DatabaseMetaData}.
+ * Drill's implementation of {@link java.sql.DatabaseMetaData}.
  */
 class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
                                 implements DrillDatabaseMetaData {
@@ -126,11 +125,10 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
    * @throws AlreadyClosedSqlException if Connection is closed
    * @throws SQLException if error in calling {@link Connection#isClosed()}
    */
-  private void throwIfClosed() throws AlreadyClosedSqlException,
-                                      SQLException {
-    if ( getConnection().isClosed() ) {
+  protected void checkOpen() throws AlreadyClosedSqlException, SQLException {
+    if (getConnection().isClosed()) {
       throw new AlreadyClosedSqlException(
-          "DatabaseMetaData's Connection is already closed." );
+          "DatabaseMetaData's Connection is already closed.");
     }
   }
 
@@ -196,13 +194,13 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean allProceduresAreCallable() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.allProceduresAreCallable();
   }
 
   @Override
   public boolean allTablesAreSelectable() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.allTablesAreSelectable();
     }
@@ -211,19 +209,19 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getURL() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getURL();
   }
 
   @Override
   public String getUserName() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getUserName();
   }
 
   @Override
   public boolean isReadOnly() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.isReadOnly();
     }
@@ -235,7 +233,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean nullsAreSortedHigh() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return true;
     }
@@ -244,7 +242,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean nullsAreSortedLow() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return false;
     }
@@ -253,7 +251,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean nullsAreSortedAtStart() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return false;
     }
@@ -262,7 +260,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean nullsAreSortedAtEnd() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return false;
     }
@@ -271,7 +269,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getDatabaseProductName() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     String name = getServerName();
     if (name == null) {
       return super.getDatabaseProductName();
@@ -281,7 +279,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getDatabaseProductVersion() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     Version version = getServerVersion();
     if (version == null) {
       return super.getDatabaseProductVersion();
@@ -291,13 +289,13 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getDriverName() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getDriverName();
   }
 
   @Override
   public String getDriverVersion() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getDriverVersion();
   }
 
@@ -315,19 +313,19 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean usesLocalFiles() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.usesLocalFiles();
   }
 
   @Override
   public boolean usesLocalFilePerTable() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.usesLocalFilePerTable();
   }
 
   @Override
   public boolean supportsMixedCaseIdentifiers() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsMixedCaseIdentifiers();
     }
@@ -336,7 +334,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean storesUpperCaseIdentifiers() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.storesUpperCaseIdentifiers();
     }
@@ -345,7 +343,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean storesLowerCaseIdentifiers() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.storesLowerCaseIdentifiers();
     }
@@ -354,7 +352,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean storesMixedCaseIdentifiers() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.storesMixedCaseIdentifiers();
     }
@@ -363,7 +361,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsMixedCaseQuotedIdentifiers() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsMixedCaseQuotedIdentifiers();
     }
@@ -372,7 +370,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean storesUpperCaseQuotedIdentifiers() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.storesUpperCaseQuotedIdentifiers();
     }
@@ -381,7 +379,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean storesLowerCaseQuotedIdentifiers() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.storesLowerCaseQuotedIdentifiers();
     }
@@ -390,7 +388,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean storesMixedCaseQuotedIdentifiers() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.storesMixedCaseQuotedIdentifiers();
     }
@@ -399,7 +397,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getIdentifierQuoteString() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return Quoting.BACK_TICK.string;
     }
@@ -408,7 +406,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getSQLKeywords() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getSQLKeywords();
     }
@@ -417,7 +415,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getNumericFunctions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getNumericFunctions();
     }
@@ -426,7 +424,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getStringFunctions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getStringFunctions();
     }
@@ -435,7 +433,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getSystemFunctions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getSystemFunctions();
     }
@@ -444,7 +442,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getTimeDateFunctions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getTimeDateFunctions();
     }
@@ -453,7 +451,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getSearchStringEscape() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getSearchStringEscape();
     }
@@ -462,7 +460,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getExtraNameCharacters() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getExtraNameCharacters();
     }
@@ -471,19 +469,19 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsAlterTableWithAddColumn() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsAlterTableWithAddColumn();
   }
 
   @Override
   public boolean supportsAlterTableWithDropColumn() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsAlterTableWithDropColumn();
   }
 
   @Override
   public boolean supportsColumnAliasing() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsColumnAliasing();
     }
@@ -492,7 +490,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean nullPlusNonNullIsNull() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.nullPlusNonNullIsNull();
     }
@@ -501,7 +499,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsConvert() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsConvert();
     }
@@ -512,7 +510,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsConvert(int fromType, int toType) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsConvert(fromType, toType);
     }
@@ -523,7 +521,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsTableCorrelationNames() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsTableCorrelationNames();
     }
@@ -533,7 +531,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsDifferentTableCorrelationNames() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsDifferentTableCorrelationNames();
     }
@@ -542,7 +540,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsExpressionsInOrderBy() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsExpressionsInOrderBy();
     }
@@ -551,7 +549,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsOrderByUnrelated() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsOrderByUnrelated();
     }
@@ -560,7 +558,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsGroupBy() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsGroupBy();
     }
@@ -569,7 +567,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsGroupByUnrelated() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsGroupByUnrelated();
     }
@@ -578,7 +576,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsGroupByBeyondSelect() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsGroupByBeyondSelect();
     }
@@ -587,7 +585,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsLikeEscapeClause() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsLikeEscapeClause();
     }
@@ -596,67 +594,67 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsMultipleResultSets() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsMultipleResultSets();
   }
 
   @Override
   public boolean supportsMultipleTransactions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsMultipleTransactions();
   }
 
   @Override
   public boolean supportsNonNullableColumns() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsNonNullableColumns();
   }
 
   @Override
   public boolean supportsMinimumSQLGrammar() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsMinimumSQLGrammar();
   }
 
   @Override
   public boolean supportsCoreSQLGrammar() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsCoreSQLGrammar();
   }
 
   @Override
   public boolean supportsExtendedSQLGrammar() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsExtendedSQLGrammar();
   }
 
   @Override
   public boolean supportsANSI92EntryLevelSQL() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsANSI92EntryLevelSQL();
   }
 
   @Override
   public boolean supportsANSI92IntermediateSQL() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsANSI92IntermediateSQL();
   }
 
   @Override
   public boolean supportsANSI92FullSQL() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsANSI92FullSQL();
   }
 
   @Override
   public boolean supportsIntegrityEnhancementFacility() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsIntegrityEnhancementFacility();
   }
 
   @Override
   public boolean supportsOuterJoins() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsOuterJoins();
     }
@@ -665,7 +663,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsFullOuterJoins() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsFullOuterJoins();
     }
@@ -674,7 +672,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsLimitedOuterJoins() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsFullOuterJoins();
     }
@@ -684,7 +682,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getSchemaTerm() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getSchemaTerm();
     }
@@ -693,13 +691,13 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getProcedureTerm() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getProcedureTerm();
   }
 
   @Override
   public String getCatalogTerm() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getCatalogTerm();
     }
@@ -708,7 +706,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean isCatalogAtStart() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.isCatalogAtStart();
     }
@@ -717,7 +715,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public String getCatalogSeparator() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getCatalogSeparator();
     }
@@ -726,79 +724,79 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsSchemasInDataManipulation() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsSchemasInDataManipulation();
   }
 
   @Override
   public boolean supportsSchemasInProcedureCalls() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsSchemasInProcedureCalls();
   }
 
   @Override
   public boolean supportsSchemasInTableDefinitions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsSchemasInTableDefinitions();
   }
 
   @Override
   public boolean supportsSchemasInIndexDefinitions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsSchemasInIndexDefinitions();
   }
 
   @Override
   public boolean supportsSchemasInPrivilegeDefinitions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsSchemasInPrivilegeDefinitions();
   }
 
   @Override
   public boolean supportsCatalogsInDataManipulation() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsCatalogsInDataManipulation();
   }
 
   @Override
   public boolean supportsCatalogsInProcedureCalls() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsCatalogsInProcedureCalls();
   }
 
   @Override
   public boolean supportsCatalogsInTableDefinitions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsCatalogsInTableDefinitions();
   }
 
   @Override
   public boolean supportsCatalogsInIndexDefinitions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsCatalogsInIndexDefinitions();
   }
 
   @Override
   public boolean supportsCatalogsInPrivilegeDefinitions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsCatalogsInPrivilegeDefinitions();
   }
 
   @Override
   public boolean supportsPositionedDelete() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsPositionedDelete();
   }
 
   @Override
   public boolean supportsPositionedUpdate() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsPositionedUpdate();
   }
 
   @Override
   public boolean supportsSelectForUpdate() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsSelectForUpdate();
     }
@@ -807,13 +805,13 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsStoredProcedures() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsStoredProcedures();
   }
 
   @Override
   public boolean supportsSubqueriesInComparisons() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsSubqueriesInComparisons();
     }
@@ -822,7 +820,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsSubqueriesInExists() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsSubqueriesInExists();
     }
@@ -831,7 +829,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsSubqueriesInIns() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsSubqueriesInIns();
     }
@@ -840,7 +838,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsSubqueriesInQuantifieds() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsSubqueriesInQuantifieds();
     }
@@ -849,7 +847,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsCorrelatedSubqueries() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsCorrelatedSubqueries();
     }
@@ -858,7 +856,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsUnion() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsUnion();
     }
@@ -867,7 +865,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsUnionAll() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsUnionAll();
     }
@@ -876,31 +874,31 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsOpenCursorsAcrossCommit() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsOpenCursorsAcrossCommit();
   }
 
   @Override
   public boolean supportsOpenCursorsAcrossRollback() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsOpenCursorsAcrossRollback();
   }
 
   @Override
   public boolean supportsOpenStatementsAcrossCommit() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsOpenStatementsAcrossCommit();
   }
 
   @Override
   public boolean supportsOpenStatementsAcrossRollback() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsOpenStatementsAcrossRollback();
   }
 
   @Override
   public int getMaxBinaryLiteralLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxBinaryLiteralLength();
     }
@@ -909,7 +907,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxCharLiteralLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxCharLiteralLength();
     }
@@ -918,7 +916,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxColumnNameLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxColumnNameLength();
     }
@@ -927,7 +925,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxColumnsInGroupBy() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxColumnsInGroupBy();
     }
@@ -936,13 +934,13 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxColumnsInIndex() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getMaxColumnsInIndex();
   }
 
   @Override
   public int getMaxColumnsInOrderBy() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxColumnsInOrderBy();
     }
@@ -951,7 +949,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxColumnsInSelect() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxColumnsInSelect();
     }
@@ -960,19 +958,19 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxColumnsInTable() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getMaxColumnsInTable();
   }
 
   @Override
   public int getMaxConnections() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getMaxConnections();
   }
 
   @Override
   public int getMaxCursorNameLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxCursorNameLength();
     }
@@ -981,13 +979,13 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxIndexLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getMaxIndexLength();
   }
 
   @Override
   public int getMaxSchemaNameLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxSchemaNameLength();
     }
@@ -996,13 +994,13 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxProcedureNameLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getMaxProcedureNameLength();
   }
 
   @Override
   public int getMaxCatalogNameLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxCatalogNameLength();
     }
@@ -1011,7 +1009,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxRowSize() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxRowSize();
     }
@@ -1020,7 +1018,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean doesMaxRowSizeIncludeBlobs() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.doesMaxRowSizeIncludeBlobs();
     }
@@ -1029,7 +1027,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxStatementLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxStatementLength();
     }
@@ -1038,7 +1036,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxStatements() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxStatements();
     }
@@ -1047,7 +1045,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxTableNameLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxTableNameLength();
     }
@@ -1056,7 +1054,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxTablesInSelect() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxTablesInSelect();
     }
@@ -1065,7 +1063,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getMaxUserNameLength() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.getMaxUserNameLength();
     }
@@ -1074,13 +1072,13 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getDefaultTransactionIsolation() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getDefaultTransactionIsolation();
   }
 
   @Override
   public boolean supportsTransactions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (!getServerMetaSupported()) {
       return super.supportsTransactions();
     }
@@ -1089,39 +1087,39 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsTransactionIsolationLevel(int level) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsTransactionIsolationLevel(level);
   }
 
   @Override
   public boolean supportsDataDefinitionAndDataManipulationTransactions()
       throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsDataDefinitionAndDataManipulationTransactions();
   }
 
   @Override
   public boolean supportsDataManipulationTransactionsOnly() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsDataManipulationTransactionsOnly();
   }
 
   @Override
   public boolean dataDefinitionCausesTransactionCommit() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.dataDefinitionCausesTransactionCommit();
   }
 
   @Override
   public boolean dataDefinitionIgnoredInTransactions() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.dataDefinitionIgnoredInTransactions();
   }
 
   @Override
   public ResultSet getProcedures(String catalog, String schemaPattern,
                                  String procedureNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getProcedures(catalog, schemaPattern, procedureNamePattern);
   }
 
@@ -1129,7 +1127,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   public ResultSet getProcedureColumns(String catalog, String schemaPattern,
                                        String procedureNamePattern,
                                        String columnNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getProcedureColumns(catalog, schemaPattern,
                                      procedureNamePattern, columnNamePattern);
   }
@@ -1139,7 +1137,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
                              String schemaPattern,
                              String tableNamePattern,
                              String[] types) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.getTables(catalog, schemaPattern,tableNamePattern, types);
     } catch(DrillRuntimeException e) {
@@ -1151,7 +1149,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public ResultSet getSchemas() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.getSchemas();
     } catch(DrillRuntimeException e) {
@@ -1162,7 +1160,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public ResultSet getCatalogs() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.getCatalogs();
     } catch(DrillRuntimeException e) {
@@ -1173,14 +1171,14 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public ResultSet getTableTypes() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getTableTypes();
   }
 
   @Override
   public ResultSet getColumns(String catalog, String schema, String table,
                               String columnNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.getColumns(catalog, schema, table, columnNamePattern);
     } catch(DrillRuntimeException e) {
@@ -1193,14 +1191,14 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   public ResultSet getColumnPrivileges(String catalog, String schema,
                                        String table,
                                        String columnNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getColumnPrivileges(catalog, schema, table, columnNamePattern);
   }
 
   @Override
   public ResultSet getTablePrivileges(String catalog, String schemaPattern,
                                       String tableNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getTablePrivileges(catalog, schemaPattern, tableNamePattern);
   }
 
@@ -1208,35 +1206,35 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   public ResultSet getBestRowIdentifier(String catalog, String schema,
                                         String table, int scope,
                                         boolean nullable) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getBestRowIdentifier(catalog, schema, table, scope, nullable);
   }
 
   @Override
   public ResultSet getVersionColumns(String catalog, String schema,
                                      String table) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getVersionColumns(catalog, schema, table);
   }
 
   @Override
   public ResultSet getPrimaryKeys(String catalog, String schema,
                                   String table) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getPrimaryKeys(catalog, schema, table);
   }
 
   @Override
   public ResultSet getImportedKeys(String catalog, String schema,
                                    String table) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getImportedKeys(catalog, schema, table);
   }
 
   @Override
   public ResultSet getExportedKeys(String catalog, String schema,
                                    String table) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getExportedKeys(catalog, schema, table);
   }
 
@@ -1245,14 +1243,14 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
       String parentCatalog, String parentSchema, String parentTable,
       String foreignCatalog, String foreignSchema,
       String foreignTable ) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getCrossReference(parentCatalog, parentSchema, parentTable,
                                    foreignCatalog, foreignSchema, foreignTable );
   }
 
   @Override
   public ResultSet getTypeInfo() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getTypeInfo();
   }
 
@@ -1260,26 +1258,26 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   public ResultSet getIndexInfo(String catalog, String schema, String table,
                                 boolean unique,
                                 boolean approximate) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getIndexInfo(catalog, schema, table, unique, approximate);
   }
 
   @Override
   public boolean supportsResultSetType(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsResultSetType(type);
   }
 
   @Override
   public boolean supportsResultSetConcurrency(int type,
                                               int concurrency) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsResultSetConcurrency(type, concurrency);
   }
 
   @Override
   public boolean ownUpdatesAreVisible(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.ownUpdatesAreVisible(type);
     }
@@ -1296,7 +1294,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean ownDeletesAreVisible(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.ownDeletesAreVisible(type);
     }
@@ -1313,7 +1311,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean ownInsertsAreVisible(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.ownInsertsAreVisible(type);
     }
@@ -1330,7 +1328,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean othersUpdatesAreVisible(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.othersUpdatesAreVisible(type);
     }
@@ -1347,7 +1345,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean othersDeletesAreVisible(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.othersDeletesAreVisible(type);
     }
@@ -1364,7 +1362,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean othersInsertsAreVisible(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.othersInsertsAreVisible(type);
     }
@@ -1381,7 +1379,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean updatesAreDetected(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.updatesAreDetected(type);
     }
@@ -1398,7 +1396,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean deletesAreDetected(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.deletesAreDetected(type);
     }
@@ -1415,7 +1413,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean insertsAreDetected(int type) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.insertsAreDetected(type);
     }
@@ -1432,7 +1430,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsBatchUpdates() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsBatchUpdates();
   }
 
@@ -1440,7 +1438,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   public ResultSet getUDTs(String catalog, String schemaPattern,
                            String typeNamePattern,
                            int[] types) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getUDTs(catalog, schemaPattern, typeNamePattern, types);
   }
 
@@ -1452,39 +1450,39 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public boolean supportsSavepoints() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsSavepoints();
   }
 
   @Override
   public boolean supportsNamedParameters() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsNamedParameters();
   }
 
   @Override
   public boolean supportsMultipleOpenResults() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsMultipleOpenResults();
   }
 
   @Override
   public boolean supportsGetGeneratedKeys() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsGetGeneratedKeys();
   }
 
   @Override
   public ResultSet getSuperTypes(String catalog, String schemaPattern,
                                  String typeNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getSuperTypes(catalog, schemaPattern, typeNamePattern);
   }
 
   @Override
   public ResultSet getSuperTables(String catalog, String schemaPattern,
                                   String tableNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getSuperTables(catalog, schemaPattern, tableNamePattern);
   }
 
@@ -1492,14 +1490,14 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   public ResultSet getAttributes(String catalog, String schemaPattern,
                                  String typeNamePattern,
                                  String attributeNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getAttributes(catalog, schemaPattern, typeNamePattern,
                                attributeNamePattern);
   }
 
   @Override
   public boolean supportsResultSetHoldability(int holdability) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.supportsResultSetHoldability(holdability);
     }
@@ -1519,9 +1517,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
     // Can't throw any SQLException because Avatica's getResultSetHoldability()
     // is missing "throws SQLException".
     try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      throw new RuntimeException(e.getMessage(), e);
+      checkOpen();
     } catch (SQLException e) {
       throw new RuntimeException(e.getMessage(), e);
     }
@@ -1530,7 +1526,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getDatabaseMajorVersion() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     Version version = getServerVersion();
     if (version == null) {
       return super.getDatabaseMajorVersion();
@@ -1540,7 +1536,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getDatabaseMinorVersion() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     Version version = getServerVersion();
     if (version == null) {
       return super.getDatabaseMinorVersion();
@@ -1550,69 +1546,69 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
 
   @Override
   public int getJDBCMajorVersion() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getJDBCMajorVersion();
   }
 
   @Override
   public int getJDBCMinorVersion() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getJDBCMinorVersion();
   }
 
   @Override
   public int getSQLStateType() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getSQLStateType();
   }
 
   @Override
   public boolean locatorsUpdateCopy() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.locatorsUpdateCopy();
   }
 
   @Override
   public boolean supportsStatementPooling() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsStatementPooling();
   }
 
   @Override
   public RowIdLifetime getRowIdLifetime() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getRowIdLifetime();
   }
 
   @Override
   public ResultSet getSchemas(String catalog,
                               String schemaPattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getSchemas(catalog, schemaPattern);
   }
 
   @Override
   public boolean supportsStoredFunctionsUsingCallSyntax() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.supportsStoredFunctionsUsingCallSyntax();
   }
 
   @Override
   public boolean autoCommitFailureClosesAllResultSets() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.autoCommitFailureClosesAllResultSets();
   }
 
   @Override
   public ResultSet getClientInfoProperties() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getClientInfoProperties();
   }
 
   @Override
   public ResultSet getFunctions(String catalog, String schemaPattern,
                                 String functionNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getFunctions(catalog, schemaPattern, functionNamePattern);
   }
 
@@ -1620,7 +1616,7 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   public ResultSet getFunctionColumns(String catalog, String schemaPattern,
                                       String functionNamePattern,
                                       String columnNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getFunctionColumns(catalog, schemaPattern, functionNamePattern,
                                     columnNamePattern);
   }
@@ -1629,14 +1625,14 @@ class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
   public ResultSet getPseudoColumns(String catalog, String schemaPattern,
                                     String tableNamePattern,
                                     String columnNamePattern) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getPseudoColumns(catalog, schemaPattern, tableNamePattern,
                                   columnNamePattern);
   }
 
   @Override
   public boolean generatedKeyAlwaysReturned() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.generatedKeyAlwaysReturned();
   }
 
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
index 308a34d..7d57c85 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
@@ -17,14 +17,9 @@
  */
 package org.apache.drill.jdbc.impl;
 
-import java.io.InputStream;
-import java.io.Reader;
-import java.sql.NClob;
 import java.sql.ResultSetMetaData;
-import java.sql.RowId;
 import java.sql.SQLException;
 import java.sql.SQLTimeoutException;
-import java.sql.SQLXML;
 import java.util.Properties;
 import java.util.TimeZone;
 
@@ -42,17 +37,17 @@ import org.apache.drill.exec.rpc.DrillRpcFuture;
 
 
 /**
- * Implementation of {@link net.hydromatic.avatica.AvaticaFactory} for Drill and
+ * Implementation of {@link org.apache.calcite.avatica.AvaticaFactory} for Drill and
  * JDBC 4.1 (corresponds to JDK 1.7).
  */
-// Note:  Must be public so net.hydromatic.avatica.UnregisteredDriver can
+// Note:  Must be public so org.apache.calcite.avatica.UnregisteredDriver can
 // (reflectively) call no-args constructor.
 public class DrillJdbc41Factory extends DrillFactory {
   private static final org.slf4j.Logger logger =
       org.slf4j.LoggerFactory.getLogger(DrillJdbc41Factory.class);
 
   /** Creates a factory for JDBC version 4.1. */
-  // Note:  Must be public so net.hydromatic.avatica.UnregisteredDriver can
+  // Note:  Must be public so org.apache.calcite.avatica.UnregisteredDriver can
   // (reflectively) call this constructor.
   public DrillJdbc41Factory() {
     this(4, 1);
@@ -161,8 +156,6 @@ public class DrillJdbc41Factory extends DrillFactory {
           resultSetHoldability);
     } catch (SQLException e) {
       throw e;
-    } catch (RuntimeException e) {
-      throw Helper.INSTANCE.createException("Error while preparing statement [" + sql + "]", e);
     } catch (Exception e) {
       throw Helper.INSTANCE.createException("Error while preparing statement [" + sql + "]", e);
     }
@@ -173,7 +166,7 @@ public class DrillJdbc41Factory extends DrillFactory {
                                          QueryState state,
                                          Meta.Signature signature,
                                          TimeZone timeZone,
-                                         Meta.Frame firstFrame) {
+                                         Meta.Frame firstFrame) throws SQLException {
     final ResultSetMetaData metaData =
         newResultSetMetaData(statement, signature);
     return new DrillResultSetImpl(statement, state, signature, metaData, timeZone, firstFrame);
@@ -201,111 +194,6 @@ public class DrillJdbc41Factory extends DrillFactory {
       super(connection, h, signature, pstmt,
             resultSetType, resultSetConcurrency, resultSetHoldability);
     }
-
-    // These don't need throwIfClosed(), since getParameter already calls it.
-
-    @Override
-    public void setRowId(int parameterIndex, RowId x) throws SQLException {
-      getSite(parameterIndex).setRowId(x);
-    }
-
-    @Override
-    public void setNString(int parameterIndex, String value) throws SQLException {
-      getSite(parameterIndex).setNString(value);
-    }
-
-    @Override
-    public void setNCharacterStream(int parameterIndex, Reader value,
-                                    long length) throws SQLException {
-      getSite(parameterIndex).setNCharacterStream(value, length);
-    }
-
-    @Override
-    public void setNClob(int parameterIndex, NClob value) throws SQLException {
-      getSite(parameterIndex).setNClob(value);
-    }
-
-    @Override
-    public void setClob(int parameterIndex, Reader reader,
-                        long length) throws SQLException {
-      getSite(parameterIndex).setClob(reader, length);
-    }
-
-    @Override
-    public void setBlob(int parameterIndex, InputStream inputStream,
-                        long length) throws SQLException {
-      getSite(parameterIndex).setBlob(inputStream, length);
-    }
-
-    @Override
-    public void setNClob(int parameterIndex, Reader reader,
-                         long length) throws SQLException {
-      getSite(parameterIndex).setNClob(reader, length);
-    }
-
-    @Override
-    public void setSQLXML(int parameterIndex, SQLXML xmlObject) throws SQLException {
-      getSite(parameterIndex).setSQLXML(xmlObject);
-    }
-
-    @Override
-    public void setAsciiStream(int parameterIndex, InputStream x,
-                               long length) throws SQLException {
-      getSite(parameterIndex).setAsciiStream(x, length);
-    }
-
-    @Override
-    public void setBinaryStream(int parameterIndex, InputStream x,
-                                long length) throws SQLException {
-      getSite(parameterIndex).setBinaryStream(x, length);
-    }
-
-    @Override
-    public void setCharacterStream(int parameterIndex, Reader reader,
-                                   long length) throws SQLException {
-      getSite(parameterIndex).setCharacterStream(reader, length);
-    }
-
-    @Override
-    public void setAsciiStream(int parameterIndex,
-                               InputStream x) throws SQLException {
-      getSite(parameterIndex).setAsciiStream(x);
-    }
-
-    @Override
-    public void setBinaryStream(int parameterIndex,
-                                InputStream x) throws SQLException {
-      getSite(parameterIndex).setBinaryStream(x);
-    }
-
-    @Override
-    public void setCharacterStream(int parameterIndex,
-                                   Reader reader) throws SQLException {
-      getSite(parameterIndex).setCharacterStream(reader);
-    }
-
-    @Override
-    public void setNCharacterStream(int parameterIndex,
-                                    Reader value) throws SQLException {
-      getSite(parameterIndex).setNCharacterStream(value);
-    }
-
-    @Override
-    public void setClob(int parameterIndex, Reader reader) throws SQLException {
-      getSite(parameterIndex).setClob(reader);
-    }
-
-    @Override
-    public void setBlob(int parameterIndex,
-                        InputStream inputStream) throws SQLException {
-      getSite(parameterIndex).setBlob(inputStream);
-    }
-
-    @Override
-    public void setNClob(int parameterIndex, Reader reader) throws SQLException {
-      getSite(parameterIndex).setNClob(reader);
-    }
-
   }
 
 }
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
index a13f936..6b6e57b 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
@@ -17,12 +17,10 @@
  */
 package org.apache.drill.jdbc.impl;
 
-import java.sql.ParameterMetaData;
 import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
-import java.sql.SQLWarning;
+import java.sql.SQLType;
 
 import org.apache.calcite.avatica.AvaticaParameter;
 import org.apache.calcite.avatica.AvaticaPreparedStatement;
@@ -68,7 +66,8 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
    *
    * @throws  AlreadyClosedSqlException  if PreparedStatement is closed
    */
-  private void throwIfClosed() throws AlreadyClosedSqlException {
+  @Override
+  protected void checkOpen() throws SQLException {
     if (isClosed()) {
       throw new AlreadyClosedSqlException("PreparedStatement is already closed.");
     }
@@ -77,22 +76,7 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
 
   // Note:  Using dynamic proxies would reduce the quantity (450?) of method
   // overrides by eliminating those that exist solely to check whether the
-  // object is closed.  It would also eliminate the need to throw non-compliant
-  // RuntimeExceptions when Avatica's method declarations won't let us throw
-  // proper SQLExceptions. (Check performance before applying to frequently
-  // called ResultSet.)
-
-  @Override
-  public DrillConnectionImpl getConnection() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getConnection() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return (DrillConnectionImpl) super.getConnection();
-  }
+  // object is closed.
 
   PreparedStatement getPreparedStatementHandle() {
     return preparedStatementHandle;
@@ -100,7 +84,7 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
 
   @Override
   protected AvaticaParameter getParameter(int param) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     throw new SQLFeatureNotSupportedException(
         "Prepared-statement dynamic parameters are not supported.");
   }
@@ -116,341 +100,130 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
   // No isWrapperFor(Class<?>) (it doesn't throw SQLException if already closed).
   // No unwrap(Class<T>) (it doesn't throw SQLException if already closed).
 
-  @Override
-  public ResultSet executeQuery(String sql) throws SQLException {
-    throwIfClosed();
-    try {
-      return super.executeQuery(sql);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public long executeLargeUpdate(String sql) throws SQLException {
-    throwIfClosed();
-    try {
-      return super.executeLargeUpdate(sql);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
   // No close() (it doesn't throw SQLException if already closed).
 
   @Override
-  public int getMaxFieldSize() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getMaxFieldSize();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public void setMaxFieldSize(int max) throws SQLException {
-    throwIfClosed();
-    try {
-      super.setMaxFieldSize(max);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public long getLargeMaxRows() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getLargeMaxRows() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getLargeMaxRows();
-  }
-
-  @Override
-  public void setLargeMaxRows(long max) throws SQLException {
-    throwIfClosed();
-    super.setLargeMaxRows(max);
-  }
-
-  @Override
   public void setEscapeProcessing(boolean enable) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.setEscapeProcessing(enable);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public int getQueryTimeout() throws SQLException {
-    throwIfClosed();
-    return super.getQueryTimeout();
-  }
-
-  @Override
-  public void setQueryTimeout(int seconds) throws SQLException {
-    throwIfClosed();
-    super.setQueryTimeout(seconds);
-  }
-
-  @Override
-  public void cancel() throws SQLException {
-    throwIfClosed();
-    super.cancel();
-  }
-
-  @Override
-  public SQLWarning getWarnings() throws SQLException {
-    throwIfClosed();
-    return super.getWarnings();
-  }
-
-  @Override
-  public void clearWarnings() throws SQLException {
-    throwIfClosed();
-    super.clearWarnings();
-  }
-
-  @Override
   public void setCursorName(String name) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.setCursorName(name);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public boolean execute(String sql) throws SQLException {
-    throwIfClosed();
-    return super.execute(sql);
-  }
-
-  @Override
-  public ResultSet getResultSet() throws SQLException {
-    throwIfClosed();
-    return super.getResultSet();
-  }
-
-  @Override
-  public int getUpdateCount() throws SQLException {
-    throwIfClosed();
-    return super.getUpdateCount();
-  }
-
-  @Override
   public boolean getMoreResults() throws SQLException {
-    throwIfClosed();
     try {
       return super.getMoreResults();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void setFetchDirection(int direction) throws SQLException {
-    throwIfClosed();
-    super.setFetchDirection(direction);
-  }
-
-  @Override
-  public int getFetchDirection(){
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getFetchDirection() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getFetchDirection();
-  }
-
-  @Override
-  public void setFetchSize(int rows) throws SQLException {
-    throwIfClosed();
-    super.setFetchSize(rows);
-  }
-
-  @Override
-  public int getFetchSize() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getFetchSize() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getFetchSize();
-  }
-
-  @Override
-  public int getResultSetConcurrency() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getResultSetConcurrency();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public int getResultSetType() throws SQLException {
-    throwIfClosed();
+  public boolean getMoreResults(int current) throws SQLException {
     try {
-      return super.getResultSetType();
-    }
-    catch (UnsupportedOperationException e) {
+      return super.getMoreResults(current);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void addBatch(String sql) throws SQLException {
-    throwIfClosed();
+  public ResultSet getGeneratedKeys() throws SQLException {
+    checkOpen();
     try {
-      super.addBatch(sql);
-    }
-    catch (UnsupportedOperationException e) {
+      return super.getGeneratedKeys();
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void clearBatch() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      throw new RuntimeException(e);
-    }
-    try {
-      super.clearBatch();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new RuntimeException(new SQLFeatureNotSupportedException(e.getMessage(), e));
-    }
+  public long executeLargeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
+    checkOpen();
+    return super.executeLargeUpdate(sql, autoGeneratedKeys);
   }
 
   @Override
-  public int[] executeBatch() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.executeBatch();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
+  public long executeLargeUpdate(String sql, int[] columnIndexes) throws SQLException {
+    checkOpen();
+    return super.executeLargeUpdate(sql, columnIndexes);
   }
 
   @Override
-  public boolean getMoreResults(int current) throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getMoreResults(current);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public ResultSet getGeneratedKeys() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getGeneratedKeys();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
+  public long executeLargeUpdate(String sql, String[] columnNames) throws SQLException {
+    checkOpen();
+    return super.executeLargeUpdate(sql, columnNames);
   }
 
   @Override
   public int executeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.executeUpdate(sql, autoGeneratedKeys);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public int executeUpdate(String sql, int columnIndexes[]) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.executeUpdate(sql, columnIndexes);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public int executeUpdate(String sql, String columnNames[]) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.executeUpdate(sql, columnNames);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean execute(String sql, int autoGeneratedKeys) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.execute(sql, autoGeneratedKeys);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean execute(String sql, int columnIndexes[]) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.execute(sql, columnIndexes);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean execute(String sql, String columnNames[]) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.execute(sql, columnNames);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public int getResultSetHoldability() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getResultSetHoldability();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
@@ -461,171 +234,29 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
       return super.isClosed();
     } catch (SQLException e) {
       throw new RuntimeException(
-          "Unexpected " + e + " from AvaticaPreparedStatement.isClosed" );
+          "Unexpected " + e + " from AvaticaPreparedStatement.isClosed");
     }
   }
 
   @Override
   public void setPoolable(boolean poolable) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.setPoolable(poolable);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public boolean isPoolable() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.isPoolable();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void closeOnCompletion() throws SQLException {
-    throwIfClosed();
-    super.closeOnCompletion();
-  }
-
-  @Override
-  public boolean isCloseOnCompletion() throws SQLException {
-    throwIfClosed();
-    return super.isCloseOnCompletion();
-  }
-
-  @Override
-  public ResultSet executeQuery() throws SQLException {
-    throwIfClosed();
-    return super.executeQuery();
-  }
-
-  @Override
-  public long executeLargeUpdate() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.executeLargeUpdate();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
+  public void setObject(int parameterIndex, Object x, SQLType targetSqlType, int scaleOrLength) throws SQLException {
+    checkOpen();
+    super.setObject(parameterIndex, x, targetSqlType, scaleOrLength);
   }
 
-  // Covered by superclass methods' calls to getParameter(int):
-  // - setNull(int, int)
-  // - setBoolean(int, boolean)
-  // - setByte(int, byte)
-  // - setShort(int, short)
-  // - setInt(int, int)
-  // - setLong(int, long)
-  // - setFloat(int, float)
-  // - setDouble(int, double)
-  // - setBigDecimal(int, BigDecimal)
-  // - setString(int, String)
-  // - setBytes(int, byte[])
-  // - setDate(int, Date)
-  // - setTime(int, Time)
-  // - setTimestamp(int, Timestamp)
-  // - setAsciiStream(int, InputStream, int)
-  // - setUnicodeStream(int, InputStream, int)
-  // - setBinaryStream(int, InputStream, int)
-
   @Override
-  public void clearParameters() throws SQLException {
-    throwIfClosed();
-    try {
-      super.clearParameters();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
+  public void setObject(int parameterIndex, Object x, SQLType targetSqlType) throws SQLException {
+    checkOpen();
+    super.setObject(parameterIndex, x, targetSqlType);
   }
-
-  // Covered by superclass methods' calls to getParameter(int):
-  // - setObject(int, Object, int)
-  // - setObject(int, Object)
-
-  @Override
-  public boolean execute() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.execute();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public void addBatch() throws SQLException {
-    throwIfClosed();
-    try {
-      super.addBatch();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  // Covered by superclass methods' calls to getParameter(int):
-  // - setCharacterStream(int, Reader, int)
-  // - setRef(int, Ref)
-  // - setBlob(int, Blob)
-  // - setClob(int, Clob)
-  // - setArray(int, Array)
-
-  @Override
-  public ResultSetMetaData getMetaData() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getMetaData() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getMetaData();
-  }
-
-  // Covered by superclass methods' calls to getParameter(int):
-  // - setDate(int, Date, Calendar)
-  // - setTime(int, Time, Calendar)
-  // - setTimestamp(int, Timestamp, Calendar)
-  // - setNull(int, int, String)
-  // - setURL(int, URL)
-
-  @Override
-  public ParameterMetaData getParameterMetaData() throws SQLException {
-    throwIfClosed();
-    return super.getParameterMetaData();
-  }
-
-  // The following methods are abstract in AvaticaPreparedStatement, and so
-  // cannot be overridden here to add throwIfClosed calls.  They are addressed
-  // via DrillJdbc41Factory (which calls back to getParameter(int) in here,
-  // which calls throwIfClosed()).
-  // - setRowId(int, RowId)
-  // - setNString(int, String)
-  // - setNCharacterStream(int, Reader, long)
-  // - setNClob(int, NClob)
-  // - setClob(int, Reader, long)
-  // - setBlob(int, InputStream, long)
-  // - setNClob(int, Reader, long)
-  // - setSQLXML(int, SQLXML xmlObject)
-  // - setObject(int, Object, int, int)
-  // - setAsciiStream(int, InputStream, long)
-  // - setBinaryStream(int, InputStream, long)
-  // - setCharacterStream(int, Reader, long)
-  // - setAsciiStream(int, InputStream)
-  // - setBinaryStream(int, InputStream)
-  // - setCharacterStream(int, Reader)
-  // - setNCharacterStream(int, Reader)
-  // - setClob(int, Reader)
-  // - setBlob(int, InputStream)
-  // - setNClob(int, Reader)
-
 }
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
index 40f5733..c0544f1 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
@@ -20,7 +20,6 @@ package org.apache.drill.jdbc.impl;
 import java.io.InputStream;
 import java.io.Reader;
 import java.math.BigDecimal;
-import java.net.URL;
 import java.sql.Array;
 import java.sql.Blob;
 import java.sql.Clob;
@@ -31,15 +30,13 @@ import java.sql.ResultSetMetaData;
 import java.sql.RowId;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
-import java.sql.SQLWarning;
+import java.sql.SQLType;
 import java.sql.SQLXML;
 import java.sql.Time;
 import java.sql.Timestamp;
 import java.sql.Types;
 import java.util.ArrayList;
-import java.util.Calendar;
 import java.util.List;
-import java.util.Map;
 import java.util.TimeZone;
 import java.util.concurrent.TimeUnit;
 
@@ -59,7 +56,7 @@ import org.apache.drill.jdbc.SqlTimeoutException;
 import com.google.common.base.Stopwatch;
 
 /**
- * Drill's implementation of {@link ResultSet}.
+ * Drill's implementation of {@link java.sql.ResultSet}.
  */
 class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
   @SuppressWarnings("unused")
@@ -75,7 +72,7 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
 
   DrillResultSetImpl(AvaticaStatement statement, QueryState state, Meta.Signature signature,
                      ResultSetMetaData resultSetMetaData, TimeZone timeZone,
-                     Meta.Frame firstFrame) {
+                     Meta.Frame firstFrame) throws SQLException {
     super(statement, state, signature, resultSetMetaData, timeZone, firstFrame);
     connection = (DrillConnectionImpl) statement.getConnection();
   }
@@ -90,7 +87,8 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
    * @throws  AlreadyClosedSqlException  if ResultSet is closed
    * @throws  SQLException  if error in calling {@link #isClosed()}
    */
-  private void throwIfClosed() throws SQLException {
+  @Override
+  protected void checkOpen() throws SQLException {
     if (isClosed()) {
       if (cursor instanceof DrillCursor && hasPendingCancelationNotification) {
         hasPendingCancelationNotification = false;
@@ -137,11 +135,11 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
   // (Not delegated.)
   @Override
   public boolean next() throws SQLException {
-    throwIfClosed();
+    checkOpen();
 
     // TODO:  Resolve following comments (possibly obsolete because of later
-    // addition of preceding call to throwIfClosed.  Also, NOTE that the
-    // following check, and maybe some throwIfClosed() calls, probably must
+    // addition of preceding call to checkOpen.  Also, NOTE that the
+    // following check, and maybe some checkOpen() calls, probably must
     // synchronize on the statement, per the comment on AvaticaStatement's
     // openResultSet:
 
@@ -163,1712 +161,1098 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
   }
 
   @Override
-  public boolean wasNull() throws SQLException {
-    throwIfClosed();
-    return super.wasNull();
-  }
-
-  // Methods for accessing results by column index
-  @Override
-  public String getString( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getString( columnIndex );
-  }
-
-  @Override
-  public boolean getBoolean( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getBoolean( columnIndex );
-  }
-
-  @Override
-  public byte getByte( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getByte( columnIndex );
-  }
-
-  @Override
-  public short getShort( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getShort( columnIndex );
-  }
-
-  @Override
-  public int getInt( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getInt( columnIndex );
-  }
-
-  @Override
-  public long getLong( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getLong( columnIndex );
-  }
-
-  @Override
-  public float getFloat( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getFloat( columnIndex );
-  }
-
-  @Override
-  public double getDouble( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getDouble( columnIndex );
-  }
-
-  @Override
-  public BigDecimal getBigDecimal( int columnIndex,
-                                   int scale ) throws SQLException {
-    throwIfClosed();
-    return super.getBigDecimal( columnIndex, scale );
-  }
-
-  @Override
-  public byte[] getBytes( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getBytes( columnIndex );
-  }
-
-  @Override
-  public Date getDate( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getDate( columnIndex );
-  }
-
-  @Override
-  public Time getTime( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getTime( columnIndex );
-  }
-
-  @Override
-  public Timestamp getTimestamp( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getTimestamp( columnIndex );
-  }
-
-  @Override
-  public InputStream getAsciiStream( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getAsciiStream( columnIndex );
-  }
-
-  @Override
-  public InputStream getUnicodeStream( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getUnicodeStream( columnIndex );
-  }
-
-  @Override
-  public InputStream getBinaryStream( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getBinaryStream( columnIndex );
-  }
-
-  // Methods for accessing results by column label
-  @Override
-  public String getString( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getString( columnLabel );
-  }
-
-  @Override
-  public boolean getBoolean( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getBoolean( columnLabel );
-  }
-
-  @Override
-  public byte getByte( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getByte( columnLabel );
-  }
-
-  @Override
-  public short getShort( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getShort( columnLabel );
-  }
-
-  @Override
-  public int getInt( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getInt( columnLabel );
-  }
-
-  @Override
-  public long getLong( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getLong( columnLabel );
-  }
-
-  @Override
-  public float getFloat( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getFloat( columnLabel );
-  }
-
-  @Override
-  public double getDouble( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getDouble( columnLabel );
-  }
-
-  @Override
-  public BigDecimal getBigDecimal( String columnLabel,
-                                   int scale ) throws SQLException {
-    throwIfClosed();
-    return super.getBigDecimal( columnLabel, scale );
-  }
-
-  @Override
-  public byte[] getBytes( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getBytes( columnLabel );
-  }
-
-  @Override
-  public Date getDate( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getDate( columnLabel );
-  }
-
-  @Override
-  public Time getTime( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getTime( columnLabel );
-  }
-
-  @Override
-  public Timestamp getTimestamp( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getTimestamp( columnLabel );
-  }
-
-  @Override
-  public InputStream getAsciiStream( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getAsciiStream( columnLabel );
-  }
-
-  @Override
-  public InputStream getUnicodeStream( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getUnicodeStream( columnLabel );
-  }
-
-  @Override
-  public InputStream getBinaryStream( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getBinaryStream( columnLabel );
-  }
-
-  // Advanced features:
-  @Override
-  public SQLWarning getWarnings() throws SQLException {
-    throwIfClosed();
-    return super.getWarnings();
-  }
-
-  @Override
-  public void clearWarnings() throws SQLException {
-    throwIfClosed();
-    super.clearWarnings();
-  }
-
-  @Override
   public String getCursorName() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.getCursorName();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
-  // (Not delegated.)
-  @Override
-  public ResultSetMetaData getMetaData() throws SQLException {
-    throwIfClosed();
-    return super.getMetaData();
-  }
-
   @Override
   public Object getObject( int columnIndex ) throws SQLException {
-    throwIfClosed();
+    checkOpen();
 
-    final Cursor.Accessor accessor;
+    Cursor.Accessor accessor;
     try {
       accessor = accessorList.get(columnIndex - 1);
     } catch (RuntimeException e) {
       throw new SQLException(e);
     }
-    final ColumnMetaData metaData = columnMetaDataList.get(columnIndex - 1);
+    ColumnMetaData metaData = columnMetaDataList.get(columnIndex - 1);
     // Drill returns a float (4bytes) for a SQL Float whereas Calcite would return a double (8bytes)
     int typeId = (metaData.type.id != Types.FLOAT) ? metaData.type.id : Types.REAL;
     return AvaticaSite.get(accessor, typeId, localCalendar);
   }
 
-  @Override
-  public Object getObject( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getObject( columnLabel );
-  }
-
-  //----------------------------------------------------------------
-  @Override
-  public int findColumn( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.findColumn( columnLabel );
-  }
-
   //--------------------------JDBC 2.0-----------------------------------
-  //---------------------------------------------------------------------
-  // Getters and Setters
-  //---------------------------------------------------------------------
-  @Override
-  public Reader getCharacterStream( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getCharacterStream( columnIndex );
-  }
-
-  @Override
-  public Reader getCharacterStream( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getCharacterStream( columnLabel );
-  }
-
-  @Override
-  public BigDecimal getBigDecimal( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getBigDecimal( columnIndex );
-  }
-
-  @Override
-  public BigDecimal getBigDecimal( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getBigDecimal( columnLabel );
-  }
 
   //---------------------------------------------------------------------
   // Traversal/Positioning
   //---------------------------------------------------------------------
   @Override
-  public boolean isBeforeFirst() throws SQLException {
-    throwIfClosed();
-    return super.isBeforeFirst();
-  }
-
-  @Override
-  public boolean isAfterLast() throws SQLException {
-    throwIfClosed();
-    return super.isAfterLast();
-  }
-
-  @Override
-  public boolean isFirst() throws SQLException {
-    throwIfClosed();
-    return super.isFirst();
-  }
-
-  @Override
   public boolean isLast() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.isLast();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void beforeFirst() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.beforeFirst();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void afterLast() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.afterLast();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean first() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.first();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean last() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.last();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public int getRow() throws SQLException {
-    throwIfClosed();
-    // Map Avatica's erroneous zero-based row numbers to 1-based, and return 0
-    // after end, per JDBC:
-    return isAfterLast() ? 0 : 1 + super.getRow();
-  }
-
-  @Override
-  public boolean absolute( int row ) throws SQLException {
-    throwIfClosed();
+  public boolean absolute(int row) throws SQLException {
+    checkOpen();
     try {
-      return super.absolute( row );
-    }
-    catch (UnsupportedOperationException e) {
+      return super.absolute(row);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean relative( int rows ) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.relative( rows );
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean previous() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.previous();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   //---------------------------------------------------------------------
-  // Properties
-  //---------------------------------------------------------------------
-
-  @Override
-  public void setFetchDirection( int direction ) throws SQLException {
-    throwIfClosed();
-    super.setFetchDirection( direction );
-  }
-
-  @Override
-  public int getFetchDirection() throws SQLException {
-    throwIfClosed();
-    return super.getFetchDirection();
-  }
-
-  @Override
-  public void setFetchSize( int rows ) throws SQLException {
-    throwIfClosed();
-    super.setFetchSize( rows );
-  }
-
-  @Override
-  public int getFetchSize() throws SQLException {
-    throwIfClosed();
-    return super.getFetchSize();
-  }
-
-  @Override
-  public int getType() throws SQLException {
-    throwIfClosed();
-    return super.getType();
-  }
-
-  @Override
-  public int getConcurrency() throws SQLException {
-    throwIfClosed();
-    return super.getConcurrency();
-  }
-
-  //---------------------------------------------------------------------
   // Updates
   //---------------------------------------------------------------------
   @Override
-  public boolean rowUpdated() throws SQLException {
-    throwIfClosed();
-    return super.rowUpdated();
-  }
-
-  @Override
-  public boolean rowInserted() throws SQLException {
-    throwIfClosed();
-    return super.rowInserted();
-  }
-
-  @Override
-  public boolean rowDeleted() throws SQLException {
-    throwIfClosed();
-    return super.rowDeleted();
-  }
-
-  @Override
-  public void updateNull( int columnIndex ) throws SQLException {
-    throwIfClosed();
+  public void updateNull(int columnIndex) throws SQLException {
+    checkOpen();
     try {
       super.updateNull( columnIndex );
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBoolean( int columnIndex, boolean x ) throws SQLException {
-    throwIfClosed();
+  public void updateBoolean(int columnIndex, boolean x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBoolean( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBoolean(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateByte( int columnIndex, byte x ) throws SQLException {
-    throwIfClosed();
+  public void updateByte(int columnIndex, byte x) throws SQLException {
+    checkOpen();
     try {
       super.updateByte( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateShort( int columnIndex, short x ) throws SQLException {
-    throwIfClosed();
+  public void updateShort(int columnIndex, short x) throws SQLException {
+    checkOpen();
     try {
-      super.updateShort( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateShort(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateInt( int columnIndex, int x ) throws SQLException {
-    throwIfClosed();
+  public void updateInt(int columnIndex, int x) throws SQLException {
+    checkOpen();
     try {
-      super.updateInt( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateInt(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateLong( int columnIndex, long x ) throws SQLException {
-    throwIfClosed();
+  public void updateLong(int columnIndex, long x) throws SQLException {
+    checkOpen();
     try {
-      super.updateLong( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateLong(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateFloat( int columnIndex, float x ) throws SQLException {
-    throwIfClosed();
+  public void updateFloat(int columnIndex, float x) throws SQLException {
+    checkOpen();
     try {
-      super.updateFloat( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateFloat(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateDouble( int columnIndex, double x ) throws SQLException {
-    throwIfClosed();
+  public void updateDouble(int columnIndex, double x) throws SQLException {
+    checkOpen();
     try {
-      super.updateDouble( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateDouble(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBigDecimal( int columnIndex,
-                                BigDecimal x ) throws SQLException {
-    throwIfClosed();
+  public void updateBigDecimal(int columnIndex, BigDecimal x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBigDecimal( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBigDecimal(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateString( int columnIndex, String x ) throws SQLException {
-    throwIfClosed();
+  public void updateString(int columnIndex, String x) throws SQLException {
+    checkOpen();
     try {
-      super.updateString( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateString(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBytes( int columnIndex, byte[] x ) throws SQLException {
-    throwIfClosed();
+  public void updateBytes(int columnIndex, byte[] x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBytes( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBytes(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateDate( int columnIndex, Date x ) throws SQLException {
-    throwIfClosed();
+  public void updateDate(int columnIndex, Date x) throws SQLException {
+    checkOpen();
     try {
-      super.updateDate( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateDate(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateTime( int columnIndex, Time x ) throws SQLException {
-    throwIfClosed();
+  public void updateTime(int columnIndex, Time x) throws SQLException {
+    checkOpen();
     try {
-      super.updateTime( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateTime(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateTimestamp( int columnIndex, Timestamp x ) throws SQLException {
-    throwIfClosed();
+  public void updateTimestamp(int columnIndex, Timestamp x) throws SQLException {
+    checkOpen();
     try {
-      super.updateTimestamp( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateTimestamp(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateAsciiStream( int columnIndex, InputStream x,
-                                 int length ) throws SQLException {
-    throwIfClosed();
+  public void updateAsciiStream(int columnIndex, InputStream x, int length) throws SQLException {
+    checkOpen();
     try {
-      super.updateAsciiStream( columnIndex, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateAsciiStream(columnIndex, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBinaryStream( int columnIndex, InputStream x,
-                                  int length ) throws SQLException {
-    throwIfClosed();
+  public void updateBinaryStream(int columnIndex, InputStream x,
+                                 int length) throws SQLException {
+    checkOpen();
     try {
-      super.updateBinaryStream( columnIndex, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBinaryStream(columnIndex, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateCharacterStream( int columnIndex, Reader x,
-                                     int length ) throws SQLException {
-    throwIfClosed();
+  public void updateCharacterStream(int columnIndex, Reader x,
+                                    int length) throws SQLException {
+    checkOpen();
     try {
-      super.updateCharacterStream( columnIndex, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateCharacterStream(columnIndex, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateObject( int columnIndex, Object x,
-                            int scaleOrLength ) throws SQLException {
-    throwIfClosed();
+  public void updateObject(int columnIndex, Object x,
+                           int scaleOrLength) throws SQLException {
+    checkOpen();
     try {
-      super.updateObject( columnIndex, x, scaleOrLength );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateObject(columnIndex, x, scaleOrLength);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateObject( int columnIndex, Object x ) throws SQLException {
-    throwIfClosed();
+  public void updateObject(int columnIndex, Object x) throws SQLException {
+    checkOpen();
     try {
-      super.updateObject( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateObject(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNull( String columnLabel ) throws SQLException {
-    throwIfClosed();
+  public void updateObject(int columnIndex, Object x, SQLType targetSqlType, int scaleOrLength) throws SQLException {
+    checkOpen();
+    super.updateObject(columnIndex, x, targetSqlType, scaleOrLength);
+  }
+
+  @Override
+  public void updateObject(String columnLabel, Object x, SQLType targetSqlType, int scaleOrLength) throws SQLException {
+    checkOpen();
+    super.updateObject(columnLabel, x, targetSqlType, scaleOrLength);
+  }
+
+  @Override
+  public void updateObject(int columnIndex, Object x, SQLType targetSqlType) throws SQLException {
+    checkOpen();
+    super.updateObject(columnIndex, x, targetSqlType);
+  }
+
+  @Override
+  public void updateObject(String columnLabel, Object x, SQLType targetSqlType) throws SQLException {
+    checkOpen();
+    super.updateObject(columnLabel, x, targetSqlType);
+  }
+
+  @Override
+  public void updateNull(String columnLabel) throws SQLException {
+    checkOpen();
     try {
-      super.updateNull( columnLabel );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNull(columnLabel);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBoolean( String columnLabel, boolean x ) throws SQLException {
-    throwIfClosed();
+  public void updateBoolean(String columnLabel, boolean x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBoolean( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBoolean(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateByte( String columnLabel, byte x ) throws SQLException {
-    throwIfClosed();
+  public void updateByte(String columnLabel, byte x) throws SQLException {
+    checkOpen();
     try {
-      super.updateByte( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateByte(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateShort( String columnLabel, short x ) throws SQLException {
-    throwIfClosed();
+  public void updateShort(String columnLabel, short x) throws SQLException {
+    checkOpen();
     try {
-      super.updateShort( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateShort(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateInt( String columnLabel, int x ) throws SQLException {
-    throwIfClosed();
+  public void updateInt(String columnLabel, int x) throws SQLException {
+    checkOpen();
     try {
-      super.updateInt( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateInt(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateLong( String columnLabel, long x ) throws SQLException {
-    throwIfClosed();
+  public void updateLong(String columnLabel, long x) throws SQLException {
+    checkOpen();
     try {
-      super.updateLong( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateLong(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateFloat( String columnLabel, float x ) throws SQLException {
-    throwIfClosed();
+  public void updateFloat(String columnLabel, float x) throws SQLException {
+    checkOpen();
     try {
-      super.updateFloat( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateFloat(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateDouble( String columnLabel, double x ) throws SQLException {
-    throwIfClosed();
+  public void updateDouble(String columnLabel, double x) throws SQLException {
+    checkOpen();
     try {
-      super.updateDouble( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateDouble(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBigDecimal( String columnLabel,
-                                BigDecimal x ) throws SQLException {
-    throwIfClosed();
+  public void updateBigDecimal(String columnLabel,
+                               BigDecimal x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBigDecimal( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBigDecimal(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateString( String columnLabel, String x ) throws SQLException {
-    throwIfClosed();
+  public void updateString(String columnLabel, String x) throws SQLException {
+    checkOpen();
     try {
-      super.updateString( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateString(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBytes( String columnLabel, byte[] x ) throws SQLException {
-    throwIfClosed();
+  public void updateBytes(String columnLabel, byte[] x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBytes( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBytes(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateDate( String columnLabel, Date x ) throws SQLException {
-    throwIfClosed();
+  public void updateDate(String columnLabel, Date x) throws SQLException {
+    checkOpen();
     try {
-      super.updateDate( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateDate(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateTime( String columnLabel, Time x ) throws SQLException {
-    throwIfClosed();
+  public void updateTime(String columnLabel, Time x) throws SQLException {
+    checkOpen();
     try {
-      super.updateTime( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateTime(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateTimestamp( String columnLabel, Timestamp x ) throws SQLException {
-    throwIfClosed();
+  public void updateTimestamp(String columnLabel, Timestamp x) throws SQLException {
+    checkOpen();
     try {
-      super.updateTimestamp( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateTimestamp(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateAsciiStream( String columnLabel, InputStream x,
-                                 int length ) throws SQLException {
-    throwIfClosed();
+  public void updateAsciiStream(String columnLabel, InputStream x,
+                                int length) throws SQLException {
+    checkOpen();
     try {
-      super.updateAsciiStream( columnLabel, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateAsciiStream(columnLabel, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBinaryStream( String columnLabel, InputStream x,
-                                  int length ) throws SQLException {
-    throwIfClosed();
+  public void updateBinaryStream(String columnLabel, InputStream x,
+                                 int length) throws SQLException {
+    checkOpen();
     try {
-      super.updateBinaryStream( columnLabel, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBinaryStream(columnLabel, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateCharacterStream( String columnLabel, Reader reader,
-                                     int length ) throws SQLException {
-    throwIfClosed();
+  public void updateCharacterStream(String columnLabel, Reader reader,
+                                    int length) throws SQLException {
+    checkOpen();
     try {
-      super.updateCharacterStream( columnLabel, reader, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateCharacterStream(columnLabel, reader, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateObject( String columnLabel, Object x,
-                            int scaleOrLength ) throws SQLException {
-    throwIfClosed();
+  public void updateObject(String columnLabel, Object x,
+                           int scaleOrLength) throws SQLException {
+    checkOpen();
     try {
-      super.updateObject( columnLabel, x, scaleOrLength );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateObject(columnLabel, x, scaleOrLength);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateObject( String columnLabel, Object x ) throws SQLException {
-    throwIfClosed();
+  public void updateObject(String columnLabel, Object x) throws SQLException {
+    checkOpen();
     try {
-      super.updateObject( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateObject(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void insertRow() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.insertRow();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void updateRow() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.updateRow();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void deleteRow() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.deleteRow();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void refreshRow() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.refreshRow();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void cancelRowUpdates() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.cancelRowUpdates();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void moveToInsertRow() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.moveToInsertRow();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void moveToCurrentRow() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.moveToCurrentRow();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
-  @Override
-  public AvaticaStatement getStatement() throws SQLException {
-    throwIfClosed();
-    return super.getStatement();
-  }
-
-  @Override
-  public Object getObject( int columnIndex,
-                           Map<String, Class<?>> map ) throws SQLException {
-    throwIfClosed();
-    return super.getObject( columnIndex, map );
-  }
-
-  @Override
-  public Ref getRef( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getRef( columnIndex );
-  }
-
-  @Override
-  public Blob getBlob( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getBlob( columnIndex );
-  }
-
-  @Override
-  public Clob getClob( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getClob( columnIndex );
-  }
-
-  @Override
-  public Array getArray( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getArray( columnIndex );
-  }
-
-  @Override
-  public Object getObject( String columnLabel,
-                           Map<String,Class<?>> map ) throws SQLException {
-    throwIfClosed();
-    return super.getObject( columnLabel, map );
-  }
-
-  @Override
-  public Ref getRef( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getRef( columnLabel );
-  }
-
-  @Override
-  public Blob getBlob( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getBlob( columnLabel );
-  }
-
-  @Override
-  public Clob getClob( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getClob( columnLabel );
-  }
-
-  @Override
-  public Array getArray( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getArray( columnLabel );
-  }
-
-  @Override
-  public Date getDate( int columnIndex, Calendar cal ) throws SQLException {
-    throwIfClosed();
-    return super.getDate( columnIndex, cal );
-  }
-
-  @Override
-  public Date getDate( String columnLabel, Calendar cal ) throws SQLException {
-    throwIfClosed();
-    return super.getDate( columnLabel, cal );
-  }
-
-  @Override
-  public Time getTime( int columnIndex, Calendar cal ) throws SQLException {
-    throwIfClosed();
-    return super.getTime( columnIndex, cal );
-  }
-
-  @Override
-  public Time getTime( String columnLabel, Calendar cal ) throws SQLException {
-    throwIfClosed();
-    return super.getTime( columnLabel, cal );
-  }
-
-  @Override
-  public Timestamp getTimestamp( int columnIndex, Calendar cal ) throws SQLException {
-    throwIfClosed();
-    return super.getTimestamp( columnIndex, cal );
-  }
-
-  @Override
-  public Timestamp getTimestamp( String columnLabel,
-                                 Calendar cal ) throws SQLException {
-    throwIfClosed();
-    return super.getTimestamp( columnLabel, cal );
-  }
-
   //-------------------------- JDBC 3.0 ----------------------------------------
 
   @Override
-  public URL getURL( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getURL( columnIndex );
-  }
-
-  @Override
-  public URL getURL( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getURL( columnLabel );
-  }
-
-  @Override
-  public void updateRef( int columnIndex, Ref x ) throws SQLException {
-    throwIfClosed();
+  public void updateRef(int columnIndex, Ref x) throws SQLException {
+    checkOpen();
     try {
-      super.updateRef( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateRef(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateRef( String columnLabel, Ref x ) throws SQLException {
-    throwIfClosed();
+  public void updateRef(String columnLabel, Ref x) throws SQLException {
+    checkOpen();
     try {
-      super.updateRef( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateRef(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBlob( int columnIndex, Blob x ) throws SQLException {
-    throwIfClosed();
+  public void updateBlob(int columnIndex, Blob x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBlob( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBlob(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBlob( String columnLabel, Blob x ) throws SQLException {
-    throwIfClosed();
+  public void updateBlob(String columnLabel, Blob x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBlob( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBlob(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateClob( int columnIndex, Clob x ) throws SQLException {
-    throwIfClosed();
+  public void updateClob(int columnIndex, Clob x) throws SQLException {
+    checkOpen();
     try {
-      super.updateClob( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateClob(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateClob( String columnLabel, Clob x ) throws SQLException {
-    throwIfClosed();
+  public void updateClob(String columnLabel, Clob x) throws SQLException {
+    checkOpen();
     try {
-      super.updateClob( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateClob(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateArray( int columnIndex, Array x ) throws SQLException {
-    throwIfClosed();
+  public void updateArray(int columnIndex, Array x) throws SQLException {
+    checkOpen();
     try {
-      super.updateArray( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateArray(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateArray( String columnLabel, Array x ) throws SQLException {
-    throwIfClosed();
+  public void updateArray(String columnLabel, Array x) throws SQLException {
+    checkOpen();
     try {
-      super.updateArray( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateArray(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   //------------------------- JDBC 4.0 -----------------------------------
   @Override
-  public RowId getRowId( int columnIndex ) throws SQLException {
-    throwIfClosed();
+  public RowId getRowId(int columnIndex) throws SQLException {
+    checkOpen();
     try {
-      return super.getRowId( columnIndex );
-    }
-    catch (UnsupportedOperationException e) {
+      return super.getRowId(columnIndex);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public RowId getRowId( String columnLabel ) throws SQLException {
-    throwIfClosed();
+  public RowId getRowId(String columnLabel) throws SQLException {
+    checkOpen();
     try {
-      return super.getRowId( columnLabel );
-    }
-    catch (UnsupportedOperationException e) {
+      return super.getRowId(columnLabel);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateRowId( int columnIndex, RowId x ) throws SQLException {
-    throwIfClosed();
+  public void updateRowId(int columnIndex, RowId x) throws SQLException {
+    checkOpen();
     try {
-      super.updateRowId( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateRowId(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateRowId( String columnLabel, RowId x ) throws SQLException {
-    throwIfClosed();
+  public void updateRowId(String columnLabel, RowId x) throws SQLException {
+    checkOpen();
     try {
-      super.updateRowId( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateRowId(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public int getHoldability() throws SQLException {
-    throwIfClosed();
-    return super.getHoldability();
-  }
-
-  @Override
-  public boolean isClosed() throws SQLException {
-    // Note:  No already-closed exception for isClosed().
-    return super.isClosed();
-  }
-
-  @Override
-  public void updateNString( int columnIndex, String nString ) throws SQLException {
-    throwIfClosed();
+  public void updateNString(int columnIndex, String nString) throws SQLException {
+    checkOpen();
     try {
-      super.updateNString( columnIndex, nString );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNString(columnIndex, nString);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNString( String columnLabel,
-                             String nString ) throws SQLException {
-    throwIfClosed();
+  public void updateNString(String columnLabel,
+                            String nString) throws SQLException {
+    checkOpen();
     try {
-      super.updateNString( columnLabel, nString );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNString(columnLabel, nString);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNClob( int columnIndex, NClob nClob ) throws SQLException {
-    throwIfClosed();
+  public void updateNClob(int columnIndex, NClob nClob) throws SQLException {
+    checkOpen();
     try {
-      super.updateNClob( columnIndex, nClob );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNClob(columnIndex, nClob);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNClob( String columnLabel, NClob nClob ) throws SQLException {
-    throwIfClosed();
+  public void updateNClob(String columnLabel, NClob nClob) throws SQLException {
+    checkOpen();
     try {
-      super.updateNClob( columnLabel, nClob );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNClob(columnLabel, nClob);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public NClob getNClob( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getNClob( columnIndex );
-  }
-
-  @Override
-  public NClob getNClob( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getNClob( columnLabel );
-  }
-
-  @Override
-  public SQLXML getSQLXML( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getSQLXML( columnIndex );
-  }
-
-  @Override
-  public SQLXML getSQLXML( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getSQLXML( columnLabel );
-  }
-
-  @Override
-  public void updateSQLXML( int columnIndex,
-                            SQLXML xmlObject ) throws SQLException {
-    throwIfClosed();
+  public void updateSQLXML(int columnIndex,
+                           SQLXML xmlObject) throws SQLException {
+    checkOpen();
     try {
-      super.updateSQLXML( columnIndex, xmlObject );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateSQLXML(columnIndex, xmlObject);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateSQLXML( String columnLabel,
-                            SQLXML xmlObject ) throws SQLException {
-    throwIfClosed();
+  public void updateSQLXML(String columnLabel,
+                           SQLXML xmlObject) throws SQLException {
+    checkOpen();
     try {
-      super.updateSQLXML( columnLabel, xmlObject );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateSQLXML(columnLabel, xmlObject);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public String getNString( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getNString( columnIndex );
-  }
-
-  @Override
-  public String getNString( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getNString( columnLabel );
-  }
-
-  @Override
-  public Reader getNCharacterStream( int columnIndex ) throws SQLException {
-    throwIfClosed();
-    return super.getNCharacterStream( columnIndex );
-  }
-
-  @Override
-  public Reader getNCharacterStream( String columnLabel ) throws SQLException {
-    throwIfClosed();
-    return super.getNCharacterStream( columnLabel );
-  }
-
-  @Override
-  public void updateNCharacterStream( int columnIndex, Reader x,
-                                      long length ) throws SQLException {
-    throwIfClosed();
+  public void updateNCharacterStream(int columnIndex, Reader x,
+                                     long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateNCharacterStream( columnIndex, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNCharacterStream(columnIndex, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNCharacterStream( String columnLabel, Reader reader,
-                                      long length ) throws SQLException {
-    throwIfClosed();
+  public void updateNCharacterStream(String columnLabel, Reader reader,
+                                     long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateNCharacterStream( columnLabel, reader, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNCharacterStream(columnLabel, reader, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateAsciiStream( int columnIndex, InputStream x,
-                                 long length ) throws SQLException {
-    throwIfClosed();
+  public void updateAsciiStream(int columnIndex, InputStream x,
+                                long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateAsciiStream( columnIndex, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateAsciiStream(columnIndex, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBinaryStream( int columnIndex, InputStream x,
-                                  long length ) throws SQLException {
-    throwIfClosed();
+  public void updateBinaryStream(int columnIndex, InputStream x,
+                                 long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateBinaryStream( columnIndex, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBinaryStream(columnIndex, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateCharacterStream( int columnIndex, Reader x,
-                                     long length ) throws SQLException {
-    throwIfClosed();
+  public void updateCharacterStream(int columnIndex, Reader x,
+                                    long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateCharacterStream( columnIndex, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateCharacterStream(columnIndex, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateAsciiStream( String columnLabel, InputStream x,
-                                 long length ) throws SQLException {
-    throwIfClosed();
+  public void updateAsciiStream(String columnLabel, InputStream x,
+                                long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateAsciiStream( columnLabel, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateAsciiStream(columnLabel, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBinaryStream( String columnLabel, InputStream x,
-                                  long length ) throws SQLException {
-    throwIfClosed();
+  public void updateBinaryStream(String columnLabel, InputStream x,
+                                 long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateBinaryStream( columnLabel, x, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBinaryStream(columnLabel, x, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateCharacterStream( String columnLabel, Reader reader,
-                                     long length ) throws SQLException {
-    throwIfClosed();
+  public void updateCharacterStream(String columnLabel, Reader reader,
+                                    long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateCharacterStream( columnLabel, reader, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateCharacterStream(columnLabel, reader, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBlob( int columnIndex, InputStream inputStream,
-                          long length ) throws SQLException {
-    throwIfClosed();
+  public void updateBlob(int columnIndex, InputStream inputStream,
+                         long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateBlob( columnIndex, inputStream, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBlob(columnIndex, inputStream, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBlob( String columnLabel, InputStream inputStream,
-                          long length ) throws SQLException {
-    throwIfClosed();
+  public void updateBlob(String columnLabel, InputStream inputStream,
+                         long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateBlob( columnLabel, inputStream, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBlob(columnLabel, inputStream, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateClob( int columnIndex,  Reader reader,
-                          long length ) throws SQLException {
-    throwIfClosed();
+  public void updateClob(int columnIndex, Reader reader,
+                         long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateClob( columnIndex, reader, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateClob(columnIndex, reader, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateClob( String columnLabel,  Reader reader,
-                          long length ) throws SQLException {
-    throwIfClosed();
+  public void updateClob(String columnLabel, Reader reader,
+                         long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateClob( columnLabel, reader, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateClob(columnLabel, reader, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNClob( int columnIndex,  Reader reader,
-                           long length ) throws SQLException {
-    throwIfClosed();
+  public void updateNClob(int columnIndex, Reader reader,
+                          long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateNClob( columnIndex, reader, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNClob(columnIndex, reader, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNClob( String columnLabel,  Reader reader,
-                           long length ) throws SQLException {
-    throwIfClosed();
+  public void updateNClob(String columnLabel, Reader reader,
+                          long length) throws SQLException {
+    checkOpen();
     try {
-      super.updateNClob( columnLabel, reader, length );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNClob(columnLabel, reader, length);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   //---
   @Override
-  public void updateNCharacterStream( int columnIndex,
-                                      Reader x ) throws SQLException {
-    throwIfClosed();
+  public void updateNCharacterStream(int columnIndex,
+                                     Reader x) throws SQLException {
+    checkOpen();
     try {
-      super.updateNCharacterStream( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNCharacterStream(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNCharacterStream( String columnLabel,
-                                      Reader reader ) throws SQLException {
-    throwIfClosed();
+  public void updateNCharacterStream(String columnLabel,
+                                     Reader reader) throws SQLException {
+    checkOpen();
     try {
-      super.updateNCharacterStream( columnLabel, reader );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNCharacterStream(columnLabel, reader);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateAsciiStream( int columnIndex,
-                                 InputStream x ) throws SQLException {
-    throwIfClosed();
+  public void updateAsciiStream(int columnIndex,
+                                InputStream x) throws SQLException {
+    checkOpen();
     try {
-      super.updateAsciiStream( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateAsciiStream(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBinaryStream( int columnIndex,
-                                  InputStream x ) throws SQLException {
-    throwIfClosed();
+  public void updateBinaryStream(int columnIndex,
+                                 InputStream x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBinaryStream( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBinaryStream(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateCharacterStream( int columnIndex,
-                                     Reader x ) throws SQLException {
-    throwIfClosed();
+  public void updateCharacterStream(int columnIndex,
+                                    Reader x) throws SQLException {
+    checkOpen();
     try {
-      super.updateCharacterStream( columnIndex, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateCharacterStream(columnIndex, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateAsciiStream( String columnLabel,
-                                 InputStream x ) throws SQLException {
-    throwIfClosed();
+  public void updateAsciiStream(String columnLabel,
+                                InputStream x) throws SQLException {
+    checkOpen();
     try {
-      super.updateAsciiStream( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateAsciiStream(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBinaryStream( String columnLabel,
-                                  InputStream x ) throws SQLException {
-    throwIfClosed();
+  public void updateBinaryStream(String columnLabel,
+                                 InputStream x) throws SQLException {
+    checkOpen();
     try {
-      super.updateBinaryStream( columnLabel, x );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBinaryStream(columnLabel, x);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateCharacterStream( String columnLabel,
-                                     Reader reader ) throws SQLException {
-    throwIfClosed();
+  public void updateCharacterStream(String columnLabel,
+                                    Reader reader) throws SQLException {
+    checkOpen();
     try {
-      super.updateCharacterStream( columnLabel, reader );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateCharacterStream(columnLabel, reader);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBlob( int columnIndex,
-                          InputStream inputStream ) throws SQLException {
-    throwIfClosed();
+  public void updateBlob(int columnIndex,
+                         InputStream inputStream) throws SQLException {
+    checkOpen();
     try {
-      super.updateBlob( columnIndex, inputStream );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBlob(columnIndex, inputStream);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateBlob( String columnLabel,
-                          InputStream inputStream ) throws SQLException {
-    throwIfClosed();
+  public void updateBlob(String columnLabel,
+                         InputStream inputStream) throws SQLException {
+    checkOpen();
     try {
-      super.updateBlob( columnLabel, inputStream );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateBlob(columnLabel, inputStream);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateClob( int columnIndex,  Reader reader ) throws SQLException {
-    throwIfClosed();
+  public void updateClob(int columnIndex, Reader reader) throws SQLException {
+    checkOpen();
     try {
-      super.updateClob( columnIndex, reader );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateClob(columnIndex, reader);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateClob( String columnLabel,  Reader reader ) throws SQLException {
-    throwIfClosed();
+  public void updateClob(String columnLabel, Reader reader) throws SQLException {
+    checkOpen();
     try {
-      super.updateClob( columnLabel, reader );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateClob(columnLabel, reader);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNClob( int columnIndex,  Reader reader ) throws SQLException {
-    throwIfClosed();
+  public void updateNClob(int columnIndex, Reader reader) throws SQLException {
+    checkOpen();
     try {
-      super.updateNClob( columnIndex, reader );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNClob(columnIndex, reader);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void updateNClob( String columnLabel,  Reader reader ) throws SQLException {
-    throwIfClosed();
+  public void updateNClob(String columnLabel, Reader reader) throws SQLException {
+    checkOpen();
     try {
-      super.updateNClob( columnLabel, reader );
-    }
-    catch (UnsupportedOperationException e) {
+      super.updateNClob(columnLabel, reader);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
-  //------------------------- JDBC 4.1 -----------------------------------
-  @Override
-  public <T> T getObject( int columnIndex, Class<T> type ) throws SQLException {
-    throwIfClosed();
-    return super.getObject( columnIndex, type );
-  }
-
-  @Override
-  public <T> T getObject( String columnLabel, Class<T> type ) throws SQLException {
-    throwIfClosed();
-    return super.getObject( columnLabel, type );
-  }
-
-
   ////////////////////////////////////////
   // DrillResultSet methods:
 
   @Override
   public String getQueryId() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     if (cursor instanceof DrillCursor) {
       return ((DrillCursor) cursor).getQueryId();
     }
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetMetaDataImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetMetaDataImpl.java
index 643f2aa..901a337 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetMetaDataImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetMetaDataImpl.java
@@ -44,8 +44,7 @@ public class DrillResultSetMetaDataImpl extends AvaticaResultSetMetaData {
    * @throws  AlreadyClosedSqlException  if ResultSet is closed
    * @throws  SQLException  if error in checking ResultSet's status
    */
-  private void throwIfClosed() throws AlreadyClosedSqlException,
-                                      SQLException {
+  private void checkOpen() throws AlreadyClosedSqlException, SQLException {
     // Statement.isClosed() call is to avoid exception from getResultSet().
     if (statement.isClosed()
         || (statement.getResultSet() != null // result set doesn't exist for prepared statement cases
@@ -56,9 +55,8 @@ public class DrillResultSetMetaDataImpl extends AvaticaResultSetMetaData {
   }
 
   private void throwIfClosedOrOutOfBounds(int columnNumber)
-      throws InvalidParameterSqlException,
-             SQLException {
-    throwIfClosed();
+      throws SQLException {
+    checkOpen();
     if (1 > columnNumber || columnNumber > getColumnCount()) {
       throw new InvalidParameterSqlException(
           "Column number " + columnNumber + " out of range of from 1 through "
@@ -81,7 +79,7 @@ public class DrillResultSetMetaDataImpl extends AvaticaResultSetMetaData {
 
   @Override
   public int getColumnCount() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     return super.getColumnCount();
   }
 
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
index a1b16cb..5386ec6 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
@@ -20,7 +20,6 @@ package org.apache.drill.jdbc.impl;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
-import java.sql.SQLWarning;
 
 import org.apache.calcite.avatica.AvaticaStatement;
 import org.apache.calcite.avatica.Meta.StatementHandle;
@@ -29,7 +28,7 @@ import org.apache.drill.jdbc.AlreadyClosedSqlException;
 import org.apache.drill.jdbc.DrillStatement;
 
 /**
- * Drill's implementation of {@link Statement}.
+ * Drill's implementation of {@link java.sql.Statement}.
  */
 // (Was abstract to avoid errors _here_ if newer versions of JDBC added
 // interface methods, but now newer versions would probably use Java 8's default
@@ -51,8 +50,9 @@ class DrillStatementImpl extends AvaticaStatement implements DrillStatement,
    *
    * @throws  AlreadyClosedSqlException  if Statement is closed
    */
-  private void throwIfClosed() throws AlreadyClosedSqlException {
-    if ( isClosed() ) {
+  @Override
+  protected void checkOpen() throws AlreadyClosedSqlException {
+    if (isClosed()) {
       throw new AlreadyClosedSqlException( "Statement is already closed." );
     }
   }
@@ -65,107 +65,52 @@ class DrillStatementImpl extends AvaticaStatement implements DrillStatement,
   // called ResultSet.)
 
   @Override
-  public DrillConnectionImpl getConnection() {
-    // Can't throw any SQLException because AvaticaConnection's getConnection() is
-    // missing "throws SQLException".
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      throw new RuntimeException(e.getMessage(), e);
-    }
+  public DrillConnectionImpl getConnection() throws SQLException {
+    checkOpen();
     return connection;
   }
 
-  // WORKAROUND:  Work around AvaticaStatement's code that wraps _any_ exception,
-  // even if SQLException, by unwrapping to get cause exception so caller can
-  // throw it directly if it's a SQLException:
-  // TODO:  Any ideas for a better name?
-  private SQLException unwrapIfExtra( final SQLException superMethodException ) {
-    final SQLException result;
-    final Throwable cause = superMethodException.getCause();
-    if ( null != cause && cause instanceof SQLException ) {
-      result = (SQLException) cause;
-    }
-    else {
-      result = superMethodException;
-    }
-    return result;
-  }
-
   @Override
-  public boolean execute( String sql ) throws SQLException {
-    throwIfClosed();
-    try {
-      return super.execute( sql );
-    }
-    catch ( final SQLException possiblyExtraWrapperException ) {
-      throw unwrapIfExtra( possiblyExtraWrapperException );
-    }
+  public long executeLargeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
+    checkOpen();
+    return super.executeLargeUpdate(sql, autoGeneratedKeys);
   }
 
   @Override
-  public ResultSet executeQuery( String sql ) throws SQLException {
-    try {
-       throwIfClosed();
-       return super.executeQuery( sql );
-    }
-    catch ( final SQLException possiblyExtraWrapperException ) {
-      throw unwrapIfExtra( possiblyExtraWrapperException );
-    }
+  public long executeLargeUpdate(String sql, int[] columnIndexes) throws SQLException {
+    checkOpen();
+    return super.executeLargeUpdate(sql, columnIndexes);
   }
 
   @Override
-  public long executeLargeUpdate( String sql ) throws SQLException {
-    throwIfClosed();
-    try {
-      return super.executeLargeUpdate( sql );
-    }
-    catch ( final SQLException possiblyExtraWrapperException ) {
-      throw unwrapIfExtra( possiblyExtraWrapperException );
-    }
+  public long executeLargeUpdate(String sql, String[] columnNames) throws SQLException {
+    checkOpen();
+    return super.executeLargeUpdate(sql, columnNames);
   }
 
   @Override
-  public int executeUpdate( String sql, int[] columnIndexes ) throws SQLException {
-    throwIfClosed();
+  public int executeUpdate(String sql, int[] columnIndexes) throws SQLException {
+    checkOpen();
     try {
-      return super.executeUpdate( sql, columnIndexes );
-    }
-    catch (UnsupportedOperationException e) {
+      return super.executeUpdate(sql, columnIndexes);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public int executeUpdate( String sql, String[] columnNames ) throws SQLException {
-    throwIfClosed();
+  public int executeUpdate(String sql, String[] columnNames) throws SQLException {
+    checkOpen();
     try {
-      return super.executeUpdate( sql, columnNames );
-    }
-    catch (UnsupportedOperationException e) {
+      return super.executeUpdate(sql, columnNames);
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void cleanUp() {
-    final DrillConnectionImpl connection1 = connection;
-    connection1.openStatementsRegistry.removeStatement(this);
-  }
-
-  @Override
-  public int getQueryTimeout() throws AlreadyClosedSqlException, SQLException
-  {
-    throwIfClosed();
-    return super.getQueryTimeout();
-  }
-
-  @Override
-  public void setQueryTimeout( int seconds )
-      throws AlreadyClosedSqlException,
-             SQLException {
-    throwIfClosed();
-    super.setQueryTimeout(seconds);
+    connection.openStatementsRegistry.removeStatement(this);
   }
 
   @Override
@@ -190,308 +135,128 @@ class DrillStatementImpl extends AvaticaStatement implements DrillStatement,
   // No close() (it doesn't throw SQLException if already closed).
 
   @Override
-  public int getMaxFieldSize() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getMaxFieldSize();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public void setMaxFieldSize(int max) throws SQLException {
-    throwIfClosed();
-    try {
-      super.setMaxFieldSize(max);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public long getLargeMaxRows() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getMaxRows() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getLargeMaxRows();
-  }
-
-  @Override
-  public void setLargeMaxRows(long max) throws SQLException {
-    throwIfClosed();
-    super.setLargeMaxRows(max);
-  }
-
-  @Override
   public void setEscapeProcessing(boolean enable) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.setEscapeProcessing(enable);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public void cancel() throws SQLException {
-    throwIfClosed();
-    super.cancel();
-  }
-
-  @Override
-  public SQLWarning getWarnings() throws SQLException {
-    throwIfClosed();
-    return super.getWarnings();
-  }
-
-  @Override
-  public void clearWarnings() throws SQLException {
-    throwIfClosed();
-    super.clearWarnings();
-  }
-
-  @Override
   public void setCursorName(String name) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.setCursorName(name);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
-  public ResultSet getResultSet() throws SQLException {
-    throwIfClosed();
-    return super.getResultSet();
-  }
-
-  @Override
-  public int getUpdateCount() throws SQLException {
-    throwIfClosed();
-    return super.getUpdateCount();
-  }
-
-  @Override
   public boolean getMoreResults() throws SQLException {
-    throwIfClosed();
     try {
       return super.getMoreResults();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public void setFetchDirection(int direction) throws SQLException {
-    throwIfClosed();
-    super.setFetchDirection(direction);
-  }
-
-  @Override
-  public int getFetchDirection() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getFetchDirection() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getFetchDirection();
-  }
-
-  @Override
-  public void setFetchSize(int rows) throws SQLException {
-    throwIfClosed();
-    super.setFetchSize(rows);
-  }
-
-  @Override
-  public int getFetchSize() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getFetchSize() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    }
-    return super.getFetchSize();
-  }
-
-  @Override
-  public int getResultSetConcurrency() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getResultSetConcurrency();
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public int getResultSetType() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getResultSetType();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void addBatch(String sql) throws SQLException {
-    throwIfClosed();
     try {
       super.addBatch(sql);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void clearBatch() throws SQLException {
-    throwIfClosed();
     try {
       super.clearBatch();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public int[] executeBatch() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.executeBatch();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean getMoreResults(int current) throws SQLException {
-    throwIfClosed();
     try {
       return super.getMoreResults(current);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public ResultSet getGeneratedKeys() throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.getGeneratedKeys();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public int executeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.executeUpdate(sql, autoGeneratedKeys);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean execute(String sql, int autoGeneratedKeys) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.execute(sql, autoGeneratedKeys);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean execute(String sql, int columnIndexes[]) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.execute(sql, columnIndexes);
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public boolean execute(String sql, String columnNames[]) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       return super.execute(sql, columnNames);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public int getResultSetHoldability() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.getResultSetHoldability();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
 
   @Override
   public void setPoolable(boolean poolable) throws SQLException {
-    throwIfClosed();
+    checkOpen();
     try {
       super.setPoolable(poolable);
-    }
-    catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
-    }
-  }
-
-  @Override
-  public boolean isPoolable() throws SQLException {
-    throwIfClosed();
-    try {
-      return super.isPoolable();
-    }
-    catch (UnsupportedOperationException e) {
+    } catch (UnsupportedOperationException e) {
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
-
-  @Override
-  public void closeOnCompletion() throws SQLException {
-    throwIfClosed();
-    super.closeOnCompletion();
-  }
-
-  @Override
-  public boolean isCloseOnCompletion() throws SQLException {
-    throwIfClosed();
-    return super.isCloseOnCompletion();
-  }
-
 }
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/JdbcTestBase.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/JdbcTestBase.java
index aa71043..48efec5 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/JdbcTestBase.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/JdbcTestBase.java
@@ -34,12 +34,11 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 
 import com.google.common.base.Function;
-import com.google.common.base.Predicate;
 import com.google.common.base.Strings;
 
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
 import org.apache.calcite.linq4j.Ord;
+import org.apache.drill.categories.JdbcTest;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.logical.LogicalPlan;
 import org.apache.drill.common.logical.data.LogicalOperator;
@@ -48,7 +47,6 @@ import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.planner.PhysicalPlanReaderTestFactory;
 import org.apache.drill.exec.util.StoragePluginTestUtils;
 import org.apache.drill.jdbc.test.Hook;
-import org.apache.drill.categories.JdbcTest;
 import org.apache.drill.test.BaseDirTestWatcher;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -85,18 +83,14 @@ public class JdbcTestBase extends ExecTest {
 
   @BeforeClass
   public static void setUpTestCase() {
-    factory = new SingleConnectionCachingFactory(new ConnectionFactory() {
-      @Override
-      public Connection getConnection(ConnectionInfo info) throws SQLException {
-        return DriverManager.getConnection(info.getUrl(), info.getParamsAsProperties());
-      }
-    });
+    factory = new SingleConnectionCachingFactory(
+        info -> DriverManager.getConnection(info.getUrl(), info.getParamsAsProperties()));
   }
 
   /**
    * Creates a {@link java.sql.Connection connection} using default parameters.
    * @param url connection URL
-   * @throws Exception if connection fails
+   * @throws SQLException if connection fails
    */
   protected static Connection connect(String url) throws SQLException {
     return connect(url, getDefaultProperties());
@@ -110,7 +104,7 @@ public class JdbcTestBase extends ExecTest {
    * Creates a {@link java.sql.Connection connection} using the given parameters.
    * @param url connection URL
    * @param info connection info
-   * @throws Exception if connection fails
+   * @throws SQLException if connection fails
    */
   protected static Connection connect(String url, Properties info) throws SQLException {
     final Connection conn = factory.getConnection(new ConnectionInfo(url, info));
@@ -278,12 +272,8 @@ public class JdbcTestBase extends ExecTest {
 
     public ModelAndSchema(final Properties info, final ConnectionFactory factory) {
       this.info = info;
-      this.adapter = new ConnectionFactoryAdapter() {
-        @Override
-        public Connection createConnection() throws SQLException {
-          return factory.getConnection(new ConnectionInfo("jdbc:drill:zk=local", ModelAndSchema.this.info));
-        }
-      };
+      this.adapter = () -> factory.getConnection(
+          new ConnectionInfo("jdbc:drill:zk=local", ModelAndSchema.this.info));
     }
 
     public TestDataConnection sql(String sql) {
@@ -291,14 +281,8 @@ public class JdbcTestBase extends ExecTest {
     }
 
     public <T> T withConnection(Function<Connection, T> function) throws Exception {
-      Connection connection = null;
-      try {
-        connection = adapter.createConnection();
+      try (Connection connection = adapter.createConnection()) {
         return function.apply(connection);
-      } finally {
-        if (connection != null) {
-          connection.close();
-        }
       }
     }
   }
@@ -316,11 +300,8 @@ public class JdbcTestBase extends ExecTest {
      * Checks that the current SQL statement returns the expected result.
      */
     public TestDataConnection returns(String expected) throws Exception {
-      Connection connection = null;
-      Statement statement = null;
-      try {
-        connection = adapter.createConnection();
-        statement = connection.createStatement();
+      try (Connection connection = adapter.createConnection();
+           Statement statement = connection.createStatement()) {
         ResultSet resultSet = statement.executeQuery(sql);
         expected = expected.trim();
         String result = JdbcTestBase.toString(resultSet).trim();
@@ -330,22 +311,12 @@ public class JdbcTestBase extends ExecTest {
           Assert.fail(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected));
         }
         return this;
-      } finally {
-        if (statement != null) {
-          statement.close();
-        }
-        if (connection != null) {
-          connection.close();
-        }
       }
     }
 
     public TestDataConnection returnsSet(Set<String> expected) throws Exception {
-      Connection connection = null;
-      Statement statement = null;
-      try {
-        connection = adapter.createConnection();
-        statement = connection.createStatement();
+      try (Connection connection = adapter.createConnection();
+           Statement statement = connection.createStatement()) {
         ResultSet resultSet = statement.executeQuery(sql);
         Set<String> result = JdbcTestBase.toStringSet(resultSet);
         resultSet.close();
@@ -354,13 +325,6 @@ public class JdbcTestBase extends ExecTest {
           Assert.fail(String.format("Generated set:\n%s\ndoes not match:\n%s", result, expected));
         }
         return this;
-      } finally {
-        if (statement != null) {
-          statement.close();
-        }
-        if (connection != null) {
-          connection.close();
-        }
       }
     }
 
@@ -369,44 +333,24 @@ public class JdbcTestBase extends ExecTest {
      * succeeds if the query returns these lines in any order.
      */
     public TestDataConnection returnsUnordered(String... expecteds) throws Exception {
-      Connection connection = null;
-      Statement statement = null;
-      try {
-        connection = adapter.createConnection();
-        statement = connection.createStatement();
+      try (Connection connection = adapter.createConnection();
+           Statement statement = connection.createStatement()) {
         ResultSet resultSet = statement.executeQuery(sql);
         Assert.assertEquals(unsortedList(Arrays.asList(expecteds)), unsortedList(JdbcTestBase.toStrings(resultSet)));
         resultSet.close();
         return this;
-      } finally {
-        if (statement != null) {
-          statement.close();
-        }
-        if (connection != null) {
-          connection.close();
-        }
       }
     }
 
     public TestDataConnection displayResults(int recordCount) throws Exception {
       // record count check is done in toString method
 
-      Connection connection = null;
-      Statement statement = null;
-      try {
-        connection = adapter.createConnection();
-        statement = connection.createStatement();
+      try (Connection connection = adapter.createConnection();
+           Statement statement = connection.createStatement()) {
         ResultSet resultSet = statement.executeQuery(sql);
         logger.debug(JdbcTestBase.toString(resultSet, recordCount));
         resultSet.close();
         return this;
-      } finally {
-        if (statement != null) {
-          statement.close();
-        }
-        if (connection != null) {
-          connection.close();
-        }
       }
     }
 
@@ -420,49 +364,26 @@ public class JdbcTestBase extends ExecTest {
 
     public LogicalPlan logicalPlan() {
       final String[] plan0 = {null};
-      Connection connection = null;
-      Statement statement = null;
-      final Hook.Closeable x = Hook.LOGICAL_PLAN.add(new Function<String, Void>() {
-        @Override
-        public Void apply(String o) {
+      try (Connection connection = adapter.createConnection();
+           Statement statement = connection.prepareStatement(sql);
+           Hook.Closeable x = Hook.LOGICAL_PLAN.add(
+        (Function<String, Void>) o -> {
           plan0[0] = o;
           return null;
-        }
-      });
-      try {
-        connection = adapter.createConnection();
-        statement = connection.prepareStatement(sql);
+        })) {
         statement.close();
         final String plan = plan0[0].trim();
         return LogicalPlan.parse(PhysicalPlanReaderTestFactory.defaultLogicalPlanPersistence(DrillConfig.create()), plan);
       } catch (Exception e) {
         throw new RuntimeException(e);
-      } finally {
-        if (statement != null) {
-          try {
-            statement.close();
-          } catch (SQLException e) {
-            // ignore
-          }
-        }
-        if (connection != null) {
-          try {
-            connection.close();
-          } catch (SQLException e) {
-            // ignore
-          }
-        }
-        x.close();
       }
     }
 
     public <T extends LogicalOperator> T planContains(final Class<T> operatorClazz) {
-      return (T) Iterables.find(logicalPlan().getSortedOperators(), new Predicate<LogicalOperator>() {
-        @Override
-        public boolean apply(LogicalOperator input) {
-          return input.getClass().equals(operatorClazz);
-        }
-      });
+      return (T) logicalPlan().getSortedOperators().stream()
+          .filter(input -> input.getClass().equals(operatorClazz))
+          .findFirst()
+          .get();
     }
   }
 
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2489CallsAfterCloseThrowExceptionsTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2489CallsAfterCloseThrowExceptionsTest.java
index 2ce2766..303565f 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2489CallsAfterCloseThrowExceptionsTest.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2489CallsAfterCloseThrowExceptionsTest.java
@@ -41,7 +41,6 @@ import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLClientInfoException;
 import java.sql.SQLException;
-import java.sql.SQLFeatureNotSupportedException;
 import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.List;
@@ -64,8 +63,8 @@ import org.apache.drill.jdbc.AlreadyClosedSqlException;
  *   {@link DatabaseMetaData}.
  * </p>
  * <p>
- *   It does not cover unimplemented {@link CallableStatement} or any relevant
- *   secondary objects such as {@link Array} or {@link Struct}).
+ *   It does not cover unimplemented {@link java.sql.CallableStatement} or any relevant
+ *   secondary objects such as {@link java.sql.Array} or {@link java.sql.Struct}).
  * </p>
  */
 @Category(JdbcTest.class)
@@ -425,15 +424,7 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTestBase {
                     || method.getName().equals("getClientInfo"))) {
         // Special good case--we had to use SQLClientInfoException from those.
         result = true;
-      }
-      else if (RuntimeException.class == cause.getClass()
-               && normalClosedExceptionText.equals(cause.getMessage())
-               && (method.getName().equals("getCatalog")
-                  || method.getName().equals("getSchema"))) {
-        // Special good-enough case--we had to use RuntimeException for now.
-        result = true;
-      }
-      else {
+      } else {
         result = false;
       }
       return result;
@@ -462,41 +453,6 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTestBase {
     ClosedPlainStatementChecker(Class<Statement> intf, Statement jdbcObject) {
       super(intf, jdbcObject, PLAIN_STATEMENT_CLOSED_MESSAGE);
     }
-
-    @Override
-    protected boolean isOkayNonthrowingMethod(Method method) {
-      // TODO: Java 8 method
-      if ("getLargeUpdateCount".equals(method.getName())) {
-        return true; }
-      return super.isOkayNonthrowingMethod(method);
-    }
-
-    @Override
-    protected boolean isOkaySpecialCaseException(Method method, Throwable cause) {
-      final boolean result;
-      if (super.isOkaySpecialCaseException(method, cause)) {
-        result = true;
-      }
-      else if (method.getName().equals("executeLargeBatch")
-               || method.getName().equals("executeLargeUpdate")) {
-        // TODO: New Java 8 methods not implemented in Avatica.
-        result = true;
-      }
-      else if (RuntimeException.class == cause.getClass()
-               && normalClosedExceptionText.equals(cause.getMessage())
-               && (method.getName().equals("getConnection")
-                   || method.getName().equals("getFetchDirection")
-                   || method.getName().equals("getFetchSize")
-                   || method.getName().equals("getMaxRows")
-                   || method.getName().equals("getLargeMaxRows"))) {
-        // Special good-enough case--we had to use RuntimeException for now.
-        result = true;
-      }
-      else {
-        result = false;
-      }
-      return result;
-    }
   } // class ClosedPlainStatementChecker
 
   @Test
@@ -521,40 +477,6 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTestBase {
                                    PreparedStatement jdbcObject) {
       super(intf, jdbcObject, PREPAREDSTATEMENT_CLOSED_MESSAGE);
     }
-
-    @Override
-    protected boolean isOkayNonthrowingMethod(Method method) {
-      // TODO: Java 8 methods not yet supported by Avatica.
-      if (method.getName().equals("getLargeUpdateCount")) {
-        return true;
-      }
-      return super.isOkayNonthrowingMethod(method);
-    }
-
-    @Override
-    protected boolean isOkaySpecialCaseException(Method method, Throwable cause) {
-      final boolean result;
-      if (super.isOkaySpecialCaseException(method, cause)) {
-        result = true;
-      }
-      else if (RuntimeException.class == cause.getClass()
-               && cause.getMessage().contains(normalClosedExceptionText)
-               && (method.getName().equals("getConnection")
-                   || method.getName().equals("getFetchDirection")
-                   || method.getName().equals("getFetchSize")
-                   || method.getName().equals("getMaxRows")
-                   || method.getName().equals("getMetaData")
-                   || method.getName().equals("clearBatch"))) {
-        // Special good-enough case--we had to use RuntimeException for now.
-        result = true;
-      } else {
-        result = method.getName().equals("setObject")
-          || method.getName().equals("executeLargeUpdate")
-          || method.getName().equals("executeLargeBatch")
-          || method.getName().equals("getLargeMaxRows");
-      }
-      return result;
-    }
   } // class closedPreparedStmtOfOpenConnChecker
 
   @Test
@@ -579,29 +501,6 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTestBase {
     ClosedResultSetChecker(Class<ResultSet> intf, ResultSet jdbcObject) {
       super(intf, jdbcObject, RESULTSET_CLOSED_MESSAGE);
     }
-
-    @Override
-    protected boolean isOkaySpecialCaseException(Method method, Throwable cause) {
-      final boolean result;
-      if (super.isOkaySpecialCaseException(method, cause)) {
-        result = true;
-      }
-      else if (RuntimeException.class == cause.getClass()
-               && normalClosedExceptionText.equals(cause.getMessage())
-               && method.getName().equals("getStatement")) {
-        // Special good-enough case--we had to use RuntimeException for now.
-        result = true;
-      }
-      else if (SQLFeatureNotSupportedException.class == cause.getClass()
-               && (method.getName().equals("updateObject"))) {
-        // TODO: Java 8 methods not yet supported by Avatica.
-        result = true;
-      }
-      else {
-        result = false;
-      }
-      return result;
-    }
   } // class ClosedResultSetChecker
 
   @Test
diff --git a/pom.xml b/pom.xml
index 165e7e2..bf9cf10 100644
--- a/pom.xml
+++ b/pom.xml
@@ -45,8 +45,8 @@
     <dep.guava.version>18.0</dep.guava.version>
     <forkCount>2</forkCount>
     <parquet.version>1.10.0</parquet.version>
-    <calcite.version>1.16.0-drill-r7</calcite.version>
-    <avatica.version>1.11.0</avatica.version>
+    <calcite.version>1.17.0-drill-r0</calcite.version>
+    <avatica.version>1.12.0</avatica.version>
     <janino.version>2.7.6</janino.version>
     <sqlline.version>1.1.9-drill-r7</sqlline.version>
     <jackson.version>2.9.5</jackson.version>