You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by vo...@apache.org on 2018/11/26 16:03:54 UTC

[drill] branch master updated (6ecaed7 -> cd4d68b)

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

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


    from 6ecaed7  DRILL-6861: Hash-Join should not exit after an empty probe-side spilled partition
     new 639be6c  DRILL-6349: Drill JDBC driver fails on Java 1.9+ with NoClassDefFoundError: sun/misc/VM
     new 5a8e6d7  DRILL-6668: In Web UI, highlight options that are not default values
     new 9174dfa  DRILL-6691: Unify checkstyle-config.xml files.
     new 597827e  DRILL-6818: Add descriptions to secondary index options.
     new 6a990c7  DRILL-6847: Add Query Metadata to RESTful Interface
     new 1c08723  DRILL-6850: JDBC integration tests failures
     new b67f77a  DRILL-6850: Force setting DRILL_LOGICAL Convention for DrillRelFactories and DrillFilterRel
     new 13ba486  DRILL-6850: Allow configuring table names case sensitivity for JDBC storage plugin
     new 8bd8192  DRILL-6857: Read only required row groups in a file when limit push down is applied
     new d0ba8ec  DRILL-6858: Add functionality to list directories / files with exceptions suppression
     new 67adde1  DRILL-6868: Upgrade Janino compiler to 3.0.11
     new 44b990b  DRILL-6870: Upgrade to ANTLR4
     new d1a082c  DRILL-6865: Query returns wrong result when filter pruning happens
     new 99a3d76  DRILL-6865: Filter is not removed from the plan when parquet table fully matches the filter
     new cd4d68b  DRILL-6834: Introduce option to disable result set for DDL queries for JDBC connection

The 15 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:
 common/pom.xml                                     |   6 -
 .../apache/drill/common/config/DrillConfig.java    |   3 +-
 .../exec/planner/index/MapRDBIndexDescriptor.java  |   6 +-
 .../exec/planner/index/MapRDBIndexDiscover.java    |  34 +-
 .../drill/exec/planner/index/MapRDBStatistics.java |  12 +-
 .../drill/exec/store/mapr/TableFormatMatcher.java  |   2 +-
 .../drill/exec/store/mapr/TableFormatPlugin.java   |   2 -
 .../store/mapr/db/MapRDBPushFilterIntoScan.java    |   2 +-
 .../drill/exec/store/mapr/db/MapRDBSubScan.java    |   1 -
 .../drill/exec/store/mapr/db/MapRDBTableCache.java |  12 +-
 .../store/mapr/db/RestrictedMapRDBSubScanSpec.java |   5 +-
 .../exec/store/mapr/db/TabletFragmentInfo.java     |  21 +-
 .../store/mapr/db/binary/MapRDBFilterBuilder.java  |   2 +-
 .../store/mapr/db/json/JsonConditionBuilder.java   |   1 -
 .../store/mapr/db/json/JsonTableGroupScan.java     |   2 -
 .../db/json/JsonTableRangePartitionFunction.java   |   6 +-
 .../store/mapr/db/json/MaprDBJsonRecordReader.java |   2 -
 .../store/mapr/db/json/OjaiFunctionsProcessor.java |   3 +-
 .../exec/store/mapr/db/json/OjaiValueWriter.java   |   4 +-
 .../db/json/ProjectionPassthroughVectorWriter.java |   1 -
 .../src/main/resources/checkstyle-config.xml       |  40 ---
 .../src/main/resources/checkstyle-suppressions.xml |  28 --
 .../tests/binary/TestMapRDBCFAsJSONString.java     |   1 -
 .../tests/binary/TestMapRDBProjectPushDown.java    |   1 -
 .../maprdb/tests/binary/TestMapRDBQueries.java     |   1 -
 .../maprdb/tests/index/IndexHintPlanTest.java      |   1 -
 .../drill/maprdb/tests/index/IndexPlanTest.java    |   2 +-
 .../drill/maprdb/tests/index/LargeTableGen.java    |   2 +-
 .../drill/maprdb/tests/index/StatisticsTest.java   |   8 -
 .../mapr/drill/maprdb/tests/json/BaseJsonTest.java |   1 -
 .../maprdb/tests/json/TestEncodedFieldPaths.java   |   2 +-
 .../drill/maprdb/tests/json/TestScanRanges.java    |   1 -
 .../native/client/src/protobuf/UserBitShared.pb.cc | 310 ++++++++++------
 .../native/client/src/protobuf/UserBitShared.pb.h  |  68 +++-
 contrib/pom.xml                                    |   4 +
 .../exec/store/hive/schema/HiveSchemaFactory.java  |  21 +-
 contrib/storage-jdbc/pom.xml                       |  24 +-
 .../drill/exec/store/jdbc/JdbcStorageConfig.java   |  16 +-
 .../drill/exec/store/jdbc/JdbcStoragePlugin.java   |  50 ++-
 .../exec/store/jdbc/TestJdbcPluginWithDerbyIT.java | 131 ++++---
 .../exec/store/jdbc/TestJdbcPluginWithMySQLIT.java | 167 +++++----
 .../test/resources/bootstrap-storage-plugins.json  |  11 +-
 .../src/test/resources/mysql-test-data.sql         |  10 +-
 distribution/src/resources/drill-config.sh         |  11 +
 distribution/src/resources/sqlline.bat             |  12 +
 exec/java-exec/pom.xml                             |  20 +-
 .../java/org/apache/drill/exec/ExecConstants.java  |  10 +-
 .../drill/exec/compile/CompilationConfig.java      |   2 +-
 .../exec/compile/bytecode/AloadPopRemover.java     | 333 ------------------
 .../compile/bytecode/ReplacingInterpreter.java     |   2 +
 .../bytecode/ValueHolderReplacementVisitor.java    |  10 +-
 .../org/apache/drill/exec/expr/ClassGenerator.java |   4 +-
 .../drill/exec/expr/fn/MethodGrabbingVisitor.java  |  57 ++-
 .../drill/exec/expr/fn/ModifiedUnparser.java       | 110 ------
 .../impl/materialize/VectorRecordMaterializer.java |  27 +-
 .../planner/FileSystemPartitionDescriptor.java     |   2 +-
 .../drill/exec/planner/logical/DrillFilterRel.java |   2 +-
 .../drill/exec/planner/logical/DrillRel.java       |   2 +-
 .../exec/planner/logical/DrillRelFactories.java    |   9 +-
 .../exec/planner/physical/PlannerSettings.java     |  39 ++-
 .../drill/exec/planner/sql/DrillSqlWorker.java     |  25 +-
 .../sql/handlers/DescribeSchemaHandler.java        |   6 +-
 .../sql/handlers/FindHardDistributionScans.java    |  10 +-
 .../planner/sql/handlers/ShowFilesHandler.java     |   5 +-
 .../exec/planner/sql/parser/SqlCreateFunction.java |   2 +-
 .../exec/planner/sql/parser/SqlCreateTable.java    |   2 +-
 .../exec/planner/sql/parser/SqlCreateView.java     |   2 +-
 .../exec/planner/sql/parser/SqlDropFunction.java   |   2 +-
 .../exec/planner/sql/parser/SqlDropTable.java      |   2 +-
 .../drill/exec/planner/sql/parser/SqlDropView.java |   2 +-
 .../planner/sql/parser/SqlRefreshMetadata.java     |   2 +-
 .../exec/planner/sql/parser/SqlUseSchema.java      |   3 +-
 .../drill/exec/server/options/OptionList.java      |   8 +
 .../exec/server/options/QueryOptionManager.java    |  14 +-
 .../exec/server/options/SystemOptionManager.java   |   3 +-
 .../drill/exec/server/rest/QueryWrapper.java       |  17 +-
 .../drill/exec/server/rest/StatusResources.java    |  11 +-
 .../drill/exec/server/rest/WebUserConnection.java  |  28 +-
 .../org/apache/drill/exec/store/SchemaFactory.java |   2 +
 .../drill/exec/store/dfs/FileSystemPlugin.java     |   2 -
 .../exec/store/dfs/FileSystemSchemaFactory.java    |   2 -
 .../store/ischema/InfoSchemaRecordGenerator.java   |   5 +-
 .../store/parquet/AbstractParquetGroupScan.java    | 207 ++++++-----
 .../exec/store/parquet/ParquetFilterBuilder.java   |  21 +-
 .../exec/store/parquet/ParquetPushDownFilter.java  |  78 +++--
 .../store/parquet/ParquetRGFilterEvaluator.java    |   4 +-
 .../drill/exec/store/parquet/RowGroupInfo.java     |   6 -
 .../exec/store/sys/ExtendedOptionIterator.java     |  17 +-
 .../drill/exec/util/DrillFileSystemUtil.java       |  57 ++-
 .../org/apache/drill/exec/util/FileSystemUtil.java | 159 ++++++---
 .../drill/exec/util/StoragePluginTestUtils.java    |   4 +-
 .../java-exec/src/main/resources/drill-module.conf |   2 +-
 exec/java-exec/src/main/resources/rest/options.ftl |   8 +
 .../apache/drill/TestUtf8SupportInQueryString.java |   3 -
 .../test/java/org/apache/drill/exec/ExecTest.java  |  27 +-
 .../drill/exec/compile/TestEvaluationVisitor.java  |  50 +--
 .../drill/exec/fn/impl/TestCastFunctions.java      |   4 -
 .../drill/exec/fn/impl/TestDateFunctions.java      |   3 -
 .../exec/fn/impl/testing/TestDateConversions.java  |   3 -
 .../drill/exec/fn/interp/TestConstantFolding.java  |   3 -
 .../impersonation/TestImpersonationMetadata.java   |  17 +-
 .../unnest/TestUnnestWithLateralCorrectness.java   |   8 +-
 ...alTestutils.java => DrillLogicalTestUtils.java} |  28 +-
 .../drill/exec/store/avro/AvroFormatTest.java      |   4 -
 .../store/parquet/TestParquetFilterPushDown.java   |  14 +
 .../store/parquet/TestParquetLimitPushDown.java    |  75 ++++
 .../drill/exec/util/DrillFileSystemUtilTest.java   |  70 ++--
 .../apache/drill/exec/util/FileSystemUtilTest.java | 122 +++----
 .../drill/exec/util/FileSystemUtilTestBase.java    |   7 +-
 .../vector/complex/writer/TestExtendedTypes.java   |   3 -
 .../java/org/apache/drill/test/BaseTestQuery.java  |   4 +-
 .../java/org/apache/drill/test/ClusterFixture.java |   4 +-
 .../apache/drill/test/PhysicalOpUnitTestBase.java  |  14 +-
 .../java/org/apache/drill/test/TestBuilder.java    |  46 +--
 .../java-exec/src/test/resources/drill-udf/pom.xml |   4 +-
 exec/jdbc-all/pom.xml                              |   5 +-
 .../org/apache/drill/jdbc/ITTestShadedJar.java     |  14 +-
 .../java/org/apache/drill/jdbc/DrillStatement.java |   5 +
 .../drill/jdbc/impl/DrillConnectionImpl.java       |   4 +-
 .../org/apache/drill/jdbc/impl/DrillCursor.java    |  14 +-
 .../apache/drill/jdbc/impl/DrillStatementImpl.java |  11 +
 ...rill2489CallsAfterCloseThrowExceptionsTest.java |  56 ++-
 .../org/apache/drill/jdbc/test/TestJdbcQuery.java  | 108 ++++++
 .../drill/exec/memory/BoundsCheckingTest.java      |   1 +
 logical/pom.xml                                    |  13 +-
 .../drill/common/expression/parser/ExprLexer.g4}   |  46 +--
 .../drill/common/expression/parser/ExprParser.g4}  |  81 +++--
 .../drill/common/expression/LogicalExpression.java |  23 +-
 .../drill/common/expression/PathSegment.java       |   8 +-
 .../apache/drill/common/expression/SchemaPath.java |  30 +-
 .../apache/drill/common/parser/ErrorListener.java  |  32 +-
 .../common/parser/LogicalExpressionParser.java     |  55 +++
 .../drill/common/expression/parser/TreeTest.java   | 106 +++---
 pom.xml                                            |  37 +-
 .../drill/exec/proto/SchemaUserBitShared.java      |  14 +
 .../org/apache/drill/exec/proto/UserBitShared.java | 390 +++++++++++++++------
 .../apache/drill/exec/proto/beans/QueryData.java   |  22 ++
 .../drill/exec/proto/beans/RecordBatchDef.java     |  22 ++
 protocol/src/main/protobuf/UserBitShared.proto     |   4 +
 139 files changed, 2230 insertions(+), 1781 deletions(-)
 delete mode 100644 contrib/format-maprdb/src/main/resources/checkstyle-config.xml
 delete mode 100644 contrib/format-maprdb/src/main/resources/checkstyle-suppressions.xml
 delete mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/AloadPopRemover.java
 delete mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparser.java
 rename exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/{DrillLogicalTestutils.java => DrillLogicalTestUtils.java} (65%)
 create mode 100644 exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetLimitPushDown.java
 rename logical/src/main/{antlr3/org/apache/drill/common/expression/parser/ExprLexer.g => antlr4/org/apache/drill/common/expression/parser/ExprLexer.g4} (79%)
 rename logical/src/main/{antlr3/org/apache/drill/common/expression/parser/ExprParser.g => antlr4/org/apache/drill/common/expression/parser/ExprParser.g4} (85%)
 copy exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/GenericExceptionMapper.java => logical/src/main/java/org/apache/drill/common/parser/ErrorListener.java (56%)
 create mode 100644 logical/src/main/java/org/apache/drill/common/parser/LogicalExpressionParser.java


[drill] 05/15: DRILL-6847: Add Query Metadata to RESTful Interface

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

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

commit 6a990c7eb928b00311935e107427c8420258fd6c
Author: Charles S. Givre <cg...@gmail.com>
AuthorDate: Wed Nov 21 08:15:00 2018 -0500

    DRILL-6847: Add Query Metadata to RESTful Interface
    
    closes #1539
---
 .../drill/exec/server/rest/QueryWrapper.java       | 17 +++++++------
 .../drill/exec/server/rest/WebUserConnection.java  | 28 +++++++++++++++++++++-
 2 files changed, 37 insertions(+), 8 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java
index 4e09858..4eb1656 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/QueryWrapper.java
@@ -115,7 +115,7 @@ public class QueryWrapper {
     }
 
     // Return the QueryResult.
-    return new QueryResult(queryId, webUserConnection.columns, webUserConnection.results);
+    return new QueryResult(queryId, webUserConnection, webUserConnection.results);
   }
 
   //Detect possible excess heap
@@ -127,12 +127,15 @@ public class QueryWrapper {
     private final String queryId;
     public final Collection<String> columns;
     public final List<Map<String, String>> rows;
-
-    public QueryResult(QueryId queryId, Collection<String> columns, List<Map<String, String>> rows) {
-      this.queryId = QueryIdHelper.getQueryId(queryId);
-      this.columns = columns;
-      this.rows = rows;
-    }
+    public final List<String> metadata;
+
+    //DRILL-6847:  Modified the constructor so that the method has access to all the properties in webUserConnection
+    public QueryResult(QueryId queryId, WebUserConnection webUserConnection, List<Map<String, String>> rows) {
+        this.queryId = QueryIdHelper.getQueryId(queryId);
+        this.columns = webUserConnection.columns;
+        this.metadata = webUserConnection.metadata;
+        this.rows = rows;
+      }
 
     public String getQueryId() {
       return queryId;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebUserConnection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebUserConnection.java
index 9d6e7e4..b70dc7b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebUserConnection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/WebUserConnection.java
@@ -37,10 +37,12 @@ import org.apache.drill.exec.rpc.ConnectionThrottle;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.vector.ValueVector.Accessor;
+import org.apache.drill.exec.record.MaterializedField;
 
 import java.net.SocketAddress;
 import java.util.List;
 import java.util.Map;
+import java.util.ArrayList;
 import java.util.Set;
 
 /**
@@ -64,6 +66,8 @@ public class WebUserConnection extends AbstractDisposableUserClientConnection im
 
   public final Set<String> columns = Sets.newLinkedHashSet();
 
+  public final List<String> metadata = new ArrayList<>();
+
   WebUserConnection(WebSessionResources webSessionResources) {
     this.webSessionResources = webSessionResources;
   }
@@ -106,7 +110,29 @@ public class WebUserConnection extends AbstractDisposableUserClientConnection im
         // TODO:  Clean:  DRILL-2933:  That load(...) no longer throws
         // SchemaChangeException, so check/clean catch clause below.
         for (int i = 0; i < loader.getSchema().getFieldCount(); ++i) {
-          columns.add(loader.getSchema().getColumn(i).getName());
+          //DRILL-6847:  This section adds query metadata to the REST results
+          MaterializedField col = loader.getSchema().getColumn(i);
+          columns.add(col.getName());
+          StringBuilder dataType = new StringBuilder(col.getType().getMinorType().name());
+
+          //For DECIMAL type
+          if (col.getType().hasPrecision()) {
+            dataType.append("(");
+            dataType.append(col.getType().getPrecision());
+
+            if (col.getType().hasScale()) {
+              dataType.append(", ");
+              dataType.append(col.getType().getScale());
+            }
+
+            dataType.append(")");
+          } else if (col.getType().hasWidth()) {
+            //Case for VARCHAR columns with specified width
+            dataType.append("(");
+            dataType.append(col.getType().getWidth());
+            dataType.append(")");
+          }
+          metadata.add(dataType.toString());
         }
         ValueVectorElementFormatter formatter = new ValueVectorElementFormatter(webSessionResources.getSession().getOptions());
         for (int i = 0; i < rows; ++i) {


[drill] 13/15: DRILL-6865: Query returns wrong result when filter pruning happens

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

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

commit d1a082cd11c79497449fda06189cd00d3510b2e9
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Wed Nov 21 14:08:17 2018 +0200

    DRILL-6865: Query returns wrong result when filter pruning happens
---
 .../store/parquet/AbstractParquetGroupScan.java    | 76 +++++++++++++++-------
 .../exec/store/parquet/ParquetFilterBuilder.java   | 21 ++++--
 .../exec/store/parquet/ParquetPushDownFilter.java  | 41 ++++++++++--
 .../store/parquet/ParquetRGFilterEvaluator.java    |  4 +-
 .../store/parquet/TestParquetFilterPushDown.java   |  5 ++
 5 files changed, 112 insertions(+), 35 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
index 0d35ddb..1bbf63b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
@@ -247,7 +247,11 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
 
     final List<RowGroupInfo> qualifiedRGs = new ArrayList<>(rowGroupInfos.size());
 
-    ParquetFilterPredicate filterPredicate = null;
+    ParquetFilterPredicate filterPredicate = getParquetFilterPredicate(filterExpr, udfUtilities, functionImplementationRegistry, optionManager, true);
+
+    if (filterPredicate == null) {
+      return null;
+    }
 
     for (RowGroupInfo rowGroup : rowGroupInfos) {
       final ColumnExplorer columnExplorer = new ColumnExplorer(optionManager, columns);
@@ -261,27 +265,8 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
 
       Map<SchemaPath, ColumnStatistics> columnStatisticsMap = statCollector.collectColStat(schemaPathsInExpr);
 
-      if (filterPredicate == null) {
-        ErrorCollector errorCollector = new ErrorCollectorImpl();
-        LogicalExpression materializedFilter = ExpressionTreeMaterializer.materializeFilterExpr(
-            filterExpr, columnStatisticsMap, errorCollector, functionImplementationRegistry);
-
-        if (errorCollector.hasErrors()) {
-          logger.error("{} error(s) encountered when materialize filter expression : {}",
-              errorCollector.getErrorCount(), errorCollector.toErrorString());
-          return null;
-        }
-        logger.debug("materializedFilter : {}", ExpressionStringBuilder.toString(materializedFilter));
-
-        Set<LogicalExpression> constantBoundaries = ConstantExpressionIdentifier.getConstantExpressionSet(materializedFilter);
-        filterPredicate = ParquetFilterBuilder.buildParquetFilterPredicate(materializedFilter, constantBoundaries, udfUtilities);
-
-        if (filterPredicate == null) {
-          return null;
-        }
-      }
-
-      ParquetFilterPredicate.RowsMatch match = ParquetRGFilterEvaluator.matches(filterPredicate, columnStatisticsMap, rowGroup.getRowCount(), parquetTableMetadata, rowGroup.getColumns(), schemaPathsInExpr);
+      ParquetFilterPredicate.RowsMatch match = ParquetRGFilterEvaluator.matches(filterPredicate,
+          columnStatisticsMap, rowGroup.getRowCount(), parquetTableMetadata, rowGroup.getColumns(), schemaPathsInExpr);
       if (match == ParquetFilterPredicate.RowsMatch.NONE) {
         continue; // No row comply to the filter => drop the row group
       }
@@ -310,6 +295,53 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
       return null;
     }
   }
+
+  /**
+   * Returns parquet filter predicate built from specified {@code filterExpr}.
+   *
+   * @param filterExpr                     filter expression to build
+   * @param udfUtilities                   udf utilities
+   * @param functionImplementationRegistry context to find drill function holder
+   * @param optionManager                  option manager
+   * @param omitUnsupportedExprs           whether expressions which cannot be converted
+   *                                       may be omitted from the resulting expression
+   * @return parquet filter predicate
+   */
+  public ParquetFilterPredicate getParquetFilterPredicate(LogicalExpression filterExpr,
+      UdfUtilities udfUtilities, FunctionImplementationRegistry functionImplementationRegistry,
+      OptionManager optionManager, boolean omitUnsupportedExprs) {
+    // used first row group to receive fields list
+    assert rowGroupInfos.size() > 0 : "row groups count cannot be 0";
+    RowGroupInfo rowGroup = rowGroupInfos.iterator().next();
+    ColumnExplorer columnExplorer = new ColumnExplorer(optionManager, columns);
+
+    Map<String, String> implicitColValues = columnExplorer.populateImplicitColumns(
+        rowGroup.getPath(),
+        getPartitionValues(rowGroup),
+        supportsFileImplicitColumns());
+
+    ParquetMetaStatCollector statCollector = new ParquetMetaStatCollector(
+        parquetTableMetadata,
+        rowGroup.getColumns(),
+        implicitColValues);
+
+    Set<SchemaPath> schemaPathsInExpr = filterExpr.accept(new ParquetRGFilterEvaluator.FieldReferenceFinder(), null);
+    Map<SchemaPath, ColumnStatistics> columnStatisticsMap = statCollector.collectColStat(schemaPathsInExpr);
+
+    ErrorCollector errorCollector = new ErrorCollectorImpl();
+    LogicalExpression materializedFilter = ExpressionTreeMaterializer.materializeFilterExpr(
+        filterExpr, columnStatisticsMap, errorCollector, functionImplementationRegistry);
+
+    if (errorCollector.hasErrors()) {
+      logger.error("{} error(s) encountered when materialize filter expression : {}",
+          errorCollector.getErrorCount(), errorCollector.toErrorString());
+      return null;
+    }
+    logger.debug("materializedFilter : {}", ExpressionStringBuilder.toString(materializedFilter));
+
+    Set<LogicalExpression> constantBoundaries = ConstantExpressionIdentifier.getConstantExpressionSet(materializedFilter);
+    return ParquetFilterBuilder.buildParquetFilterPredicate(materializedFilter, constantBoundaries, udfUtilities, omitUnsupportedExprs);
+  }
   // filter push down methods block end
 
   // limit push down methods start
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
index f0f1029..86e207f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFilterBuilder.java
@@ -63,6 +63,12 @@ public class ParquetFilterBuilder extends AbstractExprVisitor<LogicalExpression,
   static final Logger logger = LoggerFactory.getLogger(ParquetFilterBuilder.class);
 
   private final UdfUtilities udfUtilities;
+  // Flag to check whether predicate cannot be fully converted
+  // to parquet filter predicate without omitting its parts.
+  // It should be set to false for the case when we want to
+  // verify that predicate is fully convertible to parquet filter predicate,
+  // otherwise null is returned instead of the converted expression.
+  private final boolean omitUnsupportedExprs;
 
   /**
    * @param expr materialized filter expression
@@ -71,18 +77,24 @@ public class ParquetFilterBuilder extends AbstractExprVisitor<LogicalExpression,
    *
    * @return parquet filter predicate
    */
-  public static ParquetFilterPredicate buildParquetFilterPredicate(LogicalExpression expr, final Set<LogicalExpression> constantBoundaries, UdfUtilities udfUtilities) {
-    LogicalExpression logicalExpression = expr.accept(new ParquetFilterBuilder(udfUtilities), constantBoundaries);
+  public static ParquetFilterPredicate buildParquetFilterPredicate(LogicalExpression expr,
+      Set<LogicalExpression> constantBoundaries, UdfUtilities udfUtilities, boolean omitUnsupportedExprs) {
+    LogicalExpression logicalExpression =
+        expr.accept(new ParquetFilterBuilder(udfUtilities, omitUnsupportedExprs), constantBoundaries);
     if (logicalExpression instanceof ParquetFilterPredicate) {
       return (ParquetFilterPredicate) logicalExpression;
+    } else if (logicalExpression instanceof TypedFieldExpr) {
+      // Calcite simplifies `= true` expression to field name, wrap it with is true predicate
+      return (ParquetFilterPredicate) ParquetIsPredicate.createIsPredicate(FunctionGenerationHelper.IS_TRUE, logicalExpression);
     }
     logger.debug("Logical expression {} was not qualified for filter push down", logicalExpression);
     return null;
   }
 
 
-  private ParquetFilterBuilder(UdfUtilities udfUtilities) {
+  private ParquetFilterBuilder(UdfUtilities udfUtilities, boolean omitUnsupportedExprs) {
     this.udfUtilities = udfUtilities;
+    this.omitUnsupportedExprs = omitUnsupportedExprs;
   }
 
   @Override
@@ -159,8 +171,9 @@ public class ParquetFilterBuilder extends AbstractExprVisitor<LogicalExpression,
     for (LogicalExpression arg : op.args) {
       LogicalExpression childPredicate = arg.accept(this, value);
       if (childPredicate == null) {
-        if (functionName.equals("booleanOr")) {
+        if (functionName.equals("booleanOr") || !omitUnsupportedExprs) {
           // we can't include any leg of the OR if any of the predicates cannot be converted
+          // or prohibited omitting of unconverted operands
           return null;
         }
       } else {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
index c59cdce..95a0534 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
@@ -134,13 +134,32 @@ public abstract class ParquetPushDownFilter extends StoragePluginOptimizerRule {
 
     // get a conjunctions of the filter condition. For each conjunction, if it refers to ITEM or FLATTEN expression
     // then we could not pushed down. Otherwise, it's qualified to be pushed down.
-    final List<RexNode> predList = RelOptUtil.conjunctions(condition);
+    final List<RexNode> predList = RelOptUtil.conjunctions(RexUtil.toCnf(filter.getCluster().getRexBuilder(), condition));
 
     final List<RexNode> qualifiedPredList = new ArrayList<>();
 
-    for (final RexNode pred : predList) {
+    // list of predicates which cannot be converted to parquet filter predicate
+    List<RexNode> nonConvertedPredList = new ArrayList<>();
+
+    for (RexNode pred : predList) {
       if (DrillRelOptUtil.findOperators(pred, Collections.emptyList(), BANNED_OPERATORS) == null) {
+        LogicalExpression drillPredicate = DrillOptiq.toDrill(
+            new DrillParseContext(PrelUtil.getPlannerSettings(call.getPlanner())), scan, pred);
+
+        // checks whether predicate may be used for filter pushdown
+        ParquetFilterPredicate parquetFilterPredicate =
+            groupScan.getParquetFilterPredicate(drillPredicate,
+                optimizerContext,
+                optimizerContext.getFunctionRegistry(),
+                optimizerContext.getPlannerSettings().getOptions(), false);
+        // collects predicates that contain unsupported for filter pushdown expressions
+        // to build filter with them
+        if (parquetFilterPredicate == null) {
+          nonConvertedPredList.add(pred);
+        }
         qualifiedPredList.add(pred);
+      } else {
+        nonConvertedPredList.add(pred);
       }
     }
 
@@ -155,7 +174,7 @@ public abstract class ParquetPushDownFilter extends StoragePluginOptimizerRule {
 
 
     Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null;
-    final GroupScan newGroupScan = groupScan.applyFilter(conditionExp,optimizerContext,
+    final GroupScan newGroupScan = groupScan.applyFilter(conditionExp, optimizerContext,
         optimizerContext.getFunctionRegistry(), optimizerContext.getPlannerSettings().getOptions());
     if (timer != null) {
       logger.debug("Took {} ms to apply filter on parquet row groups. ", timer.elapsed(TimeUnit.MILLISECONDS));
@@ -166,10 +185,10 @@ public abstract class ParquetPushDownFilter extends StoragePluginOptimizerRule {
       return;
     }
 
-    RelNode newScan = new ScanPrel(scan.getCluster(), scan.getTraitSet(), newGroupScan, scan.getRowType(), scan.getTable());
+    RelNode newNode = new ScanPrel(scan.getCluster(), scan.getTraitSet(), newGroupScan, scan.getRowType(), scan.getTable());
 
     if (project != null) {
-      newScan = project.copy(project.getTraitSet(), Collections.singletonList(newScan));
+      newNode = project.copy(project.getTraitSet(), Collections.singletonList(newNode));
     }
 
     if (newGroupScan instanceof AbstractParquetGroupScan) {
@@ -182,12 +201,20 @@ public abstract class ParquetPushDownFilter extends StoragePluginOptimizerRule {
         }
       }
       if (matchAll == ParquetFilterPredicate.RowsMatch.ALL) {
-        call.transformTo(newScan);
+        // creates filter from the expressions which can't be pushed to the scan
+        if (nonConvertedPredList.size() > 0) {
+          newNode = filter.copy(filter.getTraitSet(), newNode,
+              RexUtil.composeConjunction(
+                  filter.getCluster().getRexBuilder(),
+                  nonConvertedPredList,
+                  true));
+        }
+        call.transformTo(newNode);
         return;
       }
     }
 
-    final RelNode newFilter = filter.copy(filter.getTraitSet(), Collections.singletonList(newScan));
+    final RelNode newFilter = filter.copy(filter.getTraitSet(), Collections.singletonList(newNode));
     call.transformTo(newFilter);
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRGFilterEvaluator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRGFilterEvaluator.java
index 281e865..0125149 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRGFilterEvaluator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRGFilterEvaluator.java
@@ -87,8 +87,8 @@ public class ParquetRGFilterEvaluator {
     }
 
     Set<LogicalExpression> constantBoundaries = ConstantExpressionIdentifier.getConstantExpressionSet(materializedFilter);
-    ParquetFilterPredicate parquetPredicate = (ParquetFilterPredicate) ParquetFilterBuilder.buildParquetFilterPredicate(
-        materializedFilter, constantBoundaries, udfUtilities);
+    ParquetFilterPredicate parquetPredicate = ParquetFilterBuilder.buildParquetFilterPredicate(
+        materializedFilter, constantBoundaries, udfUtilities, true);
 
     return matches(parquetPredicate, columnStatisticsMap, rowCount);
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
index ea12f40..ccc1480 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
@@ -70,6 +70,7 @@ public class TestParquetFilterPushDown extends PlanTestBase {
 
   @AfterClass
   public static void teardown() throws IOException {
+    fragContext.close();
     fs.close();
   }
 
@@ -294,6 +295,10 @@ public class TestParquetFilterPushDown extends PlanTestBase {
 
     PlanTestBase.testPlanMatchingPatterns(sql + "a < 1 or a > 1", new String[]{"numRowGroups=3"}); // No filter pruning
     PlanTestBase.testPlanMatchingPatterns(sql + "a < 1 or a > 2", new String[]{"numRowGroups=2"}, new String[]{"Filter\\("}); //Filter pruning
+
+    // Partial filter pruning
+    testParquetFilterPruning(sql + "a >=1 and cast(a as varchar) like '%3%'", 1, 2, new String[]{">\\($1, 1\\)"});
+    testParquetFilterPruning(sql + "a >=1 and a/3>=1", 2, 2, new String[]{">\\($1, 1\\)"});
   }
 
   @Test


[drill] 03/15: DRILL-6691: Unify checkstyle-config.xml files.

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

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

commit 9174dfa30b281983c6fab2bc55b79011e8e7968d
Author: Hanumath Rao Maduri <ha...@gmail.com>
AuthorDate: Wed Oct 31 14:39:25 2018 -0700

    DRILL-6691: Unify checkstyle-config.xml files.
    
    closes #1550
---
 .../exec/planner/index/MapRDBIndexDescriptor.java  |  6 ++--
 .../exec/planner/index/MapRDBIndexDiscover.java    |  2 +-
 .../drill/exec/planner/index/MapRDBStatistics.java | 12 +++----
 .../drill/exec/store/mapr/TableFormatMatcher.java  |  2 +-
 .../drill/exec/store/mapr/TableFormatPlugin.java   |  2 --
 .../store/mapr/db/MapRDBPushFilterIntoScan.java    |  2 +-
 .../drill/exec/store/mapr/db/MapRDBSubScan.java    |  1 -
 .../drill/exec/store/mapr/db/MapRDBTableCache.java | 12 +++----
 .../store/mapr/db/RestrictedMapRDBSubScanSpec.java |  5 +--
 .../exec/store/mapr/db/TabletFragmentInfo.java     | 21 ++++++++----
 .../store/mapr/db/binary/MapRDBFilterBuilder.java  |  2 +-
 .../store/mapr/db/json/JsonConditionBuilder.java   |  1 -
 .../store/mapr/db/json/JsonTableGroupScan.java     |  2 --
 .../db/json/JsonTableRangePartitionFunction.java   |  6 ++--
 .../store/mapr/db/json/MaprDBJsonRecordReader.java |  2 --
 .../store/mapr/db/json/OjaiFunctionsProcessor.java |  3 +-
 .../exec/store/mapr/db/json/OjaiValueWriter.java   |  4 +--
 .../db/json/ProjectionPassthroughVectorWriter.java |  1 -
 .../src/main/resources/checkstyle-config.xml       | 40 ----------------------
 .../src/main/resources/checkstyle-suppressions.xml | 28 ---------------
 .../tests/binary/TestMapRDBCFAsJSONString.java     |  1 -
 .../tests/binary/TestMapRDBProjectPushDown.java    |  1 -
 .../maprdb/tests/binary/TestMapRDBQueries.java     |  1 -
 .../maprdb/tests/index/IndexHintPlanTest.java      |  1 -
 .../drill/maprdb/tests/index/IndexPlanTest.java    |  2 +-
 .../drill/maprdb/tests/index/LargeTableGen.java    |  2 +-
 .../drill/maprdb/tests/index/StatisticsTest.java   |  8 -----
 .../mapr/drill/maprdb/tests/json/BaseJsonTest.java |  1 -
 .../maprdb/tests/json/TestEncodedFieldPaths.java   |  2 +-
 .../drill/maprdb/tests/json/TestScanRanges.java    |  1 -
 30 files changed, 43 insertions(+), 131 deletions(-)

diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDescriptor.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDescriptor.java
index ae386ab..18a22f9 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDescriptor.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDescriptor.java
@@ -22,9 +22,6 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
-import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
-
 import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.rel.RelFieldCollation.NullDirection;
@@ -35,13 +32,14 @@ import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
 import org.apache.drill.exec.planner.cost.PluginCost;
-import org.apache.drill.exec.planner.index.IndexProperties;
 import org.apache.drill.exec.store.mapr.PluginConstants;
 import org.apache.drill.exec.util.EncodedSchemaPathSet;
 import org.apache.drill.common.expression.LogicalExpression;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
 
 public class MapRDBIndexDescriptor extends DrillIndexDescriptor {
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
index d949634..95b9813 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
@@ -18,7 +18,6 @@
 
 package org.apache.drill.exec.planner.index;
 
-import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
 import com.mapr.db.Admin;
 import com.mapr.db.MapRDB;
 import com.mapr.db.exceptions.DBException;
@@ -64,6 +63,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
 
 public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDiscover {
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBStatistics.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBStatistics.java
index 6fedaff..37ea8d5 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBStatistics.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBStatistics.java
@@ -17,11 +17,6 @@
  */
 package org.apache.drill.exec.planner.index;
 
-import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
-import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
-import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
-
-import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.metadata.RelMdUtil;
@@ -40,7 +35,6 @@ import org.apache.drill.common.expression.ExpressionStringBuilder;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.exec.physical.base.DbGroupScan;
 import org.apache.drill.exec.physical.base.GroupScan;
-import org.apache.drill.exec.planner.common.DrillRelOptUtil;
 import org.apache.drill.exec.planner.common.DrillScanRelBase;
 import org.apache.drill.exec.planner.logical.DrillOptiq;
 import org.apache.drill.exec.planner.logical.DrillParseContext;
@@ -61,6 +55,10 @@ import java.util.List;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
+import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
+import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
 
 public class MapRDBStatistics implements Statistics {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBStatistics.class);
@@ -835,7 +833,7 @@ public class MapRDBStatistics implements Statistics {
           startKey = prefix.getBytes(Charsets.UTF_8);
           stopKey = startKey.clone();
           boolean isMaxVal = true;
-          for (int i = stopKey.length - 1; i >= 0 ; --i) {
+          for (int i = stopKey.length - 1; i >= 0; --i) {
             int nextByteValue = (0xff & stopKey[i]) + 1;
             if (nextByteValue < 0xff) {
               stopKey[i] = (byte) nextByteValue;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatMatcher.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatMatcher.java
index f663fce..1fca75c 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatMatcher.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatMatcher.java
@@ -58,7 +58,7 @@ public abstract class TableFormatMatcher extends FormatMatcher {
 
   @Override
   public boolean isFileReadable(DrillFileSystem fs, FileStatus status) throws IOException {
-    return (status instanceof MapRFileStatus) 
+    return (status instanceof MapRFileStatus)
         && ((MapRFileStatus) status).isTable()
         && isSupportedTable((MapRFileStatus) status);
   }
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java
index c8dfe70..aeb117a 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/TableFormatPlugin.java
@@ -33,8 +33,6 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.StoragePluginOptimizerRule;
-import org.apache.drill.exec.store.dfs.FileSystemConfig;
-import org.apache.drill.exec.store.dfs.FileSystemPlugin;
 import org.apache.drill.exec.store.dfs.FormatPlugin;
 import org.apache.hadoop.conf.Configuration;
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java
index a0f5536..6bd3368 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBPushFilterIntoScan.java
@@ -190,7 +190,7 @@ public abstract class MapRDBPushFilterIntoScan extends StoragePluginOptimizerRul
     final ScanPrel newScanPrel = new ScanPrel(scan.getCluster(), filter.getTraitSet(), newGroupsScan, scan.getRowType(), scan.getTable());
 
     // Depending on whether is a project in the middle, assign either scan or copy of project to childRel.
-    final RelNode childRel = project == null ? newScanPrel : project.copy(project.getTraitSet(), ImmutableList.of((RelNode)newScanPrel));;
+    final RelNode childRel = project == null ? newScanPrel : project.copy(project.getTraitSet(), ImmutableList.of((RelNode)newScanPrel));
 
     if (maprdbFilterBuilder.isAllExpressionsConverted()) {
         /*
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java
index 159b850..220b90e 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBSubScan.java
@@ -27,7 +27,6 @@ import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.AbstractDbSubScan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
-import org.apache.drill.exec.physical.base.SubScan;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBTableCache.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBTableCache.java
index 924d9c0..df8d59d 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBTableCache.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/MapRDBTableCache.java
@@ -17,11 +17,6 @@
  */
 package org.apache.drill.exec.store.mapr.db;
 
-import org.apache.drill.shaded.guava.com.google.common.cache.CacheBuilder;
-import org.apache.drill.shaded.guava.com.google.common.cache.CacheLoader;
-import org.apache.drill.shaded.guava.com.google.common.cache.LoadingCache;
-import org.apache.drill.shaded.guava.com.google.common.cache.RemovalListener;
-import org.apache.drill.shaded.guava.com.google.common.cache.RemovalNotification;
 import com.mapr.db.Table;
 import com.mapr.db.impl.MapRDBImpl;
 import com.mapr.db.index.IndexDesc;
@@ -34,6 +29,11 @@ import org.apache.hadoop.security.UserGroupInformation;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.concurrent.TimeUnit;
+import org.apache.drill.shaded.guava.com.google.common.cache.CacheBuilder;
+import org.apache.drill.shaded.guava.com.google.common.cache.CacheLoader;
+import org.apache.drill.shaded.guava.com.google.common.cache.LoadingCache;
+import org.apache.drill.shaded.guava.com.google.common.cache.RemovalListener;
+import org.apache.drill.shaded.guava.com.google.common.cache.RemovalNotification;
 
 public class MapRDBTableCache {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapRDBFormatPlugin.class);
@@ -209,7 +209,7 @@ public class MapRDBTableCache {
     static boolean isEqual(Object a, Object b) {
       return a == b || a != null && a.equals(b);
     }
-    
+
     public boolean equals(Object obj) {
       if (obj == this) {
         return true;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScanSpec.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScanSpec.java
index 596699f..1faf555 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScanSpec.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/RestrictedMapRDBSubScanSpec.java
@@ -20,7 +20,6 @@ package org.apache.drill.exec.store.mapr.db;
 import com.mapr.db.impl.IdCodec;
 
 import org.apache.commons.lang3.tuple.Pair;
-import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.exec.physical.impl.join.RowKeyJoin;
 import org.apache.drill.exec.record.AbstractRecordBatch.BatchState;
 import org.apache.drill.exec.vector.ValueVector;
@@ -166,7 +165,9 @@ public class RestrictedMapRDBSubScanSpec extends MapRDBSubScanSpec {
   public ByteBuffer[] getRowKeyIdsToRead(int numRowKeysToRead) {
 
     int numKeys = hasRowKeys(numRowKeysToRead);
-    if (numKeys == 0) return null;
+    if (numKeys == 0) {
+      return null;
+    }
 
     int index = 0;
     final ByteBuffer[] rowKeyIds = new ByteBuffer[numKeys];
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/TabletFragmentInfo.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/TabletFragmentInfo.java
index 66fa232..0fb2e39 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/TabletFragmentInfo.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/TabletFragmentInfo.java
@@ -79,23 +79,30 @@ public class TabletFragmentInfo  implements Comparable<TabletFragmentInfo> {
 
   @Override
   public boolean equals(Object obj) {
-    if (this == obj)
+    if (this == obj) {
       return true;
-    if (obj == null)
+    }
+    if (obj == null) {
       return false;
-    if (getClass() != obj.getClass())
+    }
+    if (getClass() != obj.getClass()) {
       return false;
+    }
     TabletFragmentInfo other = (TabletFragmentInfo) obj;
     if (regionInfo == null) {
-      if (other.regionInfo != null)
+      if (other.regionInfo != null) {
         return false;
-    } else if (!regionInfo.equals(other.regionInfo))
+      }
+    } else if (!regionInfo.equals(other.regionInfo)) {
       return false;
+    }
     if (tabletInfoImpl == null) {
-      if (other.tabletInfoImpl != null)
+      if (other.tabletInfoImpl != null) {
         return false;
-    } else if (!tabletInfoImpl.equals(other.tabletInfoImpl))
+      }
+    } else if (!tabletInfoImpl.equals(other.tabletInfoImpl)) {
       return false;
+    }
     return true;
   }
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/MapRDBFilterBuilder.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/MapRDBFilterBuilder.java
index 348853c..cdef510 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/MapRDBFilterBuilder.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/binary/MapRDBFilterBuilder.java
@@ -297,7 +297,7 @@ public class MapRDBFilterBuilder extends AbstractExprVisitor<HBaseScanSpec, Void
             startRow = prefix.getBytes(Charsets.UTF_8);
             stopRow = startRow.clone();
             boolean isMaxVal = true;
-            for (int i = stopRow.length - 1; i >= 0 ; --i) {
+            for (int i = stopRow.length - 1; i >= 0; --i) {
               int nextByteValue = (0xff & stopRow[i]) + 1;
               if (nextByteValue < 0xff) {
                 stopRow[i] = (byte) nextByteValue;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonConditionBuilder.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonConditionBuilder.java
index ce4e677..92d40f7 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonConditionBuilder.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonConditionBuilder.java
@@ -27,7 +27,6 @@ import org.ojai.store.QueryCondition;
 import org.ojai.store.QueryCondition.Op;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
-import com.mapr.db.MapRDB;
 import com.mapr.db.impl.MapRDBImpl;
 
 public class JsonConditionBuilder extends AbstractExprVisitor<JsonScanSpec, Void, RuntimeException> implements DrillHBaseConstants {
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
index 647fe82..24f370a 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableGroupScan.java
@@ -55,7 +55,6 @@ import org.apache.drill.exec.planner.physical.PrelUtil;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.dfs.FileSystemConfig;
 import org.apache.drill.exec.store.dfs.FileSystemPlugin;
-import org.apache.drill.exec.store.dfs.FileSystemPlugin;
 import org.apache.drill.exec.store.mapr.PluginConstants;
 import org.apache.drill.exec.store.mapr.db.MapRDBFormatPlugin;
 import org.apache.drill.exec.store.mapr.db.MapRDBFormatPluginConfig;
@@ -73,7 +72,6 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
-import com.mapr.db.MapRDB;
 import com.mapr.db.MetaTable;
 import com.mapr.db.Table;
 import com.mapr.db.impl.TabletInfoImpl;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
index ca508ca..b6d7bfb 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/JsonTableRangePartitionFunction.java
@@ -102,7 +102,7 @@ public class JsonTableRangePartitionFunction extends AbstractRangePartitionFunct
 
     partitionKeyVector = v.getValueVector();
 
-    Preconditions.checkArgument(partitionKeyVector != null, "Found null partitionKeVector.") ;
+    Preconditions.checkArgument(partitionKeyVector != null, "Found null partitionKeVector.");
   }
 
   @Override
@@ -130,8 +130,8 @@ public class JsonTableRangePartitionFunction extends AbstractRangePartitionFunct
   @Override
   public int eval(int index, int numPartitions) {
 
-	  String key = partitionKeyVector.getAccessor().getObject(index).toString();
-	  byte[] encodedKey = IdCodec.encodeAsBytes(key);
+    String key = partitionKeyVector.getAccessor().getObject(index).toString();
+    byte[] encodedKey = IdCodec.encodeAsBytes(key);
 
     int tabletId = -1;
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
index 5b849ea..f13d64d 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/MaprDBJsonRecordReader.java
@@ -62,8 +62,6 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Iterables;
 import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
 import org.apache.drill.shaded.guava.com.google.common.base.Predicate;
 
-import com.mapr.db.MapRDB;
-import com.mapr.org.apache.hadoop.hbase.util.Bytes;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiFunctionsProcessor.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiFunctionsProcessor.java
index e88eb9e..d07c97b 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiFunctionsProcessor.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiFunctionsProcessor.java
@@ -26,11 +26,10 @@ import org.apache.drill.common.expression.ValueExpressions.IntExpression;
 import org.apache.drill.common.expression.ValueExpressions.LongExpression;
 import org.apache.drill.common.expression.ValueExpressions.QuotedString;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
-
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
 import org.ojai.Value;
 import org.ojai.store.QueryCondition;
 
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableMap;
 import com.mapr.db.impl.ConditionImpl;
 import com.mapr.db.impl.MapRDBImpl;
 
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiValueWriter.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiValueWriter.java
index 99b2157..57aad06 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiValueWriter.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/OjaiValueWriter.java
@@ -176,14 +176,14 @@ public class OjaiValueWriter {
     buffer.setBytes(0, strBytes);
     writer.varChar(fieldName).writeVarChar(0, strBytes.length, buffer);
   }
-  
+
   protected void writeBinary(MapWriter writer, String fieldName, ByteBuffer buf) {
     int bufLen = buf.remaining();
     buffer = buffer.reallocIfNeeded(bufLen);
     buffer.setBytes(0, buf, buf.position(), bufLen);
     writer.varBinary(fieldName).writeVarBinary(0, bufLen, buffer);
   }
-  
+
   protected void writeString(MapWriter writer, String fieldName, String value) {
     final byte[] strBytes = Bytes.toBytes(value);
     buffer = buffer.reallocIfNeeded(strBytes.length);
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/ProjectionPassthroughVectorWriter.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/ProjectionPassthroughVectorWriter.java
index 8c5ed9b..cafd5f7 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/ProjectionPassthroughVectorWriter.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/store/mapr/db/json/ProjectionPassthroughVectorWriter.java
@@ -31,7 +31,6 @@ import org.ojai.DocumentConstants;
 import org.ojai.DocumentReader.EventType;
 import org.ojai.util.DocumentReaderWithProjection;
 import org.ojai.util.FieldProjector;
-
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import com.mapr.db.DBConstants;
 import com.mapr.db.ojai.DBDocumentReaderBase;
diff --git a/contrib/format-maprdb/src/main/resources/checkstyle-config.xml b/contrib/format-maprdb/src/main/resources/checkstyle-config.xml
deleted file mode 100644
index a66ded2..0000000
--- a/contrib/format-maprdb/src/main/resources/checkstyle-config.xml
+++ /dev/null
@@ -1,40 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-
-    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.
-
--->
-<!DOCTYPE module PUBLIC
-    "-//Puppy Crawl//DTD Check Configuration 1.2//EN"
-    "http://www.puppycrawl.com/dtds/configuration_1_2.dtd">
-
-<!--  Checkstyle configuration for Apache Drill -->
-
-<module name="Checker">
-
-  <module name="TreeWalker">
-
-    <module name="IllegalImport">
-      <property name="illegalPkgs" value="com.google.hive12,com.beust.jcommander.internal,jersey.repackaged"/>
-    </module>
-
-    <module name="AvoidStarImport">
-      <property name="allowStaticMemberImports" value="true"/>
-    </module>
-
-  </module>
-</module>
diff --git a/contrib/format-maprdb/src/main/resources/checkstyle-suppressions.xml b/contrib/format-maprdb/src/main/resources/checkstyle-suppressions.xml
deleted file mode 100644
index c6c34b9..0000000
--- a/contrib/format-maprdb/src/main/resources/checkstyle-suppressions.xml
+++ /dev/null
@@ -1,28 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-
-    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.
-
--->
-<!DOCTYPE suppressions PUBLIC
-    "-//Puppy Crawl//DTD Suppressions 1.1//EN"
-    "suppressions_1_1.dtd">
-    
-<!--  Checkstyle Suppressions for Apache Drill -->
-<suppressions>
-  <suppress files="[\\/]generated-sources[\\/]" checks="AvoidStarImport"/>
-</suppressions>
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/binary/TestMapRDBCFAsJSONString.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/binary/TestMapRDBCFAsJSONString.java
index c592806..4cd9f51 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/binary/TestMapRDBCFAsJSONString.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/binary/TestMapRDBCFAsJSONString.java
@@ -43,5 +43,4 @@ public class TestMapRDBCFAsJSONString extends TestHBaseCFAsJSONString {
   public static void tearDownAfterClass() throws Exception {
     MaprDBTestsSuite.cleanupTests();
   }
-  
 }
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/binary/TestMapRDBProjectPushDown.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/binary/TestMapRDBProjectPushDown.java
index 63d6abb..b5d3909 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/binary/TestMapRDBProjectPushDown.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/binary/TestMapRDBProjectPushDown.java
@@ -43,5 +43,4 @@ public class TestMapRDBProjectPushDown extends TestHBaseProjectPushDown {
   public static void tearDownAfterClass() throws Exception {
     MaprDBTestsSuite.cleanupTests();
   }
-  
 }
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/binary/TestMapRDBQueries.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/binary/TestMapRDBQueries.java
index 5393aa1..0dc30e7 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/binary/TestMapRDBQueries.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/binary/TestMapRDBQueries.java
@@ -43,5 +43,4 @@ public class TestMapRDBQueries extends TestHBaseQueries {
   public static void tearDownAfterClass() throws Exception {
     MaprDBTestsSuite.cleanupTests();
   }
-  
 }
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexHintPlanTest.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexHintPlanTest.java
index c6ec0fe..f7f0da2 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexHintPlanTest.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexHintPlanTest.java
@@ -18,7 +18,6 @@ package com.mapr.drill.maprdb.tests.index;
  * limitations under the License.
  */
 
-import com.mapr.drill.maprdb.tests.json.BaseJsonTest;
 import com.mapr.tests.annotations.ClusterTest;
 import org.apache.drill.PlanTestBase;
 import org.junit.experimental.categories.Category;
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexPlanTest.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexPlanTest.java
index fd3b5cc..18e2316 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexPlanTest.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/IndexPlanTest.java
@@ -733,7 +733,7 @@ public class IndexPlanTest extends BaseJsonTest {
   public void TestCoveringPlanJoin_1() throws Exception {
     String query = "SELECT count(*) as cnt FROM hbase.`index_test_primary` as t1 " +
         " inner join hbase.`index_test_primary` as t2 on t1.driverlicense = t2.driverlicense " +
-        " where t1.driverlicense < 100000003 and t2.driverlicense < 100000003" ;
+        " where t1.driverlicense < 100000003 and t2.driverlicense < 100000003";
     test(defaultHavingIndexPlan);
     PlanTestBase.testPlanMatchingPatterns(query,
         new String[] {".*JsonTableGroupScan.*tableName=.*index_test_primary.*indexName=",
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGen.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGen.java
index 347f7f2..1b64d8e 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGen.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/LargeTableGen.java
@@ -148,7 +148,7 @@ public class LargeTableGen extends LargeTableGenBase {
                   "\"personal\": {\"age\" : %s, \"income\": %s, \"birthdate\": {\"$dateDay\": \"%s\"} }," +
                   "\"activity\": {\"irs\" : { \"firstlogin\":  \"%s\" } }," +
                   "\"driverlicense\":{\"$numberLong\": %s} } \n",
-              i + 1, recordNumber - i , getSSN(i), getPhone(i), getEmail(i),
+              i + 1, recordNumber - i, getSSN(i), getPhone(i), getEmail(i),
               getAddress(i)[2], getAddress(i)[1], getFirstName(i), getLastName(i),
               getAge(i), getIncome(i), getBirthdate(i),
               getFirstLogin(i),
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/StatisticsTest.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/StatisticsTest.java
index 20f1090..adccac4 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/StatisticsTest.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/index/StatisticsTest.java
@@ -17,20 +17,12 @@
  */
 package com.mapr.drill.maprdb.tests.index;
 
-import com.google.common.collect.Lists;
-import com.mapr.db.Admin;
-import com.mapr.drill.maprdb.tests.MaprDBTestsSuite;
-import com.mapr.drill.maprdb.tests.json.BaseJsonTest;
 import com.mapr.tests.annotations.ClusterTest;
 import org.apache.drill.PlanTestBase;
-import org.apache.hadoop.hbase.TableName;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.util.List;
 
 @Category(ClusterTest.class)
 public class StatisticsTest extends IndexPlanTest {
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/BaseJsonTest.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/BaseJsonTest.java
index 2d6ae06..0dd6a4b 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/BaseJsonTest.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/BaseJsonTest.java
@@ -30,7 +30,6 @@ import com.mapr.drill.maprdb.tests.MaprDBTestsSuite;
 
 public class BaseJsonTest extends BaseTestQuery {
   protected static final String SCHEMA = "hbase.root";
-  
   protected String format(final String sql) {
     return String.format(sql, SCHEMA, getTablePath());
   }
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestEncodedFieldPaths.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestEncodedFieldPaths.java
index 5072374..0275f54 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestEncodedFieldPaths.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestEncodedFieldPaths.java
@@ -91,7 +91,7 @@ public class TestEncodedFieldPaths extends BaseJsonTest {
 
     // plan test
     final String[] expectedPlan = {"JsonTableGroupScan.*indexName=encoded_fields_userdata_index.*" + // scan on index
-                                   "columns=\\[`_id`, `age`\\]", 
+                                   "columns=\\[`_id`, `age`\\]",
                                    "RestrictedJsonTableGroupScan.*" + // restricted scan on the table with encoded name
                                    "columns=\\[`age`, `\\$\\$ENC00L5UWIADDN5SGK4Y`, `\\$\\$document`, `_id`\\]",
                                    "RowKeyJoin"};                             // join on row_key
diff --git a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestScanRanges.java b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestScanRanges.java
index efde774..e521c64 100644
--- a/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestScanRanges.java
+++ b/contrib/format-maprdb/src/test/java/com/mapr/drill/maprdb/tests/json/TestScanRanges.java
@@ -30,7 +30,6 @@ import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.drill.exec.rpc.user.AwaitableUserResultsListener;
 import org.apache.drill.exec.store.sys.PersistentStore;
-import org.apache.drill.exec.work.foreman.QueryManager;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;


[drill] 04/15: DRILL-6818: Add descriptions to secondary index options.

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

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

commit 597827e2d3bfe69390242b7e5a5da242d64c181a
Author: Aman Sinha <as...@maprtech.com>
AuthorDate: Mon Nov 19 11:25:59 2018 -0800

    DRILL-6818: Add descriptions to secondary index options.
    
    closes #1545
---
 .../java/org/apache/drill/exec/ExecConstants.java  |  3 +-
 .../exec/planner/physical/PlannerSettings.java     | 39 ++++++++++++++--------
 2 files changed, 28 insertions(+), 14 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 95168bc..7bfb9cf 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
@@ -823,7 +823,8 @@ public final class ExecConstants {
   public static final String ENABLE_ITERATOR_VALIDATION = "drill.exec.debug.validate_iterators";
 
   public static final String QUERY_ROWKEYJOIN_BATCHSIZE_KEY = "exec.query.rowkeyjoin_batchsize";
-  public static final PositiveLongValidator QUERY_ROWKEYJOIN_BATCHSIZE = new PositiveLongValidator(QUERY_ROWKEYJOIN_BATCHSIZE_KEY, Long.MAX_VALUE, null);
+  public static final PositiveLongValidator QUERY_ROWKEYJOIN_BATCHSIZE = new PositiveLongValidator(QUERY_ROWKEYJOIN_BATCHSIZE_KEY, Long.MAX_VALUE,
+      new OptionDescription("Batch size (in terms of number of rows) for a 'bulk get' operation from the underlying data source during a RowKeyJoin."));
   /**
    * When iterator validation is enabled, additionally validates the vectors in
    * each batch passed to each iterator.
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
index c7e8df0..69b9e3d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java
@@ -126,24 +126,37 @@ public class PlannerSettings implements Context{
 
   // ------------------------------------------- Index planning related options BEGIN --------------------------------------------------------------
   public static final String USE_SIMPLE_OPTIMIZER_KEY = "planner.use_simple_optimizer";
-  public static final BooleanValidator USE_SIMPLE_OPTIMIZER = new BooleanValidator(USE_SIMPLE_OPTIMIZER_KEY, null);
-  public static final BooleanValidator INDEX_PLANNING = new BooleanValidator("planner.enable_index_planning", null);
-  public static final BooleanValidator ENABLE_STATS = new BooleanValidator("planner.enable_statistics", null);
-  public static final BooleanValidator DISABLE_FULL_TABLE_SCAN = new BooleanValidator("planner.disable_full_table_scan", null);
-  public static final RangeLongValidator INDEX_MAX_CHOSEN_INDEXES_PER_TABLE = new RangeLongValidator("planner.index.max_chosen_indexes_per_table", 0, 100, null);
-  public static final BooleanValidator INDEX_FORCE_SORT_NONCOVERING = new BooleanValidator("planner.index.force_sort_noncovering", null);
-  public static final BooleanValidator INDEX_USE_HASHJOIN_NONCOVERING = new BooleanValidator("planner.index.use_hashjoin_noncovering", null);
+  public static final BooleanValidator USE_SIMPLE_OPTIMIZER = new BooleanValidator(USE_SIMPLE_OPTIMIZER_KEY,
+      new OptionDescription("Simple optimizer applies fewer rules to reduce planning time and is meant to be used only for simple operational queries that use limit, sort, and filter."));
+  public static final BooleanValidator INDEX_PLANNING = new BooleanValidator("planner.enable_index_planning",
+      new OptionDescription("Enables or disables index planning."));
+  public static final BooleanValidator ENABLE_STATS = new BooleanValidator("planner.enable_statistics",
+      new OptionDescription("Enable or disable statistics for the filter conditions on indexed columns."));
+  public static final BooleanValidator DISABLE_FULL_TABLE_SCAN = new BooleanValidator("planner.disable_full_table_scan",
+      new OptionDescription("Disable generating a full table scan plan (only for internal testing use)"));
+  public static final RangeLongValidator INDEX_MAX_CHOSEN_INDEXES_PER_TABLE = new RangeLongValidator("planner.index.max_chosen_indexes_per_table", 0, 100,
+      new OptionDescription("The maximum number of 'chosen' indexes for a table after index costing and ranking."));
+  public static final BooleanValidator INDEX_FORCE_SORT_NONCOVERING = new BooleanValidator("planner.index.force_sort_noncovering",
+      new OptionDescription("Forces Drill to sort for non-covering indexes. If the query has an ORDER-BY on index columns and a non-covering index is chosen, by default Drill leverages the sortedness of the index columns and does not sort. Fast changing primary table data may produce a partial sort. This option forces a sort within Drill."));
+  public static final BooleanValidator INDEX_USE_HASHJOIN_NONCOVERING = new BooleanValidator("planner.index.use_hashjoin_noncovering",
+      new OptionDescription("Enable using HashJoin for non-covering index plans instead of RowKeyJoin (only for internal testing use)."));
   public static final RangeDoubleValidator INDEX_COVERING_SELECTIVITY_THRESHOLD =
-      new RangeDoubleValidator("planner.index.covering_selectivity_threshold", 0.0, 1.0, null);
+      new RangeDoubleValidator("planner.index.covering_selectivity_threshold", 0.0, 1.0,
+          new OptionDescription("For covering indexes, this option specifies the filter selectivity that corresponds to the leading prefix of the index below which the index is considered for planning."));
   public static final RangeDoubleValidator INDEX_NONCOVERING_SELECTIVITY_THRESHOLD =
-      new RangeDoubleValidator("planner.index.noncovering_selectivity_threshold", 0.0, 1.0, null);
+      new RangeDoubleValidator("planner.index.noncovering_selectivity_threshold", 0.0, 1.0,
+          new OptionDescription("For non-covering indexes, this option specifies the filter selectivity that corresponds to the leading prefix of the index below which the index is considered for planning."));
   public static final RangeDoubleValidator INDEX_ROWKEYJOIN_COST_FACTOR =
-      new RangeDoubleValidator("planner.index.rowkeyjoin_cost_factor", 0, Double.MAX_VALUE, null);
+      new RangeDoubleValidator("planner.index.rowkeyjoin_cost_factor", 0, Double.MAX_VALUE,
+          new OptionDescription("The cost factor that provides some control over the I/O cost for non-covering indexes when the rowkey join back to the primary table causes random I/O from the primary table."));
   // TODO: Deprecate the following 2 (also in SystemOptionManager.java)
-  public static final BooleanValidator INDEX_PREFER_INTERSECT_PLANS = new BooleanValidator("planner.index.prefer_intersect_plans", null);
-  public static final RangeLongValidator INDEX_MAX_INDEXES_TO_INTERSECT = new RangeLongValidator("planner.index.max_indexes_to_intersect", 2, 100, null);
+  public static final BooleanValidator INDEX_PREFER_INTERSECT_PLANS = new BooleanValidator("planner.index.prefer_intersect_plans",
+      new OptionDescription("Given 2 or more single column indexes, this option allows preferring index intersect plans compared to single column indexes (only for internal testing use)."));
+  public static final RangeLongValidator INDEX_MAX_INDEXES_TO_INTERSECT = new RangeLongValidator("planner.index.max_indexes_to_intersect", 2, 100,
+      new OptionDescription("The maximum number of indexes to intersect in a single query (only for internal testing use)."));
   public static final RangeDoubleValidator INDEX_STATS_ROWCOUNT_SCALING_FACTOR =
-      new RangeDoubleValidator("planner.index.statistics_rowcount_scaling_factor", 0.0, 1.0, null);
+      new RangeDoubleValidator("planner.index.statistics_rowcount_scaling_factor", 0.0, 1.0,
+          new OptionDescription("A factor that allows scaling the row count estimates returned from the storage/format plugin to compensate for under or over estimation."));
   // ------------------------------------------- Index planning related options END ----------------------------------------------------------------
 
   public static final OptionValidator IDENTIFIER_MAX_LENGTH =


[drill] 12/15: DRILL-6870: Upgrade to ANTLR4

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

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

commit 44b990be5c15e1c480725cfb78fcabb40216ebf0
Author: Arina Ielchiieva <ar...@gmail.com>
AuthorDate: Tue Nov 20 14:27:55 2018 +0200

    DRILL-6870: Upgrade to ANTLR4
    
    closes #1554
---
 common/pom.xml                                     |   6 --
 .../exec/planner/index/MapRDBIndexDiscover.java    |  32 ++-----
 .../test/java/org/apache/drill/exec/ExecTest.java  |  27 +-----
 .../drill/exec/compile/TestEvaluationVisitor.java  |  50 +++-------
 .../unnest/TestUnnestWithLateralCorrectness.java   |   8 +-
 ...alTestutils.java => DrillLogicalTestUtils.java} |  28 ++----
 .../apache/drill/test/PhysicalOpUnitTestBase.java  |  14 +--
 .../java/org/apache/drill/test/TestBuilder.java    |  46 +++------
 logical/pom.xml                                    |  12 +--
 .../drill/common/expression/parser/ExprLexer.g4}   |  46 +++------
 .../drill/common/expression/parser/ExprParser.g4}  |  81 +++++++++-------
 .../drill/common/expression/LogicalExpression.java |  23 +----
 .../drill/common/expression/PathSegment.java       |   8 +-
 .../apache/drill/common/expression/SchemaPath.java |  30 ++----
 .../apache/drill/common/parser/ErrorListener.java  |  38 ++++++++
 .../common/parser/LogicalExpressionParser.java     |  55 +++++++++++
 .../drill/common/expression/parser/TreeTest.java   | 106 +++++++++++----------
 pom.xml                                            |  11 ++-
 18 files changed, 291 insertions(+), 330 deletions(-)

diff --git a/common/pom.xml b/common/pom.xml
index c931c01..fba9efc 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -102,12 +102,6 @@
     </dependency>
 
     <dependency>
-      <groupId>org.antlr</groupId>
-      <artifactId>antlr-runtime</artifactId>
-      <version>3.4</version>
-    </dependency>
-
-    <dependency>
         <groupId>joda-time</groupId>
         <artifactId>joda-time</artifactId>
         <version>2.9</version>
diff --git a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
index 95b9813..040b5e9 100644
--- a/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
+++ b/contrib/format-maprdb/src/main/java/org/apache/drill/exec/planner/index/MapRDBIndexDiscover.java
@@ -18,22 +18,19 @@
 
 package org.apache.drill.exec.planner.index;
 
+import org.apache.drill.common.parser.LogicalExpressionParser;
 import com.mapr.db.Admin;
 import com.mapr.db.MapRDB;
 import com.mapr.db.exceptions.DBException;
 import com.mapr.db.index.IndexDesc;
 import com.mapr.db.index.IndexDesc.MissingAndNullOrdering;
 import com.mapr.db.index.IndexFieldDesc;
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CommonTokenStream;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelFieldCollation.NullDirection;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.parser.ExprLexer;
-import org.apache.drill.common.expression.parser.ExprParser;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.common.util.DrillFileUtils;
@@ -67,8 +64,7 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
 
 public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDiscover {
 
-  static final String DEFAULT_STRING_CAST_LEN_STR = "256";
-  static final String FIELD_DELIMITER = ":";
+  public static final String DEFAULT_STRING_CAST_LEN_STR = "256";
 
   public MapRDBIndexDiscover(GroupScan inScan, DrillScanRelBase scanRel) {
     super((AbstractDbGroupScan) inScan, scanRel);
@@ -86,7 +82,7 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
   /**
    * For a given table name get the list of indexes defined on the table according to the visibility of
    * the indexes based on permissions.
-   * @param tableName
+   * @param tableName table name
    * @return an IndexCollection representing the list of indexes for that table
    */
   private IndexCollection getTableIndexFromMFS(String tableName) {
@@ -137,7 +133,7 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
         return null;
       }
       MapRDBFormatPlugin maprFormatPlugin = ((MapRDBGroupScan) origScan).getFormatPlugin();
-      FileSystemPlugin fsPlugin = (FileSystemPlugin) (((MapRDBGroupScan) origScan).getStoragePlugin());
+      FileSystemPlugin fsPlugin = (FileSystemPlugin) (origScan.getStoragePlugin());
 
       DrillFileSystem fs = ImpersonationUtil.createFileSystem(origScan.getUserName(), fsPlugin.getFsConf());
       MapRDBFormatMatcher matcher = (MapRDBFormatMatcher) (maprFormatPlugin.getMatcher());
@@ -240,14 +236,9 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
     }
     try {
       String castFunc = String.format("cast( %s as %s)", field, castTypeStr);
-      final ExprLexer lexer = new ExprLexer(new ANTLRStringStream(castFunc));
-      final CommonTokenStream tokens = new CommonTokenStream(lexer);
-      final ExprParser parser = new ExprParser(tokens);
-      final ExprParser.parse_return ret = parser.parse();
-      logger.trace("{}, {}", tokens, ret);
-      return ret.e;
-    }catch(Exception ex) {
-      logger.error("parse failed{}", ex);
+      return LogicalExpressionParser.parse(castFunc);
+    } catch (Exception ex) {
+      logger.error("parse failed: {}", ex);
     }
     return null;
   }
@@ -313,8 +304,7 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
     for (int i = 0; i < indexFieldCollations.size(); i++) {
       collationMap.put(indexFields.get(i), indexFieldCollations.get(i));
     }
-    CollationContext collationContext = new CollationContext(collationMap, indexFieldCollations);
-    return collationContext;
+    return new CollationContext(collationMap, indexFieldCollations);
   }
 
   private DrillIndexDescriptor buildIndexDescriptor(String tableName, IndexDesc desc)
@@ -363,11 +353,7 @@ public class MapRDBIndexDiscover extends IndexDiscoverBase implements IndexDisco
 
     final Admin admin;
     try {
-      admin = currentUser.doAs(new PrivilegedExceptionAction<Admin>() {
-        public Admin run() throws Exception {
-          return MapRDB.getAdmin(conf);
-        }
-      });
+      admin = currentUser.doAs((PrivilegedExceptionAction<Admin>) () -> MapRDB.getAdmin(conf));
     } catch (Exception e) {
       throw new DrillRuntimeException("Failed to get Admin instance for user: " + currentUser.getUserName(), e);
     }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
index 15a6e4f..6725bb9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
@@ -18,16 +18,11 @@
 package org.apache.drill.exec;
 
 import com.codahale.metrics.MetricRegistry;
-import org.apache.drill.shaded.guava.com.google.common.io.Files;
+import org.apache.drill.common.parser.LogicalExpressionParser;
 import mockit.Mock;
 import mockit.MockUp;
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.RecognitionException;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.expression.parser.ExprLexer;
-import org.apache.drill.common.expression.parser.ExprParser;
 import org.apache.drill.common.scanner.ClassPathScanner;
 import org.apache.drill.exec.compile.CodeCompilerTestFactory;
 import org.apache.drill.exec.memory.RootAllocatorFactory;
@@ -48,7 +43,6 @@ import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 
-import java.io.File;
 import java.io.IOException;
 import java.text.DateFormatSymbols;
 import java.util.Locale;
@@ -94,17 +88,6 @@ public class ExecTest extends DrillTest {
     return FileSystem.get(configuration);
   }
 
-  /**
-   * Create a temp directory to store the given <i>dirName</i>.
-   * Directory will be deleted on exit.
-   * @param dirName directory name
-   * @return Full path including temp parent directory and given directory name.
-   */
-  public static String getTempDir(final String dirName) {
-    final File dir = Files.createTempDir();
-    return dir.getAbsolutePath() + File.separator + dirName;
-  }
-
   protected DrillbitContext mockDrillbitContext() throws Exception {
     final DrillbitContext context = mock(DrillbitContext.class);
 
@@ -118,12 +101,8 @@ public class ExecTest extends DrillTest {
     return context;
   }
 
-  protected LogicalExpression parseExpr(String expr) throws RecognitionException {
-    final ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
-    final CommonTokenStream tokens = new CommonTokenStream(lexer);
-    final ExprParser parser = new ExprParser(tokens);
-    final ExprParser.parse_return ret = parser.parse();
-    return ret.e;
+  public LogicalExpression parseExpr(String expr) {
+    return LogicalExpressionParser.parse(expr);
   }
 
   /**
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestEvaluationVisitor.java b/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestEvaluationVisitor.java
index 91ce653..631f1f5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestEvaluationVisitor.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestEvaluationVisitor.java
@@ -17,68 +17,46 @@
  */
 package org.apache.drill.exec.compile;
 
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.parser.ExprLexer;
-import org.apache.drill.common.expression.parser.ExprParser;
-import org.apache.drill.common.expression.parser.ExprParser.parse_return;
+import org.apache.drill.common.parser.LogicalExpressionParser;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.expr.CodeGenerator;
 import org.apache.drill.exec.expr.EvaluationVisitor;
 import org.apache.drill.exec.expr.ValueVectorReadExpression;
 import org.apache.drill.exec.expr.ValueVectorWriteExpression;
-import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.physical.impl.project.Projector;
 import org.apache.drill.exec.record.TypedFieldId;
 import org.junit.Test;
 
 public class TestEvaluationVisitor {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestEvaluationVisitor.class);
-
 
   @Test
-  public void x() throws Exception{
-    DrillConfig c = DrillConfig.create();
-
-    FunctionImplementationRegistry reg = new FunctionImplementationRegistry(c);
+  public void testEvaluation() {
     EvaluationVisitor v = new EvaluationVisitor();
     CodeGenerator<?> g = CodeGenerator.get(Projector.TEMPLATE_DEFINITION, null);
-    SchemaPath path = (SchemaPath) getExpr("a.b[4][2].c[6]");
+    SchemaPath path = (SchemaPath) LogicalExpressionParser.parse("a.b[4][2].c[6]");
 
-    TypedFieldId id = TypedFieldId.newBuilder() //
-      .addId(1) //
-      .addId(3) //
-      .remainder(path.getRootSegment()) //
+    TypedFieldId id = TypedFieldId.newBuilder()
+      .addId(1)
+      .addId(3)
+      .remainder(path.getRootSegment())
       .intermediateType(Types.optional(MinorType.MAP))
-      .finalType(Types.repeated(MinorType.MAP)) //
-      .hyper() //
-      .withIndex() //
+      .finalType(Types.repeated(MinorType.MAP))
+      .hyper()
+      .withIndex()
       .build();
 
     ValueVectorReadExpression e = new ValueVectorReadExpression(id);
 
-    TypedFieldId outId = TypedFieldId.newBuilder() //
-        .addId(1) //
-        .finalType(Types.repeated(MinorType.MAP)) //
-        .intermediateType(Types.repeated(MinorType.MAP)) //
+    TypedFieldId outId = TypedFieldId.newBuilder()
+        .addId(1)
+        .finalType(Types.repeated(MinorType.MAP))
+        .intermediateType(Types.repeated(MinorType.MAP))
         .build();
     ValueVectorWriteExpression e2 = new ValueVectorWriteExpression(outId, e, true);
 
     v.addExpr(e2,  g.getRoot());
-    logger.debug(g.generateAndGet());
   }
 
-  private LogicalExpression getExpr(String expr) throws Exception{
-    ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
-    CommonTokenStream tokens = new CommonTokenStream(lexer);
-    ExprParser parser = new ExprParser(tokens);
-    parse_return ret = parser.parse();
-
-    return ret.e;
-
-  }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/TestUnnestWithLateralCorrectness.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/TestUnnestWithLateralCorrectness.java
index 2e21408..7d150a2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/TestUnnestWithLateralCorrectness.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/unnest/TestUnnestWithLateralCorrectness.java
@@ -37,7 +37,7 @@ import org.apache.drill.exec.physical.impl.project.ProjectRecordBatch;
 import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
 import org.apache.drill.exec.planner.common.DrillLateralJoinRelBase;
 import org.apache.drill.exec.planner.common.DrillUnnestRelBase;
-import org.apache.drill.exec.planner.logical.DrillLogicalTestutils;
+import org.apache.drill.exec.planner.logical.DrillLogicalTestUtils;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
@@ -590,7 +590,7 @@ public class TestUnnestWithLateralCorrectness extends SubOperatorTest {
 
     // project is required to rename the columns so as to disambiguate the same column name from
     // unnest operator and the regular scan.
-    final Project projectPopConfig = new Project(DrillLogicalTestutils.parseExprs("unnestColumn", "unnestColumn1",
+    final Project projectPopConfig = new Project(DrillLogicalTestUtils.parseExprs("unnestColumn", "unnestColumn1",
       unnestPopConfig.getImplicitColumn(), unnestPopConfig.getImplicitColumn()), null);
 
     final ProjectRecordBatch projectBatch =
@@ -899,12 +899,12 @@ public class TestUnnestWithLateralCorrectness extends SubOperatorTest {
 
     // Create intermediate Project
     final Project projectPopConfig1 =
-        new Project(DrillLogicalTestutils.parseExprs("unnestColumn.colB", "colB",
+        new Project(DrillLogicalTestUtils.parseExprs("unnestColumn.colB", "colB",
           unnestPopConfig1.getImplicitColumn(), unnestPopConfig1.getImplicitColumn()), unnestPopConfig1);
     final ProjectRecordBatch projectBatch1 =
         new ProjectRecordBatch(projectPopConfig1, unnestBatch1, fixture.getFragmentContext());
     final Project projectPopConfig2 =
-        new Project(DrillLogicalTestutils.parseExprs("colB", "unnestColumn2",
+        new Project(DrillLogicalTestUtils.parseExprs("colB", "unnestColumn2",
           unnestPopConfig2.getImplicitColumn(), unnestPopConfig2.getImplicitColumn()), unnestPopConfig2);
     final ProjectRecordBatch projectBatch2 =
         new ProjectRecordBatch(projectPopConfig2, unnestBatch2, fixture.getFragmentContext());
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillLogicalTestutils.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillLogicalTestUtils.java
similarity index 65%
rename from exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillLogicalTestutils.java
rename to exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillLogicalTestUtils.java
index a6b5906..d65cb3e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillLogicalTestutils.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillLogicalTestUtils.java
@@ -17,17 +17,12 @@
  */
 package org.apache.drill.exec.planner.logical;
 
+import org.apache.drill.common.parser.LogicalExpressionParser;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.RecognitionException;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.drill.common.expression.FieldReference;
-import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.PathSegment;
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.parser.ExprLexer;
-import org.apache.drill.common.expression.parser.ExprParser;
 import org.apache.drill.common.logical.data.JoinCondition;
 import org.apache.drill.common.logical.data.NamedExpression;
 import org.apache.drill.common.logical.data.Order;
@@ -35,36 +30,27 @@ import org.apache.drill.common.logical.data.Order;
 import java.util.ArrayList;
 import java.util.List;
 
-public class DrillLogicalTestutils {
+public class DrillLogicalTestUtils {
   public static Order.Ordering ordering(String expression,
                                         RelFieldCollation.Direction direction,
                                         RelFieldCollation.NullDirection nullDirection) {
-    return new Order.Ordering(direction, parseExpr(expression), nullDirection);
+    return new Order.Ordering(direction, LogicalExpressionParser.parse(expression), nullDirection);
   }
 
   public static JoinCondition joinCond(String leftExpr, String relationship, String rightExpr) {
-    return new JoinCondition(relationship, parseExpr(leftExpr), parseExpr(rightExpr));
+    return new JoinCondition(relationship, LogicalExpressionParser.parse(leftExpr), LogicalExpressionParser.parse(rightExpr));
   }
 
   public static List<NamedExpression> parseExprs(String... expressionsAndOutputNames) {
     Preconditions.checkArgument(expressionsAndOutputNames.length % 2 == 0,
-      "List of expressions and output field names" + " is not complete, each expression must explicitly give and output name,");
+      "List of expressions and output field names"
+        + " is not complete, each expression must explicitly give and output name,");
     List<NamedExpression> ret = new ArrayList<>();
     for (int i = 0; i < expressionsAndOutputNames.length; i += 2) {
-      ret.add(new NamedExpression(parseExpr(expressionsAndOutputNames[i]),
+      ret.add(new NamedExpression(LogicalExpressionParser.parse(expressionsAndOutputNames[i]),
         new FieldReference(new SchemaPath(new PathSegment.NameSegment(expressionsAndOutputNames[i + 1])))));
     }
     return ret;
   }
 
-  public static LogicalExpression parseExpr(String expr) {
-    ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
-    CommonTokenStream tokens = new CommonTokenStream(lexer);
-    ExprParser parser = new ExprParser(tokens);
-    try {
-      return parser.parse().e;
-    } catch (RecognitionException e) {
-      throw new RuntimeException("Error parsing expression: " + expr);
-    }
-  }
 }
\ No newline at end of file
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
index 300e88b..84a7c78 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
@@ -29,7 +29,7 @@ import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.FragmentStats;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
-import org.apache.drill.exec.planner.logical.DrillLogicalTestutils;
+import org.apache.drill.exec.planner.logical.DrillLogicalTestUtils;
 import org.apache.drill.exec.proto.CoordinationProtos;
 import org.apache.drill.exec.rpc.control.Controller;
 import org.apache.drill.exec.rpc.control.WorkEventBus;
@@ -42,7 +42,6 @@ import org.apache.drill.exec.work.filter.RuntimeFilterSink;
 import org.apache.drill.exec.work.filter.RuntimeFilterWritable;
 import org.apache.drill.common.config.DrillConfig;
 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.common.logical.data.JoinCondition;
 import org.apache.drill.common.logical.data.NamedExpression;
@@ -115,21 +114,16 @@ public class PhysicalOpUnitTestBase extends ExecTest {
     scanDecodeExecutor.shutdownNow();
   }
 
-  @Override
-  protected LogicalExpression parseExpr(String expr) {
-    return DrillLogicalTestutils.parseExpr(expr);
-  }
-
   protected Order.Ordering ordering(String expression, RelFieldCollation.Direction direction, RelFieldCollation.NullDirection nullDirection) {
-    return DrillLogicalTestutils.ordering(expression, direction, nullDirection);
+    return DrillLogicalTestUtils.ordering(expression, direction, nullDirection);
   }
 
   protected JoinCondition joinCond(String leftExpr, String relationship, String rightExpr) {
-    return DrillLogicalTestutils.joinCond(leftExpr, relationship, rightExpr);
+    return DrillLogicalTestUtils.joinCond(leftExpr, relationship, rightExpr);
   }
 
   protected List<NamedExpression> parseExprs(String... expressionsAndOutputNames) {
-    return DrillLogicalTestutils.parseExprs(expressionsAndOutputNames);
+    return DrillLogicalTestUtils.parseExprs(expressionsAndOutputNames);
   }
 
   protected static class BatchIterator implements Iterable<VectorAccessible> {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
index d45bd6f..37e05cd 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/TestBuilder.java
@@ -30,13 +30,10 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.RecognitionException;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.parser.ExprLexer;
-import org.apache.drill.common.expression.parser.ExprParser;
+import org.apache.drill.common.parser.LogicalExpressionParser;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.expr.fn.impl.DateUtility;
@@ -159,15 +156,13 @@ public class TestBuilder {
   }
 
   public TestBuilder sqlQueryFromFile(String queryFile) throws IOException {
-    String query = BaseTestQuery.getFile(queryFile);
-    this.query = query;
+    this.query = BaseTestQuery.getFile(queryFile);
     queryType = UserBitShared.QueryType.SQL;
     return this;
   }
 
   public TestBuilder physicalPlanFromFile(String queryFile) throws IOException {
-    String query = BaseTestQuery.getFile(queryFile);
-    this.query = query;
+    this.query =  BaseTestQuery.getFile(queryFile);
     queryType = UserBitShared.QueryType.PHYSICAL;
     return this;
   }
@@ -226,20 +221,11 @@ public class TestBuilder {
 
   // modified code from SchemaPath.De class. This should be used sparingly and only in tests if absolutely needed.
   public static SchemaPath parsePath(String path) {
-    try {
-      ExprLexer lexer = new ExprLexer(new ANTLRStringStream(path));
-      CommonTokenStream tokens = new CommonTokenStream(lexer);
-      ExprParser parser = new ExprParser(tokens);
-
-      ExprParser.parse_return ret = parser.parse();
-
-      if (ret.e instanceof SchemaPath) {
-        return (SchemaPath) ret.e;
-      } else {
-        throw new IllegalStateException("Schema path is not a valid format.");
-      }
-    } catch (RecognitionException e) {
-      throw new RuntimeException(e);
+    LogicalExpression expr = LogicalExpressionParser.parse(path);
+    if (expr instanceof SchemaPath) {
+      return (SchemaPath) expr;
+    } else {
+      throw new IllegalStateException(String.format("Schema path is not a valid format: %s.", expr));
     }
   }
 
@@ -290,11 +276,7 @@ public class TestBuilder {
   }
 
   boolean typeInfoSet() {
-    if (baselineTypeMap != null) {
-      return true;
-    } else {
-      return false;
-    }
+    return baselineTypeMap != null;
   }
 
   /**
@@ -513,11 +495,7 @@ public class TestBuilder {
 
     @Override
     boolean typeInfoSet() {
-      if (super.typeInfoSet() || baselineTypes != null) {
-        return true;
-      } else {
-        return false;
-      }
+      return super.typeInfoSet() || baselineTypes != null;
     }
 
     @Override
@@ -683,7 +661,7 @@ public class TestBuilder {
   }
 
   /**
-   * Convenience method to create a {@link JsonStringHashMap<String, Object> map} instance with the given key value sequence.
+   * Convenience method to create a {@link JsonStringHashMap<String, Object>} map instance with the given key value sequence.
    *
    * Key value sequence consists of key - value pairs such that a key precedes its value. For instance:
    *
diff --git a/logical/pom.xml b/logical/pom.xml
index aa8fb06..94bef6c 100644
--- a/logical/pom.xml
+++ b/logical/pom.xml
@@ -84,8 +84,7 @@
     
     <dependency>
       <groupId>org.antlr</groupId>
-      <artifactId>antlr-runtime</artifactId>
-      <version>3.4</version>
+      <artifactId>antlr4-runtime</artifactId>
     </dependency>
 
     <dependency>
@@ -111,15 +110,12 @@
       </plugin>
       <plugin>
         <groupId>org.antlr</groupId>
-        <artifactId>antlr3-maven-plugin</artifactId>
-        <version>3.4</version>
-        <configuration>
-
-        </configuration>
+        <artifactId>antlr4-maven-plugin</artifactId>
+        <version>${antlr.version}</version>
         <executions>
           <execution>
             <goals>
-              <goal>antlr</goal>
+              <goal>antlr4</goal>
             </goals>
           </execution>
         </executions>
diff --git a/logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g b/logical/src/main/antlr4/org/apache/drill/common/expression/parser/ExprLexer.g4
similarity index 79%
rename from logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
rename to logical/src/main/antlr4/org/apache/drill/common/expression/parser/ExprLexer.g4
index 93dba94..53b39bd 100644
--- a/logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
+++ b/logical/src/main/antlr4/org/apache/drill/common/expression/parser/ExprLexer.g4
@@ -22,8 +22,6 @@ options {
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-package org.apache.drill.common.expression.parser;
 }
 
 If       : 'if';
@@ -99,40 +97,35 @@ Bool
 Number
   :  Int ('.' Digit*)? (('e' | 'E') ('+' | '-')? Digit*)?
   ;
-
-//Float
-//  :  Int ('.' Digit*)? ('e' ('+' | '-')? Digit*)?
-//  ;
-//
-//Integer
-//  :  Digit Digit*
-//  ;
   
 Identifier
   : ('a'..'z' | 'A'..'Z' | '_' | '$') ('a'..'z' | 'A'..'Z' | '_' | '$' | Digit)*
   ;
 
 QuotedIdentifier
-@after {
-  setText(getText().substring(1, getText().length()-1).replaceAll("\\\\(.)", "$1"));
-}
-  :  '`'  (~('`' | '\\')  | '\\' ('\\' | '`'))* '`' 
+  :  '`'  (~('`' | '\\')  | '\\' ('\\' | '`'))* '`'
+  {
+    setText(getText().substring(1, getText().length()-1).replaceAll("\\\\(.)", "$1"));
+  }
   ;
 
 String
-@after {
-  setText(getText().substring(1, getText().length()-1).replaceAll("\\\\(.)", "$1"));
-}
   :  '\'' (~('\'' | '\\') | '\\' ('\\' | '\''))* '\''
+  {
+    setText(getText().substring(1, getText().length()-1).replaceAll("\\\\(.)", "$1"));
+  }
   ;
 
-Comment
-  :  '//' ~('\r' | '\n')* {skip();}
-  |  '/*' .* '*/'         {skip();}
+LineComment
+  :  '//' ~[\r\n]* -> skip
+  ;
+
+BlockComment
+  : '/*' .*? '*/' -> skip
   ;
 
 Space
-  :  (' ' | '\t' | '\r' | '\n' | '\u000C') {skip();}
+  :  [ \n\t\r\u000C]+ -> skip
   ;
 
 fragment Int
@@ -143,14 +136,3 @@ fragment Int
 fragment Digit 
   :  '0'..'9'
   ;
-
-FallThrough
-	@after{
-	  throw new RuntimeException(java.lang.String.format(
-	      "Encountered an illegal char on line \%d, column \%d: '\%s'", 
-	      getLine(), getCharPositionInLine(), getText()
-	    )
-	  );
-	}
-  :
-  ;
diff --git a/logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g b/logical/src/main/antlr4/org/apache/drill/common/expression/parser/ExprParser.g4
similarity index 85%
rename from logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
rename to logical/src/main/antlr4/org/apache/drill/common/expression/parser/ExprParser.g4
index 78a7cc3..d868cff 100644
--- a/logical/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
+++ b/logical/src/main/antlr4/org/apache/drill/common/expression/parser/ExprParser.g4
@@ -1,15 +1,10 @@
 parser grammar ExprParser;
 
 options{
-  output=AST;
   language=Java;
   tokenVocab=ExprLexer;
-  backtrack=true;
-  memoize=true;
 }
 
-
-
 @header {
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
@@ -28,11 +23,8 @@ options{
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-package org.apache.drill.common.expression.parser;
   
 //Explicit import...
-import org.antlr.runtime.BitSet;
 import java.util.*;
 import org.apache.drill.common.expression.*;
 import org.apache.drill.common.expression.PathSegment.NameSegment;
@@ -55,25 +47,20 @@ import org.apache.drill.common.exceptions.ExpressionParsingException;
   public ExpressionPosition pos(Token token){
     return new ExpressionPosition(fullExpression, token.getTokenIndex());
   }
-  
-  @Override    
-  public void displayRecognitionError(String[] tokenNames, RecognitionException e) {
-	String hdr = getErrorHeader(e);
-    String msg = getErrorMessage(e, tokenNames);
-    throw new ExpressionParsingException("Expression has syntax error! " + hdr + ":" + msg);
-  }
 }
 
 parse returns [LogicalExpression e]
   :  expression EOF {
-    $e = $expression.e; 
-    if(fullExpression == null) fullExpression = $expression.text;
+    $e = $expression.e;
+    if (fullExpression == null) fullExpression = $expression.text;
     tokenPos = $expression.start.getTokenIndex();
   }
   ;
  
 functionCall returns [LogicalExpression e]
-  :  Identifier OParen exprList? CParen {$e = FunctionCallFactory.createExpression($Identifier.text, pos($Identifier), $exprList.listE);  }
+  :  Identifier OParen exprList? CParen {$e =
+      FunctionCallFactory.createExpression($Identifier.text, pos($Identifier),
+        ($exprList.ctx == null ? new ArrayList<>() : $exprList.listE)); }
   ;
 
 convertCall returns [LogicalExpression e]
@@ -82,16 +69,18 @@ convertCall returns [LogicalExpression e]
   ;
 
 anyValueCall returns [LogicalExpression e]
-  :  AnyValue OParen exprList? CParen {$e = FunctionCallFactory.createExpression($AnyValue.text, pos($AnyValue), $exprList.listE);  }
+  :  AnyValue OParen exprList? CParen {$e =
+      FunctionCallFactory.createExpression($AnyValue.text, pos($AnyValue),
+       ($exprList.ctx == null ? new ArrayList<>() : $exprList.listE)); }
   ;
 
 castCall returns [LogicalExpression e]
 	@init{
-  	  List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
+	  List<LogicalExpression> exprs = new ArrayList<>();
 	  ExpressionPosition p = null;
 	}  
   :  Cast OParen expression As dataType repeat? CParen 
-      {  if ($repeat.isRep!=null && $repeat.isRep.compareTo(Boolean.TRUE)==0)
+      {  if ($repeat.ctx != null && $repeat.isRep.compareTo(Boolean.TRUE)==0)
            $e = FunctionCallFactory.createCast(TypeProtos.MajorType.newBuilder().mergeFrom($dataType.type).setMode(DataMode.REPEATED).build(), pos($Cast), $expression.e);
          else
            $e = FunctionCallFactory.createCast($dataType.type, pos($Cast), $expression.e);}
@@ -190,7 +179,7 @@ caseElseStat returns [LogicalExpression e]
   
 exprList returns [List<LogicalExpression> listE]
 	@init{
-	  $listE = new ArrayList<LogicalExpression>();
+	  $listE = new ArrayList<>();
 	}
   :  e1=expression {$listE.add($e1.e); } (Comma e2=expression {$listE.add($e2.e); } )*
   ;
@@ -207,7 +196,7 @@ condExpr returns [LogicalExpression e]
 
 orExpr returns [LogicalExpression e]
 	@init{
-	  List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
+	  List<LogicalExpression> exprs = new ArrayList<>();
 	  ExpressionPosition p = null;
 	}
 	@after{
@@ -222,7 +211,7 @@ orExpr returns [LogicalExpression e]
 
 andExpr returns [LogicalExpression e]
 	@init{
-	  List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
+	  List<LogicalExpression> exprs = new ArrayList<>();
 	  ExpressionPosition p = null;
 	}
 	@after{
@@ -237,8 +226,8 @@ andExpr returns [LogicalExpression e]
 
 equExpr returns [LogicalExpression e]
 	@init{
-	  List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
-	  List<String> cmps = new ArrayList();
+	  List<LogicalExpression> exprs = new ArrayList<>();
+	  List<String> cmps = new ArrayList<>();
 	  ExpressionPosition p = null;
 	}
 	@after{
@@ -254,8 +243,8 @@ relExpr returns [LogicalExpression e]
 
 addExpr returns [LogicalExpression e]
 	@init{
-	  List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
-	  List<String> ops = new ArrayList();
+	  List<LogicalExpression> exprs = new ArrayList<>();
+	  List<String> ops = new ArrayList<>();
 	  ExpressionPosition p = null;
 	}
 	@after{
@@ -266,8 +255,8 @@ addExpr returns [LogicalExpression e]
 
 mulExpr returns [LogicalExpression e]
 	@init{
-	  List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
-	  List<String> ops = new ArrayList();
+	  List<LogicalExpression> exprs = new ArrayList<>();
+	  List<String> ops = new ArrayList<>();
 	  ExpressionPosition p = null;
 	}
 	@after{
@@ -278,8 +267,8 @@ mulExpr returns [LogicalExpression e]
 
 xorExpr returns [LogicalExpression e]
 	@init{
-	  List<LogicalExpression> exprs = new ArrayList<LogicalExpression>();
-	  List<String> ops = new ArrayList();
+    List<LogicalExpression> exprs = new ArrayList<>();
+    List<String> ops = new ArrayList<>();
 	  ExpressionPosition p = null;
 	}
 	@after{
@@ -305,15 +294,35 @@ pathSegment returns [NameSegment seg]
   ;
 
 nameSegment returns [NameSegment seg]
-  : QuotedIdentifier ( (Period s1=pathSegment) | s2=arraySegment)? {$seg = new NameSegment($QuotedIdentifier.text, ($s1.seg == null ? $s2.seg : $s1.seg) ); }
-  | Identifier ( (Period s1=pathSegment) | s2=arraySegment)? {$seg = new NameSegment($Identifier.text, ($s1.seg == null ? $s2.seg : $s1.seg) ); }
+  : QuotedIdentifier ( (Period s1=pathSegment) | s2=arraySegment)?
+  {
+    if ($s1.ctx == null && $s2.ctx == null) {
+      $seg = new NameSegment($QuotedIdentifier.text);
+    } else {
+      $seg = new NameSegment($QuotedIdentifier.text, ($s1.ctx == null ? $s2.seg : $s1.seg));
+    }
+  }
+  | Identifier ( (Period s1=pathSegment) | s2=arraySegment)?
+  {
+    if ($s1.ctx == null && $s2.ctx == null) {
+      $seg = new NameSegment($Identifier.text);
+    } else {
+      $seg = new NameSegment($Identifier.text, ($s1.ctx == null ? $s2.seg : $s1.seg));
+    }
+   }
   ;
   
 arraySegment returns [PathSegment seg]
-  :  OBracket Number CBracket ( (Period s1=pathSegment) | s2=arraySegment)? {$seg = new ArraySegment($Number.text, ($s1.seg == null ? $s2.seg : $s1.seg) ); }
+  :  OBracket Number CBracket ( (Period s1=pathSegment) | s2=arraySegment)?
+  {
+    if ($s1.ctx == null && $s2.ctx == null) {
+      $seg = new ArraySegment($Number.text);
+    } else {
+      $seg = new ArraySegment($Number.text, ($s1.ctx == null ? $s2.seg : $s1.seg));
+    }
+  }
   ;
 
-
 lookup returns [LogicalExpression e]
   :  functionCall {$e = $functionCall.e ;}
   | convertCall {$e = $convertCall.e; }
diff --git a/logical/src/main/java/org/apache/drill/common/expression/LogicalExpression.java b/logical/src/main/java/org/apache/drill/common/expression/LogicalExpression.java
index cda77c3..3cfbade 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/LogicalExpression.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/LogicalExpression.java
@@ -19,14 +19,9 @@ package org.apache.drill.common.expression;
 
 import java.io.IOException;
 
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.RecognitionException;
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.expression.parser.ExprLexer;
-import org.apache.drill.common.expression.parser.ExprParser;
-import org.apache.drill.common.expression.parser.ExprParser.parse_return;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.parser.LogicalExpressionParser;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -68,20 +63,8 @@ public interface LogicalExpression extends Iterable<LogicalExpression>{
       if (expr == null || expr.isEmpty()) {
         return null;
       }
-      try {
-        ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
-        CommonTokenStream tokens = new CommonTokenStream(lexer);
-        ExprParser parser = new ExprParser(tokens);
-
-        //TODO: move functionregistry and error collector to injectables.
-        //ctxt.findInjectableValue(valueId, forProperty, beanInstance)
-        parse_return ret = parser.parse();
-
-        // ret.e.resolveAndValidate(expr, errorCollector);
-        return ret.e;
-      } catch (RecognitionException e) {
-        throw new RuntimeException(e);
-      }
+
+      return LogicalExpressionParser.parse(expr);
     }
 
   }
diff --git a/logical/src/main/java/org/apache/drill/common/expression/PathSegment.java b/logical/src/main/java/org/apache/drill/common/expression/PathSegment.java
index f198620..6c6a094 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/PathSegment.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/PathSegment.java
@@ -39,6 +39,10 @@ public abstract class PathSegment {
       this(Integer.parseInt(numberAsText), child);
     }
 
+    public ArraySegment(String numberAsText) {
+      this(Integer.parseInt(numberAsText), null);
+    }
+
     public ArraySegment(int index, PathSegment child) {
       super(child);
       this.index = index;
@@ -105,7 +109,7 @@ public abstract class PathSegment {
 
     @Override
     public PathSegment clone() {
-      PathSegment seg = index < 0 ? new ArraySegment(null) : new ArraySegment(index);
+      PathSegment seg = index < 0 ? new ArraySegment((PathSegment) null) : new ArraySegment(index);
       if (getChild() != null) {
         seg.setChild(getChild().clone());
       }
@@ -114,7 +118,7 @@ public abstract class PathSegment {
 
     @Override
     public ArraySegment cloneWithNewChild(PathSegment newChild) {
-      ArraySegment seg = index < 0 ? new ArraySegment(null) : new ArraySegment(index);
+      ArraySegment seg = index < 0 ? new ArraySegment((PathSegment) null) : new ArraySegment(index);
       if (getChild() != null) {
         seg.setChild(getChild().cloneWithNewChild(newChild));
       } else {
diff --git a/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java b/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java
index ba71712..d514d2e 100644
--- a/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java
+++ b/logical/src/main/java/org/apache/drill/common/expression/SchemaPath.java
@@ -21,15 +21,10 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.Iterator;
 
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.RecognitionException;
 import org.apache.drill.common.expression.PathSegment.ArraySegment;
 import org.apache.drill.common.expression.PathSegment.NameSegment;
-import org.apache.drill.common.expression.parser.ExprLexer;
-import org.apache.drill.common.expression.parser.ExprParser;
-import org.apache.drill.common.expression.parser.ExprParser.parse_return;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.parser.LogicalExpressionParser;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.proto.UserBitShared.NamePart;
@@ -179,23 +174,16 @@ public class SchemaPath extends LogicalExpressionBase {
     if (expr == null || expr.isEmpty()) {
       return null;
     }
-    try {
-      if (SchemaPath.DYNAMIC_STAR.equals(expr)) {
-        return SchemaPath.getSimplePath(expr);
-      }
-      ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
-      CommonTokenStream tokens = new CommonTokenStream(lexer);
-      ExprParser parser = new ExprParser(tokens);
 
-      parse_return ret = parser.parse();
+    if (SchemaPath.DYNAMIC_STAR.equals(expr)) {
+      return SchemaPath.getSimplePath(expr);
+    }
 
-      if (ret.e instanceof SchemaPath) {
-        return (SchemaPath) ret.e;
-      } else {
-        throw new IllegalStateException("Schema path is not a valid format.");
-      }
-    } catch (RecognitionException e) {
-      throw new RuntimeException(e);
+    LogicalExpression logicalExpression = LogicalExpressionParser.parse(expr);
+    if (logicalExpression instanceof SchemaPath) {
+      return (SchemaPath) logicalExpression;
+    } else {
+      throw new IllegalStateException(String.format("Schema path is not a valid format: %s.", logicalExpression));
     }
   }
 
diff --git a/logical/src/main/java/org/apache/drill/common/parser/ErrorListener.java b/logical/src/main/java/org/apache/drill/common/parser/ErrorListener.java
new file mode 100644
index 0000000..fe94368
--- /dev/null
+++ b/logical/src/main/java/org/apache/drill/common/parser/ErrorListener.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.common.parser;
+
+import org.antlr.v4.runtime.BaseErrorListener;
+import org.antlr.v4.runtime.RecognitionException;
+import org.antlr.v4.runtime.Recognizer;
+import org.apache.drill.common.exceptions.ExpressionParsingException;
+
+/**
+ * Custom error listener that converts all syntax errors into {@link ExpressionParsingException}.
+ */
+public class ErrorListener extends BaseErrorListener {
+
+  public static final ErrorListener INSTANCE = new ErrorListener();
+
+  @Override
+  public void syntaxError(Recognizer<?, ?> recognizer, Object offendingSymbol, int line,
+                          int charPositionInLine, String msg, RecognitionException e) {
+    throw new ExpressionParsingException(msg);
+  }
+
+}
diff --git a/logical/src/main/java/org/apache/drill/common/parser/LogicalExpressionParser.java b/logical/src/main/java/org/apache/drill/common/parser/LogicalExpressionParser.java
new file mode 100644
index 0000000..ad30a20
--- /dev/null
+++ b/logical/src/main/java/org/apache/drill/common/parser/LogicalExpressionParser.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.common.parser;
+
+import org.antlr.v4.runtime.CharStreams;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.parser.ExprLexer;
+import org.apache.drill.common.expression.parser.ExprParser;
+
+/**
+ * Helper class for parsing logical expression.
+ */
+public class LogicalExpressionParser {
+
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LogicalExpressionParser.class);
+
+  /**
+   * Initializes logical expression lexer and parser, add error listener that converts all
+   * syntax error into {@link org.apache.drill.common.exceptions.ExpressionParsingException}.
+   * Parses given expression into logical expression instance.
+   *
+   * @param expr expression to be parsed
+   * @return logical expression instance
+   */
+  public static LogicalExpression parse(String expr) {
+    ExprLexer lexer = new ExprLexer(CharStreams.fromString(expr));
+    lexer.removeErrorListeners(); // need to remove since default listener will output warning
+    lexer.addErrorListener(ErrorListener.INSTANCE);
+    CommonTokenStream tokens = new CommonTokenStream(lexer);
+
+    ExprParser parser = new ExprParser(tokens);
+    parser.removeErrorListeners(); // need to remove since default listener will output warning
+    parser.addErrorListener(ErrorListener.INSTANCE);
+    ExprParser.ParseContext parseContext = parser.parse();
+    logger.trace("Tokens: [{}]. Parsing details: [{}].", tokens.getText(), parseContext.toInfoString(parser));
+    return parseContext.e;
+  }
+
+}
diff --git a/logical/src/test/java/org/apache/drill/common/expression/parser/TreeTest.java b/logical/src/test/java/org/apache/drill/common/expression/parser/TreeTest.java
index 6dd489e..0d34716 100644
--- a/logical/src/test/java/org/apache/drill/common/expression/parser/TreeTest.java
+++ b/logical/src/test/java/org/apache/drill/common/expression/parser/TreeTest.java
@@ -17,103 +17,113 @@
  */
 package org.apache.drill.common.expression.parser;
 
-import java.io.IOException;
-
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.RecognitionException;
+import org.apache.drill.common.exceptions.ExpressionParsingException;
 import org.apache.drill.common.expression.ExpressionStringBuilder;
 import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.expression.parser.ExprParser.parse_return;
+import org.apache.drill.common.parser.LogicalExpressionParser;
 import org.apache.drill.test.DrillTest;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertEquals;
 
 public class TreeTest extends DrillTest {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TreeTest.class);
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
 
   @Test
-  public void escapeStringLiteral() throws Exception {
+  public void escapeStringLiteral() {
     String expr = "func(`identifier`, '\\\\d+', 0, 'fjds')";
-    testExpressionParsing(expr);
+    testExpressionParsing(expr, expr);
   }
 
   @Test
-  public void escapeQuotedIdentifier() throws Exception {
+  public void escapeQuotedIdentifier() {
     String expr = "`a\\\\b` + `c'd`";
-    testExpressionParsing(expr);
+    testExpressionParsing(expr, "add(`a\\\\b`, `c'd`)");
   }
 
   @Test
-  public void testIfWithCase() throws Exception{
-    testExpressionParsing("if ($F1) then case when (_MAP.R_NAME = 'AFRICA') then 2 else 4 end else if(4==3) then 1 else if(x==3) then 7 else (if(2==1) then 6 else 4 end) end");
+  public void testIfWithCase() {
+    testExpressionParsing("if ($F1) then case when (_MAP.R_NAME = 'AFRICA') then 2 else 4 end else if(4==3) then 1 else if(x==3) then 7 else (if(2==1) then 6 else 4 end) end",
+      "( if (equal(`x`, 3)  ) then (7 )  else ( ( if (equal(2, 1)  ) then (6 )  else (4 )  end  )  )  end  )");
   }
 
   @Test
-  public void testAdd() throws Exception{
-    testExpressionParsing("2+2");
+  public void testAdd() {
+    testExpressionParsing("2+2", "add(2, 2)");
   }
 
   @Test
-  public void testIf() throws Exception{
-    testExpressionParsing("if ('blue.red') then 'orange' else if (false) then 1 else 0 end");
+  public void testIf() {
+    testExpressionParsing("if ('blue.red') then 'orange' else if (false) then 1 else 0 end",
+      "( if (false ) then (1 )  else (0 )  end  )");
   }
 
   @Test
-  public void testQuotedIdentifier() throws Exception{
-    testExpressionParsing("`hello friend`.`goodbye`");
+  public void testQuotedIdentifier() {
+    String expr = "`hello friend`.`goodbye`";
+    testExpressionParsing(expr, expr);
   }
 
   @Test
-  public void testSpecialQuoted() throws Exception{
-    testExpressionParsing("`*0` + `*` ");
+  public void testSpecialQuoted() {
+    testExpressionParsing("`*0` + `*` ", "add(`*0`, `*`)");
   }
 
   @Test
-  public void testQuotedIdentifier2() throws Exception{
-    testExpressionParsing("`hello friend`.goodbye");
+  public void testQuotedIdentifier2() {
+    testExpressionParsing("`hello friend`.goodbye", "`hello friend`.`goodbye`");
   }
 
   @Test
-  public void testComplexIdentifier() throws Exception{
-    testExpressionParsing("goodbye[4].`hello`");
+  public void testComplexIdentifier() {
+    testExpressionParsing("goodbye[4].`hello`", "`goodbye`[4].`hello`");
   }
 
   @Test // DRILL-2606
-  public void testCastToBooleanExpr() throws Exception{
-    testExpressionParsing("cast( (cast( (`bool_col` ) as VARCHAR(100) ) ) as BIT )");
+  public void testCastToBooleanExpr() {
+    String expr = "cast( (cast( (`bool_col` ) as VARCHAR(100) ) ) as BIT )";
+    testExpressionParsing(expr, expr);
   }
 
-  private LogicalExpression parseExpression(String expr) throws RecognitionException, IOException{
+  @Test
+  public void testComments() {
+    testExpressionParsing("cast /* block comment */ ( // single comment\n" +
+      "1 as int)", "cast( (1 ) as INT )");
+  }
 
-    ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
-    CommonTokenStream tokens = new CommonTokenStream(lexer);
+  @Test
+  public void testParsingException() {
+    thrown.expect(ExpressionParsingException.class);
+    thrown.expectMessage(containsString("mismatched input 'i' expecting"));
+    testExpressionParsing("cast(1 as i)", "");
+  }
 
-    ExprParser parser = new ExprParser(tokens);
-    parse_return ret = parser.parse();
+  @Test
+  public void testFunctionCallWithoutParams() {
+    String expr = "now()";
+    testExpressionParsing(expr, expr);
+  }
 
-    return ret.e;
+  /**
+   * Attempt to parse an expression.  Once parsed, convert it to a string and then parse it again to make sure serialization works.
+   */
+  private void testExpressionParsing(String expr, String expected) {
+    LogicalExpression e1 = LogicalExpressionParser.parse(expr);
+    String newStringExpr = serializeExpression(e1);
+    assertEquals(expected, newStringExpr.trim());
+    LogicalExpressionParser.parse(newStringExpr);
   }
 
   private String serializeExpression(LogicalExpression expr){
-
     ExpressionStringBuilder b = new ExpressionStringBuilder();
     StringBuilder sb = new StringBuilder();
     expr.accept(b, sb);
     return sb.toString();
   }
 
-  /**
-   * Attempt to parse an expression.  Once parsed, convert it to a string and then parse it again to make sure serialization works.
-   * @param expr
-   * @throws RecognitionException
-   * @throws IOException
-   */
-  private void testExpressionParsing(String expr) throws RecognitionException, IOException{
-    logger.debug("-----" + expr + "-----");
-    LogicalExpression e = parseExpression(expr);
-
-    String newStringExpr = serializeExpression(e);
-    logger.debug(newStringExpr);
-    LogicalExpression e2 = parseExpression(newStringExpr);
-  }
 }
diff --git a/pom.xml b/pom.xml
index ca847d2..bd512c4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -89,6 +89,7 @@
     <rat.skip>true</rat.skip>
     <license.skip>true</license.skip>
     <docker.repository>drill/apache-drill</docker.repository>
+    <antlr.version>4.7.1</antlr.version>
   </properties>
 
   <scm>
@@ -1648,6 +1649,11 @@
           </exclusion>
         </exclusions>
       </dependency>
+      <dependency>
+        <groupId>org.antlr</groupId>
+        <artifactId>antlr4-runtime</artifactId>
+        <version>${antlr.version}</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
 
@@ -2708,11 +2714,6 @@
             </exclusions>
           </dependency>
           <dependency>
-            <groupId>org.antlr</groupId>
-            <artifactId>antlr4-runtime</artifactId>
-            <version>4.5</version>
-          </dependency>
-          <dependency>
             <groupId>org.apache.hive</groupId>
             <artifactId>hive-maprdb-json-handler</artifactId>
             <version>${hive.version}</version>


[drill] 10/15: DRILL-6858: Add functionality to list directories / files with exceptions suppression

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

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

commit d0ba8ec152976d6e7268dabfbbe0473cb909febe
Author: Arina Ielchiieva <ar...@gmail.com>
AuthorDate: Tue Nov 20 17:06:07 2018 +0200

    DRILL-6858: Add functionality to list directories / files with exceptions suppression
    
    1. Add listDirectoriesSafe, listFilesSafe, listAllSafe in FileSystemUtil and DrillFileSystemUtil classes.
    2. Use FileSystemUtil.listAllSafe during listing files in show files command and information_schema.files table.
    
    closes #1547
---
 .../planner/sql/handlers/ShowFilesHandler.java     |   5 +-
 .../store/ischema/InfoSchemaRecordGenerator.java   |   5 +-
 .../drill/exec/util/DrillFileSystemUtil.java       |  57 +++++++-
 .../org/apache/drill/exec/util/FileSystemUtil.java | 159 ++++++++++++++++-----
 .../impersonation/TestImpersonationMetadata.java   |  13 +-
 .../drill/exec/util/DrillFileSystemUtilTest.java   |  70 +++++----
 .../apache/drill/exec/util/FileSystemUtilTest.java | 122 ++++++----------
 .../drill/exec/util/FileSystemUtilTestBase.java    |   7 +-
 8 files changed, 256 insertions(+), 182 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFilesHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFilesHandler.java
index 75abfdd..9782bbf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFilesHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ShowFilesHandler.java
@@ -32,7 +32,6 @@ import org.apache.drill.exec.util.FileSystemUtil;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 import org.apache.hadoop.fs.Path;
 
-import java.io.IOException;
 import java.sql.Timestamp;
 import java.util.List;
 import java.util.stream.Collectors;
@@ -46,7 +45,7 @@ public class ShowFilesHandler extends DefaultSqlHandler {
   }
 
   @Override
-  public PhysicalPlan getPlan(SqlNode sqlNode) throws ForemanSetupException, IOException {
+  public PhysicalPlan getPlan(SqlNode sqlNode) throws ForemanSetupException {
     SchemaPlus defaultSchema = config.getConverter().getDefaultSchema();
     SchemaPlus drillSchema = defaultSchema;
     SqlShowFiles showFiles = unwrap(sqlNode, SqlShowFiles.class);
@@ -83,7 +82,7 @@ public class ShowFilesHandler extends DefaultSqlHandler {
     }
 
     Path path = new Path(wsSchema.getDefaultLocation(), fromDir);
-    List<ShowFilesCommandResult> records = FileSystemUtil.listAll(wsSchema.getFS(), path, false).stream()
+    List<ShowFilesCommandResult> records = FileSystemUtil.listAllSafe(wsSchema.getFS(), path, false).stream()
         // use ShowFilesCommandResult for backward compatibility
         .map(fileStatus -> new ShowFilesCommandResult(new Records.File(wsSchema.getFullSchemaName(), wsSchema, fileStatus)))
         .collect(Collectors.toList());
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 cfb918b..1e72840 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
@@ -31,7 +31,6 @@ import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SHRD_COL_T
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.SHRD_COL_TABLE_SCHEMA;
 import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.TBLS_COL_TABLE_TYPE;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -436,14 +435,12 @@ public abstract class InfoSchemaRecordGenerator<S> {
           String defaultLocation = wsSchema.getDefaultLocation();
           FileSystem fs = wsSchema.getFS();
           boolean recursive = optionManager.getBoolean(ExecConstants.LIST_FILES_RECURSIVELY);
-          FileSystemUtil.listAll(fs, new Path(defaultLocation), recursive).forEach(
+          FileSystemUtil.listAllSafe(fs, new Path(defaultLocation), recursive).forEach(
               fileStatus -> records.add(new Records.File(schemaName, wsSchema, fileStatus))
           );
         }
       } catch (ClassCastException | UnsupportedOperationException e) {
         // ignore the exception since either this is not a Drill schema or schema does not support files listing
-      } catch (IOException e) {
-        logger.warn("Failure while trying to list files", e);
       }
     }
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/DrillFileSystemUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/DrillFileSystemUtil.java
index 2fc135d..fcee3b0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/DrillFileSystemUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/DrillFileSystemUtil.java
@@ -36,12 +36,8 @@ public class DrillFileSystemUtil {
   /**
    * Path filter that skips all files and folders that start with dot or underscore.
    */
-  public static final PathFilter DRILL_SYSTEM_FILTER = new PathFilter() {
-    @Override
-    public boolean accept(Path path) {
-      return !path.getName().startsWith(DrillFileSystem.UNDERSCORE_PREFIX) && !path.getName().startsWith(DrillFileSystem.DOT_PREFIX);
-    }
-  };
+  public static final PathFilter DRILL_SYSTEM_FILTER = path ->
+    !path.getName().startsWith(DrillFileSystem.UNDERSCORE_PREFIX) && !path.getName().startsWith(DrillFileSystem.DOT_PREFIX);
 
   /**
    * Returns statuses of all directories present in given path applying custom filters if present.
@@ -59,6 +55,22 @@ public class DrillFileSystemUtil {
   }
 
   /**
+   * Returns statuses of all directories present in given path applying custom filters if present.
+   * Directories that start with dot or underscore are skipped.
+   * Will include nested directories if recursive flag is set to true.
+   * Will ignore all exceptions during listing if any.
+   *
+   * @param fs current file system
+   * @param path path to directory
+   * @param recursive true if nested directories should be included
+   * @param filters list of custom filters (optional)
+   * @return list of matching directory statuses
+   */
+  public static List<FileStatus> listDirectoriesSafe(final FileSystem fs, Path path, boolean recursive, PathFilter... filters) {
+    return FileSystemUtil.listDirectoriesSafe(fs, path, recursive, FileSystemUtil.mergeFilters(DRILL_SYSTEM_FILTER, filters));
+  }
+
+  /**
    * Returns statuses of all files present in given path applying custom filters if present.
    * Files and nested directories that start with dot or underscore are skipped.
    * Will also include files from nested directories if recursive flag is set to true.
@@ -74,6 +86,23 @@ public class DrillFileSystemUtil {
   }
 
   /**
+   * Returns statuses of all files present in given path applying custom filters if present.
+   * Files and nested directories that start with dot or underscore are skipped.
+   * Will include files from nested directories if recursive flag is set to true.
+   * Will ignore all exceptions during listing if any.
+   *
+   * @param fs current file system
+   * @param path path to file or directory
+   * @param recursive true if files in nested directories should be included
+   * @param filters list of custom filters (optional)
+   * @return list of matching file statuses
+   */
+  public static List<FileStatus> listFilesSafe(final FileSystem fs, Path path, boolean recursive, PathFilter... filters) {
+    return FileSystemUtil.listFilesSafe(fs, path, recursive, FileSystemUtil.mergeFilters(DRILL_SYSTEM_FILTER, filters));
+  }
+
+
+  /**
    * Returns statuses of all directories and files present in given path applying custom filters if present.
    * Directories and files that start with dot or underscore are skipped.
    * Will also include nested directories and their files if recursive flag is set to true.
@@ -88,4 +117,20 @@ public class DrillFileSystemUtil {
     return FileSystemUtil.listAll(fs, path, recursive, FileSystemUtil.mergeFilters(DRILL_SYSTEM_FILTER, filters));
   }
 
+  /**
+   * Returns statuses of all directories and files present in given path applying custom filters if present.
+   * Directories and files that start with dot or underscore are skipped.
+   * Will include nested directories and their files if recursive flag is set to true.
+   * Will ignore all exceptions during listing if any.
+   *
+   * @param fs current file system
+   * @param path path to file or directory
+   * @param recursive true if nested directories and their files should be included
+   * @param filters list of custom filters (optional)
+   * @return list of matching directory and file statuses
+   */
+  public static List<FileStatus> listAllSafe(FileSystem fs, Path path, boolean recursive, PathFilter... filters) {
+    return FileSystemUtil.listAllSafe(fs, path, recursive, FileSystemUtil.mergeFilters(DRILL_SYSTEM_FILTER, filters));
+  }
+
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/FileSystemUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/FileSystemUtil.java
index f17e215..47ac44c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/FileSystemUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/FileSystemUtil.java
@@ -26,6 +26,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.stream.Stream;
 
 /**
  * Helper class that provides methods to list directories or file or both statuses.
@@ -33,15 +34,12 @@ import java.util.List;
  */
 public class FileSystemUtil {
 
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FileSystemUtil.class);
+
   /**
    * Filter that will accept all files and directories.
    */
-  public static final PathFilter DUMMY_FILTER = new PathFilter() {
-    @Override
-    public boolean accept(Path path) {
-      return true;
-    }
-  };
+  public static final PathFilter DUMMY_FILTER = path -> true;
 
   /**
    * Returns statuses of all directories present in given path applying custom filters if present.
@@ -55,7 +53,28 @@ public class FileSystemUtil {
    */
   public static List<FileStatus> listDirectories(final FileSystem fs, Path path, boolean recursive, PathFilter... filters) throws IOException {
     List<FileStatus> statuses = new ArrayList<>();
-    listDirectories(fs, path, recursive, statuses, mergeFilters(filters));
+    listDirectories(fs, path, recursive, false, statuses, mergeFilters(filters));
+    return statuses;
+  }
+
+  /**
+   * Returns statuses of all directories present in given path applying custom filters if present.
+   * Will also include nested directories if recursive flag is set to true.
+   * Will ignore all exceptions during listing if any.
+   *
+   * @param fs current file system
+   * @param path path to directory
+   * @param recursive true if nested directories should be included
+   * @param filters list of custom filters (optional)
+   * @return list of matching directory statuses
+   */
+  public static List<FileStatus> listDirectoriesSafe(final FileSystem fs, Path path, boolean recursive, PathFilter... filters) {
+    List<FileStatus> statuses = new ArrayList<>();
+    try {
+      listDirectories(fs, path, recursive, true, statuses, mergeFilters(filters));
+    } catch (Exception e) {
+      // all exceptions are ignored
+    }
     return statuses;
   }
 
@@ -71,7 +90,27 @@ public class FileSystemUtil {
    */
   public static List<FileStatus> listFiles(FileSystem fs, Path path, boolean recursive, PathFilter... filters) throws IOException {
     List<FileStatus> statuses = new ArrayList<>();
-    listFiles(fs, path, recursive, statuses, mergeFilters(filters));
+    listFiles(fs, path, recursive, false, statuses, mergeFilters(filters));
+    return statuses;
+  }
+
+  /**
+   * Returns statuses of all files present in given path applying custom filters if present.
+   * Will also include files from nested directories if recursive flag is set to true.
+   *
+   * @param fs current file system
+   * @param path path to file or directory
+   * @param recursive true if files in nested directories should be included
+   * @param filters list of custom filters (optional)
+   * @return list of matching file statuses
+   */
+  public static List<FileStatus> listFilesSafe(FileSystem fs, Path path, boolean recursive, PathFilter... filters) {
+    List<FileStatus> statuses = new ArrayList<>();
+    try {
+      listFiles(fs, path, recursive, true, statuses, mergeFilters(filters));
+    } catch (Exception e) {
+      // all exceptions are ignored
+    }
     return statuses;
   }
 
@@ -87,7 +126,28 @@ public class FileSystemUtil {
    */
   public static List<FileStatus> listAll(FileSystem fs, Path path, boolean recursive, PathFilter... filters) throws IOException {
     List<FileStatus> statuses = new ArrayList<>();
-    listAll(fs, path, recursive, statuses, mergeFilters(filters));
+    listAll(fs, path, recursive, false, statuses, mergeFilters(filters));
+    return statuses;
+  }
+
+  /**
+   * Returns statuses of all directories and files present in given path applying custom filters if present.
+   * Will also include nested directories and their files if recursive flag is set to true.
+   * Will ignore all exceptions during listing if any.
+   *
+   * @param fs current file system
+   * @param path path to file or directory
+   * @param recursive true if nested directories and their files should be included
+   * @param filters list of custom filters (optional)
+   * @return list of matching directory and file statuses
+   */
+  public static List<FileStatus> listAllSafe(FileSystem fs, Path path, boolean recursive, PathFilter... filters) {
+    List<FileStatus> statuses = new ArrayList<>();
+    try {
+      listAll(fs, path, recursive, true, statuses, mergeFilters(filters));
+    } catch (Exception e) {
+      // all exceptions are ignored
+    }
     return statuses;
   }
 
@@ -122,39 +182,39 @@ public class FileSystemUtil {
       return DUMMY_FILTER;
     }
 
-    return new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        for (PathFilter filter : filters) {
-          if (!filter.accept(path)) {
-            return false;
-          }
-        }
-        return true;
-      }
-    };
+    return path -> Stream.of(filters).allMatch(filter -> filter.accept(path));
   }
 
   /**
    * Helper method that will store in given holder statuses of all directories present in given path applying custom filter.
    * If recursive flag is set to true, will call itself recursively to add statuses of nested directories.
+   * If suppress exceptions flag is set to true, will ignore all exceptions during listing.
    *
    * @param fs current file system
    * @param path path to directory
    * @param recursive true if nested directories should be included
+   * @param suppressExceptions indicates if exceptions should be ignored during listing
    * @param statuses holder for directory statuses
    * @param filter custom filter
    * @return holder with all matching directory statuses
    */
-  private static List<FileStatus> listDirectories(FileSystem fs, Path path, boolean recursive, List<FileStatus> statuses, PathFilter filter) throws IOException {
-    FileStatus[] fileStatuses = fs.listStatus(path, filter);
-    for (FileStatus status: fileStatuses) {
-      if (status.isDirectory()) {
-        statuses.add(status);
-        if (recursive) {
-          listDirectories(fs, status.getPath(), true, statuses, filter);
+  private static List<FileStatus> listDirectories(FileSystem fs, Path path, boolean recursive, boolean suppressExceptions,
+                                                  List<FileStatus> statuses, PathFilter filter) throws IOException {
+    try {
+      for (FileStatus status : fs.listStatus(path, filter)) {
+        if (status.isDirectory()) {
+          statuses.add(status);
+          if (recursive) {
+            listDirectories(fs, status.getPath(), true, suppressExceptions, statuses, filter);
+          }
         }
       }
+    } catch (Exception e) {
+      if (suppressExceptions) {
+        logger.debug("Exception during listing file statuses", e);
+      } else {
+        throw e;
+      }
     }
     return statuses;
   }
@@ -162,23 +222,33 @@ public class FileSystemUtil {
   /**
    * Helper method that will store in given holder statuses of all files present in given path applying custom filter.
    * If recursive flag is set to true, will call itself recursively to add file statuses from nested directories.
+   * If suppress exceptions flag is set to true, will ignore all exceptions during listing.
    *
    * @param fs current file system
    * @param path path to file or directory
    * @param recursive true if files in nested directories should be included
+   * @param suppressExceptions indicates if exceptions should be ignored during listing
    * @param statuses holder for file statuses
    * @param filter custom filter
    * @return holder with all matching file statuses
    */
-  private static List<FileStatus> listFiles(FileSystem fs, Path path, boolean recursive, List<FileStatus> statuses, PathFilter filter) throws IOException {
-    FileStatus[] fileStatuses = fs.listStatus(path, filter);
-    for (FileStatus status: fileStatuses) {
-      if (status.isDirectory()) {
-        if (recursive) {
-          listFiles(fs, status.getPath(), true, statuses, filter);
+  private static List<FileStatus> listFiles(FileSystem fs, Path path, boolean recursive, boolean suppressExceptions,
+                                            List<FileStatus> statuses, PathFilter filter) throws IOException {
+    try {
+      for (FileStatus status : fs.listStatus(path, filter)) {
+        if (status.isDirectory()) {
+          if (recursive) {
+            listFiles(fs, status.getPath(), true, suppressExceptions, statuses, filter);
+          }
+        } else {
+          statuses.add(status);
         }
+      }
+    } catch (Exception e) {
+      if (suppressExceptions) {
+        logger.debug("Exception during listing file statuses", e);
       } else {
-        statuses.add(status);
+        throw e;
       }
     }
     return statuses;
@@ -187,19 +257,30 @@ public class FileSystemUtil {
   /**
    * Helper method that will store in given holder statuses of all directories and files present in given path applying custom filter.
    * If recursive flag is set to true, will call itself recursively to add nested directories and their file statuses.
+   * If suppress exceptions flag is set to true, will ignore all exceptions during listing.
    *
    * @param fs current file system
    * @param path path to file or directory
    * @param recursive true if nested directories and their files should be included
+   * @param suppressExceptions indicates if exceptions should be ignored during listing
    * @param statuses holder for directory and file statuses
    * @param filter custom filter
    * @return holder with all matching directory and file statuses
    */
-  private static List<FileStatus> listAll(FileSystem fs, Path path, boolean recursive, List<FileStatus> statuses, PathFilter filter) throws IOException {
-    for (FileStatus status: fs.listStatus(path, filter)) {
-      statuses.add(status);
-      if (status.isDirectory() && recursive) {
-        listAll(fs, status.getPath(), true, statuses, filter);
+  private static List<FileStatus> listAll(FileSystem fs, Path path, boolean recursive, boolean suppressExceptions,
+                                          List<FileStatus> statuses, PathFilter filter) throws IOException {
+    try {
+      for (FileStatus status : fs.listStatus(path, filter)) {
+        statuses.add(status);
+        if (status.isDirectory() && recursive) {
+          listAll(fs, status.getPath(), true, suppressExceptions, statuses, filter);
+        }
+      }
+    } catch (Exception e) {
+      if (suppressExceptions) {
+        logger.debug("Exception during listing file statuses", e);
+      } else {
+        throw e;
       }
     }
     return statuses;
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 6dc3615..03754e5 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
@@ -30,7 +30,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
@@ -40,6 +39,7 @@ import org.junit.rules.ExpectedException;
 import java.util.Map;
 
 import static org.hamcrest.core.StringContains.containsString;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 /**
@@ -131,10 +131,10 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
     try {
       test("drop table parquet_table_700");
     } catch (UserException e) {
-      Assert.assertTrue(e.getMessage().contains("PERMISSION ERROR"));
+      assertTrue(e.getMessage().contains("PERMISSION ERROR"));
       dropFailed = true;
     }
-    Assert.assertTrue("Permission checking failed during drop table", dropFailed);
+    assertTrue("Permission checking failed during drop table", dropFailed);
   }
 
   @Test // DRILL-3037
@@ -176,12 +176,9 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
   @Test
   public void testShowFilesInWSWithNoPermissionsForQueryUser() throws Exception {
     updateClient(user2);
-
-    thrown.expect(UserRemoteException.class);
-    thrown.expectMessage(containsString("Permission denied: user=drillTestUser2, " +
-        "access=READ_EXECUTE, inode=\"/drill_test_grp_1_700\":drillTestUser1:drill_test_grp_1:drwx------"));
     // Try show tables in schema "drill_test_grp_1_700" which is owned by "user1"
-    test("SHOW FILES IN %s.drill_test_grp_1_700", MINI_DFS_STORAGE_PLUGIN_NAME);
+    int count = testSql(String.format("SHOW FILES IN %s.drill_test_grp_1_700", MINI_DFS_STORAGE_PLUGIN_NAME));
+    assertEquals(0, count);
   }
 
   @Test
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/util/DrillFileSystemUtilTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/util/DrillFileSystemUtilTest.java
index 175929e..8ec47be 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/util/DrillFileSystemUtilTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/util/DrillFileSystemUtilTest.java
@@ -27,6 +27,7 @@ import java.util.Collections;
 import java.util.List;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 public class DrillFileSystemUtilTest extends FileSystemUtilTestBase {
 
@@ -38,12 +39,8 @@ public class DrillFileSystemUtilTest extends FileSystemUtilTestBase {
 
   @Test
   public void testListDirectoriesWithFilter() throws IOException {
-    List<FileStatus> statuses = DrillFileSystemUtil.listDirectories(fs, base, false, new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().endsWith("a");
-      }
-    });
+    List<FileStatus> statuses = DrillFileSystemUtil.listDirectories(fs, base, false,
+      (PathFilter) path -> path.getName().endsWith("a"));
     assertEquals("Directory count should match", 1, statuses.size());
     assertEquals("Directory name should match", "a", statuses.get(0).getPath().getName());
   }
@@ -56,12 +53,8 @@ public class DrillFileSystemUtilTest extends FileSystemUtilTestBase {
 
   @Test
   public void testListDirectoriesRecursiveWithFilter() throws IOException {
-    List<FileStatus> statuses = DrillFileSystemUtil.listDirectories(fs, base, true, new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().endsWith("a");
-      }
-    });
+    List<FileStatus> statuses = DrillFileSystemUtil.listDirectories(fs, base, true,
+      (PathFilter) path -> path.getName().endsWith("a"));
     assertEquals("Directory count should match", 2, statuses.size());
 
     Collections.sort(statuses);
@@ -78,12 +71,8 @@ public class DrillFileSystemUtilTest extends FileSystemUtilTestBase {
 
   @Test
   public void testListFilesWithFilter() throws IOException {
-    List<FileStatus> statuses = DrillFileSystemUtil.listFiles(fs, new Path(base, "a"), false, new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().endsWith(".txt");
-      }
-    });
+    List<FileStatus> statuses = DrillFileSystemUtil.listFiles(fs, new Path(base, "a"), false,
+      (PathFilter) path -> path.getName().endsWith(".txt"));
     assertEquals("File count should match", 1, statuses.size());
     assertEquals("File name should match", "f.txt", statuses.get(0).getPath().getName());
   }
@@ -96,12 +85,8 @@ public class DrillFileSystemUtilTest extends FileSystemUtilTestBase {
 
   @Test
   public void testListFilesRecursiveWithFilter() throws IOException {
-    List<FileStatus> statuses = DrillFileSystemUtil.listFiles(fs, base, true, new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().endsWith("a") || path.getName().endsWith(".txt");
-      }
-    });
+    List<FileStatus> statuses = DrillFileSystemUtil.listFiles(fs, base, true,
+      (PathFilter) path -> path.getName().endsWith("a") || path.getName().endsWith(".txt"));
     assertEquals("File count should match", 2, statuses.size());
 
     Collections.sort(statuses);
@@ -121,12 +106,8 @@ public class DrillFileSystemUtilTest extends FileSystemUtilTestBase {
 
   @Test
   public void testListAllWithFilter() throws IOException {
-    List<FileStatus> statuses = DrillFileSystemUtil.listAll(fs, new Path(base, "a"), false, new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().endsWith("a") || path.getName().endsWith(".txt");
-      }
-    });
+    List<FileStatus> statuses = DrillFileSystemUtil.listAll(fs, new Path(base, "a"), false,
+      (PathFilter) path -> path.getName().endsWith("a") || path.getName().endsWith(".txt"));
     assertEquals("Directory and file count should match", 2, statuses.size());
 
     Collections.sort(statuses);
@@ -142,12 +123,8 @@ public class DrillFileSystemUtilTest extends FileSystemUtilTestBase {
 
   @Test
   public void testListAllRecursiveWithFilter() throws IOException {
-    List<FileStatus> statuses = DrillFileSystemUtil.listAll(fs, new Path(base, "a"), true, new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().startsWith("a") || path.getName().endsWith(".txt");
-      }
-    });
+    List<FileStatus> statuses = DrillFileSystemUtil.listAll(fs, new Path(base, "a"), true,
+      (PathFilter) path -> path.getName().startsWith("a") || path.getName().endsWith(".txt"));
     assertEquals("Directory and file count should match", 3, statuses.size());
 
     Collections.sort(statuses);
@@ -156,4 +133,25 @@ public class DrillFileSystemUtilTest extends FileSystemUtilTestBase {
     assertEquals("File name should match", "f.txt", statuses.get(2).getPath().getName());
   }
 
+  @Test
+  public void testListDirectoriesSafe() {
+    Path file = new Path(base, "missing");
+    List<FileStatus> fileStatuses = DrillFileSystemUtil.listDirectoriesSafe(fs, file, true);
+    assertTrue("Should return empty result", fileStatuses.isEmpty());
+  }
+
+  @Test
+  public void testListFilesSafe() {
+    Path file = new Path(base, "missing.txt");
+    List<FileStatus> fileStatuses = DrillFileSystemUtil.listFilesSafe(fs, file, true);
+    assertTrue("Should return empty result", fileStatuses.isEmpty());
+  }
+
+  @Test
+  public void testListAllSafe() {
+    Path file = new Path(base, "missing");
+    List<FileStatus> fileStatuses = DrillFileSystemUtil.listAllSafe(fs, file, true);
+    assertTrue("Should return empty result", fileStatuses.isEmpty());
+  }
+
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/util/FileSystemUtilTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/util/FileSystemUtilTest.java
index 764f1d7..299f1cc 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/util/FileSystemUtilTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/util/FileSystemUtilTest.java
@@ -40,12 +40,8 @@ public class FileSystemUtilTest extends FileSystemUtilTestBase {
 
   @Test
   public void testListDirectoriesWithFilter() throws IOException {
-    List<FileStatus> statuses = FileSystemUtil.listDirectories(fs, base, false, new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().endsWith("a");
-      }
-    });
+    List<FileStatus> statuses = FileSystemUtil.listDirectories(fs, base, false,
+      (PathFilter) path -> path.getName().endsWith("a"));
     assertEquals("Directory count should match", 3, statuses.size());
 
     Collections.sort(statuses);
@@ -62,12 +58,8 @@ public class FileSystemUtilTest extends FileSystemUtilTestBase {
 
   @Test
   public void testListDirectoriesRecursiveWithFilter() throws IOException {
-    List<FileStatus> statuses = FileSystemUtil.listDirectories(fs, base, true, new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().endsWith("a");
-      }
-    });
+    List<FileStatus> statuses = FileSystemUtil.listDirectories(fs, base, true,
+      (PathFilter) path -> path.getName().endsWith("a"));
     assertEquals("Directory count should match", 4, statuses.size());
 
     Collections.sort(statuses);
@@ -79,12 +71,8 @@ public class FileSystemUtilTest extends FileSystemUtilTestBase {
 
   @Test
   public void testListDirectoriesEmptyResult() throws IOException {
-    List<FileStatus> statuses = FileSystemUtil.listDirectories(fs, base, false, new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().startsWith("abc");
-      }
-    });
+    List<FileStatus> statuses = FileSystemUtil.listDirectories(fs, base, false,
+      (PathFilter) path -> path.getName().startsWith("abc"));
     assertEquals("Directory count should match", 0, statuses.size());
   }
 
@@ -96,12 +84,8 @@ public class FileSystemUtilTest extends FileSystemUtilTestBase {
 
   @Test
   public void testListFilesWithFilter() throws IOException {
-    List<FileStatus> statuses = FileSystemUtil.listFiles(fs, new Path(base, "a"), false, new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().endsWith(".txt");
-      }
-    });
+    List<FileStatus> statuses = FileSystemUtil.listFiles(fs, new Path(base, "a"), false,
+      (PathFilter) path -> path.getName().endsWith(".txt"));
     assertEquals("File count should match", 3, statuses.size());
 
     Collections.sort(statuses);
@@ -118,12 +102,8 @@ public class FileSystemUtilTest extends FileSystemUtilTestBase {
 
   @Test
   public void testListFilesRecursiveWithFilter() throws IOException {
-    List<FileStatus> statuses = FileSystemUtil.listFiles(fs, base, true, new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().endsWith("a") || path.getName().endsWith(".txt");
-      }
-    });
+    List<FileStatus> statuses = FileSystemUtil.listFiles(fs, base, true,
+      (PathFilter) path -> path.getName().endsWith("a") || path.getName().endsWith(".txt"));
 
     assertEquals("File count should match", 8, statuses.size());
   }
@@ -142,12 +122,8 @@ public class FileSystemUtilTest extends FileSystemUtilTestBase {
 
   @Test
   public void testListAllWithFilter() throws IOException {
-    List<FileStatus> statuses = FileSystemUtil.listAll(fs, new Path(base, "a"), false, new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().endsWith("a") || path.getName().endsWith(".txt");
-      }
-    });
+    List<FileStatus> statuses = FileSystemUtil.listAll(fs, new Path(base, "a"), false,
+      (PathFilter) path -> path.getName().endsWith("a") || path.getName().endsWith(".txt"));
     assertEquals("Directory and file count should match", 4, statuses.size());
   }
 
@@ -159,34 +135,21 @@ public class FileSystemUtilTest extends FileSystemUtilTestBase {
 
   @Test
   public void testListAllRecursiveWithFilter() throws IOException {
-    List<FileStatus> statuses = FileSystemUtil.listAll(fs, new Path(base, "a"), true, new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().endsWith("a") || path.getName().endsWith(".txt");
-      }
-    });
+    List<FileStatus> statuses = FileSystemUtil.listAll(fs, new Path(base, "a"), true,
+      (PathFilter) path -> path.getName().endsWith("a") || path.getName().endsWith(".txt"));
     assertEquals("Directory and file count should match", 7, statuses.size());
   }
 
   @Test
   public void testListAllEmptyResult() throws IOException {
-    List<FileStatus> statuses = FileSystemUtil.listAll(fs, base, false, new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().startsWith("xyz");
-      }
-    });
+    List<FileStatus> statuses = FileSystemUtil.listAll(fs, base, false,
+      (PathFilter) path -> path.getName().startsWith("xyz"));
     assertEquals("Directory and file count should match", 0, statuses.size());
   }
 
   @Test
   public void testMergeFiltersWithMissingParameters() {
-    PathFilter filter = new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().startsWith("a");
-      }
-    };
+    PathFilter filter = path -> path.getName().startsWith("a");
 
     assertEquals("Should have returned initial filter", filter, FileSystemUtil.mergeFilters(filter, null));
     assertEquals("Should have returned initial filter", filter, FileSystemUtil.mergeFilters(filter, new PathFilter[]{}));
@@ -197,19 +160,8 @@ public class FileSystemUtilTest extends FileSystemUtilTestBase {
   public void mergeFiltersTrue() {
     Path file = new Path("abc.txt");
 
-    PathFilter firstFilter = new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().startsWith("a");
-      }
-    };
-
-    PathFilter secondFilter = new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().endsWith(".txt");
-      }
-    };
+    PathFilter firstFilter = path -> path.getName().startsWith("a");
+    PathFilter secondFilter = path -> path.getName().endsWith(".txt");
 
     assertTrue("Path should have been included in the path list", FileSystemUtil.mergeFilters(firstFilter, secondFilter).accept(file));
     assertTrue("Path should have been included in the path list", FileSystemUtil.mergeFilters(firstFilter, new PathFilter[] {secondFilter}).accept(file));
@@ -219,22 +171,32 @@ public class FileSystemUtilTest extends FileSystemUtilTestBase {
   public void mergeFiltersFalse() {
     Path file = new Path("abc.txt");
 
-    PathFilter firstFilter = new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().startsWith("a");
-      }
-    };
-
-    PathFilter secondFilter = new PathFilter() {
-      @Override
-      public boolean accept(Path path) {
-        return path.getName().endsWith(".csv");
-      }
-    };
+    PathFilter firstFilter = path -> path.getName().startsWith("a");
+    PathFilter secondFilter = path -> path.getName().endsWith(".csv");
 
     assertFalse("Path should have been excluded from the path list", FileSystemUtil.mergeFilters(firstFilter, secondFilter).accept(file));
     assertFalse("Path should have been excluded from the path list", FileSystemUtil.mergeFilters(firstFilter, new PathFilter[] {secondFilter}).accept(file));
   }
 
+  @Test
+  public void testListDirectoriesSafe() {
+    Path file = new Path(base, "missing");
+    List<FileStatus> fileStatuses = FileSystemUtil.listDirectoriesSafe(fs, file, true);
+    assertTrue("Should return empty result", fileStatuses.isEmpty());
+  }
+
+  @Test
+  public void testListFilesSafe() {
+    Path file = new Path(base, "missing.txt");
+    List<FileStatus> fileStatuses = FileSystemUtil.listFilesSafe(fs, file, true);
+    assertTrue("Should return empty result", fileStatuses.isEmpty());
+  }
+
+  @Test
+  public void testListAllSafe() {
+    Path file = new Path(base, "missing");
+    List<FileStatus> fileStatuses = FileSystemUtil.listAllSafe(fs, file, true);
+    assertTrue("Should return empty result", fileStatuses.isEmpty());
+  }
+
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/util/FileSystemUtilTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/exec/util/FileSystemUtilTestBase.java
index 22cf3ea..5081bfd 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/util/FileSystemUtilTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/util/FileSystemUtilTestBase.java
@@ -68,12 +68,7 @@ public class FileSystemUtilTestBase {
 
     // create temporary directory with sub-folders and files
     final File tempDir = Files.createTempDir();
-    Runtime.getRuntime().addShutdownHook(new Thread() {
-      @Override
-      public void run() {
-        FileUtils.deleteQuietly(tempDir);
-      }
-    });
+    Runtime.getRuntime().addShutdownHook(new Thread(() -> FileUtils.deleteQuietly(tempDir)));
     base = new Path(tempDir.toURI().getPath());
 
     createDefaultStructure(fs, base, "a", 2);


[drill] 09/15: DRILL-6857: Read only required row groups in a file when limit push down is applied

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

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

commit 8bd81920c05a46e567526f1f913ce70552204734
Author: Arina Ielchiieva <ar...@gmail.com>
AuthorDate: Tue Nov 20 20:08:38 2018 +0200

    DRILL-6857: Read only required row groups in a file when limit push down is applied
    
    closes #1548
---
 .../store/parquet/AbstractParquetGroupScan.java    | 91 +++++++++++-----------
 .../store/parquet/TestParquetLimitPushDown.java    | 75 ++++++++++++++++++
 2 files changed, 119 insertions(+), 47 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
index 9bc969f..0d35ddb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
@@ -246,7 +246,6 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
     final Set<SchemaPath> schemaPathsInExpr = filterExpr.accept(new ParquetRGFilterEvaluator.FieldReferenceFinder(), null);
 
     final List<RowGroupInfo> qualifiedRGs = new ArrayList<>(rowGroupInfos.size());
-    Set<String> qualifiedFilePath = new HashSet<>(); // HashSet keeps a fileName unique.
 
     ParquetFilterPredicate filterPredicate = null;
 
@@ -289,17 +288,15 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
       rowGroup.setRowsMatch(match);
 
       qualifiedRGs.add(rowGroup);
-      qualifiedFilePath.add(rowGroup.getPath());
     }
 
     if (qualifiedRGs.size() == rowGroupInfos.size() ) {
       // There is no reduction of rowGroups. Return the original groupScan.
-      logger.debug("applyFilter does not have any pruning!");
+      logger.debug("applyFilter() does not have any pruning!");
       return null;
-    } else if (qualifiedFilePath.size() == 0) {
-      logger.debug("All rowgroups have been filtered out. Add back one to get schema from scannner");
+    } else if (qualifiedRGs.size() == 0) {
+      logger.debug("All row groups have been filtered out. Add back one to get schema from scanner.");
       RowGroupInfo rg = rowGroupInfos.iterator().next();
-      qualifiedFilePath.add(rg.getPath());
       qualifiedRGs.add(rg);
     }
 
@@ -307,11 +304,7 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
       ExpressionStringBuilder.toString(filterExpr), rowGroupInfos.size(), qualifiedRGs.size());
 
     try {
-      AbstractParquetGroupScan cloneGroupScan = cloneWithFileSelection(qualifiedFilePath);
-      cloneGroupScan.rowGroupInfos = qualifiedRGs;
-      cloneGroupScan.parquetGroupScanStatistics.collect(cloneGroupScan.rowGroupInfos, cloneGroupScan.parquetTableMetadata);
-      return cloneGroupScan;
-
+      return cloneWithRowGroupInfos(qualifiedRGs);
     } catch (IOException e) {
       logger.warn("Could not apply filter prune due to Exception : {}", e);
       return null;
@@ -330,29 +323,41 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
     maxRecords = Math.max(maxRecords, 1); // Make sure it request at least 1 row -> 1 rowGroup.
     // further optimization : minimize # of files chosen, or the affinity of files chosen.
 
+    if (parquetGroupScanStatistics.getRowCount() <= maxRecords) {
+      logger.debug("limit push down does not apply, since total number of rows [{}] is less or equal to the required [{}].",
+        parquetGroupScanStatistics.getRowCount(), maxRecords);
+      return null;
+    }
+
     // Calculate number of rowGroups to read based on maxRecords and update
     // number of records to read for each of those rowGroups.
-    int index = updateRowGroupInfo(maxRecords);
-
-    Set<String> filePaths = rowGroupInfos.subList(0, index).stream()
-        .map(ReadEntryWithPath::getPath)
-        .collect(Collectors.toSet()); // HashSet keeps a filePath unique.
+    List<RowGroupInfo> qualifiedRowGroupInfos = new ArrayList<>(rowGroupInfos.size());
+    int currentRowCount = 0;
+    for (RowGroupInfo rowGroupInfo : rowGroupInfos) {
+      long rowCount = rowGroupInfo.getRowCount();
+      if (currentRowCount + rowCount <= maxRecords) {
+        currentRowCount += rowCount;
+        rowGroupInfo.setNumRecordsToRead(rowCount);
+        qualifiedRowGroupInfos.add(rowGroupInfo);
+        continue;
+      } else if (currentRowCount < maxRecords) {
+        rowGroupInfo.setNumRecordsToRead(maxRecords - currentRowCount);
+        qualifiedRowGroupInfos.add(rowGroupInfo);
+      }
+      break;
+    }
 
-    // If there is no change in fileSet, no need to create new groupScan.
-    if (filePaths.size() == fileSet.size() ) {
-      // There is no reduction of rowGroups. Return the original groupScan.
-      logger.debug("applyLimit() does not apply!");
+    if (rowGroupInfos.size() == qualifiedRowGroupInfos.size()) {
+      logger.debug("limit push down does not apply, since number of row groups was not reduced.");
       return null;
     }
 
-    logger.debug("applyLimit() reduce parquet file # from {} to {}", fileSet.size(), filePaths.size());
+    logger.debug("applyLimit() reduce parquet row groups # from {} to {}.", rowGroupInfos.size(), qualifiedRowGroupInfos.size());
 
     try {
-      AbstractParquetGroupScan newScan = cloneWithFileSelection(filePaths);
-      newScan.updateRowGroupInfo(maxRecords);
-      return newScan;
+      return cloneWithRowGroupInfos(qualifiedRowGroupInfos);
     } catch (IOException e) {
-      logger.warn("Could not apply rowcount based prune due to Exception : {}", e);
+      logger.warn("Could not apply row count based prune due to Exception: {}", e);
       return null;
     }
   }
@@ -454,30 +459,22 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
 
   // private methods block start
   /**
-   * Based on maxRecords to read for the scan,
-   * figure out how many rowGroups to read
-   * and update number of records to read for each of them.
+   * Clones current group scan with set of file paths from given row groups,
+   * updates new scan with list of given row groups,
+   * re-calculates statistics and endpoint affinities.
    *
-   * @param maxRecords max records to read
-   * @return total number of rowGroups to read
+   * @param rowGroupInfos list of row group infos
+   * @return new parquet group scan
    */
-  private int updateRowGroupInfo(int maxRecords) {
-    long count = 0;
-    int index = 0;
-    for (RowGroupInfo rowGroupInfo : rowGroupInfos) {
-      long rowCount = rowGroupInfo.getRowCount();
-      if (count + rowCount <= maxRecords) {
-        count += rowCount;
-        rowGroupInfo.setNumRecordsToRead(rowCount);
-        index++;
-        continue;
-      } else if (count < maxRecords) {
-        rowGroupInfo.setNumRecordsToRead(maxRecords - count);
-        index++;
-      }
-      break;
-    }
-    return index;
+  private AbstractParquetGroupScan cloneWithRowGroupInfos(List<RowGroupInfo> rowGroupInfos) throws IOException {
+    Set<String> filePaths = rowGroupInfos.stream()
+      .map(ReadEntryWithPath::getPath)
+      .collect(Collectors.toSet()); // set keeps file names unique
+    AbstractParquetGroupScan scan = cloneWithFileSelection(filePaths);
+    scan.rowGroupInfos = rowGroupInfos;
+    scan.parquetGroupScanStatistics.collect(scan.rowGroupInfos, scan.parquetTableMetadata);
+    scan.endpointAffinities = AffinityCreator.getAffinityMap(scan.rowGroupInfos);
+    return scan;
   }
   // private methods block end
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetLimitPushDown.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetLimitPushDown.java
new file mode 100644
index 0000000..7749796
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetLimitPushDown.java
@@ -0,0 +1,75 @@
+/*
+ * 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.parquet;
+
+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 java.nio.file.Paths;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestParquetLimitPushDown extends ClusterTest {
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    ClusterFixtureBuilder builder = ClusterFixture.builder(dirTestWatcher);
+    dirTestWatcher.copyResourceToRoot(Paths.get("parquet", "multirowgroup.parquet"));
+    dirTestWatcher.copyResourceToRoot(Paths.get("parquet", "users"));
+    startCluster(builder);
+  }
+
+  @Test
+  public void testMultipleFiles() throws Exception {
+    String query = "select * from dfs.`parquet/users` limit 1";
+    QueryBuilder.QuerySummary summary = queryBuilder().sql(query).run();
+    assertTrue(summary.succeeded());
+    assertEquals(1, summary.recordCount());
+
+    String plan = queryBuilder().sql(query).explainText();
+    assertTrue(plan.contains("numRowGroups=1"));
+  }
+
+  @Test
+  public void testMultipleRowGroups() throws Exception {
+    String query = "select * from dfs.`parquet/multirowgroup.parquet` limit 1";
+    QueryBuilder.QuerySummary summary = queryBuilder().sql(query).run();
+    assertTrue(summary.succeeded());
+    assertEquals(1, summary.recordCount());
+
+    String plan = queryBuilder().sql(query).explainText();
+    assertTrue(plan.contains("numRowGroups=1"));
+  }
+
+  @Test
+  public void testLimitZero() throws Exception {
+    String query = "select * from dfs.`parquet/users` limit 0";
+    QueryBuilder.QuerySummary summary = queryBuilder().sql(query).run();
+    assertTrue(summary.succeeded());
+    assertEquals(0, summary.recordCount());
+
+    String plan = queryBuilder().sql(query).explainText();
+    assertTrue(plan.contains("numRowGroups=1"));
+  }
+
+}


[drill] 02/15: DRILL-6668: In Web UI, highlight options that are not default values

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

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

commit 5a8e6d782ace8fe8f3a772b2d05ea5380d7756d3
Author: Kunal Khatua <kk...@maprtech.com>
AuthorDate: Wed Nov 14 22:28:48 2018 -0800

    DRILL-6668: In Web UI, highlight options that are not default values
    
    This commit introduces a new button on the options page that allows a user to reset an option to its system default value.
    To simplify things, a tooltip is shown when the mouse hovers over the button. If the option value is already default, the button is disabled.
    
    Currently, the Update button redirects to /option/optionName . This change reuses what we already are using to set the default (using AJAX) and auto-refreshing
    Switch [Default] label to [Reset]
    Patch To Pass StatusResourcesTest
    
    closes #1543
---
 .../org/apache/drill/exec/server/rest/StatusResources.java    | 11 +++++++++--
 exec/java-exec/src/main/resources/rest/options.ftl            |  8 ++++++++
 2 files changed, 17 insertions(+), 2 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StatusResources.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StatusResources.java
index e6b116c..1ebef31 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StatusResources.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/StatusResources.java
@@ -90,7 +90,7 @@ public class StatusResources {
     OptionList optionList = internal ? optionManager.getInternalOptionList(): optionManager.getPublicOptionList();
 
     for (OptionValue option : optionList) {
-      options.add(new OptionWrapper(option.name, option.getValue(), option.accessibleScopes, option.kind, option.scope));
+      options.add(new OptionWrapper(option.name, option.getValue(), optionManager.getDefault(option.name).getValue().toString(), option.accessibleScopes, option.kind, option.scope));
     }
 
     Collections.sort(options, new Comparator<OptionWrapper>() {
@@ -171,6 +171,7 @@ public class StatusResources {
 
     private String name;
     private Object value;
+    private String defaultValue;
     private OptionValue.AccessibleScopes accessibleScopes;
     private String kind;
     private String optionScope;
@@ -178,11 +179,13 @@ public class StatusResources {
     @JsonCreator
     public OptionWrapper(@JsonProperty("name") String name,
                          @JsonProperty("value") Object value,
+                         @JsonProperty("defaultValue") String defaultValue,
                          @JsonProperty("accessibleScopes") OptionValue.AccessibleScopes type,
                          @JsonProperty("kind") Kind kind,
                          @JsonProperty("optionScope") OptionValue.OptionScope scope) {
       this.name = name;
       this.value = value;
+      this.defaultValue = defaultValue;
       this.accessibleScopes = type;
       this.kind = kind.name();
       this.optionScope = scope.name();
@@ -201,6 +204,10 @@ public class StatusResources {
       return value;
     }
 
+    public String getDefaultValue() {
+      return defaultValue;
+    }
+
     public OptionValue.AccessibleScopes getAccessibleScopes() {
       return accessibleScopes;
     }
@@ -215,7 +222,7 @@ public class StatusResources {
 
     @Override
     public String toString() {
-      return "OptionWrapper{" + "name='" + name + '\'' + ", value=" + value + ", accessibleScopes=" + accessibleScopes + ", kind='" + kind + '\'' + ", scope='" + optionScope + '\'' +'}';
+      return "OptionWrapper{" + "name='" + name + '\'' + ", value=" + value + ", default=" + defaultValue + ", accessibleScopes=" + accessibleScopes + ", kind='" + kind + '\'' + ", scope='" + optionScope + '\'' +'}';
     }
   }
 }
diff --git a/exec/java-exec/src/main/resources/rest/options.ftl b/exec/java-exec/src/main/resources/rest/options.ftl
index 085fb45..7ac8a2e 100644
--- a/exec/java-exec/src/main/resources/rest/options.ftl
+++ b/exec/java-exec/src/main/resources/rest/options.ftl
@@ -21,6 +21,11 @@
 <#macro page_head>
     <script type="text/javascript" language="javascript"  src="/static/js/jquery.dataTables-1.10.16.min.js"> </script>
     <script type="text/javascript" language="javascript" src="/static/js/dataTables.colVis-1.1.0.min.js"></script>
+    <script>
+        function resetToDefault(optionName, optionValue, optionKind) {
+            $.post("/option/"+optionName, {kind: optionKind, name: optionName, value: optionValue}, function (status) { location.reload(true); } );
+        }
+    </script>
     <!-- List of Option Descriptions -->
     <script src="/dynamic/options.describe.js"></script>
     <link href="/static/css/dataTables.colVis-1.1.0.min.css" rel="stylesheet">
@@ -85,6 +90,9 @@ table.sortable thead .sorting_desc { background-image: url("/static/img/black-de
                   </#if>
                     <div class="input-group-btn">
                       <button class="btn btn-default" type="submit">Update</button>
+                      <button class="btn btn-default" onClick="resetToDefault('${option.getName()}','${option.getDefaultValue()}', '${option.getKind()}')" type="button"
+                              <#if option.getDefaultValue() == option.getValueAsString()>disabled="true" style="pointer-events:none" <#else>
+                      title="Reset to ${option.getDefaultValue()}"</#if>>Reset</button>
                     </div>
                   </div>
                 </div>


[drill] 01/15: DRILL-6349: Drill JDBC driver fails on Java 1.9+ with NoClassDefFoundError: sun/misc/VM

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

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

commit 639be6c1cdb2747e3d4a7d04ce5966d27d1f3102
Author: ozinoviev <oz...@solit-clouds.ru>
AuthorDate: Mon Aug 27 15:48:52 2018 +0300

    DRILL-6349: Drill JDBC driver fails on Java 1.9+ with NoClassDefFoundError: sun/misc/VM
    
    closes #1446
---
 .../apache/drill/common/config/DrillConfig.java    |  3 +-
 contrib/storage-jdbc/pom.xml                       |  3 +-
 distribution/src/resources/drill-config.sh         | 11 +++++
 distribution/src/resources/sqlline.bat             | 12 +++++
 exec/java-exec/pom.xml                             | 20 +++-----
 .../drill/exec/compile/CompilationConfig.java      |  2 +-
 .../compile/bytecode/ReplacingInterpreter.java     |  2 +
 .../org/apache/drill/exec/expr/ClassGenerator.java |  4 +-
 .../planner/FileSystemPartitionDescriptor.java     |  2 +-
 .../apache/drill/TestUtf8SupportInQueryString.java |  3 --
 .../drill/exec/fn/impl/TestCastFunctions.java      |  4 --
 .../drill/exec/fn/impl/TestDateFunctions.java      |  3 --
 .../exec/fn/impl/testing/TestDateConversions.java  |  3 --
 .../drill/exec/fn/interp/TestConstantFolding.java  |  3 --
 .../impersonation/TestImpersonationMetadata.java   |  4 +-
 .../drill/exec/store/avro/AvroFormatTest.java      |  4 --
 .../vector/complex/writer/TestExtendedTypes.java   |  3 --
 .../java-exec/src/test/resources/drill-udf/pom.xml |  4 +-
 exec/jdbc-all/pom.xml                              |  5 +-
 .../org/apache/drill/jdbc/ITTestShadedJar.java     | 14 ++----
 ...rill2489CallsAfterCloseThrowExceptionsTest.java | 56 +++++++++++++++++++++-
 .../drill/exec/memory/BoundsCheckingTest.java      |  1 +
 logical/pom.xml                                    |  1 -
 pom.xml                                            | 24 ++++++----
 24 files changed, 123 insertions(+), 68 deletions(-)

diff --git a/common/src/main/java/org/apache/drill/common/config/DrillConfig.java b/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
index 423d4f0..d7d7340 100644
--- a/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
+++ b/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
@@ -27,6 +27,7 @@ import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
+import io.netty.util.internal.PlatformDependent;
 import org.apache.drill.common.exceptions.DrillConfigurationException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.scanner.ClassPathScanner;
@@ -46,7 +47,7 @@ public class DrillConfig extends NestedConfig {
   private final ImmutableList<String> startupArguments;
 
   @SuppressWarnings("restriction")
-  private static final long MAX_DIRECT_MEMORY = sun.misc.VM.maxDirectMemory();
+  private static final long MAX_DIRECT_MEMORY = PlatformDependent.maxDirectMemory();
 
   @VisibleForTesting
   public DrillConfig(Config config) {
diff --git a/contrib/storage-jdbc/pom.xml b/contrib/storage-jdbc/pom.xml
index 7de89f0..ad139d4 100755
--- a/contrib/storage-jdbc/pom.xml
+++ b/contrib/storage-jdbc/pom.xml
@@ -104,7 +104,7 @@
         <!-- Because the JDBC tests are somewhat heavyweight, we only run them in the 'verify' phase -->
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-failsafe-plugin</artifactId>
-        <version>2.18.1</version>
+        <version>2.22.0</version>
         <configuration>
           <forkCount combine.self="override">1</forkCount>
           <systemPropertyVariables>
@@ -115,6 +115,7 @@
           <includes>
             <include>**/*IT.java</include>
           </includes>
+          <useSystemClassLoader>false</useSystemClassLoader>
         </configuration>
         <executions>
           <execution>
diff --git a/distribution/src/resources/drill-config.sh b/distribution/src/resources/drill-config.sh
index d23788b..8f32a47 100644
--- a/distribution/src/resources/drill-config.sh
+++ b/distribution/src/resources/drill-config.sh
@@ -302,6 +302,17 @@ export DRILLBIT_CODE_CACHE_SIZE=${DRILLBIT_CODE_CACHE_SIZE:-"1G"}
 export DRILLBIT_OPTS="-Xms$DRILL_HEAP -Xmx$DRILL_HEAP -XX:MaxDirectMemorySize=$DRILL_MAX_DIRECT_MEMORY"
 export DRILLBIT_OPTS="$DRILLBIT_OPTS -XX:ReservedCodeCacheSize=$DRILLBIT_CODE_CACHE_SIZE -Ddrill.exec.enable-epoll=false"
 
+# Check that java is newer than 1.8
+"$JAVA" -version 2>&1 | grep "version" | egrep -e "1\.8" > /dev/null
+if [ $? -gt 0 ]; then
+  # Allow reflective access on Java 9+
+  export DRILLBIT_OPTS="$DRILLBIT_OPTS --add-opens java.base/java.lang=ALL-UNNAMED"
+  export DRILLBIT_OPTS="$DRILLBIT_OPTS --add-opens java.base/sun.nio.ch=ALL-UNNAMED"
+  export DRILLBIT_OPTS="$DRILLBIT_OPTS --add-opens java.base/java.nio=ALL-UNNAMED"
+  export DRILLBIT_OPTS="$DRILLBIT_OPTS --add-opens java.security.jgss/sun.security.krb5=ALL-UNNAMED"
+  export DRILLBIT_OPTS="$DRILLBIT_OPTS --illegal-access=permit"
+fi
+
 
 # Under YARN, the log directory is usually YARN-provided. Replace any
 # value that may have been set in drill-env.sh.
diff --git a/distribution/src/resources/sqlline.bat b/distribution/src/resources/sqlline.bat
index 7821efa..40f2771 100755
--- a/distribution/src/resources/sqlline.bat
+++ b/distribution/src/resources/sqlline.bat
@@ -205,6 +205,18 @@ set DRILL_SHELL_JAVA_OPTS=%DRILL_SHELL_JAVA_OPTS% -Dlog.path="%DRILL_LOG_DIR%\sq
 SET JAVA_CMD=%JAVA_HOME%\bin\%JAVA_EXE%
 if "%JAVA_HOME%" == "" (set JAVA_CMD=%JAVA_EXE%)
 set ERROR_CODE=0
+
+rem Check that java is newer than 1.8
+"%JAVA_CMD%" -version 2>&1 | findstr "1.8" > nul  2>&1
+if errorlevel 1 (
+  rem allow reflective access on Java 9+
+  set DRILL_SHELL_JAVA_OPTS=%DRILL_SHELL_JAVA_OPTS% --add-opens java.base/java.lang=ALL-UNNAMED
+  set DRILL_SHELL_JAVA_OPTS=%DRILL_SHELL_JAVA_OPTS% --add-opens java.base/sun.nio.ch=ALL-UNNAMED
+  set DRILL_SHELL_JAVA_OPTS=%DRILL_SHELL_JAVA_OPTS% --add-opens java.base/java.nio=ALL-UNNAMED
+  set DRILL_SHELL_JAVA_OPTS=%DRILL_SHELL_JAVA_OPTS% --add-opens java.security.jgss/sun.security.krb5=ALL-UNNAMED
+  set DRILL_SHELL_JAVA_OPTS=%DRILL_SHELL_JAVA_OPTS% --illegal-access=permit
+)
+
 set SQLLINE_CALL=sqlline.SqlLine -ac org.apache.drill.exec.client.DrillSqlLineApplication -d org.apache.drill.jdbc.Driver
 if NOT "test%QUERY%"=="test" (
   echo %QUERY% | "%JAVA_CMD%" %DRILL_SHELL_JAVA_OPTS% %DRILL_JAVA_OPTS% -cp "%DRILL_CP%" %SQLLINE_CALL% %DRILL_ARGS%
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 43897ef..cbe2285 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -64,16 +64,15 @@
       <version>${kerby.version}</version>
       <scope>test</scope>
     </dependency>
-    <!-- <dependency> -->
-    <!-- <groupId>org.ow2.asm</groupId> -->
-    <!-- <artifactId>asm-util</artifactId> -->
-    <!-- <version>5.0.3</version> -->
-    <!-- </dependency> -->
     <dependency>
       <groupId>org.ow2.asm</groupId>
-      <artifactId>asm-debug-all</artifactId>
-      <version>5.0.3</version>
-      <!-- <scope>test</scope> -->
+      <artifactId>asm-commons</artifactId>
+      <version>${asm.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.ow2.asm</groupId>
+      <artifactId>asm-util</artifactId>
+      <version>${asm.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.drill.contrib.data</groupId>
@@ -81,11 +80,6 @@
       <version>${project.version}</version>
       <scope>test</scope>
     </dependency>
-    <!-- <dependency> -->
-    <!-- <groupId>org.ow2.asm</groupId> -->
-    <!-- <artifactId>asm-commons</artifactId> -->
-    <!-- <version>5.0.3</version> -->
-    <!-- </dependency> -->
     <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-pool2</artifactId>
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CompilationConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CompilationConfig.java
index 7d7180d..6240813 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CompilationConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/CompilationConfig.java
@@ -24,5 +24,5 @@ public class CompilationConfig {
    * Never use asm.Opcodes values directly in calls that require them. Use ASM_OPCODES
    * instead, so that we can change it here once for all references.
    */
-  public final static int ASM_API_VERSION = Opcodes.ASM5;
+  public final static int ASM_API_VERSION = Opcodes.ASM7;
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ReplacingInterpreter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ReplacingInterpreter.java
index 93828c1..13063c8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ReplacingInterpreter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ReplacingInterpreter.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.compile.bytecode;
 
 import java.util.List;
 
+import org.apache.drill.exec.compile.CompilationConfig;
 import org.objectweb.asm.Opcodes;
 import org.objectweb.asm.Type;
 import org.objectweb.asm.tree.AbstractInsnNode;
@@ -40,6 +41,7 @@ public class ReplacingInterpreter extends BasicInterpreter {
   private final List<ReplacingBasicValue> valueList;
 
   public ReplacingInterpreter(final String className, final List<ReplacingBasicValue> valueList) {
+    super(CompilationConfig.ASM_API_VERSION);
     this.className = className;
     this.valueList = valueList;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
index 5b33acf..969b506 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
@@ -161,7 +161,7 @@ public class ClassGenerator<T>{
       // from the JDK Modifier values to the JCodeModel JMod values: the
       // values are different.
 
-      int mods = JMod.PRIVATE + JMod.FINAL;
+      int mods = JMod.FINAL;
       if ((innerClass.getModifiers() & Modifier.STATIC) != 0) {
         mods += JMod.STATIC;
       }
@@ -370,7 +370,7 @@ public class ClassGenerator<T>{
       // all new fields will be declared in the class from innerClassGenerator
       if (innerClassGenerator == null) {
         try {
-          JDefinedClass innerClazz = clazz._class(JMod.PRIVATE, clazz.name() + "0");
+          JDefinedClass innerClazz = clazz._class(JMod.NONE, clazz.name() + "0");
           innerClassGenerator = new ClassGenerator<>(codeGenerator, mappings, sig, evaluationVisitor, innerClazz, model, optionManager);
         } catch (JClassAlreadyExistsException e) {
           throw new DrillRuntimeException(e);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java
index 2d45701..c38cc5a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java
@@ -180,7 +180,7 @@ public class FileSystemPartitionDescriptor extends AbstractPartitionDescriptor {
 
     // build a list of DFSDirPartitionLocation.
     for (final List<String> dirs : dirToFileMap.keySet()) {
-      locations.add( new DFSDirPartitionLocation((String [])dirs.toArray(), dirToFileMap.get(dirs)));
+      locations.add( new DFSDirPartitionLocation(dirs.toArray(new String[dirs.size()]), dirToFileMap.get(dirs)));
     }
 
     locationSuperList = Lists.partition(locations, PartitionDescriptor.PARTITION_BATCH_SIZE);
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 d6a02f8..523b15b 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
@@ -19,19 +19,16 @@ package org.apache.drill;
 
 import mockit.Mock;
 import mockit.MockUp;
-import mockit.integration.junit4.JMockit;
 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.nio.charset.Charset;
 
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.junit.Assert.assertThat;
 
-@RunWith(JMockit.class)
 public class TestUtf8SupportInQueryString extends BaseTestQuery {
 
   @Test
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
index 7768909..37b260c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestCastFunctions.java
@@ -38,17 +38,13 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
 
-import mockit.integration.junit4.JMockit;
-
 import static org.hamcrest.CoreMatchers.containsString;
 import static org.hamcrest.CoreMatchers.hasItem;
 
-@RunWith(JMockit.class)
 @Category({UnlikelyTest.class, SqlFunctionTest.class})
 public class TestCastFunctions extends BaseTestQuery {
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
index d61c729..8c4bf8e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
@@ -19,7 +19,6 @@ package org.apache.drill.exec.fn.impl;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
 import org.apache.drill.shaded.guava.com.google.common.io.Files;
-import mockit.integration.junit4.JMockit;
 import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.common.util.DrillFileUtils;
@@ -37,12 +36,10 @@ import org.joda.time.LocalTime;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
 import java.util.List;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-@RunWith(JMockit.class)
 @Category({UnlikelyTest.class, SqlFunctionTest.class})
 public class TestDateFunctions extends PopUnitTestBase {
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
index e0121e6..363d326 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.fn.impl.testing;
 
-import mockit.integration.junit4.JMockit;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.SqlFunctionTest;
 import org.apache.drill.categories.UnlikelyTest;
@@ -25,7 +24,6 @@ import org.apache.drill.common.exceptions.UserException;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileWriter;
@@ -37,7 +35,6 @@ import java.time.LocalTime;
 import static org.hamcrest.CoreMatchers.startsWith;
 import static org.junit.Assert.assertThat;
 
-@RunWith(JMockit.class)
 @Category({UnlikelyTest.class, SqlFunctionTest.class})
 public class TestDateConversions extends BaseTestQuery {
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/TestConstantFolding.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/TestConstantFolding.java
index c1fea16..879dc4b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/TestConstantFolding.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/TestConstantFolding.java
@@ -19,19 +19,16 @@ package org.apache.drill.exec.fn.interp;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Joiner;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
-import mockit.integration.junit4.JMockit;
 import org.apache.drill.PlanTestBase;
 import org.apache.drill.categories.SqlTest;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
 import java.io.File;
 import java.io.PrintWriter;
 import java.util.List;
 
-@RunWith(JMockit.class)
 @Category(SqlTest.class)
 public class TestConstantFolding extends PlanTestBase {
 
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 9acd015..6dc3615 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
@@ -280,7 +280,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=\"/drill_test_grp_0_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.
@@ -351,7 +351,7 @@ public class TestImpersonationMetadata extends BaseTestImpersonation {
 
     thrown.expect(UserRemoteException.class);
     thrown.expectMessage(containsString("Permission denied: user=drillTestUser2, " +
-        "access=WRITE, inode=\"/drill_test_grp_0_755/"));
+        "access=WRITE, inode=\"/drill_test_grp_0_755"));
 
     test("CREATE TABLE %s AS SELECT c_custkey, c_nationkey " +
         "FROM cp.`tpch/customer.parquet` ORDER BY c_custkey", tableName);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java
index 2b7edc9..8f2339b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java
@@ -56,16 +56,12 @@ import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.test.TestBuilder;
 import org.junit.Assert;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
-import mockit.integration.junit4.JMockit;
-
 /**
  * Unit tests for Avro record reader.
  */
-@RunWith(JMockit.class)
 public class AvroFormatTest extends BaseTestQuery {
 
   // XXX
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
index c2d3f9a..bc27e88 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.vector.complex.writer;
 
-import mockit.integration.junit4.JMockit;
 import static org.junit.Assert.assertEquals;
 
 import java.nio.file.Files;
@@ -30,9 +29,7 @@ import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-@RunWith(JMockit.class)
 public class TestExtendedTypes extends BaseTestQuery {
   @BeforeClass
   public static void setupTestFiles() {
diff --git a/exec/java-exec/src/test/resources/drill-udf/pom.xml b/exec/java-exec/src/test/resources/drill-udf/pom.xml
index 7361845..931abed 100644
--- a/exec/java-exec/src/test/resources/drill-udf/pom.xml
+++ b/exec/java-exec/src/test/resources/drill-udf/pom.xml
@@ -63,12 +63,14 @@
           <includes>
             <include>${include.files}</include>
           </includes>
+          <source>1.8</source>
+          <target>1.8</target>
         </configuration>
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-source-plugin</artifactId>
-        <version>2.4</version>
+        <version>3.0.1</version>
         <configuration>
           <finalName>${jar.finalName}</finalName>
           <includes>
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index 7be43ab..e0ae482 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -241,7 +241,7 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-failsafe-plugin</artifactId>
-        <version>2.18.1</version>
+        <version>2.22.0</version>
         <executions>
           <execution>
             <goals>
@@ -268,6 +268,7 @@
             <app.class.path>${app.class.path}</app.class.path>
             <project.version>${project.version}</project.version>
           </systemPropertyVariables>
+          <useSystemClassLoader>false</useSystemClassLoader>
         </configuration>
       </plugin>
 
@@ -333,6 +334,7 @@
               <exclude>commons-beanutils:commons-beanutils-core:jar:*</exclude>
               <exclude>commons-beanutils:commons-beanutils:jar:*</exclude>
               <exclude>io.netty:netty-tcnative:jar:*</exclude>
+              <exclude>org.honton.chas.hocon:jackson-dataformat-hocon:*</exclude>
             </excludes>
           </artifactSet>
           <relocations>
@@ -648,6 +650,7 @@
                     <exclude>commons-io:commons-io</exclude>
                     <exclude>commons-beanutils:commons-beanutils-core:jar:*</exclude>
                     <exclude>commons-beanutils:commons-beanutils:jar:*</exclude>
+                    <exclude>org.honton.chas.hocon:jackson-dataformat-hocon:*</exclude>
                   </excludes>
                 </artifactSet>
                 <relocations>
diff --git a/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/ITTestShadedJar.java b/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/ITTestShadedJar.java
index 6319cb3..4fed146 100644
--- a/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/ITTestShadedJar.java
+++ b/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/ITTestShadedJar.java
@@ -42,7 +42,7 @@ public class ITTestShadedJar {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ITTestShadedJar.class);
 
   private static DrillbitClassLoader drillbitLoader;
-  private static URLClassLoader rootClassLoader;
+  private static ClassLoader rootClassLoader;
   private static int userPort;
 
   @ClassRule
@@ -55,7 +55,7 @@ public class ITTestShadedJar {
       try {
         drillbitLoader = new DrillbitClassLoader();
         drillbitLoader.loadClass("org.apache.commons.io.FileUtils");
-        rootClassLoader = (URLClassLoader) Thread.currentThread().getContextClassLoader();
+        rootClassLoader = Thread.currentThread().getContextClassLoader();
 
         Class<?> clazz = drillbitLoader.loadClass("org.apache.drill.test.BaseTestQuery");
         Class<?> watcherClazz = drillbitLoader.loadClass("org.apache.drill.test.BaseDirTestWatcher");
@@ -134,10 +134,7 @@ public class ITTestShadedJar {
 
   @Test
   public void testDatabaseVersion() throws Exception {
-    final URLClassLoader loader = (URLClassLoader) ClassLoader.getSystemClassLoader();
-    Method method = URLClassLoader.class.getDeclaredMethod("addURL", URL.class);
-    method.setAccessible(true);
-    method.invoke(loader, getJdbcUrl());
+    final URLClassLoader loader = URLClassLoader.newInstance(new URL[] {getJdbcUrl()});
 
     Class<?> clazz = loader.loadClass("org.apache.drill.jdbc.Driver");
     try {
@@ -155,10 +152,7 @@ public class ITTestShadedJar {
 
   @Test
   public void executeJdbcAllQuery() throws Exception {
-    final URLClassLoader loader = (URLClassLoader) ClassLoader.getSystemClassLoader();
-    Method method = URLClassLoader.class.getDeclaredMethod("addURL", URL.class);
-    method.setAccessible(true);
-    method.invoke(loader, getJdbcUrl());
+    final URLClassLoader loader = URLClassLoader.newInstance(new URL[] {getJdbcUrl()});
 
     Class<?> clazz = loader.loadClass("org.apache.drill.jdbc.Driver");
     try {
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 303565f..b868cf5 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,6 +41,7 @@ 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;
@@ -413,6 +414,15 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTestBase {
     }
 
     @Override
+    protected boolean isOkayNonthrowingMethod(Method method) {
+      return
+          super.isOkayNonthrowingMethod(method)
+          // New Java 9 methods not implemented in Avatica.
+          || method.getName().equals("beginRequest")
+          || method.getName().equals("endRequest");
+    }
+
+    @Override
     protected boolean isOkaySpecialCaseException(Method method, Throwable cause) {
       final boolean result;
       if (super.isOkaySpecialCaseException(method, cause)) {
@@ -424,6 +434,12 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTestBase {
                     || method.getName().equals("getClientInfo"))) {
         // Special good case--we had to use SQLClientInfoException from those.
         result = true;
+      }
+      else if (SQLFeatureNotSupportedException.class == cause.getClass()
+                && (method.getName().equals("setShardingKeyIfValid")
+                    || method.getName().equals("setShardingKey"))) {
+        // New Java 9 methods not implemented in Avatica.
+        result = true;
       } else {
         result = false;
       }
@@ -453,6 +469,24 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTestBase {
     ClosedPlainStatementChecker(Class<Statement> intf, Statement jdbcObject) {
       super(intf, jdbcObject, PLAIN_STATEMENT_CLOSED_MESSAGE);
     }
+
+    @Override
+    protected boolean isOkaySpecialCaseException(Method method, Throwable cause) {
+      final boolean result;
+      if (super.isOkaySpecialCaseException(method, cause)) {
+        result = true;
+      } else if (NullPointerException.class == cause.getClass()
+              && (method.getName().equals("enquoteIdentifier")
+                  || method.getName().equals("enquoteLiteral")
+                  || method.getName().equals("enquoteNCharLiteral")
+                  || method.getName().equals("isSimpleIdentifier"))) {
+        result = true;
+      } else {
+        result = false;
+      }
+
+      return result;
+    }
   } // class ClosedPlainStatementChecker
 
   @Test
@@ -477,6 +511,24 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTestBase {
                                    PreparedStatement jdbcObject) {
       super(intf, jdbcObject, PREPAREDSTATEMENT_CLOSED_MESSAGE);
     }
+
+    @Override
+    protected boolean isOkaySpecialCaseException(Method method, Throwable cause) {
+      final boolean result;
+      if (super.isOkaySpecialCaseException(method, cause)) {
+        result = true;
+      } else if (NullPointerException.class == cause.getClass()
+                 && (method.getName().equals("enquoteIdentifier")
+                     || method.getName().equals("enquoteLiteral")
+                     || method.getName().equals("enquoteNCharLiteral")
+                     || method.getName().equals("isSimpleIdentifier"))) {
+        result = true;
+      } else {
+        result = false;
+      }
+
+      return result;
+    }
   } // class closedPreparedStmtOfOpenConnChecker
 
   @Test
@@ -587,7 +639,9 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTestBase {
           || method.getName().equals("getConnection")
           // TODO: New Java 8 methods not implemented in Avatica.
           || method.getName().equals("getMaxLogicalLobSize")
-          || method.getName().equals("supportsRefCursors");
+          || method.getName().equals("supportsRefCursors")
+          // New Java 9 methods not implemented in Avatica.
+          || method.getName().equals("supportsSharding");
     }
 
     @Override
diff --git a/exec/memory/base/src/test/java/org/apache/drill/exec/memory/BoundsCheckingTest.java b/exec/memory/base/src/test/java/org/apache/drill/exec/memory/BoundsCheckingTest.java
index 5d2853b..8021228 100644
--- a/exec/memory/base/src/test/java/org/apache/drill/exec/memory/BoundsCheckingTest.java
+++ b/exec/memory/base/src/test/java/org/apache/drill/exec/memory/BoundsCheckingTest.java
@@ -46,6 +46,7 @@ public class BoundsCheckingTest
     modifiersField.setAccessible(true);
     modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
     boolean old = field.getBoolean(null);
+    field.setAccessible(true);
     field.set(null, enabled);
     return old;
   }
diff --git a/logical/pom.xml b/logical/pom.xml
index ed9abb7..aa8fb06 100644
--- a/logical/pom.xml
+++ b/logical/pom.xml
@@ -105,7 +105,6 @@
       </plugin>
       <plugin>
         <artifactId>maven-surefire-plugin</artifactId>
-        <version>2.15</version>
         <configuration>
           <useSystemClassLoader>false</useSystemClassLoader>
         </configuration>
diff --git a/pom.xml b/pom.xml
index d69c679..0cc2527 100644
--- a/pom.xml
+++ b/pom.xml
@@ -64,7 +64,7 @@
     <maven.embedder.version>3.5.3</maven.embedder.version>
     <curator.version>2.7.1</curator.version>
     <wiremock.standalone.version>2.5.1</wiremock.standalone.version>
-    <jmockit.version>1.39</jmockit.version>
+    <jmockit.version>1.43</jmockit.version>
     <logback.version>1.0.13</logback.version>
     <mockito.version>2.18.3</mockito.version>
     <!--
@@ -72,15 +72,16 @@
       Apache Hive 2.3.2. If the version is changed, make sure the jars and their dependencies are updated.
     -->
     <hive.version>2.3.2</hive.version>
-    <hadoop.version>2.7.1</hadoop.version>
-    <hbase.version>1.1.3</hbase.version>
+    <hadoop.version>2.7.4</hadoop.version>
+    <hbase.version>2.1.0</hbase.version>
     <fmpp.version>1.0</fmpp.version>
     <freemarker.version>2.3.26-incubating</freemarker.version>
-    <javassist.version>3.22.0-GA</javassist.version>
+    <javassist.version>3.24.0-GA</javassist.version>
     <msgpack.version>0.6.6</msgpack.version>
     <reflections.version>0.9.10</reflections.version>
     <avro.version>1.8.2</avro.version>
     <metrics.version>4.0.2</metrics.version>
+    <asm.version>7.0</asm.version>
     <excludedGroups />
     <memoryMb>4096</memoryMb>
     <directMemoryMb>4096</directMemoryMb>
@@ -492,7 +493,7 @@
                   <version>[3.3.1,4)</version>
                 </requireMavenVersion>
                 <requireJavaVersion>
-                  <version>[1.8,1.9)</version>
+                  <version>[1.8,12)</version>
                 </requireJavaVersion>
               </rules>
             </configuration>
@@ -681,11 +682,11 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-compiler-plugin</artifactId>
-          <version>3.2</version>
+          <version>3.8.0</version>
         </plugin>
         <plugin>
           <artifactId>maven-enforcer-plugin</artifactId>
-          <version>1.3.1</version>
+          <version>3.0.0-M1</version>
         </plugin>
         <plugin> <!-- classpath scanning  -->
           <groupId>org.codehaus.mojo</groupId>
@@ -710,7 +711,7 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-surefire-plugin</artifactId>
-          <version>2.17</version>
+          <version>2.21.0</version>
           <executions>
             <execution>
               <id>default-test</id>
@@ -722,7 +723,7 @@
             <dependency>
               <groupId>org.apache.maven.surefire</groupId>
               <artifactId>surefire-junit47</artifactId>
-              <version>2.19.1</version>
+              <version>2.21.0</version>
             </dependency>
           </dependencies>
           <configuration>
@@ -735,6 +736,8 @@
               -Djava.net.preferIPv4Stack=true
               -Djava.awt.headless=true
               -XX:+CMSClassUnloadingEnabled -ea
+              -Djdk.attach.allowAttachSelf=true
+              -javaagent:${settings.localRepository}/org/jmockit/jmockit/${jmockit.version}/jmockit-${jmockit.version}.jar
             </argLine>
             <forkCount>${forkCount}</forkCount>
             <reuseForks>true</reuseForks>
@@ -745,6 +748,7 @@
               <java.io.tmpdir>${project.build.directory}</java.io.tmpdir>
             </systemPropertyVariables>
             <excludedGroups>${excludedGroups}</excludedGroups>
+            <useSystemClassLoader>false</useSystemClassLoader>
           </configuration>
         </plugin>
         <plugin>
@@ -925,7 +929,7 @@
       <!-- JMockit needs to be on class path before JUnit. -->
       <groupId>org.jmockit</groupId>
       <artifactId>jmockit</artifactId>
-      <version>1.39</version>
+      <version>${jmockit.version}</version>
       <scope>test</scope>
     </dependency>
     <dependency>


[drill] 06/15: DRILL-6850: JDBC integration tests failures

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

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

commit 1c087237a5a23e9c97cf5318e34f0c8c58ec7af6
Author: Vitalii Diravka <vi...@gmail.com>
AuthorDate: Fri Nov 2 15:41:37 2018 +0200

    DRILL-6850: JDBC integration tests failures
    
    - Fix RDBMS integration tests (expected decimal output and testCrossSourceMultiFragmentJoin)
    - Update libraries versions
    - Resolve NPE for empty result
---
 contrib/storage-jdbc/pom.xml                       |  12 +-
 .../exec/store/jdbc/TestJdbcPluginWithDerbyIT.java | 112 ++++++++-------
 .../exec/store/jdbc/TestJdbcPluginWithMySQLIT.java | 152 +++++++++++----------
 .../sql/handlers/FindHardDistributionScans.java    |   7 +-
 4 files changed, 155 insertions(+), 128 deletions(-)

diff --git a/contrib/storage-jdbc/pom.xml b/contrib/storage-jdbc/pom.xml
index ad139d4..5bad6b1 100755
--- a/contrib/storage-jdbc/pom.xml
+++ b/contrib/storage-jdbc/pom.xml
@@ -31,7 +31,7 @@
   <name>contrib/jdbc-storage-plugin</name>
 
   <properties>
-    <mysql.connector.version>5.1.36</mysql.connector.version>
+    <mysql.connector.version>8.0.13</mysql.connector.version>
     <derby.database.name>drill_derby_test</derby.database.name>
     <mysql.database.name>drill_mysql_test</mysql.database.name>
     <skipTests>false</skipTests>
@@ -62,13 +62,13 @@
     <dependency>
       <groupId>org.apache.derby</groupId>
       <artifactId>derbyclient</artifactId>
-      <version>10.11.1.1</version>
+      <version>10.14.2.0</version>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.derby</groupId>
       <artifactId>derbynet</artifactId>
-      <version>10.11.1.1</version>
+      <version>10.14.2.0</version>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -104,7 +104,7 @@
         <!-- Because the JDBC tests are somewhat heavyweight, we only run them in the 'verify' phase -->
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-failsafe-plugin</artifactId>
-        <version>2.22.0</version>
+        <version>2.22.1</version>
         <configuration>
           <forkCount combine.self="override">1</forkCount>
           <systemPropertyVariables>
@@ -128,7 +128,7 @@
         <!-- Allows us to reserve ports for external servers that we will launch  -->
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>build-helper-maven-plugin</artifactId>
-        <version>1.9.1</version>
+        <version>3.0.0</version>
         <executions>
           <execution>
             <id>reserve-network-port</id>
@@ -147,7 +147,7 @@
       </plugin>
       <plugin>
         <artifactId>maven-dependency-plugin</artifactId>
-        <version>2.8</version>
+        <version>3.1.1</version>
         <executions>
           <execution>
             <goals>
diff --git a/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithDerbyIT.java b/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithDerbyIT.java
index e2e4089..65a1ea5 100644
--- a/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithDerbyIT.java
+++ b/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithDerbyIT.java
@@ -22,9 +22,14 @@ import org.apache.drill.PlanTestBase;
 import org.apache.drill.exec.expr.fn.impl.DateUtility;
 import org.apache.drill.exec.proto.UserBitShared;
 
+import org.apache.drill.exec.util.StoragePluginTestUtils;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import java.math.BigDecimal;
+import java.nio.file.Paths;
+
 import static org.junit.Assert.assertEquals;
 
 /**
@@ -33,107 +38,114 @@ import static org.junit.Assert.assertEquals;
 @Category(JdbcStorageTest.class)
 public class TestJdbcPluginWithDerbyIT extends PlanTestBase {
 
+  private static final String TABLE_PATH = "jdbcmulti/";
+  private static final String TABLE_NAME = String.format("%s.`%s`", StoragePluginTestUtils.DFS_PLUGIN_NAME, TABLE_PATH);
+
+  @BeforeClass
+  public static void copyData() throws Exception {
+    dirTestWatcher.copyResourceToRoot(Paths.get(TABLE_PATH));
+  }
+
   @Test
   public void testCrossSourceMultiFragmentJoin() throws Exception {
-    testNoResult("USE derby");
     testNoResult("SET `planner.slice_target` = 1");
-    String query = "select x.person_id, y.salary from DRILL_DERBY_TEST.PERSON x "
-        + "join dfs.`${WORKING_PATH}/src/test/resources/jdbcmulti/` y on x.person_id = y.person_id ";
-    test(query);
+    test("select x.person_id, y.salary from derby.drill_derby_test.person x "
+        + "join %s y on x.person_id = y.person_id ", TABLE_NAME);
   }
 
   @Test
   public void validateResult() throws Exception {
-
     // Skip date, time, and timestamp types since derby mangles these due to improper timezone support.
     testBuilder()
-            .sqlQuery(
-                    "select PERSON_ID, FIRST_NAME, LAST_NAME, ADDRESS, CITY, STATE, ZIP, JSON, BIGINT_FIELD, SMALLINT_FIELD, " +
-                            "NUMERIC_FIELD, BOOLEAN_FIELD, DOUBLE_FIELD, FLOAT_FIELD, REAL_FIELD, TIME_FIELD, TIMESTAMP_FIELD, " +
-                            "DATE_FIELD, CLOB_FIELD from derby.DRILL_DERBY_TEST.PERSON")
-            .ordered()
-            .baselineColumns("PERSON_ID", "FIRST_NAME", "LAST_NAME", "ADDRESS", "CITY", "STATE", "ZIP", "JSON",
-                    "BIGINT_FIELD", "SMALLINT_FIELD", "NUMERIC_FIELD", "BOOLEAN_FIELD", "DOUBLE_FIELD",
-                    "FLOAT_FIELD", "REAL_FIELD", "TIME_FIELD", "TIMESTAMP_FIELD", "DATE_FIELD", "CLOB_FIELD")
-            .baselineValues(1, "first_name_1", "last_name_1", "1401 John F Kennedy Blvd",   "Philadelphia",     "PA",
-                            19107, "{ a : 5, b : 6 }",            123456L,         1, 10.01, false, 1.0, 1.1, 111.00,
-                            DateUtility.parseLocalTime("13:00:01.0"), DateUtility.parseLocalDateTime("2012-02-29 13:00:01.0"), DateUtility.parseLocalDate("2012-02-29"), "some clob data 1")
-            .baselineValues(2, "first_name_2", "last_name_2", "One Ferry Building",         "San Francisco",    "CA",
-                            94111, "{ foo : \"abc\" }",            95949L,         2, 20.02, true, 2.0, 2.1, 222.00,
-                            DateUtility.parseLocalTime("23:59:59.0"),  DateUtility.parseLocalDateTime("1999-09-09 23:59:59.0"), DateUtility.parseLocalDate("1999-09-09"), "some more clob data")
-            .baselineValues(3, "first_name_3", "last_name_3", "176 Bowery",                 "New York",         "NY",
-                            10012, "{ z : [ 1, 2, 3 ] }",           45456L,        3, 30.04, true, 3.0, 3.1, 333.00,
-                            DateUtility.parseLocalTime("11:34:21.0"),  DateUtility.parseLocalDateTime("2011-10-30 11:34:21.0"), DateUtility.parseLocalDate("2011-10-30"), "clobber")
-            .baselineValues(4, null, null, "2 15th St NW", "Washington", "DC", 20007, "{ z : { a : 1, b : 2, c : 3 } " +
-                    "}", -67L, 4, 40.04, false, 4.0, 4.1, 444.00, DateUtility.parseLocalTime("16:00:01.0"), DateUtility.parseLocalDateTime("2015-06-01 16:00:01.0"),  DateUtility.parseLocalDate("2015-06-01"), "xxx")
-            .baselineValues(5, null, null, null, null, null, null, null, null, null, null, null, null, null, null,
-                            null, null, null, null)
-            .build().run();
+        .sqlQuery(
+            "select person_id, first_name, last_name, address, city, state, zip, json, bigint_field, smallint_field, " +
+                "numeric_field, boolean_field, double_field, float_field, real_field, time_field, timestamp_field, " +
+                "date_field, clob_field from derby.`drill_derby_test`.person")
+        .ordered()
+        .baselineColumns("person_id", "first_name", "last_name", "address", "city", "state", "zip", "json",
+            "bigint_field", "smallint_field", "numeric_field", "boolean_field", "double_field", "float_field",
+            "real_field", "time_field", "timestamp_field", "date_field", "clob_field")
+        .baselineValues(1, "first_name_1", "last_name_1", "1401 John F Kennedy Blvd",   "Philadelphia",     "PA", 19107,
+            "{ a : 5, b : 6 }", 123456L, 1, new BigDecimal("10.01"), false, 1.0, 1.1, 111.00,
+            DateUtility.parseLocalTime("13:00:01.0"), DateUtility.parseLocalDateTime("2012-02-29 13:00:01.0"),
+            DateUtility.parseLocalDate("2012-02-29"), "some clob data 1")
+        .baselineValues(2, "first_name_2", "last_name_2", "One Ferry Building", "San Francisco", "CA", 94111,
+            "{ foo : \"abc\" }", 95949L, 2, new BigDecimal("20.02"), true, 2.0, 2.1, 222.00,
+            DateUtility.parseLocalTime("23:59:59.0"),  DateUtility.parseLocalDateTime("1999-09-09 23:59:59.0"),
+            DateUtility.parseLocalDate("1999-09-09"), "some more clob data")
+        .baselineValues(3, "first_name_3", "last_name_3", "176 Bowery", "New York", "NY", 10012, "{ z : [ 1, 2, 3 ] }",
+            45456L, 3, new BigDecimal("30.04"), true, 3.0, 3.1, 333.00, DateUtility.parseLocalTime("11:34:21.0"),
+            DateUtility.parseLocalDateTime("2011-10-30 11:34:21.0"), DateUtility.parseLocalDate("2011-10-30"), "clobber")
+        .baselineValues(4, null, null, "2 15th St NW", "Washington", "DC", 20007, "{ z : { a : 1, b : 2, c : 3 } }",
+            -67L, 4, new BigDecimal("40.04"), false, 4.0, 4.1, 444.00, DateUtility.parseLocalTime("16:00:01.0"),
+            DateUtility.parseLocalDateTime("2015-06-01 16:00:01.0"),  DateUtility.parseLocalDate("2015-06-01"), "xxx")
+        .baselineValues(5, null, null, null, null, null, null, null, null, null, null, null, null, null, null,
+            null, null, null, null)
+        .go();
   }
 
   @Test
   public void pushdownJoin() throws Exception {
     testNoResult("use derby");
-    String query = "select x.person_id from (select person_id from DRILL_DERBY_TEST.PERSON) x "
-            + "join (select person_id from DRILL_DERBY_TEST.PERSON) y on x.person_id = y.person_id ";
+    String query = "select x.person_id from (select person_id from derby.drill_derby_test.person) x "
+            + "join (select person_id from derby.drill_derby_test.person) y on x.person_id = y.person_id ";
     testPlanMatchingPatterns(query, new String[]{}, new String[]{"Join"});
   }
 
   @Test
   public void pushdownJoinAndFilterPushDown() throws Exception {
     final String query = "select * from \n" +
-            "derby.DRILL_DERBY_TEST.PERSON e\n" +
-            "INNER JOIN \n" +
-            "derby.DRILL_DERBY_TEST.PERSON s\n" +
-            "ON e.FIRST_NAME = s.FIRST_NAME\n" +
-            "WHERE e.LAST_NAME > 'hello'";
+        "derby.drill_derby_test.person e\n" +
+        "INNER JOIN \n" +
+        "derby.drill_derby_test.person s\n" +
+        "ON e.FIRST_NAME = s.FIRST_NAME\n" +
+        "WHERE e.LAST_NAME > 'hello'";
 
     testPlanMatchingPatterns(query, new String[] {}, new String[] { "Join", "Filter" });
   }
 
   @Test
   public void pushdownAggregation() throws Exception {
-    final String query = "select count(*) from derby.DRILL_DERBY_TEST.PERSON";
+    final String query = "select count(*) from derby.drill_derby_test.person";
     testPlanMatchingPatterns(query, new String[] {}, new String[] { "Aggregate" });
   }
 
   @Test
   public void pushdownDoubleJoinAndFilter() throws Exception {
     final String query = "select * from \n" +
-            "derby.DRILL_DERBY_TEST.PERSON e\n" +
-            "INNER JOIN \n" +
-            "derby.DRILL_DERBY_TEST.PERSON s\n" +
-            "ON e.PERSON_ID = s.PERSON_ID\n" +
-            "INNER JOIN \n" +
-            "derby.DRILL_DERBY_TEST.PERSON ed\n" +
-            "ON e.PERSON_ID = ed.PERSON_ID\n" +
-            "WHERE s.FIRST_NAME > 'abc' and ed.FIRST_NAME > 'efg'";
+        "derby.drill_derby_test.person e\n" +
+        "INNER JOIN \n" +
+        "derby.drill_derby_test.person s\n" +
+        "ON e.person_ID = s.person_ID\n" +
+        "INNER JOIN \n" +
+        "derby.drill_derby_test.person ed\n" +
+        "ON e.person_ID = ed.person_ID\n" +
+        "WHERE s.first_name > 'abc' and ed.first_name > 'efg'";
     testPlanMatchingPatterns(query, new String[] {}, new String[] { "Join", "Filter" });
   }
 
   @Test
   public void showTablesDefaultSchema() throws Exception {
-    testNoResult("use derby");
-    assertEquals(1, testRunAndPrint(UserBitShared.QueryType.SQL, "show tables like 'PERSON'"));
+    test("use derby");
+    assertEquals(1, testRunAndPrint(UserBitShared.QueryType.SQL, "show tables like 'person'"));
   }
 
   @Test
   public void describe() throws Exception {
-    testNoResult("use derby");
-    assertEquals(19, testRunAndPrint(UserBitShared.QueryType.SQL, "describe PERSON"));
+    test("use derby");
+    assertEquals(19, testRunAndPrint(UserBitShared.QueryType.SQL, "describe drill_derby_test.person"));
   }
 
   @Test
   public void ensureDrillFunctionsAreNotPushedDown() throws Exception {
     // This should verify that we're not trying to push CONVERT_FROM into the JDBC storage plugin. If were pushing
     // this function down, the SQL query would fail.
-    testNoResult("select CONVERT_FROM(JSON, 'JSON') from derby.DRILL_DERBY_TEST.PERSON where PERSON_ID = 4");
+    testNoResult("select CONVERT_FROM(JSON, 'JSON') from derby.drill_derby_test.person where person_ID = 4");
   }
 
   @Test
   public void pushdownFilter() throws Exception {
-    testNoResult("use derby");
-    String query = "select * from DRILL_DERBY_TEST.PERSON where PERSON_ID = 1";
+    String query = "select * from derby.drill_derby_test.person where person_ID = 1";
     testPlanMatchingPatterns(query, new String[]{}, new String[]{"Filter"});
   }
 }
diff --git a/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithMySQLIT.java b/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithMySQLIT.java
index 0f377e3..c8396b5 100644
--- a/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithMySQLIT.java
+++ b/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithMySQLIT.java
@@ -24,6 +24,8 @@ import org.apache.drill.PlanTestBase;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import java.math.BigDecimal;
+
 /**
  * JDBC storage plugin tests against MySQL.
  * Note: it requires libaio.so library in the system
@@ -35,77 +37,77 @@ public class TestJdbcPluginWithMySQLIT extends PlanTestBase {
   public void validateResult() throws Exception {
 
     testBuilder()
-            .sqlQuery(
-                    "select person_id, " +
-                            "first_name, last_name, address, city, state, zip, " +
-                            "bigint_field, smallint_field, numeric_field, " +
-                            "boolean_field, double_field, float_field, real_field, " +
-                            "date_field, datetime_field, year_field, time_field, " +
-                            "json, text_field, tiny_text_field, medium_text_field, long_text_field, " +
-                            "blob_field, bit_field, enum_field " +
-                    "from mysql.`drill_mysql_test`.person")
-            .ordered()
-            .baselineColumns("person_id",
-                    "first_name", "last_name", "address", "city", "state", "zip",
-                    "bigint_field", "smallint_field", "numeric_field",
-                    "boolean_field",
-                    "double_field", "float_field", "real_field",
-                    "date_field", "datetime_field", "year_field", "time_field",
-                    "json", "text_field", "tiny_text_field", "medium_text_field", "long_text_field",
-                    "blob_field", "bit_field", "enum_field")
-            .baselineValues(1,
-                    "first_name_1", "last_name_1", "1401 John F Kennedy Blvd", "Philadelphia", "PA", 19107,
-                    123456789L, 1, 10.01,
-                    false,
-                    1.0, 1.1, 1.2,
-                    DateUtility.parseLocalDate("2012-02-29"), DateUtility.parseLocalDateTime("2012-02-29 13:00:01.0"), DateUtility.parseLocalDate("2015-01-01"), DateUtility.parseLocalTime("13:00:01.0"),
-                    "{ a : 5, b : 6 }",
-                    "It is a long established fact that a reader will be distracted by the readable content of a page when looking at its layout",
-                    "xxx",
-                    "a medium piece of text",
-                    "a longer piece of text this is going on.....",
-                    "this is a test".getBytes(),
-                    true, "XXX")
-            .baselineValues(2,
-                    "first_name_2", "last_name_2", "One Ferry Building", "San Francisco", "CA", 94111,
-                    45456767L, 3, 30.04,
-                    true,
-                    3.0, 3.1, 3.2,
-                    DateUtility.parseLocalDate("2011-10-30"), DateUtility.parseLocalDateTime("2011-10-30 11:34:21.0"), DateUtility.parseLocalDate("2015-01-01"), DateUtility.parseLocalTime("11:34:21.0"),
-                    "{ z : [ 1, 2, 3 ] }",
-                    "It is a long established fact that a reader will be distracted by the readable content of a page when looking at its layout",
-                    "abc",
-                    "a medium piece of text 2",
-                    "somewhat more text",
-                    "this is a test 2".getBytes(),
-                    false, "YYY")
-            .baselineValues(3,
-                    "first_name_3", "last_name_3", "176 Bowery", "New York", "NY", 10012,
-                    123090L, -3, 55.12,
-                    false,
-                    5.0, 5.1, 5.55,
-                    DateUtility.parseLocalDate("2015-06-01"), DateUtility.parseLocalDateTime("2015-09-22 15:46:10.0"), DateUtility.parseLocalDate("1901-01-01"), DateUtility.parseLocalTime("16:00:01.0"),
-                    "{ [ a, b, c ] }",
-                    "Neque porro quisquam est qui dolorem ipsum quia dolor sit amet, consectetur, adipisci velit",
-                    "abc",
-                    "a medium piece of text 3",
-                    "somewhat more text",
-                    "this is a test 3".getBytes(),
-                    true, "ZZZ")
-            .baselineValues(5,
-                    null, null, null, null, null, null,
-                    null, null, null,
-                    null,
-                    null, null, null,
-                    null, null, null, null,
-                    null,
-                    null,
-                    null,
-                    null,
-                    null,
-                    null,
-                    null, "XXX")
-                  .build().run();
+        .sqlQuery(
+            "select person_id, " +
+                "first_name, last_name, address, city, state, zip, " +
+                "bigint_field, smallint_field, numeric_field, " +
+                "boolean_field, double_field, float_field, real_field, " +
+                "date_field, datetime_field, year_field, time_field, " +
+                "json, text_field, tiny_text_field, medium_text_field, long_text_field, " +
+                "blob_field, bit_field, enum_field " +
+            "from mysql.`drill_mysql_test`.person")
+        .ordered()
+        .baselineColumns("person_id",
+            "first_name", "last_name", "address", "city", "state", "zip",
+            "bigint_field", "smallint_field", "numeric_field",
+            "boolean_field",
+            "double_field", "float_field", "real_field",
+            "date_field", "datetime_field", "year_field", "time_field",
+            "json", "text_field", "tiny_text_field", "medium_text_field", "long_text_field",
+            "blob_field", "bit_field", "enum_field")
+        .baselineValues(1,
+            "first_name_1", "last_name_1", "1401 John F Kennedy Blvd", "Philadelphia", "PA", 19107,
+            123456789L, 1, new BigDecimal("10.01"),
+            false,
+            1.0, 1.1, 1.2,
+            DateUtility.parseLocalDate("2012-02-29"), DateUtility.parseLocalDateTime("2012-02-29 13:00:01.0"), DateUtility.parseLocalDate("2015-01-01"), DateUtility.parseLocalTime("13:00:01.0"),
+            "{ a : 5, b : 6 }",
+            "It is a long established fact that a reader will be distracted by the readable content of a page when looking at its layout",
+            "xxx",
+            "a medium piece of text",
+            "a longer piece of text this is going on.....",
+            "this is a test".getBytes(),
+            true, "XXX")
+        .baselineValues(2,
+            "first_name_2", "last_name_2", "One Ferry Building", "San Francisco", "CA", 94111,
+            45456767L, 3, new BigDecimal("30.04"),
+            true,
+            3.0, 3.1, 3.2,
+            DateUtility.parseLocalDate("2011-10-30"), DateUtility.parseLocalDateTime("2011-10-30 11:34:21.0"), DateUtility.parseLocalDate("2015-01-01"), DateUtility.parseLocalTime("11:34:21.0"),
+            "{ z : [ 1, 2, 3 ] }",
+            "It is a long established fact that a reader will be distracted by the readable content of a page when looking at its layout",
+            "abc",
+            "a medium piece of text 2",
+            "somewhat more text",
+            "this is a test 2".getBytes(),
+            false, "YYY")
+        .baselineValues(3,
+            "first_name_3", "last_name_3", "176 Bowery", "New York", "NY", 10012,
+            123090L, -3, new BigDecimal("55.12"),
+            false,
+            5.0, 5.1, 5.55,
+            DateUtility.parseLocalDate("2015-06-01"), DateUtility.parseLocalDateTime("2015-09-22 15:46:10.0"), DateUtility.parseLocalDate("1901-01-01"), DateUtility.parseLocalTime("16:00:01.0"),
+            "{ [ a, b, c ] }",
+            "Neque porro quisquam est qui dolorem ipsum quia dolor sit amet, consectetur, adipisci velit",
+            "abc",
+            "a medium piece of text 3",
+            "somewhat more text",
+            "this is a test 3".getBytes(),
+            true, "ZZZ")
+        .baselineValues(5,
+            null, null, null, null, null, null,
+            null, null, null,
+            null,
+            null, null, null,
+            null, null, null, null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null,
+            null, "XXX")
+            .go();
   }
 
   @Test
@@ -132,4 +134,12 @@ public class TestJdbcPluginWithMySQLIT extends PlanTestBase {
     testPhysicalPlanExecutionBasedOnQuery("select * from mysql.`drill_mysql_test`.person");
   }
 
+  @Test
+  public void emptyOutput() throws Exception {
+    String query = "select * from mysql.`drill_mysql_test`.person e limit 0";
+
+    test(query);
+  }
+
+
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindHardDistributionScans.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindHardDistributionScans.java
index 6af8aa4..90cc178 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindHardDistributionScans.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindHardDistributionScans.java
@@ -52,7 +52,12 @@ class FindHardDistributionScans extends RelShuttleImpl {
     DrillTable unwrap;
     unwrap = scan.getTable().unwrap(DrillTable.class);
     if (unwrap == null) {
-      unwrap = scan.getTable().unwrap(DrillTranslatableTable.class).getDrillTable();
+      DrillTranslatableTable drillTranslatableTable = scan.getTable().unwrap(DrillTranslatableTable.class);
+      if (drillTranslatableTable == null) {
+        contains = true; // it rejects single mode.
+        return scan;
+      }
+      unwrap = drillTranslatableTable.getDrillTable();
     }
 
     try {


[drill] 08/15: DRILL-6850: Allow configuring table names case sensitivity for JDBC storage plugin

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

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

commit 13ba486f44315f2ed45b51c065284df62ad470be
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Thu Nov 15 13:07:44 2018 +0200

    DRILL-6850: Allow configuring table names case sensitivity for JDBC storage plugin
    
    closes #1542
---
 contrib/pom.xml                                    |  4 +++
 contrib/storage-jdbc/pom.xml                       | 11 +++++++-
 .../drill/exec/store/jdbc/JdbcStorageConfig.java   | 16 +++++++++---
 .../drill/exec/store/jdbc/JdbcStoragePlugin.java   | 29 +++++++++++++++-------
 .../exec/store/jdbc/TestJdbcPluginWithDerbyIT.java | 20 ++++++++++++++-
 .../exec/store/jdbc/TestJdbcPluginWithMySQLIT.java | 15 +++++++++++
 .../test/resources/bootstrap-storage-plugins.json  | 11 ++++++--
 .../src/test/resources/mysql-test-data.sql         | 10 ++++++--
 .../sql/handlers/FindHardDistributionScans.java    |  3 +++
 9 files changed, 101 insertions(+), 18 deletions(-)

diff --git a/contrib/pom.xml b/contrib/pom.xml
index 796e79b..59e12da 100644
--- a/contrib/pom.xml
+++ b/contrib/pom.xml
@@ -31,6 +31,10 @@
   <name>contrib/Parent Pom</name>
   <packaging>pom</packaging>
 
+  <properties>
+    <skipTests>false</skipTests>
+  </properties>
+
   <dependencies>
   </dependencies>
 
diff --git a/contrib/storage-jdbc/pom.xml b/contrib/storage-jdbc/pom.xml
index 5bad6b1..9895b1a 100755
--- a/contrib/storage-jdbc/pom.xml
+++ b/contrib/storage-jdbc/pom.xml
@@ -34,7 +34,6 @@
     <mysql.connector.version>8.0.13</mysql.connector.version>
     <derby.database.name>drill_derby_test</derby.database.name>
     <mysql.database.name>drill_mysql_test</mysql.database.name>
-    <skipTests>false</skipTests>
   </properties>
 
   <dependencies>
@@ -112,6 +111,7 @@
             <mysql.port>${mysql.reserved.port}</mysql.port>
             <mysql.name>${mysql.database.name}</mysql.name>
           </systemPropertyVariables>
+          <skipITs>${skipTests}</skipITs>
           <includes>
             <include>**/*IT.java</include>
           </includes>
@@ -121,6 +121,15 @@
           <execution>
             <id>run-IT-Tests</id>
             <phase>integration-test</phase>
+            <goals>
+              <goal>integration-test</goal>
+            </goals>
+          </execution>
+          <execution>
+            <phase>verify</phase>
+            <goals>
+              <goal>verify</goal>
+            </goals>
           </execution>
         </executions>
       </plugin>
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStorageConfig.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStorageConfig.java
index 3c3ce3c..1c607d6 100755
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStorageConfig.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStorageConfig.java
@@ -34,18 +34,21 @@ public class JdbcStorageConfig extends StoragePluginConfig {
   private final String url;
   private final String username;
   private final String password;
+  private final boolean caseInsensitiveTableNames;
 
   @JsonCreator
   public JdbcStorageConfig(
       @JsonProperty("driver") String driver,
       @JsonProperty("url") String url,
       @JsonProperty("username") String username,
-      @JsonProperty("password") String password) {
+      @JsonProperty("password") String password,
+      @JsonProperty("caseInsensitiveTableNames") boolean caseInsensitiveTableNames) {
     super();
     this.driver = driver;
     this.url = url;
     this.username = username;
     this.password = password;
+    this.caseInsensitiveTableNames = caseInsensitiveTableNames;
   }
 
   public String getDriver() {
@@ -64,6 +67,11 @@ public class JdbcStorageConfig extends StoragePluginConfig {
     return password;
   }
 
+  @JsonProperty("caseInsensitiveTableNames")
+  public boolean areTableNamesCaseInsensitive() {
+    return caseInsensitiveTableNames;
+  }
+
   @Override
   public int hashCode() {
     final int prime = 31;
@@ -72,6 +80,7 @@ public class JdbcStorageConfig extends StoragePluginConfig {
     result = prime * result + ((password == null) ? 0 : password.hashCode());
     result = prime * result + ((url == null) ? 0 : url.hashCode());
     result = prime * result + ((username == null) ? 0 : username.hashCode());
+    result = prime * result + (caseInsensitiveTableNames ? 1231 : 1237);
     return result;
   }
 
@@ -87,6 +96,9 @@ public class JdbcStorageConfig extends StoragePluginConfig {
       return false;
     }
     JdbcStorageConfig other = (JdbcStorageConfig) obj;
+    if (caseInsensitiveTableNames != other.caseInsensitiveTableNames) {
+      return false;
+    }
     if (driver == null) {
       if (other.driver != null) {
         return false;
@@ -117,6 +129,4 @@ public class JdbcStorageConfig extends StoragePluginConfig {
     }
     return true;
   }
-
-
 }
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 cd9a6c4..ebff371 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
@@ -292,10 +292,24 @@ public class JdbcStoragePlugin extends AbstractStoragePlugin {
       if (table != null) {
         return table;
       }
-      return inner.getTable(name.toUpperCase());
+      if (!areTableNamesCaseSensitive()) {
+        // Oracle and H2 changes unquoted identifiers to uppercase.
+        table = inner.getTable(name.toUpperCase());
+        if (table != null) {
+          return table;
+        }
+        // Postgres changes unquoted identifiers to lowercase.
+        return inner.getTable(name.toLowerCase());
+      }
 
+      // no table was found.
+      return null;
     }
 
+    @Override
+    public boolean areTableNamesCaseSensitive() {
+      return !config.areTableNamesCaseInsensitive();
+    }
   }
 
   private class JdbcCatalogSchema extends AbstractSchema {
@@ -335,8 +349,6 @@ public class JdbcStoragePlugin extends AbstractStoragePlugin {
       }
 
       defaultSchema = schemaMap.values().iterator().next();
-
-
     }
 
     void setHolder(SchemaPlus plusOfThis) {
@@ -405,14 +417,9 @@ public class JdbcStoragePlugin extends AbstractStoragePlugin {
 
       if (schema != null) {
         try {
-          Table t = schema.getTable(name);
-          if (t != null) {
-            return t;
-          }
-          return schema.getTable(name.toUpperCase());
+          return schema.getTable(name);
         } catch (RuntimeException e) {
           logger.warn("Failure while attempting to read table '{}' from JDBC source.", name, e);
-
         }
       }
 
@@ -425,6 +432,10 @@ public class JdbcStoragePlugin extends AbstractStoragePlugin {
       return defaultSchema.getTableNames();
     }
 
+    @Override
+    public boolean areTableNamesCaseSensitive() {
+      return defaultSchema.areTableNamesCaseSensitive();
+    }
   }
 
   @Override
diff --git a/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithDerbyIT.java b/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithDerbyIT.java
index a22b40a..168b5f3 100644
--- a/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithDerbyIT.java
+++ b/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithDerbyIT.java
@@ -41,7 +41,7 @@ public class TestJdbcPluginWithDerbyIT extends PlanTestBase {
   private static final String TABLE_NAME = String.format("%s.`%s`", StoragePluginTestUtils.DFS_PLUGIN_NAME, TABLE_PATH);
 
   @BeforeClass
-  public static void copyData() throws Exception {
+  public static void copyData() {
     dirTestWatcher.copyResourceToRoot(Paths.get(TABLE_PATH));
   }
 
@@ -127,12 +127,18 @@ public class TestJdbcPluginWithDerbyIT extends PlanTestBase {
   public void showTablesDefaultSchema() throws Exception {
     testNoResult("use derby.drill_derby_test");
     assertEquals(1, testSql("show tables like 'PERSON'"));
+
+    // check table names case insensitivity
+    assertEquals(1, testSql("show tables like 'person'"));
   }
 
   @Test
   public void describe() throws Exception {
     testNoResult("use derby.drill_derby_test");
     assertEquals(19, testSql("describe PERSON"));
+
+    // check table names case insensitivity
+    assertEquals(19, testSql("describe person"));
   }
 
   @Test
@@ -147,4 +153,16 @@ public class TestJdbcPluginWithDerbyIT extends PlanTestBase {
     String query = "select * from derby.drill_derby_test.person where person_ID = 1";
     testPlanMatchingPatterns(query, new String[]{}, new String[]{"Filter"});
   }
+
+  @Test
+  public void testCaseInsensitiveTableNames() throws Exception {
+    assertEquals(5, testSql("select * from derby.drill_derby_test.PeRsOn"));
+    assertEquals(5, testSql("select * from derby.drill_derby_test.PERSON"));
+    assertEquals(5, testSql("select * from derby.drill_derby_test.person"));
+  }
+
+  @Test
+  public void testJdbcStoragePluginSerDe() throws Exception {
+    testPhysicalPlanExecutionBasedOnQuery("select * from derby.drill_derby_test.PeRsOn");
+  }
 }
diff --git a/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithMySQLIT.java b/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithMySQLIT.java
index c8396b5..361559c 100644
--- a/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithMySQLIT.java
+++ b/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithMySQLIT.java
@@ -26,6 +26,8 @@ import org.junit.experimental.categories.Category;
 
 import java.math.BigDecimal;
 
+import static org.junit.Assert.assertEquals;
+
 /**
  * JDBC storage plugin tests against MySQL.
  * Note: it requires libaio.so library in the system
@@ -141,5 +143,18 @@ public class TestJdbcPluginWithMySQLIT extends PlanTestBase {
     test(query);
   }
 
+  @Test
+  public void testCaseSensitiveTableNames() throws Exception {
+    test("use mysqlCaseInsensitive.`drill_mysql_test`");
+    // two table names match the filter ignoring the case
+    assertEquals(2, testSql("show tables like 'caseSensitiveTable'"));
+
+    test("use mysql.`drill_mysql_test`");
+    // single table matches the filter considering table name the case
+    assertEquals(1, testSql("show tables like 'caseSensitiveTable'"));
 
+    // checks that tables with names in different case are recognized correctly
+    assertEquals(1, testSql("describe caseSensitiveTable"));
+    assertEquals(2, testSql("describe CASESENSITIVETABLE"));
+  }
 }
diff --git a/contrib/storage-jdbc/src/test/resources/bootstrap-storage-plugins.json b/contrib/storage-jdbc/src/test/resources/bootstrap-storage-plugins.json
index 4018d92..945ddeb 100755
--- a/contrib/storage-jdbc/src/test/resources/bootstrap-storage-plugins.json
+++ b/contrib/storage-jdbc/src/test/resources/bootstrap-storage-plugins.json
@@ -4,15 +4,22 @@
           type    : "jdbc",
           driver  : "org.apache.derby.jdbc.ClientDriver",
           url     : "jdbc:derby://localhost:${derby.reserved.port}/memory:${derby.database.name};user=root;password=root",
+          caseInsensitiveTableNames: true,
           enabled : true
         },
         mysql : {
           type    : "jdbc",
-          enabled : true,
           driver  : "com.mysql.jdbc.Driver",
           url     : "jdbc:mysql://localhost:${mysql.reserved.port}/${mysql.database.name}?user=root&password=root&useJDBCCompliantTimezoneShift=true",
           enabled : true
-      }
+        },
+        mysqlCaseInsensitive : {
+          type    : "jdbc",
+          driver  : "com.mysql.jdbc.Driver",
+          url     : "jdbc:mysql://localhost:${mysql.reserved.port}/${mysql.database.name}?user=root&password=root&useJDBCCompliantTimezoneShift=true",
+          caseInsensitiveTableNames: true,
+          enabled : true
+        }
     }
 }
 
diff --git a/contrib/storage-jdbc/src/test/resources/mysql-test-data.sql b/contrib/storage-jdbc/src/test/resources/mysql-test-data.sql
index 6875d99..92ad6ff 100644
--- a/contrib/storage-jdbc/src/test/resources/mysql-test-data.sql
+++ b/contrib/storage-jdbc/src/test/resources/mysql-test-data.sql
@@ -3,12 +3,18 @@ set global time_zone = "+00:00";
 
 use drill_mysql_test;
 
-create table x (
+create table caseSensitiveTable (
   a   BLOB
 );
 
-insert into x (a) values ('this is a test');
+insert into caseSensitiveTable (a) values ('this is a test');
 
+create table CASESENSITIVETABLE (
+  a   BLOB,
+  b   BLOB
+);
+
+insert into CASESENSITIVETABLE (a, b) values ('this is a test', 'for case sensitive table names');
 
 create table person (
   person_id       INT NOT NULL AUTO_INCREMENT PRIMARY KEY,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindHardDistributionScans.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindHardDistributionScans.java
index 90cc178..7d87aa2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindHardDistributionScans.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindHardDistributionScans.java
@@ -53,6 +53,9 @@ class FindHardDistributionScans extends RelShuttleImpl {
     unwrap = scan.getTable().unwrap(DrillTable.class);
     if (unwrap == null) {
       DrillTranslatableTable drillTranslatableTable = scan.getTable().unwrap(DrillTranslatableTable.class);
+      // For the case, when the underlying Table was obtained from Calcite,
+      // it extends neither DrillTable nor DrillTranslatableTable.
+      // Therefore DistributionAffinity type cannot be determined and single mode is rejected.
       if (drillTranslatableTable == null) {
         contains = true; // it rejects single mode.
         return scan;


[drill] 15/15: DRILL-6834: Introduce option to disable result set for DDL queries for JDBC connection

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

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

commit cd4d68b4c23e7c4cff9769d02bb462cc0707a4ac
Author: Bohdan Kazydub <bo...@gmail.com>
AuthorDate: Wed Nov 21 03:56:49 2018 +0200

    DRILL-6834: Introduce option to disable result set for DDL queries for JDBC connection
    
    - Added session-scoped option `drill.exec.fetch_resultset_for_ddl` to control whether update count or result set should be returned for JDBC connection session. By default the option is set to `true` which ensures that result set is returned;
    - Updated Drill JDBC: `DrillCursor` and `DrillStatement` to achieve desired behaviour.
    
    closes #1549
---
 .../native/client/src/protobuf/UserBitShared.pb.cc | 310 ++++++++++------
 .../native/client/src/protobuf/UserBitShared.pb.h  |  68 +++-
 .../java/org/apache/drill/exec/ExecConstants.java  |   7 +-
 .../impl/materialize/VectorRecordMaterializer.java |  27 +-
 .../drill/exec/planner/sql/DrillSqlWorker.java     |  25 +-
 .../exec/planner/sql/parser/SqlCreateFunction.java |   2 +-
 .../exec/planner/sql/parser/SqlCreateTable.java    |   2 +-
 .../exec/planner/sql/parser/SqlCreateView.java     |   2 +-
 .../exec/planner/sql/parser/SqlDropFunction.java   |   2 +-
 .../exec/planner/sql/parser/SqlDropTable.java      |   2 +-
 .../drill/exec/planner/sql/parser/SqlDropView.java |   2 +-
 .../planner/sql/parser/SqlRefreshMetadata.java     |   2 +-
 .../exec/planner/sql/parser/SqlUseSchema.java      |   3 +-
 .../drill/exec/server/options/OptionList.java      |   8 +
 .../exec/server/options/QueryOptionManager.java    |  14 +-
 .../exec/server/options/SystemOptionManager.java   |   3 +-
 .../exec/store/sys/ExtendedOptionIterator.java     |  17 +-
 .../java-exec/src/main/resources/drill-module.conf |   2 +-
 .../java/org/apache/drill/jdbc/DrillStatement.java |   5 +
 .../org/apache/drill/jdbc/impl/DrillCursor.java    |  14 +-
 .../apache/drill/jdbc/impl/DrillStatementImpl.java |  11 +
 .../org/apache/drill/jdbc/test/TestJdbcQuery.java  | 108 ++++++
 .../drill/exec/proto/SchemaUserBitShared.java      |  14 +
 .../org/apache/drill/exec/proto/UserBitShared.java | 390 +++++++++++++++------
 .../apache/drill/exec/proto/beans/QueryData.java   |  22 ++
 .../drill/exec/proto/beans/RecordBatchDef.java     |  22 ++
 protocol/src/main/protobuf/UserBitShared.proto     |   4 +
 27 files changed, 812 insertions(+), 276 deletions(-)

diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.cc b/contrib/native/client/src/protobuf/UserBitShared.pb.cc
index 8bb6e07..ee81ee2 100644
--- a/contrib/native/client/src/protobuf/UserBitShared.pb.cc
+++ b/contrib/native/client/src/protobuf/UserBitShared.pb.cc
@@ -213,10 +213,11 @@ void protobuf_AssignDesc_UserBitShared_2eproto() {
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(ParsingError));
   RecordBatchDef_descriptor_ = file->message_type(6);
-  static const int RecordBatchDef_offsets_[3] = {
+  static const int RecordBatchDef_offsets_[4] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RecordBatchDef, record_count_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RecordBatchDef, field_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RecordBatchDef, carries_two_byte_selection_vector_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RecordBatchDef, affected_rows_count_),
   };
   RecordBatchDef_reflection_ =
     new ::google::protobuf::internal::GeneratedMessageReflection(
@@ -302,10 +303,11 @@ void protobuf_AssignDesc_UserBitShared_2eproto() {
       sizeof(QueryResult));
   QueryResult_QueryState_descriptor_ = QueryResult_descriptor_->enum_type(0);
   QueryData_descriptor_ = file->message_type(11);
-  static const int QueryData_offsets_[3] = {
+  static const int QueryData_offsets_[4] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryData, query_id_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryData, row_count_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryData, def_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryData, affected_rows_count_),
   };
   QueryData_reflection_ =
     new ::google::protobuf::internal::GeneratedMessageReflection(
@@ -673,117 +675,119 @@ void protobuf_AddDesc_UserBitShared_2eproto() {
     "\013method_name\030\004 \001(\t\022\030\n\020is_native_method\030\005"
     " \001(\010\"\\\n\014ParsingError\022\024\n\014start_column\030\002 \001"
     "(\005\022\021\n\tstart_row\030\003 \001(\005\022\022\n\nend_column\030\004 \001("
-    "\005\022\017\n\007end_row\030\005 \001(\005\"~\n\016RecordBatchDef\022\024\n\014"
-    "record_count\030\001 \001(\005\022+\n\005field\030\002 \003(\0132\034.exec"
-    ".shared.SerializedField\022)\n!carries_two_b"
-    "yte_selection_vector\030\003 \001(\010\"\205\001\n\010NamePart\022"
-    "(\n\004type\030\001 \001(\0162\032.exec.shared.NamePart.Typ"
-    "e\022\014\n\004name\030\002 \001(\t\022$\n\005child\030\003 \001(\0132\025.exec.sh"
-    "ared.NamePart\"\033\n\004Type\022\010\n\004NAME\020\000\022\t\n\005ARRAY"
-    "\020\001\"\324\001\n\017SerializedField\022%\n\nmajor_type\030\001 \001"
-    "(\0132\021.common.MajorType\022(\n\tname_part\030\002 \001(\013"
-    "2\025.exec.shared.NamePart\022+\n\005child\030\003 \003(\0132\034"
-    ".exec.shared.SerializedField\022\023\n\013value_co"
-    "unt\030\004 \001(\005\022\027\n\017var_byte_length\030\005 \001(\005\022\025\n\rbu"
-    "ffer_length\030\007 \001(\005\"7\n\nNodeStatus\022\017\n\007node_"
-    "id\030\001 \001(\005\022\030\n\020memory_footprint\030\002 \001(\003\"\263\002\n\013Q"
-    "ueryResult\0228\n\013query_state\030\001 \001(\0162#.exec.s"
-    "hared.QueryResult.QueryState\022&\n\010query_id"
-    "\030\002 \001(\0132\024.exec.shared.QueryId\022(\n\005error\030\003 "
-    "\003(\0132\031.exec.shared.DrillPBError\"\227\001\n\nQuery"
-    "State\022\014\n\010STARTING\020\000\022\013\n\007RUNNING\020\001\022\r\n\tCOMP"
-    "LETED\020\002\022\014\n\010CANCELED\020\003\022\n\n\006FAILED\020\004\022\032\n\026CAN"
-    "CELLATION_REQUESTED\020\005\022\014\n\010ENQUEUED\020\006\022\r\n\tP"
-    "REPARING\020\007\022\014\n\010PLANNING\020\010\"p\n\tQueryData\022&\n"
-    "\010query_id\030\001 \001(\0132\024.exec.shared.QueryId\022\021\n"
-    "\trow_count\030\002 \001(\005\022(\n\003def\030\003 \001(\0132\033.exec.sha"
-    "red.RecordBatchDef\"\330\001\n\tQueryInfo\022\r\n\005quer"
-    "y\030\001 \001(\t\022\r\n\005start\030\002 \001(\003\0222\n\005state\030\003 \001(\0162#."
-    "exec.shared.QueryResult.QueryState\022\017\n\004us"
-    "er\030\004 \001(\t:\001-\022\'\n\007foreman\030\005 \001(\0132\026.exec.Dril"
-    "lbitEndpoint\022\024\n\014options_json\030\006 \001(\t\022\022\n\nto"
-    "tal_cost\030\007 \001(\001\022\025\n\nqueue_name\030\010 \001(\t:\001-\"\263\004"
-    "\n\014QueryProfile\022 \n\002id\030\001 \001(\0132\024.exec.shared"
-    ".QueryId\022$\n\004type\030\002 \001(\0162\026.exec.shared.Que"
-    "ryType\022\r\n\005start\030\003 \001(\003\022\013\n\003end\030\004 \001(\003\022\r\n\005qu"
-    "ery\030\005 \001(\t\022\014\n\004plan\030\006 \001(\t\022\'\n\007foreman\030\007 \001(\013"
-    "2\026.exec.DrillbitEndpoint\0222\n\005state\030\010 \001(\0162"
-    "#.exec.shared.QueryResult.QueryState\022\027\n\017"
-    "total_fragments\030\t \001(\005\022\032\n\022finished_fragme"
-    "nts\030\n \001(\005\022;\n\020fragment_profile\030\013 \003(\0132!.ex"
-    "ec.shared.MajorFragmentProfile\022\017\n\004user\030\014"
-    " \001(\t:\001-\022\r\n\005error\030\r \001(\t\022\024\n\014verboseError\030\016"
-    " \001(\t\022\020\n\010error_id\030\017 \001(\t\022\022\n\nerror_node\030\020 \001"
-    "(\t\022\024\n\014options_json\030\021 \001(\t\022\017\n\007planEnd\030\022 \001("
-    "\003\022\024\n\014queueWaitEnd\030\023 \001(\003\022\022\n\ntotal_cost\030\024 "
-    "\001(\001\022\025\n\nqueue_name\030\025 \001(\t:\001-\022\017\n\007queryId\030\026 "
-    "\001(\t\"t\n\024MajorFragmentProfile\022\031\n\021major_fra"
-    "gment_id\030\001 \001(\005\022A\n\026minor_fragment_profile"
-    "\030\002 \003(\0132!.exec.shared.MinorFragmentProfil"
-    "e\"\350\002\n\024MinorFragmentProfile\022)\n\005state\030\001 \001("
-    "\0162\032.exec.shared.FragmentState\022(\n\005error\030\002"
-    " \001(\0132\031.exec.shared.DrillPBError\022\031\n\021minor"
-    "_fragment_id\030\003 \001(\005\0226\n\020operator_profile\030\004"
-    " \003(\0132\034.exec.shared.OperatorProfile\022\022\n\nst"
-    "art_time\030\005 \001(\003\022\020\n\010end_time\030\006 \001(\003\022\023\n\013memo"
-    "ry_used\030\007 \001(\003\022\027\n\017max_memory_used\030\010 \001(\003\022("
-    "\n\010endpoint\030\t \001(\0132\026.exec.DrillbitEndpoint"
-    "\022\023\n\013last_update\030\n \001(\003\022\025\n\rlast_progress\030\013"
-    " \001(\003\"\377\001\n\017OperatorProfile\0221\n\rinput_profil"
-    "e\030\001 \003(\0132\032.exec.shared.StreamProfile\022\023\n\013o"
-    "perator_id\030\003 \001(\005\022\025\n\roperator_type\030\004 \001(\005\022"
-    "\023\n\013setup_nanos\030\005 \001(\003\022\025\n\rprocess_nanos\030\006 "
-    "\001(\003\022#\n\033peak_local_memory_allocated\030\007 \001(\003"
-    "\022(\n\006metric\030\010 \003(\0132\030.exec.shared.MetricVal"
-    "ue\022\022\n\nwait_nanos\030\t \001(\003\"B\n\rStreamProfile\022"
-    "\017\n\007records\030\001 \001(\003\022\017\n\007batches\030\002 \001(\003\022\017\n\007sch"
-    "emas\030\003 \001(\003\"J\n\013MetricValue\022\021\n\tmetric_id\030\001"
-    " \001(\005\022\022\n\nlong_value\030\002 \001(\003\022\024\n\014double_value"
-    "\030\003 \001(\001\")\n\010Registry\022\035\n\003jar\030\001 \003(\0132\020.exec.s"
-    "hared.Jar\"/\n\003Jar\022\014\n\004name\030\001 \001(\t\022\032\n\022functi"
-    "on_signature\030\002 \003(\t\"W\n\013SaslMessage\022\021\n\tmec"
-    "hanism\030\001 \001(\t\022\014\n\004data\030\002 \001(\014\022\'\n\006status\030\003 \001"
-    "(\0162\027.exec.shared.SaslStatus*5\n\nRpcChanne"
-    "l\022\017\n\013BIT_CONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER"
-    "\020\002*V\n\tQueryType\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n"
-    "\010PHYSICAL\020\003\022\r\n\tEXECUTION\020\004\022\026\n\022PREPARED_S"
-    "TATEMENT\020\005*\207\001\n\rFragmentState\022\013\n\007SENDING\020"
-    "\000\022\027\n\023AWAITING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022"
-    "\014\n\010FINISHED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005"
-    "\022\032\n\026CANCELLATION_REQUESTED\020\006*\222\t\n\020CoreOpe"
-    "ratorType\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAS"
-    "T_SENDER\020\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGATE"
-    "\020\003\022\r\n\tHASH_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HAS"
-    "H_PARTITION_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGI"
-    "NG_RECEIVER\020\010\022\034\n\030ORDERED_PARTITION_SENDE"
-    "R\020\t\022\013\n\007PROJECT\020\n\022\026\n\022UNORDERED_RECEIVER\020\013"
-    "\022\032\n\026RANGE_PARTITION_SENDER\020\014\022\n\n\006SCREEN\020\r"
-    "\022\034\n\030SELECTION_VECTOR_REMOVER\020\016\022\027\n\023STREAM"
-    "ING_AGGREGATE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTER"
-    "NAL_SORT\020\021\022\t\n\005TRACE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_"
-    "SORT\020\024\022\032\n\026PARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHI"
-    "VE_SUB_SCAN\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\r"
-    "MOCK_SUB_SCAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017D"
-    "IRECT_SUB_SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEX"
-    "T_SUB_SCAN\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_"
-    "SCHEMA_SUB_SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025"
-    "\n\021PRODUCER_CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020"
-    "!\022\n\n\006WINDOW\020\"\022\024\n\020NESTED_LOOP_JOIN\020#\022\021\n\rA"
-    "VRO_SUB_SCAN\020$\022\021\n\rPCAP_SUB_SCAN\020%\022\022\n\016KAF"
-    "KA_SUB_SCAN\020&\022\021\n\rKUDU_SUB_SCAN\020\'\022\013\n\007FLAT"
-    "TEN\020(\022\020\n\014LATERAL_JOIN\020)\022\n\n\006UNNEST\020*\022,\n(H"
-    "IVE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN\020"
-    "+\022\r\n\tJDBC_SCAN\020,\022\022\n\016REGEX_SUB_SCAN\020-\022\023\n\017"
-    "MAPRDB_SUB_SCAN\020.\022\022\n\016MONGO_SUB_SCAN\020/\022\017\n"
-    "\013KUDU_WRITER\0200\022\026\n\022OPEN_TSDB_SUB_SCAN\0201\022\017"
-    "\n\013JSON_WRITER\0202\022\026\n\022HTPPD_LOG_SUB_SCAN\0203\022"
-    "\022\n\016IMAGE_SUB_SCAN\0204\022\025\n\021SEQUENCE_SUB_SCAN"
-    "\0205\022\023\n\017PARTITION_LIMIT\0206\022\023\n\017PCAPNG_SUB_SC"
-    "AN\0207\022\022\n\016RUNTIME_FILTER\0208\022\017\n\013ROWKEY_JOIN\020"
-    "9*g\n\nSaslStatus\022\020\n\014SASL_UNKNOWN\020\000\022\016\n\nSAS"
-    "L_START\020\001\022\024\n\020SASL_IN_PROGRESS\020\002\022\020\n\014SASL_"
-    "SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B.\n\033org.apache"
-    ".drill.exec.protoB\rUserBitSharedH\001", 5474);
+    "\005\022\017\n\007end_row\030\005 \001(\005\"\233\001\n\016RecordBatchDef\022\024\n"
+    "\014record_count\030\001 \001(\005\022+\n\005field\030\002 \003(\0132\034.exe"
+    "c.shared.SerializedField\022)\n!carries_two_"
+    "byte_selection_vector\030\003 \001(\010\022\033\n\023affected_"
+    "rows_count\030\004 \001(\005\"\205\001\n\010NamePart\022(\n\004type\030\001 "
+    "\001(\0162\032.exec.shared.NamePart.Type\022\014\n\004name\030"
+    "\002 \001(\t\022$\n\005child\030\003 \001(\0132\025.exec.shared.NameP"
+    "art\"\033\n\004Type\022\010\n\004NAME\020\000\022\t\n\005ARRAY\020\001\"\324\001\n\017Ser"
+    "ializedField\022%\n\nmajor_type\030\001 \001(\0132\021.commo"
+    "n.MajorType\022(\n\tname_part\030\002 \001(\0132\025.exec.sh"
+    "ared.NamePart\022+\n\005child\030\003 \003(\0132\034.exec.shar"
+    "ed.SerializedField\022\023\n\013value_count\030\004 \001(\005\022"
+    "\027\n\017var_byte_length\030\005 \001(\005\022\025\n\rbuffer_lengt"
+    "h\030\007 \001(\005\"7\n\nNodeStatus\022\017\n\007node_id\030\001 \001(\005\022\030"
+    "\n\020memory_footprint\030\002 \001(\003\"\263\002\n\013QueryResult"
+    "\0228\n\013query_state\030\001 \001(\0162#.exec.shared.Quer"
+    "yResult.QueryState\022&\n\010query_id\030\002 \001(\0132\024.e"
+    "xec.shared.QueryId\022(\n\005error\030\003 \003(\0132\031.exec"
+    ".shared.DrillPBError\"\227\001\n\nQueryState\022\014\n\010S"
+    "TARTING\020\000\022\013\n\007RUNNING\020\001\022\r\n\tCOMPLETED\020\002\022\014\n"
+    "\010CANCELED\020\003\022\n\n\006FAILED\020\004\022\032\n\026CANCELLATION_"
+    "REQUESTED\020\005\022\014\n\010ENQUEUED\020\006\022\r\n\tPREPARING\020\007"
+    "\022\014\n\010PLANNING\020\010\"\215\001\n\tQueryData\022&\n\010query_id"
+    "\030\001 \001(\0132\024.exec.shared.QueryId\022\021\n\trow_coun"
+    "t\030\002 \001(\005\022(\n\003def\030\003 \001(\0132\033.exec.shared.Recor"
+    "dBatchDef\022\033\n\023affected_rows_count\030\004 \001(\005\"\330"
+    "\001\n\tQueryInfo\022\r\n\005query\030\001 \001(\t\022\r\n\005start\030\002 \001"
+    "(\003\0222\n\005state\030\003 \001(\0162#.exec.shared.QueryRes"
+    "ult.QueryState\022\017\n\004user\030\004 \001(\t:\001-\022\'\n\007forem"
+    "an\030\005 \001(\0132\026.exec.DrillbitEndpoint\022\024\n\014opti"
+    "ons_json\030\006 \001(\t\022\022\n\ntotal_cost\030\007 \001(\001\022\025\n\nqu"
+    "eue_name\030\010 \001(\t:\001-\"\263\004\n\014QueryProfile\022 \n\002id"
+    "\030\001 \001(\0132\024.exec.shared.QueryId\022$\n\004type\030\002 \001"
+    "(\0162\026.exec.shared.QueryType\022\r\n\005start\030\003 \001("
+    "\003\022\013\n\003end\030\004 \001(\003\022\r\n\005query\030\005 \001(\t\022\014\n\004plan\030\006 "
+    "\001(\t\022\'\n\007foreman\030\007 \001(\0132\026.exec.DrillbitEndp"
+    "oint\0222\n\005state\030\010 \001(\0162#.exec.shared.QueryR"
+    "esult.QueryState\022\027\n\017total_fragments\030\t \001("
+    "\005\022\032\n\022finished_fragments\030\n \001(\005\022;\n\020fragmen"
+    "t_profile\030\013 \003(\0132!.exec.shared.MajorFragm"
+    "entProfile\022\017\n\004user\030\014 \001(\t:\001-\022\r\n\005error\030\r \001"
+    "(\t\022\024\n\014verboseError\030\016 \001(\t\022\020\n\010error_id\030\017 \001"
+    "(\t\022\022\n\nerror_node\030\020 \001(\t\022\024\n\014options_json\030\021"
+    " \001(\t\022\017\n\007planEnd\030\022 \001(\003\022\024\n\014queueWaitEnd\030\023 "
+    "\001(\003\022\022\n\ntotal_cost\030\024 \001(\001\022\025\n\nqueue_name\030\025 "
+    "\001(\t:\001-\022\017\n\007queryId\030\026 \001(\t\"t\n\024MajorFragment"
+    "Profile\022\031\n\021major_fragment_id\030\001 \001(\005\022A\n\026mi"
+    "nor_fragment_profile\030\002 \003(\0132!.exec.shared"
+    ".MinorFragmentProfile\"\350\002\n\024MinorFragmentP"
+    "rofile\022)\n\005state\030\001 \001(\0162\032.exec.shared.Frag"
+    "mentState\022(\n\005error\030\002 \001(\0132\031.exec.shared.D"
+    "rillPBError\022\031\n\021minor_fragment_id\030\003 \001(\005\0226"
+    "\n\020operator_profile\030\004 \003(\0132\034.exec.shared.O"
+    "peratorProfile\022\022\n\nstart_time\030\005 \001(\003\022\020\n\010en"
+    "d_time\030\006 \001(\003\022\023\n\013memory_used\030\007 \001(\003\022\027\n\017max"
+    "_memory_used\030\010 \001(\003\022(\n\010endpoint\030\t \001(\0132\026.e"
+    "xec.DrillbitEndpoint\022\023\n\013last_update\030\n \001("
+    "\003\022\025\n\rlast_progress\030\013 \001(\003\"\377\001\n\017OperatorPro"
+    "file\0221\n\rinput_profile\030\001 \003(\0132\032.exec.share"
+    "d.StreamProfile\022\023\n\013operator_id\030\003 \001(\005\022\025\n\r"
+    "operator_type\030\004 \001(\005\022\023\n\013setup_nanos\030\005 \001(\003"
+    "\022\025\n\rprocess_nanos\030\006 \001(\003\022#\n\033peak_local_me"
+    "mory_allocated\030\007 \001(\003\022(\n\006metric\030\010 \003(\0132\030.e"
+    "xec.shared.MetricValue\022\022\n\nwait_nanos\030\t \001"
+    "(\003\"B\n\rStreamProfile\022\017\n\007records\030\001 \001(\003\022\017\n\007"
+    "batches\030\002 \001(\003\022\017\n\007schemas\030\003 \001(\003\"J\n\013Metric"
+    "Value\022\021\n\tmetric_id\030\001 \001(\005\022\022\n\nlong_value\030\002"
+    " \001(\003\022\024\n\014double_value\030\003 \001(\001\")\n\010Registry\022\035"
+    "\n\003jar\030\001 \003(\0132\020.exec.shared.Jar\"/\n\003Jar\022\014\n\004"
+    "name\030\001 \001(\t\022\032\n\022function_signature\030\002 \003(\t\"W"
+    "\n\013SaslMessage\022\021\n\tmechanism\030\001 \001(\t\022\014\n\004data"
+    "\030\002 \001(\014\022\'\n\006status\030\003 \001(\0162\027.exec.shared.Sas"
+    "lStatus*5\n\nRpcChannel\022\017\n\013BIT_CONTROL\020\000\022\014"
+    "\n\010BIT_DATA\020\001\022\010\n\004USER\020\002*V\n\tQueryType\022\007\n\003S"
+    "QL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL\020\003\022\r\n\tEXECU"
+    "TION\020\004\022\026\n\022PREPARED_STATEMENT\020\005*\207\001\n\rFragm"
+    "entState\022\013\n\007SENDING\020\000\022\027\n\023AWAITING_ALLOCA"
+    "TION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHED\020\003\022\r\n\tCAN"
+    "CELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCELLATION_REQ"
+    "UESTED\020\006*\222\t\n\020CoreOperatorType\022\021\n\rSINGLE_"
+    "SENDER\020\000\022\024\n\020BROADCAST_SENDER\020\001\022\n\n\006FILTER"
+    "\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHASH_JOIN\020\004\022\016\n"
+    "\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITION_SENDER\020\006"
+    "\022\t\n\005LIMIT\020\007\022\024\n\020MERGING_RECEIVER\020\010\022\034\n\030ORD"
+    "ERED_PARTITION_SENDER\020\t\022\013\n\007PROJECT\020\n\022\026\n\022"
+    "UNORDERED_RECEIVER\020\013\022\032\n\026RANGE_PARTITION_"
+    "SENDER\020\014\022\n\n\006SCREEN\020\r\022\034\n\030SELECTION_VECTOR"
+    "_REMOVER\020\016\022\027\n\023STREAMING_AGGREGATE\020\017\022\016\n\nT"
+    "OP_N_SORT\020\020\022\021\n\rEXTERNAL_SORT\020\021\022\t\n\005TRACE\020"
+    "\022\022\t\n\005UNION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n\026PARQUET_RO"
+    "W_GROUP_SCAN\020\025\022\021\n\rHIVE_SUB_SCAN\020\026\022\025\n\021SYS"
+    "TEM_TABLE_SCAN\020\027\022\021\n\rMOCK_SUB_SCAN\020\030\022\022\n\016P"
+    "ARQUET_WRITER\020\031\022\023\n\017DIRECT_SUB_SCAN\020\032\022\017\n\013"
+    "TEXT_WRITER\020\033\022\021\n\rTEXT_SUB_SCAN\020\034\022\021\n\rJSON"
+    "_SUB_SCAN\020\035\022\030\n\024INFO_SCHEMA_SUB_SCAN\020\036\022\023\n"
+    "\017COMPLEX_TO_JSON\020\037\022\025\n\021PRODUCER_CONSUMER\020"
+    " \022\022\n\016HBASE_SUB_SCAN\020!\022\n\n\006WINDOW\020\"\022\024\n\020NES"
+    "TED_LOOP_JOIN\020#\022\021\n\rAVRO_SUB_SCAN\020$\022\021\n\rPC"
+    "AP_SUB_SCAN\020%\022\022\n\016KAFKA_SUB_SCAN\020&\022\021\n\rKUD"
+    "U_SUB_SCAN\020\'\022\013\n\007FLATTEN\020(\022\020\n\014LATERAL_JOI"
+    "N\020)\022\n\n\006UNNEST\020*\022,\n(HIVE_DRILL_NATIVE_PAR"
+    "QUET_ROW_GROUP_SCAN\020+\022\r\n\tJDBC_SCAN\020,\022\022\n\016"
+    "REGEX_SUB_SCAN\020-\022\023\n\017MAPRDB_SUB_SCAN\020.\022\022\n"
+    "\016MONGO_SUB_SCAN\020/\022\017\n\013KUDU_WRITER\0200\022\026\n\022OP"
+    "EN_TSDB_SUB_SCAN\0201\022\017\n\013JSON_WRITER\0202\022\026\n\022H"
+    "TPPD_LOG_SUB_SCAN\0203\022\022\n\016IMAGE_SUB_SCAN\0204\022"
+    "\025\n\021SEQUENCE_SUB_SCAN\0205\022\023\n\017PARTITION_LIMI"
+    "T\0206\022\023\n\017PCAPNG_SUB_SCAN\0207\022\022\n\016RUNTIME_FILT"
+    "ER\0208\022\017\n\013ROWKEY_JOIN\0209*g\n\nSaslStatus\022\020\n\014S"
+    "ASL_UNKNOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SASL_IN"
+    "_PROGRESS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SASL_FA"
+    "ILED\020\004B.\n\033org.apache.drill.exec.protoB\rU"
+    "serBitSharedH\001", 5534);
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile(
     "UserBitShared.proto", &protobuf_RegisterTypes);
   UserCredentials::default_instance_ = new UserCredentials();
@@ -3076,6 +3080,7 @@ void ParsingError::Swap(ParsingError* other) {
 const int RecordBatchDef::kRecordCountFieldNumber;
 const int RecordBatchDef::kFieldFieldNumber;
 const int RecordBatchDef::kCarriesTwoByteSelectionVectorFieldNumber;
+const int RecordBatchDef::kAffectedRowsCountFieldNumber;
 #endif  // !_MSC_VER
 
 RecordBatchDef::RecordBatchDef()
@@ -3096,6 +3101,7 @@ void RecordBatchDef::SharedCtor() {
   _cached_size_ = 0;
   record_count_ = 0;
   carries_two_byte_selection_vector_ = false;
+  affected_rows_count_ = 0;
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
 }
 
@@ -3133,6 +3139,7 @@ void RecordBatchDef::Clear() {
   if (_has_bits_[0 / 32] & (0xffu << (0 % 32))) {
     record_count_ = 0;
     carries_two_byte_selection_vector_ = false;
+    affected_rows_count_ = 0;
   }
   field_.Clear();
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
@@ -3187,6 +3194,22 @@ bool RecordBatchDef::MergePartialFromCodedStream(
         } else {
           goto handle_uninterpreted;
         }
+        if (input->ExpectTag(32)) goto parse_affected_rows_count;
+        break;
+      }
+
+      // optional int32 affected_rows_count = 4;
+      case 4: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
+         parse_affected_rows_count:
+          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+                   ::google::protobuf::int32, ::google::protobuf::internal::WireFormatLite::TYPE_INT32>(
+                 input, &affected_rows_count_)));
+          set_has_affected_rows_count();
+        } else {
+          goto handle_uninterpreted;
+        }
         if (input->ExpectAtEnd()) return true;
         break;
       }
@@ -3225,6 +3248,11 @@ void RecordBatchDef::SerializeWithCachedSizes(
     ::google::protobuf::internal::WireFormatLite::WriteBool(3, this->carries_two_byte_selection_vector(), output);
   }
 
+  // optional int32 affected_rows_count = 4;
+  if (has_affected_rows_count()) {
+    ::google::protobuf::internal::WireFormatLite::WriteInt32(4, this->affected_rows_count(), output);
+  }
+
   if (!unknown_fields().empty()) {
     ::google::protobuf::internal::WireFormat::SerializeUnknownFields(
         unknown_fields(), output);
@@ -3250,6 +3278,11 @@ void RecordBatchDef::SerializeWithCachedSizes(
     target = ::google::protobuf::internal::WireFormatLite::WriteBoolToArray(3, this->carries_two_byte_selection_vector(), target);
   }
 
+  // optional int32 affected_rows_count = 4;
+  if (has_affected_rows_count()) {
+    target = ::google::protobuf::internal::WireFormatLite::WriteInt32ToArray(4, this->affected_rows_count(), target);
+  }
+
   if (!unknown_fields().empty()) {
     target = ::google::protobuf::internal::WireFormat::SerializeUnknownFieldsToArray(
         unknown_fields(), target);
@@ -3273,6 +3306,13 @@ int RecordBatchDef::ByteSize() const {
       total_size += 1 + 1;
     }
 
+    // optional int32 affected_rows_count = 4;
+    if (has_affected_rows_count()) {
+      total_size += 1 +
+        ::google::protobuf::internal::WireFormatLite::Int32Size(
+          this->affected_rows_count());
+    }
+
   }
   // repeated .exec.shared.SerializedField field = 2;
   total_size += 1 * this->field_size();
@@ -3315,6 +3355,9 @@ void RecordBatchDef::MergeFrom(const RecordBatchDef& from) {
     if (from.has_carries_two_byte_selection_vector()) {
       set_carries_two_byte_selection_vector(from.carries_two_byte_selection_vector());
     }
+    if (from.has_affected_rows_count()) {
+      set_affected_rows_count(from.affected_rows_count());
+    }
   }
   mutable_unknown_fields()->MergeFrom(from.unknown_fields());
 }
@@ -3341,6 +3384,7 @@ void RecordBatchDef::Swap(RecordBatchDef* other) {
     std::swap(record_count_, other->record_count_);
     field_.Swap(&other->field_);
     std::swap(carries_two_byte_selection_vector_, other->carries_two_byte_selection_vector_);
+    std::swap(affected_rows_count_, other->affected_rows_count_);
     std::swap(_has_bits_[0], other->_has_bits_[0]);
     _unknown_fields_.Swap(&other->_unknown_fields_);
     std::swap(_cached_size_, other->_cached_size_);
@@ -4699,6 +4743,7 @@ void QueryResult::Swap(QueryResult* other) {
 const int QueryData::kQueryIdFieldNumber;
 const int QueryData::kRowCountFieldNumber;
 const int QueryData::kDefFieldNumber;
+const int QueryData::kAffectedRowsCountFieldNumber;
 #endif  // !_MSC_VER
 
 QueryData::QueryData()
@@ -4722,6 +4767,7 @@ void QueryData::SharedCtor() {
   query_id_ = NULL;
   row_count_ = 0;
   def_ = NULL;
+  affected_rows_count_ = 0;
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
 }
 
@@ -4766,6 +4812,7 @@ void QueryData::Clear() {
     if (has_def()) {
       if (def_ != NULL) def_->::exec::shared::RecordBatchDef::Clear();
     }
+    affected_rows_count_ = 0;
   }
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
   mutable_unknown_fields()->Clear();
@@ -4816,6 +4863,22 @@ bool QueryData::MergePartialFromCodedStream(
         } else {
           goto handle_uninterpreted;
         }
+        if (input->ExpectTag(32)) goto parse_affected_rows_count;
+        break;
+      }
+
+      // optional int32 affected_rows_count = 4;
+      case 4: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
+         parse_affected_rows_count:
+          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+                   ::google::protobuf::int32, ::google::protobuf::internal::WireFormatLite::TYPE_INT32>(
+                 input, &affected_rows_count_)));
+          set_has_affected_rows_count();
+        } else {
+          goto handle_uninterpreted;
+        }
         if (input->ExpectAtEnd()) return true;
         break;
       }
@@ -4855,6 +4918,11 @@ void QueryData::SerializeWithCachedSizes(
       3, this->def(), output);
   }
 
+  // optional int32 affected_rows_count = 4;
+  if (has_affected_rows_count()) {
+    ::google::protobuf::internal::WireFormatLite::WriteInt32(4, this->affected_rows_count(), output);
+  }
+
   if (!unknown_fields().empty()) {
     ::google::protobuf::internal::WireFormat::SerializeUnknownFields(
         unknown_fields(), output);
@@ -4882,6 +4950,11 @@ void QueryData::SerializeWithCachedSizes(
         3, this->def(), target);
   }
 
+  // optional int32 affected_rows_count = 4;
+  if (has_affected_rows_count()) {
+    target = ::google::protobuf::internal::WireFormatLite::WriteInt32ToArray(4, this->affected_rows_count(), target);
+  }
+
   if (!unknown_fields().empty()) {
     target = ::google::protobuf::internal::WireFormat::SerializeUnknownFieldsToArray(
         unknown_fields(), target);
@@ -4914,6 +4987,13 @@ int QueryData::ByteSize() const {
           this->def());
     }
 
+    // optional int32 affected_rows_count = 4;
+    if (has_affected_rows_count()) {
+      total_size += 1 +
+        ::google::protobuf::internal::WireFormatLite::Int32Size(
+          this->affected_rows_count());
+    }
+
   }
   if (!unknown_fields().empty()) {
     total_size +=
@@ -4950,6 +5030,9 @@ void QueryData::MergeFrom(const QueryData& from) {
     if (from.has_def()) {
       mutable_def()->::exec::shared::RecordBatchDef::MergeFrom(from.def());
     }
+    if (from.has_affected_rows_count()) {
+      set_affected_rows_count(from.affected_rows_count());
+    }
   }
   mutable_unknown_fields()->MergeFrom(from.unknown_fields());
 }
@@ -4976,6 +5059,7 @@ void QueryData::Swap(QueryData* other) {
     std::swap(query_id_, other->query_id_);
     std::swap(row_count_, other->row_count_);
     std::swap(def_, other->def_);
+    std::swap(affected_rows_count_, other->affected_rows_count_);
     std::swap(_has_bits_[0], other->_has_bits_[0]);
     _unknown_fields_.Swap(&other->_unknown_fields_);
     std::swap(_cached_size_, other->_cached_size_);
diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.h b/contrib/native/client/src/protobuf/UserBitShared.pb.h
index ab3063d..3fa1911 100644
--- a/contrib/native/client/src/protobuf/UserBitShared.pb.h
+++ b/contrib/native/client/src/protobuf/UserBitShared.pb.h
@@ -1122,21 +1122,31 @@ class RecordBatchDef : public ::google::protobuf::Message {
   inline bool carries_two_byte_selection_vector() const;
   inline void set_carries_two_byte_selection_vector(bool value);
 
+  // optional int32 affected_rows_count = 4;
+  inline bool has_affected_rows_count() const;
+  inline void clear_affected_rows_count();
+  static const int kAffectedRowsCountFieldNumber = 4;
+  inline ::google::protobuf::int32 affected_rows_count() const;
+  inline void set_affected_rows_count(::google::protobuf::int32 value);
+
   // @@protoc_insertion_point(class_scope:exec.shared.RecordBatchDef)
  private:
   inline void set_has_record_count();
   inline void clear_has_record_count();
   inline void set_has_carries_two_byte_selection_vector();
   inline void clear_has_carries_two_byte_selection_vector();
+  inline void set_has_affected_rows_count();
+  inline void clear_has_affected_rows_count();
 
   ::google::protobuf::UnknownFieldSet _unknown_fields_;
 
   ::google::protobuf::RepeatedPtrField< ::exec::shared::SerializedField > field_;
   ::google::protobuf::int32 record_count_;
   bool carries_two_byte_selection_vector_;
+  ::google::protobuf::int32 affected_rows_count_;
 
   mutable int _cached_size_;
-  ::google::protobuf::uint32 _has_bits_[(3 + 31) / 32];
+  ::google::protobuf::uint32 _has_bits_[(4 + 31) / 32];
 
   friend void  protobuf_AddDesc_UserBitShared_2eproto();
   friend void protobuf_AssignDesc_UserBitShared_2eproto();
@@ -1728,6 +1738,13 @@ class QueryData : public ::google::protobuf::Message {
   inline ::exec::shared::RecordBatchDef* release_def();
   inline void set_allocated_def(::exec::shared::RecordBatchDef* def);
 
+  // optional int32 affected_rows_count = 4;
+  inline bool has_affected_rows_count() const;
+  inline void clear_affected_rows_count();
+  static const int kAffectedRowsCountFieldNumber = 4;
+  inline ::google::protobuf::int32 affected_rows_count() const;
+  inline void set_affected_rows_count(::google::protobuf::int32 value);
+
   // @@protoc_insertion_point(class_scope:exec.shared.QueryData)
  private:
   inline void set_has_query_id();
@@ -1736,15 +1753,18 @@ class QueryData : public ::google::protobuf::Message {
   inline void clear_has_row_count();
   inline void set_has_def();
   inline void clear_has_def();
+  inline void set_has_affected_rows_count();
+  inline void clear_has_affected_rows_count();
 
   ::google::protobuf::UnknownFieldSet _unknown_fields_;
 
   ::exec::shared::QueryId* query_id_;
   ::exec::shared::RecordBatchDef* def_;
   ::google::protobuf::int32 row_count_;
+  ::google::protobuf::int32 affected_rows_count_;
 
   mutable int _cached_size_;
-  ::google::protobuf::uint32 _has_bits_[(3 + 31) / 32];
+  ::google::protobuf::uint32 _has_bits_[(4 + 31) / 32];
 
   friend void  protobuf_AddDesc_UserBitShared_2eproto();
   friend void protobuf_AssignDesc_UserBitShared_2eproto();
@@ -4250,6 +4270,28 @@ inline void RecordBatchDef::set_carries_two_byte_selection_vector(bool value) {
   carries_two_byte_selection_vector_ = value;
 }
 
+// optional int32 affected_rows_count = 4;
+inline bool RecordBatchDef::has_affected_rows_count() const {
+  return (_has_bits_[0] & 0x00000008u) != 0;
+}
+inline void RecordBatchDef::set_has_affected_rows_count() {
+  _has_bits_[0] |= 0x00000008u;
+}
+inline void RecordBatchDef::clear_has_affected_rows_count() {
+  _has_bits_[0] &= ~0x00000008u;
+}
+inline void RecordBatchDef::clear_affected_rows_count() {
+  affected_rows_count_ = 0;
+  clear_has_affected_rows_count();
+}
+inline ::google::protobuf::int32 RecordBatchDef::affected_rows_count() const {
+  return affected_rows_count_;
+}
+inline void RecordBatchDef::set_affected_rows_count(::google::protobuf::int32 value) {
+  set_has_affected_rows_count();
+  affected_rows_count_ = value;
+}
+
 // -------------------------------------------------------------------
 
 // NamePart
@@ -4796,6 +4838,28 @@ inline void QueryData::set_allocated_def(::exec::shared::RecordBatchDef* def) {
   }
 }
 
+// optional int32 affected_rows_count = 4;
+inline bool QueryData::has_affected_rows_count() const {
+  return (_has_bits_[0] & 0x00000008u) != 0;
+}
+inline void QueryData::set_has_affected_rows_count() {
+  _has_bits_[0] |= 0x00000008u;
+}
+inline void QueryData::clear_has_affected_rows_count() {
+  _has_bits_[0] &= ~0x00000008u;
+}
+inline void QueryData::clear_affected_rows_count() {
+  affected_rows_count_ = 0;
+  clear_has_affected_rows_count();
+}
+inline ::google::protobuf::int32 QueryData::affected_rows_count() const {
+  return affected_rows_count_;
+}
+inline void QueryData::set_affected_rows_count(::google::protobuf::int32 value) {
+  set_has_affected_rows_count();
+  affected_rows_count_ = value;
+}
+
 // -------------------------------------------------------------------
 
 // QueryInfo
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 7bfb9cf..fb46572 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
@@ -876,5 +876,10 @@ public final class ExecConstants {
 
   public static final String LIST_FILES_RECURSIVELY = "storage.list_files_recursively";
   public static final BooleanValidator LIST_FILES_RECURSIVELY_VALIDATOR = new BooleanValidator(LIST_FILES_RECURSIVELY,
-      new OptionDescription("Enables recursive files listing when querying the `INFORMATION_SCHEMA.FILES` table or executing the SHOW FILES command. Default is false. (Drill 1.15+"));
+      new OptionDescription("Enables recursive files listing when querying the `INFORMATION_SCHEMA.FILES` table or executing the SHOW FILES command. Default is false. (Drill 1.15+)"));
+
+  public static final String RETURN_RESULT_SET_FOR_DDL = "exec.return_result_set_for_ddl";
+  public static final BooleanValidator RETURN_RESULT_SET_FOR_DDL_VALIDATOR = new BooleanValidator(RETURN_RESULT_SET_FOR_DDL,
+      new OptionDescription("Controls whether to return result set for CREATE TABLE/VIEW, DROP TABLE/VIEW, SET, USE etc. queries. " +
+          "If set to false affected rows count will be returned instead and result set will be null. Default is true. (Drill 1.15+)"));
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
index 2d8c231..7cdf9b3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.materialize;
 
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OperatorContext;
@@ -25,13 +26,15 @@ import org.apache.drill.exec.proto.UserBitShared.QueryData;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.server.options.OptionManager;
 
-public class VectorRecordMaterializer implements RecordMaterializer{
+public class VectorRecordMaterializer implements RecordMaterializer {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VectorRecordMaterializer.class);
 
   private QueryId queryId;
   private RecordBatch batch;
   private BufferAllocator allocator;
+  private OptionManager options;
 
   public VectorRecordMaterializer(FragmentContext context, OperatorContext oContext, RecordBatch batch) {
     this.queryId = context.getHandle().getQueryId();
@@ -39,21 +42,19 @@ public class VectorRecordMaterializer implements RecordMaterializer{
     this.allocator = oContext.getAllocator();
     BatchSchema schema = batch.getSchema();
     assert schema != null : "Schema must be defined.";
-
-//    for (MaterializedField f : batch.getSchema()) {
-//      logger.debug("New Field: {}", f);
-//    }
+    options = context.getOptions();
   }
 
   public QueryWritableBatch convertNext() {
-    //batch.getWritableBatch().getDef().getRecordCount()
     WritableBatch w = batch.getWritableBatch().transfer(allocator);
-
-    QueryData header = QueryData.newBuilder() //
-        .setQueryId(queryId) //
-        .setRowCount(batch.getRecordCount()) //
-        .setDef(w.getDef()).build();
-    QueryWritableBatch batch = new QueryWritableBatch(header, w.getBuffers());
-    return batch;
+    QueryData.Builder builder = QueryData.newBuilder()
+        .setQueryId(queryId)
+        .setRowCount(batch.getRecordCount())
+        .setDef(w.getDef());
+    if (!options.getBoolean(ExecConstants.RETURN_RESULT_SET_FOR_DDL)) {
+      int count = w.getDef().getAffectedRowsCount();
+      builder.setAffectedRowsCount(count == -1 ? 0 : count);
+    }
+    return new QueryWritableBatch(builder.build(), w.getBuffers());
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
index 41faea9..7a4fcdf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
@@ -20,10 +20,12 @@ package org.apache.drill.exec.planner.sql;
 import java.io.IOException;
 
 import org.apache.calcite.sql.SqlDescribeSchema;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.tools.RelConversionException;
 import org.apache.calcite.tools.ValidationException;
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.planner.sql.handlers.AbstractSqlHandler;
@@ -35,7 +37,6 @@ import org.apache.drill.exec.planner.sql.handlers.SetOptionHandler;
 import org.apache.drill.exec.planner.sql.handlers.SqlHandlerConfig;
 import org.apache.drill.exec.planner.sql.parser.DrillSqlCall;
 import org.apache.drill.exec.planner.sql.parser.DrillSqlDescribeTable;
-import org.apache.drill.exec.planner.sql.parser.SqlCreateTable;
 import org.apache.drill.exec.testing.ControlsInjector;
 import org.apache.drill.exec.testing.ControlsInjectorFactory;
 import org.apache.drill.exec.util.Pointer;
@@ -110,7 +111,7 @@ public class DrillSqlWorker {
     final AbstractSqlHandler handler;
     final SqlHandlerConfig config = new SqlHandlerConfig(context, parser);
 
-    switch(sqlNode.getKind()){
+    switch(sqlNode.getKind()) {
     case EXPLAIN:
       handler = new ExplainHandler(config, textPlan);
       break;
@@ -127,14 +128,16 @@ public class DrillSqlWorker {
         handler = new DescribeSchemaHandler(config);
         break;
       }
+    case CREATE_TABLE:
+      handler = ((DrillSqlCall) sqlNode).getSqlHandler(config, textPlan);
+      break;
+    case DROP_TABLE:
+    case CREATE_VIEW:
+    case DROP_VIEW:
+    case OTHER_DDL:
     case OTHER:
-      if(sqlNode instanceof SqlCreateTable) {
-        handler = ((DrillSqlCall)sqlNode).getSqlHandler(config, textPlan);
-        break;
-      }
-
       if (sqlNode instanceof DrillSqlCall) {
-        handler = ((DrillSqlCall)sqlNode).getSqlHandler(config);
+        handler = ((DrillSqlCall) sqlNode).getSqlHandler(config);
         break;
       }
       // fallthrough
@@ -142,6 +145,12 @@ public class DrillSqlWorker {
       handler = new DefaultSqlHandler(config, textPlan);
     }
 
+    boolean returnResultSet = context.getOptions().getBoolean(ExecConstants.RETURN_RESULT_SET_FOR_DDL);
+    // Determine whether result set should be returned for the query based on `exec.return_result_set_for_ddl`
+    // and sql node kind. Overrides the option on a query level.
+    context.getOptions().setLocalOption(ExecConstants.RETURN_RESULT_SET_FOR_DDL,
+        returnResultSet || !SqlKind.DDL.contains(sqlNode.getKind()));
+
     try {
       return handler.getPlan(sqlNode);
     } catch(ValidationException e) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateFunction.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateFunction.java
index eeab237..8861887 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateFunction.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateFunction.java
@@ -36,7 +36,7 @@ public class SqlCreateFunction extends DrillSqlCall {
 
   private final SqlNode jar;
 
-  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("CREATE_FUNCTION", SqlKind.OTHER) {
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("CREATE_FUNCTION", SqlKind.OTHER_DDL) {
     @Override
     public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
       return new SqlCreateFunction(pos, operands[0]);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateTable.java
index 48b2e1b..11e33aa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateTable.java
@@ -41,7 +41,7 @@ import org.apache.drill.exec.planner.sql.handlers.SqlHandlerUtil;
 import org.apache.drill.exec.util.Pointer;
 
 public class SqlCreateTable extends DrillSqlCall {
-  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("CREATE_TABLE", SqlKind.OTHER) {
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("CREATE_TABLE", SqlKind.CREATE_TABLE) {
     @Override
     public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
       Preconditions.checkArgument(operands.length == 6, "SqlCreateTable.createCall() has to get 6 operands!");
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateView.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateView.java
index 9e8bd8a..f61aeaa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateView.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateView.java
@@ -37,7 +37,7 @@ import org.apache.calcite.sql.SqlWriter;
 import java.util.List;
 
 public class SqlCreateView extends DrillSqlCall {
-  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("CREATE_VIEW", SqlKind.OTHER) {
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("CREATE_VIEW", SqlKind.CREATE_VIEW) {
     @Override
     public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
       return new SqlCreateView(pos, (SqlIdentifier) operands[0], (SqlNodeList) operands[1], operands[2], (SqlLiteral) operands[3]);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropFunction.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropFunction.java
index b7fef1b..94839f3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropFunction.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropFunction.java
@@ -36,7 +36,7 @@ public class SqlDropFunction extends DrillSqlCall {
 
   private final SqlNode jar;
 
-  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("DROP_FUNCTION", SqlKind.OTHER) {
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("DROP_FUNCTION", SqlKind.OTHER_DDL) {
     @Override
     public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
       return new SqlDropFunction(pos, operands[0]);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTable.java
index cbdb415..dbc6788 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropTable.java
@@ -35,7 +35,7 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 
 public class SqlDropTable extends DrillSqlCall {
-  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("DROP_TABLE", SqlKind.OTHER) {
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("DROP_TABLE", SqlKind.DROP_TABLE) {
     @Override
     public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
       return new SqlDropTable(pos, (SqlIdentifier) operands[0], (SqlLiteral) operands[1]);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropView.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropView.java
index d137cb9..bfd3474 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropView.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropView.java
@@ -35,7 +35,7 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 
 public class SqlDropView extends DrillSqlCall {
-  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("DROP_VIEW", SqlKind.OTHER) {
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("DROP_VIEW", SqlKind.DROP_VIEW) {
     @Override
     public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
       return new SqlDropView(pos, (SqlIdentifier) operands[0], (SqlLiteral) operands[1]);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlRefreshMetadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlRefreshMetadata.java
index 72fc397..84d95f4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlRefreshMetadata.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlRefreshMetadata.java
@@ -40,7 +40,7 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
  * REFRESH TABLE METADATA tblname
  */
 public class SqlRefreshMetadata extends DrillSqlCall {
-  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("REFRESH_TABLE_METADATA", SqlKind.OTHER) {
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("REFRESH_TABLE_METADATA", SqlKind.OTHER_DDL) {
     @Override
     public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
       return new SqlRefreshMetadata(pos, (SqlIdentifier) operands[0]);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlUseSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlUseSchema.java
index df2bc1d..8c05dc4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlUseSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlUseSchema.java
@@ -38,8 +38,7 @@ import java.util.List;
  */
 public class SqlUseSchema extends DrillSqlCall {
 
-  public static final SqlSpecialOperator OPERATOR =
-      new SqlSpecialOperator("USE_SCHEMA", SqlKind.OTHER){
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("USE_SCHEMA", SqlKind.OTHER_DDL) {
     @Override
     public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
       return new SqlUseSchema(pos, (SqlIdentifier) operands[0]);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionList.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionList.java
index e2161aa..bef6990 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionList.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/OptionList.java
@@ -18,10 +18,18 @@
 package org.apache.drill.exec.server.options;
 
 import java.util.ArrayList;
+import java.util.Collection;
 
 @SuppressWarnings("serial")
 public class OptionList extends ArrayList<OptionValue>{
 
+  public OptionList() {
+  }
+
+  public OptionList(Collection<OptionValue> options) {
+    super(options);
+  }
+
   public void merge(OptionList list){
     this.addAll(list);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/QueryOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/QueryOptionManager.java
index 1c7d298..1241726 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/QueryOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/QueryOptionManager.java
@@ -19,6 +19,9 @@ package org.apache.drill.exec.server.options;
 
 import org.apache.drill.common.map.CaseInsensitiveMap;
 
+import java.util.HashMap;
+import java.util.Map;
+
 /**
  * {@link OptionManager} that holds options within {@link org.apache.drill.exec.ops.QueryContext}.
  */
@@ -31,9 +34,14 @@ public class QueryOptionManager extends InMemoryOptionManager {
 
   @Override
   public OptionList getOptionList() {
-    OptionList list = super.getOptionList();
-    list.merge(fallback.getOptionList());
-    return list;
+    Map<String, OptionValue> optionMap = new HashMap<>();
+    for (OptionValue option : fallback.getOptionList()) {
+      optionMap.put(option.name, option);
+    }
+    for (OptionValue option : super.getOptionList()) {
+      optionMap.put(option.name, option);
+    }
+    return new OptionList(optionMap.values());
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index 7897c3b..37934c8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -267,7 +267,8 @@ public class SystemOptionManager extends BaseOptionManager implements AutoClosea
       new OptionDefinition(ExecConstants.OUTPUT_BATCH_SIZE_AVAIL_MEM_FACTOR_VALIDATOR, new OptionMetaData(OptionValue.AccessibleScopes.SYSTEM, true, false)),
       new OptionDefinition(ExecConstants.FRAG_RUNNER_RPC_TIMEOUT_VALIDATOR, new OptionMetaData(OptionValue.AccessibleScopes.SYSTEM, true, true)),
       new OptionDefinition(ExecConstants.LIST_FILES_RECURSIVELY_VALIDATOR),
-      new OptionDefinition(ExecConstants.QUERY_ROWKEYJOIN_BATCHSIZE)
+      new OptionDefinition(ExecConstants.QUERY_ROWKEYJOIN_BATCHSIZE),
+      new OptionDefinition(ExecConstants.RETURN_RESULT_SET_FOR_DDL_VALIDATOR)
     };
 
     CaseInsensitiveMap<OptionDefinition> map = Arrays.stream(definitions)
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ExtendedOptionIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ExtendedOptionIterator.java
index a8c3c84..f6c7c08 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ExtendedOptionIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/ExtendedOptionIterator.java
@@ -35,7 +35,7 @@ import org.apache.drill.exec.store.pojo.NonNullable;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
-/*
+/**
  * Extends the original Option iterator. The idea is to hide the implementation details and present the
  * user with the rows which have values set at the top level of hierarchy and exclude the values set
  * at lower levels. This is done by examining the scope and the precedence order of scope is session - system - default.
@@ -55,22 +55,16 @@ import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
  *  only the value set at SESSION level.
  */
 public class ExtendedOptionIterator implements Iterator<Object> {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExtendedOptionIterator.class);
 
   private final OptionManager fragmentOptions;
   private final Iterator<OptionValue> mergedOptions;
   private Map<OptionValue.Kind, String> typeMapping;
-  private Map<OptionScope, Integer> preference;
   private static final int SHORT_DESCRIP_MAX_SIZE = 110;
 
   public ExtendedOptionIterator(FragmentContext context, boolean internal) {
     fragmentOptions = context.getOptions();
-    preference = new HashMap<OptionScope, Integer>();
-    preference.put(OptionScope.SESSION, 0);
-    preference.put(OptionScope.SYSTEM, 1);
-    preference.put(OptionScope.BOOT, 2);
 
-    typeMapping = new HashMap<Kind, String>();
+    typeMapping = new HashMap<>();
     typeMapping.put(Kind.STRING, "VARCHAR");
     typeMapping.put(Kind.DOUBLE, "FLOAT");
     typeMapping.put(Kind.LONG, "BIGINT");
@@ -92,9 +86,14 @@ public class ExtendedOptionIterator implements Iterator<Object> {
     HashMap<String, OptionValue> optionsmap = new HashMap<>();
 
     for (OptionValue option : optionslist) {
+      if (option.scope == OptionScope.QUERY) {
+        // Option set on query level should be ignored here as its value should not be shown to user
+        continue;
+      }
+
       if (optionsmap.containsKey(option.getName())) {
 
-        if (preference.get(option.scope) < preference.get(optionsmap.get(option.getName()).scope)) {
+        if (option.scope.compareTo(optionsmap.get(option.getName()).scope) > 0) {
           optionsmap.put(option.getName(), option);
         }
 
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index 632b4ed..8aa3233 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -644,5 +644,5 @@ drill.exec.options: {
     planner.index.prefer_intersect_plans: false,
     planner.index.max_indexes_to_intersect: 5,
     exec.query.rowkeyjoin_batchsize: 128,
-
+    exec.return_result_set_for_ddl: true,
 }
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java
index 1bddacb..f93ef3b 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.jdbc;
 
+import org.apache.calcite.avatica.AvaticaResultSet;
+
 import java.sql.SQLException;
 import java.sql.Statement;
 
@@ -60,4 +62,7 @@ public interface DrillStatement extends Statement {
   @Override
   boolean isClosed();
 
+  void setResultSet(AvaticaResultSet resultSet);
+
+  void setUpdateCount(int value);
 }
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
index 20b0652..888a909 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java
@@ -29,6 +29,7 @@ import java.util.concurrent.LinkedBlockingDeque;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.drill.jdbc.DrillStatement;
 import org.apache.drill.shaded.guava.com.google.common.base.Stopwatch;
 import org.apache.calcite.avatica.AvaticaStatement;
 import org.apache.calcite.avatica.ColumnMetaData;
@@ -325,7 +326,7 @@ class DrillCursor implements Cursor {
    * <p>
    *   (Relates to {@link #loadInitialSchema()}'s calling
    *   {@link #nextRowInternally()} one "extra" time (extra relative to number
-   *   of {@link ResultSet#next()} calls) at the beginning to get first batch
+   *   of {@link java.sql.ResultSet#next()} calls) at the beginning to get first batch
    *   and schema before {@code Statement.execute...(...)} even returns.)
    * </p>
    */
@@ -450,7 +451,7 @@ class DrillCursor implements Cursor {
    * <p>
    *   Is to be called (once) from {@link #loadInitialSchema} for
    *   {@link DrillResultSetImpl#execute()}, and then (repeatedly) from
-   *   {@link #next()} for {@link AvaticaResultSet#next()}.
+   *   {@link #next()} for {@link org.apache.calcite.avatica.AvaticaResultSet#next()}.
    * </p>
    *
    * @return  whether cursor is positioned at a row (false when after end of
@@ -499,6 +500,13 @@ class DrillCursor implements Cursor {
 
           currentRecordNumber = 0;
 
+          if (qrb.getHeader().hasAffectedRowsCount()) {
+            int updateCount = qrb.getHeader().getAffectedRowsCount();
+            int currentUpdateCount = statement.getUpdateCount() == -1 ? 0 : statement.getUpdateCount();
+            ((DrillStatement) statement).setUpdateCount(updateCount + currentUpdateCount);
+            ((DrillStatement) statement).setResultSet(null);
+          }
+
           final boolean schemaChanged;
           try {
             schemaChanged = currentBatchHolder.load(qrb.getHeader().getDef(),
@@ -549,7 +557,7 @@ class DrillCursor implements Cursor {
    * Advances to first batch to load schema data into result set metadata.
    * <p>
    *   To be called once from {@link DrillResultSetImpl#execute()} before
-   *   {@link #next()} is called from {@link AvaticaResultSet#next()}.
+   *   {@link #next()} is called from {@link org.apache.calcite.avatica.AvaticaResultSet#next()}.
    * <p>
    */
   void loadInitialSchema() throws SQLException {
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 5386ec6..7255c75 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
@@ -21,6 +21,7 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 
+import org.apache.calcite.avatica.AvaticaResultSet;
 import org.apache.calcite.avatica.AvaticaStatement;
 import org.apache.calcite.avatica.Meta.StatementHandle;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
@@ -259,4 +260,14 @@ class DrillStatementImpl extends AvaticaStatement implements DrillStatement,
       throw new SQLFeatureNotSupportedException(e.getMessage(), e);
     }
   }
+
+  @Override
+  public void setResultSet(AvaticaResultSet resultSet) {
+    openResultSet = resultSet;
+  }
+
+  @Override
+  public void setUpdateCount(int value) {
+    updateCount = value;
+  }
 }
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java
index 24a5661..ab854aa 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java
@@ -18,6 +18,7 @@
 package org.apache.drill.jdbc.test;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 
 import java.nio.file.Paths;
 import java.sql.Connection;
@@ -26,6 +27,7 @@ import java.sql.Statement;
 import java.sql.Types;
 
 import org.apache.drill.categories.JdbcTest;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.jdbc.JdbcTestBase;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
@@ -403,4 +405,110 @@ public class TestJdbcQuery extends JdbcTestQueryBase {
         .sql("SELECT CONVERT_FROM(columns[1], 'JSON') as col1 from cp.`empty.csv`")
         .returns("");
   }
+
+  @Test
+  public void testResultSetIsNotReturnedSet() throws Exception {
+    try (Connection conn = connect();
+         Statement s = conn.createStatement()) {
+
+      s.execute(String.format("SET `%s` = false", ExecConstants.RETURN_RESULT_SET_FOR_DDL));
+
+      // Set any option
+      s.execute(String.format("SET `%s` = 'json'", ExecConstants.OUTPUT_FORMAT_OPTION));
+      assertNull("No result", s.getResultSet());
+    }
+  }
+
+  @Test
+  public void testResultSetIsNotReturnedCTAS() throws Exception {
+    String tableName = "dfs.tmp.`ctas`";
+
+    try (Connection conn = connect();
+         Statement s = conn.createStatement()) {
+      s.execute(String.format("SET `%s` = false", ExecConstants.RETURN_RESULT_SET_FOR_DDL));
+
+      s.execute(String.format("CREATE TABLE %s AS SELECT * FROM cp.`employee.json`", tableName));
+      assertNull("No result", s.getResultSet());
+    } finally {
+      execute("DROP TABLE IF EXISTS %s", tableName);
+    }
+  }
+
+  @Test
+  public void testResultSetIsNotReturnedCreateView() throws Exception {
+    String viewName = "dfs.tmp.`cv`";
+
+    try (Connection conn = connect();
+         Statement s = conn.createStatement()) {
+      s.execute(String.format("SET `%s` = false", ExecConstants.RETURN_RESULT_SET_FOR_DDL));
+
+      s.execute(String.format("CREATE VIEW %s AS SELECT * FROM cp.`employee.json`", viewName));
+      assertNull("No result", s.getResultSet());
+    } finally {
+      execute("DROP VIEW IF EXISTS %s", viewName);
+    }
+  }
+
+  @Test
+  public void testResultSetIsNotReturnedDropTable() throws Exception {
+    String tableName = "dfs.tmp.`dt`";
+
+    try (Connection conn = connect();
+         Statement s = conn.createStatement()) {
+      s.execute(String.format("SET `%s` = false", ExecConstants.RETURN_RESULT_SET_FOR_DDL));
+
+      s.execute(String.format("CREATE TABLE %s AS SELECT * FROM cp.`employee.json`", tableName));
+
+      s.execute(String.format("DROP TABLE %s", tableName));
+      assertNull("No result", s.getResultSet());
+    }
+  }
+
+  @Test
+  public void testResultSetIsNotReturnedDropView() throws Exception {
+    String viewName = "dfs.tmp.`dv`";
+
+    try (Connection conn = connect();
+         Statement stmt = conn.createStatement()) {
+      stmt.execute(String.format("SET `%s` = false", ExecConstants.RETURN_RESULT_SET_FOR_DDL));
+
+      stmt.execute(String.format("CREATE VIEW %s AS SELECT * FROM cp.`employee.json`", viewName));
+
+      stmt.execute(String.format("DROP VIEW %s", viewName));
+      assertNull("No result", stmt.getResultSet());
+    }
+  }
+
+  @Test
+  public void testResultSetIsNotReturnedUse() throws Exception {
+    try (Connection conn = connect();
+         Statement s = conn.createStatement()) {
+      s.execute(String.format("SET `%s` = false", ExecConstants.RETURN_RESULT_SET_FOR_DDL));
+
+      s.execute("USE dfs.tmp");
+      assertNull("No result", s.getResultSet());
+    }
+  }
+
+  @Test
+  public void testResultSetIsNotReturnedRefreshMetadata() throws Exception {
+    String tableName = "dfs.tmp.`rm`";
+
+    try (Connection conn = connect();
+         Statement s = conn.createStatement()) {
+      s.execute(String.format("SET `%s` = false", ExecConstants.RETURN_RESULT_SET_FOR_DDL));
+
+      s.execute(String.format("CREATE TABLE %s AS SELECT * FROM cp.`employee.json`", tableName));
+
+      s.execute(String.format("REFRESH TABLE METADATA %s", tableName));
+      assertNull("No result", s.getResultSet());
+    }
+  }
+
+  private static void execute(String sql, Object... params) throws Exception {
+    try (Connection conn = connect();
+         Statement s = conn.createStatement()) {
+      s.execute(String.format(sql, params));
+    }
+  }
 }
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java
index 7b845a4..ba64762 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaUserBitShared.java
@@ -830,6 +830,8 @@ public final class SchemaUserBitShared
 
                 if(message.hasCarriesTwoByteSelectionVector())
                     output.writeBool(3, message.getCarriesTwoByteSelectionVector(), false);
+                if(message.hasAffectedRowsCount())
+                    output.writeInt32(4, message.getAffectedRowsCount(), false);
             }
             public boolean isInitialized(org.apache.drill.exec.proto.UserBitShared.RecordBatchDef message)
             {
@@ -879,6 +881,9 @@ public final class SchemaUserBitShared
                         case 3:
                             builder.setCarriesTwoByteSelectionVector(input.readBool());
                             break;
+                        case 4:
+                            builder.setAffectedRowsCount(input.readInt32());
+                            break;
                         default:
                             input.handleUnknownField(number, this);
                     }
@@ -922,6 +927,7 @@ public final class SchemaUserBitShared
                 case 1: return "recordCount";
                 case 2: return "field";
                 case 3: return "carriesTwoByteSelectionVector";
+                case 4: return "affectedRowsCount";
                 default: return null;
             }
         }
@@ -936,6 +942,7 @@ public final class SchemaUserBitShared
             fieldMap.put("recordCount", 1);
             fieldMap.put("field", 2);
             fieldMap.put("carriesTwoByteSelectionVector", 3);
+            fieldMap.put("affectedRowsCount", 4);
         }
     }
 
@@ -1484,6 +1491,8 @@ public final class SchemaUserBitShared
                 if(message.hasDef())
                     output.writeObject(3, message.getDef(), org.apache.drill.exec.proto.SchemaUserBitShared.RecordBatchDef.WRITE, false);
 
+                if(message.hasAffectedRowsCount())
+                    output.writeInt32(4, message.getAffectedRowsCount(), false);
             }
             public boolean isInitialized(org.apache.drill.exec.proto.UserBitShared.QueryData message)
             {
@@ -1534,6 +1543,9 @@ public final class SchemaUserBitShared
                             builder.setDef(input.mergeObject(org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.newBuilder(), org.apache.drill.exec.proto.SchemaUserBitShared.RecordBatchDef.MERGE));
 
                             break;
+                        case 4:
+                            builder.setAffectedRowsCount(input.readInt32());
+                            break;
                         default:
                             input.handleUnknownField(number, this);
                     }
@@ -1577,6 +1589,7 @@ public final class SchemaUserBitShared
                 case 1: return "queryId";
                 case 2: return "rowCount";
                 case 3: return "def";
+                case 4: return "affectedRowsCount";
                 default: return null;
             }
         }
@@ -1591,6 +1604,7 @@ public final class SchemaUserBitShared
             fieldMap.put("queryId", 1);
             fieldMap.put("rowCount", 2);
             fieldMap.put("def", 3);
+            fieldMap.put("affectedRowsCount", 4);
         }
     }
 
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
index 2f5c3de..f969639 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
@@ -6796,6 +6796,16 @@ public final class UserBitShared {
      * <code>optional bool carries_two_byte_selection_vector = 3;</code>
      */
     boolean getCarriesTwoByteSelectionVector();
+
+    // optional int32 affected_rows_count = 4;
+    /**
+     * <code>optional int32 affected_rows_count = 4;</code>
+     */
+    boolean hasAffectedRowsCount();
+    /**
+     * <code>optional int32 affected_rows_count = 4;</code>
+     */
+    int getAffectedRowsCount();
   }
   /**
    * Protobuf type {@code exec.shared.RecordBatchDef}
@@ -6866,6 +6876,11 @@ public final class UserBitShared {
               carriesTwoByteSelectionVector_ = input.readBool();
               break;
             }
+            case 32: {
+              bitField0_ |= 0x00000004;
+              affectedRowsCount_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -6977,10 +6992,27 @@ public final class UserBitShared {
       return carriesTwoByteSelectionVector_;
     }
 
+    // optional int32 affected_rows_count = 4;
+    public static final int AFFECTED_ROWS_COUNT_FIELD_NUMBER = 4;
+    private int affectedRowsCount_;
+    /**
+     * <code>optional int32 affected_rows_count = 4;</code>
+     */
+    public boolean hasAffectedRowsCount() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional int32 affected_rows_count = 4;</code>
+     */
+    public int getAffectedRowsCount() {
+      return affectedRowsCount_;
+    }
+
     private void initFields() {
       recordCount_ = 0;
       field_ = java.util.Collections.emptyList();
       carriesTwoByteSelectionVector_ = false;
+      affectedRowsCount_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -7003,6 +7035,9 @@ public final class UserBitShared {
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeBool(3, carriesTwoByteSelectionVector_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeInt32(4, affectedRowsCount_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -7024,6 +7059,10 @@ public final class UserBitShared {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, carriesTwoByteSelectionVector_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(4, affectedRowsCount_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -7151,6 +7190,8 @@ public final class UserBitShared {
         }
         carriesTwoByteSelectionVector_ = false;
         bitField0_ = (bitField0_ & ~0x00000004);
+        affectedRowsCount_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000008);
         return this;
       }
 
@@ -7196,6 +7237,10 @@ public final class UserBitShared {
           to_bitField0_ |= 0x00000002;
         }
         result.carriesTwoByteSelectionVector_ = carriesTwoByteSelectionVector_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.affectedRowsCount_ = affectedRowsCount_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -7244,6 +7289,9 @@ public final class UserBitShared {
         if (other.hasCarriesTwoByteSelectionVector()) {
           setCarriesTwoByteSelectionVector(other.getCarriesTwoByteSelectionVector());
         }
+        if (other.hasAffectedRowsCount()) {
+          setAffectedRowsCount(other.getAffectedRowsCount());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -7577,6 +7625,39 @@ public final class UserBitShared {
         return this;
       }
 
+      // optional int32 affected_rows_count = 4;
+      private int affectedRowsCount_ ;
+      /**
+       * <code>optional int32 affected_rows_count = 4;</code>
+       */
+      public boolean hasAffectedRowsCount() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional int32 affected_rows_count = 4;</code>
+       */
+      public int getAffectedRowsCount() {
+        return affectedRowsCount_;
+      }
+      /**
+       * <code>optional int32 affected_rows_count = 4;</code>
+       */
+      public Builder setAffectedRowsCount(int value) {
+        bitField0_ |= 0x00000008;
+        affectedRowsCount_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 affected_rows_count = 4;</code>
+       */
+      public Builder clearAffectedRowsCount() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        affectedRowsCount_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:exec.shared.RecordBatchDef)
     }
 
@@ -11586,6 +11667,16 @@ public final class UserBitShared {
      * <code>optional .exec.shared.RecordBatchDef def = 3;</code>
      */
     org.apache.drill.exec.proto.UserBitShared.RecordBatchDefOrBuilder getDefOrBuilder();
+
+    // optional int32 affected_rows_count = 4;
+    /**
+     * <code>optional int32 affected_rows_count = 4;</code>
+     */
+    boolean hasAffectedRowsCount();
+    /**
+     * <code>optional int32 affected_rows_count = 4;</code>
+     */
+    int getAffectedRowsCount();
   }
   /**
    * Protobuf type {@code exec.shared.QueryData}
@@ -11674,6 +11765,11 @@ public final class UserBitShared {
               bitField0_ |= 0x00000004;
               break;
             }
+            case 32: {
+              bitField0_ |= 0x00000008;
+              affectedRowsCount_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -11774,10 +11870,27 @@ public final class UserBitShared {
       return def_;
     }
 
+    // optional int32 affected_rows_count = 4;
+    public static final int AFFECTED_ROWS_COUNT_FIELD_NUMBER = 4;
+    private int affectedRowsCount_;
+    /**
+     * <code>optional int32 affected_rows_count = 4;</code>
+     */
+    public boolean hasAffectedRowsCount() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional int32 affected_rows_count = 4;</code>
+     */
+    public int getAffectedRowsCount() {
+      return affectedRowsCount_;
+    }
+
     private void initFields() {
       queryId_ = org.apache.drill.exec.proto.UserBitShared.QueryId.getDefaultInstance();
       rowCount_ = 0;
       def_ = org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance();
+      affectedRowsCount_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -11800,6 +11913,9 @@ public final class UserBitShared {
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeMessage(3, def_);
       }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeInt32(4, affectedRowsCount_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -11821,6 +11937,10 @@ public final class UserBitShared {
         size += com.google.protobuf.CodedOutputStream
           .computeMessageSize(3, def_);
       }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(4, affectedRowsCount_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -11958,6 +12078,8 @@ public final class UserBitShared {
           defBuilder_.clear();
         }
         bitField0_ = (bitField0_ & ~0x00000004);
+        affectedRowsCount_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000008);
         return this;
       }
 
@@ -12006,6 +12128,10 @@ public final class UserBitShared {
         } else {
           result.def_ = defBuilder_.build();
         }
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.affectedRowsCount_ = affectedRowsCount_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -12031,6 +12157,9 @@ public final class UserBitShared {
         if (other.hasDef()) {
           mergeDef(other.getDef());
         }
+        if (other.hasAffectedRowsCount()) {
+          setAffectedRowsCount(other.getAffectedRowsCount());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -12325,6 +12454,39 @@ public final class UserBitShared {
         return defBuilder_;
       }
 
+      // optional int32 affected_rows_count = 4;
+      private int affectedRowsCount_ ;
+      /**
+       * <code>optional int32 affected_rows_count = 4;</code>
+       */
+      public boolean hasAffectedRowsCount() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional int32 affected_rows_count = 4;</code>
+       */
+      public int getAffectedRowsCount() {
+        return affectedRowsCount_;
+      }
+      /**
+       * <code>optional int32 affected_rows_count = 4;</code>
+       */
+      public Builder setAffectedRowsCount(int value) {
+        bitField0_ |= 0x00000008;
+        affectedRowsCount_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 affected_rows_count = 4;</code>
+       */
+      public Builder clearAffectedRowsCount() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        affectedRowsCount_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:exec.shared.QueryData)
     }
 
@@ -24354,117 +24516,119 @@ public final class UserBitShared {
       "\013method_name\030\004 \001(\t\022\030\n\020is_native_method\030\005" +
       " \001(\010\"\\\n\014ParsingError\022\024\n\014start_column\030\002 \001" +
       "(\005\022\021\n\tstart_row\030\003 \001(\005\022\022\n\nend_column\030\004 \001(" +
-      "\005\022\017\n\007end_row\030\005 \001(\005\"~\n\016RecordBatchDef\022\024\n\014" +
-      "record_count\030\001 \001(\005\022+\n\005field\030\002 \003(\0132\034.exec" +
-      ".shared.SerializedField\022)\n!carries_two_b" +
-      "yte_selection_vector\030\003 \001(\010\"\205\001\n\010NamePart\022",
-      "(\n\004type\030\001 \001(\0162\032.exec.shared.NamePart.Typ" +
-      "e\022\014\n\004name\030\002 \001(\t\022$\n\005child\030\003 \001(\0132\025.exec.sh" +
-      "ared.NamePart\"\033\n\004Type\022\010\n\004NAME\020\000\022\t\n\005ARRAY" +
-      "\020\001\"\324\001\n\017SerializedField\022%\n\nmajor_type\030\001 \001" +
-      "(\0132\021.common.MajorType\022(\n\tname_part\030\002 \001(\013" +
-      "2\025.exec.shared.NamePart\022+\n\005child\030\003 \003(\0132\034" +
-      ".exec.shared.SerializedField\022\023\n\013value_co" +
-      "unt\030\004 \001(\005\022\027\n\017var_byte_length\030\005 \001(\005\022\025\n\rbu" +
-      "ffer_length\030\007 \001(\005\"7\n\nNodeStatus\022\017\n\007node_" +
-      "id\030\001 \001(\005\022\030\n\020memory_footprint\030\002 \001(\003\"\263\002\n\013Q",
-      "ueryResult\0228\n\013query_state\030\001 \001(\0162#.exec.s" +
-      "hared.QueryResult.QueryState\022&\n\010query_id" +
-      "\030\002 \001(\0132\024.exec.shared.QueryId\022(\n\005error\030\003 " +
-      "\003(\0132\031.exec.shared.DrillPBError\"\227\001\n\nQuery" +
-      "State\022\014\n\010STARTING\020\000\022\013\n\007RUNNING\020\001\022\r\n\tCOMP" +
-      "LETED\020\002\022\014\n\010CANCELED\020\003\022\n\n\006FAILED\020\004\022\032\n\026CAN" +
-      "CELLATION_REQUESTED\020\005\022\014\n\010ENQUEUED\020\006\022\r\n\tP" +
-      "REPARING\020\007\022\014\n\010PLANNING\020\010\"p\n\tQueryData\022&\n" +
-      "\010query_id\030\001 \001(\0132\024.exec.shared.QueryId\022\021\n" +
-      "\trow_count\030\002 \001(\005\022(\n\003def\030\003 \001(\0132\033.exec.sha",
-      "red.RecordBatchDef\"\330\001\n\tQueryInfo\022\r\n\005quer" +
-      "y\030\001 \001(\t\022\r\n\005start\030\002 \001(\003\0222\n\005state\030\003 \001(\0162#." +
-      "exec.shared.QueryResult.QueryState\022\017\n\004us" +
-      "er\030\004 \001(\t:\001-\022\'\n\007foreman\030\005 \001(\0132\026.exec.Dril" +
-      "lbitEndpoint\022\024\n\014options_json\030\006 \001(\t\022\022\n\nto" +
-      "tal_cost\030\007 \001(\001\022\025\n\nqueue_name\030\010 \001(\t:\001-\"\263\004" +
-      "\n\014QueryProfile\022 \n\002id\030\001 \001(\0132\024.exec.shared" +
-      ".QueryId\022$\n\004type\030\002 \001(\0162\026.exec.shared.Que" +
-      "ryType\022\r\n\005start\030\003 \001(\003\022\013\n\003end\030\004 \001(\003\022\r\n\005qu" +
-      "ery\030\005 \001(\t\022\014\n\004plan\030\006 \001(\t\022\'\n\007foreman\030\007 \001(\013",
-      "2\026.exec.DrillbitEndpoint\0222\n\005state\030\010 \001(\0162" +
-      "#.exec.shared.QueryResult.QueryState\022\027\n\017" +
-      "total_fragments\030\t \001(\005\022\032\n\022finished_fragme" +
-      "nts\030\n \001(\005\022;\n\020fragment_profile\030\013 \003(\0132!.ex" +
-      "ec.shared.MajorFragmentProfile\022\017\n\004user\030\014" +
-      " \001(\t:\001-\022\r\n\005error\030\r \001(\t\022\024\n\014verboseError\030\016" +
-      " \001(\t\022\020\n\010error_id\030\017 \001(\t\022\022\n\nerror_node\030\020 \001" +
-      "(\t\022\024\n\014options_json\030\021 \001(\t\022\017\n\007planEnd\030\022 \001(" +
-      "\003\022\024\n\014queueWaitEnd\030\023 \001(\003\022\022\n\ntotal_cost\030\024 " +
-      "\001(\001\022\025\n\nqueue_name\030\025 \001(\t:\001-\022\017\n\007queryId\030\026 ",
-      "\001(\t\"t\n\024MajorFragmentProfile\022\031\n\021major_fra" +
-      "gment_id\030\001 \001(\005\022A\n\026minor_fragment_profile" +
-      "\030\002 \003(\0132!.exec.shared.MinorFragmentProfil" +
-      "e\"\350\002\n\024MinorFragmentProfile\022)\n\005state\030\001 \001(" +
-      "\0162\032.exec.shared.FragmentState\022(\n\005error\030\002" +
-      " \001(\0132\031.exec.shared.DrillPBError\022\031\n\021minor" +
-      "_fragment_id\030\003 \001(\005\0226\n\020operator_profile\030\004" +
-      " \003(\0132\034.exec.shared.OperatorProfile\022\022\n\nst" +
-      "art_time\030\005 \001(\003\022\020\n\010end_time\030\006 \001(\003\022\023\n\013memo" +
-      "ry_used\030\007 \001(\003\022\027\n\017max_memory_used\030\010 \001(\003\022(",
-      "\n\010endpoint\030\t \001(\0132\026.exec.DrillbitEndpoint" +
-      "\022\023\n\013last_update\030\n \001(\003\022\025\n\rlast_progress\030\013" +
-      " \001(\003\"\377\001\n\017OperatorProfile\0221\n\rinput_profil" +
-      "e\030\001 \003(\0132\032.exec.shared.StreamProfile\022\023\n\013o" +
-      "perator_id\030\003 \001(\005\022\025\n\roperator_type\030\004 \001(\005\022" +
-      "\023\n\013setup_nanos\030\005 \001(\003\022\025\n\rprocess_nanos\030\006 " +
-      "\001(\003\022#\n\033peak_local_memory_allocated\030\007 \001(\003" +
-      "\022(\n\006metric\030\010 \003(\0132\030.exec.shared.MetricVal" +
-      "ue\022\022\n\nwait_nanos\030\t \001(\003\"B\n\rStreamProfile\022" +
-      "\017\n\007records\030\001 \001(\003\022\017\n\007batches\030\002 \001(\003\022\017\n\007sch",
-      "emas\030\003 \001(\003\"J\n\013MetricValue\022\021\n\tmetric_id\030\001" +
-      " \001(\005\022\022\n\nlong_value\030\002 \001(\003\022\024\n\014double_value" +
-      "\030\003 \001(\001\")\n\010Registry\022\035\n\003jar\030\001 \003(\0132\020.exec.s" +
-      "hared.Jar\"/\n\003Jar\022\014\n\004name\030\001 \001(\t\022\032\n\022functi" +
-      "on_signature\030\002 \003(\t\"W\n\013SaslMessage\022\021\n\tmec" +
-      "hanism\030\001 \001(\t\022\014\n\004data\030\002 \001(\014\022\'\n\006status\030\003 \001" +
-      "(\0162\027.exec.shared.SaslStatus*5\n\nRpcChanne" +
-      "l\022\017\n\013BIT_CONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER" +
-      "\020\002*V\n\tQueryType\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n" +
-      "\010PHYSICAL\020\003\022\r\n\tEXECUTION\020\004\022\026\n\022PREPARED_S",
-      "TATEMENT\020\005*\207\001\n\rFragmentState\022\013\n\007SENDING\020" +
-      "\000\022\027\n\023AWAITING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022" +
-      "\014\n\010FINISHED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005" +
-      "\022\032\n\026CANCELLATION_REQUESTED\020\006*\222\t\n\020CoreOpe" +
-      "ratorType\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAS" +
-      "T_SENDER\020\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGATE" +
-      "\020\003\022\r\n\tHASH_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HAS" +
-      "H_PARTITION_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGI" +
-      "NG_RECEIVER\020\010\022\034\n\030ORDERED_PARTITION_SENDE" +
-      "R\020\t\022\013\n\007PROJECT\020\n\022\026\n\022UNORDERED_RECEIVER\020\013",
-      "\022\032\n\026RANGE_PARTITION_SENDER\020\014\022\n\n\006SCREEN\020\r" +
-      "\022\034\n\030SELECTION_VECTOR_REMOVER\020\016\022\027\n\023STREAM" +
-      "ING_AGGREGATE\020\017\022\016\n\nTOP_N_SORT\020\020\022\021\n\rEXTER" +
-      "NAL_SORT\020\021\022\t\n\005TRACE\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_" +
-      "SORT\020\024\022\032\n\026PARQUET_ROW_GROUP_SCAN\020\025\022\021\n\rHI" +
-      "VE_SUB_SCAN\020\026\022\025\n\021SYSTEM_TABLE_SCAN\020\027\022\021\n\r" +
-      "MOCK_SUB_SCAN\020\030\022\022\n\016PARQUET_WRITER\020\031\022\023\n\017D" +
-      "IRECT_SUB_SCAN\020\032\022\017\n\013TEXT_WRITER\020\033\022\021\n\rTEX" +
-      "T_SUB_SCAN\020\034\022\021\n\rJSON_SUB_SCAN\020\035\022\030\n\024INFO_" +
-      "SCHEMA_SUB_SCAN\020\036\022\023\n\017COMPLEX_TO_JSON\020\037\022\025",
-      "\n\021PRODUCER_CONSUMER\020 \022\022\n\016HBASE_SUB_SCAN\020" +
-      "!\022\n\n\006WINDOW\020\"\022\024\n\020NESTED_LOOP_JOIN\020#\022\021\n\rA" +
-      "VRO_SUB_SCAN\020$\022\021\n\rPCAP_SUB_SCAN\020%\022\022\n\016KAF" +
-      "KA_SUB_SCAN\020&\022\021\n\rKUDU_SUB_SCAN\020\'\022\013\n\007FLAT" +
-      "TEN\020(\022\020\n\014LATERAL_JOIN\020)\022\n\n\006UNNEST\020*\022,\n(H" +
-      "IVE_DRILL_NATIVE_PARQUET_ROW_GROUP_SCAN\020" +
-      "+\022\r\n\tJDBC_SCAN\020,\022\022\n\016REGEX_SUB_SCAN\020-\022\023\n\017" +
-      "MAPRDB_SUB_SCAN\020.\022\022\n\016MONGO_SUB_SCAN\020/\022\017\n" +
-      "\013KUDU_WRITER\0200\022\026\n\022OPEN_TSDB_SUB_SCAN\0201\022\017" +
-      "\n\013JSON_WRITER\0202\022\026\n\022HTPPD_LOG_SUB_SCAN\0203\022",
-      "\022\n\016IMAGE_SUB_SCAN\0204\022\025\n\021SEQUENCE_SUB_SCAN" +
-      "\0205\022\023\n\017PARTITION_LIMIT\0206\022\023\n\017PCAPNG_SUB_SC" +
-      "AN\0207\022\022\n\016RUNTIME_FILTER\0208\022\017\n\013ROWKEY_JOIN\020" +
-      "9*g\n\nSaslStatus\022\020\n\014SASL_UNKNOWN\020\000\022\016\n\nSAS" +
-      "L_START\020\001\022\024\n\020SASL_IN_PROGRESS\020\002\022\020\n\014SASL_" +
-      "SUCCESS\020\003\022\017\n\013SASL_FAILED\020\004B.\n\033org.apache" +
-      ".drill.exec.protoB\rUserBitSharedH\001"
+      "\005\022\017\n\007end_row\030\005 \001(\005\"\233\001\n\016RecordBatchDef\022\024\n" +
+      "\014record_count\030\001 \001(\005\022+\n\005field\030\002 \003(\0132\034.exe" +
+      "c.shared.SerializedField\022)\n!carries_two_" +
+      "byte_selection_vector\030\003 \001(\010\022\033\n\023affected_",
+      "rows_count\030\004 \001(\005\"\205\001\n\010NamePart\022(\n\004type\030\001 " +
+      "\001(\0162\032.exec.shared.NamePart.Type\022\014\n\004name\030" +
+      "\002 \001(\t\022$\n\005child\030\003 \001(\0132\025.exec.shared.NameP" +
+      "art\"\033\n\004Type\022\010\n\004NAME\020\000\022\t\n\005ARRAY\020\001\"\324\001\n\017Ser" +
+      "ializedField\022%\n\nmajor_type\030\001 \001(\0132\021.commo" +
+      "n.MajorType\022(\n\tname_part\030\002 \001(\0132\025.exec.sh" +
+      "ared.NamePart\022+\n\005child\030\003 \003(\0132\034.exec.shar" +
+      "ed.SerializedField\022\023\n\013value_count\030\004 \001(\005\022" +
+      "\027\n\017var_byte_length\030\005 \001(\005\022\025\n\rbuffer_lengt" +
+      "h\030\007 \001(\005\"7\n\nNodeStatus\022\017\n\007node_id\030\001 \001(\005\022\030",
+      "\n\020memory_footprint\030\002 \001(\003\"\263\002\n\013QueryResult" +
+      "\0228\n\013query_state\030\001 \001(\0162#.exec.shared.Quer" +
+      "yResult.QueryState\022&\n\010query_id\030\002 \001(\0132\024.e" +
+      "xec.shared.QueryId\022(\n\005error\030\003 \003(\0132\031.exec" +
+      ".shared.DrillPBError\"\227\001\n\nQueryState\022\014\n\010S" +
+      "TARTING\020\000\022\013\n\007RUNNING\020\001\022\r\n\tCOMPLETED\020\002\022\014\n" +
+      "\010CANCELED\020\003\022\n\n\006FAILED\020\004\022\032\n\026CANCELLATION_" +
+      "REQUESTED\020\005\022\014\n\010ENQUEUED\020\006\022\r\n\tPREPARING\020\007" +
+      "\022\014\n\010PLANNING\020\010\"\215\001\n\tQueryData\022&\n\010query_id" +
+      "\030\001 \001(\0132\024.exec.shared.QueryId\022\021\n\trow_coun",
+      "t\030\002 \001(\005\022(\n\003def\030\003 \001(\0132\033.exec.shared.Recor" +
+      "dBatchDef\022\033\n\023affected_rows_count\030\004 \001(\005\"\330" +
+      "\001\n\tQueryInfo\022\r\n\005query\030\001 \001(\t\022\r\n\005start\030\002 \001" +
+      "(\003\0222\n\005state\030\003 \001(\0162#.exec.shared.QueryRes" +
+      "ult.QueryState\022\017\n\004user\030\004 \001(\t:\001-\022\'\n\007forem" +
+      "an\030\005 \001(\0132\026.exec.DrillbitEndpoint\022\024\n\014opti" +
+      "ons_json\030\006 \001(\t\022\022\n\ntotal_cost\030\007 \001(\001\022\025\n\nqu" +
+      "eue_name\030\010 \001(\t:\001-\"\263\004\n\014QueryProfile\022 \n\002id" +
+      "\030\001 \001(\0132\024.exec.shared.QueryId\022$\n\004type\030\002 \001" +
+      "(\0162\026.exec.shared.QueryType\022\r\n\005start\030\003 \001(",
+      "\003\022\013\n\003end\030\004 \001(\003\022\r\n\005query\030\005 \001(\t\022\014\n\004plan\030\006 " +
+      "\001(\t\022\'\n\007foreman\030\007 \001(\0132\026.exec.DrillbitEndp" +
+      "oint\0222\n\005state\030\010 \001(\0162#.exec.shared.QueryR" +
+      "esult.QueryState\022\027\n\017total_fragments\030\t \001(" +
+      "\005\022\032\n\022finished_fragments\030\n \001(\005\022;\n\020fragmen" +
+      "t_profile\030\013 \003(\0132!.exec.shared.MajorFragm" +
+      "entProfile\022\017\n\004user\030\014 \001(\t:\001-\022\r\n\005error\030\r \001" +
+      "(\t\022\024\n\014verboseError\030\016 \001(\t\022\020\n\010error_id\030\017 \001" +
+      "(\t\022\022\n\nerror_node\030\020 \001(\t\022\024\n\014options_json\030\021" +
+      " \001(\t\022\017\n\007planEnd\030\022 \001(\003\022\024\n\014queueWaitEnd\030\023 ",
+      "\001(\003\022\022\n\ntotal_cost\030\024 \001(\001\022\025\n\nqueue_name\030\025 " +
+      "\001(\t:\001-\022\017\n\007queryId\030\026 \001(\t\"t\n\024MajorFragment" +
+      "Profile\022\031\n\021major_fragment_id\030\001 \001(\005\022A\n\026mi" +
+      "nor_fragment_profile\030\002 \003(\0132!.exec.shared" +
+      ".MinorFragmentProfile\"\350\002\n\024MinorFragmentP" +
+      "rofile\022)\n\005state\030\001 \001(\0162\032.exec.shared.Frag" +
+      "mentState\022(\n\005error\030\002 \001(\0132\031.exec.shared.D" +
+      "rillPBError\022\031\n\021minor_fragment_id\030\003 \001(\005\0226" +
+      "\n\020operator_profile\030\004 \003(\0132\034.exec.shared.O" +
+      "peratorProfile\022\022\n\nstart_time\030\005 \001(\003\022\020\n\010en",
+      "d_time\030\006 \001(\003\022\023\n\013memory_used\030\007 \001(\003\022\027\n\017max" +
+      "_memory_used\030\010 \001(\003\022(\n\010endpoint\030\t \001(\0132\026.e" +
+      "xec.DrillbitEndpoint\022\023\n\013last_update\030\n \001(" +
+      "\003\022\025\n\rlast_progress\030\013 \001(\003\"\377\001\n\017OperatorPro" +
+      "file\0221\n\rinput_profile\030\001 \003(\0132\032.exec.share" +
+      "d.StreamProfile\022\023\n\013operator_id\030\003 \001(\005\022\025\n\r" +
+      "operator_type\030\004 \001(\005\022\023\n\013setup_nanos\030\005 \001(\003" +
+      "\022\025\n\rprocess_nanos\030\006 \001(\003\022#\n\033peak_local_me" +
+      "mory_allocated\030\007 \001(\003\022(\n\006metric\030\010 \003(\0132\030.e" +
+      "xec.shared.MetricValue\022\022\n\nwait_nanos\030\t \001",
+      "(\003\"B\n\rStreamProfile\022\017\n\007records\030\001 \001(\003\022\017\n\007" +
+      "batches\030\002 \001(\003\022\017\n\007schemas\030\003 \001(\003\"J\n\013Metric" +
+      "Value\022\021\n\tmetric_id\030\001 \001(\005\022\022\n\nlong_value\030\002" +
+      " \001(\003\022\024\n\014double_value\030\003 \001(\001\")\n\010Registry\022\035" +
+      "\n\003jar\030\001 \003(\0132\020.exec.shared.Jar\"/\n\003Jar\022\014\n\004" +
+      "name\030\001 \001(\t\022\032\n\022function_signature\030\002 \003(\t\"W" +
+      "\n\013SaslMessage\022\021\n\tmechanism\030\001 \001(\t\022\014\n\004data" +
+      "\030\002 \001(\014\022\'\n\006status\030\003 \001(\0162\027.exec.shared.Sas" +
+      "lStatus*5\n\nRpcChannel\022\017\n\013BIT_CONTROL\020\000\022\014" +
+      "\n\010BIT_DATA\020\001\022\010\n\004USER\020\002*V\n\tQueryType\022\007\n\003S",
+      "QL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL\020\003\022\r\n\tEXECU" +
+      "TION\020\004\022\026\n\022PREPARED_STATEMENT\020\005*\207\001\n\rFragm" +
+      "entState\022\013\n\007SENDING\020\000\022\027\n\023AWAITING_ALLOCA" +
+      "TION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHED\020\003\022\r\n\tCAN" +
+      "CELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCELLATION_REQ" +
+      "UESTED\020\006*\222\t\n\020CoreOperatorType\022\021\n\rSINGLE_" +
+      "SENDER\020\000\022\024\n\020BROADCAST_SENDER\020\001\022\n\n\006FILTER" +
+      "\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHASH_JOIN\020\004\022\016\n" +
+      "\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITION_SENDER\020\006" +
+      "\022\t\n\005LIMIT\020\007\022\024\n\020MERGING_RECEIVER\020\010\022\034\n\030ORD",
+      "ERED_PARTITION_SENDER\020\t\022\013\n\007PROJECT\020\n\022\026\n\022" +
+      "UNORDERED_RECEIVER\020\013\022\032\n\026RANGE_PARTITION_" +
+      "SENDER\020\014\022\n\n\006SCREEN\020\r\022\034\n\030SELECTION_VECTOR" +
+      "_REMOVER\020\016\022\027\n\023STREAMING_AGGREGATE\020\017\022\016\n\nT" +
+      "OP_N_SORT\020\020\022\021\n\rEXTERNAL_SORT\020\021\022\t\n\005TRACE\020" +
+      "\022\022\t\n\005UNION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n\026PARQUET_RO" +
+      "W_GROUP_SCAN\020\025\022\021\n\rHIVE_SUB_SCAN\020\026\022\025\n\021SYS" +
+      "TEM_TABLE_SCAN\020\027\022\021\n\rMOCK_SUB_SCAN\020\030\022\022\n\016P" +
+      "ARQUET_WRITER\020\031\022\023\n\017DIRECT_SUB_SCAN\020\032\022\017\n\013" +
+      "TEXT_WRITER\020\033\022\021\n\rTEXT_SUB_SCAN\020\034\022\021\n\rJSON",
+      "_SUB_SCAN\020\035\022\030\n\024INFO_SCHEMA_SUB_SCAN\020\036\022\023\n" +
+      "\017COMPLEX_TO_JSON\020\037\022\025\n\021PRODUCER_CONSUMER\020" +
+      " \022\022\n\016HBASE_SUB_SCAN\020!\022\n\n\006WINDOW\020\"\022\024\n\020NES" +
+      "TED_LOOP_JOIN\020#\022\021\n\rAVRO_SUB_SCAN\020$\022\021\n\rPC" +
+      "AP_SUB_SCAN\020%\022\022\n\016KAFKA_SUB_SCAN\020&\022\021\n\rKUD" +
+      "U_SUB_SCAN\020\'\022\013\n\007FLATTEN\020(\022\020\n\014LATERAL_JOI" +
+      "N\020)\022\n\n\006UNNEST\020*\022,\n(HIVE_DRILL_NATIVE_PAR" +
+      "QUET_ROW_GROUP_SCAN\020+\022\r\n\tJDBC_SCAN\020,\022\022\n\016" +
+      "REGEX_SUB_SCAN\020-\022\023\n\017MAPRDB_SUB_SCAN\020.\022\022\n" +
+      "\016MONGO_SUB_SCAN\020/\022\017\n\013KUDU_WRITER\0200\022\026\n\022OP",
+      "EN_TSDB_SUB_SCAN\0201\022\017\n\013JSON_WRITER\0202\022\026\n\022H" +
+      "TPPD_LOG_SUB_SCAN\0203\022\022\n\016IMAGE_SUB_SCAN\0204\022" +
+      "\025\n\021SEQUENCE_SUB_SCAN\0205\022\023\n\017PARTITION_LIMI" +
+      "T\0206\022\023\n\017PCAPNG_SUB_SCAN\0207\022\022\n\016RUNTIME_FILT" +
+      "ER\0208\022\017\n\013ROWKEY_JOIN\0209*g\n\nSaslStatus\022\020\n\014S" +
+      "ASL_UNKNOWN\020\000\022\016\n\nSASL_START\020\001\022\024\n\020SASL_IN" +
+      "_PROGRESS\020\002\022\020\n\014SASL_SUCCESS\020\003\022\017\n\013SASL_FA" +
+      "ILED\020\004B.\n\033org.apache.drill.exec.protoB\rU" +
+      "serBitSharedH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -24512,7 +24676,7 @@ public final class UserBitShared {
           internal_static_exec_shared_RecordBatchDef_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_shared_RecordBatchDef_descriptor,
-              new java.lang.String[] { "RecordCount", "Field", "CarriesTwoByteSelectionVector", });
+              new java.lang.String[] { "RecordCount", "Field", "CarriesTwoByteSelectionVector", "AffectedRowsCount", });
           internal_static_exec_shared_NamePart_descriptor =
             getDescriptor().getMessageTypes().get(7);
           internal_static_exec_shared_NamePart_fieldAccessorTable = new
@@ -24542,7 +24706,7 @@ public final class UserBitShared {
           internal_static_exec_shared_QueryData_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_shared_QueryData_descriptor,
-              new java.lang.String[] { "QueryId", "RowCount", "Def", });
+              new java.lang.String[] { "QueryId", "RowCount", "Def", "AffectedRowsCount", });
           internal_static_exec_shared_QueryInfo_descriptor =
             getDescriptor().getMessageTypes().get(12);
           internal_static_exec_shared_QueryInfo_fieldAccessorTable = new
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryData.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryData.java
index 36f2747..7744848 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryData.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/QueryData.java
@@ -50,6 +50,7 @@ public final class QueryData implements Externalizable, Message<QueryData>, Sche
     private QueryId queryId;
     private int rowCount;
     private RecordBatchDef def;
+    private int affectedRowsCount;
 
     public QueryData()
     {
@@ -97,6 +98,19 @@ public final class QueryData implements Externalizable, Message<QueryData>, Sche
         return this;
     }
 
+    // affectedRowsCount
+
+    public int getAffectedRowsCount()
+    {
+        return affectedRowsCount;
+    }
+
+    public QueryData setAffectedRowsCount(int affectedRowsCount)
+    {
+        this.affectedRowsCount = affectedRowsCount;
+        return this;
+    }
+
     // java serialization
 
     public void readExternal(ObjectInput in) throws IOException
@@ -162,6 +176,9 @@ public final class QueryData implements Externalizable, Message<QueryData>, Sche
                     message.def = input.mergeObject(message.def, RecordBatchDef.getSchema());
                     break;
 
+                case 4:
+                    message.affectedRowsCount = input.readInt32();
+                    break;
                 default:
                     input.handleUnknownField(number, this);
             }   
@@ -181,6 +198,9 @@ public final class QueryData implements Externalizable, Message<QueryData>, Sche
         if(message.def != null)
              output.writeObject(3, message.def, RecordBatchDef.getSchema(), false);
 
+
+        if(message.affectedRowsCount != 0)
+            output.writeInt32(4, message.affectedRowsCount, false);
     }
 
     public String getFieldName(int number)
@@ -190,6 +210,7 @@ public final class QueryData implements Externalizable, Message<QueryData>, Sche
             case 1: return "queryId";
             case 2: return "rowCount";
             case 3: return "def";
+            case 4: return "affectedRowsCount";
             default: return null;
         }
     }
@@ -206,6 +227,7 @@ public final class QueryData implements Externalizable, Message<QueryData>, Sche
         __fieldMap.put("queryId", 1);
         __fieldMap.put("rowCount", 2);
         __fieldMap.put("def", 3);
+        __fieldMap.put("affectedRowsCount", 4);
     }
     
 }
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/RecordBatchDef.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/RecordBatchDef.java
index 53bfc91..8376013 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/RecordBatchDef.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/RecordBatchDef.java
@@ -52,6 +52,7 @@ public final class RecordBatchDef implements Externalizable, Message<RecordBatch
     private int recordCount;
     private List<SerializedField> field;
     private Boolean carriesTwoByteSelectionVector;
+    private int affectedRowsCount;
 
     public RecordBatchDef()
     {
@@ -99,6 +100,19 @@ public final class RecordBatchDef implements Externalizable, Message<RecordBatch
         return this;
     }
 
+    // affectedRowsCount
+
+    public int getAffectedRowsCount()
+    {
+        return affectedRowsCount;
+    }
+
+    public RecordBatchDef setAffectedRowsCount(int affectedRowsCount)
+    {
+        this.affectedRowsCount = affectedRowsCount;
+        return this;
+    }
+
     // java serialization
 
     public void readExternal(ObjectInput in) throws IOException
@@ -165,6 +179,9 @@ public final class RecordBatchDef implements Externalizable, Message<RecordBatch
                 case 3:
                     message.carriesTwoByteSelectionVector = input.readBool();
                     break;
+                case 4:
+                    message.affectedRowsCount = input.readInt32();
+                    break;
                 default:
                     input.handleUnknownField(number, this);
             }   
@@ -189,6 +206,9 @@ public final class RecordBatchDef implements Externalizable, Message<RecordBatch
 
         if(message.carriesTwoByteSelectionVector != null)
             output.writeBool(3, message.carriesTwoByteSelectionVector, false);
+
+        if(message.affectedRowsCount != 0)
+            output.writeInt32(4, message.affectedRowsCount, false);
     }
 
     public String getFieldName(int number)
@@ -198,6 +218,7 @@ public final class RecordBatchDef implements Externalizable, Message<RecordBatch
             case 1: return "recordCount";
             case 2: return "field";
             case 3: return "carriesTwoByteSelectionVector";
+            case 4: return "affectedRowsCount";
             default: return null;
         }
     }
@@ -214,6 +235,7 @@ public final class RecordBatchDef implements Externalizable, Message<RecordBatch
         __fieldMap.put("recordCount", 1);
         __fieldMap.put("field", 2);
         __fieldMap.put("carriesTwoByteSelectionVector", 3);
+        __fieldMap.put("affectedRowsCount", 4);
     }
     
 }
diff --git a/protocol/src/main/protobuf/UserBitShared.proto b/protocol/src/main/protobuf/UserBitShared.proto
index 843c6d8..4e2644f 100644
--- a/protocol/src/main/protobuf/UserBitShared.proto
+++ b/protocol/src/main/protobuf/UserBitShared.proto
@@ -140,6 +140,9 @@ message RecordBatchDef {
   optional int32 record_count = 1;
   repeated SerializedField field = 2;
   optional bool carries_two_byte_selection_vector = 3;
+  // The value is set when result set is disabled and its value corresponds to number
+  // of rows affected by query (see JDBC java.sql.ResultSet#getUpdateCount())
+  optional int32 affected_rows_count = 4;
 }
 
 message NamePart{
@@ -197,6 +200,7 @@ message QueryData {
   optional QueryId query_id = 1;
   optional int32 row_count = 2;
   optional RecordBatchDef def = 3;
+  optional int32 affected_rows_count = 4;
 }
 
 message QueryInfo {


[drill] 14/15: DRILL-6865: Filter is not removed from the plan when parquet table fully matches the filter

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

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

commit 99a3d76551d1a08958c7cd7670df189963fbc943
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Fri Nov 23 11:34:13 2018 +0200

    DRILL-6865: Filter is not removed from the plan when parquet table fully matches the filter
    
    closes #1552
---
 .../store/parquet/AbstractParquetGroupScan.java    | 44 ++++++++++++-----
 .../exec/store/parquet/ParquetPushDownFilter.java  | 57 +++++++++++++---------
 .../drill/exec/store/parquet/RowGroupInfo.java     |  6 ---
 .../store/parquet/TestParquetFilterPushDown.java   |  9 ++++
 4 files changed, 74 insertions(+), 42 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
index 1bbf63b..a366339 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetGroupScan.java
@@ -85,6 +85,8 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
 
   private List<EndpointAffinity> endpointAffinities;
   private ParquetGroupScanStatistics parquetGroupScanStatistics;
+  // whether all row groups of this group scan fully match the filter
+  private boolean matchAllRowGroups = false;
 
   protected AbstractParquetGroupScan(String userName,
                                      List<SchemaPath> columns,
@@ -111,6 +113,7 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
     this.fileSet = that.fileSet == null ? null : new HashSet<>(that.fileSet);
     this.entries = that.entries == null ? null : new ArrayList<>(that.entries);
     this.readerConfig = that.readerConfig;
+    this.matchAllRowGroups = that.matchAllRowGroups;
   }
 
   @JsonProperty
@@ -136,6 +139,11 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
   }
 
   @JsonIgnore
+  public boolean isMatchAllRowGroups() {
+    return matchAllRowGroups;
+  }
+
+  @JsonIgnore
   @Override
   public Collection<String> getFiles() {
     return fileSet;
@@ -229,15 +237,12 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
   }
 
   @Override
-  public GroupScan applyFilter(LogicalExpression filterExpr, UdfUtilities udfUtilities,
-                               FunctionImplementationRegistry functionImplementationRegistry, OptionManager optionManager) {
-
-    if (rowGroupInfos.size() == 1 ||
-        ! (parquetTableMetadata.isRowGroupPrunable()) ||
-        rowGroupInfos.size() > optionManager.getOption(PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD)
-        ) {
-      // Stop pruning for 3 cases:
-      //    -  1 single parquet file,
+  public AbstractParquetGroupScan applyFilter(LogicalExpression filterExpr, UdfUtilities udfUtilities,
+      FunctionImplementationRegistry functionImplementationRegistry, OptionManager optionManager) {
+
+    if (!parquetTableMetadata.isRowGroupPrunable() ||
+        rowGroupInfos.size() > optionManager.getOption(PlannerSettings.PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD)) {
+      // Stop pruning for 2 cases:
       //    -  metadata does not have proper format to support row group level filter pruning,
       //    -  # of row groups is beyond PARQUET_ROWGROUP_FILTER_PUSHDOWN_PLANNING_THRESHOLD.
       return null;
@@ -253,6 +258,8 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
       return null;
     }
 
+    boolean matchAllRowGroupsLocal = true;
+
     for (RowGroupInfo rowGroup : rowGroupInfos) {
       final ColumnExplorer columnExplorer = new ColumnExplorer(optionManager, columns);
       List<String> partitionValues = getPartitionValues(rowGroup);
@@ -270,16 +277,27 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
       if (match == ParquetFilterPredicate.RowsMatch.NONE) {
         continue; // No row comply to the filter => drop the row group
       }
-      rowGroup.setRowsMatch(match);
+      // for the case when any of row groups partially matches the filter,
+      // matchAllRowGroupsLocal should be set to false
+      if (matchAllRowGroupsLocal) {
+        matchAllRowGroupsLocal = match == ParquetFilterPredicate.RowsMatch.ALL;
+      }
 
       qualifiedRGs.add(rowGroup);
     }
 
-    if (qualifiedRGs.size() == rowGroupInfos.size() ) {
+    if (qualifiedRGs.size() == rowGroupInfos.size()) {
       // There is no reduction of rowGroups. Return the original groupScan.
       logger.debug("applyFilter() does not have any pruning!");
+      matchAllRowGroups = matchAllRowGroupsLocal;
       return null;
     } else if (qualifiedRGs.size() == 0) {
+      if (rowGroupInfos.size() == 1) {
+        // For the case when group scan has single row group and it was filtered,
+        // no need to create new group scan with the same row group.
+        return null;
+      }
+      matchAllRowGroupsLocal = false;
       logger.debug("All row groups have been filtered out. Add back one to get schema from scanner.");
       RowGroupInfo rg = rowGroupInfos.iterator().next();
       qualifiedRGs.add(rg);
@@ -289,7 +307,9 @@ public abstract class AbstractParquetGroupScan extends AbstractFileGroupScan {
       ExpressionStringBuilder.toString(filterExpr), rowGroupInfos.size(), qualifiedRGs.size());
 
     try {
-      return cloneWithRowGroupInfos(qualifiedRGs);
+      AbstractParquetGroupScan cloneGroupScan = cloneWithRowGroupInfos(qualifiedRGs);
+      cloneGroupScan.matchAllRowGroups = matchAllRowGroupsLocal;
+      return cloneGroupScan;
     } catch (IOException e) {
       logger.warn("Could not apply filter prune due to Exception : {}", e);
       return null;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
index 95a0534..c12ea73 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
@@ -28,9 +28,7 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.ValueExpressions;
 import org.apache.drill.exec.expr.stat.ParquetFilterPredicate;
-import org.apache.drill.exec.expr.stat.ParquetFilterPredicate.RowsMatch;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
-import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.planner.common.DrillRelOptUtil;
 import org.apache.drill.exec.planner.logical.DrillOptiq;
 import org.apache.drill.exec.planner.logical.DrillParseContext;
@@ -174,14 +172,35 @@ public abstract class ParquetPushDownFilter extends StoragePluginOptimizerRule {
 
 
     Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null;
-    final GroupScan newGroupScan = groupScan.applyFilter(conditionExp, optimizerContext,
+    AbstractParquetGroupScan newGroupScan = groupScan.applyFilter(conditionExp, optimizerContext,
         optimizerContext.getFunctionRegistry(), optimizerContext.getPlannerSettings().getOptions());
     if (timer != null) {
       logger.debug("Took {} ms to apply filter on parquet row groups. ", timer.elapsed(TimeUnit.MILLISECONDS));
       timer.stop();
     }
 
-    if (newGroupScan == null ) {
+    // For the case when newGroupScan wasn't created, the old one may
+    // fully match the filter for the case when row group pruning did not happen.
+    if (newGroupScan == null) {
+      if (groupScan.isMatchAllRowGroups()) {
+        RelNode child = project == null ? scan : project;
+        // If current row group fully matches filter,
+        // but row group pruning did not happen, remove the filter.
+        if (nonConvertedPredList.size() == 0) {
+          call.transformTo(child);
+        } else if (nonConvertedPredList.size() == predList.size()) {
+          // None of the predicates participated in filter pushdown.
+          return;
+        } else {
+          // If some of the predicates weren't used in the filter, creates new filter with them
+          // on top of current scan. Excludes the case when all predicates weren't used in the filter.
+          call.transformTo(filter.copy(filter.getTraitSet(), child,
+              RexUtil.composeConjunction(
+                  filter.getCluster().getRexBuilder(),
+                  nonConvertedPredList,
+                  true)));
+        }
+      }
       return;
     }
 
@@ -191,27 +210,17 @@ public abstract class ParquetPushDownFilter extends StoragePluginOptimizerRule {
       newNode = project.copy(project.getTraitSet(), Collections.singletonList(newNode));
     }
 
-    if (newGroupScan instanceof AbstractParquetGroupScan) {
-      RowsMatch matchAll = RowsMatch.ALL;
-      List<RowGroupInfo> rowGroupInfos = ((AbstractParquetGroupScan) newGroupScan).rowGroupInfos;
-      for (RowGroupInfo rowGroup : rowGroupInfos) {
-        if (rowGroup.getRowsMatch() != RowsMatch.ALL) {
-          matchAll = RowsMatch.SOME;
-          break;
-        }
-      }
-      if (matchAll == ParquetFilterPredicate.RowsMatch.ALL) {
-        // creates filter from the expressions which can't be pushed to the scan
-        if (nonConvertedPredList.size() > 0) {
-          newNode = filter.copy(filter.getTraitSet(), newNode,
-              RexUtil.composeConjunction(
-                  filter.getCluster().getRexBuilder(),
-                  nonConvertedPredList,
-                  true));
-        }
-        call.transformTo(newNode);
-        return;
+    if (newGroupScan.isMatchAllRowGroups()) {
+      // creates filter from the expressions which can't be pushed to the scan
+      if (nonConvertedPredList.size() > 0) {
+        newNode = filter.copy(filter.getTraitSet(), newNode,
+            RexUtil.composeConjunction(
+                filter.getCluster().getRexBuilder(),
+                nonConvertedPredList,
+                true));
       }
+      call.transformTo(newNode);
+      return;
     }
 
     final RelNode newFilter = filter.copy(filter.getTraitSet(), Collections.singletonList(newNode));
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/RowGroupInfo.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/RowGroupInfo.java
index 7d2143c..1c9ce10 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/RowGroupInfo.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/RowGroupInfo.java
@@ -19,7 +19,6 @@ package org.apache.drill.exec.store.parquet;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.drill.exec.expr.stat.ParquetFilterPredicate.RowsMatch;
 import org.apache.drill.exec.store.dfs.ReadEntryFromHDFS;
 import org.apache.drill.exec.store.dfs.easy.FileWork;
 import org.apache.drill.exec.store.schedule.CompleteWork;
@@ -36,7 +35,6 @@ public class RowGroupInfo extends ReadEntryFromHDFS implements CompleteWork, Fil
   private List<? extends ColumnMetadata> columns;
   private long rowCount;  // rowCount = -1 indicates to include all rows.
   private long numRecordsToRead;
-  private RowsMatch rowsMatch = RowsMatch.SOME;
 
   @JsonCreator
   public RowGroupInfo(@JsonProperty("path") String path,
@@ -96,8 +94,4 @@ public class RowGroupInfo extends ReadEntryFromHDFS implements CompleteWork, Fil
   public void setColumns(List<? extends ColumnMetadata> columns) {
     this.columns = columns;
   }
-
-  public RowsMatch getRowsMatch() { return rowsMatch; }
-
-  public void setRowsMatch(RowsMatch rowsMatch) { this.rowsMatch = rowsMatch; }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
index ccc1480..80b06d9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
@@ -649,6 +649,15 @@ public class TestParquetFilterPushDown extends PlanTestBase {
     assertEquals(RowsMatch.ALL, isNotFalse.matches(re));
   }
 
+  @Test
+  public void testParquetSingleRowGroupFilterRemoving() throws Exception {
+    test("create table dfs.tmp.`singleRowGroupTable` as select * from cp.`tpch/nation.parquet`");
+
+    String query = "select * from dfs.tmp.`singleRowGroupTable` where n_nationkey > -1";
+
+    testParquetFilterPruning(query, 25, 1, new String[]{"Filter\\("});
+  }
+
   //////////////////////////////////////////////////////////////////////////////////////////////////
   // Some test helper functions.
   //////////////////////////////////////////////////////////////////////////////////////////////////


[drill] 07/15: DRILL-6850: Force setting DRILL_LOGICAL Convention for DrillRelFactories and DrillFilterRel

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

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

commit b67f77a18ac4b67044fa9b4b962d0536fdc54af5
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Tue Nov 13 15:21:39 2018 +0200

    DRILL-6850: Force setting DRILL_LOGICAL Convention for DrillRelFactories and DrillFilterRel
    
    - Fix workspace case insensitivity for JDBC storage plugin
---
 .../exec/store/hive/schema/HiveSchemaFactory.java   | 21 ++++++++++-----------
 .../drill/exec/store/jdbc/JdbcStoragePlugin.java    | 21 +++++++++++----------
 .../exec/store/jdbc/TestJdbcPluginWithDerbyIT.java  |  9 ++++-----
 .../drill/exec/planner/logical/DrillFilterRel.java  |  2 +-
 .../apache/drill/exec/planner/logical/DrillRel.java |  2 +-
 .../exec/planner/logical/DrillRelFactories.java     |  9 +++++----
 .../planner/sql/handlers/DescribeSchemaHandler.java |  6 +++---
 .../org/apache/drill/exec/store/SchemaFactory.java  |  2 ++
 .../drill/exec/store/dfs/FileSystemPlugin.java      |  2 --
 .../exec/store/dfs/FileSystemSchemaFactory.java     |  2 --
 .../drill/exec/util/StoragePluginTestUtils.java     |  4 ++--
 .../java/org/apache/drill/test/BaseTestQuery.java   |  4 ++--
 .../java/org/apache/drill/test/ClusterFixture.java  |  4 ++--
 .../apache/drill/jdbc/impl/DrillConnectionImpl.java |  4 ++--
 14 files changed, 45 insertions(+), 47 deletions(-)

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 f857ec6..abb8676 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
@@ -18,6 +18,8 @@
 package org.apache.drill.exec.store.hive.schema;
 
 import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
@@ -45,9 +47,6 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.thrift.TException;
 
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
-import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
-
 public class HiveSchemaFactory extends AbstractSchemaFactory {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveSchemaFactory.class);
 
@@ -137,10 +136,10 @@ public class HiveSchemaFactory extends AbstractSchemaFactory {
     private HiveDatabaseSchema defaultSchema;
 
     HiveSchema(final SchemaConfig schemaConfig, final DrillHiveMetaStoreClient mClient, final String name) {
-      super(ImmutableList.<String>of(), name);
+      super(Collections.emptyList(), name);
       this.schemaConfig = schemaConfig;
       this.mClient = mClient;
-      getSubSchema("default");
+      getSubSchema(DEFAULT_WS_NAME);
     }
 
     @Override
@@ -152,7 +151,7 @@ public class HiveSchemaFactory extends AbstractSchemaFactory {
           return null;
         }
         HiveDatabaseSchema schema = getSubSchemaKnownExists(name);
-        if (name.equals("default")) {
+        if (DEFAULT_WS_NAME.equals(name)) {
           this.defaultSchema = schema;
         }
         return schema;
@@ -181,8 +180,8 @@ public class HiveSchemaFactory extends AbstractSchemaFactory {
     public Set<String> getSubSchemaNames() {
       try {
         List<String> dbs = mClient.getDatabases(schemaConfig.getIgnoreAuthErrors());
-        return Sets.newHashSet(dbs);
-      } catch (final TException e) {
+        return new HashSet<>(dbs);
+      } catch (TException e) {
         logger.warn("Failure while getting Hive database list.", e);
       }
       return super.getSubSchemaNames();
@@ -227,11 +226,11 @@ public class HiveSchemaFactory extends AbstractSchemaFactory {
 
     HiveReadEntry getSelectionBaseOnName(String dbName, String t) {
       if (dbName == null) {
-        dbName = "default";
+        dbName = DEFAULT_WS_NAME;
       }
-      try{
+      try {
         return mClient.getHiveReadEntry(dbName, t, schemaConfig.getIgnoreAuthErrors());
-      }catch(final TException e) {
+      } catch (TException e) {
         logger.warn("Exception occurred while trying to read table. {}.{}", dbName, t, e.getCause());
         return null;
       }
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 b0338cb..cd9a6c4 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
@@ -22,6 +22,8 @@ import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -59,13 +61,12 @@ 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.SchemaFactory;
 import org.apache.drill.exec.store.jdbc.DrillJdbcRuleBase.DrillJdbcFilterRule;
 import org.apache.drill.exec.store.jdbc.DrillJdbcRuleBase.DrillJdbcProjectRule;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Joiner;
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
-import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
 
 public class JdbcStoragePlugin extends AbstractStoragePlugin {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JdbcStoragePlugin.class);
@@ -157,10 +158,10 @@ public class JdbcStoragePlugin extends AbstractStoragePlugin {
   }
 
   /**
-   * Returns whether a condition is supported by {@link JdbcJoin}.
+   * Returns whether a condition is supported by {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcJoin}.
    *
    * <p>Corresponds to the capabilities of
-   * {@link SqlImplementor#convertConditionToSqlNode}.
+   * {@link org.apache.calcite.rel.rel2sql.SqlImplementor#convertConditionToSqlNode}.
    *
    * @param node Condition
    * @return Whether condition is supported
@@ -234,7 +235,7 @@ public class JdbcStoragePlugin extends AbstractStoragePlugin {
 
   private class CapitalizingJdbcSchema extends AbstractSchema {
 
-    final Map<String, CapitalizingJdbcSchema> schemaMap = Maps.newHashMap();
+    private final Map<String, CapitalizingJdbcSchema> schemaMap = new HashMap<>();
     private final JdbcSchema inner;
 
     public CapitalizingJdbcSchema(List<String> parentSchemaPath, String name, DataSource dataSource,
@@ -299,11 +300,11 @@ public class JdbcStoragePlugin extends AbstractStoragePlugin {
 
   private class JdbcCatalogSchema extends AbstractSchema {
 
-    private final Map<String, CapitalizingJdbcSchema> schemaMap = Maps.newHashMap();
+    private final Map<String, CapitalizingJdbcSchema> schemaMap = new HashMap<>();
     private final CapitalizingJdbcSchema defaultSchema;
 
     public JdbcCatalogSchema(String name) {
-      super(ImmutableList.<String> of(), name);
+      super(Collections.emptyList(), name);
 
       try (Connection con = source.getConnection();
            ResultSet set = con.getMetaData().getCatalogs()) {
@@ -311,7 +312,7 @@ public class JdbcStoragePlugin extends AbstractStoragePlugin {
           final String catalogName = set.getString(1);
           CapitalizingJdbcSchema schema = new CapitalizingJdbcSchema(
               getSchemaPath(), catalogName, source, dialect, convention, catalogName, null);
-          schemaMap.put(catalogName, schema);
+          schemaMap.put(schema.getName(), schema);
         }
       } catch (SQLException e) {
         logger.warn("Failure while attempting to load JDBC schema.", e);
@@ -325,7 +326,7 @@ public class JdbcStoragePlugin extends AbstractStoragePlugin {
 
         if (!schemasAdded) {
           // there were no schemas, just create a default one (the jdbc system doesn't support catalogs/schemas).
-          schemaMap.put("default", new CapitalizingJdbcSchema(ImmutableList.<String> of(), name, source, dialect,
+          schemaMap.put(SchemaFactory.DEFAULT_WS_NAME, new CapitalizingJdbcSchema(Collections.emptyList(), name, source, dialect,
               convention, null, null));
         }
       } else {
@@ -360,7 +361,7 @@ public class JdbcStoragePlugin extends AbstractStoragePlugin {
                 convention, catalogName, schemaName);
 
             // if a catalog schema doesn't exist, we'll add this at the top level.
-            schemaMap.put(schemaName, schema);
+            schemaMap.put(schema.getName(), schema);
           } else {
             CapitalizingJdbcSchema schema = new CapitalizingJdbcSchema(parentSchema.getSchemaPath(), schemaName,
                 source, dialect,
diff --git a/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithDerbyIT.java b/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithDerbyIT.java
index 65a1ea5..a22b40a 100644
--- a/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithDerbyIT.java
+++ b/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPluginWithDerbyIT.java
@@ -20,7 +20,6 @@ package org.apache.drill.exec.store.jdbc;
 import org.apache.drill.categories.JdbcStorageTest;
 import org.apache.drill.PlanTestBase;
 import org.apache.drill.exec.expr.fn.impl.DateUtility;
-import org.apache.drill.exec.proto.UserBitShared;
 
 import org.apache.drill.exec.util.StoragePluginTestUtils;
 import org.junit.BeforeClass;
@@ -126,14 +125,14 @@ public class TestJdbcPluginWithDerbyIT extends PlanTestBase {
 
   @Test
   public void showTablesDefaultSchema() throws Exception {
-    test("use derby");
-    assertEquals(1, testRunAndPrint(UserBitShared.QueryType.SQL, "show tables like 'person'"));
+    testNoResult("use derby.drill_derby_test");
+    assertEquals(1, testSql("show tables like 'PERSON'"));
   }
 
   @Test
   public void describe() throws Exception {
-    test("use derby");
-    assertEquals(19, testRunAndPrint(UserBitShared.QueryType.SQL, "describe drill_derby_test.person"));
+    testNoResult("use derby.drill_derby_test");
+    assertEquals(19, testSql("describe PERSON"));
   }
 
   @Test
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRel.java
index 7497783..9e351f7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRel.java
@@ -51,7 +51,7 @@ public class DrillFilterRel extends DrillFilterRelBase implements DrillRel {
   }
 
   public static DrillFilterRel create(RelNode child, RexNode condition) {
-    return new DrillFilterRel(child.getCluster(), child.getTraitSet(), child, condition);
+    return new DrillFilterRel(child.getCluster(), child.getTraitSet().plus(DRILL_LOGICAL), child, condition);
   }
 
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRel.java
index 2de63ab..51bfb46 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRel.java
@@ -27,7 +27,7 @@ import org.apache.calcite.plan.Convention;
 public interface DrillRel extends DrillRelNode {
   /** Calling convention for relational expressions that are "implemented" by
    * generating Drill logical plans. */
-  public static final Convention DRILL_LOGICAL = new Convention.Impl("LOGICAL", DrillRel.class);
+  Convention DRILL_LOGICAL = new Convention.Impl("LOGICAL", DrillRel.class);
 
   LogicalOperator implement(DrillImplementor implementor);
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRelFactories.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRelFactories.java
index a0b727d..feccce0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRelFactories.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRelFactories.java
@@ -44,6 +44,7 @@ import static org.apache.calcite.rel.core.RelFactories.DEFAULT_SET_OP_FACTORY;
 import static org.apache.calcite.rel.core.RelFactories.DEFAULT_SORT_FACTORY;
 import static org.apache.calcite.rel.core.RelFactories.DEFAULT_TABLE_SCAN_FACTORY;
 import static org.apache.calcite.rel.core.RelFactories.DEFAULT_VALUES_FACTORY;
+import static org.apache.drill.exec.planner.logical.DrillRel.DRILL_LOGICAL;
 
 /**
  * Contains factory implementation for creating various Drill Logical Rel nodes.
@@ -100,7 +101,7 @@ public class DrillRelFactories {
       final RelDataType rowType =
           RexUtil.createStructType(cluster.getTypeFactory(), childExprs, fieldNames, null);
 
-      return DrillProjectRel.create(cluster, child.getTraitSet(), child, childExprs, rowType);
+      return DrillProjectRel.create(cluster, child.getTraitSet().plus(DRILL_LOGICAL), child, childExprs, rowType);
     }
   }
 
@@ -125,14 +126,14 @@ public class DrillRelFactories {
     public RelNode createJoin(RelNode left, RelNode right,
                               RexNode condition, Set<CorrelationId> variablesSet,
                               JoinRelType joinType, boolean semiJoinDone) {
-      return new DrillJoinRel(left.getCluster(), left.getTraitSet(), left, right, condition, joinType);
+      return new DrillJoinRel(left.getCluster(), left.getTraitSet().plus(DRILL_LOGICAL), left, right, condition, joinType);
     }
 
     @Override
     public RelNode createJoin(RelNode left, RelNode right,
                               RexNode condition, JoinRelType joinType,
                               Set<String> variablesStopped, boolean semiJoinDone) {
-      return new DrillJoinRel(left.getCluster(), left.getTraitSet(), left, right, condition, joinType);
+      return new DrillJoinRel(left.getCluster(), left.getTraitSet().plus(DRILL_LOGICAL), left, right, condition, joinType);
     }
   }
 
@@ -145,7 +146,7 @@ public class DrillRelFactories {
     @Override
     public RelNode createAggregate(RelNode input, boolean indicator, ImmutableBitSet groupSet,
                                    com.google.common.collect.ImmutableList<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
-      return new DrillAggregateRel(input.getCluster(), input.getTraitSet(), input, indicator, groupSet, groupSets, aggCalls);
+      return new DrillAggregateRel(input.getCluster(), input.getTraitSet().plus(DRILL_LOGICAL), input, indicator, groupSet, groupSets, aggCalls);
     }
   }
 }
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 7f7dbe9..92a07c5 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
@@ -21,6 +21,7 @@ import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.core.SerializableString;
 import com.fasterxml.jackson.core.io.CharacterEscapes;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.drill.shaded.guava.com.google.common.base.Joiner;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlDescribeSchema;
@@ -35,7 +36,6 @@ import org.apache.drill.exec.planner.sql.SchemaUtilites;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.StoragePlugin;
 import org.apache.drill.exec.store.dfs.FileSystemPlugin;
-import org.apache.drill.exec.store.dfs.FileSystemSchemaFactory;
 import org.apache.drill.exec.store.dfs.WorkspaceConfig;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 
@@ -111,12 +111,12 @@ public class DescribeSchemaHandler extends DefaultSqlHandler {
     Object workspaces = configMap.remove("workspaces");
     if (workspaces != null) {
       Map map = (Map) workspaces;
-      String key = names.size() > 1 ? names.get(1) : FileSystemSchemaFactory.DEFAULT_WS_NAME;
+      String key = names.size() > 1 ? names.get(1) : SchemaFactory.DEFAULT_WS_NAME;
       Object workspace = map.get(key);
       if (workspace != null) {
         Map workspaceMap = (Map) map.get(key);
         configMap.putAll(workspaceMap);
-      } else if (FileSystemSchemaFactory.DEFAULT_WS_NAME.equals(key)) {
+      } else if (SchemaFactory.DEFAULT_WS_NAME.equals(key)) {
         configMap.putAll(mapper.convertValue(WorkspaceConfig.DEFAULT, Map.class));
       }
     }
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 4545169..4766e96 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
@@ -26,6 +26,8 @@ import java.io.IOException;
  */
 public interface SchemaFactory {
 
+  String DEFAULT_WS_NAME = "default";
+
   /**
    * Register the schemas provided by this SchemaFactory implementation under the given parent schema.
    *
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 dd1c91c..5656c55 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
@@ -17,8 +17,6 @@
  */
 package org.apache.drill.exec.store.dfs;
 
-import static org.apache.drill.exec.store.dfs.FileSystemSchemaFactory.DEFAULT_WS_NAME;
-
 import java.io.IOException;
 import java.net.URI;
 import java.util.ArrayList;
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 1a97e60..795cbd2 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
@@ -47,8 +47,6 @@ import org.apache.hadoop.fs.Path;
  */
 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;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/StoragePluginTestUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/StoragePluginTestUtils.java
index e73c1b6..1c7b2ab 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/StoragePluginTestUtils.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/StoragePluginTestUtils.java
@@ -22,6 +22,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Optional;
 
+import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.logical.FormatPluginConfig;
@@ -43,11 +44,10 @@ public class StoragePluginTestUtils {
   public static final String DFS_PLUGIN_NAME = "dfs";
 
   public static final String TMP_SCHEMA = "tmp";
-  public static final String DEFAULT_SCHEMA = "default";
   public static final String ROOT_SCHEMA = "root";
 
   public static final String DFS_TMP_SCHEMA = DFS_PLUGIN_NAME + "." + TMP_SCHEMA;
-  public static final String DFS_DEFAULT_SCHEMA = DFS_PLUGIN_NAME + "." + DEFAULT_SCHEMA;
+  public static final String DFS_DEFAULT_SCHEMA = DFS_PLUGIN_NAME + "." + SchemaFactory.DEFAULT_WS_NAME;
   public static final String DFS_ROOT_SCHEMA = DFS_PLUGIN_NAME + "." + ROOT_SCHEMA;
 
   public static final String UNIT_TEST_PROP_PREFIX = "drillJDBCUnitTests";
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/BaseTestQuery.java b/exec/java-exec/src/test/java/org/apache/drill/test/BaseTestQuery.java
index 71aa240..250f646 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/BaseTestQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/BaseTestQuery.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.test;
 
-import static org.apache.drill.exec.util.StoragePluginTestUtils.DEFAULT_SCHEMA;
 import static org.apache.drill.exec.util.StoragePluginTestUtils.ROOT_SCHEMA;
 import static org.apache.drill.exec.util.StoragePluginTestUtils.TMP_SCHEMA;
 import static org.hamcrest.core.StringContains.containsString;
@@ -35,6 +34,7 @@ import java.util.List;
 import java.util.Properties;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.drill.test.DrillTestWrapper.TestServices;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.config.DrillProperties;
@@ -194,7 +194,7 @@ public class BaseTestQuery extends ExecTest {
       StoragePluginTestUtils.updateSchemaLocation(StoragePluginTestUtils.DFS_PLUGIN_NAME, pluginRegistry,
         dirTestWatcher.getRootDir(), ROOT_SCHEMA);
       StoragePluginTestUtils.updateSchemaLocation(StoragePluginTestUtils.DFS_PLUGIN_NAME, pluginRegistry,
-        dirTestWatcher.getRootDir(), DEFAULT_SCHEMA);
+        dirTestWatcher.getRootDir(), SchemaFactory.DEFAULT_WS_NAME);
     }
 
     if (!properties.containsKey(DrillProperties.DRILLBIT_CONNECTION)) {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
index 996898e..a9d2977 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
@@ -33,6 +33,7 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.Properties;
 
+import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.drill.test.DrillTestWrapper.TestServices;
 import org.apache.drill.common.config.DrillProperties;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -61,7 +62,6 @@ import org.apache.drill.shaded.guava.com.google.common.base.Charsets;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.io.Resources;
 
-import static org.apache.drill.exec.util.StoragePluginTestUtils.DEFAULT_SCHEMA;
 import static org.apache.drill.exec.util.StoragePluginTestUtils.DFS_TMP_SCHEMA;
 import static org.apache.drill.exec.util.StoragePluginTestUtils.ROOT_SCHEMA;
 import static org.apache.drill.exec.util.StoragePluginTestUtils.TMP_SCHEMA;
@@ -267,7 +267,7 @@ public class ClusterFixture extends BaseFixture implements AutoCloseable {
 
     StoragePluginTestUtils.updateSchemaLocation(StoragePluginTestUtils.DFS_PLUGIN_NAME, pluginRegistry, builder.dirTestWatcher.getDfsTestTmpDir(), TMP_SCHEMA);
     StoragePluginTestUtils.updateSchemaLocation(StoragePluginTestUtils.DFS_PLUGIN_NAME, pluginRegistry, builder.dirTestWatcher.getRootDir(), ROOT_SCHEMA);
-    StoragePluginTestUtils.updateSchemaLocation(StoragePluginTestUtils.DFS_PLUGIN_NAME, pluginRegistry, builder.dirTestWatcher.getRootDir(), DEFAULT_SCHEMA);
+    StoragePluginTestUtils.updateSchemaLocation(StoragePluginTestUtils.DFS_PLUGIN_NAME, pluginRegistry, builder.dirTestWatcher.getRootDir(), SchemaFactory.DEFAULT_WS_NAME);
 
     // Create the mock data plugin
 
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 b7e61b9..820044e 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
@@ -55,6 +55,7 @@ import org.apache.drill.exec.memory.RootAllocatorFactory;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.jdbc.AlreadyClosedSqlException;
 import org.apache.drill.jdbc.DrillConnection;
@@ -65,7 +66,6 @@ import org.slf4j.Logger;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Throwables;
 
-import static org.apache.drill.exec.util.StoragePluginTestUtils.DEFAULT_SCHEMA;
 import static org.apache.drill.exec.util.StoragePluginTestUtils.DFS_PLUGIN_NAME;
 import static org.apache.drill.exec.util.StoragePluginTestUtils.ROOT_SCHEMA;
 import static org.apache.drill.exec.util.StoragePluginTestUtils.TMP_SCHEMA;
@@ -666,7 +666,7 @@ class DrillConnectionImpl extends AvaticaConnection
         if (dfsDefaultPath == null) {
           logger.warn(logMessage, UNIT_TEST_DFS_DEFAULT_PROP);
         } else {
-          updateSchemaLocation(DFS_PLUGIN_NAME, pluginRegistry, new File(dfsDefaultPath), DEFAULT_SCHEMA);
+          updateSchemaLocation(DFS_PLUGIN_NAME, pluginRegistry, new File(dfsDefaultPath), SchemaFactory.DEFAULT_WS_NAME);
         }
       }
     } catch(Throwable e) {


[drill] 11/15: DRILL-6868: Upgrade Janino compiler to 3.0.11

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

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

commit 67adde12853f14aab8d0e2ffc162a2daaf53d954
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Fri Oct 19 18:04:12 2018 +0300

    DRILL-6868: Upgrade Janino compiler to 3.0.11
    
    - Remove workaround where removing adjacent ALOAD-POP instruction pairs
    - Remove ModifiedUnparser and use DeepCopier for modifying methods instead of modifying it with custom Unparser implementation
    
    closes #1553
---
 .../exec/compile/bytecode/AloadPopRemover.java     | 333 ---------------------
 .../bytecode/ValueHolderReplacementVisitor.java    |  10 +-
 .../drill/exec/expr/fn/MethodGrabbingVisitor.java  |  57 +++-
 .../drill/exec/expr/fn/ModifiedUnparser.java       | 110 -------
 pom.xml                                            |   2 +-
 5 files changed, 58 insertions(+), 454 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/AloadPopRemover.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/AloadPopRemover.java
deleted file mode 100644
index a6df261..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/AloadPopRemover.java
+++ /dev/null
@@ -1,333 +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.compile.bytecode;
-
-import org.objectweb.asm.AnnotationVisitor;
-import org.objectweb.asm.Attribute;
-import org.objectweb.asm.Handle;
-import org.objectweb.asm.Label;
-import org.objectweb.asm.MethodVisitor;
-import org.objectweb.asm.Opcodes;
-import org.objectweb.asm.TypePath;
-
-
-/**
- * Remove any adjacent ALOAD-POP instruction pairs.
- *
- * The Janino compiler generates an instruction stream where it will ALOAD a
- * holder's objectref, and then immediately POP it because the compiler has
- * recognized that the method call that it loaded the objectref for is static
- * (i.e., no this pointer is required). This causes a problem with our scalar
- * replacement strategy, because once we remove the holders, we simply eliminate
- * the ALOAD instructions. In this case, the POP gets left behind, and breaks
- * the program stack.
- *
- * This class looks for ALOADs that are immediately followed by a POP, and removes
- * the pair of instructions altogether.
- *
- * It is unknown if other compilers besides Janino generate this instruction sequence,
- * but to be on the safe side, we'll use this class unconditionally to filter bytecode.
- *
- * TODO: this might be easier to do by going off an InsnList; the state machine would
- * be in the loop that visits the instructions then.
- */
-public class AloadPopRemover extends MethodVisitor {
-  private final static int NONE = -1; // var value to indicate no captured ALOAD
-  private int var = NONE;
-
-  /**
-   * Constructor.
-   *
-   * See {@link org.objectweb.asm.MethodVisitor#MethodVisitor(int)}.
-   */
-  public AloadPopRemover(final int api) {
-    super(api);
-  }
-
-  /**
-   * Constructor.
-   *
-   * See {@link org.objectweb.asm.MethodVisitor#MethodVisitor(int, MethodVisitor)}.
-   */
-  public AloadPopRemover(final int api, final MethodVisitor mv) {
-    super(api, mv);
-  }
-
-  /**
-   * Process a deferred ALOAD instruction, if there is one.
-   *
-   * If there is no deferred ALOAD, does nothing, and returns false.
-   *
-   * If there is a deferred ALOAD, and we're on a POP instruction
-   * (indicated by onPop), does nothing (the ALOAD is not forwarded),
-   * and returns true.
-   *
-   * If there is a deferred ALOAD and we're not on a POP instruction,
-   * forwards the deferred ALOAD, and returns false
-   *
-   * @param onPop true if the current instruction is POP
-   * @return true if onPop and there was a deferred ALOAD, false otherwise;
-   *   basically, returns true if the ALOAD-POP optimization is required
-   */
-  private boolean processDeferredAload(final boolean onPop) {
-    // if the previous instruction wasn't an ALOAD, then there's nothing to do
-    if (var == NONE) {
-      return false;
-    }
-
-    // clear the variable index, but save it for local use
-    final int localVar = var;
-    var = NONE;
-
-    // if the next instruction is a POP, don't emit the deferred ALOAD
-    if (onPop) {
-      return true;
-    }
-
-    // if we got here, we're not on a POP, so emit the deferred ALOAD
-    super.visitVarInsn(Opcodes.ALOAD, localVar);
-    return false;
-  }
-
-  @Override
-  public AnnotationVisitor visitAnnotation(final String desc, final boolean visible) {
-    processDeferredAload(false);
-    final AnnotationVisitor annotationVisitor = super.visitAnnotation(desc, visible);
-    return annotationVisitor;
-  }
-
-  @Override
-  public AnnotationVisitor visitAnnotationDefault() {
-    processDeferredAload(false);
-    final AnnotationVisitor annotationVisitor = super.visitAnnotationDefault();
-    return annotationVisitor;
-  }
-
-  @Override
-  public void visitAttribute(final Attribute attr) {
-    processDeferredAload(false);
-    super.visitAttribute(attr);
-  }
-
-  @Override
-  public void visitCode() {
-    processDeferredAload(false);
-    super.visitCode();
-  }
-
-  @Override
-  public void visitEnd() {
-    processDeferredAload(false);
-    super.visitEnd();
-  }
-
-  @Override
-  public void visitFieldInsn(final int opcode, final String owner, final String name,
-      final String desc) {
-    processDeferredAload(false);
-    super.visitFieldInsn(opcode, owner, name, desc);
-  }
-
-  @Override
-  public void visitFrame(final int type, final int nLocal,
-      final Object[] local, final int nStack, final Object[] stack) {
-    processDeferredAload(false);
-    super.visitFrame(type, nLocal, local, nStack, stack);
-  }
-
-  @Override
-  public void visitIincInsn(final int var, final int increment) {
-    processDeferredAload(false);
-    super.visitIincInsn(var, increment);
-  }
-
-  @Override
-  public void visitInsn(final int opcode) {
-    /*
-     * If we don't omit an ALOAD with a following POP, then forward this.
-     * In other words, if we omit an ALOAD because we're on the following POP,
-     * don't forward this POP, which omits the ALOAD-POP pair.
-     */
-    if (!processDeferredAload(Opcodes.POP == opcode)) {
-      super.visitInsn(opcode);
-    }
-  }
-
-  @Override
-  public AnnotationVisitor visitInsnAnnotation(final int typeRef,
-      final TypePath typePath, final String desc, final boolean visible) {
-    processDeferredAload(false);
-    final AnnotationVisitor annotationVisitor = super.visitInsnAnnotation(
-        typeRef, typePath, desc, visible);
-    return annotationVisitor;
-  }
-
-  @Override
-  public void visitIntInsn(final int opcode, final int operand) {
-    processDeferredAload(false);
-    super.visitIntInsn(opcode, operand);
-  }
-
-  @Override
-  public void visitInvokeDynamicInsn(final String name, final String desc,
-      final Handle bsm, final Object... bsmArgs) {
-    processDeferredAload(false);
-    super.visitInvokeDynamicInsn(name, desc, bsm, bsmArgs);
-  }
-
-  @Override
-  public void visitJumpInsn(final int opcode, final Label label) {
-    processDeferredAload(false);
-    super.visitJumpInsn(opcode, label);
-  }
-
-  @Override
-  public void visitLabel(final Label label) {
-    processDeferredAload(false);
-    super.visitLabel(label);
-  }
-
-  @Override
-  public void visitLdcInsn(final Object cst) {
-    processDeferredAload(false);
-    super.visitLdcInsn(cst);
-  }
-
-  @Override
-  public void visitLineNumber(final int line, final Label start) {
-    processDeferredAload(false);
-    super.visitLineNumber(line, start);
-  }
-
-  @Override
-  public void visitLocalVariable(final String name, final String desc,
-      final String signature, final Label start, final Label end, final int index) {
-    processDeferredAload(false);
-    super.visitLocalVariable(name, desc, signature, start, end, index);
-  }
-
-  @Override
-  public AnnotationVisitor visitLocalVariableAnnotation(final int typeRef,
-      final TypePath typePath, final Label[] start, final Label[] end,
-      final int[] index, final String desc, final boolean visible) {
-    processDeferredAload(false);
-    final AnnotationVisitor annotationVisitor =
-        super.visitLocalVariableAnnotation(typeRef, typePath, start, end, index,
-            desc, visible);
-    return annotationVisitor;
-  }
-
-  @Override
-  public void visitLookupSwitchInsn(final Label dflt, final int[] keys,
-      final Label[] labels) {
-    processDeferredAload(false);
-    super.visitLookupSwitchInsn(dflt, keys, labels);
-  }
-
-  @Override
-  public void visitMaxs(final int maxStack, final int maxLocals) {
-    processDeferredAload(false);
-    super.visitMaxs(maxStack, maxLocals);
-  }
-
-  @Deprecated
-  @Override
-  public void visitMethodInsn(final int opcode, final String owner,
-      final String name, final String desc) {
-    processDeferredAload(false);
-    super.visitMethodInsn(opcode, owner, name, desc);
-  }
-
-  @Override
-  public void visitMethodInsn(final int opcode, final String owner,
-      final String name, final String desc, final boolean itf) {
-    processDeferredAload(false);
-    super.visitMethodInsn(opcode, owner, name, desc, itf);
-  }
-
-  @Override
-  public void visitMultiANewArrayInsn(final String desc, final int dims) {
-    processDeferredAload(false);
-    super.visitMultiANewArrayInsn(desc, dims);
-  }
-
-  @Override
-  public void visitParameter(final String name, final int access) {
-    processDeferredAload(false);
-    super.visitParameter(name, access);
-  }
-
-  @Override
-  public AnnotationVisitor visitParameterAnnotation(final int parameter,
-      final String desc, final boolean visible) {
-    processDeferredAload(false);
-    final AnnotationVisitor annotationVisitor =
-        super.visitParameterAnnotation(parameter, desc, visible);
-    return annotationVisitor;
-  }
-
-  @Override
-  public void visitTableSwitchInsn(final int min, final int max,
-      final Label dflt, final Label... labels) {
-    processDeferredAload(false);
-    super.visitTableSwitchInsn(min, max, dflt, labels);
-  }
-
-  @Override
-  public AnnotationVisitor visitTryCatchAnnotation(final int typeRef,
-      final TypePath typePath, final String desc, final boolean visible) {
-    processDeferredAload(false);
-    final AnnotationVisitor annotationVisitor =
-        super.visitTryCatchAnnotation(typeRef, typePath, desc, visible);
-    return annotationVisitor;
-  }
-
-  @Override
-  public void visitTryCatchBlock(final Label start, final Label end,
-      final Label handler, final String type) {
-    processDeferredAload(false);
-    super.visitTryCatchBlock(start, end, handler, type);
-  }
-
-  @Override
-  public AnnotationVisitor visitTypeAnnotation(final int typeRef,
-      final TypePath typePath, final String desc, final boolean visible) {
-    processDeferredAload(false);
-    final AnnotationVisitor annotationVisitor =
-        super.visitTypeAnnotation(typeRef, typePath, desc, visible);
-    return annotationVisitor;
-  }
-
-  @Override
-  public void visitTypeInsn(final int opcode, final String type) {
-    processDeferredAload(false);
-    super.visitTypeInsn(opcode, type);
-  }
-
-  @Override
-  public void visitVarInsn(final int opcode, final int var) {
-    processDeferredAload(false);
-
-    // if we see an ALOAD, defer forwarding it until we know what the next instruction is
-    if (Opcodes.ALOAD == opcode) {
-      this.var = var;
-    } else {
-      super.visitVarInsn(opcode, var);
-    }
-  }
-}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ValueHolderReplacementVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ValueHolderReplacementVisitor.java
index 1ed9ea5..9094b33 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ValueHolderReplacementVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/bytecode/ValueHolderReplacementVisitor.java
@@ -55,14 +55,8 @@ public class ValueHolderReplacementVisitor extends ClassVisitor {
       innerVisitor = new CheckMethodVisitorFsm(api, innerVisitor);
     }
 
-    /*
-     * Before using the ScalarReplacementNode to rewrite method code, use the
-     * AloadPopRemover to eliminate unnecessary ALOAD-POP pairs; see the
-     * AloadPopRemover javadoc for a detailed explanation.
-     */
-    return new AloadPopRemover(api,
-        new ScalarReplacementNode(
-            className, access, name, desc, signature, exceptions, innerVisitor, verifyBytecode));
+    return new ScalarReplacementNode(className, access, name, desc, signature,
+        exceptions,innerVisitor, verifyBytecode);
   }
 
   private static class Debugger extends MethodNode {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
index 782e1e2..18cc4e4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/MethodGrabbingVisitor.java
@@ -21,10 +21,13 @@ import java.io.StringWriter;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.codehaus.commons.compiler.CompileException;
 import org.codehaus.janino.Java;
 import org.codehaus.janino.Java.AbstractClassDeclaration;
 import org.codehaus.janino.Java.MethodDeclarator;
+import org.codehaus.janino.Unparser;
 import org.codehaus.janino.util.AbstractTraverser;
+import org.codehaus.janino.util.DeepCopier;
 
 public class MethodGrabbingVisitor {
 
@@ -64,9 +67,59 @@ public class MethodGrabbingVisitor {
     @Override
     public void traverseMethodDeclarator(MethodDeclarator methodDeclarator) {
       if (captureMethods) {
+        // Generates a "labeled statement".
+        // This code takes code from the method body, wraps it into the labeled statement
+        // and replaces all the return statements by break command with label.
+        //
+        // For example, the following method
+        //    public void foo(int a) {
+        //      if (a < 0) {
+        //        return;
+        //      } else {
+        //        do something;
+        //      }
+        //    }
+        //
+        // will be converted to
+        //    MethodClassName_foo: {
+        //      if (a < 0) {
+        //        break MethodClassName_foo;
+        //      } else {
+        //        do something;
+        //      }
+        //    }
+
+        // Constructs a name of the resulting label
+        // using methods class name and method name itself.
+        String[] fQCN = methodDeclarator.getDeclaringType().getClassName().split("\\.");
+        String returnLabel = fQCN[fQCN.length - 1] + "_" + methodDeclarator.name;
+        Java.Block methodBodyBlock = new Java.Block(methodDeclarator.getLocation());
+
+        // DeepCopier implementation which returns break statement with label
+        // instead if return statement.
+        DeepCopier returnStatementReplacer = new DeepCopier() {
+          @Override
+          public Java.BlockStatement copyReturnStatement(Java.ReturnStatement subject) {
+            return new Java.BreakStatement(subject.getLocation(), returnLabel);
+          }
+        };
+        try {
+          // replaces return statements and stores the result into methodBodyBlock
+          methodBodyBlock.addStatements(
+              returnStatementReplacer.copyBlockStatements(methodDeclarator.optionalStatements));
+        } catch (CompileException e) {
+          throw new RuntimeException(e);
+        }
+
+        // wraps method code with replaced return statements into label statement.
+        Java.LabeledStatement labeledStatement =
+            new Java.LabeledStatement(methodDeclarator.getLocation(), returnLabel, methodBodyBlock);
+
+        // Unparse the labeled statement.
         StringWriter writer = new StringWriter();
-        ModifiedUnparser unparser = new ModifiedUnparser(writer);
-        unparser.visitMethodDeclarator(methodDeclarator);
+        Unparser unparser = new Unparser(writer);
+        // unparses labeledStatement and stores unparsed code into writer
+        unparser.unparseBlockStatement(labeledStatement);
         unparser.close();
         writer.flush();
         methods.put(methodDeclarator.name, writer.getBuffer().toString());
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparser.java
deleted file mode 100644
index e7b2244..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/ModifiedUnparser.java
+++ /dev/null
@@ -1,110 +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.expr.fn;
-
-import java.io.Writer;
-import java.util.List;
-
-import org.codehaus.janino.Java;
-import org.codehaus.janino.Unparser;
-import org.codehaus.janino.util.AutoIndentWriter;
-
-/**
- * This is a modified version of {@link Unparser} so that we can avoid
- * rendering few things.
- */
-public class ModifiedUnparser extends Unparser {
-
-  private String returnLabel;
-
-  public ModifiedUnparser(Writer writer) {
-    super(writer);
-  }
-
-  @Override
-  public void unparseBlockStatement(Java.BlockStatement blockStatement) {
-    // Unparser uses anonymous classes for visiting statements,
-    // therefore added this check for customizing of handling ReturnStatement.
-    if (blockStatement instanceof Java.ReturnStatement) {
-      visitReturnStatement((Java.ReturnStatement) blockStatement);
-    } else {
-      super.unparseBlockStatement(blockStatement);
-    }
-  }
-
-  /**
-   * Parses specified {@link Java.MethodDeclarator}, wraps its content
-   * with replaced {@code return} statements by {@code break} ones into the
-   * block with label and stores it into {@link java.io.PrintWriter}.
-   *
-   * @param methodDeclarator method to parse
-   */
-  public void visitMethodDeclarator(Java.MethodDeclarator methodDeclarator) {
-    if (methodDeclarator.optionalStatements == null) {
-      pw.print(';');
-    } else if (methodDeclarator.optionalStatements.isEmpty()) {
-      pw.print(" {}");
-    } else {
-      pw.println(' ');
-      // Add labels to handle return statements within function templates
-      String[] fQCN = methodDeclarator.getDeclaringType().getClassName().split("\\.");
-      returnLabel = fQCN[fQCN.length - 1] + "_" + methodDeclarator.name;
-      pw.print(returnLabel);
-      pw.println(": {");
-      pw.print(AutoIndentWriter.INDENT);
-      unparseStatements(methodDeclarator.optionalStatements);
-      pw.print(AutoIndentWriter.UNINDENT);
-      pw.println("}");
-      pw.print(' ');
-    }
-  }
-
-  private void visitReturnStatement(Java.ReturnStatement returnStatement) {
-    pw.print("break ");
-    pw.print(returnLabel);
-    if (returnStatement.optionalReturnValue != null) {
-      pw.print(' ');
-      unparseAtom(returnStatement.optionalReturnValue);
-    }
-    pw.print(';');
-  }
-
-  /**
-   * The following helper method is copied from the parent class since it
-   * is declared as private in the parent class and can not be used in the child
-   * class (this).
-   */
-  private void unparseStatements(List<? extends Java.BlockStatement> statements) {
-    int state = -1;
-    for (Java.BlockStatement bs : statements) {
-      int x = (
-        bs instanceof Java.Block ? 1 :
-          bs instanceof Java.LocalClassDeclarationStatement ? 2 :
-            bs instanceof Java.LocalVariableDeclarationStatement ? 3 :
-              bs instanceof Java.SynchronizedStatement ? 4 : 99
-      );
-      if (state != -1 && state != x) {
-        pw.println(AutoIndentWriter.CLEAR_TABULATORS);
-      }
-      state = x;
-
-      unparseBlockStatement(bs);
-      pw.println();
-    }
-  }
-}
diff --git a/pom.xml b/pom.xml
index 0cc2527..ca847d2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -48,7 +48,7 @@
     <parquet.version>1.10.0</parquet.version>
     <calcite.version>1.17.0-drill-r1</calcite.version>
     <avatica.version>1.12.0</avatica.version>
-    <janino.version>3.0.10</janino.version>
+    <janino.version>3.0.11</janino.version>
     <sqlline.version>1.5.0</sqlline.version>
     <jackson.version>2.9.5</jackson.version>
     <jackson.databind.version>2.9.5</jackson.databind.version>