You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by am...@apache.org on 2018/01/18 23:17:28 UTC

[04/18] drill git commit: DRILL-3993: Fix unit test failures connected with support Calcite 1.13

DRILL-3993: Fix unit test failures connected with support Calcite 1.13

- Use root schema as default for describe table statement.
Fix TestOpenTSDBPlugin.testDescribe() and TestInfoSchemaOnHiveStorage.varCharMaxLengthAndDecimalPrecisionInInfoSchema() unit tests.
- Modify expected results for tests:
TestPreparedStatementProvider.invalidQueryValidationError();
TestProjectPushDown.testTPCH1();
TestProjectPushDown.testTPCH3();
TestStorageBasedHiveAuthorization.selectUser1_db_u0_only();
TestStorageBasedHiveAuthorization.selectUser0_db_u1g1_only()
- Fix TestCTAS.whenTableQueryColumnHasStarAndTableFiledListIsSpecified(), TestViewSupport.createViewWhenViewQueryColumnHasStarAndViewFiledListIsSpecified(), TestInbuiltHiveUDFs.testIf(), testDisableUtf8SupportInQueryString unit tests.
- Fix UnsupportedOperationException and NPE for jdbc tests.
- Fix AssertionError: Conversion to relational algebra failed to preserve datatypes

*DrillCompoundIdentifier:
According to the changes, made in [CALCITE-546], star Identifier is replaced by empty string during parsing the query. Since Drill uses its own DrillCompoundIdentifier, it should also replace star by empty string before creating SqlIdentifier instance to avoid further errors connected with star column. see SqlIdentifier.isStar() method.

*SqlConverter:
In [CALCITE-1417] added simplification of expressions which should be projected every time when a new project rel node is created using RelBuilder. It causes assertion errors connected with types nullability. This hook was set to false to avoid project expressions simplification. See usage of this hook and RelBuilder.project() method.

In Drill the type nullability of the function depends on only the nullability of its arguments. In some cases, a function may return null value even if it had non-nullable arguments. When Calice simplifies expressions, it checks that the type of the result is the same as the type of the expression. Otherwise, makeCast() method is called. But when a function returns null literal, this cast does nothing, even when the function has a non-nullable type. So to avoid this issue, method makeCast() was overridden.

*DrillAvgVarianceConvertlet:
Problem with sum0 and specific changes in old Calcite (it is CALCITE-777). (see HistogramShuttle.visitCall method) Changes were made to avoid changes in Calcite.

*SqlConverter, DescribeTableHandler, ShowTablesHandler:
New Calcite tries to combine both default and specified workspaces during the query validation. In some cases, for example, when describe table statement is used, Calcite tries to find INFORMATION_SCHEMA in the schema used as default. When it does not find the schema, it tries to find a table with such name. For some storage plugins, such as opentsdb and hbase, when a table was not found, the error is thrown, and the query fails. To avoid this issue, default schema was changed to root schema for validation stage for describe table and show tables queries.


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

Branch: refs/heads/master
Commit: 3896a58243f310c5d9466a98edc205b61f9dd2e7
Parents: 9fabb61
Author: Volodymyr Vysotskyi <vv...@gmail.com>
Authored: Fri Nov 3 12:18:09 2017 +0000
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Tue Jan 16 12:10:13 2018 +0200

----------------------------------------------------------------------
 .../drill/exec/fn/hive/TestInbuiltHiveUDFs.java |  2 +-
 .../hive/TestStorageBasedHiveAuthorization.java |  6 +-
 .../planner/sql/DrillAvgVarianceConvertlet.java |  8 +--
 .../sql/DrillCalciteSqlAggFunctionWrapper.java  |  6 ++
 .../drill/exec/planner/sql/SqlConverter.java    | 53 +++++++++++++-
 .../planner/sql/handlers/DefaultSqlHandler.java |  4 +-
 .../sql/handlers/DescribeTableHandler.java      | 16 ++++-
 .../planner/sql/handlers/ShowTablesHandler.java | 15 ++++
 .../planner/sql/handlers/SqlHandlerUtil.java    |  4 +-
 .../sql/parser/DrillCompoundIdentifier.java     | 72 ++++++++++----------
 .../ischema/InfoSchemaRecordGenerator.java      |  3 +-
 .../org/apache/drill/TestProjectPushDown.java   |  4 +-
 .../drill/TestUtf8SupportInQueryString.java     | 24 +++----
 .../prepare/TestPreparedStatementProvider.java  |  5 +-
 .../apache/drill/jdbc/impl/DrillMetaImpl.java   | 14 ++--
 .../jdbc/impl/DrillPreparedStatementImpl.java   |  8 ++-
 ...l2489CallsAfterCloseThrowExceptionsTest.java | 45 +++++-------
 ...69UnsupportedReportsUseSqlExceptionTest.java | 15 ++--
 18 files changed, 189 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
