DRILL-6294: Changes to support Calcite 1.16.0 , and remove deprecated API usage

closes #1198


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

Branch: refs/heads/master
Commit: 379196970a0eb4d1c1059902b5daa0673dfb7fbf
Parents: ead1e9d
Author: Volodymyr Vysotskyi <vvo...@gmail.com>
Authored: Wed Feb 7 16:24:50 2018 +0200
Committer: Ben-Zvi <bben-...@mapr.com>
Committed: Fri Apr 13 16:01:15 2018 -0700

----------------------------------------------------------------------
 .../apache/drill/exec/store/jdbc/JdbcPrel.java  |  2 +-
 .../exec/store/jdbc/JdbcStoragePlugin.java      |  6 ++-
 .../apache/drill/exec/planner/RuleInstance.java |  4 +-
 .../exec/planner/common/DrillFilterRelBase.java |  3 +-
 .../exec/planner/common/DrillJoinRelBase.java   |  6 ++-
 .../planner/common/DrillProjectRelBase.java     |  4 +-
 .../exec/planner/common/DrillRelOptUtil.java    | 27 ++++++-------
 .../drill/exec/planner/cost/DrillCostBase.java  |  9 +++--
 .../cost/DrillRelMdDistinctRowCount.java        |  2 +-
 .../planner/logical/DrillConstExecutor.java     |  2 +-
 .../logical/DrillPushFilterPastProjectRule.java | 13 +++----
 .../logical/DrillPushLimitToScanRule.java       | 17 +++-----
 .../logical/DrillReduceAggregatesRule.java      | 37 ++++++++++--------
 .../logical/DrillReduceExpressionsRule.java     |  4 +-
 .../exec/planner/logical/DrillSortRel.java      |  7 ++--
 .../logical/partition/PruneScanRule.java        |  6 +--
 .../exec/planner/physical/AggPrelBase.java      | 10 -----
 .../physical/HashToRandomExchangePrel.java      | 12 +++---
 .../drill/exec/planner/physical/JoinPrel.java   | 20 ++++------
 .../exec/planner/physical/MergeJoinPrule.java   | 11 +++---
 .../planner/physical/PhysicalPlanCreator.java   |  4 +-
 .../exec/planner/physical/ProjectPrule.java     | 14 +++----
 .../exec/planner/physical/StreamAggPrule.java   |  6 +--
 .../exec/planner/physical/WindowPrule.java      | 12 ++----
 .../exec/planner/physical/WriterPrule.java      |  8 ++--
 .../physical/explain/NumberingRelWriter.java    |  3 +-
 .../visitor/ExcessiveExchangeIdentifier.java    | 10 ++---
 .../physical/visitor/StarColumnConverter.java   | 41 ++++++++++----------
 .../physical/visitor/SwapHashJoinVisitor.java   | 12 +++---
 .../exec/planner/sql/DrillSqlAggOperator.java   |  6 ++-
 .../drill/exec/planner/sql/SqlConverter.java    | 15 ++++---
 .../exec/planner/sql/TypeInferenceUtils.java    |  1 +
 .../planner/sql/handlers/DefaultSqlHandler.java |  3 +-
 .../planner/sql/handlers/SqlHandlerUtil.java    |  8 ++--
 .../sql/parser/UnsupportedOperatorsVisitor.java | 22 +++++------
 .../store/parquet/ParquetPushDownFilter.java    |  8 ++--
 exec/jdbc-all/pom.xml                           |  2 +-
 .../apache/drill/jdbc/impl/DrillMetaImpl.java   |  2 +
 .../drill/jdbc/impl/DrillResultSetImpl.java     | 26 ++++---------
 pom.xml                                         |  4 +-
 40 files changed, 194 insertions(+), 215 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
----------------------------------------------------------------------
diff --git 
a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
 
b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
index 63752fa..abeca23 100644
--- 
a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
+++ 
b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
@@ -52,7 +52,7 @@ public class JdbcPrel extends AbstractRelNode implements Prel 
{
   public JdbcPrel(RelOptCluster cluster, RelTraitSet traitSet, 
JdbcIntermediatePrel prel) {
     super(cluster, traitSet);
     final RelNode input = prel.getInput();
-    rows = input.getRows();
+    rows = input.estimateRowCount(cluster.getMetadataQuery());
     convention = (DrillJdbcConvention) 
input.getTraitSet().getTrait(ConventionTraitDef.INSTANCE);
 
     // generate sql for tree.

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStoragePlugin.java
----------------------------------------------------------------------
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 47f0f21..6502e24 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
@@ -38,6 +38,7 @@ import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.rel2sql.SqlImplementor;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
@@ -46,6 +47,7 @@ import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlDialectFactoryImpl;
 import org.apache.commons.dbcp.BasicDataSource;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.expression.SchemaPath;
@@ -101,7 +103,7 @@ public class JdbcStoragePlugin extends 
AbstractStoragePlugin {
     }
 
     this.source = source;
-    this.dialect = JdbcSchema.createDialect(source);
+    this.dialect = JdbcSchema.createDialect(SqlDialectFactoryImpl.INSTANCE, 
source);
     this.convention = new DrillJdbcConvention(dialect, name);
   }
 
@@ -163,7 +165,7 @@ public class JdbcStoragePlugin extends 
AbstractStoragePlugin {
    * Returns whether a condition is supported by {@link JdbcJoin}.
    *
    * <p>Corresponds to the capabilities of
-   * {@link JdbcJoin#convertConditionToSqlNode}.
+   * {@link SqlImplementor#convertConditionToSqlNode}.
    *
    * @param node Condition
    * @return Whether condition is supported

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/RuleInstance.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/RuleInstance.java 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/RuleInstance.java
index a390ff4..67ddf06 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/RuleInstance.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/RuleInstance.java
@@ -44,7 +44,7 @@ import 
org.apache.drill.exec.planner.logical.DrillRelFactories;
 public interface RuleInstance {
 
   ReduceExpressionsRule PROJECT_INSTANCE =
-      new 
ReduceExpressionsRule.ProjectReduceExpressionsRule(LogicalProject.class,
+      new 
ReduceExpressionsRule.ProjectReduceExpressionsRule(LogicalProject.class, true,
           DrillRelFactories.LOGICAL_BUILDER);
 
   UnionToDistinctRule UNION_TO_DISTINCT_RULE =
@@ -82,7 +82,7 @@ public interface RuleInstance {
           LogicalJoin.class, DrillRelFactories.LOGICAL_BUILDER);
 
   ReduceExpressionsRule CALC_INSTANCE =
-      new ReduceExpressionsRule.CalcReduceExpressionsRule(LogicalCalc.class,
+      new ReduceExpressionsRule.CalcReduceExpressionsRule(LogicalCalc.class, 
true,
           DrillRelFactories.LOGICAL_BUILDER);
 
   FilterSetOpTransposeRule FILTER_SET_OP_TRANSPOSE_RULE =

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillFilterRelBase.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillFilterRelBase.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillFilterRelBase.java
index b87e974..b0bca28 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillFilterRelBase.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillFilterRelBase.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.planner.common;
 
+import org.apache.calcite.rel.metadata.RelMdUtil;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
@@ -90,7 +91,7 @@ public abstract class DrillFilterRelBase extends Filter 
implements DrillRelNode
 
     for (int i = 0; i< numConjuncts; i++) {
       RexNode conjFilter = 
RexUtil.composeConjunction(this.getCluster().getRexBuilder(), 
conjunctions.subList(0, i + 1), false);
-      compNum += Filter.estimateFilteredRows(child, conjFilter);
+      compNum += RelMdUtil.estimateFilteredRows(child, conjFilter, mq);
     }
 
     return compNum * DrillCostBase.COMPARE_CPU_COST;

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
index ba146f5..06f02c0 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
@@ -21,6 +21,7 @@ import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.expr.holders.IntHolder;
@@ -55,8 +56,9 @@ public abstract class DrillJoinRelBase extends Join 
implements DrillRelNode {
   private final double joinRowFactor;
 
   public DrillJoinRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode 
left, RelNode right, RexNode condition,
-      JoinRelType joinType){
-    super(cluster, traits, left, right, condition, joinType, 
Collections.<String> emptySet());
+      JoinRelType joinType) {
+    super(cluster, traits, left, right, condition,
+        CorrelationId.setOf(Collections.<String> emptySet()), joinType);
     this.joinRowFactor = 
PrelUtil.getPlannerSettings(cluster.getPlanner()).getRowCountEstimateFactor();
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java
index 8141a8c..b7881c2 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java
@@ -64,7 +64,7 @@ public abstract class DrillProjectRelBase extends Project 
implements DrillRelNod
 
   protected DrillProjectRelBase(Convention convention, RelOptCluster cluster, 
RelTraitSet traits, RelNode child, List<? extends RexNode> exps,
       RelDataType rowType) {
-    super(cluster, traits, child, exps, rowType, Flags.BOXED);
+    super(cluster, traits, child, exps, rowType);
     assert getConvention() == convention;
     nonSimpleFieldCount = this.getRowType().getFieldCount() - 
getSimpleFieldCount();
   }
@@ -92,7 +92,7 @@ public abstract class DrillProjectRelBase extends Project 
implements DrillRelNod
   protected List<NamedExpression> getProjectExpressions(DrillParseContext 
context) {
     List<NamedExpression> expressions = Lists.newArrayList();
 
-    HashMap<String, String> starColPrefixes = new HashMap<String, String>();
+    HashMap<String, String> starColPrefixes = new HashMap<>();
 
     // T1.* will subsume T1.*0, but will not subsume any regular 
column/expression.
     // Select *, col1, *, col2 : the intermediate will output one set of 
regular columns expanded from star with prefix,

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
index dff83f6..36d7db2 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelOptUtil.java
@@ -89,11 +89,7 @@ public abstract class DrillRelOptUtil {
         List<TypeProtos.MinorType> types = Lists.newArrayListWithCapacity(2);
         
types.add(Types.getMinorTypeFromName(type1.getSqlTypeName().getName()));
         
types.add(Types.getMinorTypeFromName(type2.getSqlTypeName().getName()));
-        if(TypeCastRules.getLeastRestrictiveType(types) != null) {
-          return true;
-        }
-
-        return false;
+        return TypeCastRules.getLeastRestrictiveType(types) != null;
       }
     }
     return true;
@@ -124,8 +120,11 @@ public abstract class DrillRelOptUtil {
           }
         };
 
-    return RelOptUtil.createProject(rel, refs, fieldNames, false,
-        DrillRelFactories.LOGICAL_BUILDER.create(rel.getCluster(), null));
+    return DrillRelFactories.LOGICAL_BUILDER
+        .create(rel.getCluster(), null)
+        .push(rel)
+        .projectNamed(refs, fieldNames, true)
+        .build();
   }
 
   public static boolean isTrivialProject(Project project, boolean 
useNamesInIdentityProjCalc) {
@@ -140,11 +139,11 @@ public abstract class DrillRelOptUtil {
    *
    * @param rowType : input rowType
    * @param typeFactory : type factory used to create a new row type.
-   * @return
+   * @return a rowType having all unique field name.
    */
   public static RelDataType uniqifyFieldName(final RelDataType rowType, final 
RelDataTypeFactory typeFactory) {
     return typeFactory.createStructType(RelOptUtil.getFieldTypeList(rowType),
-        SqlValidatorUtil.uniquify(rowType.getFieldNames()));
+        SqlValidatorUtil.uniquify(rowType.getFieldNames(), 
SqlValidatorUtil.EXPR_SUGGESTER, true));
   }
 
   /**
@@ -245,9 +244,8 @@ public abstract class DrillRelOptUtil {
    * @param project : The project rel
    * @return : Return true if the rowcount is unknown. Otherwise, false.
    */
-  public static boolean isProjectOutputRowcountUnknown(RelNode project) {
-    assert project instanceof Project : "Rel is NOT an instance of project!";
-    for (RexNode rex : project.getChildExps()) {
+  public static boolean isProjectOutputRowcountUnknown(Project project) {
+    for (RexNode rex : project.getProjects()) {
       if (rex instanceof RexCall) {
         if ("flatten".equals(((RexCall) 
rex).getOperator().getName().toLowerCase())) {
           return true;
@@ -263,8 +261,7 @@ public abstract class DrillRelOptUtil {
    * @param project : The project rel
    * @return : Return true if the project output schema is unknown. Otherwise, 
false.
    */
-  public static boolean isProjectOutputSchemaUnknown(RelNode project) {
-    assert project instanceof Project : "Rel is NOT an instance of project!";
+  public static boolean isProjectOutputSchemaUnknown(Project project) {
     try {
       RexVisitor<Void> visitor =
           new RexVisitorImpl<Void>(true) {
@@ -276,7 +273,7 @@ public abstract class DrillRelOptUtil {
               return super.visitCall(call);
             }
           };
-      for (RexNode rex : ((Project) project).getProjects()) {
+      for (RexNode rex : project.getProjects()) {
         rex.accept(visitor);
       }
     } catch (Util.FoundOne e) {

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillCostBase.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillCostBase.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillCostBase.java
index 229a1cb..e225977 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillCostBase.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillCostBase.java
@@ -1,4 +1,4 @@
-/*******************************************************************************
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -14,13 +14,13 @@
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * See the License for the specific language governing permissions and
  * limitations under the License.
- 
******************************************************************************/
+ */
 
 package org.apache.drill.exec.planner.cost;
 
 import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.util.Util;
+import org.apache.calcite.runtime.Utilities;
 
 /**
  * Implementation of the DrillRelOptCost, modeled similar to VolcanoCost
@@ -159,7 +159,8 @@ public class DrillCostBase implements DrillRelOptCost {
 
   @Override
   public int hashCode() {
-    return Util.hashCode(rowCount) + Util.hashCode(cpu) + Util.hashCode(io) + 
Util.hashCode(network);
+    return Utilities.hashCode(rowCount) + Utilities.hashCode(cpu)
+        + Utilities.hashCode(io) + Utilities.hashCode(network);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java
index ab7fb87..054ff5e 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdDistinctRowCount.java
@@ -60,6 +60,6 @@ public class DrillRelMdDistinctRowCount extends 
RelMdDistinctRowCount {
   public Double getDistinctRowCount(DrillScanRel scan, RelMetadataQuery mq,
       ImmutableBitSet groupKey, RexNode predicate) {
     // Consistent with the estimation of Aggregate row count in RelMdRowCount 
: distinctRowCount = rowCount * 10%.
-    return scan.getRows() * 0.1;
+    return scan.estimateRowCount(mq) * 0.1;
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java
index 0cc016b..a8649da 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java
@@ -157,7 +157,7 @@ public class DrillConstExecutor implements RexExecutor {
             .message(message)
             .build(logger);
         }
-        reducedValues.add(rexBuilder.makeNullLiteral(sqlTypeName));
+        
reducedValues.add(rexBuilder.makeNullLiteral(typeFactory.createSqlType(sqlTypeName)));
         continue;
       }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushFilterPastProjectRule.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushFilterPastProjectRule.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushFilterPastProjectRule.java
index 7b978be..c7f2838 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushFilterPastProjectRule.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushFilterPastProjectRule.java
@@ -77,7 +77,7 @@ public class DrillPushFilterPastProjectRule extends 
RelOptRule {
       }
     }
 
-    final RexNode qualifedPred 
=RexUtil.composeConjunction(filterRel.getCluster().getRexBuilder(), 
qualifiedPredList, true);
+    final RexNode qualifedPred = 
RexUtil.composeConjunction(filterRel.getCluster().getRexBuilder(), 
qualifiedPredList, true);
 
     if (qualifedPred == null) {
       return;
@@ -90,12 +90,11 @@ public class DrillPushFilterPastProjectRule extends 
RelOptRule {
     Filter newFilterRel = LogicalFilter.create(projRel.getInput(), 
newCondition);
 
     Project newProjRel =
-        (Project) RelOptUtil.createProject(
-            newFilterRel,
-            Pair.left(projRel.getNamedProjects()),
-            Pair.right(projRel.getNamedProjects()),
-            false,
-            relBuilderFactory.create(projRel.getCluster(), null));
+        (Project) relBuilderFactory
+            .create(newFilterRel.getCluster(), null)
+            .push(newFilterRel)
+            .projectNamed(Pair.left(projRel.getNamedProjects()), 
Pair.right(projRel.getNamedProjects()), true)
+            .build();
 
     final RexNode unqualifiedPred = 
RexUtil.composeConjunction(filterRel.getCluster().getRexBuilder(), 
unqualifiedPredList, true);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushLimitToScanRule.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushLimitToScanRule.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushLimitToScanRule.java
index 068252d..99d51e2 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushLimitToScanRule.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushLimitToScanRule.java
@@ -23,12 +23,6 @@ import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.rex.RexVisitor;
-import org.apache.calcite.rex.RexVisitorImpl;
-import org.apache.calcite.util.Util;
 import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.planner.common.DrillRelOptUtil;
 
@@ -100,9 +94,10 @@ public abstract class DrillPushLimitToScanRule extends 
RelOptRule {
     }
   };
 
-  protected void doOnMatch(RelOptRuleCall call, DrillLimitRel limitRel, 
DrillScanRel scanRel, DrillProjectRel projectRel){
+  protected void doOnMatch(RelOptRuleCall call, DrillLimitRel limitRel,
+      DrillScanRel scanRel, DrillProjectRel projectRel) {
     try {
-      final int rowCountRequested = (int) limitRel.getRows();
+      final int rowCountRequested = (int) 
limitRel.estimateRowCount(limitRel.getCluster().getMetadataQuery());
 
       final GroupScan newGroupScan = 
scanRel.getGroupScan().applyLimit(rowCountRequested);
 
@@ -120,10 +115,10 @@ public abstract class DrillPushLimitToScanRule extends 
RelOptRule {
 
       final RelNode newLimit;
       if (projectRel != null) {
-        final RelNode newProject = projectRel.copy(projectRel.getTraitSet(), 
ImmutableList.of((RelNode)newScanRel));
-        newLimit = limitRel.copy(limitRel.getTraitSet(), 
ImmutableList.of((RelNode)newProject));
+        final RelNode newProject = projectRel.copy(projectRel.getTraitSet(), 
ImmutableList.of((RelNode) newScanRel));
+        newLimit = limitRel.copy(limitRel.getTraitSet(), 
ImmutableList.of(newProject));
       } else {
-        newLimit = limitRel.copy(limitRel.getTraitSet(), 
ImmutableList.of((RelNode)newScanRel));
+        newLimit = limitRel.copy(limitRel.getTraitSet(), 
ImmutableList.of((RelNode) newScanRel));
       }
 
       call.transformTo(newLimit);

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceAggregatesRule.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceAggregatesRule.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceAggregatesRule.java
index 7ff286f..8ef4923 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceAggregatesRule.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceAggregatesRule.java
@@ -18,7 +18,6 @@
 
 package org.apache.drill.exec.planner.logical;
 
-
 import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -28,6 +27,8 @@ import java.util.Map;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.Window;
@@ -48,7 +49,6 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
-import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -199,26 +199,29 @@ public class DrillReduceAggregatesRule extends RelOptRule 
{
         inputExprs.size() - input.getRowType().getFieldCount();
     if (extraArgCount > 0) {
       input =
-          RelOptUtil.createProject(
-              input,
-              inputExprs,
-              CompositeList.of(
+          relBuilderFactory
+              .create(input.getCluster(), null)
+              .push(input)
+              .projectNamed(
+                  inputExprs,
+                  CompositeList.of(
                   input.getRowType().getFieldNames(),
-                  Collections.<String>nCopies(
+                  Collections.nCopies(
                       extraArgCount,
                       null)),
-              false,
-              relBuilderFactory.create(input.getCluster(), null));
+                  true)
+              .build();
     }
     Aggregate newAggRel =
         newAggregateRel(
             oldAggRel, input, newCalls);
 
     RelNode projectRel =
-        RelOptUtil.createProject(
-            newAggRel,
-            projList,
-            oldAggRel.getRowType().getFieldNames());
+        relBuilderFactory
+            .create(newAggRel.getCluster(), null)
+            .push(newAggRel)
+            .projectNamed(projList, oldAggRel.getRowType().getFieldNames(), 
true)
+            .build();
 
     ruleCall.transformTo(projectRel);
   }
@@ -317,7 +320,7 @@ public class DrillReduceAggregatesRule extends RelOptRule {
             iAvgInput);
     RelDataType sumType =
         TypeInferenceUtils.getDrillSqlReturnTypeInference(SqlKind.SUM.name(),
-            ImmutableList.<DrillFuncHolder>of())
+            ImmutableList.of())
           .inferReturnType(oldCall.createBinding(oldAggRel));
     sumType =
         typeFactory.createTypeWithNullability(
@@ -507,7 +510,7 @@ public class DrillReduceAggregatesRule extends RelOptRule {
 
     RelDataType sumType =
         TypeInferenceUtils.getDrillSqlReturnTypeInference(SqlKind.SUM.name(),
-            ImmutableList.<DrillFuncHolder>of())
+            ImmutableList.of())
           .inferReturnType(oldCall.createBinding(oldAggRel));
     sumType = typeFactory.createTypeWithNullability(sumType, true);
     final AggregateCall sumArgSquaredAggCall =
@@ -666,7 +669,9 @@ public class DrillReduceAggregatesRule extends RelOptRule {
       Aggregate oldAggRel,
       RelNode inputRel,
       List<AggregateCall> newCalls) {
-    return LogicalAggregate.create(inputRel, oldAggRel.indicator, 
oldAggRel.getGroupSet(), oldAggRel.getGroupSets(), newCalls);
+    RelOptCluster cluster = inputRel.getCluster();
+    return new LogicalAggregate(cluster, cluster.traitSetOf(Convention.NONE),
+        inputRel, oldAggRel.indicator, oldAggRel.getGroupSet(), 
oldAggRel.getGroupSets(), newCalls);
   }
 
   private RelDataType getFieldType(RelNode relNode, int i) {

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceExpressionsRule.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceExpressionsRule.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceExpressionsRule.java
index 6214912..f053f3f 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceExpressionsRule.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceExpressionsRule.java
@@ -39,7 +39,7 @@ public class DrillReduceExpressionsRule {
   private static class DrillReduceFilterRule extends 
ReduceExpressionsRule.FilterReduceExpressionsRule {
 
     DrillReduceFilterRule() {
-      super(Filter.class, DrillRelFactories.LOGICAL_BUILDER);
+      super(Filter.class, true, DrillRelFactories.LOGICAL_BUILDER);
     }
 
     /**
@@ -58,7 +58,7 @@ public class DrillReduceExpressionsRule {
   private static class DrillReduceCalcRule extends 
ReduceExpressionsRule.CalcReduceExpressionsRule {
 
     DrillReduceCalcRule() {
-      super(Calc.class, DrillRelFactories.LOGICAL_BUILDER);
+      super(Calc.class, true, DrillRelFactories.LOGICAL_BUILDER);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRel.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRel.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRel.java
index 3436aba..8b68ef3 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRel.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRel.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -20,6 +20,7 @@ package org.apache.drill.exec.planner.logical;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.calcite.rel.RelCollations;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.logical.data.LogicalOperator;
 import org.apache.drill.common.logical.data.Order;
@@ -27,7 +28,6 @@ import org.apache.drill.common.logical.data.Order.Ordering;
 import org.apache.drill.exec.planner.torel.ConversionContext;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelCollationImpl;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Sort;
@@ -93,8 +93,9 @@ public class DrillSortRel extends Sort implements DrillRel {
       String fieldName = ExprHelper.getFieldName(o.getExpr());
       int fieldId = fieldMap.get(fieldName);
       RelFieldCollation c = new RelFieldCollation(fieldId, o.getDirection(), 
o.getNullDirection());
+      collations.add(c);
     }
-    return new DrillSortRel(context.getCluster(), context.getLogicalTraits(), 
input, RelCollationImpl.of(collations));
+    return new DrillSortRel(context.getCluster(), context.getLogicalTraits(), 
input, RelCollations.of(collations));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
index 837ddd8..5f679a4 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
@@ -136,11 +136,11 @@ public abstract class PruneScanRule extends 
StoragePluginOptimizerRule {
     }
   }
 
-  public static final RelOptRule getDirFilterOnProject(OptimizerRulesContext 
optimizerRulesContext) {
+  public static RelOptRule getDirFilterOnProject(OptimizerRulesContext 
optimizerRulesContext) {
     return new DirPruneScanFilterOnProjectRule(optimizerRulesContext);
   }
 
-  public static final RelOptRule getDirFilterOnScan(OptimizerRulesContext 
optimizerRulesContext) {
+  public static RelOptRule getDirFilterOnScan(OptimizerRulesContext 
optimizerRulesContext) {
     return new DirPruneScanFilterOnScanRule(optimizerRulesContext);
   }
 
@@ -165,7 +165,7 @@ public abstract class PruneScanRule extends 
StoragePluginOptimizerRule {
       condition = filterRel.getCondition();
     } else {
       // get the filter as if it were below the projection.
-      condition = RelOptUtil.pushFilterPastProject(filterRel.getCondition(), 
projectRel);
+      condition = RelOptUtil.pushPastProject(filterRel.getCondition(), 
projectRel);
     }
 
     RewriteAsBinaryOperators visitor = new RewriteAsBinaryOperators(true, 
filterRel.getCluster().getRexBuilder());

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPrelBase.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPrelBase.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPrelBase.java
index 232473b..c84f0fc 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPrelBase.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPrelBase.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.planner.physical;
 
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.util.BitSets;
@@ -80,18 +79,9 @@ public abstract class AggPrelBase extends 
DrillAggregateRelBase implements Prel
       this.type = type;
     }
 
-    public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) 
{
-      return ImmutableList.of(type);
-    }
-
     public RelDataType getType() {
       return type;
     }
-
-    public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-      return type;
-    }
-
   }
 
   public AggPrelBase(RelOptCluster cluster,

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
index 2254c56..7618edf 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
@@ -44,10 +44,8 @@ import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.drill.exec.server.options.OptionManager;
 
-
 public class HashToRandomExchangePrel extends ExchangePrel {
 
-
   private final List<DistributionField> fields;
 
   public HashToRandomExchangePrel(RelOptCluster cluster, RelTraitSet traitSet, 
RelNode input, List<DistributionField> fields) {
@@ -145,8 +143,8 @@ public class HashToRandomExchangePrel extends ExchangePrel {
                                     new 
HashPrelUtil.RexNodeBasedHashExpressionCreatorHelper(rexBuilder);
 
       final List<RexNode> distFieldRefs = 
Lists.newArrayListWithExpectedSize(distFields.size());
-      for (int i=0; i<distFields.size(); i++) {
-        final int fieldId = distFields.get(i).getFieldId();
+      for (DistributionField distField : distFields) {
+        final int fieldId = distField.getFieldId();
         
distFieldRefs.add(rexBuilder.makeInputRef(childRowTypeFields.get(fieldId).getType(),
 fieldId));
       }
 
@@ -162,7 +160,8 @@ public class HashToRandomExchangePrel extends ExchangePrel {
       final RexNode distSeed = 
rexBuilder.makeBigintLiteral(BigDecimal.valueOf(HashPrelUtil.DIST_SEED)); // 
distribution seed
       
updatedExpr.add(HashPrelUtil.createHashBasedPartitionExpression(distFieldRefs, 
distSeed, hashHelper));
 
-      RelDataType rowType = 
RexUtil.createStructType(getCluster().getTypeFactory(), updatedExpr, 
outputFieldNames);
+      RelDataType rowType = 
RexUtil.createStructType(getCluster().getTypeFactory(),
+          updatedExpr, outputFieldNames, null);
 
       ProjectPrel addColumnprojectPrel = new ProjectPrel(child.getCluster(), 
child.getTraitSet(), child, updatedExpr, rowType);
 
@@ -180,7 +179,8 @@ public class HashToRandomExchangePrel extends ExchangePrel {
 
     if (isMuxEnabled) {
       // remove earlier inserted Project Operator - since it creates issues 
down the road in HashJoin
-      RelDataType removeRowType = 
RexUtil.createStructType(newPrel.getCluster().getTypeFactory(), 
removeUpdatedExpr, childFields);
+      RelDataType removeRowType = 
RexUtil.createStructType(newPrel.getCluster().getTypeFactory(),
+          removeUpdatedExpr, childFields, null);
 
       ProjectPrel removeColumnProjectPrel = new 
ProjectPrel(newPrel.getCluster(), newPrel.getTraitSet(),
                                                             newPrel, 
removeUpdatedExpr, removeRowType);

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrel.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrel.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrel.java
index dbacee8..e2f49c9 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrel.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrel.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -27,7 +27,6 @@ import org.apache.drill.common.logical.data.JoinCondition;
 import org.apache.drill.exec.physical.impl.join.JoinUtils;
 import org.apache.drill.exec.planner.common.DrillJoinRelBase;
 import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
-import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.plan.RelOptCluster;
@@ -47,13 +46,13 @@ import com.google.common.collect.Lists;
  * Base class for MergeJoinPrel and HashJoinPrel
  *
  */
-public abstract class JoinPrel extends DrillJoinRelBase implements Prel{
+public abstract class JoinPrel extends DrillJoinRelBase implements Prel {
   private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(JoinPrel.class);
 
   protected JoinUtils.JoinCategory joincategory;
 
   public JoinPrel(RelOptCluster cluster, RelTraitSet traits, RelNode left, 
RelNode right, RexNode condition,
-      JoinRelType joinType) throws InvalidRelException{
+      JoinRelType joinType) {
     super(cluster, traits, left, right, condition, joinType);
   }
 
@@ -68,12 +67,8 @@ public abstract class JoinPrel extends DrillJoinRelBase 
implements Prel{
   }
 
   /**
-   * Check to make sure that the fields of the inputs are the same as the 
output field names.  If not, insert a project renaming them.
-   * @param implementor
-   * @param i
-   * @param offset
-   * @param input
-   * @return
+   * Check to make sure that the fields of the inputs are the same as the 
output field names.
+   * If not, insert a project renaming them.
    */
   public RelNode getJoinInput(int offset, RelNode input) {
     assert uniqueFieldNames(input.getRowType());
@@ -99,7 +94,8 @@ public abstract class JoinPrel extends DrillJoinRelBase 
implements Prel{
       exprs.add(expr);
     }
 
-    RelDataType rowType = 
RexUtil.createStructType(input.getCluster().getTypeFactory(), exprs, 
outputFieldNames);
+    RelDataType rowType = 
RexUtil.createStructType(input.getCluster().getTypeFactory(),
+        exprs, outputFieldNames, null);
 
     ProjectPrel proj = new ProjectPrel(input.getCluster(), 
input.getTraitSet(), input, exprs, rowType);
 
@@ -116,7 +112,7 @@ public abstract class JoinPrel extends DrillJoinRelBase 
implements Prel{
    * A join condition is built only for equality and IS NOT DISTINCT FROM 
comparisons. The difference is:
    * null == null is FALSE whereas null IS NOT DISTINCT FROM null is TRUE
    * For a use case of the IS NOT DISTINCT FROM comparison, see
-   * {@link org.apache.calcite.rel.rules.RemoveDistinctAggregateRule}
+   * {@link org.apache.calcite.rel.rules.AggregateRemoveRule}
    * @param conditions populated list of join conditions
    * @param leftFields join fields from the left input
    * @param rightFields join fields from the right input

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrule.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrule.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrule.java
index 3f40720..13abc26 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrule.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrule.java
@@ -19,11 +19,11 @@ package org.apache.drill.exec.planner.physical;
 
 import java.util.List;
 
+import org.apache.calcite.rel.RelCollations;
 import org.apache.drill.exec.planner.logical.DrillJoinRel;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelCollationImpl;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.plan.RelOptRule;
@@ -40,7 +40,8 @@ public class MergeJoinPrule extends JoinPruleBase {
 
   protected static final Logger tracer = CalciteTrace.getPlannerTracer();
 
-  final boolean isDist;
+  private final boolean isDist;
+
   private MergeJoinPrule(String name, RelOptRuleOperand operand, boolean 
isDist) {
     super(operand, name);
     this.isDist = isDist;
@@ -54,7 +55,7 @@ public class MergeJoinPrule extends JoinPruleBase {
   @Override
   public void onMatch(RelOptRuleCall call) {
     PlannerSettings settings = PrelUtil.getPlannerSettings(call.getPlanner());
-    final DrillJoinRel join = (DrillJoinRel) call.rel(0);
+    final DrillJoinRel join = call.rel(0);
     final RelNode left = join.getLeft();
     final RelNode right = join.getRight();
 
@@ -82,12 +83,12 @@ public class MergeJoinPrule extends JoinPruleBase {
     }
   }
 
-  private RelCollation getCollation(List<Integer> keys){
+  private RelCollation getCollation(List<Integer> keys) {
     List<RelFieldCollation> fields = Lists.newArrayList();
     for (int key : keys) {
       fields.add(new RelFieldCollation(key));
     }
-    return RelCollationImpl.of(fields);
+    return RelCollations.of(fields);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
index 6094221..6a94662 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -59,7 +59,7 @@ public class PhysicalPlanCreator {
 
   public PhysicalOperator addMetadata(Prel originalPrel, PhysicalOperator op){
     op.setOperatorId(opIdMap.get(originalPrel).getAsSingleInt());
-    op.setCost(originalPrel.getRows());
+    
op.setCost(originalPrel.estimateRowCount(originalPrel.getCluster().getMetadataQuery()));
     return op;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java
index 4aca314..a6d9719 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -23,12 +23,12 @@ import java.util.Map;
 
 import org.apache.calcite.linq4j.Ord;
 
+import org.apache.calcite.rel.RelCollations;
 import org.apache.drill.exec.planner.logical.DrillProjectRel;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
 import 
org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
 import 
org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionType;
 import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelCollationImpl;
 import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
@@ -52,7 +52,7 @@ public class ProjectPrule extends Prule {
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final DrillProjectRel project = (DrillProjectRel) call.rel(0);
+    final DrillProjectRel project = call.rel(0);
     final RelNode input = project.getInput();
 
     RelTraitSet traits = input.getTraitSet().plus(Prel.DRILL_PHYSICAL);
@@ -126,14 +126,14 @@ public class ProjectPrule extends Prule {
     }
 
     if (newFields.isEmpty()) {
-      return RelCollationImpl.of();
+      return RelCollations.of();
     } else {
-      return RelCollationImpl.of(newFields);
+      return RelCollations.of(newFields);
     }
   }
 
   private Map<Integer, Integer> getDistributionMap(DrillProjectRel project) {
-    Map<Integer, Integer> m = new HashMap<Integer, Integer>();
+    Map<Integer, Integer> m = new HashMap<>();
 
     for (Ord<RexNode> node : Ord.zip(project.getProjects())) {
       // For distribution, either $0 or cast($0 as ...) would keep the 
distribution after projection.
@@ -151,7 +151,7 @@ public class ProjectPrule extends Prule {
   }
 
   private Map<Integer, Integer> getCollationMap(DrillProjectRel project) {
-    Map<Integer, Integer> m = new HashMap<Integer, Integer>();
+    Map<Integer, Integer> m = new HashMap<>();
 
     for (Ord<RexNode> node : Ord.zip(project.getProjects())) {
       // For collation, only $0 will keep the sort-ness after projection.

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrule.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrule.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrule.java
index 29fa750..85f516a 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrule.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrule.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.planner.physical;
 
 import java.util.List;
 
+import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.util.BitSets;
 
 import org.apache.calcite.util.ImmutableBitSet;
@@ -27,7 +28,6 @@ import org.apache.drill.exec.planner.logical.RelOptHelper;
 import org.apache.drill.exec.planner.physical.AggPrelBase.OperatorPhase;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelCollationImpl;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.plan.RelOptRule;
@@ -210,12 +210,12 @@ public class StreamAggPrule extends AggPruleBase {
   }
 
 
-  private RelCollation getCollation(DrillAggregateRel rel){
+  private RelCollation getCollation(DrillAggregateRel rel) {
 
     List<RelFieldCollation> fields = Lists.newArrayList();
     for (int group : BitSets.toIter(rel.getGroupSet())) {
       fields.add(new RelFieldCollation(group));
     }
-    return RelCollationImpl.of(fields);
+    return RelCollations.of(fields);
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WindowPrule.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WindowPrule.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WindowPrule.java
index 4fdb3b9..475acec 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WindowPrule.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WindowPrule.java
@@ -24,6 +24,7 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 
 import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.core.Window;
 import org.apache.calcite.util.BitSets;
 import org.apache.drill.exec.planner.logical.DrillRel;
@@ -31,7 +32,6 @@ import org.apache.drill.exec.planner.logical.DrillWindowRel;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
 import 
org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
 import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelCollationImpl;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.plan.RelOptRule;
@@ -115,9 +115,7 @@ public class WindowPrule extends Prule {
       }
 
       List<RelDataTypeField> newRowFields = Lists.newArrayList();
-      for(RelDataTypeField field : convertedInput.getRowType().getFieldList()) 
{
-        newRowFields.add(field);
-      }
+      newRowFields.addAll(convertedInput.getRowType().getFieldList());
 
       Iterable<RelDataTypeField> newWindowFields = 
Iterables.filter(window.getRowType().getFieldList(), new 
Predicate<RelDataTypeField>() {
             @Override
@@ -200,11 +198,9 @@ public class WindowPrule extends Prule {
       fields.add(new RelFieldCollation(group));
     }
 
-    for (RelFieldCollation field : window.orderKeys.getFieldCollations()) {
-      fields.add(field);
-    }
+    fields.addAll(window.orderKeys.getFieldCollations());
 
-    return RelCollationImpl.of(fields);
+    return RelCollations.of(fields);
   }
 
   private List<DrillDistributionTrait.DistributionField> 
getDistributionFields(Window.Group window) {

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrule.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrule.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrule.java
index d4e3d0e..959d9c9 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrule.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrule.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -17,13 +17,11 @@
  */
 package org.apache.drill.exec.planner.physical;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import org.apache.calcite.rel.RelCollation;
-import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelFieldCollation;
-import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.planner.common.DrillWriterRelBase;
 import org.apache.drill.exec.planner.logical.DrillRel;
@@ -73,7 +71,7 @@ public class WriterPrule extends Prule{
     for (int key : keys) {
       fields.add(new RelFieldCollation(key));
     }
-    return RelCollationImpl.of(fields);
+    return RelCollations.of(fields);
   }
 
   private DrillDistributionTrait getDistribution(List<Integer> keys) {

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java
index f078b6e..045dba9 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java
@@ -46,7 +46,7 @@ class NumberingRelWriter implements RelWriter {
   protected final PrintWriter pw;
   private final SqlExplainLevel detailLevel;
   protected final Spacer spacer = new Spacer();
-  private final List<Pair<String, Object>> values = new ArrayList<Pair<String, 
Object>>();
+  private final List<Pair<String, Object>> values = new ArrayList<>();
 
   private final Map<Prel, OpId> ids;
   //~ Constructors -----------------------------------------------------------
@@ -160,6 +160,7 @@ class NumberingRelWriter implements RelWriter {
     return this;
   }
 
+  @SuppressWarnings("deprecation")
   public RelWriter done(RelNode node) {
     int i = 0;
     if (values.size() > 0 && values.get(0).left.equals("subset")) {

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/ExcessiveExchangeIdentifier.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/ExcessiveExchangeIdentifier.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/ExcessiveExchangeIdentifier.java
index 2e95e7c..7bfe214 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/ExcessiveExchangeIdentifier.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/ExcessiveExchangeIdentifier.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -30,8 +30,6 @@ import org.apache.calcite.rel.RelNode;
 import com.google.common.collect.Lists;
 
 public class ExcessiveExchangeIdentifier extends BasePrelVisitor<Prel, 
ExcessiveExchangeIdentifier.MajorFragmentStat, RuntimeException> {
-  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(ExcessiveExchangeIdentifier.class);
-
   private final long targetSliceSize;
 
   public ExcessiveExchangeIdentifier(long targetSliceSize) {
@@ -62,8 +60,8 @@ public class ExcessiveExchangeIdentifier extends 
BasePrelVisitor<Prel, Excessive
   @Override
   public Prel visitScreen(ScreenPrel prel, MajorFragmentStat s) throws 
RuntimeException {
     s.addScreen(prel);
-    RelNode child = ((Prel)prel.getInput()).accept(this, s);
-    return (Prel) prel.copy(prel.getTraitSet(), 
Collections.singletonList(child));
+    RelNode child = ((Prel) prel.getInput()).accept(this, s);
+    return prel.copy(prel.getTraitSet(), Collections.singletonList(child));
   }
 
   @Override
@@ -102,7 +100,7 @@ public class ExcessiveExchangeIdentifier extends 
BasePrelVisitor<Prel, Excessive
     private boolean isMultiSubScan = false;
 
     public void add(Prel prel) {
-      maxRows = Math.max(prel.getRows(), maxRows);
+      maxRows = 
Math.max(prel.estimateRowCount(prel.getCluster().getMetadataQuery()), maxRows);
     }
 
     public void addScreen(ScreenPrel screenPrel) {

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/StarColumnConverter.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/StarColumnConverter.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/StarColumnConverter.java
index 0148d47..2ee46f0 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/StarColumnConverter.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/StarColumnConverter.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -18,7 +18,6 @@
 
 package org.apache.drill.exec.planner.physical.visitor;
 
-import java.beans.Statement;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
@@ -42,12 +41,12 @@ import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.Lists;
 
-public class StarColumnConverter extends BasePrelVisitor<Prel, Void, 
RuntimeException>{
+public class StarColumnConverter extends BasePrelVisitor<Prel, Void, 
RuntimeException> {
 
   private static final AtomicLong tableNumber = new AtomicLong(0);
 
-  private boolean prefixedForStar = false;
-  private boolean prefixedForWriter = false;
+  private boolean prefixedForStar;
+  private boolean prefixedForWriter;
 
   private StarColumnConverter() {
     prefixedForStar = false;
@@ -82,7 +81,7 @@ public class StarColumnConverter extends 
BasePrelVisitor<Prel, Void, RuntimeExce
         return insertProjUnderScreenOrWriter(prel, 
prel.getInput().getRowType(), child);
       } else {
         // Prefix is added under CTAS Writer. We need create a new Screen with 
the converted child.
-        return (Prel) prel.copy(prel.getTraitSet(), 
Collections.<RelNode>singletonList(child));
+        return prel.copy(prel.getTraitSet(), 
Collections.<RelNode>singletonList(child));
       }
     } else {
       // No prefix is
@@ -98,7 +97,7 @@ public class StarColumnConverter extends 
BasePrelVisitor<Prel, Void, RuntimeExce
     if (prefixedForStar) {
       prefixedForWriter = true;
       // return insertProjUnderScreenOrWriter(prel, 
prel.getInput().getRowType(), child);
-      return (Prel) prel.copy(prel.getTraitSet(), 
Collections.singletonList(child));
+      return prel.copy(prel.getTraitSet(), Collections.singletonList(child));
     } else {
       return prel;
     }
@@ -107,7 +106,7 @@ public class StarColumnConverter extends 
BasePrelVisitor<Prel, Void, RuntimeExce
   // insert PUS or PUW: Project Under Screen/Writer, when necessary.
   private Prel insertProjUnderScreenOrWriter(Prel prel, RelDataType 
origRowType, Prel child) {
 
-    ProjectPrel proj = null;
+    ProjectPrel proj;
     List<RelNode> children = Lists.newArrayList();
 
     List<RexNode> exprs = Lists.newArrayList();
@@ -116,7 +115,8 @@ public class StarColumnConverter extends 
BasePrelVisitor<Prel, Void, RuntimeExce
       exprs.add(expr);
     }
 
-    RelDataType newRowType = 
RexUtil.createStructType(child.getCluster().getTypeFactory(), exprs, 
origRowType.getFieldNames());
+    RelDataType newRowType = 
RexUtil.createStructType(child.getCluster().getTypeFactory(),
+        exprs, origRowType.getFieldNames(), null);
 
     int fieldCount = prel.getRowType().isStruct()? 
prel.getRowType().getFieldCount():1;
 
@@ -130,13 +130,12 @@ public class StarColumnConverter extends 
BasePrelVisitor<Prel, Void, RuntimeExce
     children.add(proj);
     return (Prel) prel.copy(prel.getTraitSet(), children);
   }
-      @Override
-  public Prel visitProject(ProjectPrel prel, Void value) throws 
RuntimeException {
-    ProjectPrel proj = (ProjectPrel) prel;
 
+  @Override
+  public Prel visitProject(ProjectPrel prel, Void value) throws 
RuntimeException {
     // Require prefix rename : there exists other expression, in addition to a 
star column.
     if (!prefixedForStar  // not set yet.
-        && 
StarColumnHelper.containsStarColumnInProject(prel.getInput().getRowType(), 
proj.getProjects())
+        && 
StarColumnHelper.containsStarColumnInProject(prel.getInput().getRowType(), 
prel.getProjects())
         && prel.getRowType().getFieldNames().size() > 1) {
       prefixedForStar = true;
     }
@@ -149,7 +148,7 @@ public class StarColumnConverter extends 
BasePrelVisitor<Prel, Void, RuntimeExce
 
     List<String> fieldNames = Lists.newArrayList();
 
-    for (Pair<String, RexNode> pair : 
Pair.zip(prel.getRowType().getFieldNames(), proj.getProjects())) {
+    for (Pair<String, RexNode> pair : 
Pair.zip(prel.getRowType().getFieldNames(), prel.getProjects())) {
       if (pair.right instanceof RexInputRef) {
         String name = child.getRowType().getFieldNames().get(((RexInputRef) 
pair.right).getIndex());
         fieldNames.add(name);
@@ -161,9 +160,10 @@ public class StarColumnConverter extends 
BasePrelVisitor<Prel, Void, RuntimeExce
     // Make sure the field names are unique : no allow of duplicate field 
names in a rowType.
     fieldNames = makeUniqueNames(fieldNames);
 
-    RelDataType rowType = 
RexUtil.createStructType(prel.getCluster().getTypeFactory(), 
proj.getProjects(), fieldNames);
+    RelDataType rowType = 
RexUtil.createStructType(prel.getCluster().getTypeFactory(),
+        prel.getProjects(), fieldNames, null);
 
-    ProjectPrel newProj = (ProjectPrel) proj.copy(proj.getTraitSet(), child, 
proj.getProjects(), rowType);
+    ProjectPrel newProj = (ProjectPrel) prel.copy(prel.getTraitSet(), child, 
prel.getProjects(), rowType);
 
     if (ProjectRemoveRule.isTrivial(newProj)) {
       return (Prel) child;
@@ -192,7 +192,7 @@ public class StarColumnConverter extends 
BasePrelVisitor<Prel, Void, RuntimeExce
 
   @Override
   public Prel visitScan(ScanPrel scanPrel, Void value) throws RuntimeException 
{
-    if (StarColumnHelper.containsStarColumn(scanPrel.getRowType()) && 
prefixedForStar ) {
+    if (StarColumnHelper.containsStarColumn(scanPrel.getRowType()) && 
prefixedForStar) {
 
       List<RexNode> exprs = Lists.newArrayList();
 
@@ -212,7 +212,8 @@ public class StarColumnConverter extends 
BasePrelVisitor<Prel, Void, RuntimeExce
           fieldNames.add(name);  // Keep regular column as it is.
         }
       }
-      RelDataType rowType = 
RexUtil.createStructType(scanPrel.getCluster().getTypeFactory(), exprs, 
fieldNames);
+      RelDataType rowType = 
RexUtil.createStructType(scanPrel.getCluster().getTypeFactory(),
+          exprs, fieldNames, null);
 
       // insert a PAS.
       ProjectPrel proj = new ProjectPrel(scanPrel.getCluster(), 
scanPrel.getTraitSet(), scanPrel, exprs, rowType);
@@ -231,8 +232,8 @@ public class StarColumnConverter extends 
BasePrelVisitor<Prel, Void, RuntimeExce
     // That means we should pick a different name that does not conflict with 
the original names, in additional
     // to make sure it's unique in the set of unique names.
 
-    HashSet<String> uniqueNames = new HashSet<String>();
-    HashSet<String> origNames = new HashSet<String>(names);
+    HashSet<String> uniqueNames = new HashSet<>();
+    HashSet<String> origNames = new HashSet<>(names);
 
     List<String> newNames = Lists.newArrayList();
 

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/SwapHashJoinVisitor.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/SwapHashJoinVisitor.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/SwapHashJoinVisitor.java
index 96c7902..d84cbb4 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/SwapHashJoinVisitor.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/SwapHashJoinVisitor.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -19,13 +19,12 @@
 package org.apache.drill.exec.planner.physical.visitor;
 
 import com.google.common.collect.Lists;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.drill.exec.planner.physical.HashJoinPrel;
 import org.apache.drill.exec.planner.physical.JoinPrel;
 import org.apache.drill.exec.planner.physical.Prel;
-import org.apache.drill.exec.planner.physical.PrelUtil;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rex.RexNode;
 
 import java.util.List;
 
@@ -38,7 +37,7 @@ import java.util.List;
  * @see org.apache.drill.exec.planner.physical.HashJoinPrel
  */
 
-public class SwapHashJoinVisitor extends BasePrelVisitor<Prel, Double, 
RuntimeException>{
+public class SwapHashJoinVisitor extends BasePrelVisitor<Prel, Double, 
RuntimeException> {
 
   private static SwapHashJoinVisitor INSTANCE = new SwapHashJoinVisitor();
 
@@ -67,9 +66,10 @@ public class SwapHashJoinVisitor extends 
BasePrelVisitor<Prel, Double, RuntimeEx
 
     if (prel instanceof HashJoinPrel) {
       // Mark left/right is swapped, when INNER hash join's left row count < ( 
1+ margin factor) right row count.
-      if (newJoin.getLeft().getRows() < (1 + value.doubleValue() ) * 
newJoin.getRight().getRows() &&
+      RelMetadataQuery mq = newJoin.getCluster().getMetadataQuery();
+      if (newJoin.getLeft().estimateRowCount(mq) < (1 + value) * 
newJoin.getRight().estimateRowCount(mq) &&
           newJoin.getJoinType() == JoinRelType.INNER) {
-        ( (HashJoinPrel) newJoin).setSwapped(true);
+        ((HashJoinPrel) newJoin).setSwapped(true);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java
index 73ff200..9d7fdce 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -40,7 +40,9 @@ public class DrillSqlAggOperator extends SqlAggFunction {
         sqlReturnTypeInference,
         null,
         Checker.getChecker(argCountMin, argCountMax),
-        SqlFunctionCategory.USER_DEFINED_FUNCTION);
+        SqlFunctionCategory.USER_DEFINED_FUNCTION,
+        false,
+        false);
     this.functions = functions;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
index 9821bf3..69362d9 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
@@ -26,7 +26,6 @@ import java.util.Set;
 import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.config.CalciteConnectionConfigImpl;
 import org.apache.calcite.config.CalciteConnectionProperty;
@@ -59,7 +58,7 @@ import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
-import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.sql.validate.SqlValidatorCatalogReader;
 import org.apache.calcite.sql.validate.SqlValidatorImpl;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
@@ -140,7 +139,7 @@ public class SqlConverter {
         session);
     this.opTab = new 
ChainedSqlOperatorTable(Arrays.asList(context.getDrillOperatorTable(), 
catalog));
     this.costFactory = (settings.useDefaultCosting()) ? null : new 
DrillCostBase.DrillCostFactory();
-    this.validator = new DrillValidator(opTab, catalog, typeFactory, 
SqlConformance.DEFAULT);
+    this.validator = new DrillValidator(opTab, catalog, typeFactory, 
SqlConformanceEnum.DEFAULT);
     validator.setIdentifierExpansion(true);
     cluster = null;
   }
@@ -160,7 +159,7 @@ public class SqlConverter {
     this.catalog = catalog;
     this.opTab = parent.opTab;
     this.planner = parent.planner;
-    this.validator = new DrillValidator(opTab, catalog, typeFactory, 
SqlConformance.DEFAULT);
+    this.validator = new DrillValidator(opTab, catalog, typeFactory, 
SqlConformanceEnum.DEFAULT);
     this.temporarySchema = parent.temporarySchema;
     this.session = parent.session;
     this.drillConfig = parent.drillConfig;
@@ -239,10 +238,9 @@ public class SqlConverter {
   }
 
   private class DrillValidator extends SqlValidatorImpl {
-    private final Set<SqlValidatorScope> identitySet = 
Sets.newIdentityHashSet();
 
     protected DrillValidator(SqlOperatorTable opTab, SqlValidatorCatalogReader 
catalogReader,
-        RelDataTypeFactory typeFactory, SqlConformance conformance) {
+        RelDataTypeFactory typeFactory, SqlConformanceEnum conformance) {
       super(opTab, catalogReader, typeFactory, conformance);
     }
 
@@ -377,9 +375,10 @@ public class SqlConverter {
     //To avoid unexpected column errors set a value of top to false
     final RelRoot rel = sqlToRelConverter.convertQuery(validatedNode, false, 
false);
     final RelRoot rel2 = rel.withRel(sqlToRelConverter.flattenTypes(rel.rel, 
true));
-    final RelRoot rel3 = 
rel2.withRel(RelDecorrelator.decorrelateQuery(rel2.rel));
+    final RelRoot rel3 = rel2.withRel(
+        RelDecorrelator.decorrelateQuery(rel2.rel,
+            sqlToRelConverterConfig.getRelBuilderFactory().create(cluster, 
null)));
     return rel3;
-
   }
 
   private class Expander implements RelOptTable.ViewExpander {

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java
index af544b5..078094b 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java
@@ -601,6 +601,7 @@ public class TypeInferenceUtils {
     private static final DrillCastSqlReturnTypeInference INSTANCE = new 
DrillCastSqlReturnTypeInference();
 
     @Override
+    @SuppressWarnings("deprecation")
     public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
       final RelDataTypeFactory factory = opBinding.getTypeFactory();
       final boolean isNullable = opBinding

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
index 58fac66..b9fe9ff 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
@@ -690,7 +690,8 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
         SqlValidatorUtil.EXPR_SUGGESTER,
         
rel.getCluster().getTypeFactory().getTypeSystem().isSchemaCaseSensitive());
 
-    RelDataType newRowType = 
RexUtil.createStructType(rel.getCluster().getTypeFactory(), projections, 
fieldNames2);
+    RelDataType newRowType = 
RexUtil.createStructType(rel.getCluster().getTypeFactory(),
+        projections, fieldNames2, null);
 
     DrillProjectRel topProj = DrillProjectRel.create(rel.getCluster(), 
rel.getTraitSet(), rel, projections, newRowType);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
index 134c28f..02f6114 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
@@ -37,7 +37,6 @@ import org.apache.drill.exec.store.AbstractSchema;
 
 import org.apache.calcite.tools.ValidationException;
 import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.type.RelDataType;
 
 import java.io.IOException;
@@ -206,8 +205,11 @@ public class SqlHandlerUtil {
             }
           };
 
-      return RelOptUtil.createProject(input, refs, names, false,
-          DrillRelFactories.LOGICAL_BUILDER.create(input.getCluster(), null));
+      return DrillRelFactories.LOGICAL_BUILDER
+          .create(input.getCluster(), null)
+          .push(input)
+          .projectNamed(refs, names, true)
+          .build();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/UnsupportedOperatorsVisitor.java
----------------------------------------------------------------------
diff --git 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/UnsupportedOperatorsVisitor.java
 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/UnsupportedOperatorsVisitor.java
index 917353e..da7b108 100644
--- 
a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/UnsupportedOperatorsVisitor.java
+++ 
b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/UnsupportedOperatorsVisitor.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -21,11 +21,11 @@ import org.apache.calcite.sql.SqlNumericLiteral;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.util.Litmus;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.UnsupportedOperatorCollector;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
-import org.apache.drill.exec.planner.sql.DrillCalciteSqlWrapper;
 import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
 
 import org.apache.calcite.sql.SqlSelectKeyword;
@@ -479,7 +479,7 @@ public class UnsupportedOperatorsVisitor extends SqlShuttle 
{
    * If the condition is true, mark flag 'find' as true.
    */
   private static class ExprFinder extends SqlBasicVisitor<Void> {
-    private boolean find = false;
+    private boolean find;
     private final SqlNodeCondition condition;
 
     public ExprFinder(SqlNodeCondition condition) {
@@ -512,15 +512,15 @@ public class UnsupportedOperatorsVisitor extends 
SqlShuttle {
    * @param sqlSelect SELECT-CLAUSE in the query
    */
   private void detectMultiplePartitions(SqlSelect sqlSelect) {
-    for(SqlNode nodeInSelectList : sqlSelect.getSelectList()) {
+    for (SqlNode nodeInSelectList : sqlSelect.getSelectList()) {
       // If the window function is used with an alias,
       // enter the first operand of AS operator
-      if(nodeInSelectList.getKind() == SqlKind.AS
+      if (nodeInSelectList.getKind() == SqlKind.AS
           && (((SqlCall) nodeInSelectList).getOperandList().get(0).getKind() 
== SqlKind.OVER)) {
         nodeInSelectList = ((SqlCall) 
nodeInSelectList).getOperandList().get(0);
       }
 
-      if(nodeInSelectList.getKind() != SqlKind.OVER) {
+      if (nodeInSelectList.getKind() != SqlKind.OVER) {
         continue;
       }
 
@@ -530,10 +530,10 @@ public class UnsupportedOperatorsVisitor extends 
SqlShuttle {
 
       // Partition window is referenced as a SqlIdentifier,
       // which is defined in the window list
-      if(window instanceof SqlIdentifier) {
+      if (window instanceof SqlIdentifier) {
         // Expand the SqlIdentifier as the expression defined in the window 
list
-        for(SqlNode sqlNode : sqlSelect.getWindowList()) {
-          if(((SqlWindow) sqlNode).getDeclName().equalsDeep(window, false)) {
+        for (SqlNode sqlNode : sqlSelect.getWindowList()) {
+          if (((SqlWindow) sqlNode).getDeclName().equalsDeep(window, 
Litmus.IGNORE)) {
             window = sqlNode;
             break;
           }
@@ -543,10 +543,10 @@ public class UnsupportedOperatorsVisitor extends 
SqlShuttle {
       }
 
       // In a SELECT-SCOPE, only a partition can be defined
-      if(definedWindow == null) {
+      if (definedWindow == null) {
         definedWindow = window;
       } else {
-        if(!definedWindow.equalsDeep(window, false)) {
+        if (!definedWindow.equalsDeep(window, Litmus.IGNORE)) {
           
unsupportedOperatorCollector.setException(SqlUnsupportedException.ExceptionType.FUNCTION,
               "Multiple window definitions in a single SELECT list is not 
currently supported \n" +
               "See Apache Drill JIRA: DRILL-3196");

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetPushDownFilter.java
----------------------------------------------------------------------
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 4257150..8d845a5 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
@@ -117,15 +117,15 @@ public abstract class ParquetPushDownFilter extends 
StoragePluginOptimizerRule {
       return;
     }
 
-    RexNode condition = null;
+    RexNode condition;
     if (project == null) {
       condition = filter.getCondition();
     } else {
       // get the filter as if it were below the projection.
-      condition = RelOptUtil.pushFilterPastProject(filter.getCondition(), 
project);
+      condition = RelOptUtil.pushPastProject(filter.getCondition(), project);
     }
 
-    if (condition == null || 
condition.equals(ValueExpressions.BooleanExpression.TRUE)) {
+    if (condition == null || condition.isAlwaysTrue()) {
       return;
     }
 
@@ -167,7 +167,7 @@ public abstract class ParquetPushDownFilter extends 
StoragePluginOptimizerRule {
       inputRel = project.copy(project.getTraitSet(), 
ImmutableList.of(inputRel));
     }
 
-    final RelNode newFilter = filter.copy(filter.getTraitSet(), 
ImmutableList.<RelNode>of(inputRel));
+    final RelNode newFilter = filter.copy(filter.getTraitSet(), 
ImmutableList.of(inputRel));
 
     call.transformTo(newFilter);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/jdbc-all/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index a10c37a..b1fe99e 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -499,7 +499,7 @@
                   This is likely due to you adding new dependencies to a 
java-exec and not updating the excludes in this module. This is important as it 
minimizes the size of the dependency of Drill application users.
 
                   </message>
-                  <maxsize>35000000</maxsize>
+                  <maxsize>36000000</maxsize>
                   <minsize>15000000</minsize>
                   <files>
                    
<file>${project.build.directory}/drill-jdbc-all-${project.version}.jar</file>

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java
----------------------------------------------------------------------
diff --git 
a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java 
b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java
index 0b33167..b08c0f5 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillMetaImpl.java
@@ -1096,6 +1096,7 @@ class DrillMetaImpl extends MetaImpl {
   }
 
   @Override
+  @SuppressWarnings("deprecation")
   public ExecuteResult prepareAndExecute(StatementHandle h, String sql, long 
maxRowCount, PrepareCallback callback) {
     final Signature signature = newSignature(sql);
     try {
@@ -1133,6 +1134,7 @@ class DrillMetaImpl extends MetaImpl {
   }
 
   @Override
+  @SuppressWarnings("deprecation")
   public ExecuteResult execute(StatementHandle statementHandle,
         List<TypedValue> list, long l) throws NoSuchStatementException {
     return new ExecuteResult(Collections.singletonList(

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
----------------------------------------------------------------------
diff --git 
a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java 
b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
index 6ca8ee2..4dfce48 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
@@ -92,18 +92,14 @@ class DrillResultSetImpl extends AvaticaResultSet 
implements DrillResultSet {
    * @throws  AlreadyClosedSqlException  if ResultSet is closed
    * @throws  SQLException  if error in calling {@link #isClosed()}
    */
-  private void throwIfClosed() throws AlreadyClosedSqlException,
-                                      ExecutionCanceledSqlException,
-                                      SQLTimeoutException,
-                                      SQLException {
-    if ( isClosed() ) {
+  private void throwIfClosed() throws SQLException {
+    if (isClosed()) {
       if (cursor instanceof DrillCursor && hasPendingCancelationNotification) {
         hasPendingCancelationNotification = false;
         throw new ExecutionCanceledSqlException(
-            "SQL statement execution canceled; ResultSet now closed." );
-      }
-      else {
-        throw new AlreadyClosedSqlException( "ResultSet is already closed." );
+            "SQL statement execution canceled; ResultSet now closed.");
+      } else {
+        throw new AlreadyClosedSqlException("ResultSet is already closed.");
       }
     }
 
@@ -1142,16 +1138,8 @@ class DrillResultSetImpl extends AvaticaResultSet 
implements DrillResultSet {
   }
 
   @Override
-  public AvaticaStatement getStatement() {
-    try {
-      throwIfClosed();
-    } catch (AlreadyClosedSqlException e) {
-      // Can't throw any SQLException because AvaticaConnection's
-      // getStatement() is missing "throws SQLException".
-      throw new RuntimeException(e.getMessage(), e);
-    } catch (SQLException e) {
-      throw new RuntimeException(e.getMessage(), e);
-    }
+  public AvaticaStatement getStatement() throws SQLException {
+    throwIfClosed();
     return super.getStatement();
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/37919697/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4b69426..5a68ffd 100644
--- a/pom.xml
+++ b/pom.xml
@@ -36,8 +36,8 @@
     <dep.guava.version>18.0</dep.guava.version>
     <forkCount>2</forkCount>
     <parquet.version>1.8.1-drill-r0</parquet.version>
-    <calcite.version>1.15.0-drill-r0</calcite.version>
-    <avatica.version>1.10.0</avatica.version>
+    <calcite.version>1.16.0-drill-r0</calcite.version>
+    <avatica.version>1.11.0</avatica.version>
     <janino.version>2.7.6</janino.version>
     <sqlline.version>1.1.9-drill-r7</sqlline.version>
     <jackson.version>2.7.9</jackson.version>

Reply via email to