index 5654ff0..d4e0b5c 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
@@ -95,7 +95,7 @@ public class TestInbuiltHiveUDFs extends HiveTestBase {
   @Test // DRILL-3272
   public void testIf() throws Exception {
     testBuilder()
-        .sqlQuery("select `if`(1999 > 2000, 'latest', 'old') Period from hive.kv limit 1")
+        .sqlQuery("select `if`(1999 > 2000, 'latest', 'old') `Period` from hive.kv limit 1")
         .ordered()
         .baselineColumns("Period")
         .baselineValues("old")

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestStorageBasedHiveAuthorization.java
----------------------------------------------------------------------
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 26a8d67..685d3bf 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
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -469,7 +469,7 @@ public class TestStorageBasedHiveAuthorization extends BaseTestHiveImpersonation
 
     errorMsgTestHelper(
         String.format("SELECT * FROM hive.%s.%s ORDER BY gpa DESC LIMIT 2", db_u1g1_only, u1g1_student_all_755),
-        String.format("Table 'hive.%s.%s' not found", db_u1g1_only, u1g1_student_all_755));
+        String.format("Object '%s' not found within 'hive.%s'", u1g1_student_all_755, db_u1g1_only));
   }
 
   // Try to read the tables "user1" has access to read in db_general.
@@ -489,7 +489,7 @@ public class TestStorageBasedHiveAuthorization extends BaseTestHiveImpersonation
 
     errorMsgTestHelper(
         String.format("SELECT * FROM hive.%s.%s ORDER BY gpa DESC LIMIT 2", db_u0_only, u0_student_all_755),
-        String.format("Table 'hive.%s.%s' not found", db_u0_only, u0_student_all_755));
+        String.format("Object '%s' not found within 'hive.%s'", u0_student_all_755, db_u0_only));
   }
 
   private static void queryViewHelper(final String queryUser, final String query) throws Exception {

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillAvgVarianceConvertlet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillAvgVarianceConvertlet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillAvgVarianceConvertlet.java
index 01c7616..bfb4c05 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillAvgVarianceConvertlet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillAvgVarianceConvertlet.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -88,7 +88,7 @@ public class DrillAvgVarianceConvertlet implements SqlRexConvertlet {
       final SqlNode arg) {
     final SqlParserPos pos = SqlParserPos.ZERO;
     final SqlNode sum =
-        SqlStdOperatorTable.SUM.createCall(pos, arg);
+        DrillCalciteSqlAggFunctionWrapper.SUM.createCall(pos, arg);
     final SqlNode count =
         SqlStdOperatorTable.COUNT.createCall(pos, arg);
     final SqlNode sumAsDouble =
@@ -128,9 +128,9 @@ public class DrillAvgVarianceConvertlet implements SqlRexConvertlet {
     final SqlNode argSquared =
         SqlStdOperatorTable.MULTIPLY.createCall(pos, castHighArg, castHighArg);
     final SqlNode sumArgSquared =
-        SqlStdOperatorTable.SUM.createCall(pos, argSquared);
+        DrillCalciteSqlAggFunctionWrapper.SUM.createCall(pos, argSquared);
     final SqlNode sum =
-        SqlStdOperatorTable.SUM.createCall(pos, castHighArg);
+        DrillCalciteSqlAggFunctionWrapper.SUM.createCall(pos, castHighArg);
     final SqlNode sumSquared =
         SqlStdOperatorTable.MULTIPLY.createCall(pos, sum, sum);
     final SqlNode count =

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlAggFunctionWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlAggFunctionWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlAggFunctionWrapper.java
index a87247b..bd46d2d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlAggFunctionWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlAggFunctionWrapper.java
@@ -24,6 +24,7 @@ import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.SqlSyntax;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
@@ -42,6 +43,11 @@ import java.util.List;
  * simply forwards the method calls to the wrapped SqlAggFunction.
  */
 public class DrillCalciteSqlAggFunctionWrapper extends SqlAggFunction implements DrillCalciteSqlWrapper {
+
+  public final static DrillCalciteSqlAggFunctionWrapper SUM =
+      new DrillCalciteSqlAggFunctionWrapper(SqlStdOperatorTable.SUM,
+          SqlStdOperatorTable.SUM.getReturnTypeInference());
+
   private final SqlAggFunction operator;
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
----------------------------------------------------------------------
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 6f59fcf..f900587 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
@@ -22,6 +22,7 @@ import java.util.List;
 import java.util.Set;
 
 import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
@@ -43,7 +44,10 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
 import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperatorTable;
@@ -73,7 +77,6 @@ import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.logical.DrillConstExecutor;
 import org.apache.drill.exec.planner.physical.DrillDistributionTraitDef;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
-import org.apache.drill.exec.planner.sql.parser.impl.DrillParserWithCompoundIdConverter;
 import org.apache.drill.exec.rpc.user.UserSession;
 
 import com.google.common.base.Joiner;
@@ -106,6 +109,7 @@ public class SqlConverter {
 
   private String sql;
   private VolcanoPlanner planner;
+  private boolean useRootSchema = false;
 
 
   public SqlConverter(QueryContext context) {
@@ -217,6 +221,15 @@ public class SqlConverter {
     catalog.disallowTemporaryTables();
   }
 
+  /**
+   * Is root schema path should be used as default schema path.
+   *
+   * @param useRoot flag
+   */
+  public void useRootSchemaAsDefault(boolean useRoot) {
+    useRootSchema = useRoot;
+  }
+
   private class DrillValidator extends SqlValidatorImpl {
     private final Set<SqlValidatorScope> identitySet = Sets.newIdentityHashSet();
 
@@ -273,6 +286,14 @@ public class SqlConverter {
     final SqlToRelConverter sqlToRelConverter =
         new SqlToRelConverter(new Expander(), validator, catalog, cluster, DrillConvertletTable.INSTANCE,
             sqlToRelConverterConfig);
+
+    /*
+     * Sets value to false to avoid simplifying project expressions
+     * 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));
+
     //To avoid unexpected column errors set a value of top to false
     final RelRoot rel = sqlToRelConverter.convertQuery(validatedNode, false, false);
     final RelRoot rel2 = rel.withRel(sqlToRelConverter.flattenTypes(rel.rel, true));
@@ -430,6 +451,28 @@ public class SqlConverter {
         boolean matchNullability) {
       return node;
     }
+
+    /**
+     * Creates a call to the CAST operator, expanding if possible, and optionally
+     * also preserving nullability.
+     *
+     * <p>Tries to expand the cast, and therefore the result may be something
+     * other than a {@link RexCall} to the CAST operator, such as a
+     * {@link RexLiteral} if {@code matchNullability} is false.
+     *
+     * @param type             Type to cast to
+     * @param exp              Expression being cast
+     * @param matchNullability Whether to ensure the result has the same
+     *                         nullability as {@code type}
+     * @return Call to CAST operator
+     */
+    @Override
+    public RexNode makeCast(RelDataType type, RexNode exp, boolean matchNullability) {
+      if (matchNullability) {
+        return makeAbstractCast(type, exp);
+      }
+      return super.makeCast(type, exp, false);
+    }
   }
 
   /**
@@ -506,6 +549,14 @@ public class SqlConverter {
       return table;
     }
 
+    @Override
+    public List<List<String>> getSchemaPaths() {
+      if (useRootSchema) {
+        return ImmutableList.<List<String>>of(ImmutableList.<String>of());
+      }
+      return super.getSchemaPaths();
+    }
+
     /**
      * check if the schema provided is a valid schema:
      * <li>schema is not indicated (only one element in the names list)<li/>

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
----------------------------------------------------------------------
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 93113cb..5c34323 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
@@ -41,8 +41,6 @@ import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.TableFunctionScan;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.logical.LogicalValues;
-import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
-import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
 import org.apache.calcite.rel.metadata.JaninoRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
@@ -622,7 +620,7 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
 
   }
 
-  private Pair<SqlNode, RelDataType> validateNode(SqlNode sqlNode) throws ValidationException, RelConversionException, ForemanSetupException {
+  protected Pair<SqlNode, RelDataType> validateNode(SqlNode sqlNode) throws ValidationException, RelConversionException, ForemanSetupException {
     final SqlNode sqlNodeValidated = config.getConverter().validate(sqlNode);
     final Pair<SqlNode, RelDataType> typedSqlNode = new Pair<>(sqlNodeValidated, config.getConverter().getOutputType(
         sqlNodeValidated));

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeTableHandler.java
----------------------------------------------------------------------
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 0311dfc..4d01424 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
@@ -28,8 +28,8 @@ import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.TAB_COLUMN
 
 import java.util.List;
 
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.sql.SqlDescribeTable;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
@@ -38,9 +38,12 @@ import org.apache.calcite.sql.SqlSelect;
 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.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.drill.common.exceptions.UserException;
 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.work.foreman.ForemanSetupException;
@@ -134,4 +137,15 @@ public class DescribeTableHandler extends DefaultSqlHandler {
           .build(logger);
     }
   }
+
+  @Override
+  protected Pair<SqlNode, RelDataType> validateNode(SqlNode sqlNode) throws ValidationException,
+      RelConversionException, ForemanSetupException {
+    SqlConverter converter = config.getConverter();
+    // set this to true since INFORMATION_SCHEMA in the root schema, not in the default
+    converter.useRootSchemaAsDefault(true);
+    Pair<SqlNode, RelDataType> sqlNodeRelDataTypePair = super.validateNode(sqlNode);
+    converter.useRootSchemaAsDefault(false);
+    return sqlNodeRelDataTypePair;
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowTablesHandler.java
----------------------------------------------------------------------
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 ada7c3a..7084877 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
@@ -25,6 +25,7 @@ import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.TAB_TABLES
 
 import java.util.List;
 
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlLiteral;
@@ -34,9 +35,12 @@ import org.apache.calcite.sql.SqlSelect;
 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.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.drill.common.exceptions.UserException;
 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.SqlShowTables;
 import org.apache.drill.exec.store.AbstractSchema;
@@ -105,4 +109,15 @@ public class ShowTablesHandler extends DefaultSqlHandler {
     return new SqlSelect(SqlParserPos.ZERO, null, new SqlNodeList(selectList, SqlParserPos.ZERO),
         fromClause, where, null, null, null, null, null, null);
   }
+
+  @Override
+  protected Pair<SqlNode, RelDataType> validateNode(SqlNode sqlNode) throws ValidationException,
+      RelConversionException, ForemanSetupException {
+    SqlConverter converter = config.getConverter();
+    // set this to true since INFORMATION_SCHEMA in the root schema, not in the default
+    converter.useRootSchemaAsDefault(true);
+    Pair<SqlNode, RelDataType> sqlNodeRelDataTypePair = super.validateNode(sqlNode);
+    converter.useRootSchemaAsDefault(false);
+    return sqlNodeRelDataTypePair;
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
index 04930a8..72d2699 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
@@ -86,9 +86,9 @@ public class SqlHandlerUtil {
             .build(logger);
       }
 
-      // CTAS's query field list shouldn't have "*" when table's field list is specified.
+      // CTAS's query field list shouldn't have "**" when table's field list is specified.
       for (String field : validatedRowtype.getFieldNames()) {
-        if (field.equals("*")) {
+        if (field.equals("**")) {
           final String tblType = isNewTableView ? "view" : "table";
           throw UserException.validationError()
               .message("%s's query field list has a '*', which is invalid when %s's field list is specified.",

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillCompoundIdentifier.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillCompoundIdentifier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillCompoundIdentifier.java
index fe96be4..a6c75c1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillCompoundIdentifier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillCompoundIdentifier.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -20,6 +20,7 @@ package org.apache.drill.exec.planner.sql.parser;
 import java.util.Collections;
 import java.util.List;
 
+import com.google.common.base.Function;
 import org.apache.calcite.sql.SqlBasicCall;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlLiteral;
@@ -30,13 +31,19 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
-public class DrillCompoundIdentifier extends SqlIdentifier{
+public class DrillCompoundIdentifier extends SqlIdentifier {
 
-  List<IdentifierHolder> ids;
+  private static final Function<String, String> STAR_TO_EMPTY = new Function<String, String>() {
+    public String apply(String s) {
+      return s.equals("*") ? "" : s;
+    }
+  };
+
+  private final List<IdentifierHolder> ids;
 
-  private static List<String> getNames(List<IdentifierHolder> identifiers){
+  private static List<String> getNames(List<IdentifierHolder> identifiers) {
     List<String> names = Lists.newArrayListWithCapacity(identifiers.size());
-    for(IdentifierHolder h : identifiers){
+    for (IdentifierHolder h : identifiers) {
       names.add(h.value);
     }
     return names;
@@ -47,74 +54,69 @@ public class DrillCompoundIdentifier extends SqlIdentifier{
     this.ids = identifiers;
   }
 
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillCompoundIdentifier.class);
-
-  public static Builder newBuilder(){
+  public static Builder newBuilder() {
     return new Builder();
   }
 
   public static class Builder {
     private List<IdentifierHolder> identifiers = Lists.newArrayList();
 
-    public DrillCompoundIdentifier build(){
+    public DrillCompoundIdentifier build() {
       return new DrillCompoundIdentifier(identifiers);
     }
 
-    public void addString(String name, SqlParserPos pos){
+    public void addString(String name, SqlParserPos pos) {
       identifiers.add(new IdentifierHolder(name, pos, false));
     }
 
-    public void addIndex(int index, SqlParserPos pos){
+    public void addIndex(int index, SqlParserPos pos) {
       identifiers.add(new IdentifierHolder(Integer.toString(index), pos, true));
     }
   }
 
-  public SqlNode getAsSqlNode(){
-    if(ids.size() == 1){
+  public SqlNode getAsSqlNode() {
+    if (ids.size() == 1) {
       return new SqlIdentifier(Collections.singletonList(ids.get(0).value), ids.get(0).parserPos);
     }
 
     int startIndex;
     SqlNode node;
 
-    if(ids.get(1).isArray()){
+    if (ids.get(1).isArray()) {
       // handle everything post zero index as item operator.
       startIndex = 1;
-      node = new SqlIdentifier( //
-          ImmutableList.of(ids.get(0).value), //
-          null, //
-          ids.get(0).parserPos, //
+      node = new SqlIdentifier(
+          ImmutableList.of(ids.get(0).value),
+          null,
+          ids.get(0).parserPos,
           ImmutableList.of(ids.get(0).parserPos));
-    }else{
+    } else {
       // handle everything post two index as item operator.
       startIndex = 2;
-      node = new SqlIdentifier( //
-          ImmutableList.of(ids.get(0).value, ids.get(1).value), //
-          null, //
-          ids.get(0).parserPos, //
+      node = new SqlIdentifier(
+          // Replaces star by empty string. See SqlIdentifier#isStar()
+          ImmutableList.of(ids.get(0).value, STAR_TO_EMPTY.apply(ids.get(1).value)), null,
+          ids.get(0).parserPos,
           ImmutableList.of(ids.get(0).parserPos, ids.get(1).parserPos));
-
     }
-    for(int i = startIndex ; i < ids.size(); i++){
+    for (int i = startIndex; i < ids.size(); i++) {
       node = ids.get(i).getNode(node);
     }
 
     return node;
   }
 
-
-  public SqlNode getAsCompoundIdentifier(){
+  public SqlNode getAsCompoundIdentifier() {
     List<String> names = Lists.newArrayListWithCapacity(ids.size());
     List<SqlParserPos> pos = Lists.newArrayListWithCapacity(ids.size());
-    for(int i =0; i < ids.size(); i++){
-      IdentifierHolder holder = ids.get(i);
+    for (IdentifierHolder holder : ids) {
       names.add(holder.value);
       pos.add(holder.parserPos);
     }
     return new SqlIdentifier(names, null, pos.get(0), pos);
   }
 
-  private static class IdentifierHolder{
+  private static class IdentifierHolder {
     String value;
     SqlParserPos parserPos;
     boolean isArray;
@@ -126,18 +128,18 @@ public class DrillCompoundIdentifier extends SqlIdentifier{
       this.parserPos = parserPos;
     }
 
-    public boolean isArray(){
+    public boolean isArray() {
       return isArray;
     }
 
-    public SqlNode getNode(SqlNode node){
+    public SqlNode getNode(SqlNode node) {
       SqlLiteral literal;
-      if(isArray){
+      if (isArray) {
         literal = SqlLiteral.createExactNumeric(value, parserPos);
-      }else{
+      } else {
         literal = SqlLiteral.createCharString(value, parserPos);
       }
-      return new SqlBasicCall(SqlStdOperatorTable.ITEM, new SqlNode[]{ node, literal }, parserPos);
+      return new SqlBasicCall(SqlStdOperatorTable.ITEM, new SqlNode[]{node, literal}, parserPos);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaRecordGenerator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaRecordGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaRecordGenerator.java
index e96ec68..d2c8c6f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaRecordGenerator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaRecordGenerator.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.store.ischema;
 
+import static org.apache.drill.exec.planner.types.DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.CATS_COL_CATALOG_NAME;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.COLS_COL_COLUMN_NAME;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.IS_CATALOG_CONNECT;
@@ -231,7 +232,7 @@ public abstract class InfoSchemaRecordGenerator<S> {
       // Visit the table, and if requested ...
       if(shouldVisitTable(schemaPath, tableName, tableType) && visitTable(schemaPath, tableName, table)) {
         // ... do for each of the table's fields.
-        final RelDataType tableRow = table.getRowType(new JavaTypeFactoryImpl());
+        final RelDataType tableRow = table.getRowType(new JavaTypeFactoryImpl(DRILL_REL_DATATYPE_SYSTEM));
         for (RelDataTypeField field: tableRow.getFieldList()) {
           if (shouldVisitColumn(schemaPath, tableName, field.getName())) {
             visitField(schemaPath, tableName, field);

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/exec/java-exec/src/test/java/org/apache/drill/TestProjectPushDown.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestProjectPushDown.java b/exec/java-exec/src/test/java/org/apache/drill/TestProjectPushDown.java
index 41d1227..ad55a0d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestProjectPushDown.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestProjectPushDown.java
@@ -102,14 +102,14 @@ public class TestProjectPushDown extends PlanTestBase {
 
   @Test
   public void testTPCH1() throws Exception {
-    String expectedColNames = " \"columns\" : [ \"`l_returnflag`\", \"`l_linestatus`\", \"`l_shipdate`\", \"`l_quantity`\", \"`l_extendedprice`\", \"`l_discount`\", \"`l_tax`\" ]";
+    String expectedColNames = " \"columns\" : [ \"`l_shipdate`\", \"`l_returnflag`\", \"`l_linestatus`\", \"`l_quantity`\", \"`l_extendedprice`\", \"`l_discount`\", \"`l_tax`\" ]";
     testPhysicalPlanFromFile("queries/tpch/01.sql", expectedColNames);
   }
 
   @Test
   public void testTPCH3() throws Exception {
     String expectedColNames1 = "\"columns\" : [ \"`c_mktsegment`\", \"`c_custkey`\" ]";
-    String expectedColNames2 = " \"columns\" : [ \"`o_orderdate`\", \"`o_shippriority`\", \"`o_custkey`\", \"`o_orderkey`\" ";
+    String expectedColNames2 = " \"columns\" : [ \"`o_custkey`\", \"`o_orderkey`\", \"`o_orderdate`\", \"`o_shippriority`\" ]";
     String expectedColNames3 = "\"columns\" : [ \"`l_orderkey`\", \"`l_shipdate`\", \"`l_extendedprice`\", \"`l_discount`\" ]";
     testPhysicalPlanFromFile("queries/tpch/03.sql", expectedColNames1, expectedColNames2, expectedColNames3);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/exec/java-exec/src/test/java/org/apache/drill/TestUtf8SupportInQueryString.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestUtf8SupportInQueryString.java b/exec/java-exec/src/test/java/org/apache/drill/TestUtf8SupportInQueryString.java
index 8dfddde..a515763 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestUtf8SupportInQueryString.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestUtf8SupportInQueryString.java
@@ -16,17 +16,16 @@
  */
 package org.apache.drill;
 
-import mockit.Deencapsulation;
 import mockit.Mock;
 import mockit.MockUp;
 import mockit.integration.junit4.JMockit;
-import org.apache.calcite.util.SaffronProperties;
+import org.apache.calcite.util.Util;
 import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.test.BaseTestQuery;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
-import java.util.Properties;
+import java.nio.charset.Charset;
 
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.junit.Assert.assertThat;
@@ -47,19 +46,16 @@ public class TestUtf8SupportInQueryString extends BaseTestQuery {
 
   @Test(expected = UserRemoteException.class)
   public void testDisableUtf8SupportInQueryString() throws Exception {
-    Deencapsulation.setField(SaffronProperties.class, "properties", null);
-    final Properties properties = System.getProperties();
     final String charset = "ISO-8859-1";
-    new MockUp<System>()
+
+    // Mocked Util.getDefaultCharset() since it uses static field Util.DEFAULT_CHARSET
+    // which is initialized when declared using SaffronProperties.INSTANCE field which also is initialized
+    // when declared.
+    new MockUp<Util>()
     {
       @Mock
-      Properties getProperties() {
-        Properties newProperties = new Properties();
-        newProperties.putAll(properties);
-        newProperties.put("saffron.default.charset", charset);
-        newProperties.put("saffron.default.nationalcharset", charset);
-        newProperties.put("saffron.default.collation.name", charset + "$en_US");
-        return newProperties;
+      Charset getDefaultCharset() {
+        return Charset.forName(charset);
       }
     };
 
@@ -70,8 +66,6 @@ public class TestUtf8SupportInQueryString extends BaseTestQuery {
       assertThat(e.getMessage(), containsString(
           String.format("Failed to encode '%s' in character set '%s'", hello, charset)));
       throw e;
-    } finally {
-      Deencapsulation.setField(SaffronProperties.class, "properties", null);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/exec/java-exec/src/test/java/org/apache/drill/exec/work/prepare/TestPreparedStatementProvider.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/work/prepare/TestPreparedStatementProvider.java b/exec/java-exec/src/test/java/org/apache/drill/exec/work/prepare/TestPreparedStatementProvider.java
index ca47a02..9745297 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/work/prepare/TestPreparedStatementProvider.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/work/prepare/TestPreparedStatementProvider.java
@@ -120,6 +120,9 @@ public class TestPreparedStatementProvider extends PreparedStatementTestBase {
    */
   @Test
   public void invalidQueryValidationError() throws Exception {
-    createPrepareStmt("SELECT * sdflkgdh", true, ErrorType.PARSE /** Drill returns incorrect error for parse error*/);
+    // CALCITE-1120 allows SELECT without from syntax.
+    // So with this change the query fails with VALIDATION error.
+    createPrepareStmt("SELECT * sdflkgdh", true,
+        ErrorType.VALIDATION /* Drill returns incorrect error for parse error*/);
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java
index 810ffef..0b33167 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java
@@ -1113,7 +1113,7 @@ class DrillMetaImpl extends MetaImpl {
 
   @Override
   public ExecuteResult prepareAndExecute(final StatementHandle handle, final String sql, final long maxRowCount,
-                                         int maxRowsInFirstFrame, final PrepareCallback callback) throws NoSuchStatementException {
+        int maxRowsInFirstFrame, final PrepareCallback callback) throws NoSuchStatementException {
     return prepareAndExecute(handle, sql, maxRowCount, callback);
   }
 
@@ -1133,13 +1133,17 @@ class DrillMetaImpl extends MetaImpl {
   }
 
   @Override
-  public ExecuteResult execute(StatementHandle statementHandle, List<TypedValue> list, long l) throws NoSuchStatementException {
-    throw new UnsupportedOperationException(this.getClass().getSimpleName());
+  public ExecuteResult execute(StatementHandle statementHandle,
+        List<TypedValue> list, long l) throws NoSuchStatementException {
+    return new ExecuteResult(Collections.singletonList(
+        MetaResultSet.create(statementHandle.connectionId, statementHandle.id,
+            true, statementHandle.signature, null)));
   }
 
   @Override
-  public ExecuteResult execute(StatementHandle statementHandle, List<TypedValue> list, int i) throws NoSuchStatementException {
-    return null;
+  public ExecuteResult execute(StatementHandle statementHandle,
+      List<TypedValue> list, int i) throws NoSuchStatementException {
+    return execute(statementHandle, list, (long) i);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
----------------------------------------------------------------------
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 a45412f..a13f936 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
@@ -38,7 +38,7 @@ import org.apache.drill.jdbc.DrillPreparedStatement;
  * <p>
  * This class has sub-classes which implement JDBC 3.0 and JDBC 4.0 APIs; it is
  * instantiated using
- * {@link net.hydromatic.avatica.AvaticaFactory#newPreparedStatement}.
+ * {@link org.apache.calcite.avatica.AvaticaFactory#newPreparedStatement}.
  * </p>
  */
 abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
@@ -58,7 +58,9 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
           resultSetType, resultSetConcurrency, resultSetHoldability);
     connection.openStatementsRegistry.addStatement(this);
     this.preparedStatementHandle = preparedStatementHandle;
-    ((DrillColumnMetaDataList) signature.columns).updateColumnMetaData(preparedStatementHandle.getColumnsList());
+    if (preparedStatementHandle != null) {
+      ((DrillColumnMetaDataList) signature.columns).updateColumnMetaData(preparedStatementHandle.getColumnsList());
+    }
   }
 
   /**
@@ -329,7 +331,7 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
   }
 
   @Override
-  public void clearBatch() throws RuntimeException {
+  public void clearBatch() {
     try {
       throwIfClosed();
     } catch (AlreadyClosedSqlException e) {

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2489CallsAfterCloseThrowExceptionsTest.java
----------------------------------------------------------------------
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 26ab7bd..8521586 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
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -421,19 +421,15 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTestBase {
       }
       else if (SQLClientInfoException.class == cause.getClass()
                 && normalClosedExceptionText.equals(cause.getMessage())
-                && (false
-                    || method.getName().equals("setClientInfo")
-                    || method.getName().equals("getClientInfo")
-                    )) {
+                && (method.getName().equals("setClientInfo")
+                    || 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())
-               && (false
-                   || method.getName().equals("getCatalog")
-                   || method.getName().equals("getSchema")
-                   )) {
+               && (method.getName().equals("getCatalog")
+                  || method.getName().equals("getSchema"))) {
         // Special good-enough case--we had to use RuntimeException for now.
         result = true;
       }
@@ -481,20 +477,18 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTestBase {
       if (super.isOkaySpecialCaseException(method, cause)) {
         result = true;
       }
-      else if (   method.getName().equals("executeLargeBatch")
+      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())
-               && (false
-                   || method.getName().equals("getConnection")
+               && (method.getName().equals("getConnection")
                    || method.getName().equals("getFetchDirection")
                    || method.getName().equals("getFetchSize")
                    || method.getName().equals("getMaxRows")
-                   || method.getName().equals("getLargeMaxRows") // TODO: Java 8
-                   )) {
+                   || method.getName().equals("getLargeMaxRows"))) {
         // Special good-enough case--we had to use RuntimeException for now.
         result = true;
       }
@@ -544,27 +538,20 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTestBase {
         result = true;
       }
       else if (RuntimeException.class == cause.getClass()
-               && normalClosedExceptionText.equals(cause.getMessage())
-               && (false
-                   || method.getName().equals("getConnection")
+               && 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 if (  method.getName().equals("setObject")
-              || method.getName().equals("executeLargeUpdate")
-              || method.getName().equals("executeLargeBatch")
-              || method.getName().equals("getLargeMaxRows")
-              ) {
-        // TODO: Java 8 methods not yet supported by Avatica.
-        result = true;
-      }
-      else {
-        result = false;
+      } else {
+        result = method.getName().equals("setObject")
+          || method.getName().equals("executeLargeUpdate")
+          || method.getName().equals("executeLargeBatch")
+          || method.getName().equals("getLargeMaxRows");
       }
       return result;
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/3896a582/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2769UnsupportedReportsUseSqlExceptionTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2769UnsupportedReportsUseSqlExceptionTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2769UnsupportedReportsUseSqlExceptionTest.java
index 1a6ac40..9e2399b 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2769UnsupportedReportsUseSqlExceptionTest.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2769UnsupportedReportsUseSqlExceptionTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -96,13 +96,8 @@ public class Drill2769UnsupportedReportsUseSqlExceptionTest extends JdbcTestBase
     catch (SQLException | UnsupportedOperationException e) {
       // Expected.
     }
-    try {
-      connection.createArrayOf("INTEGER", new Object[0]);
-      fail("Test seems to be out of date.  Were arrays implemented?");
-    }
-    catch (SQLException | UnsupportedOperationException e) {
-      // Expected.
-    }
+
+    connection.createArrayOf("INTEGER", new Object[0]);
 
     resultSet = plainStatement.executeQuery("VALUES 'plain Statement query'");
     resultSet.next();
@@ -161,7 +156,9 @@ public class Drill2769UnsupportedReportsUseSqlExceptionTest extends JdbcTestBase
      */
     private static Object getDummyValueForType(Class<?> type) {
       final Object result;
-      if (! type.isPrimitive()) {
+      if (type.equals(String.class)) {
+        result = "";
+      } else if (! type.isPrimitive()) {
         result = null;
       }
       else {