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

[01/18] drill git commit: DRILL-5089: Dynamically load schema of storage plugin only when needed for every query

Repository: drill
Updated Branches:
  refs/heads/master ef0fafea2 -> 9e944c97e


DRILL-5089: Dynamically load schema of storage plugin only when needed for every query

For each query, loading all storage plugins and loading all workspaces under file system plugins is not needed.

This patch use DynamicRootSchema as the root schema for Drill. Which loads correspondent storage only when needed.

infoschema to read full schema information and load second level schema accordingly.

for workspaces under the same Filesyetm, no need to create FileSystem for each workspace.

use fs.access API to check permission which is available after HDFS 2.6 except for windows + local file system case.

Add unit tests to test with a broken mock storage: with a storage that will throw Exception in regiterSchema method,
all queries even on good storages shall fail without this fix(Drill still load all schemas from all storages).

(cherry picked from commit a66d1d7)


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

Branch: refs/heads/master
Commit: 18a71a38f6bd1fd33d21d1c68fc23c5901b0080a
Parents: 3f0e517
Author: chunhui-shi <cs...@maprtech.com>
Authored: Fri Nov 3 02:06:25 2017 +0200
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Tue Jan 16 12:10:13 2018 +0200

----------------------------------------------------------------------
 .../apache/calcite/jdbc/DynamicRootSchema.java  | 129 +++++++++++++++++++
 .../org/apache/calcite/jdbc/DynamicSchema.java  |  57 ++++++++
 .../apache/drill/exec/ops/FragmentContext.java  |   9 +-
 .../org/apache/drill/exec/ops/QueryContext.java |  11 +-
 .../drill/exec/planner/sql/SqlConverter.java    |  10 +-
 .../drill/exec/store/SchemaTreeProvider.java    |  31 ++++-
 .../exec/store/StoragePluginRegistryImpl.java   |   2 +
 .../exec/store/dfs/FileSystemSchemaFactory.java |  23 +++-
 .../exec/store/dfs/WorkspaceSchemaFactory.java  |  87 ++++++++-----
 .../store/ischema/InfoSchemaBatchCreator.java   |   2 +-
 .../exec/store/mock/MockBreakageStorage.java    |  47 +++++++
 .../exec/store/mock/MockStorageEngine.java      |   7 +-
 .../exec/work/metadata/MetadataProvider.java    |   2 +-
 .../drill/exec/physical/impl/TestSchema.java    |  87 +++++++++++++
 .../drill/test/ClusterFixtureBuilder.java       |   4 +
 .../drill/test/ClusterMockStorageFixture.java   |  51 ++++++++
 16 files changed, 512 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/18a71a38/exec/java-exec/src/main/java/org/apache/calcite/jdbc/DynamicRootSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/calcite/jdbc/DynamicRootSchema.java b/exec/java-exec/src/main/java/org/apache/calcite/jdbc/DynamicRootSchema.java
new file mode 100644
index 0000000..cde46f2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/calcite/jdbc/DynamicRootSchema.java
@@ -0,0 +1,129 @@
+/*
+ * 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.calcite.jdbc;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.DataContext;
+
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.store.SchemaConfig;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.SubSchemaWrapper;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * This class is to allow us loading schemas from storage plugins later when {@link #getSubSchema(String, boolean)}
+ * is called.
+ */
+public class DynamicRootSchema extends DynamicSchema {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DynamicRootSchema.class);
+
+  protected SchemaConfig schemaConfig;
+  protected StoragePluginRegistry storages;
+
+  public StoragePluginRegistry getSchemaFactories() {
+    return storages;
+  }
+
+  /** Creates a root schema. */
+  DynamicRootSchema(StoragePluginRegistry storages, SchemaConfig schemaConfig) {
+    super(null, new RootSchema(), "");
+    this.schemaConfig = schemaConfig;
+    this.storages = storages;
+  }
+
+  @Override
+  protected CalciteSchema getImplicitSubSchema(String schemaName,
+                                               boolean caseSensitive) {
+    CalciteSchema retSchema = getSubSchemaMap().get(schemaName);
+    if (retSchema != null) {
+      return retSchema;
+    }
+
+    loadSchemaFactory(schemaName, caseSensitive);
+    retSchema = getSubSchemaMap().get(schemaName);
+    return retSchema;
+  }
+
+  /**
+   * load schema factory(storage plugin) for schemaName
+   * @param schemaName
+   * @param caseSensitive
+   */
+  public void loadSchemaFactory(String schemaName, boolean caseSensitive) {
+    try {
+      SchemaPlus thisPlus = this.plus();
+      StoragePlugin plugin = getSchemaFactories().getPlugin(schemaName);
+      if (plugin != null) {
+        plugin.registerSchemas(schemaConfig, thisPlus);
+        return;
+      }
+
+      // Could not find the plugin of schemaName. The schemaName could be `dfs.tmp`, a 2nd level schema under 'dfs'
+      String[] paths = schemaName.split("\\.");
+      if (paths.length == 2) {
+        plugin = getSchemaFactories().getPlugin(paths[0]);
+        if (plugin == null) {
+          return;
+        }
+
+        // Found the storage plugin for first part(e.g. 'dfs') of schemaName (e.g. 'dfs.tmp')
+        // register schema for this storage plugin to 'this'.
+        plugin.registerSchemas(schemaConfig, thisPlus);
+
+        // Load second level schemas for this storage plugin
+        final SchemaPlus firstlevelSchema = thisPlus.getSubSchema(paths[0]);
+        final List<SchemaPlus> secondLevelSchemas = Lists.newArrayList();
+        for (String secondLevelSchemaName : firstlevelSchema.getSubSchemaNames()) {
+          secondLevelSchemas.add(firstlevelSchema.getSubSchema(secondLevelSchemaName));
+        }
+
+        for (SchemaPlus schema : secondLevelSchemas) {
+          org.apache.drill.exec.store.AbstractSchema drillSchema;
+          try {
+            drillSchema = schema.unwrap(org.apache.drill.exec.store.AbstractSchema.class);
+          } catch (ClassCastException e) {
+            throw new RuntimeException(String.format("Schema '%s' is not expected under root schema", schema.getName()));
+          }
+          SubSchemaWrapper wrapper = new SubSchemaWrapper(drillSchema);
+          thisPlus.add(wrapper.getName(), wrapper);
+        }
+      }
+    } catch(ExecutionSetupException | IOException ex) {
+      logger.warn("Failed to load schema for \"" + schemaName + "\"!", ex);
+    }
+  }
+
+  static class RootSchema extends AbstractSchema {
+    @Override public Expression getExpression(SchemaPlus parentSchema,
+                                              String name) {
+      return Expressions.call(
+          DataContext.ROOT,
+          BuiltInMethod.DATA_CONTEXT_GET_ROOT_SCHEMA.method);
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/drill/blob/18a71a38/exec/java-exec/src/main/java/org/apache/calcite/jdbc/DynamicSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/calcite/jdbc/DynamicSchema.java b/exec/java-exec/src/main/java/org/apache/calcite/jdbc/DynamicSchema.java
new file mode 100644
index 0000000..01c38c2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/calcite/jdbc/DynamicSchema.java
@@ -0,0 +1,57 @@
+/*
+ * 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.calcite.jdbc;
+
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.drill.exec.store.SchemaConfig;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+
+
+/**
+ * Unlike SimpleCalciteSchema, DynamicSchema could have an empty or partial schemaMap, but it could maintain a map of
+ * name->SchemaFactory, and only register schema when the corresponsdent name is requested.
+ */
+public class DynamicSchema extends SimpleCalciteSchema {
+
+  public DynamicSchema(CalciteSchema parent, Schema schema, String name) {
+    super(parent, schema, name);
+  }
+
+  @Override
+  protected CalciteSchema getImplicitSubSchema(String schemaName,
+                                               boolean caseSensitive) {
+    Schema s = schema.getSubSchema(schemaName);
+    if (s != null) {
+      return new DynamicSchema(this, s, schemaName);
+    }
+    CalciteSchema ret = getSubSchemaMap().get(schemaName);
+    return ret;
+  }
+
+  @Override
+  public SchemaPlus plus() {
+    return super.plus();
+  }
+
+  public static SchemaPlus createRootSchema(StoragePluginRegistry storages, SchemaConfig schemaConfig) {
+    DynamicRootSchema rootSchema = new DynamicRootSchema(storages, schemaConfig);
+    return rootSchema.plus();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/18a71a38/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
index 736d550..210d0d4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
@@ -230,7 +230,12 @@ public class FragmentContext extends BaseFragmentContext implements AutoCloseabl
     return context;
   }
 
-  public SchemaPlus getRootSchema() {
+  /**
+   * This method is only used to construt InfoSchemaReader, it is for the reader to get full schema, so here we
+   * are going to return a fully initialized schema tree.
+   * @return root schema's plus
+   */
+  public SchemaPlus getFullRootSchema() {
     if (queryContext == null) {
       fail(new UnsupportedOperationException("Schema tree can only be created in root fragment. " +
           "This is a non-root fragment."));
@@ -248,7 +253,7 @@ public class FragmentContext extends BaseFragmentContext implements AutoCloseabl
         .setIgnoreAuthErrors(isImpersonationEnabled)
         .build();
 
-    return queryContext.getRootSchema(schemaConfig);
+    return queryContext.getFullRootSchema(schemaConfig);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/drill/blob/18a71a38/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index 8dbddbf..eb32bc6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -163,14 +163,23 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
   }
 
   /**
-   *  Create and return a SchemaTree with given <i>schemaConfig</i>.
+   *  Create and return a SchemaTree with given <i>schemaConfig</i> but some schemas (from storage plugins)
+   *  could be initialized later.
    * @param schemaConfig
    * @return
    */
   public SchemaPlus getRootSchema(SchemaConfig schemaConfig) {
     return schemaTreeProvider.createRootSchema(schemaConfig);
   }
+  /**
+   *  Create and return a fully initialized SchemaTree with given <i>schemaConfig</i>.
+   * @param schemaConfig
+   * @return
+   */
 
+  public SchemaPlus getFullRootSchema(SchemaConfig schemaConfig) {
+    return schemaTreeProvider.createFullRootSchema(schemaConfig);
+  }
   /**
    * Get the user name of the user who issued the query that is managed by this QueryContext.
    * @return

http://git-wip-us.apache.org/repos/asf/drill/blob/18a71a38/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 466d2fe..af3c2bf 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
@@ -29,7 +29,7 @@ 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;
-import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.DynamicSchema;
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 import org.apache.calcite.plan.ConventionTraitDef;
 import org.apache.calcite.plan.RelOptCluster;
@@ -130,9 +130,9 @@ public class SqlConverter {
     this.session = context.getSession();
     this.drillConfig = context.getConfig();
     this.catalog = new DrillCalciteCatalogReader(
-        this.rootSchema,
+        rootSchema,
         parserConfig.caseSensitive(),
-        CalciteSchema.from(defaultSchema).path(null),
+        DynamicSchema.from(defaultSchema).path(null),
         typeFactory,
         drillConfig,
         session);
@@ -375,7 +375,7 @@ public class SqlConverter {
     @Override
     public RelRoot expandView(RelDataType rowType, String queryString, SchemaPlus rootSchema, List<String> schemaPath) {
       final DrillCalciteCatalogReader catalogReader = new DrillCalciteCatalogReader(
-          rootSchema, // new root schema
+          rootSchema,
           parserConfig.caseSensitive(),
           schemaPath,
           typeFactory,
@@ -555,7 +555,7 @@ public class SqlConverter {
                               JavaTypeFactory typeFactory,
                               DrillConfig drillConfig,
                               UserSession session) {
-      super(CalciteSchema.from(rootSchema), defaultSchema,
+      super(DynamicSchema.from(rootSchema), defaultSchema,
           typeFactory, getConnectionConfig(caseSensitive));
       this.drillConfig = drillConfig;
       this.session = session;

http://git-wip-us.apache.org/repos/asf/drill/blob/18a71a38/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java
index 21ab39f..0731387 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java
@@ -20,12 +20,12 @@ package org.apache.drill.exec.store;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.ViewExpansionContext;
+import org.apache.calcite.jdbc.DynamicSchema;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.server.options.OptionValue;
@@ -105,12 +105,36 @@ public class SchemaTreeProvider implements AutoCloseable {
    * @return
    */
   public SchemaPlus createRootSchema(SchemaConfig schemaConfig) {
+      final SchemaPlus rootSchema = DynamicSchema.createRootSchema(dContext.getStorage(), schemaConfig);
+      schemaTreesToClose.add(rootSchema);
+      return rootSchema;
+  }
+
+  /**
+   * Return full root schema with schema owner as the given user.
+   *
+   * @param userName Name of the user who is accessing the storage sources.
+   * @param provider {@link SchemaConfigInfoProvider} instance
+   * @return Root of the schema tree.
+   */
+  public SchemaPlus createFullRootSchema(final String userName, final SchemaConfigInfoProvider provider) {
+    final String schemaUser = isImpersonationEnabled ? userName : ImpersonationUtil.getProcessUserName();
+    final SchemaConfig schemaConfig = SchemaConfig.newBuilder(schemaUser, provider).build();
+    return createFullRootSchema(schemaConfig);
+  }
+  /**
+   * Create and return a Full SchemaTree with given <i>schemaConfig</i>.
+   * @param schemaConfig
+   * @return
+   */
+  public SchemaPlus createFullRootSchema(SchemaConfig schemaConfig) {
     try {
-      final SchemaPlus rootSchema = CalciteSchema.createRootSchema(false, false).plus();
+      final SchemaPlus rootSchema = DynamicSchema.createRootSchema(dContext.getStorage(), schemaConfig);
       dContext.getSchemaFactory().registerSchemas(schemaConfig, rootSchema);
       schemaTreesToClose.add(rootSchema);
       return rootSchema;
-    } catch(IOException e) {
+    }
+    catch(IOException e) {
       // We can't proceed further without a schema, throw a runtime exception.
       // Improve the error message for client side.
 
@@ -124,6 +148,7 @@ public class SchemaTreeProvider implements AutoCloseable {
           .addContext(contextString)
           .build(logger);
     }
+
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/18a71a38/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java
index 3fb1c3a..f2edf5e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java
@@ -494,4 +494,6 @@ public class StoragePluginRegistryImpl implements StoragePluginRegistry {
     return availablePlugins;
   }
 
+
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/18a71a38/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
----------------------------------------------------------------------
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 5d99377..6d88d04 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
@@ -27,6 +27,7 @@ import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
 
+import org.apache.drill.exec.store.StoragePlugin;
 import org.apache.drill.exec.store.StorageStrategy;
 import org.apache.drill.exec.planner.logical.CreateTableEntry;
 import org.apache.drill.exec.store.AbstractSchema;
@@ -38,7 +39,9 @@ import org.apache.drill.exec.store.dfs.WorkspaceSchemaFactory.WorkspaceSchema;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Maps;
 import org.apache.drill.exec.util.DrillFileSystemUtil;
+import org.apache.drill.exec.util.ImpersonationUtil;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 
@@ -49,11 +52,23 @@ public class FileSystemSchemaFactory implements SchemaFactory{
 
   public static final String DEFAULT_WS_NAME = "default";
 
+  public static final String LOCAL_FS_SCHEME = "file";
+
   private List<WorkspaceSchemaFactory> factories;
   private String schemaName;
+  protected FileSystemPlugin plugin;
 
   public FileSystemSchemaFactory(String schemaName, List<WorkspaceSchemaFactory> factories) {
-    super();
+    // when the correspondent FileSystemPlugin is not passed in, we dig into ANY workspace factory to get it.
+    if (factories.size() > 0) {
+      this.plugin = factories.get(0).getPlugin();
+    }
+    this.schemaName = schemaName;
+    this.factories = factories;
+  }
+
+  public FileSystemSchemaFactory(FileSystemPlugin plugin, String schemaName, List<WorkspaceSchemaFactory> factories) {
+    this.plugin = plugin;
     this.schemaName = schemaName;
     this.factories = factories;
   }
@@ -73,10 +88,10 @@ public class FileSystemSchemaFactory implements SchemaFactory{
 
     public FileSystemSchema(String name, SchemaConfig schemaConfig) throws IOException {
       super(ImmutableList.<String>of(), name);
+      final DrillFileSystem fs = ImpersonationUtil.createFileSystem(schemaConfig.getUserName(), plugin.getFsConf());
       for(WorkspaceSchemaFactory f :  factories){
-        if (f.accessible(schemaConfig.getUserName())) {
-          @SuppressWarnings("resource")
-          WorkspaceSchema s = f.createSchema(getSchemaPath(), schemaConfig);
+        WorkspaceSchema s = f.createSchema(getSchemaPath(), schemaConfig, fs);
+        if (s != null) {
           schemaMap.put(s.getName(), s);
         }
       }

http://git-wip-us.apache.org/repos/asf/drill/blob/18a71a38/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
index bbf013d..a3886bb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
@@ -42,6 +42,7 @@ import org.apache.calcite.schema.FunctionParameter;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.TableMacro;
 import org.apache.calcite.schema.TranslatableTable;
+import org.apache.commons.lang3.SystemUtils;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.common.config.LogicalPlanPersistence;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -70,6 +71,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.AccessControlException;
 
@@ -150,14 +152,30 @@ public class WorkspaceSchemaFactory {
    * @return True if the user has access. False otherwise.
    */
   public boolean accessible(final String userName) throws IOException {
-    final FileSystem fs = ImpersonationUtil.createFileSystem(userName, fsConf);
+    final DrillFileSystem fs = ImpersonationUtil.createFileSystem(userName, fsConf);
+    return accessible(fs);
+  }
+
+  /**
+   * Checks whether a FileSystem object has the permission to list/read workspace directory
+   * @param fs a DrillFileSystem object that was created with certain user privilege
+   * @return True if the user has access. False otherwise.
+   * @throws IOException
+   */
+  public boolean accessible(DrillFileSystem fs) throws IOException {
     try {
-      // We have to rely on the listStatus as a FileSystem can have complicated controls such as regular unix style
-      // permissions, Access Control Lists (ACLs) or Access Control Expressions (ACE). Hadoop 2.7 version of FileSystem
-      // has a limited private API (FileSystem.access) to check the permissions directly
-      // (see https://issues.apache.org/jira/browse/HDFS-6570). Drill currently relies on Hadoop 2.5.0 version of
-      // FileClient. TODO: Update this when DRILL-3749 is fixed.
-      fs.listStatus(wsPath);
+      /**
+       * For Windows local file system, fs.access ends up using DeprecatedRawLocalFileStatus which has
+       * TrustedInstaller as owner, and a member of Administrators group could not satisfy the permission.
+       * In this case, we will still use method listStatus.
+       * In other cases, we use access method since it is cheaper.
+       */
+      if (SystemUtils.IS_OS_WINDOWS && fs.getUri().getScheme().equalsIgnoreCase(FileSystemSchemaFactory.LOCAL_FS_SCHEME)) {
+        fs.listStatus(wsPath);
+      }
+      else {
+        fs.access(wsPath, FsAction.READ);
+      }
     } catch (final UnsupportedOperationException e) {
       logger.trace("The filesystem for this workspace does not support this operation.", e);
     } catch (final FileNotFoundException | AccessControlException e) {
@@ -171,8 +189,19 @@ public class WorkspaceSchemaFactory {
     return DotDrillType.VIEW.getPath(config.getLocation(), name);
   }
 
-  public WorkspaceSchema createSchema(List<String> parentSchemaPath, SchemaConfig schemaConfig) throws IOException {
-    return new WorkspaceSchema(parentSchemaPath, schemaName, schemaConfig);
+  public WorkspaceSchema createSchema(List<String> parentSchemaPath, SchemaConfig schemaConfig, DrillFileSystem fs) throws IOException {
+    if (!accessible(fs)) {
+      return null;
+    }
+    return new WorkspaceSchema(parentSchemaPath, schemaName, schemaConfig, fs);
+  }
+
+  public String getSchemaName() {
+    return schemaName;
+  }
+
+  public FileSystemPlugin getPlugin() {
+    return plugin;
   }
 
   /**
@@ -380,12 +409,12 @@ public class WorkspaceSchemaFactory {
   public class WorkspaceSchema extends AbstractSchema implements ExpandingConcurrentMap.MapValueFactory<TableInstance, DrillTable> {
     private final ExpandingConcurrentMap<TableInstance, DrillTable> tables = new ExpandingConcurrentMap<>(this);
     private final SchemaConfig schemaConfig;
-    private final DrillFileSystem fs;
+    private DrillFileSystem fs;
 
-    public WorkspaceSchema(List<String> parentSchemaPath, String wsName, SchemaConfig schemaConfig) throws IOException {
+    public WorkspaceSchema(List<String> parentSchemaPath, String wsName, SchemaConfig schemaConfig, DrillFileSystem fs) throws IOException {
       super(parentSchemaPath, wsName);
       this.schemaConfig = schemaConfig;
-      this.fs = ImpersonationUtil.createFileSystem(schemaConfig.getUserName(), fsConf);
+      this.fs = fs;
     }
 
     DrillTable getDrillTable(TableInstance key) {
@@ -395,10 +424,10 @@ public class WorkspaceSchemaFactory {
     @Override
     public boolean createView(View view) throws IOException {
       Path viewPath = getViewPath(view.getName());
-      boolean replaced = fs.exists(viewPath);
+      boolean replaced = getFS().exists(viewPath);
       final FsPermission viewPerms =
           new FsPermission(schemaConfig.getOption(ExecConstants.NEW_VIEW_DEFAULT_PERMS_KEY).string_val);
-      try (OutputStream stream = DrillFileSystem.create(fs, viewPath, viewPerms)) {
+      try (OutputStream stream = DrillFileSystem.create(getFS(), viewPath, viewPerms)) {
         mapper.writeValue(stream, view);
       }
       return replaced;
@@ -421,7 +450,7 @@ public class WorkspaceSchemaFactory {
 
     @Override
     public void dropView(String viewName) throws IOException {
-      fs.delete(getViewPath(viewName), false);
+      getFS().delete(getViewPath(viewName), false);
     }
 
     private Set<String> getViews() {
@@ -429,7 +458,7 @@ public class WorkspaceSchemaFactory {
       // Look for files with ".view.drill" extension.
       List<DotDrillFile> files;
       try {
-        files = DotDrillUtil.getDotDrills(fs, new Path(config.getLocation()), DotDrillType.VIEW);
+        files = DotDrillUtil.getDotDrills(getFS(), new Path(config.getLocation()), DotDrillType.VIEW);
         for (DotDrillFile f : files) {
           viewSet.add(f.getBaseName());
         }
@@ -498,7 +527,7 @@ public class WorkspaceSchemaFactory {
       List<DotDrillFile> files = Collections.emptyList();
       try {
         try {
-          files = DotDrillUtil.getDotDrills(fs, new Path(config.getLocation()), tableName, DotDrillType.VIEW);
+          files = DotDrillUtil.getDotDrills(getFS(), new Path(config.getLocation()), tableName, DotDrillType.VIEW);
         } catch (AccessControlException e) {
           if (!schemaConfig.getIgnoreAuthErrors()) {
             logger.debug(e.getMessage());
@@ -570,18 +599,18 @@ public class WorkspaceSchemaFactory {
     }
 
     private DrillTable isReadable(FormatMatcher m, FileSelection fileSelection) throws IOException {
-      return m.isReadable(fs, fileSelection, plugin, storageEngineName, schemaConfig.getUserName());
+      return m.isReadable(getFS(), fileSelection, plugin, storageEngineName, schemaConfig.getUserName());
     }
 
     @Override
     public DrillTable create(TableInstance key) {
       try {
-        final FileSelection fileSelection = FileSelection.create(fs, config.getLocation(), key.sig.name, config.allowAccessOutsideWorkspace());
+        final FileSelection fileSelection = FileSelection.create(getFS(), config.getLocation(), key.sig.name, config.allowAccessOutsideWorkspace());
         if (fileSelection == null) {
           return null;
         }
 
-        final boolean hasDirectories = fileSelection.containsDirectories(fs);
+        final boolean hasDirectories = fileSelection.containsDirectories(getFS());
         if (key.sig.params.size() > 0) {
           FormatPluginConfig fconfig = optionExtractor.createConfigForTable(key);
           return new DynamicDrillTable(
@@ -591,7 +620,7 @@ public class WorkspaceSchemaFactory {
         if (hasDirectories) {
           for (final FormatMatcher matcher : dirMatchers) {
             try {
-              DrillTable table = matcher.isReadable(fs, fileSelection, plugin, storageEngineName, schemaConfig.getUserName());
+              DrillTable table = matcher.isReadable(getFS(), fileSelection, plugin, storageEngineName, schemaConfig.getUserName());
               if (table != null) {
                 return table;
               }
@@ -601,13 +630,13 @@ public class WorkspaceSchemaFactory {
           }
         }
 
-        final FileSelection newSelection = hasDirectories ? fileSelection.minusDirectories(fs) : fileSelection;
+        final FileSelection newSelection = hasDirectories ? fileSelection.minusDirectories(getFS()) : fileSelection;
         if (newSelection == null) {
           return null;
         }
 
         for (final FormatMatcher matcher : fileMatchers) {
-          DrillTable table = matcher.isReadable(fs, newSelection, plugin, storageEngineName, schemaConfig.getUserName());
+          DrillTable table = matcher.isReadable(getFS(), newSelection, plugin, storageEngineName, schemaConfig.getUserName());
           if (table != null) {
             return table;
           }
@@ -632,7 +661,7 @@ public class WorkspaceSchemaFactory {
       FormatMatcher matcher = null;
       try {
         for (FormatMatcher m : dropFileMatchers) {
-          if (m.isFileReadable(fs, file)) {
+          if (m.isFileReadable(getFS(), file)) {
             return m;
           }
         }
@@ -655,7 +684,7 @@ public class WorkspaceSchemaFactory {
      * @throws IOException is case of problems accessing table files
      */
     private boolean isHomogeneous(String tableName) throws IOException {
-      FileSelection fileSelection = FileSelection.create(fs, config.getLocation(), tableName, config.allowAccessOutsideWorkspace());
+      FileSelection fileSelection = FileSelection.create(getFS(), config.getLocation(), tableName, config.allowAccessOutsideWorkspace());
 
       if (fileSelection == null) {
         throw UserException
@@ -666,15 +695,15 @@ public class WorkspaceSchemaFactory {
 
       FormatMatcher matcher = null;
       Queue<FileStatus> listOfFiles = new LinkedList<>();
-      listOfFiles.addAll(fileSelection.getStatuses(fs));
+      listOfFiles.addAll(fileSelection.getStatuses(getFS()));
 
       while (!listOfFiles.isEmpty()) {
         FileStatus currentFile = listOfFiles.poll();
         if (currentFile.isDirectory()) {
-          listOfFiles.addAll(DrillFileSystemUtil.listFiles(fs, currentFile.getPath(), true));
+          listOfFiles.addAll(DrillFileSystemUtil.listFiles(getFS(), currentFile.getPath(), true));
         } else {
           if (matcher != null) {
-            if (!matcher.isFileReadable(fs, currentFile)) {
+            if (!matcher.isFileReadable(getFS(), currentFile)) {
               return false;
             }
           } else {
@@ -763,7 +792,7 @@ public class WorkspaceSchemaFactory {
       // Then look for files that start with this name and end in .drill.
       List<DotDrillFile> files = Collections.emptyList();
       try {
-        files = DotDrillUtil.getDotDrills(fs, new Path(config.getLocation()), DotDrillType.VIEW);
+        files = DotDrillUtil.getDotDrills(getFS(), new Path(config.getLocation()), DotDrillType.VIEW);
       } catch (AccessControlException e) {
         if (!schemaConfig.getIgnoreAuthErrors()) {
           logger.debug(e.getMessage());

http://git-wip-us.apache.org/repos/asf/drill/blob/18a71a38/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaBatchCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaBatchCreator.java
index 60581a7..ce05543 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaBatchCreator.java
@@ -33,7 +33,7 @@ public class InfoSchemaBatchCreator implements BatchCreator<InfoSchemaSubScan>{
   @Override
   public ScanBatch getBatch(FragmentContext context, InfoSchemaSubScan config, List<RecordBatch> children)
       throws ExecutionSetupException {
-    RecordReader rr = config.getTable().getRecordReader(context.getRootSchema(), config.getFilter(), context.getOptions());
+    RecordReader rr = config.getTable().getRecordReader(context.getFullRootSchema(), config.getFilter(), context.getOptions());
     return new ScanBatch(config, context, Collections.singletonList(rr));
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/18a71a38/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockBreakageStorage.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockBreakageStorage.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockBreakageStorage.java
new file mode 100644
index 0000000..f2c2d9f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockBreakageStorage.java
@@ -0,0 +1,47 @@
+/*
+ * 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.mock;
+
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.SchemaConfig;
+
+import java.io.IOException;
+
+public class MockBreakageStorage extends MockStorageEngine {
+
+  private boolean breakRegister;
+
+  public MockBreakageStorage(MockStorageEngineConfig configuration, DrillbitContext context, String name) {
+    super(configuration, context, name);
+    breakRegister = false;
+  }
+
+  public void setBreakRegister(boolean breakRegister) {
+    this.breakRegister = breakRegister;
+  }
+
+  @Override
+  public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException {
+    if (breakRegister) {
+      throw new IOException("mock breakRegister!");
+    }
+    super.registerSchemas(schemaConfig, parent);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/18a71a38/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
index 0edf65f..8dd4e2e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
@@ -57,7 +57,7 @@ public class MockStorageEngine extends AbstractStoragePlugin {
 
   public MockStorageEngine(MockStorageEngineConfig configuration, DrillbitContext context, String name) {
     this.configuration = configuration;
-    this.schema = new MockSchema(this);
+    this.schema = new MockSchema(this, name);
   }
 
   @Override
@@ -123,6 +123,11 @@ public class MockStorageEngine extends AbstractStoragePlugin {
       this.engine = engine;
     }
 
+    public MockSchema(MockStorageEngine engine, String name) {
+      super(ImmutableList.<String>of(), name);
+      this.engine = engine;
+    }
+
     @Override
     public Table getTable(String name) {
       Table table = tableCache.get(name);

http://git-wip-us.apache.org/repos/asf/drill/blob/18a71a38/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java
index cf64b20..f26848d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java
@@ -543,7 +543,7 @@ public class MetadataProvider {
   private static <S> PojoRecordReader<S> getPojoRecordReader(final InfoSchemaTableType tableType, final InfoSchemaFilter filter, final DrillConfig config,
       final SchemaTreeProvider provider, final UserSession userSession) {
     final SchemaPlus rootSchema =
-        provider.createRootSchema(userSession.getCredentials().getUserName(), newSchemaConfigInfoProvider(config, userSession, provider));
+        provider.createFullRootSchema(userSession.getCredentials().getUserName(), newSchemaConfigInfoProvider(config, userSession, provider));
     return tableType.getRecordReader(rootSchema, filter, userSession.getOptions());
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/18a71a38/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSchema.java
new file mode 100644
index 0000000..9282eed
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSchema.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.physical.impl;
+
+import org.apache.drill.test.BaseDirTestWatcher;
+import org.apache.drill.test.ClientFixture;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterMockStorageFixture;
+import org.apache.drill.test.DrillTest;
+
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+
+public class TestSchema extends DrillTest {
+
+  @ClassRule
+  public static final BaseDirTestWatcher dirTestWatcher = new BaseDirTestWatcher();
+
+  private static ClusterMockStorageFixture cluster;
+  private static ClientFixture client;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    cluster = ClusterFixture.builder(dirTestWatcher).buildCustomMockStorage();
+    boolean breakRegisterSchema = true;
+    // With a broken storage which will throw exception in regiterSchema, every query (even on other storage)
+    // shall fail if Drill is still loading all schemas (include the broken schema) before a query.
+    cluster.insertMockStorage("mock_broken", breakRegisterSchema);
+    cluster.insertMockStorage("mock_good", !breakRegisterSchema);
+    client = cluster.clientFixture();
+  }
+
+  @Test (expected = Exception.class)
+  public void testQueryBrokenStorage() throws Exception {
+    String sql = "SELECT id_i, name_s10 FROM `mock_broken`.`employees_5`";
+    try {
+      client.queryBuilder().sql(sql).run();
+    } catch (Exception ex) {
+      assertTrue(ex.getMessage().contains("VALIDATION ERROR: Schema"));
+      throw ex;
+    }
+  }
+
+  @Test
+  public void testQueryGoodStorage() throws Exception {
+    String sql = "SELECT id_i, name_s10 FROM `mock_good`.`employees_5`";
+    client.queryBuilder().sql(sql).run();
+  }
+
+  @Test
+  public void testQueryGoodStorageWithDefaultSchema() throws Exception {
+    String use_dfs = "use dfs.tmp";
+    client.queryBuilder().sql(use_dfs).run();
+    String sql = "SELECT id_i, name_s10 FROM `mock_good`.`employees_5`";
+    client.queryBuilder().sql(sql).run();
+  }
+
+  @Test (expected = Exception.class)
+  public void testUseBrokenStorage() throws Exception {
+    try {
+      String use_dfs = "use mock_broken";
+      client.queryBuilder().sql(use_dfs).run();
+    } catch(Exception ex) {
+      assertTrue(ex.getMessage().contains("VALIDATION ERROR: Schema"));
+      throw ex;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/18a71a38/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixtureBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixtureBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixtureBuilder.java
index 82bcf75..dfd63de 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixtureBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixtureBuilder.java
@@ -282,4 +282,8 @@ public class ClusterFixtureBuilder {
   public ClusterFixture build() {
     return new ClusterFixture(this);
   }
+
+  public ClusterMockStorageFixture buildCustomMockStorage() {
+    return new ClusterMockStorageFixture(this);
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/18a71a38/exec/java-exec/src/test/java/org/apache/drill/test/ClusterMockStorageFixture.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterMockStorageFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterMockStorageFixture.java
new file mode 100644
index 0000000..54d7bf0
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterMockStorageFixture.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.test;
+
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.StoragePluginRegistryImpl;
+import org.apache.drill.exec.store.mock.MockBreakageStorage;
+import org.apache.drill.exec.store.mock.MockStorageEngineConfig;
+
+public class ClusterMockStorageFixture extends ClusterFixture {
+  ClusterMockStorageFixture(ClusterFixtureBuilder builder) {
+    super(builder);
+
+  }
+
+  /**
+   * This should be called after bits are started
+   * @param name nthe mock storage name we are going to create
+   */
+  public void insertMockStorage(String name, boolean breakRegisterSchema) {
+    for (Drillbit bit : drillbits()) {
+
+      // Bit name and registration.
+      final StoragePluginRegistry pluginRegistry = bit.getContext().getStorage();
+      MockStorageEngineConfig config = MockStorageEngineConfig.INSTANCE;
+      @SuppressWarnings("resource")
+      MockBreakageStorage plugin = new MockBreakageStorage(
+          MockStorageEngineConfig.INSTANCE, bit.getContext(), name);
+      ((StoragePluginRegistryImpl) pluginRegistry).definePlugin(name, config, plugin);
+
+      plugin.setBreakRegister(breakRegisterSchema);
+    }
+  }
+
+}


[13/18] drill git commit: DRILL-3993: Changes to support Calcite 1.13

Posted by am...@apache.org.
DRILL-3993: Changes to support Calcite 1.13

- fixed all compiling errors (main changes were: Maven changes, chenges RelNode -> RelRoot, implementing some new methods from updated interfaces, chenges some literals, logger changes);
- fixed unexpected column errors, validation errors and assertion errors after Calcite update;
- fixed describe table/schema statement according to updated logic;
- added fixes with time-intervals;
- changed precision of BINARY to 65536 (was 1048576) according to updated logic (Calcite overrides bigger precision to own maxPrecision);
- ignored some incorrect tests with DRILL-3244;
- changed "Table not found" message to "Object not found within" according to new Calcite changes.


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

Branch: refs/heads/master
Commit: 9fabb612f16f6f541b3bde68ad7d734cad26df33
Parents: 450e670
Author: Roman Kulyk <ro...@gmail.com>
Authored: Tue Aug 29 14:10:24 2017 +0000
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Tue Jan 16 12:10:13 2018 +0200

----------------------------------------------------------------------
 contrib/storage-hive/hive-exec-shade/pom.xml    |   6 +-
 .../exec/store/jdbc/JdbcExpressionCheck.java    |  20 +++-
 .../apache/drill/exec/store/jdbc/JdbcPrel.java  |   2 +-
 exec/java-exec/pom.xml                          |   4 +
 exec/java-exec/src/main/codegen/data/Parser.tdd |  23 ++++-
 .../src/main/codegen/includes/parserImpls.ftl   |  25 ++++-
 .../org/apache/drill/exec/dotdrill/View.java    |  15 ++-
 .../planner/FileSystemPartitionDescriptor.java  |   6 +-
 .../exec/planner/common/DrillStoreRelBase.java  |   9 +-
 .../exec/planner/common/DrillValuesRelBase.java |  13 ++-
 .../planner/logical/DrillAggregateRule.java     |  10 +-
 .../exec/planner/logical/DrillConditions.java   |   8 +-
 .../planner/logical/DrillConstExecutor.java     |  14 ++-
 .../exec/planner/logical/DrillJoinRule.java     |  12 +--
 .../drill/exec/planner/logical/DrillOptiq.java  |  32 +++++-
 .../logical/DrillProjectSetOpTransposeRule.java |   5 +-
 .../logical/DrillPushProjectPastFilterRule.java |   7 +-
 .../logical/DrillPushProjectPastJoinRule.java   |   5 +-
 .../logical/DrillReduceAggregatesRule.java      |   7 +-
 .../logical/DrillReduceExpressionsRule.java     |  25 ++---
 .../exec/planner/logical/DrillRelFactories.java |  11 +-
 .../exec/planner/logical/DrillStoreRel.java     |   9 +-
 .../exec/planner/logical/DrillUnionAllRule.java |   8 +-
 .../exec/planner/logical/DrillViewTable.java    |   9 +-
 .../exec/planner/physical/HashAggPrel.java      |   5 -
 .../exec/planner/physical/HashAggPrule.java     |   7 +-
 .../exec/planner/physical/HashJoinPrel.java     |   1 -
 .../exec/planner/physical/HashJoinPrule.java    |   9 +-
 .../exec/planner/physical/MergeJoinPrule.java   |   6 +-
 .../planner/physical/NestedLoopJoinPrule.java   |   6 +-
 .../exec/planner/physical/StreamAggPrule.java   |   6 +-
 .../exec/planner/physical/UnionAllPrule.java    |  13 ++-
 .../planner/physical/UnionDistinctPrule.java    |  14 +--
 .../exec/planner/physical/WindowPrule.java      |   8 +-
 .../physical/visitor/TopProjectVisitor.java     |   2 +-
 .../sql/DrillCalciteSqlAggFunctionWrapper.java  |   5 +-
 .../sql/DrillCalciteSqlFunctionWrapper.java     |   5 +-
 .../sql/DrillCalciteSqlOperatorWrapper.java     |   5 +-
 .../exec/planner/sql/DrillParserConfig.java     |  12 +++
 .../drill/exec/planner/sql/DrillSqlWorker.java  |  15 +++
 .../drill/exec/planner/sql/SqlConverter.java    |  41 ++++----
 .../exec/planner/sql/TypeInferenceUtils.java    |  20 +++-
 .../planner/sql/handlers/DefaultSqlHandler.java |  29 ++++--
 .../sql/handlers/DescribeSchemaHandler.java     |   2 +-
 .../sql/handlers/DescribeTableHandler.java      |   5 +-
 .../planner/sql/handlers/FindLimit0Visitor.java |  29 +++++-
 .../sql/parser/CompoundIdentifierConverter.java |   3 +-
 .../sql/parser/DrillSqlDescribeTable.java       |  73 +++++++++++++
 .../planner/sql/parser/SqlDescribeSchema.java   |  81 ---------------
 .../planner/sql/parser/SqlDescribeTable.java    | 102 -------------------
 .../exec/planner/torel/ConversionContext.java   |   7 +-
 .../planner/types/RelDataTypeDrillImpl.java     |   6 +-
 .../exec/planner/types/RelDataTypeHolder.java   |  12 ++-
 .../apache/drill/exec/store/AbstractSchema.java |   5 +
 .../drill/exec/store/SchemaTreeProvider.java    |   4 +-
 .../drill/exec/store/ischema/Records.java       |  41 +++++++-
 .../java/org/apache/drill/PlanningBase.java     |   4 +-
 .../exec/planner/logical/DrillOptiqTest.java    |   2 +-
 .../exec/sql/TestSqlBracketlessSyntax.java      |   4 +-
 .../store/parquet/TestParquetGroupScan.java     |   4 +-
 exec/jdbc-all/pom.xml                           |   4 +-
 exec/jdbc/pom.xml                               |   5 +-
 .../drill/jdbc/impl/DrillConnectionImpl.java    |  12 ++-
 .../drill/jdbc/impl/DrillJdbc41Factory.java     |   6 +-
 .../apache/drill/jdbc/impl/DrillMetaImpl.java   |  73 +++++++++++--
 .../jdbc/impl/DrillPreparedStatementImpl.java   |  18 ++--
 .../drill/jdbc/impl/DrillResultSetImpl.java     |   7 +-
 .../jdbc/DatabaseMetaDataGetColumnsTest.java    |  43 +++++---
 .../jdbc/test/TestInformationSchemaColumns.java |  40 ++++----
 pom.xml                                         |  28 ++++-
 tools/fmpp/pom.xml                              |  14 ++-
 71 files changed, 692 insertions(+), 436 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/contrib/storage-hive/hive-exec-shade/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/hive-exec-shade/pom.xml b/contrib/storage-hive/hive-exec-shade/pom.xml
index 30472e5..de9e055 100644
--- a/contrib/storage-hive/hive-exec-shade/pom.xml
+++ b/contrib/storage-hive/hive-exec-shade/pom.xml
@@ -42,6 +42,10 @@
           <groupId>commons-codec</groupId>
           <artifactId>commons-codec</artifactId>
         </exclusion>
+        <exclusion>
+          <artifactId>calcite-avatica</artifactId>
+          <groupId>org.apache.calcite</groupId>
+        </exclusion>
       </exclusions>
     </dependency>
   </dependencies>
@@ -71,7 +75,7 @@
                   <include>com.twitter:parquet-encoding</include>
                   <include>com.twitter:parquet-generator</include>
                   <include>org.apache.calcite:calcite-core</include>
-                  <include>org.apache.calcite:calcite-avatica</include>
+                  <include>org.apache.calcite.avatica:avatica-core</include>
                 </includes>
               </artifactSet>
               <createDependencyReducedPom>false</createDependencyReducedPom>

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcExpressionCheck.java
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcExpressionCheck.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcExpressionCheck.java
index 2015a77..e0b01b3 100644
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcExpressionCheck.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcExpressionCheck.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,10 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexLocalRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexPatternFieldRef;
 import org.apache.calcite.rex.RexRangeRef;
+import org.apache.calcite.rex.RexSubQuery;
+import org.apache.calcite.rex.RexTableInputRef;
 import org.apache.calcite.rex.RexVisitor;
 import org.apache.calcite.rex.RexWindow;
 import org.apache.drill.exec.planner.sql.DrillSqlOperator;
@@ -116,4 +119,19 @@ class JdbcExpressionCheck implements RexVisitor<Boolean> {
     return paramRexFieldAccess.getReferenceExpr().accept(this);
   }
 
+  @Override
+  public Boolean visitSubQuery(RexSubQuery subQuery) {
+    return null;
+  }
+
+  @Override
+  public Boolean visitTableInputRef(RexTableInputRef fieldRef) {
+    return false;
+  }
+
+  @Override
+  public Boolean visitPatternFieldRef(RexPatternFieldRef fieldRef) {
+    return false;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/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 58549db..e335c08 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
@@ -62,7 +62,7 @@ public class JdbcPrel extends AbstractRelNode implements Prel {
         (JavaTypeFactory) getCluster().getTypeFactory());
     final JdbcImplementor.Result result =
         jdbcImplementor.visitChild(0, input.accept(new SubsetRemover()));
-    sql = result.asQuery().toSqlString(dialect).getSql();
+    sql = result.asSelect().toSqlString(dialect).getSql();
     rowType = input.getRowType();
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 728477b..8e64224 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -212,6 +212,10 @@
       <artifactId>calcite-core</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.calcite.avatica</groupId>
+      <artifactId>avatica</artifactId>
+    </dependency>
+    <dependency>
       <groupId>net.sf.jpam</groupId>
       <artifactId>jpam</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/codegen/data/Parser.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/Parser.tdd b/exec/java-exec/src/main/codegen/data/Parser.tdd
index 6c23808..507d692 100644
--- a/exec/java-exec/src/main/codegen/data/Parser.tdd
+++ b/exec/java-exec/src/main/codegen/data/Parser.tdd
@@ -29,15 +29,12 @@
   # List of keywords.
   keywords: [
     "DATABASES",
-    "REPLACE",
     "SCHEMAS",
-    "SHOW",
     "TABLES",
     "USE",
     "FILES",
     "REFRESH",
     "METADATA",
-    "DATABASE",
     "IF",
     "JAR"
   ]
@@ -75,6 +72,26 @@
   implementationFiles: [
     "parserImpls.ftl"
   ]
+
+  # List of methods for parsing extensions to "CREATE [OR REPLACE]" calls.
+  # Each must accept arguments "(SqlParserPos pos, boolean replace)".
+  createStatementParserMethods: [
+  ]
+
+  # List of methods for parsing extensions to "ALTER <scope>" calls.
+  # Each must accept arguments "(SqlParserPos pos, String scope)".
+  alterStatementParserMethods: [
+  ]
+
+  # List of methods for parsing extensions to "DROP" calls.
+  # Each must accept arguments "(SqlParserPos pos)".
+    dropStatementParserMethods: [
+  ]
+
+  # List of keywords from "keywords" section that are not reserved.
+    nonReservedKeywords: [
+  ]
+
   
   includeCompoundIdentifier: false,
   includeBraces: true,

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
index 67d1501..502c5b3 100644
--- a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
+++ b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
@@ -118,7 +118,7 @@ SqlNode SqlDescribeTable() :
         E()
     )
     {
-        return new SqlDescribeTable(pos, table, column, columnPattern);
+        return new DrillSqlDescribeTable(pos, table, column, columnPattern);
     }
 }
 
@@ -158,7 +158,7 @@ SqlNodeList ParseRequiredFieldList(String relType) :
 }
 {
     <LPAREN>
-    fieldList = SimpleIdentifierCommaList()
+    fieldList = ParenthesizedCompoundIdentifierList()
     <RPAREN>
     {
         for(SqlNode node : fieldList)
@@ -351,4 +351,23 @@ SqlNode SqlDropFunction() :
    {
        return new SqlDropFunction(pos, jar);
    }
-}
\ No newline at end of file
+}
+
+<#if !parser.includeCompoundIdentifier >
+/**
+* Parses a comma-separated list of simple identifiers.
+*/
+SqlNodeList ParenthesizedCompoundIdentifierList() :
+{
+    List<SqlIdentifier> list = new ArrayList<SqlIdentifier>();
+    SqlIdentifier id;
+}
+{
+    id = SimpleIdentifier() {list.add(id);}
+    (
+   <COMMA> id = SimpleIdentifier() {list.add(id);}) *
+    {
+       return new SqlNodeList(list, getPos());
+    }
+}
+</#if>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/View.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/View.java b/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/View.java
index 97be73d..2b69f00 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/View.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/View.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
@@ -106,8 +106,19 @@ public class View {
         p = dataType.getPrecision();
         s = dataType.getScale();
         break;
+      case INTERVAL_YEAR:
       case INTERVAL_YEAR_MONTH:
-      case INTERVAL_DAY_TIME:
+      case INTERVAL_MONTH:
+      case INTERVAL_DAY:
+      case INTERVAL_DAY_HOUR:
+      case INTERVAL_DAY_MINUTE:
+      case INTERVAL_DAY_SECOND:
+      case INTERVAL_HOUR:
+      case INTERVAL_HOUR_MINUTE:
+      case INTERVAL_HOUR_SECOND:
+      case INTERVAL_MINUTE:
+      case INTERVAL_MINUTE_SECOND:
+      case INTERVAL_SECOND:
         p = dataType.getIntervalQualifier().getStartPrecisionPreservingDefault();
       default:
         break;

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java
----------------------------------------------------------------------
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 a04f34d..49c9726 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
@@ -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
@@ -28,6 +28,7 @@ import java.util.Map;
 
 import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
@@ -259,7 +260,8 @@ public class FileSystemPartitionDescriptor extends AbstractPartitionDescriptor {
             new DynamicDrillTable(table.getPlugin(), table.getStorageEngineName(),
             table.getUserName(),
             newFormatSelection));
-    final RelOptTableImpl newOptTableImpl = RelOptTableImpl.create(t.getRelOptSchema(), t.getRowType(), newTable);
+    final RelOptTableImpl newOptTableImpl = RelOptTableImpl.create(t.getRelOptSchema(), t.getRowType(), newTable,
+        ImmutableList.<String>of());
 
     // return an EnumerableTableScan with fileSelection being part of digest of TableScan node.
     return DirPrunedEnumerableTableScan.create(oldScan.getCluster(), newOptTableImpl, newFileSelection.toString());

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRelBase.java
index d61f271..2781564 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRelBase.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
@@ -26,6 +26,7 @@ import org.apache.calcite.rel.core.TableModify;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rex.RexNode;
 
 /**
  * Base class for logical and physical Store implemented in Drill
@@ -33,9 +34,9 @@ import org.apache.calcite.plan.RelTraitSet;
 public abstract class DrillStoreRelBase extends TableModify implements DrillRelNode {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillStoreRelBase.class);
 
-  protected DrillStoreRelBase(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, CatalogReader catalogReader,
-      RelNode child, Operation operation, List<String> updateColumnList, boolean flattened) {
-    super(cluster, traits, table, catalogReader, child, operation, updateColumnList, flattened);
+  protected DrillStoreRelBase(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, CatalogReader catalogReader, RelNode child,
+                              Operation operation, List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
+    super(cluster, traits, table, catalogReader, child, operation, updateColumnList, sourceExpressionList, flattened);
 
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java
index 4d54fce..a182d35 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java
@@ -216,7 +216,9 @@ public abstract class DrillValuesRelBase extends Values implements DrillRelNode
         }
         return;
 
+      case INTERVAL_YEAR:
       case INTERVAL_YEAR_MONTH:
+      case INTERVAL_MONTH:
         if (isLiteralNull(literal)) {
           out.writeIntervalNull();
         } else {
@@ -225,7 +227,16 @@ public abstract class DrillValuesRelBase extends Values implements DrillRelNode
         }
         return;
 
-      case INTERVAL_DAY_TIME:
+      case INTERVAL_DAY:
+      case INTERVAL_DAY_HOUR:
+      case INTERVAL_DAY_MINUTE:
+      case INTERVAL_DAY_SECOND:
+      case INTERVAL_HOUR:
+      case INTERVAL_HOUR_MINUTE:
+      case INTERVAL_HOUR_SECOND:
+      case INTERVAL_MINUTE:
+      case INTERVAL_MINUTE_SECOND:
+      case INTERVAL_SECOND:
         if (isLiteralNull(literal)) {
           out.writeIntervalNull();
         } else {

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java
index 0ca4447..cbb5aee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.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,9 +17,6 @@
  */
 package org.apache.drill.exec.planner.logical;
 
-import java.util.logging.Logger;
-
-import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.plan.Convention;
@@ -28,6 +25,7 @@ import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.util.trace.CalciteTrace;
+import org.slf4j.Logger;
 
 /**
  * Rule that converts an {@link LogicalAggregate} to a {@link DrillAggregateRel}, implemented by a Drill "segment" operation
@@ -43,7 +41,7 @@ public class DrillAggregateRule extends RelOptRule {
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final LogicalAggregate aggregate = (LogicalAggregate) call.rel(0);
+    final LogicalAggregate aggregate = call.rel(0);
     final RelNode input = call.rel(1);
 
     if (aggregate.containsDistinctCall()) {
@@ -57,7 +55,7 @@ public class DrillAggregateRule extends RelOptRule {
       call.transformTo(new DrillAggregateRel(aggregate.getCluster(), traits, convertedInput, aggregate.indicator,
           aggregate.getGroupSet(), aggregate.getGroupSets(), aggregate.getAggCallList()));
     } catch (InvalidRelException e) {
-      tracer.warning(e.toString());
+      tracer.warn(e.toString());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConditions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConditions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConditions.java
index f68007d..266ce64 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConditions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConditions.java
@@ -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
@@ -24,6 +24,12 @@ import org.apache.calcite.rex.RexNode;
 public final class DrillConditions {
 
   public static PushProjector.ExprCondition PRESERVE_ITEM = new PushProjector.ExprCondition() {
+
+    @Override
+    public boolean apply(RexNode rexNode) {
+      return false;
+    }
+
     @Override
     public boolean test(RexNode expr) {
       if (expr instanceof RexCall) {

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/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 19c7524..cdba8b8 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
@@ -20,6 +20,10 @@ package org.apache.drill.exec.planner.logical;
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import io.netty.buffer.DrillBuf;
+import org.apache.calcite.rex.RexExecutor;
+import org.apache.calcite.util.DateString;
+import org.apache.calcite.util.TimeString;
+import org.apache.calcite.util.TimestampString;
 import org.apache.calcite.rel.RelNode;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.ErrorCollectorImpl;
@@ -79,7 +83,7 @@ import java.math.BigInteger;
 import java.util.Calendar;
 import java.util.List;
 
-public class DrillConstExecutor implements RelOptPlanner.Executor {
+public class DrillConstExecutor implements RexExecutor {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillConstExecutor.class);
 
   private final PlannerSettings plannerSettings;
@@ -203,7 +207,7 @@ public class DrillConstExecutor implements RelOptPlanner.Executor {
               Calendar value = (materializedExpr.getMajorType().getMode() == TypeProtos.DataMode.OPTIONAL) ?
                 new DateTime(((NullableDateHolder) output).value, DateTimeZone.UTC).toCalendar(null) :
                 new DateTime(((DateHolder) output).value, DateTimeZone.UTC).toCalendar(null);
-              return rexBuilder.makeLiteral(value,
+              return rexBuilder.makeLiteral(DateString.fromCalendarFields(value),
                 TypeInferenceUtils.createCalciteTypeWithNullability(typeFactory, SqlTypeName.DATE, newCall.getType().isNullable()), false);
             }
             case DECIMAL9: {
@@ -282,14 +286,14 @@ public class DrillConstExecutor implements RelOptPlanner.Executor {
               Calendar value = (materializedExpr.getMajorType().getMode() == TypeProtos.DataMode.OPTIONAL) ?
                 new DateTime(((NullableTimeHolder) output).value, DateTimeZone.UTC).toCalendar(null) :
                 new DateTime(((TimeHolder) output).value, DateTimeZone.UTC).toCalendar(null);
-              return rexBuilder.makeLiteral(value,
+              return rexBuilder.makeLiteral(TimeString.fromCalendarFields(value),
                 TypeInferenceUtils.createCalciteTypeWithNullability(typeFactory, SqlTypeName.TIME, newCall.getType().isNullable()), false);
             }
             case TIMESTAMP: {
               Calendar value = (materializedExpr.getMajorType().getMode() == TypeProtos.DataMode.OPTIONAL) ?
                 new DateTime(((NullableTimeStampHolder) output).value, DateTimeZone.UTC).toCalendar(null) :
                 new DateTime(((TimeStampHolder) output).value, DateTimeZone.UTC).toCalendar(null);
-              return rexBuilder.makeLiteral(value,
+              return rexBuilder.makeLiteral(TimestampString.fromCalendarFields(value),
                 TypeInferenceUtils.createCalciteTypeWithNullability(typeFactory, SqlTypeName.TIMESTAMP, newCall.getType().isNullable()), false);
             }
             case INTERVALYEAR: {
@@ -313,7 +317,7 @@ public class DrillConstExecutor implements RelOptPlanner.Executor {
               }
               return rexBuilder.makeLiteral(
                 new BigDecimal(days * DateUtility.daysToStandardMillis + milliseconds),
-                TypeInferenceUtils.createCalciteTypeWithNullability(typeFactory, SqlTypeName.INTERVAL_DAY_TIME, newCall.getType().isNullable()),
+                TypeInferenceUtils.createCalciteTypeWithNullability(typeFactory, SqlTypeName.INTERVAL_DAY, newCall.getType().isNullable()),
                 false);
             }
             // The list of known unsupported types is used to trigger this behavior of re-using the input expression

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
index d41ae72..deb8ace 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.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.logical;
 
 import java.util.List;
-import java.util.logging.Logger;
 
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.core.JoinRelType;
@@ -37,6 +36,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.collect.Lists;
+import org.slf4j.Logger;
 
 /**
  * Rule that converts a {@link org.apache.calcite.rel.logical.LogicalJoin} to a {@link DrillJoinRel}, which is implemented by Drill "join" operation.
@@ -53,7 +53,7 @@ public class DrillJoinRule extends RelOptRule {
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final LogicalJoin join = (LogicalJoin) call.rel(0);
+    final LogicalJoin join = call.rel(0);
     final RelNode left = join.getLeft();
     final RelNode right = join.getRight();
     final RelTraitSet traits = join.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
@@ -97,7 +97,7 @@ public class DrillJoinRule extends RelOptRule {
         call.transformTo(new DrillFilterRel(join.getCluster(), traits, joinRel, remaining));
       }
     } catch (InvalidRelException e) {
-      tracer.warning(e.toString());
+      tracer.warn(e.toString());
     }
   }
 
@@ -109,8 +109,8 @@ public class DrillJoinRule extends RelOptRule {
     List<RelDataTypeField> rightTypes = convertedRight.getRowType().getFieldList();
 
     for (int i=0; i < leftKeys.size(); i++) {
-      int leftKeyOrdinal = leftKeys.get(i).intValue();
-      int rightKeyOrdinal = rightKeys.get(i).intValue();
+      int leftKeyOrdinal = leftKeys.get(i);
+      int rightKeyOrdinal = rightKeys.get(i);
 
       equijoinList.add(builder.makeCall(
            filterNulls.get(i) ? SqlStdOperatorTable.EQUALS : SqlStdOperatorTable.IS_NOT_DISTINCT_FROM,

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
index 0d8efd5..7dc2050 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
@@ -226,6 +226,10 @@ public class DrillOptiq {
           return doFunction(call, "+");
         }
 
+        if (call.getOperator() == SqlStdOperatorTable.MINUS_DATE) {
+          return doFunction(call, "-");
+        }
+
         // fall through
       default:
         throw new AssertionError("todo: implement syntax " + syntax + "(" + call + ")");
@@ -325,8 +329,19 @@ public class DrillOptiq {
         }
         break;
 
-        case "INTERVAL_YEAR_MONTH": castType = Types.required(MinorType.INTERVALYEAR); break;
-        case "INTERVAL_DAY_TIME": castType = Types.required(MinorType.INTERVALDAY); break;
+        case "INTERVAL_YEAR":
+        case "INTERVAL_YEAR_MONTH":
+        case "INTERVAL_MONTH": castType = Types.required(MinorType.INTERVALYEAR); break;
+        case "INTERVAL_DAY":
+        case "INTERVAL_DAY_HOUR":
+        case "INTERVAL_DAY_MINUTE":
+        case "INTERVAL_DAY_SECOND":
+        case "INTERVAL_HOUR":
+        case "INTERVAL_HOUR_MINUTE":
+        case "INTERVAL_HOUR_SECOND":
+        case "INTERVAL_MINUTE":
+        case "INTERVAL_MINUTE_SECOND":
+        case "INTERVAL_SECOND": castType = Types.required(MinorType.INTERVALDAY); break;
         case "BOOLEAN": castType = Types.required(MinorType.BIT); break;
         case "BINARY": castType = Types.required(MinorType.VARBINARY); break;
         case "ANY": return arg; // Type will be same as argument.
@@ -574,11 +589,22 @@ public class DrillOptiq {
         }
         return (ValueExpressions.getTimeStamp((GregorianCalendar) literal.getValue()));
       case INTERVAL_YEAR_MONTH:
+      case INTERVAL_YEAR:
+      case INTERVAL_MONTH:
         if (isLiteralNull(literal)) {
           return createNullExpr(MinorType.INTERVALYEAR);
         }
         return (ValueExpressions.getIntervalYear(((BigDecimal) (literal.getValue())).intValue()));
-      case INTERVAL_DAY_TIME:
+      case INTERVAL_DAY:
+      case INTERVAL_DAY_HOUR:
+      case INTERVAL_DAY_MINUTE:
+      case INTERVAL_DAY_SECOND:
+      case INTERVAL_HOUR:
+      case INTERVAL_HOUR_MINUTE:
+      case INTERVAL_HOUR_SECOND:
+      case INTERVAL_MINUTE:
+      case INTERVAL_MINUTE_SECOND:
+      case INTERVAL_SECOND:
         if (isLiteralNull(literal)) {
           return createNullExpr(MinorType.INTERVALDAY);
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectSetOpTransposeRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectSetOpTransposeRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectSetOpTransposeRule.java
index bde82e7..baff3b7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectSetOpTransposeRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectSetOpTransposeRule.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,6 +19,7 @@
 package org.apache.drill.exec.planner.logical;
 
 import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.rules.ProjectSetOpTransposeRule;
 import org.apache.calcite.rel.rules.PushProjector;
 
@@ -26,6 +27,6 @@ public class DrillProjectSetOpTransposeRule extends ProjectSetOpTransposeRule {
   public final static RelOptRule INSTANCE = new DrillProjectSetOpTransposeRule(DrillConditions.PRESERVE_ITEM);
 
   protected DrillProjectSetOpTransposeRule(PushProjector.ExprCondition preserveExprCondition) {
-    super(preserveExprCondition);
+    super(preserveExprCondition, RelFactories.LOGICAL_BUILDER);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastFilterRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastFilterRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastFilterRule.java
index 4f950d3..e5c40b9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastFilterRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastFilterRule.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,6 +17,9 @@
  */
 package org.apache.drill.exec.planner.logical;
 
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.rules.ProjectFilterTransposeRule;
 import org.apache.calcite.rel.rules.PushProjector;
 import org.apache.calcite.plan.RelOptRule;
@@ -26,7 +29,7 @@ public class DrillPushProjectPastFilterRule extends ProjectFilterTransposeRule {
   public final static RelOptRule INSTANCE = new DrillPushProjectPastFilterRule(DrillConditions.PRESERVE_ITEM);
 
   protected DrillPushProjectPastFilterRule(PushProjector.ExprCondition preserveExprCondition) {
-    super(preserveExprCondition);
+    super(LogicalProject.class, LogicalFilter.class, RelFactories.LOGICAL_BUILDER, preserveExprCondition);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastJoinRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastJoinRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastJoinRule.java
index e27360f..269f6d5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastJoinRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastJoinRule.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,6 +17,7 @@
  */
 package org.apache.drill.exec.planner.logical;
 
+import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.rules.ProjectJoinTransposeRule;
 import org.apache.calcite.rel.rules.PushProjector;
 import org.apache.calcite.plan.RelOptRule;
@@ -26,7 +27,7 @@ public class DrillPushProjectPastJoinRule extends ProjectJoinTransposeRule {
   public static final RelOptRule INSTANCE = new DrillPushProjectPastJoinRule(DrillConditions.PRESERVE_ITEM);
 
   protected DrillPushProjectPastJoinRule(PushProjector.ExprCondition preserveExprCondition) {
-    super(preserveExprCondition);
+    super(preserveExprCondition, RelFactories.LOGICAL_BUILDER);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/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 9f8d062..bb972d0 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
@@ -25,7 +25,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.logging.Logger;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -66,6 +65,7 @@ import org.apache.calcite.util.Util;
 import com.google.common.collect.ImmutableList;
 import org.apache.drill.exec.planner.sql.TypeInferenceUtils;
 import org.apache.drill.exec.planner.sql.parser.DrillCalciteWrapperUtility;
+import org.slf4j.Logger;
 
 /**
  * Rule to reduce aggregates to simpler forms. Currently only AVG(x) to
@@ -718,7 +718,7 @@ public class DrillReduceAggregatesRule extends RelOptRule {
             oldAggRel.getGroupSets(),
             newAggregateCalls));
       } catch (InvalidRelException e) {
-        tracer.warning(e.toString());
+        tracer.warn(e.toString());
       }
     }
   }
@@ -760,7 +760,8 @@ public class DrillReduceAggregatesRule extends RelOptRule {
                     sumZeroAgg,
                     sumType,
                     rexWinAggCall.operands,
-                    rexWinAggCall.ordinal);
+                    rexWinAggCall.ordinal,
+                    rexWinAggCall.distinct);
             aggCalls.add(sumZeroCall);
           } else {
             aggCalls.add(rexWinAggCall);

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/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 2b65831..96bcf02 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
@@ -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,17 +14,18 @@
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * See the License for the specific language governing permissions and
  * limitations under the License.
- ******************************************************************************/
+ */
 package org.apache.drill.exec.planner.logical;
 
+import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.core.Calc;
-import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.RelCollationImpl;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.SingleRel;
-import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalSort;
 import org.apache.calcite.rel.rules.ReduceExpressionsRule;
 
@@ -38,10 +39,10 @@ public class DrillReduceExpressionsRule {
   public static final DrillReduceCalcRule CALC_INSTANCE_DRILL =
       new DrillReduceCalcRule();
 
-  private static class DrillReduceFilterRule extends ReduceExpressionsRule.ReduceFilterRule {
+  private static class DrillReduceFilterRule extends ReduceExpressionsRule.FilterReduceExpressionsRule {
 
     DrillReduceFilterRule() {
-      super("DrillReduceExpressionsRule(Filter)");
+      super(DrillFilterRel.class, RelFactories.LOGICAL_BUILDER);
     }
 
     /**
@@ -51,16 +52,16 @@ public class DrillReduceExpressionsRule {
      * expose the planning time known schema. Instead we have to insert a limit 0.
      */
     @Override
-    protected RelNode createEmptyRelOrEquivalent(Filter filter) {
+    protected RelNode createEmptyRelOrEquivalent(RelOptRuleCall call, Filter filter) {
       return createEmptyEmptyRelHelper(filter);
     }
 
   }
 
-  private static class DrillReduceCalcRule extends ReduceExpressionsRule.ReduceCalcRule {
+  private static class DrillReduceCalcRule extends ReduceExpressionsRule.CalcReduceExpressionsRule {
 
     DrillReduceCalcRule() {
-      super("DrillReduceExpressionsRule(Calc)");
+      super(Calc.class, RelFactories.LOGICAL_BUILDER);
     }
 
     /**
@@ -70,8 +71,8 @@ public class DrillReduceExpressionsRule {
      * expose the planning time known schema. Instead we have to insert a limit 0.
      */
     @Override
-    protected RelNode createEmptyRelOrEquivalent(Calc calc) {
-      return createEmptyEmptyRelHelper(calc);
+    protected RelNode createEmptyRelOrEquivalent(RelOptRuleCall call, Calc input) {
+      return createEmptyEmptyRelHelper(input);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRelFactories.java
----------------------------------------------------------------------
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 0348a5b..487da4d 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
@@ -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 org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.logical.LogicalFilter;
@@ -79,6 +80,14 @@ public class DrillRelFactories {
    * {@link org.apache.calcite.rel.logical.LogicalJoin}.
    */
   private static class DrillJoinFactoryImpl implements RelFactories.JoinFactory {
+
+    @Override
+    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);
+    }
+
     @Override
     public RelNode createJoin(RelNode left, RelNode right,
                               RexNode condition, JoinRelType joinType,

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java
index 7c498f3..9872f95 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.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,6 +21,7 @@ import java.util.List;
 
 import org.apache.calcite.prepare.Prepare.CatalogReader;
 
+import org.apache.calcite.rex.RexNode;
 import org.apache.drill.common.logical.data.LogicalOperator;
 import org.apache.drill.exec.planner.common.DrillStoreRelBase;
 import org.apache.calcite.rel.RelNode;
@@ -31,9 +32,9 @@ import org.apache.calcite.plan.RelTraitSet;
 public class DrillStoreRel extends DrillStoreRelBase implements DrillRel{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillStoreRel.class);
 
-  protected DrillStoreRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, CatalogReader catalogReader,
-      RelNode child, Operation operation, List<String> updateColumnList, boolean flattened) {
-    super(cluster, traits, table, catalogReader, child, operation, updateColumnList, flattened);
+  protected DrillStoreRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, CatalogReader catalogReader, RelNode child,
+                          Operation operation, List<String> updateColumnList, List<RexNode> sourceExpressionList, boolean flattened) {
+    super(cluster, traits, table, catalogReader, child, operation, updateColumnList, sourceExpressionList, flattened);
 
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionAllRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionAllRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionAllRule.java
index 10c0118..a074952 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionAllRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionAllRule.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,7 +19,6 @@ package org.apache.drill.exec.planner.logical;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.logging.Logger;
 
 import org.apache.drill.exec.planner.common.DrillUnionRelBase;
 import org.apache.calcite.rel.InvalidRelException;
@@ -30,6 +29,7 @@ import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.util.trace.CalciteTrace;
+import org.slf4j.Logger;
 
 /**
  * Rule that converts a {@link LogicalUnion} to a {@link DrillUnionRelBase}, implemented by a "union" operation.
@@ -44,7 +44,7 @@ public class DrillUnionAllRule extends RelOptRule {
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final LogicalUnion union = (LogicalUnion) call.rel(0);
+    final LogicalUnion union = call.rel(0);
 
     // This rule applies to Union-All only
     if(!union.all) {
@@ -61,7 +61,7 @@ public class DrillUnionAllRule extends RelOptRule {
       call.transformTo(new DrillUnionRel(union.getCluster(), traits, convertedInputs, union.all,
           true /* check compatibility */));
     } catch (InvalidRelException e) {
-      tracer.warning(e.toString()) ;
+      tracer.warn(e.toString()) ;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillViewTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillViewTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillViewTable.java
index d2e1ace..0f0278e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillViewTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillViewTable.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,8 +17,7 @@
  */
 package org.apache.drill.exec.planner.logical;
 
-import java.util.List;
-
+import com.google.common.collect.ImmutableList;
 import org.apache.calcite.schema.Schema.TableType;
 import org.apache.calcite.schema.Statistic;
 import org.apache.calcite.schema.Statistics;
@@ -65,9 +64,9 @@ public class DrillViewTable implements TranslatableTable, DrillViewInfoProvider
 
       if (viewExpansionContext.isImpersonationEnabled()) {
         token = viewExpansionContext.reserveViewExpansionToken(viewOwner);
-        rel = context.expandView(rowType, view.getSql(), token.getSchemaTree(), view.getWorkspaceSchemaPath());
+        rel = context.expandView(rowType, view.getSql(), token.getSchemaTree(), view.getWorkspaceSchemaPath()).rel;
       } else {
-        rel = context.expandView(rowType, view.getSql(), view.getWorkspaceSchemaPath());
+        rel = context.expandView(rowType, view.getSql(), view.getWorkspaceSchemaPath(), ImmutableList.<String>of()).rel;
       }
 
       // If the View's field list is not "*", create a cast.

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
index 09d33fd..b647279 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
@@ -21,13 +21,8 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.drill.common.logical.data.NamedExpression;
-import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.expr.holders.IntHolder;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.HashAggregate;
-import org.apache.drill.exec.planner.cost.DrillCostBase;
-import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Aggregate;

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrule.java
index e16593b..f4cdf62 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrule.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,8 +17,6 @@
  */
 package org.apache.drill.exec.planner.physical;
 
-import java.util.logging.Logger;
-
 import org.apache.drill.exec.planner.logical.DrillAggregateRel;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
 import org.apache.drill.exec.planner.physical.AggPrelBase.OperatorPhase;
@@ -31,6 +29,7 @@ import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.collect.ImmutableList;
+import org.slf4j.Logger;
 
 public class HashAggPrule extends AggPruleBase {
   public static final RelOptRule INSTANCE = new HashAggPrule();
@@ -94,7 +93,7 @@ public class HashAggPrule extends AggPruleBase {
         }
       }
     } catch (InvalidRelException e) {
-      tracer.warning(e.toString());
+      tracer.warn(e.toString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
index 1795bf1..31b7772 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
@@ -27,7 +27,6 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.HashJoinPOP;
 import org.apache.drill.exec.physical.impl.join.JoinUtils;
 import org.apache.drill.exec.physical.impl.join.JoinUtils.JoinCategory;
-import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.core.JoinRelType;

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrule.java
index 1fd0e51..d07cf51 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrule.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,8 +17,6 @@
  */
 package org.apache.drill.exec.planner.physical;
 
-import java.util.logging.Logger;
-
 import org.apache.drill.exec.planner.logical.DrillJoinRel;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
 import org.apache.calcite.rel.InvalidRelException;
@@ -27,6 +25,7 @@ import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.util.trace.CalciteTrace;
+import org.slf4j.Logger;
 
 public class HashJoinPrule extends JoinPruleBase {
   public static final RelOptRule DIST_INSTANCE = new HashJoinPrule("Prel.HashJoinDistPrule", RelOptHelper.any(DrillJoinRel.class), true);
@@ -53,7 +52,7 @@ public class HashJoinPrule extends JoinPruleBase {
       return;
     }
 
-    final DrillJoinRel join = (DrillJoinRel) call.rel(0);
+    final DrillJoinRel join = call.rel(0);
     final RelNode left = join.getLeft();
     final RelNode right = join.getRight();
 
@@ -77,7 +76,7 @@ public class HashJoinPrule extends JoinPruleBase {
 
 
     } catch (InvalidRelException e) {
-      tracer.warning(e.toString());
+      tracer.warn(e.toString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/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 dd587b8..3f40720 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
@@ -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;
 
 import java.util.List;
-import java.util.logging.Logger;
 
 import org.apache.drill.exec.planner.logical.DrillJoinRel;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
@@ -33,6 +32,7 @@ import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.collect.Lists;
+import org.slf4j.Logger;
 
 public class MergeJoinPrule extends JoinPruleBase {
   public static final RelOptRule DIST_INSTANCE = new MergeJoinPrule("Prel.MergeJoinDistPrule", RelOptHelper.any(DrillJoinRel.class), true);
@@ -78,7 +78,7 @@ public class MergeJoinPrule extends JoinPruleBase {
       }
 
     } catch (InvalidRelException e) {
-      tracer.warning(e.toString());
+      tracer.warn(e.toString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/NestedLoopJoinPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/NestedLoopJoinPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/NestedLoopJoinPrule.java
index b98976b..26e1b73 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/NestedLoopJoinPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/NestedLoopJoinPrule.java
@@ -18,7 +18,6 @@
 package org.apache.drill.exec.planner.physical;
 
 import java.util.List;
-import java.util.logging.Logger;
 
 import org.apache.drill.exec.physical.impl.join.JoinUtils;
 import org.apache.drill.exec.physical.impl.join.JoinUtils.JoinCategory;
@@ -33,6 +32,7 @@ import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.collect.Lists;
+import org.slf4j.Logger;
 
 
 public class NestedLoopJoinPrule extends JoinPruleBase {
@@ -81,7 +81,7 @@ public class NestedLoopJoinPrule extends JoinPruleBase {
       return;
     }
 
-    final DrillJoinRel join = (DrillJoinRel) call.rel(0);
+    final DrillJoinRel join = call.rel(0);
     final RelNode left = join.getLeft();
     final RelNode right = join.getRight();
 
@@ -97,7 +97,7 @@ public class NestedLoopJoinPrule extends JoinPruleBase {
       }
 
     } catch (InvalidRelException e) {
-      tracer.warning(e.toString());
+      tracer.warn(e.toString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/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 ea7927c..a6a8f28 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
@@ -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;
 
 import java.util.List;
-import java.util.logging.Logger;
 
 import org.apache.calcite.util.BitSets;
 
@@ -37,6 +36,7 @@ import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+import org.slf4j.Logger;
 
 public class StreamAggPrule extends AggPruleBase {
   public static final RelOptRule INSTANCE = new StreamAggPrule();
@@ -174,7 +174,7 @@ public class StreamAggPrule extends AggPruleBase {
         }
       }
     } catch (InvalidRelException e) {
-      tracer.warning(e.toString());
+      tracer.warn(e.toString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionAllPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionAllPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionAllPrule.java
index 9d93220..336ab3a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionAllPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionAllPrule.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;
 
 import java.util.List;
-import java.util.logging.Logger;
 
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
@@ -34,6 +33,7 @@ import org.apache.drill.exec.planner.physical.DrillDistributionTrait.Distributio
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+import org.slf4j.Logger;
 
 public class UnionAllPrule extends Prule {
   public static final RelOptRule INSTANCE = new UnionAllPrule();
@@ -46,20 +46,19 @@ public class UnionAllPrule extends Prule {
 
   @Override
   public boolean matches(RelOptRuleCall call) {
-    DrillUnionRel union = (DrillUnionRel) call.rel(0);
+    DrillUnionRel union = call.rel(0);
     return (! union.isDistinct());
   }
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final DrillUnionRel union = (DrillUnionRel) call.rel(0);
+    final DrillUnionRel union = call.rel(0);
     final List<RelNode> inputs = union.getInputs();
     List<RelNode> convertedInputList = Lists.newArrayList();
     PlannerSettings settings = PrelUtil.getPlannerSettings(call.getPlanner());
     boolean allHashDistributed = true;
 
-    for (int i = 0; i < inputs.size(); i++) {
-      RelNode child = inputs.get(i);
+    for (RelNode child : inputs) {
       List<DistributionField> childDistFields = Lists.newArrayList();
       RelNode convertedChild;
 
@@ -110,7 +109,7 @@ public class UnionAllPrule extends Prule {
       call.transformTo(left);
 
     } catch (InvalidRelException e) {
-      tracer.warning(e.toString());
+      tracer.warn(e.toString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionDistinctPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionDistinctPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionDistinctPrule.java
index 8b4d62d..f353802 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionDistinctPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionDistinctPrule.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;
 
 import java.util.List;
-import java.util.logging.Logger;
 
 import org.apache.drill.exec.planner.logical.DrillUnionRel;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
@@ -30,6 +29,7 @@ import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.collect.Lists;
+import org.slf4j.Logger;
 
 public class UnionDistinctPrule extends Prule {
   public static final RelOptRule INSTANCE = new UnionDistinctPrule();
@@ -42,20 +42,20 @@ public class UnionDistinctPrule extends Prule {
 
   @Override
   public boolean matches(RelOptRuleCall call) {
-    DrillUnionRel union = (DrillUnionRel) call.rel(0);
+    DrillUnionRel union = call.rel(0);
     return (union.isDistinct() && union.isHomogeneous(false /* don't compare names */));
   }
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final DrillUnionRel union = (DrillUnionRel) call.rel(0);
+    final DrillUnionRel union = call.rel(0);
     final List<RelNode> inputs = union.getInputs();
     List<RelNode> convertedInputList = Lists.newArrayList();
     RelTraitSet traits = call.getPlanner().emptyTraitSet().plus(Prel.DRILL_PHYSICAL);
 
     try {
-      for (int i = 0; i < inputs.size(); i++) {
-        RelNode convertedInput = convert(inputs.get(i), PrelUtil.fixTraits(call, traits));
+      for (RelNode input : inputs) {
+        RelNode convertedInput = convert(input, PrelUtil.fixTraits(call, traits));
         convertedInputList.add(convertedInput);
       }
 
@@ -67,7 +67,7 @@ public class UnionDistinctPrule extends Prule {
       call.transformTo(unionDistinct);
 
     } catch (InvalidRelException e) {
-      tracer.warning(e.toString());
+      tracer.warn(e.toString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/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 1980335..4fdb3b9 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
@@ -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
@@ -154,15 +154,15 @@ public class WindowPrule extends Prule {
 
           newOperandsOfWindowFunction.add(operand);
         }
-
         aggCall = new Window.RexWinAggCall(
             (SqlAggFunction) aggCall.getOperator(),
             aggCall.getType(),
             newOperandsOfWindowFunction,
-            aggCall.ordinal);
+            aggCall.ordinal,
+            aggCall.distinct);
 
         newWinAggCalls.add(new Window.RexWinAggCall(
-            (SqlAggFunction)aggCall.getOperator(), aggCall.getType(), aggCall.getOperands(), aggOrd.i)
+            (SqlAggFunction)aggCall.getOperator(), aggCall.getType(), aggCall.getOperands(), aggOrd.i, aggCall.distinct)
         );
       }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/TopProjectVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/TopProjectVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/TopProjectVisitor.java
index 08bd9e7..6b81b70 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/TopProjectVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/TopProjectVisitor.java
@@ -129,7 +129,7 @@ public class TopProjectVisitor extends BasePrelVisitor<Prel, Void, RuntimeExcept
 
     List<String> fieldNames = SqlValidatorUtil.uniquify(
         validatedRowType.getFieldNames(),
-        SqlValidatorUtil.F_SUGGESTER2,
+        SqlValidatorUtil.EXPR_SUGGESTER,
         prel.getCluster().getTypeFactory().getTypeSystem().isSchemaCaseSensitive());
 
     RelDataType newRowType = RexUtil.createStructType(prel.getCluster().getTypeFactory(), projections, fieldNames, null);

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlAggFunctionWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlAggFunctionWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlAggFunctionWrapper.java
index 3795dd4..a87247b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlAggFunctionWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlAggFunctionWrapper.java
@@ -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
@@ -29,6 +29,7 @@ import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 
+import org.apache.calcite.util.Litmus;
 import org.apache.drill.exec.expr.fn.DrillFuncHolder;
 
 import java.util.List;
@@ -84,7 +85,7 @@ public class DrillCalciteSqlAggFunctionWrapper extends SqlAggFunction implements
   }
 
   @Override
-  public boolean validRexOperands(int count, boolean fail) {
+  public boolean validRexOperands(int count, Litmus litmus) {
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlFunctionWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlFunctionWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlFunctionWrapper.java
index 1c61d08..cf3b866 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlFunctionWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlFunctionWrapper.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
@@ -29,6 +29,7 @@ import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 
+import org.apache.calcite.util.Litmus;
 import org.apache.drill.exec.expr.fn.DrillFuncHolder;
 
 import java.util.List;
@@ -65,7 +66,7 @@ public class DrillCalciteSqlFunctionWrapper extends SqlFunction implements Drill
   }
 
   @Override
-  public boolean validRexOperands(int count, boolean fail) {
+  public boolean validRexOperands(int count, Litmus litmus) {
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlOperatorWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlOperatorWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlOperatorWrapper.java
index 8258120..2bf04b9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlOperatorWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlOperatorWrapper.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
@@ -28,6 +28,7 @@ import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Litmus;
 import org.apache.drill.exec.expr.fn.DrillFuncHolder;
 
 import java.util.List;
@@ -88,7 +89,7 @@ public class DrillCalciteSqlOperatorWrapper extends SqlOperator implements Drill
   }
 
   @Override
-  public boolean validRexOperands(int count, boolean fail) {
+  public boolean validRexOperands(int count, Litmus litmus) {
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserConfig.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserConfig.java
index d8441ae..1f67c5e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserConfig.java
@@ -21,6 +21,8 @@ import org.apache.calcite.avatica.util.Casing;
 import org.apache.calcite.avatica.util.Quoting;
 import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.parser.SqlParserImplFactory;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.planner.sql.parser.impl.DrillParserWithCompoundIdConverter;
 
@@ -60,6 +62,16 @@ public class DrillParserConfig implements SqlParser.Config {
   }
 
   @Override
+  public SqlConformance conformance() {
+    return SqlConformanceEnum.DEFAULT;
+  }
+
+  @Override
+  public boolean allowBangEqual() {
+    return conformance().isBangEqualAllowed();
+  }
+
+  @Override
   public SqlParserImplFactory parserFactory() {
     return DrillParserWithCompoundIdConverter.FACTORY;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
----------------------------------------------------------------------
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 3bc0922..345e4b7 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
@@ -19,6 +19,8 @@ package org.apache.drill.exec.planner.sql;
 
 import java.io.IOException;
 
+import org.apache.calcite.sql.SqlDescribeSchema;
+import org.apache.calcite.sql.SqlDescribeTable;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.tools.RelConversionException;
@@ -28,10 +30,13 @@ import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.planner.sql.handlers.AbstractSqlHandler;
 import org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler;
+import org.apache.drill.exec.planner.sql.handlers.DescribeSchemaHandler;
+import org.apache.drill.exec.planner.sql.handlers.DescribeTableHandler;
 import org.apache.drill.exec.planner.sql.handlers.ExplainHandler;
 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;
@@ -112,6 +117,16 @@ public class DrillSqlWorker {
     case SET_OPTION:
       handler = new SetOptionHandler(context);
       break;
+    case DESCRIBE_TABLE:
+      if (sqlNode instanceof DrillSqlDescribeTable) {
+        handler = new DescribeTableHandler(config);
+        break;
+      }
+    case DESCRIBE_SCHEMA:
+      if (sqlNode instanceof SqlDescribeSchema) {
+        handler = new DescribeSchemaHandler(config);
+        break;
+      }
     case OTHER:
       if(sqlNode instanceof SqlCreateTable) {
         handler = ((DrillSqlCall)sqlNode).getSqlHandler(config, textPlan);

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/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 dbe16bd..6f59fcf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
@@ -22,9 +22,12 @@ import java.util.List;
 import java.util.Set;
 
 import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.util.Casing;
+import org.apache.calcite.avatica.util.Quoting;
 import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.jdbc.CalciteSchemaImpl;
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 import org.apache.calcite.plan.ConventionTraitDef;
 import org.apache.calcite.plan.RelOptCluster;
@@ -32,9 +35,10 @@ import org.apache.calcite.plan.RelOptCostFactory;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.volcano.VolcanoPlanner;
 import org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.calcite.prepare.Prepare;
 import org.apache.calcite.prepare.RelOptTableImpl;
 import org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
@@ -45,10 +49,12 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.SqlParserImplFactory;
 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;
@@ -67,11 +73,10 @@ import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.logical.DrillConstExecutor;
 import org.apache.drill.exec.planner.physical.DrillDistributionTraitDef;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.planner.sql.parser.impl.DrillParserWithCompoundIdConverter;
 import org.apache.drill.exec.rpc.user.UserSession;
 
 import com.google.common.base.Joiner;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
 
 /**
  * Class responsible for managing parsing, validation and toRel conversion for sql statements.
@@ -119,7 +124,7 @@ public class SqlConverter {
     this.catalog = new DrillCalciteCatalogReader(
         this.rootSchema,
         parserConfig.caseSensitive(),
-        CalciteSchemaImpl.from(defaultSchema).path(null),
+        CalciteSchema.from(defaultSchema).path(null),
         typeFactory,
         drillConfig,
         session);
@@ -253,8 +258,7 @@ public class SqlConverter {
     }
   }
 
-  public RelNode toRel(
-      final SqlNode validatedNode) {
+  public RelRoot toRel(final SqlNode validatedNode) {
     final RexBuilder rexBuilder = new DrillRexBuilder(typeFactory);
     if (planner == null) {
       planner = new VolcanoPlanner(costFactory, settings);
@@ -269,9 +273,10 @@ public class SqlConverter {
     final SqlToRelConverter sqlToRelConverter =
         new SqlToRelConverter(new Expander(), validator, catalog, cluster, DrillConvertletTable.INSTANCE,
             sqlToRelConverterConfig);
-    final RelNode rel = sqlToRelConverter.convertQuery(validatedNode, false, !isInnerQuery);
-    final RelNode rel2 = sqlToRelConverter.flattenTypes(rel, true);
-    final RelNode rel3 = RelDecorrelator.decorrelateQuery(rel2);
+    //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));
     return rel3;
 
   }
@@ -282,7 +287,7 @@ public class SqlConverter {
     }
 
     @Override
-    public RelNode expandView(RelDataType rowType, String queryString, List<String> schemaPath) {
+    public RelRoot expandView(RelDataType rowType, String queryString, List<String> schemaPath, List<String> viewPath) {
       final DrillCalciteCatalogReader catalogReader = new DrillCalciteCatalogReader(
           rootSchema,
           parserConfig.caseSensitive(),
@@ -295,7 +300,7 @@ public class SqlConverter {
     }
 
     @Override
-    public RelNode expandView(RelDataType rowType, String queryString, SchemaPlus rootSchema, List<String> schemaPath) {
+    public RelRoot expandView(RelDataType rowType, String queryString, SchemaPlus rootSchema, List<String> schemaPath) {
       final DrillCalciteCatalogReader catalogReader = new DrillCalciteCatalogReader(
           rootSchema, // new root schema
           parserConfig.caseSensitive(),
@@ -324,7 +329,7 @@ public class SqlConverter {
       return expandView(queryString, parser);
     }
 
-    private RelNode expandView(String queryString, SqlConverter converter) {
+    private RelRoot expandView(String queryString, SqlConverter converter) {
       converter.disallowTemporaryTables();
       final SqlNode parsedNode = converter.parse(queryString);
       final SqlNode validatedNode = converter.validate(parsedNode);
@@ -368,7 +373,7 @@ public class SqlConverter {
     }
 
     @Override
-    public int getInSubqueryThreshold() {
+    public int getInSubQueryThreshold() {
       return inSubqueryThreshold;
     }
   }
@@ -446,7 +451,7 @@ public class SqlConverter {
                               JavaTypeFactory typeFactory,
                               DrillConfig drillConfig,
                               UserSession session) {
-      super(CalciteSchemaImpl.from(rootSchema), caseSensitive, defaultSchema, typeFactory);
+      super(CalciteSchema.from(rootSchema), caseSensitive, defaultSchema, typeFactory);
       this.drillConfig = drillConfig;
       this.session = session;
       this.allowTemporaryTables = true;
@@ -471,8 +476,8 @@ public class SqlConverter {
      * @throws UserException if temporary tables usage is disallowed
      */
     @Override
-    public RelOptTableImpl getTable(final List<String> names) {
-      RelOptTableImpl temporaryTable = null;
+    public Prepare.PreparingTable getTable(final List<String> names) {
+      Prepare.PreparingTable temporaryTable = null;
 
       if (mightBeTemporaryTable(names, session.getDefaultSchemaPath(), drillConfig)) {
         String temporaryTableName = session.resolveTemporaryTableName(names.get(names.size() - 1));
@@ -491,7 +496,7 @@ public class SqlConverter {
             .build(logger);
       }
 
-      RelOptTableImpl table = super.getTable(names);
+      Prepare.PreparingTable table = super.getTable(names);
 
       // Check the schema and throw a valid SchemaNotFound exception instead of TableNotFound exception.
       if (table == null) {


[10/18] drill git commit: DRILL-3993: Changes after code review.

Posted by am...@apache.org.
DRILL-3993: Changes after code review.


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

Branch: refs/heads/master
Commit: 663518e8ae1a05e29895a3994213a1e9819c955b
Parents: 688e043
Author: Volodymyr Vysotskyi <vv...@gmail.com>
Authored: Thu Dec 21 15:01:50 2017 +0200
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Tue Jan 16 12:10:13 2018 +0200

----------------------------------------------------------------------
 .../apache/drill/exec/store/hbase/HBaseSchemaFactory.java    | 8 ++++----
 .../main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java | 2 +-
 .../exec/store/openTSDB/schema/OpenTSDBSchemaFactory.java    | 8 ++++----
 .../java/org/apache/drill/exec/planner/sql/SqlConverter.java | 8 +-------
 exec/jdbc-all/pom.xml                                        | 2 +-
 5 files changed, 11 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/663518e8/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
index 810daaa..b8e825b 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
@@ -75,10 +75,10 @@ public class HBaseSchemaFactory implements SchemaFactory {
       try {
         return new DrillHBaseTable(schemaName, plugin, scanSpec);
       } catch (Exception e) {
-        // Calcite firstly is looking for a table in the default schema, if a table was not found,
-        // it is looking in root schema.
-        // If a table does not exist, a query will fail at validation stage,
-        // so the error should not be thrown there.
+        // Calcite firstly looks for a table in the default schema, if the table was not found,
+        // it looks in the root schema.
+        // If the table does not exist, a query will fail at validation stage,
+        // so the error should not be thrown here.
         logger.warn("Failure while loading table '{}' for database '{}'.", name, schemaName, e.getCause());
         return null;
       }

http://git-wip-us.apache.org/repos/asf/drill/blob/663518e8/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 e335c08..63752fa 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
@@ -62,7 +62,7 @@ public class JdbcPrel extends AbstractRelNode implements Prel {
         (JavaTypeFactory) getCluster().getTypeFactory());
     final JdbcImplementor.Result result =
         jdbcImplementor.visitChild(0, input.accept(new SubsetRemover()));
-    sql = result.asSelect().toSqlString(dialect).getSql();
+    sql = result.asStatement().toSqlString(dialect).getSql();
     rowType = input.getRowType();
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/663518e8/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/schema/OpenTSDBSchemaFactory.java
----------------------------------------------------------------------
diff --git a/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/schema/OpenTSDBSchemaFactory.java b/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/schema/OpenTSDBSchemaFactory.java
index ca93445..3b86a95 100644
--- a/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/schema/OpenTSDBSchemaFactory.java
+++ b/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/schema/OpenTSDBSchemaFactory.java
@@ -64,10 +64,10 @@ public class OpenTSDBSchemaFactory implements SchemaFactory {
       try {
         return new DrillOpenTSDBTable(schemaName, plugin, new Schema(plugin.getClient(), name), scanSpec);
       } catch (Exception e) {
-        // Calcite firstly is looking for a table in the default schema, if a table was not found,
-        // it is looking in root schema.
-        // If a table does not exist, a query will fail at validation stage,
-        // so the error should not be thrown there.
+        // Calcite firstly looks for a table in the default schema, if the table was not found,
+        // it looks in the root schema.
+        // If the table does not exist, a query will fail at validation stage,
+        // so the error should not be thrown here.
         logger.warn("Failure while loading table '{}' for database '{}'.", name, schemaName, e.getCause());
         return null;
       }

http://git-wip-us.apache.org/repos/asf/drill/blob/663518e8/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 ae9b37f..563f58e 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
@@ -586,13 +586,7 @@ public class SqlConverter {
               .build(logger);
         }
       }
-      // Fix for select from hbase table with schema name in query (example: "SELECT col FROM hbase.t)
-      // from hbase schema (did "USE hbase" before).
-      if (names.size() == getSchemaPaths().size() && getSchemaPaths().size() > 1) {
-        if (names.get(0).equals(getSchemaPaths().get(0).get(0))) {
-          useRootSchemaAsDefault(true);
-        }
-      }
+
       return super.getTable(names);
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/663518e8/exec/jdbc-all/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index f59780d..80d3716 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -569,7 +569,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>31000000</maxsize>
+                        <maxsize>32000000</maxsize>
                         <minsize>15000000</minsize>
                         <files>
                           <file>${project.build.directory}/drill-jdbc-all-${project.version}.jar</file>


[07/18] drill git commit: DRILL-3993: Changes to support Calcite 1.15.

Posted by am...@apache.org.
DRILL-3993: Changes to support Calcite 1.15.

Fix AssertionError: type mismatch for tests with aggregate functions.
Fix VARIANCE agg function
Remove using deprecated Subtype enum
Fix 'Failure while loading table a in database hbase' error
Fix 'Field ordinal 1 is invalid for  type '(DrillRecordRow[*])'' unit test failures


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

Branch: refs/heads/master
Commit: d59f0cda481d98a15d11ecbfb1463db53c954dfb
Parents: 9274cb9
Author: Volodymyr Vysotskyi <vv...@gmail.com>
Authored: Fri Dec 8 15:05:03 2017 +0200
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Tue Jan 16 12:10:13 2018 +0200

----------------------------------------------------------------------
 .../exec/store/hbase/HBaseSchemaFactory.java    | 11 ++-
 .../store/hive/schema/HiveDatabaseSchema.java   | 23 ++++--
 .../openTSDB/schema/OpenTSDBSchemaFactory.java  |  9 +++
 .../src/main/codegen/includes/parserImpls.ftl   | 12 +--
 .../logical/DrillReduceAggregatesRule.java      | 83 ++++++++++++--------
 .../drill/exec/planner/logical/DrillTable.java  | 15 +++-
 .../planner/logical/DrillTranslatableTable.java | 17 +++-
 .../exec/planner/logical/DrillViewTable.java    | 13 +++
 .../exec/planner/physical/AggPruleBase.java     | 10 ++-
 .../planner/sql/DrillAvgVarianceConvertlet.java |  6 +-
 .../exec/planner/sql/DrillConvertletTable.java  | 16 ++--
 .../drill/exec/planner/sql/SqlConverter.java    | 19 ++++-
 .../exec/planner/sql/TypeInferenceUtils.java    | 35 ++++-----
 .../apache/drill/exec/store/AbstractSchema.java |  8 +-
 .../exec/store/mock/MockStorageEngine.java      | 16 +++-
 .../java/org/apache/drill/TestBugFixes.java     |  2 +-
 .../apache/drill/exec/TestWindowFunctions.java  |  4 +-
 pom.xml                                         |  2 +-
 18 files changed, 207 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
index 548b679..810daaa 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
@@ -72,7 +72,16 @@ public class HBaseSchemaFactory implements SchemaFactory {
     @Override
     public Table getTable(String name) {
       HBaseScanSpec scanSpec = new HBaseScanSpec(name);
-      return new DrillHBaseTable(schemaName, plugin, scanSpec);
+      try {
+        return new DrillHBaseTable(schemaName, plugin, scanSpec);
+      } catch (Exception e) {
+        // Calcite firstly is looking for a table in the default schema, if a table was not found,
+        // it is looking in root schema.
+        // If a table does not exist, a query will fail at validation stage,
+        // so the error should not be thrown there.
+        logger.warn("Failure while loading table '{}' for database '{}'.", name, schemaName, e.getCause());
+        return null;
+      }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveDatabaseSchema.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveDatabaseSchema.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveDatabaseSchema.java
index 90f30d8..ec1d0c6 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveDatabaseSchema.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveDatabaseSchema.java
@@ -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,14 +17,16 @@
  */
 package org.apache.drill.exec.store.hive.schema;
 
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
+import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.Statistic;
 import org.apache.calcite.schema.Table;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlNode;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.SchemaConfig;
@@ -37,15 +39,15 @@ import java.util.List;
 import java.util.Set;
 
 public class HiveDatabaseSchema extends AbstractSchema{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveDatabaseSchema.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveDatabaseSchema.class);
 
   private final HiveSchema hiveSchema;
   private Set<String> tables;
   private final DrillHiveMetaStoreClient mClient;
   private final SchemaConfig schemaConfig;
 
-  public HiveDatabaseSchema( //
-      HiveSchema hiveSchema, //
+  public HiveDatabaseSchema(
+      HiveSchema hiveSchema,
       String name,
       DrillHiveMetaStoreClient mClient,
       SchemaConfig schemaConfig) {
@@ -126,6 +128,17 @@ public class HiveDatabaseSchema extends AbstractSchema{
     public Schema.TableType getJdbcTableType() {
       return tableType;
     }
+
+    @Override
+    public boolean rolledUpColumnValidInsideAgg(String column,
+        SqlCall call, SqlNode parent, CalciteConnectionConfig config) {
+      return true;
+    }
+
+    @Override
+    public boolean isRolledUp(String column) {
+      return false;
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/schema/OpenTSDBSchemaFactory.java
----------------------------------------------------------------------
diff --git a/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/schema/OpenTSDBSchemaFactory.java b/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/schema/OpenTSDBSchemaFactory.java
index cca39d8..ca93445 100644
--- a/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/schema/OpenTSDBSchemaFactory.java
+++ b/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/schema/OpenTSDBSchemaFactory.java
@@ -61,7 +61,16 @@ public class OpenTSDBSchemaFactory implements SchemaFactory {
     @Override
     public Table getTable(String name) {
       OpenTSDBScanSpec scanSpec = new OpenTSDBScanSpec(name);
+      try {
         return new DrillOpenTSDBTable(schemaName, plugin, new Schema(plugin.getClient(), name), scanSpec);
+      } catch (Exception e) {
+        // Calcite firstly is looking for a table in the default schema, if a table was not found,
+        // it is looking in root schema.
+        // If a table does not exist, a query will fail at validation stage,
+        // so the error should not be thrown there.
+        logger.warn("Failure while loading table '{}' for database '{}'.", name, schemaName, e.getCause());
+        return null;
+      }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
index 502c5b3..813461d 100644
--- a/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
+++ b/exec/java-exec/src/main/codegen/includes/parserImpls.ftl
@@ -154,19 +154,19 @@ SqlNodeList ParseOptionalFieldList(String relType) :
 /** Parses a required field list and makes sure no field is a "*". */
 SqlNodeList ParseRequiredFieldList(String relType) :
 {
-    SqlNodeList fieldList;
+    Pair<SqlNodeList, SqlNodeList> fieldList;
 }
 {
     <LPAREN>
     fieldList = ParenthesizedCompoundIdentifierList()
     <RPAREN>
     {
-        for(SqlNode node : fieldList)
+        for(SqlNode node : fieldList.left)
         {
-            if (((SqlIdentifier)node).isStar())
+            if (((SqlIdentifier) node).isStar())
                 throw new ParseException(String.format("%s's field list has a '*', which is invalid.", relType));
         }
-        return fieldList;
+        return fieldList.left;
     }
 }
 
@@ -357,7 +357,7 @@ SqlNode SqlDropFunction() :
 /**
 * Parses a comma-separated list of simple identifiers.
 */
-SqlNodeList ParenthesizedCompoundIdentifierList() :
+Pair<SqlNodeList, SqlNodeList> ParenthesizedCompoundIdentifierList() :
 {
     List<SqlIdentifier> list = new ArrayList<SqlIdentifier>();
     SqlIdentifier id;
@@ -367,7 +367,7 @@ SqlNodeList ParenthesizedCompoundIdentifierList() :
     (
    <COMMA> id = SimpleIdentifier() {list.add(id);}) *
     {
-       return new SqlNodeList(list, getPos());
+       return Pair.of(new SqlNodeList(list, getPos()), null);
     }
 }
 </#if>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/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 21ff9a9..7ff286f 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
@@ -32,12 +32,14 @@ import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.Window;
 import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.fun.SqlCountAggFunction;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.trace.CalciteTrace;
+import org.apache.drill.exec.expr.fn.DrillFuncHolder;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.planner.sql.DrillCalciteSqlAggFunctionWrapper;
 import org.apache.drill.exec.planner.sql.DrillSqlOperator;
@@ -234,7 +236,7 @@ public class DrillReduceAggregatesRule extends RelOptRule {
       return reduceSum(oldAggRel, oldCall, newCalls, aggCallMapping);
     }
     if (sqlAggFunction instanceof SqlAvgAggFunction) {
-      final SqlAvgAggFunction.Subtype subtype = ((SqlAvgAggFunction) sqlAggFunction).getSubtype();
+      final SqlKind subtype = sqlAggFunction.getKind();
       switch (subtype) {
       case AVG:
         // replace original AVG(x) with SUM(x) / COUNT(x)
@@ -314,14 +316,21 @@ public class DrillReduceAggregatesRule extends RelOptRule {
             oldAggRel.getInput(),
             iAvgInput);
     RelDataType sumType =
+        TypeInferenceUtils.getDrillSqlReturnTypeInference(SqlKind.SUM.name(),
+            ImmutableList.<DrillFuncHolder>of())
+          .inferReturnType(oldCall.createBinding(oldAggRel));
+    sumType =
         typeFactory.createTypeWithNullability(
-            avgInputType,
-            avgInputType.isNullable() || nGroups == 0);
-    SqlAggFunction sumAgg = new SqlSumEmptyIsZeroAggFunction();
-    AggregateCall sumCall = AggregateCall.create(sumAgg, oldCall.isDistinct(), oldCall.getArgList(), -1, sumType, null);
+            sumType,
+            sumType.isNullable() || nGroups == 0);
+    SqlAggFunction sumAgg =
+        new DrillCalciteSqlAggFunctionWrapper(new SqlSumEmptyIsZeroAggFunction(), sumType);
+    AggregateCall sumCall = AggregateCall.create(sumAgg, oldCall.isDistinct(),
+        oldCall.isApproximate(), oldCall.getArgList(), -1, sumType, null);
     final SqlCountAggFunction countAgg = (SqlCountAggFunction) SqlStdOperatorTable.COUNT;
     final RelDataType countType = countAgg.getReturnType(typeFactory);
-    AggregateCall countCall = AggregateCall.create(countAgg, oldCall.isDistinct(), oldCall.getArgList(), -1, countType, null);
+    AggregateCall countCall = AggregateCall.create(countAgg, oldCall.isDistinct(),
+        oldCall.isApproximate(), oldCall.getArgList(), -1, countType, null);
 
     RexNode tmpsumRef =
         rexBuilder.addAggCall(
@@ -370,7 +379,7 @@ public class DrillReduceAggregatesRule extends RelOptRule {
             newCalls,
             aggCallMapping,
             ImmutableList.of(avgInputType));
-    if(isInferenceEnabled) {
+    if (isInferenceEnabled) {
       return rexBuilder.makeCall(
           new DrillSqlOperator(
               "divide",
@@ -408,20 +417,21 @@ public class DrillReduceAggregatesRule extends RelOptRule {
             arg);
     final RelDataType sumType;
     final SqlAggFunction sumZeroAgg;
-    if(isInferenceEnabled) {
+    if (isInferenceEnabled) {
       sumType = oldCall.getType();
-      sumZeroAgg = new DrillCalciteSqlAggFunctionWrapper(
-          new SqlSumEmptyIsZeroAggFunction(), sumType);
     } else {
       sumType =
           typeFactory.createTypeWithNullability(
-              argType, argType.isNullable());
-      sumZeroAgg = new SqlSumEmptyIsZeroAggFunction();
+              oldCall.getType(), argType.isNullable());
     }
-    AggregateCall sumZeroCall =AggregateCall.create(sumZeroAgg, oldCall.isDistinct(), oldCall.getArgList(), -1, sumType, null);
+    sumZeroAgg = new DrillCalciteSqlAggFunctionWrapper(
+        new SqlSumEmptyIsZeroAggFunction(), sumType);
+    AggregateCall sumZeroCall = AggregateCall.create(sumZeroAgg, oldCall.isDistinct(),
+        oldCall.isApproximate(), oldCall.getArgList(), -1, sumType, null);
     final SqlCountAggFunction countAgg = (SqlCountAggFunction) SqlStdOperatorTable.COUNT;
     final RelDataType countType = countAgg.getReturnType(typeFactory);
-    AggregateCall countCall = AggregateCall.create(countAgg, oldCall.isDistinct(), oldCall.getArgList(), -1, countType, null);
+    AggregateCall countCall = AggregateCall.create(countAgg, oldCall.isDistinct(),
+        oldCall.isApproximate(), oldCall.getArgList(), -1, countType, null);
     // NOTE:  these references are with respect to the output
     // of newAggRel
     RexNode sumZeroRef =
@@ -495,14 +505,17 @@ public class DrillReduceAggregatesRule extends RelOptRule {
             SqlStdOperatorTable.MULTIPLY, argRef, argRef);
     final int argSquaredOrdinal = lookupOrAdd(inputExprs, argSquared);
 
-    final RelDataType sumType =
-        typeFactory.createTypeWithNullability(
-            argType,
-            true);
+    RelDataType sumType =
+        TypeInferenceUtils.getDrillSqlReturnTypeInference(SqlKind.SUM.name(),
+            ImmutableList.<DrillFuncHolder>of())
+          .inferReturnType(oldCall.createBinding(oldAggRel));
+    sumType = typeFactory.createTypeWithNullability(sumType, true);
     final AggregateCall sumArgSquaredAggCall =
         AggregateCall.create(
-            new SqlSumAggFunction(sumType),
+            new DrillCalciteSqlAggFunctionWrapper(
+                new SqlSumAggFunction(sumType), sumType),
             oldCall.isDistinct(),
+            oldCall.isApproximate(),
             ImmutableIntList.of(argSquaredOrdinal),
             -1,
             sumType,
@@ -518,8 +531,10 @@ public class DrillReduceAggregatesRule extends RelOptRule {
 
     final AggregateCall sumArgAggCall =
         AggregateCall.create(
-            new SqlSumAggFunction(sumType),
+            new DrillCalciteSqlAggFunctionWrapper(
+                new SqlSumAggFunction(sumType), sumType),
             oldCall.isDistinct(),
+            oldCall.isApproximate(),
             ImmutableIntList.of(argOrdinal),
             -1,
             sumType,
@@ -539,7 +554,8 @@ public class DrillReduceAggregatesRule extends RelOptRule {
 
     final SqlCountAggFunction countAgg = (SqlCountAggFunction) SqlStdOperatorTable.COUNT;
     final RelDataType countType = countAgg.getReturnType(typeFactory);
-    final AggregateCall countArgAggCall = AggregateCall.create(countAgg, oldCall.isDistinct(), oldCall.getArgList(), -1, countType, null);
+    final AggregateCall countArgAggCall = AggregateCall.create(countAgg, oldCall.isDistinct(),
+        oldCall.isApproximate(), oldCall.getArgList(), -1, countType, null);
     final RexNode countArg =
         rexBuilder.addAggCall(
             countArgAggCall,
@@ -566,7 +582,7 @@ public class DrillReduceAggregatesRule extends RelOptRule {
       final RexLiteral one =
           rexBuilder.makeExactLiteral(BigDecimal.ONE);
       final RexNode nul =
-          rexBuilder.makeNullLiteral(countArg.getType().getSqlTypeName());
+          rexBuilder.makeNullLiteral(countArg.getType());
       final RexNode countMinusOne =
           rexBuilder.makeCall(
               SqlStdOperatorTable.MINUS, countArg, one);
@@ -580,7 +596,7 @@ public class DrillReduceAggregatesRule extends RelOptRule {
     }
 
     final SqlOperator divide;
-    if(isInferenceEnabled) {
+    if (isInferenceEnabled) {
       divide = new DrillSqlOperator(
           "divide",
           2,
@@ -603,7 +619,7 @@ public class DrillReduceAggregatesRule extends RelOptRule {
               SqlStdOperatorTable.POWER, div, half);
     }
 
-    if(isInferenceEnabled) {
+    if (isInferenceEnabled) {
       return result;
     } else {
      /*
@@ -670,7 +686,7 @@ public class DrillReduceAggregatesRule extends RelOptRule {
     public boolean matches(RelOptRuleCall call) {
       DrillAggregateRel oldAggRel = (DrillAggregateRel) call.rels[0];
       for (AggregateCall aggregateCall : oldAggRel.getAggCallList()) {
-        if(isConversionToSumZeroNeeded(aggregateCall.getAggregation(), aggregateCall.getType())) {
+        if (isConversionToSumZeroNeeded(aggregateCall.getAggregation(), aggregateCall.getType())) {
           return true;
         }
       }
@@ -684,7 +700,7 @@ public class DrillReduceAggregatesRule extends RelOptRule {
       final Map<AggregateCall, RexNode> aggCallMapping = Maps.newHashMap();
       final List<AggregateCall> newAggregateCalls = Lists.newArrayList();
       for (AggregateCall oldAggregateCall : oldAggRel.getAggCallList()) {
-        if(isConversionToSumZeroNeeded(oldAggregateCall.getAggregation(), oldAggregateCall.getType())) {
+        if (isConversionToSumZeroNeeded(oldAggregateCall.getAggregation(), oldAggregateCall.getType())) {
           final RelDataType argType = oldAggregateCall.getType();
           final RelDataType sumType = oldAggRel.getCluster().getTypeFactory()
               .createTypeWithNullability(argType, argType.isNullable());
@@ -694,6 +710,7 @@ public class DrillReduceAggregatesRule extends RelOptRule {
               AggregateCall.create(
                   sumZeroAgg,
                   oldAggregateCall.isDistinct(),
+                  oldAggregateCall.isApproximate(),
                   oldAggregateCall.getArgList(),
                   -1,
                   sumType,
@@ -733,9 +750,9 @@ public class DrillReduceAggregatesRule extends RelOptRule {
     @Override
     public boolean matches(RelOptRuleCall call) {
       final DrillWindowRel oldWinRel = (DrillWindowRel) call.rels[0];
-      for(Window.Group group : oldWinRel.groups) {
-        for(Window.RexWinAggCall rexWinAggCall : group.aggCalls) {
-          if(isConversionToSumZeroNeeded(rexWinAggCall.getOperator(), rexWinAggCall.getType())) {
+      for (Window.Group group : oldWinRel.groups) {
+        for (Window.RexWinAggCall rexWinAggCall : group.aggCalls) {
+          if (isConversionToSumZeroNeeded(rexWinAggCall.getOperator(), rexWinAggCall.getType())) {
             return true;
           }
         }
@@ -748,10 +765,10 @@ public class DrillReduceAggregatesRule extends RelOptRule {
       final DrillWindowRel oldWinRel = (DrillWindowRel) call.rels[0];
       final ImmutableList.Builder<Window.Group> builder = ImmutableList.builder();
 
-      for(Window.Group group : oldWinRel.groups) {
+      for (Window.Group group : oldWinRel.groups) {
         final List<Window.RexWinAggCall> aggCalls = Lists.newArrayList();
-        for(Window.RexWinAggCall rexWinAggCall : group.aggCalls) {
-          if(isConversionToSumZeroNeeded(rexWinAggCall.getOperator(), rexWinAggCall.getType())) {
+        for (Window.RexWinAggCall rexWinAggCall : group.aggCalls) {
+          if (isConversionToSumZeroNeeded(rexWinAggCall.getOperator(), rexWinAggCall.getType())) {
             final RelDataType argType = rexWinAggCall.getType();
             final RelDataType sumType = oldWinRel.getCluster().getTypeFactory()
                 .createTypeWithNullability(argType, argType.isNullable());
@@ -792,7 +809,7 @@ public class DrillReduceAggregatesRule extends RelOptRule {
 
   private static boolean isConversionToSumZeroNeeded(SqlOperator sqlOperator, RelDataType type) {
     sqlOperator = DrillCalciteWrapperUtility.extractSqlOperatorFromWrapper(sqlOperator);
-    if(sqlOperator instanceof SqlSumAggFunction
+    if (sqlOperator instanceof SqlSumAggFunction
         && !type.isNullable()) {
       // If SUM(x) is not nullable, the validator must have determined that
       // nulls are impossible (because the group is never empty and x is never

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
index 96043d3..9a0d369 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.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,12 +19,15 @@ package org.apache.drill.exec.planner.logical;
 
 import java.io.IOException;
 
+import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.schema.Schema.TableType;
 import org.apache.calcite.schema.Statistic;
 import org.apache.calcite.schema.Statistics;
 import org.apache.calcite.schema.Table;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlNode;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.GroupScan;
@@ -124,6 +127,16 @@ public abstract class DrillTable implements Table {
   }
 
   @Override
+  public boolean rolledUpColumnValidInsideAgg(String column,
+      SqlCall call, SqlNode parent, CalciteConnectionConfig config) {
+    return true;
+  }
+
+  @Override public boolean isRolledUp(String column) {
+    return false;
+  }
+
+  @Override
   public int hashCode() {
     final int prime = 31;
     int result = 1;

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTranslatableTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTranslatableTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTranslatableTable.java
index 8ec805f..6576c25 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTranslatableTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTranslatableTable.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,6 +17,7 @@
  */
 package org.apache.drill.exec.planner.logical;
 
+import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelOptTable.ToRelContext;
 import org.apache.calcite.rel.RelNode;
@@ -25,7 +26,8 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.Statistic;
 import org.apache.calcite.schema.TranslatableTable;
 import org.apache.calcite.schema.Schema.TableType;
-import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlNode;
 
 /**
  * TableMacros must return a TranslatableTable
@@ -65,6 +67,17 @@ public class DrillTranslatableTable implements TranslatableTable {
   }
 
   @Override
+  public boolean rolledUpColumnValidInsideAgg(String column,
+      SqlCall call, SqlNode parent, CalciteConnectionConfig config) {
+    return true;
+  }
+
+  @Override
+  public boolean isRolledUp(String column) {
+    return false;
+  }
+
+  @Override
   public int hashCode() {
     return drillTable.hashCode();
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillViewTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillViewTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillViewTable.java
index 0f0278e..82bdc56 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillViewTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillViewTable.java
@@ -18,11 +18,14 @@
 package org.apache.drill.exec.planner.logical;
 
 import com.google.common.collect.ImmutableList;
+import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.schema.Schema.TableType;
 import org.apache.calcite.schema.Statistic;
 import org.apache.calcite.schema.Statistics;
 import org.apache.calcite.schema.TranslatableTable;
 
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlNode;
 import org.apache.drill.exec.dotdrill.View;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.plan.RelOptTable;
@@ -91,4 +94,14 @@ public class DrillViewTable implements TranslatableTable, DrillViewInfoProvider
   public String getViewSql() {
     return view.getSql();
   }
+
+  @Override
+  public boolean rolledUpColumnValidInsideAgg(String column,
+      SqlCall call, SqlNode parent, CalciteConnectionConfig config) {
+    return true;
+  }
+
+  @Override public boolean isRolledUp(String column) {
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.java
index 3de5fca..84e37fc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.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.physical;
 
 import java.util.List;
 
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.util.BitSets;
 
 import org.apache.drill.exec.planner.logical.DrillAggregateRel;
@@ -71,8 +72,11 @@ public abstract class AggPruleBase extends Prule {
 
     for (AggregateCall aggCall : aggregate.getAggCallList()) {
       String name = aggCall.getAggregation().getName();
-      if ( ! (name.equals("SUM") || name.equals("MIN") || name.equals("MAX") || name.equals("COUNT")
-              || name.equals("$SUM0"))) {
+      if (!(name.equals(SqlKind.SUM.name())
+          || name.equals(SqlKind.MIN.name())
+          || name.equals(SqlKind.MAX.name())
+          || name.equals(SqlKind.COUNT.name())
+          || name.equals("$SUM0"))) {
         return false;
       }
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillAvgVarianceConvertlet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillAvgVarianceConvertlet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillAvgVarianceConvertlet.java
index bfb4c05..f25ceee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillAvgVarianceConvertlet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillAvgVarianceConvertlet.java
@@ -20,11 +20,11 @@ package org.apache.drill.exec.planner.sql;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNumericLiteral;
 import org.apache.calcite.sql.SqlOperatorBinding;
-import org.apache.calcite.sql.fun.SqlAvgAggFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
@@ -42,7 +42,7 @@ import org.apache.calcite.util.Util;
  */
 public class DrillAvgVarianceConvertlet implements SqlRexConvertlet {
 
-  private final SqlAvgAggFunction.Subtype subtype;
+  private final SqlKind subtype;
   private static final DrillSqlOperator CastHighOp = new DrillSqlOperator("CastHigh", 1, false,
       new SqlReturnTypeInference() {
         @Override
@@ -54,7 +54,7 @@ public class DrillAvgVarianceConvertlet implements SqlRexConvertlet {
         }
       }, false);
 
-  public DrillAvgVarianceConvertlet(SqlAvgAggFunction.Subtype subtype) {
+  public DrillAvgVarianceConvertlet(SqlKind subtype) {
     this.subtype = subtype;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillConvertletTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillConvertletTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillConvertletTable.java
index 511eed7..7b66d12 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillConvertletTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillConvertletTable.java
@@ -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,8 +21,8 @@ import java.util.HashMap;
 
 import org.apache.calcite.sql.SqlBasicCall;
 import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.fun.SqlAvgAggFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql2rel.SqlRexConvertlet;
 import org.apache.calcite.sql2rel.SqlRexConvertletTable;
@@ -38,11 +38,13 @@ public class DrillConvertletTable implements SqlRexConvertletTable{
   static {
     // Use custom convertlet for extract function
     map.put(SqlStdOperatorTable.EXTRACT, DrillExtractConvertlet.INSTANCE);
-    map.put(SqlStdOperatorTable.AVG, new DrillAvgVarianceConvertlet(SqlAvgAggFunction.Subtype.AVG));
-    map.put(SqlStdOperatorTable.STDDEV_POP, new DrillAvgVarianceConvertlet(SqlAvgAggFunction.Subtype.STDDEV_POP));
-    map.put(SqlStdOperatorTable.STDDEV_SAMP, new DrillAvgVarianceConvertlet(SqlAvgAggFunction.Subtype.STDDEV_SAMP));
-    map.put(SqlStdOperatorTable.VAR_POP, new DrillAvgVarianceConvertlet(SqlAvgAggFunction.Subtype.VAR_POP));
-    map.put(SqlStdOperatorTable.VAR_SAMP, new DrillAvgVarianceConvertlet(SqlAvgAggFunction.Subtype.VAR_SAMP));
+    map.put(SqlStdOperatorTable.AVG, new DrillAvgVarianceConvertlet(SqlKind.AVG));
+    map.put(SqlStdOperatorTable.STDDEV_POP, new DrillAvgVarianceConvertlet(SqlKind.STDDEV_POP));
+    map.put(SqlStdOperatorTable.STDDEV_SAMP, new DrillAvgVarianceConvertlet(SqlKind.STDDEV_SAMP));
+    map.put(SqlStdOperatorTable.STDDEV, new DrillAvgVarianceConvertlet(SqlKind.STDDEV_SAMP));
+    map.put(SqlStdOperatorTable.VAR_POP, new DrillAvgVarianceConvertlet(SqlKind.VAR_POP));
+    map.put(SqlStdOperatorTable.VAR_SAMP, new DrillAvgVarianceConvertlet(SqlKind.VAR_SAMP));
+    map.put(SqlStdOperatorTable.VARIANCE, new DrillAvgVarianceConvertlet(SqlKind.VAR_SAMP));
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/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 8224d97..ae9b37f 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
@@ -19,6 +19,7 @@ package org.apache.drill.exec.planner.sql;
 
 import java.util.Arrays;
 import java.util.List;
+import java.util.Properties;
 import java.util.Set;
 
 import com.google.common.base.Strings;
@@ -26,6 +27,8 @@ 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;
 import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 import org.apache.calcite.plan.ConventionTraitDef;
@@ -537,7 +540,8 @@ public class SqlConverter {
                               JavaTypeFactory typeFactory,
                               DrillConfig drillConfig,
                               UserSession session) {
-      super(CalciteSchema.from(rootSchema), caseSensitive, defaultSchema, typeFactory);
+      super(CalciteSchema.from(rootSchema), defaultSchema,
+          typeFactory, getConnectionConfig(caseSensitive));
       this.drillConfig = drillConfig;
       this.session = session;
       this.allowTemporaryTables = true;
@@ -650,4 +654,17 @@ public class SqlConverter {
               SchemaUtilites.SCHEMA_PATH_JOINER.join(defaultSchemaPath, schemaPath), drillConfig);
     }
   }
+
+  /**
+   * Creates {@link CalciteConnectionConfigImpl} instance with specified caseSensitive property.
+   *
+   * @param caseSensitive is case sensitive.
+   * @return {@link CalciteConnectionConfigImpl} instance
+   */
+  private static CalciteConnectionConfigImpl getConnectionConfig(boolean caseSensitive) {
+    Properties properties = new Properties();
+    properties.setProperty(CalciteConnectionProperty.CASE_SENSITIVE.camelName(),
+        String.valueOf(caseSensitive));
+    return new CalciteConnectionConfigImpl(properties);
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/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 ddf48ed..af544b5 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
@@ -32,7 +32,6 @@ import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNumericLiteral;
 import org.apache.calcite.sql.SqlOperatorBinding;
-import org.apache.calcite.sql.fun.SqlAvgAggFunction;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeFamily;
@@ -134,21 +133,21 @@ public class TypeInferenceUtils {
 
   private static final ImmutableMap<String, SqlReturnTypeInference> funcNameToInference = ImmutableMap.<String, SqlReturnTypeInference> builder()
       .put("DATE_PART", DrillDatePartSqlReturnTypeInference.INSTANCE)
-      .put("SUM", DrillSumSqlReturnTypeInference.INSTANCE)
-      .put("COUNT", DrillCountSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.SUM.name(), DrillSumSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.COUNT.name(), DrillCountSqlReturnTypeInference.INSTANCE)
       .put("CONCAT", DrillConcatSqlReturnTypeInference.INSTANCE_CONCAT)
       .put("CONCATOPERATOR", DrillConcatSqlReturnTypeInference.INSTANCE_CONCAT_OP)
       .put("LENGTH", DrillLengthSqlReturnTypeInference.INSTANCE)
       .put("LPAD", DrillPadSqlReturnTypeInference.INSTANCE)
       .put("RPAD", DrillPadSqlReturnTypeInference.INSTANCE)
-      .put("LTRIM", DrillTrimSqlReturnTypeInference.INSTANCE)
-      .put("RTRIM", DrillTrimSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.LTRIM.name(), DrillTrimSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.RTRIM.name(), DrillTrimSqlReturnTypeInference.INSTANCE)
       .put("BTRIM", DrillTrimSqlReturnTypeInference.INSTANCE)
-      .put("TRIM", DrillTrimSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.TRIM.name(), DrillTrimSqlReturnTypeInference.INSTANCE)
       .put("CONVERT_TO", DrillConvertToSqlReturnTypeInference.INSTANCE)
-      .put("EXTRACT", DrillExtractSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.EXTRACT.name(), DrillExtractSqlReturnTypeInference.INSTANCE)
       .put("SQRT", DrillSqrtSqlReturnTypeInference.INSTANCE)
-      .put("CAST", DrillCastSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.CAST.name(), DrillCastSqlReturnTypeInference.INSTANCE)
       .put("FLATTEN", DrillDeferToExecSqlReturnTypeInference.INSTANCE)
       .put("KVGEN", DrillDeferToExecSqlReturnTypeInference.INSTANCE)
       .put("CONVERT_FROM", DrillDeferToExecSqlReturnTypeInference.INSTANCE)
@@ -168,22 +167,22 @@ public class TypeInferenceUtils {
       .put(SqlKind.ROW_NUMBER.name(), DrillRankingSqlReturnTypeInference.INSTANCE_BIGINT)
 
       // NTILE
-      .put("NTILE", DrillNTILESqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.NTILE.name(), DrillNTILESqlReturnTypeInference.INSTANCE)
 
       // LEAD, LAG
-      .put("LEAD", DrillLeadLagSqlReturnTypeInference.INSTANCE)
-      .put("LAG", DrillLeadLagSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.LEAD.name(), DrillLeadLagSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.LAG.name(), DrillLeadLagSqlReturnTypeInference.INSTANCE)
 
       // FIRST_VALUE, LAST_VALUE
-      .put("FIRST_VALUE", DrillSameSqlReturnTypeInference.INSTANCE)
-      .put("LAST_VALUE", DrillSameSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.FIRST_VALUE.name(), DrillSameSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.LAST_VALUE.name(), DrillSameSqlReturnTypeInference.INSTANCE)
 
       // Functions rely on DrillReduceAggregatesRule for expression simplification as opposed to getting evaluated directly
-      .put(SqlAvgAggFunction.Subtype.AVG.name(), DrillAvgAggSqlReturnTypeInference.INSTANCE)
-      .put(SqlAvgAggFunction.Subtype.STDDEV_POP.name(), DrillAvgAggSqlReturnTypeInference.INSTANCE)
-      .put(SqlAvgAggFunction.Subtype.STDDEV_SAMP.name(), DrillAvgAggSqlReturnTypeInference.INSTANCE)
-      .put(SqlAvgAggFunction.Subtype.VAR_POP.name(), DrillAvgAggSqlReturnTypeInference.INSTANCE)
-      .put(SqlAvgAggFunction.Subtype.VAR_SAMP.name(), DrillAvgAggSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.AVG.name(), DrillAvgAggSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.STDDEV_POP.name(), DrillAvgAggSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.STDDEV_SAMP.name(), DrillAvgAggSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.VAR_POP.name(), DrillAvgAggSqlReturnTypeInference.INSTANCE)
+      .put(SqlKind.VAR_SAMP.name(), DrillAvgAggSqlReturnTypeInference.INSTANCE)
       .build();
 
   /**

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
index c4586ef..ed02db9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
@@ -29,6 +29,7 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.SchemaVersion;
 import org.apache.calcite.schema.Table;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.common.exceptions.UserException;
@@ -197,17 +198,12 @@ public abstract class AbstractSchema implements Schema, SchemaPartitionExplorer,
   }
 
   @Override
-  public boolean contentsHaveChangedSince(long lastCheck, long now) {
-    return true;
-  }
-
-  @Override
   public void close() throws Exception {
     // no-op: default implementation for most implementations.
   }
 
   @Override
-  public Schema snapshot(long now) {
+  public Schema snapshot(SchemaVersion version) {
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
index 90644b5..0edf65f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
@@ -22,7 +22,9 @@ import java.net.URL;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
+import java.util.WeakHashMap;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -114,6 +116,7 @@ public class MockStorageEngine extends AbstractStoragePlugin {
   private static class MockSchema extends AbstractSchema {
 
     private MockStorageEngine engine;
+    private final Map<String, Table> tableCache = new WeakHashMap<>();
 
     public MockSchema(MockStorageEngine engine) {
       super(ImmutableList.<String>of(), MockStorageEngineConfig.NAME);
@@ -122,11 +125,16 @@ public class MockStorageEngine extends AbstractStoragePlugin {
 
     @Override
     public Table getTable(String name) {
-      if (name.toLowerCase().endsWith(".json")) {
-        return getConfigFile(name);
-      } else {
-        return getDirectTable(name);
+      Table table = tableCache.get(name);
+      if (table == null) {
+        if (name.toLowerCase().endsWith(".json")) {
+          table = getConfigFile(name);
+        } else {
+          table = getDirectTable(name);
+        }
+        tableCache.put(name, table);
       }
+      return table;
     }
 
     private Table getConfigFile(String name) {

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java b/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java
index e88e5a4..100d194 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestBugFixes.java
@@ -198,7 +198,7 @@ public class TestBugFixes extends BaseTestQuery {
             + " from cp.`employee.json` emp\n"
             + " group by gender";
     final String[] expectedPlans1 = {
-            ".*Agg\\(group=\\[\\{0\\}\\], agg#0=\\[\\$SUM0\\(\\$2\\)\\], agg#1=\\[\\$SUM0\\(\\$1\\)\\], agg#2=\\[COUNT\\(\\$1\\)\\]\\)",
+            ".*Agg\\(group=\\[\\{0\\}\\], cnt=\\[\\$SUM0\\(\\$2\\)\\], agg#1=\\[\\$SUM0\\(\\$1\\)\\], agg#2=\\[COUNT\\(\\$1\\)\\]\\)",
             ".*Agg\\(group=\\[\\{0, 1\\}\\], cnt=\\[COUNT\\(\\)\\]\\)"};
     final String[] excludedPlans1 = {".*Join\\(condition=\\[true\\], joinType=\\[inner\\]\\).*"};
     PlanTestBase.testPlanMatchingPatterns(query1, expectedPlans1, excludedPlans1);

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
index 7fac487..3851228 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
@@ -907,7 +907,7 @@ public class TestWindowFunctions extends BaseTestQuery {
           + " window w1 as (partition by l_suppkey)";
       test(query);
     } catch(UserException ex) {
-      assert(ex.getMessage().contains("Expression 'tpch/nation.parquet.l_suppkey' is not being grouped"));
+      assert(ex.getMessage().contains("Expression 'l_suppkey' is not being grouped"));
     }
 
     try {
@@ -932,7 +932,7 @@ public class TestWindowFunctions extends BaseTestQuery {
           + " window w2 as (partition by n_nationkey)";
       test(query);
     } catch(UserException ex) {
-      assert(ex.getMessage().contains("Expression 'tpch/nation.parquet.n_nationkey' is not being grouped"));
+      assert(ex.getMessage().contains("Expression 'n_nationkey' is not being grouped"));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/d59f0cda/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ce64591..c64788c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -36,7 +36,7 @@
     <dep.guava.version>18.0</dep.guava.version>
     <forkCount>2</forkCount>
     <parquet.version>1.8.1-drill-r0</parquet.version>
-    <calcite.version>1.13.0-drill-r0</calcite.version>
+    <calcite.version>1.15.0-drill-r0</calcite.version>
     <avatica.version>1.10.0</avatica.version>
     <janino.version>2.7.6</janino.version>
     <sqlline.version>1.1.9-drill-r7</sqlline.version>


[05/18] drill git commit: REVERTED: DRILL-5089

Posted by am...@apache.org.
REVERTED:  DRILL-5089

Dynamically load schema of storage plugin only when needed for every query


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

Branch: refs/heads/master
Commit: 450e67094eb6e9a6484d7f86c49b51c77a08d7b2
Parents: ef0fafe
Author: Roman Kulyk <ro...@gmail.com>
Authored: Thu Nov 30 16:19:12 2017 +0000
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Tue Jan 16 12:10:13 2018 +0200

----------------------------------------------------------------------
 .../apache/drill/exec/ops/FragmentContext.java  |   9 +-
 .../org/apache/drill/exec/ops/QueryContext.java |  11 +-
 .../exec/planner/sql/DynamicRootSchema.java     | 142 -------------------
 .../drill/exec/planner/sql/DynamicSchema.java   |  58 --------
 .../drill/exec/planner/sql/SqlConverter.java    |   9 +-
 .../drill/exec/store/SchemaTreeProvider.java    |  31 +---
 .../exec/store/StoragePluginRegistryImpl.java   |   3 -
 .../exec/store/dfs/FileSystemSchemaFactory.java |  23 +--
 .../exec/store/dfs/WorkspaceSchemaFactory.java  |  87 ++++--------
 .../store/ischema/InfoSchemaBatchCreator.java   |   2 +-
 .../exec/store/mock/MockBreakageStorage.java    |  47 ------
 .../exec/store/mock/MockStorageEngine.java      |   7 +-
 .../exec/work/metadata/MetadataProvider.java    |   2 +-
 .../drill/exec/physical/impl/TestSchema.java    |  87 ------------
 .../drill/test/ClusterFixtureBuilder.java       |   4 -
 .../drill/test/ClusterMockStorageFixture.java   |  51 -------
 16 files changed, 46 insertions(+), 527 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/450e6709/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
index 210d0d4..736d550 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
@@ -230,12 +230,7 @@ public class FragmentContext extends BaseFragmentContext implements AutoCloseabl
     return context;
   }
 
-  /**
-   * This method is only used to construt InfoSchemaReader, it is for the reader to get full schema, so here we
-   * are going to return a fully initialized schema tree.
-   * @return root schema's plus
-   */
-  public SchemaPlus getFullRootSchema() {
+  public SchemaPlus getRootSchema() {
     if (queryContext == null) {
       fail(new UnsupportedOperationException("Schema tree can only be created in root fragment. " +
           "This is a non-root fragment."));
@@ -253,7 +248,7 @@ public class FragmentContext extends BaseFragmentContext implements AutoCloseabl
         .setIgnoreAuthErrors(isImpersonationEnabled)
         .build();
 
-    return queryContext.getFullRootSchema(schemaConfig);
+    return queryContext.getRootSchema(schemaConfig);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/drill/blob/450e6709/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index eb32bc6..8dbddbf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -163,23 +163,14 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
   }
 
   /**
-   *  Create and return a SchemaTree with given <i>schemaConfig</i> but some schemas (from storage plugins)
-   *  could be initialized later.
+   *  Create and return a SchemaTree with given <i>schemaConfig</i>.
    * @param schemaConfig
    * @return
    */
   public SchemaPlus getRootSchema(SchemaConfig schemaConfig) {
     return schemaTreeProvider.createRootSchema(schemaConfig);
   }
-  /**
-   *  Create and return a fully initialized SchemaTree with given <i>schemaConfig</i>.
-   * @param schemaConfig
-   * @return
-   */
 
-  public SchemaPlus getFullRootSchema(SchemaConfig schemaConfig) {
-    return schemaTreeProvider.createFullRootSchema(schemaConfig);
-  }
   /**
    * Get the user name of the user who issued the query that is managed by this QueryContext.
    * @return

http://git-wip-us.apache.org/repos/asf/drill/blob/450e6709/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DynamicRootSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DynamicRootSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DynamicRootSchema.java
deleted file mode 100644
index df504b7..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DynamicRootSchema.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.planner.sql;
-
-import com.google.common.collect.ImmutableSortedSet;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import org.apache.calcite.DataContext;
-import org.apache.calcite.jdbc.CalciteRootSchema;
-import org.apache.calcite.jdbc.CalciteSchema;
-
-import org.apache.calcite.linq4j.tree.Expression;
-import org.apache.calcite.linq4j.tree.Expressions;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.impl.AbstractSchema;
-import org.apache.calcite.util.BuiltInMethod;
-import org.apache.calcite.util.Compatible;
-import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.exec.store.SchemaConfig;
-import org.apache.drill.exec.store.StoragePlugin;
-import org.apache.drill.exec.store.StoragePluginRegistry;
-import org.apache.drill.exec.store.SubSchemaWrapper;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableSet;
-import java.util.Set;
-
-/**
- * This class is to allow us loading schemas from storage plugins later when {@link #getSubSchema(String, boolean)}
- * is called.
- */
-public class DynamicRootSchema extends DynamicSchema
-    implements CalciteRootSchema {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DynamicRootSchema.class);
-
-  protected SchemaConfig schemaConfig;
-  protected StoragePluginRegistry storages;
-
-  public StoragePluginRegistry getSchemaFactories() {
-    return storages;
-  }
-
-  /** Creates a root schema. */
-  DynamicRootSchema(StoragePluginRegistry storages, SchemaConfig schemaConfig) {
-    super(null, new RootSchema(), "");
-    this.schemaConfig = schemaConfig;
-    this.storages = storages;
-  }
-
-  @Override
-  public CalciteSchema getSubSchema(String schemaName, boolean caseSensitive) {
-    CalciteSchema retSchema = getSubSchemaMap().get(schemaName);
-    if (retSchema != null) {
-      return retSchema;
-    }
-
-    loadSchemaFactory(schemaName, caseSensitive);
-    retSchema = getSubSchemaMap().get(schemaName);
-    return retSchema;
-  }
-
-  @Override
-  public NavigableSet<String> getTableNames() {
-    return Compatible.INSTANCE.navigableSet(ImmutableSortedSet.<String>of());
-  }
-
-  /**
-   * load schema factory(storage plugin) for schemaName
-   * @param schemaName
-   * @param caseSensitive
-   */
-  public void loadSchemaFactory(String schemaName, boolean caseSensitive) {
-    try {
-      SchemaPlus thisPlus = this.plus();
-      StoragePlugin plugin = getSchemaFactories().getPlugin(schemaName);
-      if (plugin != null) {
-        plugin.registerSchemas(schemaConfig, thisPlus);
-        return;
-      }
-
-      // Could not find the plugin of schemaName. The schemaName could be `dfs.tmp`, a 2nd level schema under 'dfs'
-      String[] paths = schemaName.split("\\.");
-      if (paths.length == 2) {
-        plugin = getSchemaFactories().getPlugin(paths[0]);
-        if (plugin == null) {
-          return;
-        }
-
-        // Found the storage plugin for first part(e.g. 'dfs') of schemaName (e.g. 'dfs.tmp')
-        // register schema for this storage plugin to 'this'.
-        plugin.registerSchemas(schemaConfig, thisPlus);
-
-        // Load second level schemas for this storage plugin
-        final SchemaPlus firstlevelSchema = thisPlus.getSubSchema(paths[0]);
-        final List<SchemaPlus> secondLevelSchemas = Lists.newArrayList();
-        for (String secondLevelSchemaName : firstlevelSchema.getSubSchemaNames()) {
-          secondLevelSchemas.add(firstlevelSchema.getSubSchema(secondLevelSchemaName));
-        }
-
-        for (SchemaPlus schema : secondLevelSchemas) {
-          org.apache.drill.exec.store.AbstractSchema drillSchema;
-          try {
-            drillSchema = schema.unwrap(org.apache.drill.exec.store.AbstractSchema.class);
-          } catch (ClassCastException e) {
-            throw new RuntimeException(String.format("Schema '%s' is not expected under root schema", schema.getName()));
-          }
-          SubSchemaWrapper wrapper = new SubSchemaWrapper(drillSchema);
-          thisPlus.add(wrapper.getName(), wrapper);
-        }
-      }
-    } catch(ExecutionSetupException | IOException ex) {
-      logger.warn("Failed to load schema for \"" + schemaName + "\"!", ex);
-    }
-  }
-
-  static class RootSchema extends AbstractSchema {
-    @Override public Expression getExpression(SchemaPlus parentSchema,
-                                              String name) {
-      return Expressions.call(
-          DataContext.ROOT,
-          BuiltInMethod.DATA_CONTEXT_GET_ROOT_SCHEMA.method);
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/drill/blob/450e6709/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DynamicSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DynamicSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DynamicSchema.java
deleted file mode 100644
index 7211f0d..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DynamicSchema.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.planner.sql;
-
-import org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.calcite.jdbc.SimpleCalciteSchema;
-import org.apache.calcite.schema.Schema;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.drill.exec.store.SchemaConfig;
-import org.apache.drill.exec.store.StoragePluginRegistry;
-
-
-/**
- * Unlike SimpleCalciteSchema, DynamicSchema could have an empty or partial schemaMap, but it could maintain a map of
- * name->SchemaFactory, and only register schema when the corresponsdent name is requested.
- */
-public class DynamicSchema extends SimpleCalciteSchema {
-
-  public DynamicSchema(CalciteSchema parent, Schema schema, String name) {
-    super(parent, schema, name);
-  }
-
-  @Override
-  public CalciteSchema getSubSchema(String schemaName, boolean caseSensitive) {
-    Schema s = schema.getSubSchema(schemaName);
-    if (s != null) {
-      return new DynamicSchema(this, s, schemaName);
-    }
-    CalciteSchema ret = getSubSchemaMap().get(schemaName);
-    return ret;
-  }
-
-  @Override
-  public SchemaPlus plus() {
-    return super.plus();
-  }
-
-  public static SchemaPlus createRootSchema(StoragePluginRegistry storages, SchemaConfig schemaConfig) {
-    DynamicRootSchema rootSchema = new DynamicRootSchema(storages, schemaConfig);
-    return rootSchema.plus();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/450e6709/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 11b90e2..dbe16bd 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
@@ -24,6 +24,7 @@ import java.util.Set;
 import com.google.common.base.Strings;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.CalciteSchemaImpl;
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 import org.apache.calcite.plan.ConventionTraitDef;
 import org.apache.calcite.plan.RelOptCluster;
@@ -116,9 +117,9 @@ public class SqlConverter {
     this.session = context.getSession();
     this.drillConfig = context.getConfig();
     this.catalog = new DrillCalciteCatalogReader(
-        rootSchema,
+        this.rootSchema,
         parserConfig.caseSensitive(),
-        DynamicSchema.from(defaultSchema).path(null),
+        CalciteSchemaImpl.from(defaultSchema).path(null),
         typeFactory,
         drillConfig,
         session);
@@ -296,7 +297,7 @@ public class SqlConverter {
     @Override
     public RelNode expandView(RelDataType rowType, String queryString, SchemaPlus rootSchema, List<String> schemaPath) {
       final DrillCalciteCatalogReader catalogReader = new DrillCalciteCatalogReader(
-          rootSchema,
+          rootSchema, // new root schema
           parserConfig.caseSensitive(),
           schemaPath,
           typeFactory,
@@ -445,7 +446,7 @@ public class SqlConverter {
                               JavaTypeFactory typeFactory,
                               DrillConfig drillConfig,
                               UserSession session) {
-      super(DynamicSchema.from(rootSchema), caseSensitive, defaultSchema, typeFactory);
+      super(CalciteSchemaImpl.from(rootSchema), caseSensitive, defaultSchema, typeFactory);
       this.drillConfig = drillConfig;
       this.session = session;
       this.allowTemporaryTables = true;

http://git-wip-us.apache.org/repos/asf/drill/blob/450e6709/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java
index e1a1ede..23441bd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java
@@ -20,12 +20,12 @@ package org.apache.drill.exec.store;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.calcite.jdbc.SimpleCalciteSchema;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.ViewExpansionContext;
-import org.apache.drill.exec.planner.sql.DynamicSchema;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.server.options.OptionValue;
@@ -105,36 +105,12 @@ public class SchemaTreeProvider implements AutoCloseable {
    * @return
    */
   public SchemaPlus createRootSchema(SchemaConfig schemaConfig) {
-      final SchemaPlus rootSchema = DynamicSchema.createRootSchema(dContext.getStorage(), schemaConfig);
-      schemaTreesToClose.add(rootSchema);
-      return rootSchema;
-  }
-
-  /**
-   * Return full root schema with schema owner as the given user.
-   *
-   * @param userName Name of the user who is accessing the storage sources.
-   * @param provider {@link SchemaConfigInfoProvider} instance
-   * @return Root of the schema tree.
-   */
-  public SchemaPlus createFullRootSchema(final String userName, final SchemaConfigInfoProvider provider) {
-    final String schemaUser = isImpersonationEnabled ? userName : ImpersonationUtil.getProcessUserName();
-    final SchemaConfig schemaConfig = SchemaConfig.newBuilder(schemaUser, provider).build();
-    return createFullRootSchema(schemaConfig);
-  }
-  /**
-   * Create and return a Full SchemaTree with given <i>schemaConfig</i>.
-   * @param schemaConfig
-   * @return
-   */
-  public SchemaPlus createFullRootSchema(SchemaConfig schemaConfig) {
     try {
-      final SchemaPlus rootSchema = DynamicSchema.createRootSchema(dContext.getStorage(), schemaConfig);
+      final SchemaPlus rootSchema = SimpleCalciteSchema.createRootSchema(false);
       dContext.getSchemaFactory().registerSchemas(schemaConfig, rootSchema);
       schemaTreesToClose.add(rootSchema);
       return rootSchema;
-    }
-    catch(IOException e) {
+    } catch(IOException e) {
       // We can't proceed further without a schema, throw a runtime exception.
       // Improve the error message for client side.
 
@@ -148,7 +124,6 @@ public class SchemaTreeProvider implements AutoCloseable {
           .addContext(contextString)
           .build(logger);
     }
-
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/450e6709/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java
index 2b4d648..3fb1c3a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java
@@ -46,7 +46,6 @@ import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.exception.StoreException;
 import org.apache.drill.exec.planner.logical.StoragePlugins;
-import org.apache.drill.exec.planner.sql.DynamicSchema;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.dfs.FileSystemPlugin;
 import org.apache.drill.exec.store.dfs.FormatPlugin;
@@ -495,6 +494,4 @@ public class StoragePluginRegistryImpl implements StoragePluginRegistry {
     return availablePlugins;
   }
 
-
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/450e6709/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
----------------------------------------------------------------------
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 6d88d04..5d99377 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
@@ -27,7 +27,6 @@ import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
 
-import org.apache.drill.exec.store.StoragePlugin;
 import org.apache.drill.exec.store.StorageStrategy;
 import org.apache.drill.exec.planner.logical.CreateTableEntry;
 import org.apache.drill.exec.store.AbstractSchema;
@@ -39,9 +38,7 @@ import org.apache.drill.exec.store.dfs.WorkspaceSchemaFactory.WorkspaceSchema;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Maps;
 import org.apache.drill.exec.util.DrillFileSystemUtil;
-import org.apache.drill.exec.util.ImpersonationUtil;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 
@@ -52,23 +49,11 @@ public class FileSystemSchemaFactory implements SchemaFactory{
 
   public static final String DEFAULT_WS_NAME = "default";
 
-  public static final String LOCAL_FS_SCHEME = "file";
-
   private List<WorkspaceSchemaFactory> factories;
   private String schemaName;
-  protected FileSystemPlugin plugin;
 
   public FileSystemSchemaFactory(String schemaName, List<WorkspaceSchemaFactory> factories) {
-    // when the correspondent FileSystemPlugin is not passed in, we dig into ANY workspace factory to get it.
-    if (factories.size() > 0) {
-      this.plugin = factories.get(0).getPlugin();
-    }
-    this.schemaName = schemaName;
-    this.factories = factories;
-  }
-
-  public FileSystemSchemaFactory(FileSystemPlugin plugin, String schemaName, List<WorkspaceSchemaFactory> factories) {
-    this.plugin = plugin;
+    super();
     this.schemaName = schemaName;
     this.factories = factories;
   }
@@ -88,10 +73,10 @@ public class FileSystemSchemaFactory implements SchemaFactory{
 
     public FileSystemSchema(String name, SchemaConfig schemaConfig) throws IOException {
       super(ImmutableList.<String>of(), name);
-      final DrillFileSystem fs = ImpersonationUtil.createFileSystem(schemaConfig.getUserName(), plugin.getFsConf());
       for(WorkspaceSchemaFactory f :  factories){
-        WorkspaceSchema s = f.createSchema(getSchemaPath(), schemaConfig, fs);
-        if (s != null) {
+        if (f.accessible(schemaConfig.getUserName())) {
+          @SuppressWarnings("resource")
+          WorkspaceSchema s = f.createSchema(getSchemaPath(), schemaConfig);
           schemaMap.put(s.getName(), s);
         }
       }

http://git-wip-us.apache.org/repos/asf/drill/blob/450e6709/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
index 3934958..bbf013d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
@@ -42,7 +42,6 @@ import org.apache.calcite.schema.FunctionParameter;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.TableMacro;
 import org.apache.calcite.schema.TranslatableTable;
-import org.apache.commons.lang3.SystemUtils;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.common.config.LogicalPlanPersistence;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -71,7 +70,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.AccessControlException;
 
@@ -152,30 +150,14 @@ public class WorkspaceSchemaFactory {
    * @return True if the user has access. False otherwise.
    */
   public boolean accessible(final String userName) throws IOException {
-    final DrillFileSystem fs = ImpersonationUtil.createFileSystem(userName, fsConf);
-    return accessible(fs);
-  }
-
-  /**
-   * Checks whether a FileSystem object has the permission to list/read workspace directory
-   * @param fs a DrillFileSystem object that was created with certain user privilege
-   * @return True if the user has access. False otherwise.
-   * @throws IOException
-   */
-  public boolean accessible(DrillFileSystem fs) throws IOException {
+    final FileSystem fs = ImpersonationUtil.createFileSystem(userName, fsConf);
     try {
-      /**
-       * For Windows local file system, fs.access ends up using DeprecatedRawLocalFileStatus which has
-       * TrustedInstaller as owner, and a member of Administrators group could not satisfy the permission.
-       * In this case, we will still use method listStatus.
-       * In other cases, we use access method since it is cheaper.
-       */
-      if (SystemUtils.IS_OS_WINDOWS && fs.getUri().getScheme().equalsIgnoreCase(FileSystemSchemaFactory.LOCAL_FS_SCHEME)) {
+      // We have to rely on the listStatus as a FileSystem can have complicated controls such as regular unix style
+      // permissions, Access Control Lists (ACLs) or Access Control Expressions (ACE). Hadoop 2.7 version of FileSystem
+      // has a limited private API (FileSystem.access) to check the permissions directly
+      // (see https://issues.apache.org/jira/browse/HDFS-6570). Drill currently relies on Hadoop 2.5.0 version of
+      // FileClient. TODO: Update this when DRILL-3749 is fixed.
       fs.listStatus(wsPath);
-      }
-      else {
-        fs.access(wsPath, FsAction.READ);
-      }
     } catch (final UnsupportedOperationException e) {
       logger.trace("The filesystem for this workspace does not support this operation.", e);
     } catch (final FileNotFoundException | AccessControlException e) {
@@ -189,19 +171,8 @@ public class WorkspaceSchemaFactory {
     return DotDrillType.VIEW.getPath(config.getLocation(), name);
   }
 
-  public WorkspaceSchema createSchema(List<String> parentSchemaPath, SchemaConfig schemaConfig, DrillFileSystem fs) throws IOException {
-    if (!accessible(fs)) {
-      return null;
-  }
-    return new WorkspaceSchema(parentSchemaPath, schemaName, schemaConfig, fs);
-  }
-
-  public String getSchemaName() {
-    return schemaName;
-  }
-
-  public FileSystemPlugin getPlugin() {
-    return plugin;
+  public WorkspaceSchema createSchema(List<String> parentSchemaPath, SchemaConfig schemaConfig) throws IOException {
+    return new WorkspaceSchema(parentSchemaPath, schemaName, schemaConfig);
   }
 
   /**
@@ -409,12 +380,12 @@ public class WorkspaceSchemaFactory {
   public class WorkspaceSchema extends AbstractSchema implements ExpandingConcurrentMap.MapValueFactory<TableInstance, DrillTable> {
     private final ExpandingConcurrentMap<TableInstance, DrillTable> tables = new ExpandingConcurrentMap<>(this);
     private final SchemaConfig schemaConfig;
-    private DrillFileSystem fs;
+    private final DrillFileSystem fs;
 
-    public WorkspaceSchema(List<String> parentSchemaPath, String wsName, SchemaConfig schemaConfig, DrillFileSystem fs) throws IOException {
+    public WorkspaceSchema(List<String> parentSchemaPath, String wsName, SchemaConfig schemaConfig) throws IOException {
       super(parentSchemaPath, wsName);
       this.schemaConfig = schemaConfig;
-      this.fs = fs;
+      this.fs = ImpersonationUtil.createFileSystem(schemaConfig.getUserName(), fsConf);
     }
 
     DrillTable getDrillTable(TableInstance key) {
@@ -424,10 +395,10 @@ public class WorkspaceSchemaFactory {
     @Override
     public boolean createView(View view) throws IOException {
       Path viewPath = getViewPath(view.getName());
-      boolean replaced = getFS().exists(viewPath);
+      boolean replaced = fs.exists(viewPath);
       final FsPermission viewPerms =
           new FsPermission(schemaConfig.getOption(ExecConstants.NEW_VIEW_DEFAULT_PERMS_KEY).string_val);
-      try (OutputStream stream = DrillFileSystem.create(getFS(), viewPath, viewPerms)) {
+      try (OutputStream stream = DrillFileSystem.create(fs, viewPath, viewPerms)) {
         mapper.writeValue(stream, view);
       }
       return replaced;
@@ -450,7 +421,7 @@ public class WorkspaceSchemaFactory {
 
     @Override
     public void dropView(String viewName) throws IOException {
-      getFS().delete(getViewPath(viewName), false);
+      fs.delete(getViewPath(viewName), false);
     }
 
     private Set<String> getViews() {
@@ -458,7 +429,7 @@ public class WorkspaceSchemaFactory {
       // Look for files with ".view.drill" extension.
       List<DotDrillFile> files;
       try {
-        files = DotDrillUtil.getDotDrills(getFS(), new Path(config.getLocation()), DotDrillType.VIEW);
+        files = DotDrillUtil.getDotDrills(fs, new Path(config.getLocation()), DotDrillType.VIEW);
         for (DotDrillFile f : files) {
           viewSet.add(f.getBaseName());
         }
@@ -527,7 +498,7 @@ public class WorkspaceSchemaFactory {
       List<DotDrillFile> files = Collections.emptyList();
       try {
         try {
-          files = DotDrillUtil.getDotDrills(getFS(), new Path(config.getLocation()), tableName, DotDrillType.VIEW);
+          files = DotDrillUtil.getDotDrills(fs, new Path(config.getLocation()), tableName, DotDrillType.VIEW);
         } catch (AccessControlException e) {
           if (!schemaConfig.getIgnoreAuthErrors()) {
             logger.debug(e.getMessage());
@@ -599,19 +570,18 @@ public class WorkspaceSchemaFactory {
     }
 
     private DrillTable isReadable(FormatMatcher m, FileSelection fileSelection) throws IOException {
-      return m.isReadable(getFS(), fileSelection, plugin, storageEngineName, schemaConfig.getUserName());
+      return m.isReadable(fs, fileSelection, plugin, storageEngineName, schemaConfig.getUserName());
     }
 
     @Override
     public DrillTable create(TableInstance key) {
       try {
-        final FileSelection fileSelection = FileSelection
-            .create(getFS(), config.getLocation(), key.sig.name, config.allowAccessOutsideWorkspace());
+        final FileSelection fileSelection = FileSelection.create(fs, config.getLocation(), key.sig.name, config.allowAccessOutsideWorkspace());
         if (fileSelection == null) {
           return null;
         }
 
-        final boolean hasDirectories = fileSelection.containsDirectories(getFS());
+        final boolean hasDirectories = fileSelection.containsDirectories(fs);
         if (key.sig.params.size() > 0) {
           FormatPluginConfig fconfig = optionExtractor.createConfigForTable(key);
           return new DynamicDrillTable(
@@ -621,7 +591,7 @@ public class WorkspaceSchemaFactory {
         if (hasDirectories) {
           for (final FormatMatcher matcher : dirMatchers) {
             try {
-              DrillTable table = matcher.isReadable(getFS(), fileSelection, plugin, storageEngineName, schemaConfig.getUserName());
+              DrillTable table = matcher.isReadable(fs, fileSelection, plugin, storageEngineName, schemaConfig.getUserName());
               if (table != null) {
                 return table;
               }
@@ -631,13 +601,13 @@ public class WorkspaceSchemaFactory {
           }
         }
 
-        final FileSelection newSelection = hasDirectories ? fileSelection.minusDirectories(getFS()) : fileSelection;
+        final FileSelection newSelection = hasDirectories ? fileSelection.minusDirectories(fs) : fileSelection;
         if (newSelection == null) {
           return null;
         }
 
         for (final FormatMatcher matcher : fileMatchers) {
-          DrillTable table = matcher.isReadable(getFS(), newSelection, plugin, storageEngineName, schemaConfig.getUserName());
+          DrillTable table = matcher.isReadable(fs, newSelection, plugin, storageEngineName, schemaConfig.getUserName());
           if (table != null) {
             return table;
           }
@@ -662,7 +632,7 @@ public class WorkspaceSchemaFactory {
       FormatMatcher matcher = null;
       try {
         for (FormatMatcher m : dropFileMatchers) {
-          if (m.isFileReadable(getFS(), file)) {
+          if (m.isFileReadable(fs, file)) {
             return m;
           }
         }
@@ -685,8 +655,7 @@ public class WorkspaceSchemaFactory {
      * @throws IOException is case of problems accessing table files
      */
     private boolean isHomogeneous(String tableName) throws IOException {
-      FileSelection fileSelection =
-          FileSelection.create(getFS(), config.getLocation(), tableName, config.allowAccessOutsideWorkspace());
+      FileSelection fileSelection = FileSelection.create(fs, config.getLocation(), tableName, config.allowAccessOutsideWorkspace());
 
       if (fileSelection == null) {
         throw UserException
@@ -697,15 +666,15 @@ public class WorkspaceSchemaFactory {
 
       FormatMatcher matcher = null;
       Queue<FileStatus> listOfFiles = new LinkedList<>();
-      listOfFiles.addAll(fileSelection.getStatuses(getFS()));
+      listOfFiles.addAll(fileSelection.getStatuses(fs));
 
       while (!listOfFiles.isEmpty()) {
         FileStatus currentFile = listOfFiles.poll();
         if (currentFile.isDirectory()) {
-          listOfFiles.addAll(DrillFileSystemUtil.listFiles(getFS(), currentFile.getPath(), true));
+          listOfFiles.addAll(DrillFileSystemUtil.listFiles(fs, currentFile.getPath(), true));
         } else {
           if (matcher != null) {
-            if (!matcher.isFileReadable(getFS(), currentFile)) {
+            if (!matcher.isFileReadable(fs, currentFile)) {
               return false;
             }
           } else {
@@ -794,7 +763,7 @@ public class WorkspaceSchemaFactory {
       // Then look for files that start with this name and end in .drill.
       List<DotDrillFile> files = Collections.emptyList();
       try {
-        files = DotDrillUtil.getDotDrills(getFS(), new Path(config.getLocation()), DotDrillType.VIEW);
+        files = DotDrillUtil.getDotDrills(fs, new Path(config.getLocation()), DotDrillType.VIEW);
       } catch (AccessControlException e) {
         if (!schemaConfig.getIgnoreAuthErrors()) {
           logger.debug(e.getMessage());

http://git-wip-us.apache.org/repos/asf/drill/blob/450e6709/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaBatchCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaBatchCreator.java
index ce05543..60581a7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaBatchCreator.java
@@ -33,7 +33,7 @@ public class InfoSchemaBatchCreator implements BatchCreator<InfoSchemaSubScan>{
   @Override
   public ScanBatch getBatch(FragmentContext context, InfoSchemaSubScan config, List<RecordBatch> children)
       throws ExecutionSetupException {
-    RecordReader rr = config.getTable().getRecordReader(context.getFullRootSchema(), config.getFilter(), context.getOptions());
+    RecordReader rr = config.getTable().getRecordReader(context.getRootSchema(), config.getFilter(), context.getOptions());
     return new ScanBatch(config, context, Collections.singletonList(rr));
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/450e6709/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockBreakageStorage.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockBreakageStorage.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockBreakageStorage.java
deleted file mode 100644
index f2c2d9f..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockBreakageStorage.java
+++ /dev/null
@@ -1,47 +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.store.mock;
-
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.store.SchemaConfig;
-
-import java.io.IOException;
-
-public class MockBreakageStorage extends MockStorageEngine {
-
-  private boolean breakRegister;
-
-  public MockBreakageStorage(MockStorageEngineConfig configuration, DrillbitContext context, String name) {
-    super(configuration, context, name);
-    breakRegister = false;
-  }
-
-  public void setBreakRegister(boolean breakRegister) {
-    this.breakRegister = breakRegister;
-  }
-
-  @Override
-  public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException {
-    if (breakRegister) {
-      throw new IOException("mock breakRegister!");
-    }
-    super.registerSchemas(schemaConfig, parent);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/450e6709/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
index 76363f3..90644b5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
@@ -55,7 +55,7 @@ public class MockStorageEngine extends AbstractStoragePlugin {
 
   public MockStorageEngine(MockStorageEngineConfig configuration, DrillbitContext context, String name) {
     this.configuration = configuration;
-    this.schema = new MockSchema(this, name);
+    this.schema = new MockSchema(this);
   }
 
   @Override
@@ -120,11 +120,6 @@ public class MockStorageEngine extends AbstractStoragePlugin {
       this.engine = engine;
     }
 
-    public MockSchema(MockStorageEngine engine, String name) {
-      super(ImmutableList.<String>of(), name);
-      this.engine = engine;
-    }
-
     @Override
     public Table getTable(String name) {
       if (name.toLowerCase().endsWith(".json")) {

http://git-wip-us.apache.org/repos/asf/drill/blob/450e6709/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java
index f26848d..cf64b20 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/metadata/MetadataProvider.java
@@ -543,7 +543,7 @@ public class MetadataProvider {
   private static <S> PojoRecordReader<S> getPojoRecordReader(final InfoSchemaTableType tableType, final InfoSchemaFilter filter, final DrillConfig config,
       final SchemaTreeProvider provider, final UserSession userSession) {
     final SchemaPlus rootSchema =
-        provider.createFullRootSchema(userSession.getCredentials().getUserName(), newSchemaConfigInfoProvider(config, userSession, provider));
+        provider.createRootSchema(userSession.getCredentials().getUserName(), newSchemaConfigInfoProvider(config, userSession, provider));
     return tableType.getRecordReader(rootSchema, filter, userSession.getOptions());
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/450e6709/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSchema.java
deleted file mode 100644
index 9282eed..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSchema.java
+++ /dev/null
@@ -1,87 +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.physical.impl;
-
-import org.apache.drill.test.BaseDirTestWatcher;
-import org.apache.drill.test.ClientFixture;
-import org.apache.drill.test.ClusterFixture;
-import org.apache.drill.test.ClusterMockStorageFixture;
-import org.apache.drill.test.DrillTest;
-
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Test;
-
-import static org.junit.Assert.assertTrue;
-
-public class TestSchema extends DrillTest {
-
-  @ClassRule
-  public static final BaseDirTestWatcher dirTestWatcher = new BaseDirTestWatcher();
-
-  private static ClusterMockStorageFixture cluster;
-  private static ClientFixture client;
-
-  @BeforeClass
-  public static void setup() throws Exception {
-    cluster = ClusterFixture.builder(dirTestWatcher).buildCustomMockStorage();
-    boolean breakRegisterSchema = true;
-    // With a broken storage which will throw exception in regiterSchema, every query (even on other storage)
-    // shall fail if Drill is still loading all schemas (include the broken schema) before a query.
-    cluster.insertMockStorage("mock_broken", breakRegisterSchema);
-    cluster.insertMockStorage("mock_good", !breakRegisterSchema);
-    client = cluster.clientFixture();
-  }
-
-  @Test (expected = Exception.class)
-  public void testQueryBrokenStorage() throws Exception {
-    String sql = "SELECT id_i, name_s10 FROM `mock_broken`.`employees_5`";
-    try {
-      client.queryBuilder().sql(sql).run();
-    } catch (Exception ex) {
-      assertTrue(ex.getMessage().contains("VALIDATION ERROR: Schema"));
-      throw ex;
-    }
-  }
-
-  @Test
-  public void testQueryGoodStorage() throws Exception {
-    String sql = "SELECT id_i, name_s10 FROM `mock_good`.`employees_5`";
-    client.queryBuilder().sql(sql).run();
-  }
-
-  @Test
-  public void testQueryGoodStorageWithDefaultSchema() throws Exception {
-    String use_dfs = "use dfs.tmp";
-    client.queryBuilder().sql(use_dfs).run();
-    String sql = "SELECT id_i, name_s10 FROM `mock_good`.`employees_5`";
-    client.queryBuilder().sql(sql).run();
-  }
-
-  @Test (expected = Exception.class)
-  public void testUseBrokenStorage() throws Exception {
-    try {
-      String use_dfs = "use mock_broken";
-      client.queryBuilder().sql(use_dfs).run();
-    } catch(Exception ex) {
-      assertTrue(ex.getMessage().contains("VALIDATION ERROR: Schema"));
-      throw ex;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/450e6709/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixtureBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixtureBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixtureBuilder.java
index dfd63de..82bcf75 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixtureBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixtureBuilder.java
@@ -282,8 +282,4 @@ public class ClusterFixtureBuilder {
   public ClusterFixture build() {
     return new ClusterFixture(this);
   }
-
-  public ClusterMockStorageFixture buildCustomMockStorage() {
-    return new ClusterMockStorageFixture(this);
-  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/450e6709/exec/java-exec/src/test/java/org/apache/drill/test/ClusterMockStorageFixture.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterMockStorageFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterMockStorageFixture.java
deleted file mode 100644
index 54d7bf0..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterMockStorageFixture.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.test;
-
-import org.apache.drill.exec.server.Drillbit;
-import org.apache.drill.exec.store.StoragePluginRegistry;
-import org.apache.drill.exec.store.StoragePluginRegistryImpl;
-import org.apache.drill.exec.store.mock.MockBreakageStorage;
-import org.apache.drill.exec.store.mock.MockStorageEngineConfig;
-
-public class ClusterMockStorageFixture extends ClusterFixture {
-  ClusterMockStorageFixture(ClusterFixtureBuilder builder) {
-    super(builder);
-
-  }
-
-  /**
-   * This should be called after bits are started
-   * @param name nthe mock storage name we are going to create
-   */
-  public void insertMockStorage(String name, boolean breakRegisterSchema) {
-    for (Drillbit bit : drillbits()) {
-
-      // Bit name and registration.
-      final StoragePluginRegistry pluginRegistry = bit.getContext().getStorage();
-      MockStorageEngineConfig config = MockStorageEngineConfig.INSTANCE;
-      @SuppressWarnings("resource")
-      MockBreakageStorage plugin = new MockBreakageStorage(
-          MockStorageEngineConfig.INSTANCE, bit.getContext(), name);
-      ((StoragePluginRegistryImpl) pluginRegistry).definePlugin(name, config, plugin);
-
-      plugin.setBreakRegister(breakRegisterSchema);
-    }
-  }
-
-}


[11/18] drill git commit: DRILL-3993: Changes for CALCITE-2092

Posted by am...@apache.org.
DRILL-3993: Changes for CALCITE-2092


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

Branch: refs/heads/master
Commit: 6580674134339fb31cd42f8003e2494f04cac205
Parents: 663518e
Author: Volodymyr Vysotskyi <vv...@gmail.com>
Authored: Tue Dec 26 14:03:54 2017 +0200
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Tue Jan 16 12:10:13 2018 +0200

----------------------------------------------------------------------
 .../java/org/apache/drill/exec/planner/sql/SqlConverter.java  | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/65806741/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 563f58e..8ad4df7 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
@@ -65,6 +65,7 @@ import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.sql2rel.RelDecorrelator;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
+import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.Util;
 import org.apache.commons.collections.ListUtils;
 import org.apache.drill.common.config.DrillConfig;
@@ -76,6 +77,7 @@ import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.ops.UdfUtilities;
 import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.logical.DrillConstExecutor;
+import org.apache.drill.exec.planner.logical.DrillRelFactories;
 import org.apache.drill.exec.planner.physical.DrillDistributionTraitDef;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.rpc.user.UserSession;
@@ -443,6 +445,11 @@ public class SqlConverter {
     public int getInSubQueryThreshold() {
       return inSubqueryThreshold;
     }
+
+    @Override
+    public RelBuilderFactory getRelBuilderFactory() {
+      return DrillRelFactories.LOGICAL_BUILDER;
+    }
   }
 
   /**


[02/18] drill git commit: DRILL-3993: Add check for OOM in HashAgg

Posted by am...@apache.org.
DRILL-3993: Add check for OOM in HashAgg


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

Branch: refs/heads/master
Commit: d06a7cbbd7e5046e5017ffeb010b8b3c74123184
Parents: 3c9093e
Author: Volodymyr Vysotskyi <vv...@gmail.com>
Authored: Fri Dec 29 17:36:08 2017 +0200
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Tue Jan 16 12:10:13 2018 +0200

----------------------------------------------------------------------
 .../physical/impl/aggregate/HashAggTemplate.java     | 15 +++++++++++++++
 1 file changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/d06a7cbb/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
index 4a81f3c..4b43b22 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
@@ -1280,6 +1280,8 @@ public abstract class HashAggTemplate implements HashAggregator {
       logger.trace("Reserved memory runs short, trying to {} a partition and retry Hash Table put() again.",
         is1stPhase ? "early return" : "spill");
 
+      checkForSpillPossibility(currentPartition);
+
       doSpill(currentPartition); // spill to free some memory
 
       retrySameIndex = true;
@@ -1303,6 +1305,8 @@ public abstract class HashAggTemplate implements HashAggregator {
       long memDiff = allocator.getAllocatedMemory() - allocatedBeforeHTput;
       if ( memDiff > 0 ) { logger.warn("Leak: HashTable put() OOM left behind {} bytes allocated",memDiff); }
 
+      checkForSpillPossibility(currentPartition);
+
       doSpill(currentPartition); // spill to free some memory
 
       retrySameIndex = true;
@@ -1379,6 +1383,17 @@ public abstract class HashAggTemplate implements HashAggregator {
     }
   }
 
+  /**
+   * Checks that spill is possible, otherwise throws {@link OutOfMemoryException}.
+   *
+   * @param currentPartition the partition that hit the memory limit
+   */
+  private void checkForSpillPossibility(int currentPartition) {
+    if (chooseAPartitionToFlush(currentPartition, true) < 0) {
+      throw new OutOfMemoryException(getOOMErrorMsg("AGGR"));
+    }
+  }
+
   private void spillIfNeeded(int currentPartition) { spillIfNeeded(currentPartition, false);}
   private void doSpill(int currentPartition) { spillIfNeeded(currentPartition, true);}
   /**


[17/18] drill git commit: DRILL-3993: Changes after review:

Posted by am...@apache.org.
DRILL-3993: Changes after review:

Moved checks for OOM into spillIfNeeded method to avoid excessive call of chooseAPartitionToFlush method.
Ignored unit tests until DRILL-6018 is fixed.
These failures appears because new Calcite tries to simplify expressions from the query when applying ReduceExpressionsRule and fails with NFE.
Similar problem, but with old version of Calcite was described in DRILL-6018, therefore these unit tests are marked as ignored until DRILL-6018 is fixed.


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

Branch: refs/heads/master
Commit: de3889ac1e6c1808b1f24b32039dd3c2fff658d7
Parents: 18a71a3
Author: Volodymyr Vysotskyi <vv...@gmail.com>
Authored: Tue Jan 16 15:38:17 2018 +0200
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Thu Jan 18 12:56:01 2018 +0200

----------------------------------------------------------------------
 .../impl/aggregate/HashAggTemplate.java         | 52 ++++++++++----------
 .../fn/impl/TestMathFunctionsWithNanInf.java    |  3 ++
 .../vector/complex/writer/TestJsonNanInf.java   |  3 ++
 3 files changed, 32 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/de3889ac/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
index 4b43b22..2f181fe 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggTemplate.java
@@ -1280,8 +1280,6 @@ public abstract class HashAggTemplate implements HashAggregator {
       logger.trace("Reserved memory runs short, trying to {} a partition and retry Hash Table put() again.",
         is1stPhase ? "early return" : "spill");
 
-      checkForSpillPossibility(currentPartition);
-
       doSpill(currentPartition); // spill to free some memory
 
       retrySameIndex = true;
@@ -1305,8 +1303,6 @@ public abstract class HashAggTemplate implements HashAggregator {
       long memDiff = allocator.getAllocatedMemory() - allocatedBeforeHTput;
       if ( memDiff > 0 ) { logger.warn("Leak: HashTable put() OOM left behind {} bytes allocated",memDiff); }
 
-      checkForSpillPossibility(currentPartition);
-
       doSpill(currentPartition); // spill to free some memory
 
       retrySameIndex = true;
@@ -1383,17 +1379,6 @@ public abstract class HashAggTemplate implements HashAggregator {
     }
   }
 
-  /**
-   * Checks that spill is possible, otherwise throws {@link OutOfMemoryException}.
-   *
-   * @param currentPartition the partition that hit the memory limit
-   */
-  private void checkForSpillPossibility(int currentPartition) {
-    if (chooseAPartitionToFlush(currentPartition, true) < 0) {
-      throw new OutOfMemoryException(getOOMErrorMsg("AGGR"));
-    }
-  }
-
   private void spillIfNeeded(int currentPartition) { spillIfNeeded(currentPartition, false);}
   private void doSpill(int currentPartition) { spillIfNeeded(currentPartition, true);}
   /**
@@ -1426,9 +1411,17 @@ public abstract class HashAggTemplate implements HashAggregator {
       // Pick a "victim" partition to spill or return
       int victimPartition = chooseAPartitionToFlush(currentPartition, forceSpill);
 
-      // In case no partition has more than one batch -- try and "push the limits"; maybe next
-      // time the spill could work.
-      if ( victimPartition < 0 ) { return; }
+      // In case no partition has more than one batch and
+      // non-forced spill -- try and "push the limits";
+      // maybe next time the spill could work.
+      if (victimPartition < 0) {
+        // In the case of the forced spill, there is not enough memory to continue.
+        // Throws OOM to avoid the infinite loop.
+        if (forceSpill) {
+          throw new OutOfMemoryException(getOOMErrorMsg("AGGR"));
+        }
+        return;
+      }
 
       if ( is2ndPhase ) {
         long before = allocator.getAllocatedMemory();
@@ -1443,14 +1436,21 @@ public abstract class HashAggTemplate implements HashAggregator {
         boolean spillAgain = reserveOutgoingMemory == 0 || reserveValueBatchMemory == 0;
         // in some "edge" cases (e.g. testing), spilling one partition may not be enough
         if ( spillAgain || allocator.getAllocatedMemory() + maxMemoryNeeded > allocator.getLimit() ) {
-            int victimPartition2 = chooseAPartitionToFlush(victimPartition, true);
-            if ( victimPartition2 < 0 ) { return; }
-            long after = allocator.getAllocatedMemory();
-            spillAPartition(victimPartition2);
-            reinitPartition(victimPartition2);
-            logger.warn("A Second Spill was Needed: allocated before {}, after first spill {}, after second {}, memory needed {}",
-                before, after, allocator.getAllocatedMemory(), maxMemoryNeeded);
-            logger.trace("Second Partition Spilled: {}",victimPartition2);
+          int victimPartition2 = chooseAPartitionToFlush(victimPartition, true);
+          if (victimPartition2 < 0) {
+            // In the case of the forced spill, there is not enough memory to continue.
+            // Throws OOM to avoid the infinite loop.
+            if (forceSpill) {
+              throw new OutOfMemoryException(getOOMErrorMsg("AGGR"));
+            }
+            return;
+          }
+          long after = allocator.getAllocatedMemory();
+          spillAPartition(victimPartition2);
+          reinitPartition(victimPartition2);
+          logger.warn("A Second Spill was Needed: allocated before {}, after first spill {}, after second {}, memory needed {}",
+              before, after, allocator.getAllocatedMemory(), maxMemoryNeeded);
+          logger.trace("Second Partition Spilled: {}",victimPartition2);
         }
       }
       else {

http://git-wip-us.apache.org/repos/asf/drill/blob/de3889ac/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctionsWithNanInf.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctionsWithNanInf.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctionsWithNanInf.java
index c82689e..4003bbc 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctionsWithNanInf.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctionsWithNanInf.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.fn.impl;
 import org.apache.commons.io.FileUtils;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.test.BaseTestQuery;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import java.io.File;
@@ -90,6 +91,7 @@ public class TestMathFunctionsWithNanInf extends BaseTestQuery {
 
 
     @Test
+    @Ignore // see DRILL-6018
     public void tesGreaterThanOrEqualToFunction() throws Exception {
       String table_name = "nan_test.json";
       String json = "{\"nan_col\":NaN, \"inf_col\":Infinity}";
@@ -101,6 +103,7 @@ public class TestMathFunctionsWithNanInf extends BaseTestQuery {
     }
 
     @Test
+    @Ignore // see DRILL-6018
     public void testLessThanOrEqualToFunction() throws Exception {
       String table_name = "nan_test.json";
       String json = "{\"nan_col\":NaN, \"inf_col\":Infinity}";

http://git-wip-us.apache.org/repos/asf/drill/blob/de3889ac/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonNanInf.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonNanInf.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonNanInf.java
index 60d4b7a..2d19c17 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonNanInf.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonNanInf.java
@@ -60,6 +60,7 @@ public class TestJsonNanInf extends BaseTestQuery {
   }
 
   @Test
+  @Ignore // see DRILL-6018
   public void testExcludePositiveInfinity() throws Exception {
     String table = "nan_test.json";
     File file = new File(dirTestWatcher.getRootDir(), table);
@@ -81,6 +82,7 @@ public class TestJsonNanInf extends BaseTestQuery {
   }
 
   @Test
+  @Ignore // see DRILL-6018
   public void testExcludeNegativeInfinity() throws Exception {
     String table = "nan_test.json";
     File file = new File(dirTestWatcher.getRootDir(), table);
@@ -102,6 +104,7 @@ public class TestJsonNanInf extends BaseTestQuery {
   }
 
   @Test
+  @Ignore // see DRILL-6018
   public void testIncludePositiveInfinity() throws Exception {
     String table = "nan_test.json";
     File file = new File(dirTestWatcher.getRootDir(), table);


[18/18] drill git commit: DRILL-3993: Fix typo in DrillRelBuilder Javadoc

Posted by am...@apache.org.
DRILL-3993: Fix typo in DrillRelBuilder Javadoc


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

Branch: refs/heads/master
Commit: 9e944c97ee6f6c0d1705f09d531af35deed2e310
Parents: de3889a
Author: Volodymyr Vysotskyi <vv...@gmail.com>
Authored: Wed Jan 17 21:50:34 2018 +0200
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Thu Jan 18 12:56:05 2018 +0200

----------------------------------------------------------------------
 .../main/java/org/apache/drill/exec/planner/DrillRelBuilder.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/9e944c97/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DrillRelBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DrillRelBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DrillRelBuilder.java
index 3923ba3..9010e30 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DrillRelBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DrillRelBuilder.java
@@ -38,7 +38,7 @@ public class DrillRelBuilder extends RelBuilder {
 
   /**
    * Original method {@link RelBuilder#empty} returns empty values rel.
-   * In the order to preserve dara row types, filter with false predicate is created.
+   * In the order to preserve data row types, filter with false predicate is created.
    */
   @Override
   public RelBuilder empty() {


[12/18] drill git commit: DRILL-3993: Changes to support Calcite 1.13

Posted by am...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/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 523b721..ddf48ed 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
@@ -35,6 +35,7 @@ import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.fun.SqlAvgAggFunction;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 
 import org.apache.drill.common.expression.ExpressionPosition;
@@ -73,7 +74,7 @@ public class TypeInferenceUtils {
       .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
       .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
       .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
-      .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
+      .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY)
       .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
       .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
       .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
@@ -97,8 +98,19 @@ public class TypeInferenceUtils {
       .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
       .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
       .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
+      .put(SqlTypeName.INTERVAL_YEAR, TypeProtos.MinorType.INTERVALYEAR)
       .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
-      .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
+      .put(SqlTypeName.INTERVAL_MONTH, TypeProtos.MinorType.INTERVALYEAR)
+      .put(SqlTypeName.INTERVAL_DAY, TypeProtos.MinorType.INTERVALDAY)
+      .put(SqlTypeName.INTERVAL_DAY_HOUR, TypeProtos.MinorType.INTERVALDAY)
+      .put(SqlTypeName.INTERVAL_DAY_MINUTE, TypeProtos.MinorType.INTERVALDAY)
+      .put(SqlTypeName.INTERVAL_DAY_SECOND, TypeProtos.MinorType.INTERVALDAY)
+      .put(SqlTypeName.INTERVAL_HOUR, TypeProtos.MinorType.INTERVALDAY)
+      .put(SqlTypeName.INTERVAL_HOUR_MINUTE, TypeProtos.MinorType.INTERVALDAY)
+      .put(SqlTypeName.INTERVAL_HOUR_SECOND, TypeProtos.MinorType.INTERVALDAY)
+      .put(SqlTypeName.INTERVAL_MINUTE, TypeProtos.MinorType.INTERVALDAY)
+      .put(SqlTypeName.INTERVAL_MINUTE_SECOND, TypeProtos.MinorType.INTERVALDAY)
+      .put(SqlTypeName.INTERVAL_SECOND, TypeProtos.MinorType.INTERVALDAY)
 
       // SqlTypeName.CHAR is the type for Literals in Calcite, Drill treats Literals as VARCHAR also
       .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
@@ -734,13 +746,13 @@ public class TypeInferenceUtils {
                                                              SqlTypeName sqlTypeName,
                                                              boolean isNullable) {
     RelDataType type;
-    if (sqlTypeName == SqlTypeName.INTERVAL_DAY_TIME) {
+    if (sqlTypeName.getFamily() == SqlTypeFamily.INTERVAL_DAY_TIME) {
       type = typeFactory.createSqlIntervalType(
           new SqlIntervalQualifier(
               TimeUnit.DAY,
               TimeUnit.MINUTE,
               SqlParserPos.ZERO));
-    } else if (sqlTypeName == SqlTypeName.INTERVAL_YEAR_MONTH) {
+    } else if (sqlTypeName.getFamily() == SqlTypeFamily.INTERVAL_YEAR_MONTH) {
       type = typeFactory.createSqlIntervalType(
           new SqlIntervalQualifier(
               TimeUnit.YEAR,

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/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 9515b3b..93113cb 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
@@ -23,6 +23,10 @@ import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.plan.RelOptCostImpl;
+import org.apache.calcite.plan.RelOptLattice;
+import org.apache.calcite.plan.RelOptMaterialization;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptUtil;
@@ -37,6 +41,8 @@ import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.TableFunctionScan;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
+import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
 import org.apache.calcite.rel.metadata.JaninoRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
@@ -46,13 +52,13 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.TypedSqlNode;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.tools.Program;
 import org.apache.calcite.tools.Programs;
 import org.apache.calcite.tools.RelConversionException;
 import org.apache.calcite.tools.RuleSet;
 import org.apache.calcite.tools.ValidationException;
+import org.apache.calcite.util.Pair;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.logical.PlanProperties;
 import org.apache.drill.common.logical.PlanProperties.Generator.ResultMode;
@@ -189,13 +195,13 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
 
   protected ConvertedRelNode validateAndConvert(SqlNode sqlNode) throws ForemanSetupException, RelConversionException, ValidationException {
     final SqlNode rewrittenSqlNode = rewrite(sqlNode);
-    final TypedSqlNode validatedTypedSqlNode = validateNode(rewrittenSqlNode);
-    final SqlNode validated = validatedTypedSqlNode.getSqlNode();
+    final Pair<SqlNode, RelDataType> validatedTypedSqlNode = validateNode(rewrittenSqlNode);
+    final SqlNode validated = validatedTypedSqlNode.getKey();
 
     RelNode rel = convertToRel(validated);
     rel = preprocessNode(rel);
 
-    return new ConvertedRelNode(rel, validatedTypedSqlNode.getType());
+    return new ConvertedRelNode(rel, validatedTypedSqlNode.getValue());
   }
 
   /**
@@ -375,7 +381,9 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
         hepPgmBldr.addRuleInstance(rule);
       }
 
-      final HepPlanner planner = new HepPlanner(hepPgmBldr.build(), context.getPlannerSettings());
+      // Set noDAG = true to avoid caching problems which lead to incorrect Drill work.
+      final HepPlanner planner = new HepPlanner(hepPgmBldr.build(), context.getPlannerSettings(), true, null,
+          RelOptCostImpl.FACTORY);
 
       JaninoRelMetadataProvider relMetadataProvider = JaninoRelMetadataProvider.of(DrillDefaultRelMetadataProvider.INSTANCE);
       RelMetadataQuery.THREAD_PROVIDERS.set(relMetadataProvider);
@@ -397,7 +405,8 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
       Preconditions.checkArgument(planner instanceof VolcanoPlanner,
           "Cluster is expected to be constructed using VolcanoPlanner. Was actually of type %s.", planner.getClass()
               .getName());
-      output = program.run(planner, input, toTraits);
+      output = program.run(planner, input, toTraits,
+          ImmutableList.<RelOptMaterialization>of(), ImmutableList.<RelOptLattice>of());
 
       break;
     }
@@ -613,9 +622,9 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
 
   }
 
-  private TypedSqlNode validateNode(SqlNode sqlNode) throws ValidationException, RelConversionException, ForemanSetupException {
+  private Pair<SqlNode, RelDataType> validateNode(SqlNode sqlNode) throws ValidationException, RelConversionException, ForemanSetupException {
     final SqlNode sqlNodeValidated = config.getConverter().validate(sqlNode);
-    final TypedSqlNode typedSqlNode = new TypedSqlNode(sqlNodeValidated, config.getConverter().getOutputType(
+    final Pair<SqlNode, RelDataType> typedSqlNode = new Pair<>(sqlNodeValidated, config.getConverter().getOutputType(
         sqlNodeValidated));
 
     // Check if the unsupported functionality is used
@@ -634,7 +643,7 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
   }
 
   private RelNode convertToRel(SqlNode node) throws RelConversionException {
-    final RelNode convertedNode = config.getConverter().toRel(node);
+    final RelNode convertedNode = config.getConverter().toRel(node).rel;
     log("INITIAL", convertedNode, logger, null);
     return transform(PlannerType.HEP, PlannerPhase.WINDOW_REWRITE, convertedNode);
   }
@@ -674,7 +683,7 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
 
     final List<String> fieldNames2 = SqlValidatorUtil.uniquify(
         validatedRowType.getFieldNames(),
-        SqlValidatorUtil.F_SUGGESTER2,
+        SqlValidatorUtil.EXPR_SUGGESTER,
         rel.getCluster().getTypeFactory().getTypeSystem().isSchemaCaseSensitive());
 
     RelDataType newRowType = RexUtil.createStructType(rel.getCluster().getTypeFactory(), projections, fieldNames2);

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java
----------------------------------------------------------------------
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 869829a..b41f880 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
@@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.io.CharacterEscapes;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Joiner;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlDescribeSchema;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
@@ -31,7 +32,6 @@ import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.planner.sql.DirectPlan;
 import org.apache.drill.exec.planner.sql.SchemaUtilites;
-import org.apache.drill.exec.planner.sql.parser.SqlDescribeSchema;
 import org.apache.drill.exec.store.StoragePlugin;
 import org.apache.drill.exec.store.dfs.FileSystemPlugin;
 import org.apache.drill.exec.store.dfs.FileSystemSchemaFactory;

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeTableHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeTableHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeTableHandler.java
index 72b1aef..0311dfc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeTableHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeTableHandler.java
@@ -29,6 +29,7 @@ import static org.apache.drill.exec.store.ischema.InfoSchemaConstants.TAB_COLUMN
 import java.util.List;
 
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlDescribeTable;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
@@ -41,7 +42,7 @@ import org.apache.calcite.util.Util;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.planner.sql.SchemaUtilites;
 import org.apache.drill.exec.planner.sql.parser.DrillParserUtil;
-import org.apache.drill.exec.planner.sql.parser.SqlDescribeTable;
+import org.apache.drill.exec.planner.sql.parser.DrillSqlDescribeTable;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 
 import com.google.common.collect.ImmutableList;
@@ -54,7 +55,7 @@ public class DescribeTableHandler extends DefaultSqlHandler {
   /** Rewrite the parse tree as SELECT ... FROM INFORMATION_SCHEMA.COLUMNS ... */
   @Override
   public SqlNode rewrite(SqlNode sqlNode) throws RelConversionException, ForemanSetupException {
-    SqlDescribeTable node = unwrap(sqlNode, SqlDescribeTable.class);
+    DrillSqlDescribeTable node = unwrap(sqlNode, DrillSqlDescribeTable.class);
 
     try {
       List<SqlNode> selectList =

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java
index d5216e7..166c350 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/FindLimit0Visitor.java
@@ -49,6 +49,7 @@ import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.store.AbstractRecordReader;
 import org.apache.drill.exec.store.direct.DirectGroupScan;
 
+import java.util.ArrayList;
 import java.util.List;
 
 /**
@@ -71,8 +72,11 @@ public class FindLimit0Visitor extends RelShuttleImpl {
       ImmutableSet.<SqlTypeName>builder()
           .add(SqlTypeName.INTEGER, SqlTypeName.BIGINT, SqlTypeName.FLOAT, SqlTypeName.DOUBLE,
               SqlTypeName.VARCHAR, SqlTypeName.BOOLEAN, SqlTypeName.DATE, SqlTypeName.TIME,
-              SqlTypeName.TIMESTAMP, SqlTypeName.INTERVAL_YEAR_MONTH, SqlTypeName.INTERVAL_DAY_TIME,
-              SqlTypeName.CHAR)
+              SqlTypeName.TIMESTAMP, SqlTypeName.INTERVAL_YEAR, SqlTypeName.INTERVAL_YEAR_MONTH,
+              SqlTypeName.INTERVAL_MONTH, SqlTypeName.INTERVAL_DAY, SqlTypeName.INTERVAL_DAY_HOUR,
+              SqlTypeName.INTERVAL_DAY_MINUTE, SqlTypeName.INTERVAL_DAY_SECOND, SqlTypeName.INTERVAL_HOUR,
+              SqlTypeName.INTERVAL_HOUR_MINUTE, SqlTypeName.INTERVAL_HOUR_SECOND, SqlTypeName.INTERVAL_MINUTE,
+              SqlTypeName.INTERVAL_MINUTE_SECOND, SqlTypeName.INTERVAL_SECOND, SqlTypeName.CHAR)
           .build();
 
   /**
@@ -231,5 +235,26 @@ public class FindLimit0Visitor extends RelShuttleImpl {
     @Override
     public void close() throws Exception {
     }
+
+    /**
+     * Represents RelDataTypeReader content as string, used in query plan json.
+     * Example: RelDataTypeReader{columnNames=[col1], columnTypes=[INTERVALYEAR-OPTIONAL]}
+     *
+     * @return string representation of RelDataTypeReader content
+     */
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("RelDataTypeReader{columnNames=");
+      builder.append(columnNames).append(", columnTypes=");
+      List<String> columnTypesList = new ArrayList<>(columnTypes.size());
+      for (TypeProtos.MajorType columnType : columnTypes) {
+        columnTypesList.add(columnType.getMinorType().toString() + "-" + columnType.getMode().toString());
+      }
+      builder.append(columnTypesList);
+      builder.append("}");
+
+      return builder.toString();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
index e9d0dca..4d0f34c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
@@ -163,7 +163,7 @@ public class CompoundIdentifierConverter extends SqlShuttle {
     rules.put(SqlSelect.class, R(D, E, D, E, E, E, E, E, D, D));
     rules.put(SqlCreateTable.class, R(D, D, D, E, D, D));
     rules.put(SqlCreateView.class, R(D, E, E, D));
-    rules.put(SqlDescribeTable.class, R(D, D, E));
+    rules.put(DrillSqlDescribeTable.class, R(D, D, E));
     rules.put(SqlDropView.class, R(D, D));
     rules.put(SqlShowFiles.class, R(D));
     rules.put(SqlShowSchemas.class, R(D, D));
@@ -173,7 +173,6 @@ public class CompoundIdentifierConverter extends SqlShuttle {
     rules.put(SqlDropTable.class, R(D, D));
     rules.put(SqlRefreshMetadata.class, R(D));
     rules.put(SqlSetOption.class, R(D, D, D));
-    rules.put(SqlDescribeSchema.class, R(D));
     rules.put(SqlCreateFunction.class, R(D));
     rules.put(SqlDropFunction.class, R(D));
     REWRITE_RULES = ImmutableMap.copyOf(rules);

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillSqlDescribeTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillSqlDescribeTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillSqlDescribeTable.java
new file mode 100644
index 0000000..c97d8c3
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillSqlDescribeTable.java
@@ -0,0 +1,73 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to you under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.drill.exec.planner.sql.parser;
+
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDescribeTable;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.ImmutableNullableList;
+
+import java.util.List;
+
+/**
+ * Sql parser tree node to represent statement:
+ * { DESCRIBE | DESC } tblname [col_name | wildcard ]
+ */
+public class DrillSqlDescribeTable extends SqlDescribeTable {
+
+  private final SqlNode columnQualifier;
+
+  public static final SqlSpecialOperator OPERATOR =
+    new SqlSpecialOperator("DESCRIBE_TABLE", SqlKind.DESCRIBE_TABLE) {
+    @Override
+    public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
+      return new DrillSqlDescribeTable(pos, (SqlIdentifier) operands[0], (SqlIdentifier) operands[1], operands[2]);
+    }
+  };
+
+  public DrillSqlDescribeTable(SqlParserPos pos, SqlIdentifier table, SqlIdentifier column, SqlNode columnQualifier) {
+    super(pos, table, column);
+    this.columnQualifier = columnQualifier;
+  }
+
+  @Override
+  public List<SqlNode> getOperandList() {
+    return ImmutableNullableList.<SqlNode>of(getTable(), getColumn(), columnQualifier);
+  }
+
+  @Override
+  public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+    super.unparse(writer, leftPrec, rightPrec);
+    if (columnQualifier != null) {
+      columnQualifier.unparse(writer, leftPrec, rightPrec);
+    }
+  }
+
+  public SqlNode getColumnQualifier() { return columnQualifier; }
+
+  @Override public SqlOperator getOperator() {
+    return OPERATOR;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDescribeSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDescribeSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDescribeSchema.java
deleted file mode 100644
index 7ea6940..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDescribeSchema.java
+++ /dev/null
@@ -1,81 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.planner.sql.parser;
-
-import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlSpecialOperator;
-import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.drill.exec.planner.sql.handlers.AbstractSqlHandler;
-import org.apache.drill.exec.planner.sql.handlers.DescribeSchemaHandler;
-import org.apache.drill.exec.planner.sql.handlers.SqlHandlerConfig;
-
-import java.util.Collections;
-import java.util.List;
-
-/**
- * Sql parse tree node to represent statement:
- * DESCRIBE {SCHEMA | DATABASE} schema_name
- */
-public class SqlDescribeSchema extends DrillSqlCall {
-
-  private final SqlIdentifier schema;
-
-  public static final SqlSpecialOperator OPERATOR =
-      new SqlSpecialOperator("DESCRIBE_SCHEMA", SqlKind.OTHER) {
-        @Override
-        public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
-          return new SqlDescribeSchema(pos, (SqlIdentifier) operands[0]);
-        }
-      };
-
-  public SqlDescribeSchema(SqlParserPos pos, SqlIdentifier schema) {
-    super(pos);
-    this.schema = schema;
-  }
-
-  @Override
-  public SqlOperator getOperator() {
-    return OPERATOR;
-  }
-
-  @Override
-  public List<SqlNode> getOperandList() {
-    return Collections.singletonList((SqlNode) schema);
-  }
-
-  @Override
-  public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
-    writer.keyword("DESCRIBE");
-    writer.keyword("SCHEMA");
-    schema.unparse(writer, leftPrec, rightPrec);
-  }
-
-  @Override
-  public AbstractSqlHandler getSqlHandler(SqlHandlerConfig config) {
-    return new DescribeSchemaHandler(config);
-  }
-
-  public SqlIdentifier getSchema() { return schema; }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDescribeTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDescribeTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDescribeTable.java
deleted file mode 100644
index 3a09aea..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDescribeTable.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.planner.sql.parser;
-
-import java.util.List;
-
-import org.apache.calcite.tools.Planner;
-
-import org.apache.drill.exec.ops.QueryContext;
-import org.apache.drill.exec.planner.sql.handlers.AbstractSqlHandler;
-import org.apache.drill.exec.planner.sql.handlers.DescribeTableHandler;
-import org.apache.drill.exec.planner.sql.handlers.SqlHandlerConfig;
-import org.apache.calcite.plan.hep.HepPlanner;
-import org.apache.calcite.sql.SqlCall;
-import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlLiteral;
-import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlSpecialOperator;
-import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.parser.SqlParserPos;
-
-import com.google.common.collect.Lists;
-
-/**
- * Sql parser tree node to represent statement:
- * { DESCRIBE | DESC } tblname [col_name | wildcard ]
- */
-public class SqlDescribeTable extends DrillSqlCall {
-
-  private final SqlIdentifier table;
-  private final SqlIdentifier column;
-  private final SqlNode columnQualifier;
-
-  public static final SqlSpecialOperator OPERATOR =
-    new SqlSpecialOperator("DESCRIBE_TABLE", SqlKind.OTHER) {
-    @Override
-    public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
-      return new SqlDescribeTable(pos, (SqlIdentifier) operands[0], (SqlIdentifier) operands[1], operands[2]);
-    }
-  };
-
-  public SqlDescribeTable(SqlParserPos pos, SqlIdentifier table, SqlIdentifier column, SqlNode columnQualifier) {
-    super(pos);
-    this.table = table;
-    this.column = column;
-    this.columnQualifier = columnQualifier;
-  }
-
-  @Override
-  public SqlOperator getOperator() {
-    return OPERATOR;
-  }
-
-  @Override
-  public List<SqlNode> getOperandList() {
-    List<SqlNode> opList = Lists.newArrayList();
-    opList.add(table);
-    opList.add(column);
-    opList.add(columnQualifier);
-    return opList;
-  }
-
-  @Override
-  public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
-    writer.keyword("DESCRIBE");
-    writer.keyword("TABLE");
-    table.unparse(writer, leftPrec, rightPrec);
-    if (column != null) {
-      column.unparse(writer, leftPrec, rightPrec);
-    }
-    if (columnQualifier != null) {
-      columnQualifier.unparse(writer, leftPrec, rightPrec);
-    }
-  }
-
-  @Override
-  public AbstractSqlHandler getSqlHandler(SqlHandlerConfig config) {
-    return new DescribeTableHandler(config);
-  }
-
-  public SqlIdentifier getTable() { return table; }
-  public SqlIdentifier getColumn() { return column; }
-  public SqlNode getColumnQualifier() { return columnQualifier; }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/torel/ConversionContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/torel/ConversionContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/torel/ConversionContext.java
index a88f523..e4b9668 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/torel/ConversionContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/torel/ConversionContext.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
@@ -22,6 +22,7 @@ import java.util.Map;
 
 import org.apache.calcite.prepare.Prepare;
 
+import org.apache.calcite.rel.RelRoot;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.logical.LogicalPlan;
 import org.apache.drill.common.logical.data.Filter;
@@ -111,12 +112,12 @@ public class ConversionContext implements ToRelContext {
   }
 
   @Override
-  public RelNode expandView(RelDataType rowType, String queryString, List<String> schemaPath) {
+  public RelRoot expandView(RelDataType rowType, String queryString, List<String> schemaPath, List<String> viewPath) {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public RelNode expandView(RelDataType rowType, String queryString, SchemaPlus rootSchema, List<String> schemaPath) {
+  public RelRoot expandView(RelDataType rowType, String queryString, SchemaPlus rootSchema, List<String> schemaPath) {
     throw new UnsupportedOperationException();
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeDrillImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeDrillImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeDrillImpl.java
index 04c35c1..d332434 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeDrillImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeDrillImpl.java
@@ -20,21 +20,21 @@ package org.apache.drill.exec.planner.types;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.calcite.rel.type.DynamicRecordType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeFamily;
 import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rel.type.RelDataTypeImpl;
 import org.apache.calcite.rel.type.RelDataTypePrecedenceList;
 import org.apache.calcite.sql.type.SqlTypeExplicitPrecedenceList;
 import org.apache.calcite.sql.type.SqlTypeName;
 
-/* We use an instance of this class as the row type for
+/* We use an instance of this class as the dynamic row type for
  * Drill table. Since we don't know the schema before hand
  * whenever optiq requires us to validate that a field exists
  * we always return true and indicate that the type of that
  * field is 'ANY'
  */
-public class RelDataTypeDrillImpl extends RelDataTypeImpl {
+public class RelDataTypeDrillImpl extends DynamicRecordType {
 
     private final RelDataTypeFactory typeFactory;
     private final RelDataTypeHolder holder;

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java
index 483b277..aa3542c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java
@@ -19,12 +19,15 @@ package org.apache.drill.exec.planner.types;
 
 import java.util.List;
 
+import org.apache.calcite.rel.type.DynamicRecordType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
 import org.apache.calcite.sql.type.SqlTypeName;
 
 import com.google.common.collect.Lists;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
 public class RelDataTypeHolder {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RelDataTypeHolder.class);
@@ -59,7 +62,14 @@ public class RelDataTypeHolder {
     }
 
     /* This field does not exist in our field list add it */
-    RelDataTypeField newField = new RelDataTypeFieldImpl(fieldName, fields.size(), typeFactory.createTypeWithNullability(typeFactory.createSqlType(SqlTypeName.ANY), true));
+    final SqlTypeName typeName = DynamicRecordType.isDynamicStarColName(fieldName)
+        ? SqlTypeName.DYNAMIC_STAR : SqlTypeName.ANY;
+
+    // This field does not exist in our field list add it
+    RelDataTypeField newField = new RelDataTypeFieldImpl(
+        fieldName,
+        fields.size(),
+        typeFactory.createTypeWithNullability(typeFactory.createSqlType(typeName), true));
 
     /* Add the name to our list of field names */
     fields.add(newField);

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
index 7d6bfe3..c4586ef 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
@@ -206,6 +206,11 @@ public abstract class AbstractSchema implements Schema, SchemaPartitionExplorer,
     // no-op: default implementation for most implementations.
   }
 
+  @Override
+  public Schema snapshot(long now) {
+    return this;
+  }
+
   public void dropTable(String tableName) {
     throw UserException.unsupportedError()
         .message("Dropping tables is not supported in schema [%s]", getSchemaPath())

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java
index 23441bd..21ab39f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaTreeProvider.java
@@ -20,7 +20,7 @@ package org.apache.drill.exec.store;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.calcite.jdbc.SimpleCalciteSchema;
+import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.exceptions.UserException;
@@ -106,7 +106,7 @@ public class SchemaTreeProvider implements AutoCloseable {
    */
   public SchemaPlus createRootSchema(SchemaConfig schemaConfig) {
     try {
-      final SchemaPlus rootSchema = SimpleCalciteSchema.createRootSchema(false);
+      final SchemaPlus rootSchema = CalciteSchema.createRootSchema(false, false).plus();
       dContext.getSchemaFactory().registerSchemas(schemaConfig, rootSchema);
       schemaTreesToClose.add(rootSchema);
       return rootSchema;

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java
index 49d1423..81ac8b6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/Records.java
@@ -134,8 +134,19 @@ public class Records {
         case CHAR:                this.DATA_TYPE = "CHARACTER";         break;
         case VARCHAR:             this.DATA_TYPE = "CHARACTER VARYING"; break;
         case VARBINARY:           this.DATA_TYPE = "BINARY VARYING";    break;
-        case INTERVAL_YEAR_MONTH: this.DATA_TYPE = "INTERVAL";          break;
-        case INTERVAL_DAY_TIME:   this.DATA_TYPE = "INTERVAL";          break;
+        case INTERVAL_YEAR:
+        case INTERVAL_YEAR_MONTH:
+        case INTERVAL_MONTH:
+        case INTERVAL_DAY:
+        case INTERVAL_DAY_HOUR:
+        case INTERVAL_DAY_MINUTE:
+        case INTERVAL_DAY_SECOND:
+        case INTERVAL_HOUR:
+        case INTERVAL_HOUR_MINUTE:
+        case INTERVAL_HOUR_SECOND:
+        case INTERVAL_MINUTE:
+        case INTERVAL_MINUTE_SECOND:
+        case INTERVAL_SECOND:     this.DATA_TYPE = "INTERVAL";          break;
         // 3:  SqlTypeName enumerators not yet seen and confirmed or handled.
         default:
           logger.warn( "Type not handled explicitly (code needs review): "
@@ -292,21 +303,43 @@ public class Records {
 
           }
           break;
+        case INTERVAL_YEAR:
         case INTERVAL_YEAR_MONTH:
-        case INTERVAL_DAY_TIME:
+        case INTERVAL_MONTH:
+        case INTERVAL_DAY:
+        case INTERVAL_DAY_HOUR:
+        case INTERVAL_DAY_MINUTE:
+        case INTERVAL_DAY_SECOND:
+        case INTERVAL_HOUR:
+        case INTERVAL_HOUR_MINUTE:
+        case INTERVAL_HOUR_SECOND:
+        case INTERVAL_MINUTE:
+        case INTERVAL_MINUTE_SECOND:
+        case INTERVAL_SECOND:
           this.CHARACTER_MAXIMUM_LENGTH = null;
           this.CHARACTER_OCTET_LENGTH = null;
           this.NUMERIC_PRECISION = null;
           this.NUMERIC_PRECISION_RADIX = null;
           this.NUMERIC_SCALE = null;
           switch ( sqlTypeName ) {
+            case INTERVAL_YEAR:
             case INTERVAL_YEAR_MONTH:
+            case INTERVAL_MONTH:
               // NOTE:  Apparently can't get use RelDataType, etc.; it seems to
               // apply a default fractional seconds precision of 6 for SECOND,
               // even though SECOND does not exist for this case.
               this.DATETIME_PRECISION = 0;
               break;
-            case INTERVAL_DAY_TIME:
+            case INTERVAL_DAY:
+            case INTERVAL_DAY_HOUR:
+            case INTERVAL_DAY_MINUTE:
+            case INTERVAL_DAY_SECOND:
+            case INTERVAL_HOUR:
+            case INTERVAL_HOUR_MINUTE:
+            case INTERVAL_HOUR_SECOND:
+            case INTERVAL_MINUTE:
+            case INTERVAL_MINUTE_SECOND:
+            case INTERVAL_SECOND:
               this.DATETIME_PRECISION =
                   relDataType
                   .getIntervalQualifier()

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
index ca643f5..8a8e8d2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
@@ -23,7 +23,7 @@ import java.net.URL;
 import mockit.Mocked;
 import mockit.NonStrictExpectations;
 
-import org.apache.calcite.jdbc.SimpleCalciteSchema;
+import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.config.LogicalPlanPersistence;
@@ -111,7 +111,7 @@ public class PlanningBase extends ExecTest{
     registry.init();
     final FunctionImplementationRegistry functionRegistry = new FunctionImplementationRegistry(config);
     final DrillOperatorTable table = new DrillOperatorTable(functionRegistry, systemOptions);
-    final SchemaPlus root = SimpleCalciteSchema.createRootSchema(false);
+    final SchemaPlus root = CalciteSchema.createRootSchema(false, false).plus();
     registry.getSchemaFactory().registerSchemas(SchemaConfig.newBuilder("foo", context).build(), root);
 
     new NonStrictExpectations() {

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillOptiqTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillOptiqTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillOptiqTest.java
index e449602..57b7466 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillOptiqTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/logical/DrillOptiqTest.java
@@ -56,7 +56,7 @@ public class DrillOptiqTest {
 
       // create a dummy RexOver object.
       RexNode window = rex.makeOver(anyType, SqlStdOperatorTable.AVG, emptyList, emptyList, e, null, null, true,
-          false, false);
+          false, false, false);
       DrillOptiq.toDrill(null, (RelNode) null, window);
     } catch (UserException e) {
       if (e.getMessage().contains(DrillOptiq.UNSUPPORTED_REX_NODE_ERROR)) {

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSqlBracketlessSyntax.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSqlBracketlessSyntax.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSqlBracketlessSyntax.java
index f94c40e..57ca0c1 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSqlBracketlessSyntax.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSqlBracketlessSyntax.java
@@ -18,7 +18,7 @@
 package org.apache.drill.exec.sql;
 
 import org.apache.calcite.config.Lex;
-import org.apache.calcite.jdbc.SimpleCalciteSchema;
+import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.tools.FrameworkConfig;
 import org.apache.calcite.tools.Frameworks;
 import org.apache.calcite.tools.Planner;
@@ -46,7 +46,7 @@ public class TestSqlBracketlessSyntax {
             .setIdentifierMaxLength(PlannerSettings.DEFAULT_IDENTIFIER_MAX_LENGTH)
             .setParserFactory(DrillParserImpl.FACTORY)
             .build()) //
-        .defaultSchema(SimpleCalciteSchema.createRootSchema(false)) //
+        .defaultSchema(CalciteSchema.createRootSchema(false, false).plus()) //
         .convertletTable(DrillConvertletTable.INSTANCE) //
         .build();
     Planner planner = Frameworks.getPlanner(config);

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetGroupScan.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetGroupScan.java
index d7ccea4..6dd2e66 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetGroupScan.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetGroupScan.java
@@ -85,7 +85,7 @@ public class TestParquetGroupScan extends BaseTestQuery {
       runSQL("SELECT COUNT(*) AS `count` FROM dfs.tmp.`4376_3/604*`");
       fail("Query should've failed!");
     } catch (UserRemoteException uex) {
-      final String expectedMsg = "Table 'dfs.tmp.4376_3/604*' not found";
+      final String expectedMsg = "Object '4376_3/604*' not found within 'dfs.tmp'";
       assertTrue(String.format("Error message should contain \"%s\" but was instead \"%s\"", expectedMsg,
         uex.getMessage()), uex.getMessage().contains(expectedMsg));
     }
@@ -112,7 +112,7 @@ public class TestParquetGroupScan extends BaseTestQuery {
       runSQL("SELECT COUNT(*) AS `count` FROM dfs.tmp.`4376_5/6041`");
       fail("Query should've failed!");
     } catch (UserRemoteException uex) {
-      final String expectedMsg = "Table 'dfs.tmp.4376_5/6041' not found";
+      final String expectedMsg = "Object '4376_5/6041' not found within 'dfs.tmp'";
       assertTrue(String.format("Error message should contain \"%s\" but was instead \"%s\"", expectedMsg,
         uex.getMessage()), uex.getMessage().contains(expectedMsg));
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/jdbc-all/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index 9db9c58..f59780d 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -509,7 +509,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>32000000</maxsize>
+                  <maxsize>35000000</maxsize>
                   <minsize>15000000</minsize>
                   <files>
                    <file>${project.build.directory}/drill-jdbc-all-${project.version}.jar</file>
@@ -569,7 +569,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>29000000</maxsize>
+                        <maxsize>31000000</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/9fabb612/exec/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc/pom.xml b/exec/jdbc/pom.xml
index dfeaf82..d7087f4 100644
--- a/exec/jdbc/pom.xml
+++ b/exec/jdbc/pom.xml
@@ -21,9 +21,8 @@
 
   <dependencies>
     <dependency>
-      <groupId>org.apache.calcite</groupId>
-      <artifactId>calcite-avatica</artifactId>
-      <version>${calcite.version}</version>
+      <groupId>org.apache.calcite.avatica</groupId>
+      <artifactId>avatica</artifactId>
       <exclusions>
         <exclusion>
           <artifactId>jackson-core</artifactId>

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java
----------------------------------------------------------------------
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 a2b9211..689041c 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
@@ -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
@@ -46,6 +46,8 @@ import org.apache.calcite.avatica.AvaticaFactory;
 import org.apache.calcite.avatica.AvaticaStatement;
 import org.apache.calcite.avatica.Meta.ExecuteResult;
 import org.apache.calcite.avatica.Meta.MetaResultSet;
+import org.apache.calcite.avatica.NoSuchStatementException;
+import org.apache.calcite.avatica.QueryState;
 import org.apache.calcite.avatica.UnregisteredDriver;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
@@ -180,16 +182,16 @@ class DrillConnectionImpl extends AvaticaConnection
 
 
   @Override
-  protected ResultSet createResultSet(MetaResultSet metaResultSet) throws SQLException {
-    return super.createResultSet(metaResultSet);
+  protected ResultSet createResultSet(MetaResultSet metaResultSet, QueryState state) throws SQLException {
+    return super.createResultSet(metaResultSet, state);
   }
 
   @Override
   protected ExecuteResult prepareAndExecuteInternal(AvaticaStatement statement, String sql, long maxRowCount)
-      throws SQLException {
+      throws SQLException, NoSuchStatementException {
     try {
       return super.prepareAndExecuteInternal(statement, sql, maxRowCount);
-    } catch(RuntimeException e) {
+    } catch (RuntimeException e) {
       Throwables.propagateIfInstanceOf(e.getCause(), SQLException.class);
       throw e;
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
index 629e47b..38715e9 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillJdbc41Factory.java
@@ -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
@@ -34,6 +34,7 @@ import org.apache.calcite.avatica.AvaticaStatement;
 import org.apache.calcite.avatica.Helper;
 import org.apache.calcite.avatica.Meta;
 import org.apache.calcite.avatica.Meta.StatementHandle;
+import org.apache.calcite.avatica.QueryState;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.client.ServerMethod;
 import org.apache.drill.exec.proto.UserProtos.CreatePreparedStatementResp;
@@ -170,12 +171,13 @@ public class DrillJdbc41Factory extends DrillFactory {
 
   @Override
   public DrillResultSetImpl newResultSet(AvaticaStatement statement,
+                                         QueryState state,
                                          Meta.Signature signature,
                                          TimeZone timeZone,
                                          Meta.Frame firstFrame) {
     final ResultSetMetaData metaData =
         newResultSetMetaData(statement, signature);
-    return new DrillResultSetImpl(statement, signature, metaData, timeZone, firstFrame);
+    return new DrillResultSetImpl(statement, state, signature, metaData, timeZone, firstFrame);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/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 b78e93a..810ffef 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
@@ -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
@@ -38,6 +38,10 @@ import org.apache.calcite.avatica.ColumnMetaData;
 import org.apache.calcite.avatica.ColumnMetaData.StructType;
 import org.apache.calcite.avatica.Meta;
 import org.apache.calcite.avatica.MetaImpl;
+import org.apache.calcite.avatica.MissingResultsException;
+import org.apache.calcite.avatica.NoSuchStatementException;
+import org.apache.calcite.avatica.QueryState;
+import org.apache.calcite.avatica.remote.TypedValue;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.util.DrillStringUtils;
 import org.apache.drill.exec.client.ServerMethod;
@@ -92,8 +96,8 @@ class DrillMetaImpl extends MetaImpl {
         sql,
         Collections.<AvaticaParameter> emptyList(),
         Collections.<String, Object>emptyMap(),
-        null // CursorFactory set to null, as SQL requests use DrillCursor
-        );
+        null, // CursorFactory set to null, as SQL requests use DrillCursor
+        Meta.StatementType.SELECT);
   }
 
   private MetaResultSet s(String s) {
@@ -322,7 +326,7 @@ class DrillMetaImpl extends MetaImpl {
         StructType fieldMetaData = drillFieldMetaData(clazz);
         Meta.Signature signature = Meta.Signature.create(
             fieldMetaData.columns, "",
-            Collections.<AvaticaParameter>emptyList(), CursorFactory.record(clazz));
+            Collections.<AvaticaParameter>emptyList(), CursorFactory.record(clazz), Meta.StatementType.SELECT);
 
         AvaticaStatement statement = connection.createStatement();
         return MetaResultSet.create(connection.id, statement.getId(), true,
@@ -419,8 +423,11 @@ class DrillMetaImpl extends MetaImpl {
    * Implements {@link DatabaseMetaData#getTables}.
    */
   @Override
-  public MetaResultSet getTables(String catalog, final Pat schemaPattern, final Pat tableNamePattern,
-      final List<String> typeList) {
+  public MetaResultSet getTables(ConnectionHandle ch,
+                                 String catalog,
+                                 Pat schemaPattern,
+                                 Pat tableNamePattern,
+                                 List<String> typeList) {
     if (connection.getConfig().isServerMetadataDisabled() || ! connection.getClient().getSupportedMethods().contains(ServerMethod.GET_TABLES)) {
       return clientGetTables(catalog, schemaPattern, tableNamePattern, typeList);
     }
@@ -962,8 +969,7 @@ class DrillMetaImpl extends MetaImpl {
    * Implements {@link DatabaseMetaData#getColumns}.
    */
   @Override
-  public MetaResultSet getColumns(String catalog, Pat schemaPattern,
-                              Pat tableNamePattern, Pat columnNamePattern) {
+  public MetaResultSet getColumns(ConnectionHandle ch, String catalog, Pat schemaPattern, Pat tableNamePattern, Pat columnNamePattern) {
     if (connection.getConfig().isServerMetadataDisabled() || ! connection.getClient().getSupportedMethods().contains(ServerMethod.GET_COLUMNS)) {
       return clientGetColumns(catalog, schemaPattern, tableNamePattern, columnNamePattern);
     }
@@ -1022,7 +1028,7 @@ class DrillMetaImpl extends MetaImpl {
    * Implements {@link DatabaseMetaData#getSchemas}.
    */
   @Override
-  public MetaResultSet getSchemas(String catalog, Pat schemaPattern) {
+  public MetaResultSet getSchemas(ConnectionHandle ch, String catalog, Pat schemaPattern) {
     if (connection.getConfig().isServerMetadataDisabled() || ! connection.getClient().getSupportedMethods().contains(ServerMethod.GET_SCHEMAS)) {
       return clientGetSchemas(catalog, schemaPattern);
     }
@@ -1069,7 +1075,7 @@ class DrillMetaImpl extends MetaImpl {
    * Implements {@link DatabaseMetaData#getCatalogs}.
    */
   @Override
-  public MetaResultSet getCatalogs() {
+  public MetaResultSet getCatalogs(ConnectionHandle ch) {
     if (connection.getConfig().isServerMetadataDisabled() || ! connection.getClient().getSupportedMethods().contains(ServerMethod.GET_CATALOGS)) {
       return clientGetCatalogs();
     }
@@ -1106,7 +1112,54 @@ class DrillMetaImpl extends MetaImpl {
   }
 
   @Override
+  public ExecuteResult prepareAndExecute(final StatementHandle handle, final String sql, final long maxRowCount,
+                                         int maxRowsInFirstFrame, final PrepareCallback callback) throws NoSuchStatementException {
+    return prepareAndExecute(handle, sql, maxRowCount, callback);
+  }
+
+  @Override
+  public ExecuteBatchResult prepareAndExecuteBatch(StatementHandle statementHandle, List<String> list) throws NoSuchStatementException {
+    throw new UnsupportedOperationException(this.getClass().getSimpleName());
+  }
+
+  @Override
+  public ExecuteBatchResult executeBatch(StatementHandle statementHandle, List<List<TypedValue>> list) throws NoSuchStatementException {
+    throw new UnsupportedOperationException(this.getClass().getSimpleName());
+  }
+
+  @Override
+  public Frame fetch(StatementHandle statementHandle, long l, int i) throws NoSuchStatementException, MissingResultsException {
+    throw new UnsupportedOperationException(this.getClass().getSimpleName());
+  }
+
+  @Override
+  public ExecuteResult execute(StatementHandle statementHandle, List<TypedValue> list, long l) throws NoSuchStatementException {
+    throw new UnsupportedOperationException(this.getClass().getSimpleName());
+  }
+
+  @Override
+  public ExecuteResult execute(StatementHandle statementHandle, List<TypedValue> list, int i) throws NoSuchStatementException {
+    return null;
+  }
+
+  @Override
   public void closeStatement(StatementHandle h) {
     // Nothing
   }
+
+  @Override
+  public boolean syncResults(StatementHandle statementHandle, QueryState queryState, long l) throws NoSuchStatementException {
+    throw new UnsupportedOperationException(this.getClass().getSimpleName());
+  }
+
+  @Override
+  public void commit(ConnectionHandle connectionHandle) {
+    throw new UnsupportedOperationException(this.getClass().getSimpleName());
+  }
+
+  @Override
+  public void rollback(ConnectionHandle connectionHandle) {
+    throw new UnsupportedOperationException(this.getClass().getSimpleName());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
index f1ba4c1..a45412f 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java
@@ -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
@@ -58,13 +58,9 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
           resultSetType, resultSetConcurrency, resultSetHoldability);
     connection.openStatementsRegistry.addStatement(this);
     this.preparedStatementHandle = preparedStatementHandle;
-    if (preparedStatementHandle != null) {
-      ((DrillColumnMetaDataList) signature.columns).updateColumnMetaData(preparedStatementHandle.getColumnsList());
-    }
+    ((DrillColumnMetaDataList) signature.columns).updateColumnMetaData(preparedStatementHandle.getColumnsList());
   }
 
-
-
   /**
    * Throws AlreadyClosedSqlException <i>iff</i> this PreparedStatement is closed.
    *
@@ -333,13 +329,17 @@ abstract class DrillPreparedStatementImpl extends AvaticaPreparedStatement
   }
 
   @Override
-  public void clearBatch() throws SQLException {
-    throwIfClosed();
+  public void clearBatch() throws RuntimeException {
+    try {
+      throwIfClosed();
+    } catch (AlreadyClosedSqlException e) {
+      throw new RuntimeException(e);
+    }
     try {
       super.clearBatch();
     }
     catch (UnsupportedOperationException e) {
-      throw new SQLFeatureNotSupportedException(e.getMessage(), e);
+      throw new RuntimeException(new SQLFeatureNotSupportedException(e.getMessage(), e));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/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 c8b4e3d..f4fc588 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
@@ -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
@@ -48,6 +48,7 @@ import org.apache.calcite.avatica.AvaticaSite;
 import org.apache.calcite.avatica.AvaticaStatement;
 import org.apache.calcite.avatica.ColumnMetaData;
 import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.QueryState;
 import org.apache.calcite.avatica.util.Cursor;
 import org.apache.calcite.avatica.util.Cursor.Accessor;
 import org.apache.drill.jdbc.AlreadyClosedSqlException;
@@ -66,10 +67,10 @@ class DrillResultSetImpl extends AvaticaResultSet implements DrillResultSet {
   private final DrillConnectionImpl connection;
   private volatile boolean hasPendingCancelationNotification = false;
 
-  DrillResultSetImpl(AvaticaStatement statement, Meta.Signature signature,
+  DrillResultSetImpl(AvaticaStatement statement, QueryState state, Meta.Signature signature,
                      ResultSetMetaData resultSetMetaData, TimeZone timeZone,
                      Meta.Frame firstFrame) {
-    super(statement, signature, resultSetMetaData, timeZone, firstFrame);
+    super(statement, state, signature, resultSetMetaData, timeZone, firstFrame);
     connection = (DrillConnectionImpl) statement.getConnection();
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataGetColumnsTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataGetColumnsTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataGetColumnsTest.java
index ed95162..ac84844 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataGetColumnsTest.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/DatabaseMetaDataGetColumnsTest.java
@@ -128,7 +128,7 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTestBase {
   private static ResultSet mdrReqCHAR_5;
   // No NCHAR, etc., in Drill (?).
   private static ResultSet mdrOptVARBINARY_16;
-  private static ResultSet mdrOptBINARY_1048576;
+  private static ResultSet mdrOptBINARY_65536;
 
   private static ResultSet mdrReqDATE;
   private static ResultSet mdrReqTIME;
@@ -256,7 +256,7 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTestBase {
         + "\n  CAST( NULL    AS VARCHAR            ) AS mdrOptVARCHAR,        "
         + "\n  CAST( '55'    AS CHAR(5)            ) AS mdrReqCHAR_5,         "
         + "\n  CAST( NULL    AS VARBINARY(16)      ) AS mdrOptVARBINARY_16,   "
-        + "\n  CAST( NULL    AS VARBINARY(1048576) ) AS mdrOptBINARY_1048576, "
+        + "\n  CAST( NULL    AS VARBINARY(65536)   ) AS mdrOptBINARY_65536,   "
         + "\n  CAST( NULL    AS BINARY(8)          ) AS mdrOptBINARY_8,       "
         + "\n  "
         + "\n                   DATE '2015-01-01'    AS mdrReqDATE,           "
@@ -307,7 +307,7 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTestBase {
     mdrOptVARCHAR        = setUpRow( DFS_TMP_SCHEMA, VIEW_NAME, "mdrOptVARCHAR" );
     mdrReqCHAR_5         = setUpRow( DFS_TMP_SCHEMA, VIEW_NAME, "mdrReqCHAR_5" );
     mdrOptVARBINARY_16   = setUpRow( DFS_TMP_SCHEMA, VIEW_NAME, "mdrOptVARBINARY_16" );
-    mdrOptBINARY_1048576 = setUpRow( DFS_TMP_SCHEMA, VIEW_NAME, "mdrOptBINARY_1048576" );
+    mdrOptBINARY_65536   = setUpRow( DFS_TMP_SCHEMA, VIEW_NAME, "mdrOptBINARY_65536" );
 
     mdrReqDATE           = setUpRow( DFS_TMP_SCHEMA, VIEW_NAME, "mdrReqDATE" );
     mdrReqTIME           = setUpRow( DFS_TMP_SCHEMA, VIEW_NAME, "mdrReqTIME" );
@@ -671,7 +671,7 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTestBase {
 
   @Test
   public void test_DATA_TYPE_hasRightValue_mdrOptBINARY_1048576CHECK() throws SQLException {
-    assertThat( getIntOrNull( mdrOptBINARY_1048576, "DATA_TYPE" ), equalTo( Types.VARBINARY ) );
+    assertThat( getIntOrNull(mdrOptBINARY_65536, "DATA_TYPE" ), equalTo( Types.VARBINARY ) );
   }
 
   @Test
@@ -849,7 +849,7 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTestBase {
 
   @Test
   public void test_TYPE_NAME_hasRightValue_mdrOptBINARY_1048576CHECK() throws SQLException {
-    assertThat( mdrOptBINARY_1048576.getString( "TYPE_NAME" ),
+    assertThat( mdrOptBINARY_65536.getString( "TYPE_NAME" ),
                 equalTo( "BINARY VARYING" ) );
   }
 
@@ -1039,7 +1039,7 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTestBase {
 
   @Test
   public void test_COLUMN_SIZE_hasRightValue_mdrOptBINARY_1048576() throws SQLException {
-    assertThat( getIntOrNull( mdrOptBINARY_1048576, "COLUMN_SIZE" ), equalTo( 1048576 ) );
+    assertThat( getIntOrNull(mdrOptBINARY_65536, "COLUMN_SIZE" ), equalTo( 65536 ) );
   }
 
   @Test
@@ -1109,13 +1109,16 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTestBase {
                 equalTo( 12 ) );  // "P123DT12H12M"
   }
 
-  @Ignore( "TODO(DRILL-3244): unignore when fractional secs. prec. is right" )
+  //Fixed with Calcite update
+  //@Ignore( "TODO(DRILL-3244): unignore when fractional secs. prec. is right" )
   @Test
   public void test_COLUMN_SIZE_hasRightValue_mdrReqINTERVAL_2D_S5() throws SQLException {
     assertThat( getIntOrNull( mdrReqINTERVAL_2D_S5, "COLUMN_SIZE" ),
                 equalTo( 20 ) );  // "P12DT12H12M12.12345S"
   }
 
+  @Ignore( "Ignored after Calcite update" )
+  @Test
   public void test_COLUMN_SIZE_hasINTERIMValue_mdrReqINTERVAL_2D_S5() throws SQLException {
     assertThat( "When DRILL-3244 fixed, un-ignore above method and purge this.",
                 getIntOrNull( mdrReqINTERVAL_2D_S5, "COLUMN_SIZE" ),
@@ -1134,13 +1137,15 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTestBase {
                 equalTo( 7 ) );  // "PT1H12M"
   }
 
-  @Ignore( "TODO(DRILL-3244): unignore when fractional secs. prec. is right" )
+  //Fixed with Calcite update
+  //@Ignore( "TODO(DRILL-3244): unignore when fractional secs. prec. is right" )
   @Test
   public void test_COLUMN_SIZE_hasRightValue_mdrReqINTERVAL_3H_S1() throws SQLException {
     assertThat( getIntOrNull( mdrReqINTERVAL_3H_S1, "COLUMN_SIZE" ),
                 equalTo( 14 ) );  // "PT123H12M12.1S"
   }
 
+  @Ignore( "Ignored after Calcite update" )
   @Test
   public void test_COLUMN_SIZE_hasINTERIMValue_mdrReqINTERVAL_3H_S1() throws SQLException {
     assertThat( "When DRILL-3244 fixed, un-ignore above method and purge this.",
@@ -1172,13 +1177,15 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTestBase {
                 equalTo( 13 ) );  // "PT123.123456S"
   }
 
-  @Ignore( "TODO(DRILL-3244): unignore when fractional secs. prec. is right" )
+  //Fixed with Calcite update
+  //@Ignore( "TODO(DRILL-3244): unignore when fractional secs. prec. is right" )
   @Test
   public void test_COLUMN_SIZE_hasRightValue_mdrReqINTERVAL_3S1() throws SQLException {
     assertThat( getIntOrNull( mdrReqINTERVAL_3S1, "COLUMN_SIZE" ),
                 equalTo( 8 ) );  // "PT123.1S"
   }
 
+  @Ignore( "Ignored after Calcite update" )
   @Test
   public void test_COLUMN_SIZE_hasINTERIMValue_mdrReqINTERVAL_3S1() throws SQLException {
     assertThat( "When DRILL-3244 fixed, un-ignore above method and purge this.",
@@ -1342,7 +1349,7 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTestBase {
 
   @Test
   public void test_DECIMAL_DIGITS_hasRightValue_mdrOptBINARY_1048576CHECK() throws SQLException {
-    assertThat( getIntOrNull( mdrOptBINARY_1048576, "DECIMAL_DIGITS" ), nullValue() );
+    assertThat( getIntOrNull(mdrOptBINARY_65536, "DECIMAL_DIGITS" ), nullValue() );
   }
 
   @Test
@@ -1420,12 +1427,14 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTestBase {
     assertThat( getIntOrNull( mdrReqINTERVAL_3D_Mi, "DECIMAL_DIGITS" ), equalTo( 6 ) );
   }
 
-  @Ignore( "TODO(DRILL-3244): unignore when fractional secs. prec. is right" )
+  //Fixed with Calcite update
+  //@Ignore( "TODO(DRILL-3244): unignore when fractional secs. prec. is right" )
   @Test
   public void test_DECIMAL_DIGITS_hasRightValue_mdrReqINTERVAL_2D_S5() throws SQLException {
     assertThat( getIntOrNull( mdrReqINTERVAL_2D_S5, "DECIMAL_DIGITS" ), equalTo( 5 ) );
   }
 
+  @Ignore( "Ignored after Calcite update" )
   @Test
   public void test_DECIMAL_DIGITS_hasINTERIMValue_mdrReqINTERVAL_2D_S5() throws SQLException {
     assertThat( "When DRILL-3244 fixed, un-ignore above method and purge this.",
@@ -1450,12 +1459,14 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTestBase {
     assertThat( getIntOrNull( mdrReqINTERVAL_1H_Mi, "DECIMAL_DIGITS" ), equalTo( 6 ) );
   }
 
-  @Ignore( "TODO(DRILL-3244): unignore when fractional secs. prec. is right" )
+  //Fixed with Calcite update
+  //@Ignore( "TODO(DRILL-3244): unignore when fractional secs. prec. is right" )
   @Test
   public void test_DECIMAL_DIGITS_hasRightValue_mdrReqINTERVAL_3H_S1() throws SQLException {
     assertThat( getIntOrNull( mdrReqINTERVAL_3H_S1, "DECIMAL_DIGITS" ), equalTo( 1 ) );
   }
 
+  @Ignore( "Ignored after Calcite update" )
   @Test
   public void test_DECIMAL_DIGITS_hasINTERIMValue_mdrReqINTERVAL_3H_S1() throws SQLException {
     assertThat( "When DRILL-3244 fixed, un-ignore above method and purge this.",
@@ -1640,7 +1651,7 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTestBase {
 
   @Test
   public void test_NUM_PREC_RADIX_hasRightValue_mdrOptBINARY_1048576CHECK() throws SQLException {
-    assertThat( getIntOrNull( mdrOptBINARY_1048576, "NUM_PREC_RADIX" ), nullValue() );
+    assertThat( getIntOrNull(mdrOptBINARY_65536, "NUM_PREC_RADIX" ), nullValue() );
   }
 
   @Test
@@ -1824,7 +1835,7 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTestBase {
   @Test
   public void test_NULLABLE_hasRightValue_mdrOptBINARY_1048576() throws SQLException {
     assertThat( "ResultSetMetaData.column...Null... nullability code:",
-                getIntOrNull( mdrOptBINARY_1048576, "NULLABLE" ), equalTo( columnNullable ) );
+                getIntOrNull(mdrOptBINARY_65536, "NULLABLE" ), equalTo( columnNullable ) );
   }
 
   @Test
@@ -2186,7 +2197,7 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTestBase {
 
   @Test
   public void test_CHAR_OCTET_LENGTH_hasRightValue_mdrOptBINARY_1048576CHECK() throws SQLException {
-    assertThat( getIntOrNull( mdrOptBINARY_1048576, "CHAR_OCTET_LENGTH" ), nullValue() );
+    assertThat( getIntOrNull(mdrOptBINARY_65536, "CHAR_OCTET_LENGTH" ), nullValue() );
   }
 
   @Test
@@ -2443,7 +2454,7 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTestBase {
 
   @Test
   public void test_IS_NULLABLE_hasRightValue_mdrOptBINARY_1048576CHECK() throws SQLException {
-    assertThat( mdrOptBINARY_1048576.getString( "IS_NULLABLE" ), equalTo( "YES" ) );
+    assertThat( mdrOptBINARY_65536.getString( "IS_NULLABLE" ), equalTo( "YES" ) );
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestInformationSchemaColumns.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestInformationSchemaColumns.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestInformationSchemaColumns.java
index 7e0934c..214b7ca 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestInformationSchemaColumns.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestInformationSchemaColumns.java
@@ -99,7 +99,7 @@ public class TestInformationSchemaColumns extends JdbcTestBase {
   private static ResultSet mdrReqCHAR_5;
   // No NCHAR, etc., in Drill (?).
   private static ResultSet mdrOptVARBINARY_16;
-  private static ResultSet mdrOptBINARY_1048576;
+  private static ResultSet mdrOptBINARY_65536;
 
   private static ResultSet mdrReqDATE;
   private static ResultSet mdrReqTIME;
@@ -236,7 +236,7 @@ public class TestInformationSchemaColumns extends JdbcTestBase {
         + "\n  CAST( NULL    AS VARCHAR            ) AS mdrOptVARCHAR,        "
         + "\n  CAST( '55'    AS CHAR(5)            ) AS mdrReqCHAR_5,         "
         + "\n  CAST( NULL    AS VARBINARY(16)      ) AS mdrOptVARBINARY_16,   "
-        + "\n  CAST( NULL    AS VARBINARY(1048576) ) AS mdrOptBINARY_1048576, "
+        + "\n  CAST( NULL    AS VARBINARY(65536)   ) AS mdrOptBINARY_65536,   "
         + "\n  CAST( NULL    AS BINARY(8)          ) AS mdrOptBINARY_8,       "
         + "\n  "
         + "\n                   DATE '2015-01-01'    AS mdrReqDATE,           "
@@ -287,7 +287,7 @@ public class TestInformationSchemaColumns extends JdbcTestBase {
     mdrOptVARCHAR        = setUpRow( DFS_TMP_SCHEMA, VIEW_NAME, "mdrOptVARCHAR" );
     mdrReqCHAR_5         = setUpRow( DFS_TMP_SCHEMA, VIEW_NAME, "mdrReqCHAR_5" );
     mdrOptVARBINARY_16   = setUpRow( DFS_TMP_SCHEMA, VIEW_NAME, "mdrOptVARBINARY_16" );
-    mdrOptBINARY_1048576 = setUpRow( DFS_TMP_SCHEMA, VIEW_NAME, "mdrOptBINARY_1048576" );
+    mdrOptBINARY_65536   = setUpRow( DFS_TMP_SCHEMA, VIEW_NAME, "mdrOptBINARY_65536" );
 
     mdrReqDATE           = setUpRow( DFS_TMP_SCHEMA, VIEW_NAME, "mdrReqDATE" );
     mdrReqTIME           = setUpRow( DFS_TMP_SCHEMA, VIEW_NAME, "mdrReqTIME" );
@@ -829,7 +829,7 @@ public class TestInformationSchemaColumns extends JdbcTestBase {
   @Test
   public void test_IS_NULLABLE_hasRightValue_mdrOptBINARY_1048576() throws SQLException {
     assertThat( "ResultSetMetaData.column...Null... nullability code:",
-                mdrOptBINARY_1048576.getString( "IS_NULLABLE" ), equalTo( "YES" ) );
+                mdrOptBINARY_65536.getString( "IS_NULLABLE" ), equalTo( "YES" ) );
   }
 
   @Test
@@ -1006,8 +1006,8 @@ public class TestInformationSchemaColumns extends JdbcTestBase {
   @Ignore( "TODO(DRILL-3368): unignore when BINARY is implemented enough" )
   @Test
   public void test_DATA_TYPE_hasRightValue_mdrOptBINARY_1048576() throws SQLException {
-    assertThat( mdrOptBINARY_1048576.getString( "DATA_TYPE" ), equalTo( "BINARY VARYING" ) ); // ?? current
-    assertThat( mdrOptBINARY_1048576.getString( "DATA_TYPE" ), equalTo( "BINARY" ) );  // ?? should be
+    assertThat( mdrOptBINARY_65536.getString( "DATA_TYPE" ), equalTo( "BINARY VARYING" ) ); // ?? current
+    assertThat( mdrOptBINARY_65536.getString( "DATA_TYPE" ), equalTo( "BINARY" ) );  // ?? should be
   }
 
   @Test
@@ -1172,7 +1172,7 @@ public class TestInformationSchemaColumns extends JdbcTestBase {
 
   @Test
   public void test_CHARACTER_MAXIMUM_LENGTH_hasRightValue_mdrOptBINARY_1048576() throws SQLException {
-    assertThat( getIntOrNull( mdrOptBINARY_1048576, "CHARACTER_MAXIMUM_LENGTH" ), equalTo( 1048576 ) );
+    assertThat( getIntOrNull(mdrOptBINARY_65536, "CHARACTER_MAXIMUM_LENGTH" ), equalTo( 65536 ) );
   }
 
   @Test
@@ -1339,7 +1339,7 @@ public class TestInformationSchemaColumns extends JdbcTestBase {
 
   @Test
   public void test_CHARACTER_OCTET_LENGTH_hasRightValue_mdrOptBINARY_1048576() throws SQLException {
-    assertThat( getIntOrNull( mdrOptBINARY_1048576, "CHARACTER_OCTET_LENGTH" ), equalTo( 1048576 ));
+    assertThat( getIntOrNull(mdrOptBINARY_65536, "CHARACTER_OCTET_LENGTH" ), equalTo( 65536 ));
   }
 
   @Test
@@ -1499,7 +1499,7 @@ public class TestInformationSchemaColumns extends JdbcTestBase {
 
   @Test
   public void test_NUMERIC_PRECISION_hasRightValue_mdrOptBINARY_1048576() throws SQLException {
-    assertThat( getIntOrNull( mdrOptBINARY_1048576, "NUMERIC_PRECISION" ), nullValue() );
+    assertThat( getIntOrNull(mdrOptBINARY_65536, "NUMERIC_PRECISION" ), nullValue() );
   }
 
   @Test
@@ -1659,7 +1659,7 @@ public class TestInformationSchemaColumns extends JdbcTestBase {
 
   @Test
   public void test_NUMERIC_PRECISION_RADIX_hasRightValue_mdrOptBINARY_1048576() throws SQLException {
-    assertThat( getIntOrNull( mdrOptBINARY_1048576, "NUMERIC_PRECISION_RADIX" ), nullValue() );
+    assertThat( getIntOrNull(mdrOptBINARY_65536, "NUMERIC_PRECISION_RADIX" ), nullValue() );
   }
 
   @Test
@@ -1819,7 +1819,7 @@ public class TestInformationSchemaColumns extends JdbcTestBase {
 
   @Test
   public void test_NUMERIC_SCALE_hasRightValue_mdrOptBINARY_1048576() throws SQLException {
-    assertThat( getIntOrNull( mdrOptBINARY_1048576, "NUMERIC_SCALE" ), nullValue() );
+    assertThat( getIntOrNull(mdrOptBINARY_65536, "NUMERIC_SCALE" ), nullValue() );
   }
 
   @Test
@@ -1979,7 +1979,7 @@ public class TestInformationSchemaColumns extends JdbcTestBase {
 
   @Test
   public void test_DATETIME_PRECISION_hasRightValue_mdrOptBINARY_1048576() throws SQLException {
-    assertThat( getIntOrNull( mdrOptBINARY_1048576, "DATETIME_PRECISION" ), nullValue() );
+    assertThat( getIntOrNull(mdrOptBINARY_65536, "DATETIME_PRECISION" ), nullValue() );
   }
 
   @Test
@@ -2050,12 +2050,14 @@ public class TestInformationSchemaColumns extends JdbcTestBase {
     assertThat( getIntOrNull( mdrReqINTERVAL_3D_Mi, "DATETIME_PRECISION" ), equalTo( 6 ) );
   }
 
-  @Ignore( "TODO(DRILL-3244): unignore when fractional secs. prec. is right" )
+  //Fixed with Calcite update
+  //@Ignore( "TODO(DRILL-3244): unignore when fractional secs. prec. is right" )
   @Test
   public void test_DATETIME_PRECISION_hasRightValue_mdrReqINTERVAL_2D_S5() throws SQLException {
     assertThat( getIntOrNull( mdrReqINTERVAL_2D_S5, "DATETIME_PRECISION" ), equalTo( 5 ) );
   }
 
+  @Ignore( "Ignored after Calcite update" )
   @Test
   public void test_DATETIME_PRECISION_hasINTERIMValue_mdrReqINTERVAL_2D_S5() throws SQLException {
     assertThat( "When DRILL-3244 fixed, un-ignore above method and purge this.",
@@ -2080,12 +2082,14 @@ public class TestInformationSchemaColumns extends JdbcTestBase {
     assertThat( getIntOrNull( mdrReqINTERVAL_1H_Mi, "DATETIME_PRECISION" ), equalTo( 6 ) );
   }
 
-  @Ignore( "TODO(DRILL-3244): unignore when fractional secs. prec. is right" )
+  //Fixed with Calcite update
+  //@Ignore( "TODO(DRILL-3244): unignore when fractional secs. prec. is right" )
   @Test
   public void test_DATETIME_PRECISION_hasRightValue_mdrReqINTERVAL_H_S3() throws SQLException {
     assertThat( getIntOrNull( mdrReqINTERVAL_3H_S1, "DATETIME_PRECISION" ), equalTo( 1 ) );
   }
 
+  @Ignore( "Ignored after Calcite update" )
   @Test
   public void test_DATETIME_PRECISION_hasINTERIMValue_mdrReqINTERVAL_H_S3() throws SQLException {
     assertThat( "When DRILL-3244 fixed, un-ignore above method and purge this.",
@@ -2119,12 +2123,14 @@ public class TestInformationSchemaColumns extends JdbcTestBase {
     assertThat( getIntOrNull( mdrReqINTERVAL_3S, "DATETIME_PRECISION" ), equalTo( 6 ) );
   }
 
-  @Ignore( "TODO(DRILL-3244): unignore when fractional secs. prec. is right" )
+  //Fixed with Calcite update
+  //@Ignore( "TODO(DRILL-3244): unignore when fractional secs. prec. is right" )
   @Test
   public void test_DATETIME_PRECISION_hasRightValue_mdrReqINTERVAL_3S1() throws SQLException {
     assertThat( getIntOrNull( mdrReqINTERVAL_3S1, "DATETIME_PRECISION" ), equalTo( 1 ) );
   }
 
+  @Ignore( "Ignored after Calcite update" )
   @Test
   public void test_DATETIME_PRECISION_hasINTERIMValue_mdrReqINTERVAL_3S1() throws SQLException {
     assertThat( "When DRILL-3244 fixed, un-ignore above method and purge this.",
@@ -2258,7 +2264,7 @@ public class TestInformationSchemaColumns extends JdbcTestBase {
 
   @Test
   public void test_INTERVAL_TYPE_hasRightValue_mdrOptBINARY_1048576() throws SQLException {
-    assertThat( mdrOptBINARY_1048576.getString( "INTERVAL_TYPE" ), nullValue() );
+    assertThat( mdrOptBINARY_65536.getString( "INTERVAL_TYPE" ), nullValue() );
   }
 
   @Test
@@ -2491,7 +2497,7 @@ public class TestInformationSchemaColumns extends JdbcTestBase {
 
   @Test
   public void test_INTERVAL_PRECISION_hasRightValue_mdrOptBINARY_1048576() throws SQLException {
-    assertThat( getIntOrNull( mdrOptBINARY_1048576, "INTERVAL_PRECISION" ), nullValue() );
+    assertThat( getIntOrNull(mdrOptBINARY_65536, "INTERVAL_PRECISION" ), nullValue() );
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6189cf2..ce64591 100644
--- a/pom.xml
+++ b/pom.xml
@@ -36,7 +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.4.0-drill-r23</calcite.version>
+    <calcite.version>1.13.0-drill-r0</calcite.version>
+    <avatica.version>1.10.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>
@@ -57,7 +58,7 @@
     <hive.version>1.2.1</hive.version>
     <hadoop.version>2.7.1</hadoop.version>
     <hbase.version>1.1.3</hbase.version>
-    <fmpp.version>0.9.15</fmpp.version>
+    <fmpp.version>1.0</fmpp.version>
     <freemarker.version>2.3.26-incubating</freemarker.version>
     <javassist.version>3.16.1-GA</javassist.version>
     <msgpack.version>0.6.6</msgpack.version>
@@ -717,6 +718,25 @@
   <dependencyManagement>
     <dependencies>
       <dependency>
+        <groupId>org.apache.calcite.avatica</groupId>
+        <artifactId>avatica</artifactId>
+        <version>${avatica.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.apache.calcite.avatica</groupId>
+            <artifactId>avatica-core</artifactId>
+          </exclusion>
+          <exclusion>
+            <groupId>commons-logging</groupId>
+            <artifactId>commons-logging</artifactId>
+          </exclusion>
+          <exclusion>
+            <artifactId>protobuf-java</artifactId>
+            <groupId>com.google.protobuf</groupId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
         <groupId>org.msgpack</groupId>
         <artifactId>msgpack</artifactId>
         <version>${msgpack.version}</version>
@@ -1723,6 +1743,10 @@
             <version>${calcite.version}</version>
             <exclusions>
               <exclusion>
+                <groupId>org.apache.calcite.avatica</groupId>
+                <artifactId>avatica-core</artifactId>
+              </exclusion>
+              <exclusion>
                 <groupId>org.jgrapht</groupId>
                 <artifactId>jgrapht-jdk1.5</artifactId>
               </exclusion>

http://git-wip-us.apache.org/repos/asf/drill/blob/9fabb612/tools/fmpp/pom.xml
----------------------------------------------------------------------
diff --git a/tools/fmpp/pom.xml b/tools/fmpp/pom.xml
index 91e686c..4f4b7e9 100644
--- a/tools/fmpp/pom.xml
+++ b/tools/fmpp/pom.xml
@@ -45,9 +45,19 @@
       <version>3.3.3</version>
     </dependency>
     <dependency>
-      <groupId>net.sourceforge.fmpp</groupId>
-      <artifactId>fmpp</artifactId>
+      <groupId>com.googlecode.fmpp-maven-plugin</groupId>
+      <artifactId>fmpp-maven-plugin</artifactId>
       <version>${fmpp.version}</version>
+      <exclusions>
+        <exclusion>
+          <artifactId>log4j</artifactId>
+          <groupId>log4j</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>commons-logging-api</artifactId>
+          <groupId>commons-logging</groupId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.freemarker</groupId>


[06/18] drill git commit: DRILL-3993: Use custom RelBuilder implementation in rules

Posted by am...@apache.org.
DRILL-3993: Use custom RelBuilder implementation in rules

After the changes, made in CALCITE-1056 if the filter has a predicate that is always false, RelBuilder.filter() method returns values rel node instead of filter rel node. In order to preserve column types, DrillRelBuilder.empty() method, which is returned by filter method  was overridden, and now it returns filter with a false predicate. (advice to override this method was in its javadoc) The goal of all other changes in this commit is to use our custom RelBuilder for all rules that are used in Drill.


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

Branch: refs/heads/master
Commit: 0a525aab638016a74cae11f8b2430b5a96ffc4af
Parents: 3896a58
Author: Volodymyr Vysotskyi <vv...@gmail.com>
Authored: Wed Nov 15 12:37:50 2017 +0200
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Tue Jan 16 12:10:13 2018 +0200

----------------------------------------------------------------------
 .../exec/store/jdbc/DrillJdbcRuleBase.java      |  10 +-
 .../exec/store/jdbc/JdbcStoragePlugin.java      |  18 ++--
 .../drill/exec/planner/DrillRelBuilder.java     |  69 ++++++++++++
 .../apache/drill/exec/planner/PlannerPhase.java |  64 +++++------
 .../apache/drill/exec/planner/RuleInstance.java | 105 +++++++++++++++++++
 .../drill/exec/planner/StarColumnHelper.java    |   5 +-
 .../exec/planner/common/DrillRelOptUtil.java    |   7 +-
 .../planner/logical/DrillAggregateRule.java     |   6 +-
 .../planner/logical/DrillConstExecutor.java     |   7 +-
 .../DrillFilterAggregateTransposeRule.java      |  14 +--
 .../planner/logical/DrillFilterJoinRules.java   |  52 +++++----
 .../exec/planner/logical/DrillFilterRule.java   |  16 +--
 .../exec/planner/logical/DrillJoinRule.java     |   5 +-
 .../exec/planner/logical/DrillLimitRule.java    |   9 +-
 .../planner/logical/DrillMergeProjectRule.java  |  13 ++-
 .../exec/planner/logical/DrillProjectRule.java  |  11 +-
 .../logical/DrillProjectSetOpTransposeRule.java |   3 +-
 .../logical/DrillPushFilterPastProjectRule.java |  14 ++-
 .../logical/DrillPushLimitToScanRule.java       |  22 ++--
 .../planner/logical/DrillPushProjIntoScan.java  |  15 ++-
 .../logical/DrillPushProjectPastFilterRule.java |   5 +-
 .../logical/DrillPushProjectPastJoinRule.java   |   3 +-
 .../logical/DrillReduceAggregatesRule.java      |  10 +-
 .../logical/DrillReduceExpressionsRule.java     |   7 +-
 .../exec/planner/logical/DrillRelFactories.java |  38 ++++++-
 .../exec/planner/logical/DrillScanRule.java     |   7 +-
 .../exec/planner/logical/DrillSortRule.java     |   5 +-
 .../exec/planner/logical/DrillUnionAllRule.java |   3 +-
 .../exec/planner/logical/DrillValuesRule.java   |   7 +-
 .../exec/planner/logical/DrillWindowRule.java   |   5 +-
 .../drill/exec/planner/physical/Prule.java      |  15 ++-
 .../exec/planner/physical/ValuesPrule.java      |  10 +-
 .../planner/sql/handlers/SqlHandlerUtil.java    |   4 +-
 .../exec/store/StoragePluginOptimizerRule.java  |   5 +-
 .../TestCorruptParquetDateCorrection.java       |  70 ++++---------
 .../resources/record/test_recorditerator.json   |   2 +-
 36 files changed, 422 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/DrillJdbcRuleBase.java
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/DrillJdbcRuleBase.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/DrillJdbcRuleBase.java
index bbb4daf..963e80a 100644
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/DrillJdbcRuleBase.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/DrillJdbcRuleBase.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.store.jdbc;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.base.Predicates;
 import org.apache.calcite.adapter.jdbc.JdbcConvention;
 import org.apache.calcite.adapter.jdbc.JdbcRules;
 import org.apache.calcite.plan.Convention;
@@ -34,6 +35,7 @@ import org.apache.calcite.rex.RexNode;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
+import org.apache.drill.exec.planner.logical.DrillRelFactories;
 
 abstract class DrillJdbcRuleBase extends ConverterRule {
 
@@ -50,7 +52,7 @@ abstract class DrillJdbcRuleBase extends ConverterRule {
   protected final JdbcConvention out;
 
   private DrillJdbcRuleBase(Class<? extends RelNode> clazz, RelTrait in, JdbcConvention out, String description) {
-    super(clazz, in, out, description);
+    super(clazz, Predicates.<RelNode>alwaysTrue(), in, out, DrillRelFactories.LOGICAL_BUILDER, description);
     this.out = out;
   }
 
@@ -71,7 +73,7 @@ abstract class DrillJdbcRuleBase extends ConverterRule {
     public boolean matches(RelOptRuleCall call) {
       try {
 
-        final LogicalProject project = (LogicalProject) call.rel(0);
+        final LogicalProject project = call.rel(0);
         for (RexNode node : project.getChildExps()) {
           if (!checkedExpressions.get(node)) {
             return false;
@@ -102,7 +104,7 @@ abstract class DrillJdbcRuleBase extends ConverterRule {
     public boolean matches(RelOptRuleCall call) {
       try {
 
-        final LogicalFilter filter = (LogicalFilter) call.rel(0);
+        final LogicalFilter filter = call.rel(0);
         for (RexNode node : filter.getChildExps()) {
           if (!checkedExpressions.get(node)) {
             return false;

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/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 9efb7af..4b41696 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
@@ -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
@@ -28,6 +28,7 @@ import java.util.Set;
 
 import javax.sql.DataSource;
 
+import com.google.common.base.Predicates;
 import org.apache.calcite.adapter.jdbc.JdbcConvention;
 import org.apache.calcite.adapter.jdbc.JdbcRules;
 import org.apache.calcite.adapter.jdbc.JdbcRules.JdbcJoin;
@@ -37,8 +38,6 @@ 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.rules.FilterSetOpTransposeRule;
-import org.apache.calcite.rel.rules.ProjectRemoveRule;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
@@ -52,7 +51,9 @@ import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.planner.RuleInstance;
 import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.DrillRelFactories;
 import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractSchema;
@@ -136,8 +137,8 @@ public class JdbcStoragePlugin extends AbstractStoragePlugin {
         builder.add(rule);
       }
 
-      builder.add(FilterSetOpTransposeRule.INSTANCE);
-      builder.add(ProjectRemoveRule.INSTANCE);
+      builder.add(RuleInstance.FILTER_SET_OP_TRANSPOSE_RULE);
+      builder.add(RuleInstance.PROJECT_REMOVE_RULE);
 
       rules = builder.build();
     }
@@ -199,13 +200,13 @@ public class JdbcStoragePlugin extends AbstractStoragePlugin {
     }
   }
 
-
   private static final JdbcPrule JDBC_PRULE_INSTANCE = new JdbcPrule();
 
   private static class JdbcPrule extends ConverterRule {
 
     private JdbcPrule() {
-      super(JdbcDrel.class, DrillRel.DRILL_LOGICAL, Prel.DRILL_PHYSICAL, "JDBC_PREL_Converter");
+      super(JdbcDrel.class, Predicates.<RelNode>alwaysTrue(), DrillRel.DRILL_LOGICAL,
+          Prel.DRILL_PHYSICAL, DrillRelFactories.LOGICAL_BUILDER, "JDBC_PREL_Converter");
     }
 
     @Override
@@ -222,7 +223,8 @@ public class JdbcStoragePlugin extends AbstractStoragePlugin {
   private class JdbcDrelConverterRule extends ConverterRule {
 
     public JdbcDrelConverterRule(DrillJdbcConvention in) {
-      super(RelNode.class, in, DrillRel.DRILL_LOGICAL, "JDBC_DREL_Converter" + in.getName());
+      super(RelNode.class, Predicates.<RelNode>alwaysTrue(), in, DrillRel.DRILL_LOGICAL,
+          DrillRelFactories.LOGICAL_BUILDER, "JDBC_DREL_Converter" + in.getName());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DrillRelBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DrillRelBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DrillRelBuilder.java
new file mode 100644
index 0000000..3923ba3
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DrillRelBuilder.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner;
+
+import org.apache.calcite.plan.Context;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptSchema;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.Util;
+
+public class DrillRelBuilder extends RelBuilder {
+  private final RelFactories.FilterFactory filterFactory;
+
+  protected DrillRelBuilder(Context context, RelOptCluster cluster, RelOptSchema relOptSchema) {
+    super(context, cluster, relOptSchema);
+    this.filterFactory =
+        Util.first(context.unwrap(RelFactories.FilterFactory.class),
+            RelFactories.DEFAULT_FILTER_FACTORY);
+  }
+
+  /**
+   * Original method {@link RelBuilder#empty} returns empty values rel.
+   * In the order to preserve dara row types, filter with false predicate is created.
+   */
+  @Override
+  public RelBuilder empty() {
+    // pops the frame from the stack and returns its relational expression
+    RelNode relNode = build();
+
+    // creates filter with false in the predicate
+    final RelNode filter = filterFactory.createFilter(relNode, cluster.getRexBuilder().makeLiteral(false));
+    push(filter);
+
+    return this;
+  }
+
+  /** Creates a {@link RelBuilderFactory}, a partially-created DrillRelBuilder.
+   * Just add a {@link RelOptCluster} and a {@link RelOptSchema} */
+  public static RelBuilderFactory proto(final Context context) {
+    return new RelBuilderFactory() {
+      public RelBuilder create(RelOptCluster cluster, RelOptSchema schema) {
+        return new DrillRelBuilder(context, cluster, schema);
+      }
+    };
+  }
+
+  /** Creates a {@link RelBuilderFactory} that uses a given set of factories. */
+  public static RelBuilderFactory proto(Object... factories) {
+    return proto(Contexts.of(factories));
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
index 513db9b..28ac51e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java
@@ -21,21 +21,9 @@ import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.ImmutableSet.Builder;
 import com.google.common.collect.Lists;
 import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.volcano.AbstractConverter.ExpandConversionRule;
 import org.apache.calcite.rel.core.RelFactories;
-import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
-import org.apache.calcite.rel.rules.AggregateRemoveRule;
-import org.apache.calcite.rel.rules.FilterMergeRule;
-import org.apache.calcite.rel.rules.JoinPushExpressionsRule;
-import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
 import org.apache.calcite.rel.rules.JoinToMultiJoinRule;
 import org.apache.calcite.rel.rules.LoptOptimizeJoinRule;
-import org.apache.calcite.rel.rules.ProjectRemoveRule;
-import org.apache.calcite.rel.rules.ProjectToWindowRule;
-import org.apache.calcite.rel.rules.ProjectWindowTransposeRule;
-import org.apache.calcite.rel.rules.ReduceExpressionsRule;
-import org.apache.calcite.rel.rules.SortRemoveRule;
-import org.apache.calcite.rel.rules.UnionToDistinctRule;
 import org.apache.calcite.tools.RuleSet;
 import org.apache.calcite.tools.RuleSets;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
@@ -92,6 +80,10 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
+/**
+ * Returns RuleSet for concrete planner phase.
+ * Only rules which use DrillRelFactories should be used in this enum.
+ */
 public enum PlannerPhase {
   //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillRuleSets.class);
 
@@ -109,8 +101,8 @@ public enum PlannerPhase {
   WINDOW_REWRITE("Window Function rewrites") {
     public RuleSet getRules(OptimizerRulesContext context, Collection<StoragePlugin> plugins) {
       return RuleSets.ofList(
-          ReduceExpressionsRule.CALC_INSTANCE,
-          ProjectToWindowRule.PROJECT
+          RuleInstance.CALC_INSTANCE,
+          RuleInstance.PROJECT_TO_LOGICAL_PROJECT_AND_WINDOW_RULE
           );
     }
   },
@@ -132,7 +124,7 @@ public enum PlannerPhase {
         rules.add(DRILL_JOIN_TO_MULTIJOIN_RULE);
         rules.add(DRILL_LOPT_OPTIMIZE_JOIN_RULE);
       }
-      rules.add(ProjectRemoveRule.INSTANCE);
+      rules.add(RuleInstance.PROJECT_REMOVE_RULE);
       return PlannerPhase.mergedRuleSets(
           RuleSets.ofList(rules),
           getStorageRules(context, plugins, this)
@@ -208,11 +200,13 @@ public enum PlannerPhase {
   }
 
 
-  static final RelOptRule DRILL_JOIN_TO_MULTIJOIN_RULE = new JoinToMultiJoinRule(DrillJoinRel.class);
-  static final RelOptRule DRILL_LOPT_OPTIMIZE_JOIN_RULE = new LoptOptimizeJoinRule(
-      DrillRelFactories.DRILL_LOGICAL_JOIN_FACTORY,
-      DrillRelFactories.DRILL_LOGICAL_PROJECT_FACTORY,
-      DrillRelFactories.DRILL_LOGICAL_FILTER_FACTORY);
+  static final RelOptRule DRILL_JOIN_TO_MULTIJOIN_RULE =
+      new JoinToMultiJoinRule(DrillJoinRel.class, DrillRelFactories.LOGICAL_BUILDER);
+  static final RelOptRule DRILL_LOPT_OPTIMIZE_JOIN_RULE =
+      new LoptOptimizeJoinRule(DrillRelBuilder.proto(
+          DrillRelFactories.DRILL_LOGICAL_JOIN_FACTORY,
+          DrillRelFactories.DRILL_LOGICAL_PROJECT_FACTORY,
+          DrillRelFactories.DRILL_LOGICAL_FILTER_FACTORY));
 
   /**
    * Get a list of logical rules that can be turned on or off by session/system options.
@@ -236,7 +230,7 @@ public enum PlannerPhase {
 
     if (ps.isConstantFoldingEnabled()) {
       // TODO - DRILL-2218
-      userConfigurableRules.add(ReduceExpressionsRule.PROJECT_INSTANCE);
+      userConfigurableRules.add(RuleInstance.PROJECT_INSTANCE);
       userConfigurableRules.add(DrillReduceExpressionsRule.FILTER_INSTANCE_DRILL);
       userConfigurableRules.add(DrillReduceExpressionsRule.CALC_INSTANCE_DRILL);
     }
@@ -250,12 +244,12 @@ public enum PlannerPhase {
    */
   final static ImmutableSet<RelOptRule> staticRuleSet = ImmutableSet.<RelOptRule> builder().add(
       // Add support for Distinct Union (by using Union-All followed by Distinct)
-      UnionToDistinctRule.INSTANCE,
+      RuleInstance.UNION_TO_DISTINCT_RULE,
 
       // Add support for WHERE style joins.
       DrillFilterJoinRules.DRILL_FILTER_ON_JOIN,
       DrillFilterJoinRules.DRILL_JOIN,
-      JoinPushExpressionsRule.INSTANCE,
+      RuleInstance.JOIN_PUSH_EXPRESSIONS_RULE,
       // End support for WHERE style joins.
 
       /*
@@ -266,12 +260,12 @@ public enum PlannerPhase {
       //FilterSetOpTransposeRule.INSTANCE,
       DrillFilterAggregateTransposeRule.INSTANCE,
 
-      FilterMergeRule.INSTANCE,
-      AggregateRemoveRule.INSTANCE,
-      ProjectRemoveRule.INSTANCE,
-      SortRemoveRule.INSTANCE,
+      RuleInstance.FILTER_MERGE_RULE,
+      RuleInstance.AGGREGATE_REMOVE_RULE,
+      RuleInstance.PROJECT_REMOVE_RULE,
+      RuleInstance.SORT_REMOVE_RULE,
 
-      AggregateExpandDistinctAggregatesRule.JOIN,
+      RuleInstance.AGGREGATE_EXPAND_DISTINCT_AGGREGATES_RULE,
       DrillReduceAggregatesRule.INSTANCE,
 
       /*
@@ -281,13 +275,13 @@ public enum PlannerPhase {
       DrillPushProjectPastJoinRule.INSTANCE,
       // Due to infinite loop in planning (DRILL-3257), temporarily disable this rule
       //DrillProjectSetOpTransposeRule.INSTANCE,
-      ProjectWindowTransposeRule.INSTANCE,
+      RuleInstance.PROJECT_WINDOW_TRANSPOSE_RULE,
       DrillPushProjIntoScan.INSTANCE,
 
       /*
        Convert from Calcite Logical to Drill Logical Rules.
        */
-      ExpandConversionRule.INSTANCE,
+      RuleInstance.EXPAND_CONVERSION_RULE,
       DrillScanRule.INSTANCE,
       DrillFilterRule.INSTANCE,
       DrillProjectRule.INSTANCE,
@@ -393,9 +387,9 @@ public enum PlannerPhase {
 
   // Ruleset for join permutation, used only in VolcanoPlanner.
   static RuleSet getJoinPermRules(OptimizerRulesContext optimizerRulesContext) {
-    return RuleSets.ofList(ImmutableSet.<RelOptRule> builder().add( //
-        JoinPushThroughJoinRule.RIGHT,
-        JoinPushThroughJoinRule.LEFT
+    return RuleSets.ofList(ImmutableSet.<RelOptRule> builder().add(
+        RuleInstance.JOIN_PUSH_THROUGH_JOIN_RULE_RIGHT,
+        RuleInstance.JOIN_PUSH_THROUGH_JOIN_RULE_LEFT
         ).build());
   }
 
@@ -404,7 +398,7 @@ public enum PlannerPhase {
     ));
 
   static final RuleSet getPhysicalRules(OptimizerRulesContext optimizerRulesContext) {
-    final List<RelOptRule> ruleList = new ArrayList<RelOptRule>();
+    final List<RelOptRule> ruleList = new ArrayList<>();
     final PlannerSettings ps = optimizerRulesContext.getPlannerSettings();
 
     ruleList.add(ConvertCountToDirectScan.AGG_ON_PROJ_ON_SCAN);
@@ -414,7 +408,7 @@ public enum PlannerPhase {
     ruleList.add(ProjectPrule.INSTANCE);
     ruleList.add(ScanPrule.INSTANCE);
     ruleList.add(ScreenPrule.INSTANCE);
-    ruleList.add(ExpandConversionRule.INSTANCE);
+    ruleList.add(RuleInstance.EXPAND_CONVERSION_RULE);
     ruleList.add(FilterPrule.INSTANCE);
     ruleList.add(LimitPrule.INSTANCE);
     ruleList.add(WriterPrule.INSTANCE);

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/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
new file mode 100644
index 0000000..a390ff4
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/RuleInstance.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.volcano.AbstractConverter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
+import org.apache.calcite.rel.rules.AggregateRemoveRule;
+import org.apache.calcite.rel.rules.FilterMergeRule;
+import org.apache.calcite.rel.rules.FilterSetOpTransposeRule;
+import org.apache.calcite.rel.rules.JoinPushExpressionsRule;
+import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.rel.rules.ProjectToWindowRule;
+import org.apache.calcite.rel.rules.ProjectWindowTransposeRule;
+import org.apache.calcite.rel.rules.ReduceExpressionsRule;
+import org.apache.calcite.rel.rules.SortRemoveRule;
+import org.apache.calcite.rel.rules.UnionToDistinctRule;
+import org.apache.drill.exec.planner.logical.DrillRelFactories;
+
+/**
+ * Contains rule instances which use custom RelBuilder.
+ */
+public interface RuleInstance {
+
+  ReduceExpressionsRule PROJECT_INSTANCE =
+      new ReduceExpressionsRule.ProjectReduceExpressionsRule(LogicalProject.class,
+          DrillRelFactories.LOGICAL_BUILDER);
+
+  UnionToDistinctRule UNION_TO_DISTINCT_RULE =
+      new UnionToDistinctRule(LogicalUnion.class,
+          DrillRelFactories.LOGICAL_BUILDER);
+
+  JoinPushExpressionsRule JOIN_PUSH_EXPRESSIONS_RULE =
+      new JoinPushExpressionsRule(Join.class,
+          DrillRelFactories.LOGICAL_BUILDER);
+
+  FilterMergeRule FILTER_MERGE_RULE =
+      new FilterMergeRule(DrillRelFactories.LOGICAL_BUILDER);
+
+  AggregateRemoveRule AGGREGATE_REMOVE_RULE =
+      new AggregateRemoveRule(LogicalAggregate.class, DrillRelFactories.LOGICAL_BUILDER);
+
+  AggregateExpandDistinctAggregatesRule AGGREGATE_EXPAND_DISTINCT_AGGREGATES_RULE =
+      new AggregateExpandDistinctAggregatesRule(LogicalAggregate.class, false,
+          DrillRelFactories.LOGICAL_BUILDER);
+
+  /**
+   * Instance of the rule that works on logical joins only, and pushes to the
+   * right.
+   */
+  RelOptRule JOIN_PUSH_THROUGH_JOIN_RULE_RIGHT =
+      new JoinPushThroughJoinRule("JoinPushThroughJoinRule:right", true,
+          LogicalJoin.class, DrillRelFactories.LOGICAL_BUILDER);
+
+  /**
+   * Instance of the rule that works on logical joins only, and pushes to the
+   * left.
+   */
+  RelOptRule JOIN_PUSH_THROUGH_JOIN_RULE_LEFT =
+      new JoinPushThroughJoinRule("JoinPushThroughJoinRule:left", false,
+          LogicalJoin.class, DrillRelFactories.LOGICAL_BUILDER);
+
+  ReduceExpressionsRule CALC_INSTANCE =
+      new ReduceExpressionsRule.CalcReduceExpressionsRule(LogicalCalc.class,
+          DrillRelFactories.LOGICAL_BUILDER);
+
+  FilterSetOpTransposeRule FILTER_SET_OP_TRANSPOSE_RULE =
+      new FilterSetOpTransposeRule(DrillRelFactories.LOGICAL_BUILDER);
+
+  ProjectRemoveRule PROJECT_REMOVE_RULE =
+      new ProjectRemoveRule(DrillRelFactories.LOGICAL_BUILDER);
+
+  ProjectToWindowRule PROJECT_TO_LOGICAL_PROJECT_AND_WINDOW_RULE =
+      new ProjectToWindowRule.ProjectToLogicalProjectAndWindowRule(DrillRelFactories.LOGICAL_BUILDER);
+
+  SortRemoveRule SORT_REMOVE_RULE =
+      new SortRemoveRule(DrillRelFactories.LOGICAL_BUILDER);
+
+  ProjectWindowTransposeRule PROJECT_WINDOW_TRANSPOSE_RULE =
+      new ProjectWindowTransposeRule(DrillRelFactories.LOGICAL_BUILDER);
+
+  AbstractConverter.ExpandConversionRule EXPAND_CONVERSION_RULE =
+      new AbstractConverter.ExpandConversionRule(DrillRelFactories.LOGICAL_BUILDER);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/StarColumnHelper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/StarColumnHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/StarColumnHelper.java
index 9034f01..672af42 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/StarColumnHelper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/StarColumnHelper.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,7 +20,6 @@ package org.apache.drill.exec.planner;
 
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexInputRef;
@@ -30,7 +29,7 @@ public class StarColumnHelper {
 
   public final static String PREFIX_DELIMITER = "\u00a6\u00a6";
 
-  public final static String STAR_COLUMN = "*";
+  public final static String STAR_COLUMN = "**";
 
   public final static String PREFIXED_STAR_COLUMN = PREFIX_DELIMITER + STAR_COLUMN;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/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 b3e261c..91c33bd 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
@@ -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
@@ -24,7 +24,6 @@ import com.google.common.collect.Lists;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.logical.LogicalCalc;
 import org.apache.calcite.rel.rules.ProjectRemoveRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -40,6 +39,7 @@ import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.planner.logical.DrillRelFactories;
 import org.apache.drill.exec.resolver.TypeCastRules;
 
 /**
@@ -121,7 +121,8 @@ public abstract class DrillRelOptUtil {
           }
         };
 
-    return RelOptUtil.createProject(rel, refs, fieldNames, false);
+    return RelOptUtil.createProject(rel, refs, fieldNames, false,
+        DrillRelFactories.LOGICAL_BUILDER.create(rel.getCluster(), null));
   }
 
   public static boolean isTrivialProject(Project project, boolean useNamesInIdentityProjCalc) {

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java
index cbb5aee..70f2215 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java
@@ -36,7 +36,11 @@ public class DrillAggregateRule extends RelOptRule {
   protected static final Logger tracer = CalciteTrace.getPlannerTracer();
 
   private DrillAggregateRule() {
-    super(RelOptHelper.some(LogicalAggregate.class, Convention.NONE, RelOptHelper.any(RelNode.class)), "DrillAggregateRule");
+    super(
+        RelOptHelper.some(LogicalAggregate.class,
+            Convention.NONE, RelOptHelper.any(RelNode.class)),
+        DrillRelFactories.LOGICAL_BUILDER,
+        "DrillAggregateRule");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/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 cdba8b8..7b52eda 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
@@ -68,7 +68,6 @@ import org.apache.drill.exec.expr.holders.TimeStampHolder;
 import org.apache.drill.exec.expr.holders.ValueHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
 import org.apache.drill.exec.ops.UdfUtilities;
-import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
@@ -316,9 +315,9 @@ public class DrillConstExecutor implements RexExecutor {
                 milliseconds = intervalDayOut.milliseconds;
               }
               return rexBuilder.makeLiteral(
-                new BigDecimal(days * DateUtility.daysToStandardMillis + milliseconds),
-                TypeInferenceUtils.createCalciteTypeWithNullability(typeFactory, SqlTypeName.INTERVAL_DAY, newCall.getType().isNullable()),
-                false);
+                  new BigDecimal(days * (long) DateUtility.daysToStandardMillis + milliseconds),
+                  TypeInferenceUtils.createCalciteTypeWithNullability(typeFactory, SqlTypeName.INTERVAL_DAY,
+                      newCall.getType().isNullable()), false);
             }
             // The list of known unsupported types is used to trigger this behavior of re-using the input expression
             // before the expression is even attempted to be evaluated, this is just here as a last precaution a

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterAggregateTransposeRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterAggregateTransposeRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterAggregateTransposeRule.java
index 9d5e444..8a6c0aa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterAggregateTransposeRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterAggregateTransposeRule.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
@@ -25,7 +25,7 @@ import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.rules.FilterAggregateTransposeRule;
-import org.apache.calcite.tools.RelBuilder;
+import org.apache.drill.exec.planner.DrillRelBuilder;
 
 public class DrillFilterAggregateTransposeRule extends FilterAggregateTransposeRule{
 
@@ -35,14 +35,16 @@ public class DrillFilterAggregateTransposeRule extends FilterAggregateTransposeR
   public static final FilterAggregateTransposeRule INSTANCE = new DrillFilterAggregateTransposeRule();
 
   private DrillFilterAggregateTransposeRule() {
-    super(Filter.class, RelBuilder.proto(Contexts.of(RelFactories.DEFAULT_FILTER_FACTORY)), Aggregate.class);
+    super(Filter.class, DrillRelBuilder.proto(Contexts.of(RelFactories.DEFAULT_FILTER_FACTORY)),
+        Aggregate.class);
   }
 
   @Override
   public boolean matches(RelOptRuleCall call) {
-    final Filter filter = (Filter) call.rel(0);
-    final Aggregate aggregate = (Aggregate) call.rel(1);
-    return filter.getTraitSet().getTrait(ConventionTraitDef.INSTANCE) == aggregate.getTraitSet().getTrait(ConventionTraitDef.INSTANCE);
+    final Filter filter = call.rel(0);
+    final Aggregate aggregate = call.rel(1);
+    return filter.getTraitSet().getTrait(ConventionTraitDef.INSTANCE)
+        == aggregate.getTraitSet().getTrait(ConventionTraitDef.INSTANCE);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterJoinRules.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterJoinRules.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterJoinRules.java
index 0c1fdb3..6d186e1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterJoinRules.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterJoinRules.java
@@ -1,20 +1,4 @@
-package org.apache.drill.exec.planner.logical;
-
-import com.google.common.collect.Lists;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.rel.core.RelFactories;
-import org.apache.calcite.rel.rules.FilterJoinRule;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.SqlKind;
-
-import java.util.List;
-
-/**
+/*
  * 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
@@ -31,6 +15,19 @@ import java.util.List;
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.drill.exec.planner.logical;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.rules.FilterJoinRule;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.exec.planner.DrillRelBuilder;
+
+import java.util.List;
 
 public class DrillFilterJoinRules {
   /** Predicate that always returns true for any filter in OUTER join, and only true
@@ -55,26 +52,27 @@ public class DrillFilterJoinRules {
           List<RelDataTypeField> sysFields = Lists.newArrayList();
           List<Integer> filterNulls = Lists.newArrayList();
 
-          RexNode remaining = RelOptUtil.splitJoinCondition(sysFields, join.getLeft(), join.getRight(), exp, tmpLeftKeys, tmpRightKeys, filterNulls, null);
-
-          if (remaining.isAlwaysTrue()) {
-            return true;
-          }
+          RexNode remaining = RelOptUtil.splitJoinCondition(sysFields, join.getLeft(), join.getRight(),
+              exp, tmpLeftKeys, tmpRightKeys, filterNulls, null);
 
-          return false;
+          return remaining.isAlwaysTrue();
         }
       };
 
 
   /** Rule that pushes predicates from a Filter into the Join below them. */
   public static final FilterJoinRule DRILL_FILTER_ON_JOIN =
-      new FilterJoinRule.FilterIntoJoinRule(true, RelFactories.DEFAULT_FILTER_FACTORY,
-          RelFactories.DEFAULT_PROJECT_FACTORY, EQUAL_IS_DISTINCT_FROM);
+      new FilterJoinRule.FilterIntoJoinRule(true,
+          DrillRelBuilder.proto(RelFactories.DEFAULT_FILTER_FACTORY,
+              RelFactories.DEFAULT_PROJECT_FACTORY),
+          EQUAL_IS_DISTINCT_FROM);
 
 
   /** Rule that pushes predicates in a Join into the inputs to the join. */
   public static final FilterJoinRule DRILL_JOIN =
-      new FilterJoinRule.JoinConditionPushRule(RelFactories.DEFAULT_FILTER_FACTORY,
-          RelFactories.DEFAULT_PROJECT_FACTORY, EQUAL_IS_DISTINCT_FROM);
+      new FilterJoinRule.JoinConditionPushRule(
+          DrillRelBuilder.proto(RelFactories.DEFAULT_FILTER_FACTORY,
+              RelFactories.DEFAULT_PROJECT_FACTORY),
+          EQUAL_IS_DISTINCT_FROM);
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRule.java
index dd7da48..f3769f0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRule.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,26 +21,26 @@ import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.logical.LogicalFilter;
 
 /**
- * Rule that converts a {@link org.apache.calcite.rel.FilterRel} to a Drill "filter" operation.
+ * Rule that converts a {@link org.apache.calcite.rel.logical.LogicalFilter} to a Drill "filter" operation.
  */
 public class DrillFilterRule extends RelOptRule {
   public static final RelOptRule INSTANCE = new DrillFilterRule();
 
   private DrillFilterRule() {
-    super(RelOptHelper.any(LogicalFilter.class, Convention.NONE), "DrillFilterRule");
+    super(RelOptHelper.any(LogicalFilter.class, Convention.NONE),
+        DrillRelFactories.LOGICAL_BUILDER, "DrillFilterRule");
   }
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final LogicalFilter filter = (LogicalFilter) call.rel(0);
+    final LogicalFilter filter = call.rel(0);
     final RelNode input = filter.getInput();
-    //final RelTraitSet traits = filter.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
     final RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL));
-    call.transformTo(new DrillFilterRel(filter.getCluster(), convertedInput.getTraitSet(), convertedInput, filter.getCondition()));
+    call.transformTo(new DrillFilterRel(
+        filter.getCluster(), convertedInput.getTraitSet(),
+        convertedInput, filter.getCondition()));
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
index deb8ace..766509e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
@@ -46,8 +46,8 @@ public class DrillJoinRule extends RelOptRule {
   protected static final Logger tracer = CalciteTrace.getPlannerTracer();
 
   private DrillJoinRule() {
-    super(
-        RelOptHelper.any(LogicalJoin.class, Convention.NONE),
+    super(RelOptHelper.any(LogicalJoin.class, Convention.NONE),
+        DrillRelFactories.LOGICAL_BUILDER,
         "DrillJoinRule");
   }
 
@@ -64,7 +64,6 @@ public class DrillJoinRule extends RelOptRule {
     List<Integer> leftKeys = Lists.newArrayList();
     List<Integer> rightKeys = Lists.newArrayList();
     List<Boolean> filterNulls = Lists.newArrayList();
-    int numLeftFields = convertedLeft.getRowType().getFieldCount();
 
     boolean addFilter = false;
     RexNode origJoinCondition = join.getCondition();

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java
index 03ef1e8..6d2da9d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.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
@@ -32,7 +32,8 @@ public class DrillLimitRule extends RelOptRule {
   public static DrillLimitRule INSTANCE = new DrillLimitRule();
 
   private DrillLimitRule() {
-    super(RelOptHelper.any(LogicalSort.class, Convention.NONE), "DrillLimitRule");
+    super(RelOptHelper.any(LogicalSort.class, Convention.NONE),
+        DrillRelFactories.LOGICAL_BUILDER, "DrillLimitRule");
   }
 
   @Override
@@ -54,7 +55,9 @@ public class DrillLimitRule extends RelOptRule {
     }
 
     RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL));
-    call.transformTo(new DrillLimitRel(incomingSort.getCluster(), convertedInput.getTraitSet().plus(DrillRel.DRILL_LOGICAL), convertedInput, incomingSort.offset, incomingSort.fetch));
+    call.transformTo(new DrillLimitRel(
+        incomingSort.getCluster(), convertedInput.getTraitSet().plus(DrillRel.DRILL_LOGICAL),
+        convertedInput, incomingSort.offset, incomingSort.fetch));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillMergeProjectRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillMergeProjectRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillMergeProjectRule.java
index 6c69e28..94964ef 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillMergeProjectRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillMergeProjectRule.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,22 +27,25 @@ import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.calcite.rel.core.RelFactories.ProjectFactory;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexNode;
+import org.apache.drill.exec.planner.DrillRelBuilder;
 
 public class DrillMergeProjectRule extends ProjectMergeRule {
 
   private FunctionImplementationRegistry functionRegistry;
   private static DrillMergeProjectRule INSTANCE = null;
 
-  public static DrillMergeProjectRule getInstance(boolean force, ProjectFactory pFactory, FunctionImplementationRegistry functionRegistry) {
+  public static DrillMergeProjectRule getInstance(boolean force, ProjectFactory pFactory,
+      FunctionImplementationRegistry functionRegistry) {
     if (INSTANCE == null) {
       INSTANCE = new DrillMergeProjectRule(force, pFactory, functionRegistry);
     }
     return INSTANCE;
   }
 
-  private DrillMergeProjectRule(boolean force, ProjectFactory pFactory, FunctionImplementationRegistry functionRegistry) {
-    super(force, pFactory);
-   this.functionRegistry = functionRegistry;
+  private DrillMergeProjectRule(boolean force, ProjectFactory pFactory,
+      FunctionImplementationRegistry functionRegistry) {
+    super(force, DrillRelBuilder.proto(pFactory));
+    this.functionRegistry = functionRegistry;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRule.java
index e44df33..14c6533 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRule.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
@@ -32,16 +32,17 @@ public class DrillProjectRule extends RelOptRule {
   public static final RelOptRule INSTANCE = new DrillProjectRule();
 
   private DrillProjectRule() {
-    super(RelOptHelper.any(LogicalProject.class, Convention.NONE), "DrillProjectRule");
+    super(RelOptHelper.any(LogicalProject.class, Convention.NONE),
+        DrillRelFactories.LOGICAL_BUILDER, "DrillProjectRule");
   }
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final Project project = (Project) call.rel(0);
+    final Project project = call.rel(0);
     final RelNode input = project.getInput();
     final RelTraitSet traits = project.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
     final RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL));
-    call.transformTo(new DrillProjectRel(project.getCluster(), traits, convertedInput, project.getProjects(), project
-        .getRowType()));
+    call.transformTo(new DrillProjectRel(
+        project.getCluster(), traits, convertedInput, project.getProjects(), project.getRowType()));
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectSetOpTransposeRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectSetOpTransposeRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectSetOpTransposeRule.java
index baff3b7..86a39dd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectSetOpTransposeRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectSetOpTransposeRule.java
@@ -19,7 +19,6 @@
 package org.apache.drill.exec.planner.logical;
 
 import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.rules.ProjectSetOpTransposeRule;
 import org.apache.calcite.rel.rules.PushProjector;
 
@@ -27,6 +26,6 @@ public class DrillProjectSetOpTransposeRule extends ProjectSetOpTransposeRule {
   public final static RelOptRule INSTANCE = new DrillProjectSetOpTransposeRule(DrillConditions.PRESERVE_ITEM);
 
   protected DrillProjectSetOpTransposeRule(PushProjector.ExprCondition preserveExprCondition) {
-    super(preserveExprCondition, RelFactories.LOGICAL_BUILDER);
+    super(preserveExprCondition, DrillRelFactories.LOGICAL_BUILDER);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/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 c2dbfb9..d24abcc 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
@@ -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,6 +27,7 @@ import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.util.Pair;
 import org.apache.drill.exec.planner.common.DrillRelOptUtil;
 
 import java.util.List;
@@ -39,7 +40,8 @@ public class DrillPushFilterPastProjectRule extends RelOptRule {
     super(
         operand(
             LogicalFilter.class,
-            operand(LogicalProject.class, any())));
+            operand(LogicalProject.class, any())),
+        DrillRelFactories.LOGICAL_BUILDER, null);
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -73,15 +75,17 @@ public class DrillPushFilterPastProjectRule extends RelOptRule {
 
     // convert the filter to one that references the child of the project
     RexNode newCondition =
-        RelOptUtil.pushFilterPastProject(qualifedPred, projRel);
+        RelOptUtil.pushPastProject(qualifedPred, projRel);
 
     Filter newFilterRel = LogicalFilter.create(projRel.getInput(), newCondition);
 
     Project newProjRel =
         (Project) RelOptUtil.createProject(
             newFilterRel,
-            projRel.getNamedProjects(),
-            false);
+            Pair.left(projRel.getNamedProjects()),
+            Pair.right(projRel.getNamedProjects()),
+            false,
+            relBuilderFactory.create(projRel.getCluster(), null));
 
     final RexNode unqualifiedPred = RexUtil.composeConjunction(filterRel.getCluster().getRexBuilder(), unqualifiedPredList, true);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/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 8ce26c8..9c06897 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
@@ -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,23 +23,18 @@ 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.util.Pair;
 import org.apache.drill.exec.physical.base.GroupScan;
-import org.apache.drill.exec.planner.logical.partition.PruneScanRule;
-import org.apache.drill.exec.store.parquet.ParquetGroupScan;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
 
 public abstract class DrillPushLimitToScanRule extends RelOptRule {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillPushLimitToScanRule.class);
 
   private DrillPushLimitToScanRule(RelOptRuleOperand operand, String description) {
-    super(operand, description);
+    super(operand, DrillRelFactories.LOGICAL_BUILDER, description);
   }
 
-  public static DrillPushLimitToScanRule LIMIT_ON_SCAN = new DrillPushLimitToScanRule(
-      RelOptHelper.some(DrillLimitRel.class, RelOptHelper.any(DrillScanRel.class)), "DrillPushLimitToScanRule_LimitOnScan") {
+  public static DrillPushLimitToScanRule LIMIT_ON_SCAN =
+      new DrillPushLimitToScanRule(RelOptHelper.some(DrillLimitRel.class, RelOptHelper.any(DrillScanRel.class)),
+          "DrillPushLimitToScanRule_LimitOnScan") {
     @Override
     public boolean matches(RelOptRuleCall call) {
       DrillLimitRel limitRel = call.rel(0);
@@ -60,8 +55,11 @@ public abstract class DrillPushLimitToScanRule extends RelOptRule {
     }
   };
 
-  public static DrillPushLimitToScanRule LIMIT_ON_PROJECT = new DrillPushLimitToScanRule(
-      RelOptHelper.some(DrillLimitRel.class, RelOptHelper.some(DrillProjectRel.class, RelOptHelper.any(DrillScanRel.class))), "DrillPushLimitToScanRule_LimitOnProject") {
+  public static DrillPushLimitToScanRule LIMIT_ON_PROJECT =
+      new DrillPushLimitToScanRule(
+          RelOptHelper.some(DrillLimitRel.class, RelOptHelper.some(
+              DrillProjectRel.class, RelOptHelper.any(DrillScanRel.class))),
+          "DrillPushLimitToScanRule_LimitOnProject") {
     @Override
     public boolean matches(RelOptRuleCall call) {
       DrillLimitRel limitRel = call.rel(0);

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjIntoScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjIntoScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjIntoScan.java
index b15a843..f6bc6b7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjIntoScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjIntoScan.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
@@ -38,17 +38,16 @@ import com.google.common.collect.Lists;
 public class DrillPushProjIntoScan extends RelOptRule {
   public static final RelOptRule INSTANCE = new DrillPushProjIntoScan(LogicalProject.class, EnumerableTableScan.class);
 
-  public static final RelOptRule DRILL_LOGICAL_INSTANCE = new DrillPushProjIntoScan(DrillProjectRel.class, DrillScanRel.class);
-
-  private DrillPushProjIntoScan(Class<? extends Project> projectClass, Class<? extends TableScan> scanClass) {
-    super(RelOptHelper.some(projectClass, RelOptHelper.any(scanClass)), "DrillPushProjIntoScan");
+  private DrillPushProjIntoScan(Class<? extends Project> projectClass,
+      Class<? extends TableScan> scanClass) {
+    super(RelOptHelper.some(projectClass, RelOptHelper.any(scanClass)),
+        DrillRelFactories.LOGICAL_BUILDER, "DrillPushProjIntoScan");
   }
 
-
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final Project proj = (Project) call.rel(0);
-    final TableScan scan = (TableScan) call.rel(1);
+    final Project proj = call.rel(0);
+    final TableScan scan = call.rel(1);
 
     try {
       ProjectPushInfo columnInfo = PrelUtil.getColumns(scan.getRowType(), proj.getProjects());

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastFilterRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastFilterRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastFilterRule.java
index e5c40b9..6ea2c92 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastFilterRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastFilterRule.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.planner.logical;
 
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.rules.ProjectFilterTransposeRule;
@@ -29,7 +28,7 @@ public class DrillPushProjectPastFilterRule extends ProjectFilterTransposeRule {
   public final static RelOptRule INSTANCE = new DrillPushProjectPastFilterRule(DrillConditions.PRESERVE_ITEM);
 
   protected DrillPushProjectPastFilterRule(PushProjector.ExprCondition preserveExprCondition) {
-    super(LogicalProject.class, LogicalFilter.class, RelFactories.LOGICAL_BUILDER, preserveExprCondition);
+    super(LogicalProject.class, LogicalFilter.class,
+        DrillRelFactories.LOGICAL_BUILDER, preserveExprCondition);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastJoinRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastJoinRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastJoinRule.java
index 269f6d5..972a141 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastJoinRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillPushProjectPastJoinRule.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.planner.logical;
 
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.rules.ProjectJoinTransposeRule;
 import org.apache.calcite.rel.rules.PushProjector;
 import org.apache.calcite.plan.RelOptRule;
@@ -27,7 +26,7 @@ public class DrillPushProjectPastJoinRule extends ProjectJoinTransposeRule {
   public static final RelOptRule INSTANCE = new DrillPushProjectPastJoinRule(DrillConditions.PRESERVE_ITEM);
 
   protected DrillPushProjectPastJoinRule(PushProjector.ExprCondition preserveExprCondition) {
-    super(preserveExprCondition, RelFactories.LOGICAL_BUILDER);
+    super(preserveExprCondition, DrillRelFactories.LOGICAL_BUILDER);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/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 bb972d0..21ff9a9 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
@@ -99,7 +99,7 @@ public class DrillReduceAggregatesRule extends RelOptRule {
   //~ Constructors -----------------------------------------------------------
 
   protected DrillReduceAggregatesRule(RelOptRuleOperand operand) {
-    super(operand);
+    super(operand, DrillRelFactories.LOGICAL_BUILDER, null);
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -204,7 +204,9 @@ public class DrillReduceAggregatesRule extends RelOptRule {
                   input.getRowType().getFieldNames(),
                   Collections.<String>nCopies(
                       extraArgCount,
-                      null)));
+                      null)),
+              false,
+              relBuilderFactory.create(input.getCluster(), null));
     }
     Aggregate newAggRel =
         newAggregateRel(
@@ -661,7 +663,7 @@ public class DrillReduceAggregatesRule extends RelOptRule {
     protected static final Logger tracer = CalciteTrace.getPlannerTracer();
 
     public DrillConvertSumToSumZero(RelOptRuleOperand operand) {
-      super(operand);
+      super(operand, DrillRelFactories.LOGICAL_BUILDER, null);
     }
 
     @Override
@@ -725,7 +727,7 @@ public class DrillReduceAggregatesRule extends RelOptRule {
 
   private static class DrillConvertWindowSumToSumZero extends RelOptRule {
     public DrillConvertWindowSumToSumZero(RelOptRuleOperand operand) {
-      super(operand);
+      super(operand, DrillRelFactories.LOGICAL_BUILDER, null);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/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 96bcf02..6214912 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
@@ -20,12 +20,9 @@ package org.apache.drill.exec.planner.logical;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.core.Calc;
-import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.SingleRel;
-import org.apache.calcite.rel.logical.LogicalCalc;
-import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalSort;
 import org.apache.calcite.rel.rules.ReduceExpressionsRule;
 
@@ -42,7 +39,7 @@ public class DrillReduceExpressionsRule {
   private static class DrillReduceFilterRule extends ReduceExpressionsRule.FilterReduceExpressionsRule {
 
     DrillReduceFilterRule() {
-      super(DrillFilterRel.class, RelFactories.LOGICAL_BUILDER);
+      super(Filter.class, DrillRelFactories.LOGICAL_BUILDER);
     }
 
     /**
@@ -61,7 +58,7 @@ public class DrillReduceExpressionsRule {
   private static class DrillReduceCalcRule extends ReduceExpressionsRule.CalcReduceExpressionsRule {
 
     DrillReduceCalcRule() {
-      super(Calc.class, RelFactories.LOGICAL_BUILDER);
+      super(Calc.class, DrillRelFactories.LOGICAL_BUILDER);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRelFactories.java
----------------------------------------------------------------------
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 487da4d..ce35612 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
@@ -18,6 +18,7 @@
 
 package org.apache.drill.exec.planner.logical;
 
+import org.apache.calcite.plan.Contexts;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.CorrelationId;
@@ -27,10 +28,23 @@ import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.drill.exec.planner.DrillRelBuilder;
 
 import java.util.List;
 import java.util.Set;
 
+import static org.apache.calcite.rel.core.RelFactories.DEFAULT_AGGREGATE_FACTORY;
+import static org.apache.calcite.rel.core.RelFactories.DEFAULT_FILTER_FACTORY;
+import static org.apache.calcite.rel.core.RelFactories.DEFAULT_JOIN_FACTORY;
+import static org.apache.calcite.rel.core.RelFactories.DEFAULT_MATCH_FACTORY;
+import static org.apache.calcite.rel.core.RelFactories.DEFAULT_PROJECT_FACTORY;
+import static org.apache.calcite.rel.core.RelFactories.DEFAULT_SEMI_JOIN_FACTORY;
+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;
+
 /**
  * Contains factory implementation for creating various Drill Logical Rel nodes.
  */
@@ -45,6 +59,22 @@ public class DrillRelFactories {
 
   public static final RelFactories.JoinFactory DRILL_LOGICAL_JOIN_FACTORY = new DrillJoinFactoryImpl();
 
+  /**
+   * A {@link RelBuilderFactory} that creates a {@link DrillRelBuilder} that will
+   * create logical relational expressions for everything.
+   */
+  public static final RelBuilderFactory LOGICAL_BUILDER =
+      DrillRelBuilder.proto(
+          Contexts.of(DEFAULT_PROJECT_FACTORY,
+              DEFAULT_FILTER_FACTORY,
+              DEFAULT_JOIN_FACTORY,
+              DEFAULT_SEMI_JOIN_FACTORY,
+              DEFAULT_SORT_FACTORY,
+              DEFAULT_AGGREGATE_FACTORY,
+              DEFAULT_MATCH_FACTORY,
+              DEFAULT_SET_OP_FACTORY,
+              DEFAULT_VALUES_FACTORY,
+              DEFAULT_TABLE_SCAN_FACTORY));
 
   /**
    * Implementation of {@link RelFactories.ProjectFactory} that returns a vanilla
@@ -55,14 +85,13 @@ public class DrillRelFactories {
     public RelNode createProject(RelNode child,
                                  List<? extends RexNode> childExprs, List<String> fieldNames) {
       final RelOptCluster cluster = child.getCluster();
-      final RelDataType rowType = RexUtil.createStructType(cluster.getTypeFactory(), childExprs, fieldNames);
-      final RelNode project = DrillProjectRel.create(cluster, child.getTraitSet(), child, childExprs, rowType);
+      final RelDataType rowType =
+          RexUtil.createStructType(cluster.getTypeFactory(), childExprs, fieldNames, null);
 
-      return project;
+      return DrillProjectRel.create(cluster, child.getTraitSet(), child, childExprs, rowType);
     }
   }
 
-
   /**
    * Implementation of {@link RelFactories.FilterFactory} that
    * returns a vanilla {@link LogicalFilter}.
@@ -74,7 +103,6 @@ public class DrillRelFactories {
     }
   }
 
-
   /**
    * Implementation of {@link RelFactories.JoinFactory} that returns a vanilla
    * {@link org.apache.calcite.rel.logical.LogicalJoin}.

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java
index 9a63cb2..722ea85 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.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,12 +27,13 @@ public class DrillScanRule  extends RelOptRule {
   public static final RelOptRule INSTANCE = new DrillScanRule();
 
   private DrillScanRule() {
-    super(RelOptHelper.any(EnumerableTableScan.class), "DrillTableRule");
+    super(RelOptHelper.any(EnumerableTableScan.class),
+        DrillRelFactories.LOGICAL_BUILDER, "DrillTableRule");
   }
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final EnumerableTableScan access = (EnumerableTableScan) call.rel(0);
+    final EnumerableTableScan access = call.rel(0);
     final RelTraitSet traits = access.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
     call.transformTo(new DrillScanRel(access.getCluster(), traits, access.getTable()));
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRule.java
index 5ae2b39..1ab061b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRule.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
@@ -31,7 +31,8 @@ public class DrillSortRule extends RelOptRule {
   public static final RelOptRule INSTANCE = new DrillSortRule();
 
   private DrillSortRule() {
-    super(RelOptHelper.any(Sort.class, Convention.NONE), "DrillSortRule");
+    super(RelOptHelper.any(Sort.class, Convention.NONE),
+        DrillRelFactories.LOGICAL_BUILDER, "DrillSortRule");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionAllRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionAllRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionAllRule.java
index a074952..91697a2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionAllRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionAllRule.java
@@ -39,7 +39,8 @@ public class DrillUnionAllRule extends RelOptRule {
   protected static final Logger tracer = CalciteTrace.getPlannerTracer();
 
   private DrillUnionAllRule() {
-    super(RelOptHelper.any(LogicalUnion.class, Convention.NONE), "DrillUnionRule");
+    super(RelOptHelper.any(LogicalUnion.class, Convention.NONE),
+        DrillRelFactories.LOGICAL_BUILDER, "DrillUnionRule");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRule.java
index 066e57b..692afbb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRule.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,12 +30,13 @@ public class DrillValuesRule extends RelOptRule {
   public static final RelOptRule INSTANCE = new DrillValuesRule();
 
   private DrillValuesRule() {
-    super(RelOptHelper.any(LogicalValues.class, Convention.NONE), "DrillValuesRule");
+    super(RelOptHelper.any(LogicalValues.class, Convention.NONE),
+        DrillRelFactories.LOGICAL_BUILDER, "DrillValuesRule");
   }
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final LogicalValues values = (LogicalValues) call.rel(0);
+    final LogicalValues values = call.rel(0);
     final RelTraitSet traits = values.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
     call.transformTo(new DrillValuesRel(values.getCluster(), values.getRowType(), values.getTuples(), traits));
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWindowRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWindowRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWindowRule.java
index 1cefcd9..4461796 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWindowRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWindowRule.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
@@ -29,7 +29,8 @@ public class DrillWindowRule extends RelOptRule {
   public static final RelOptRule INSTANCE = new DrillWindowRule();
 
   private DrillWindowRule() {
-    super(RelOptHelper.some(Window.class, Convention.NONE, RelOptHelper.any(RelNode.class)), "DrillWindowRule");
+    super(RelOptHelper.some(Window.class, Convention.NONE, RelOptHelper.any(RelNode.class)),
+        DrillRelFactories.LOGICAL_BUILDER, "DrillWindowRule");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prule.java
index e0a2839..148e74b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prule.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
@@ -22,18 +22,18 @@ import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelTraitSet;
+import org.apache.drill.exec.planner.logical.DrillRelFactories;
 
-public abstract class Prule extends RelOptRule{
+public abstract class Prule extends RelOptRule {
   public Prule(RelOptRuleOperand operand, String description) {
-    super(operand, description);
+    super(operand, DrillRelFactories.LOGICAL_BUILDER, description);
   }
 
   public Prule(RelOptRuleOperand operand) {
-    super(operand);
+    this(operand, null);
   }
 
-
-  public static RelNode convert(RelNode rel, RelTraitSet toTraits){
+  public static RelNode convert(RelNode rel, RelTraitSet toTraits) {
     toTraits = toTraits.simplify();
 
     PlannerSettings settings = PrelUtil.getSettings(rel.getCluster());
@@ -44,8 +44,7 @@ public abstract class Prule extends RelOptRule{
     return RelOptRule.convert(rel, toTraits);
   }
 
-  public static boolean isSingleMode(RelOptRuleCall call){
+  public static boolean isSingleMode(RelOptRuleCall call) {
     return PrelUtil.getPlannerSettings(call.getPlanner()).isSingleMode();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrule.java
index d1f2e28..1251ddf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ValuesPrule.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.planner.physical;
 
 import org.apache.calcite.plan.RelOptRule;
+import org.apache.drill.exec.planner.logical.DrillRelFactories;
 import org.apache.drill.exec.planner.logical.DrillValuesRel;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
 import org.apache.calcite.plan.RelOptRuleCall;
@@ -26,14 +27,15 @@ public class ValuesPrule extends RelOptRule {
 
   public static final ValuesPrule INSTANCE = new ValuesPrule();
 
-  private ValuesPrule(){
-    super(RelOptHelper.any(DrillValuesRel.class), "Prel.ValuesPrule");
+  private ValuesPrule() {
+    super(RelOptHelper.any(DrillValuesRel.class), DrillRelFactories.LOGICAL_BUILDER, "Prel.ValuesPrule");
   }
 
   @Override
   public void onMatch(final RelOptRuleCall call) {
-    final DrillValuesRel rel = (DrillValuesRel) call.rel(0);
-    call.transformTo(new ValuesPrel(rel.getCluster(), rel.getRowType(), rel.getTuples(), rel.getTraitSet().plus(Prel.DRILL_PHYSICAL), rel.getContent()));
+    final DrillValuesRel rel = call.rel(0);
+    call.transformTo(new ValuesPrel(rel.getCluster(), rel.getRowType(), rel.getTuples(),
+        rel.getTraitSet().plus(Prel.DRILL_PHYSICAL), rel.getContent()));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/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 72d2699..69458d4 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
@@ -32,6 +32,7 @@ import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.planner.StarColumnHelper;
 import org.apache.drill.exec.planner.common.DrillRelOptUtil;
+import org.apache.drill.exec.planner.logical.DrillRelFactories;
 import org.apache.drill.exec.store.AbstractSchema;
 
 import org.apache.calcite.tools.ValidationException;
@@ -203,7 +204,8 @@ public class SqlHandlerUtil {
             }
           };
 
-      return RelOptUtil.createProject(input, refs, names, false);
+      return RelOptUtil.createProject(input, refs, names, false,
+          DrillRelFactories.LOGICAL_BUILDER.create(input.getCluster(), null));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginOptimizerRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginOptimizerRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginOptimizerRule.java
index 8b4f0e6..f81c783 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginOptimizerRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginOptimizerRule.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,11 +19,12 @@ package org.apache.drill.exec.store;
 
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.drill.exec.planner.logical.DrillRelFactories;
 
 public abstract class StoragePluginOptimizerRule extends RelOptRule {
 
   public StoragePluginOptimizerRule(RelOptRuleOperand operand, String description) {
-    super(operand, description);
+    super(operand, DrillRelFactories.LOGICAL_BUILDER, description);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestCorruptParquetDateCorrection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestCorruptParquetDateCorrection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestCorruptParquetDateCorrection.java
index 1013511..507e102 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestCorruptParquetDateCorrection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestCorruptParquetDateCorrection.java
@@ -23,9 +23,11 @@ import org.apache.drill.categories.ParquetTest;
 import org.apache.drill.PlanTestBase;
 import org.apache.drill.test.TestBuilder;
 import org.apache.drill.categories.UnlikelyTest;
+import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.store.parquet.Metadata;
 import org.joda.time.DateTime;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -188,34 +190,26 @@ public class TestCorruptParquetDateCorrection extends PlanTestBase {
         .go();
   }
 
-  @Test
-  public void testReadPartitionedOnCorruptedDates_UserDisabledCorrection() throws Exception {
+  // according to SQL spec. '4.4.3.5 Datetime types' year should be less than 9999
+  @Test(expected = UserRemoteException.class)
+  public void testQueryWithCorruptedDates() throws Exception {
     try {
-      for (String selection : new String[]{"*", "date_col"}) {
-        for (Path table : new Path[]{CORRUPTED_PARTITIONED_DATES_1_2_PATH, CORRUPTED_PARTITIONED_DATES_1_4_0_PATH}) {
-          // for sanity, try reading all partitions without a filter
-          TestBuilder builder = testBuilder()
-              .sqlQuery("select %s from table(dfs.`%s` (type => 'parquet', autoCorrectCorruptDates => false))",
-                  selection, table)
-              .unOrdered()
-              .baselineColumns("date_col");
-          addCorruptedDateBaselineValues(builder);
-          builder.go();
+      TestBuilder builder = testBuilder()
+          .sqlQuery("select * from table(dfs.`%s` (type => 'parquet', autoCorrectCorruptDates => false))",
+              CORRUPTED_PARTITIONED_DATES_1_2_PATH)
+          .unOrdered()
+          .baselineColumns("date_col");
+      addCorruptedDateBaselineValues(builder);
+      builder.go();
 
-          String query = format("select %s from table(dfs.`%s` (type => 'parquet', " +
-              "autoCorrectCorruptDates => false)) where date_col = cast('15334-03-17' as date)", selection, table);
-          // verify that pruning is actually taking place
-          testPlanMatchingPatterns(query, new String[]{"numFiles=1"}, null);
+      String query = "select * from table(dfs.`%s` (type => 'parquet', " +
+          "autoCorrectCorruptDates => false)) where date_col = cast('15334-03-17' as date)";
 
-          // read with a filter on the partition column
-          testBuilder()
-              .sqlQuery(query)
-              .unOrdered()
-              .baselineColumns("date_col")
-              .baselineValues(new DateTime(15334, 3, 17, 0, 0))
-              .go();
-        }
-      }
+      test(query, CORRUPTED_PARTITIONED_DATES_1_2_PATH);
+    } catch (UserRemoteException e) {
+      Assert.assertTrue(e.getMessage()
+          .contains("Year out of range"));
+      throw e;
     } finally {
       test("alter session reset all");
     }
@@ -347,32 +341,6 @@ public class TestCorruptParquetDateCorrection extends PlanTestBase {
   }
 
   @Test
-  public void testReadOldMetadataCacheFileOverrideCorrection() throws Exception {
-    // for sanity, try reading all partitions without a filter
-    TestBuilder builder = testBuilder()
-        .sqlQuery("select date_col from table(dfs.`%s` (type => 'parquet', autoCorrectCorruptDates => false))",
-          PARTITIONED_1_2_FOLDER)
-        .unOrdered()
-        .baselineColumns("date_col");
-    addCorruptedDateBaselineValues(builder);
-    builder.go();
-
-    String query = format("select date_col from table(dfs.`%s` (type => 'parquet', " +
-        "autoCorrectCorruptDates => false)) where date_col = cast('15334-03-17' as date)",
-      PARTITIONED_1_2_FOLDER);
-    // verify that pruning is actually taking place
-    testPlanMatchingPatterns(query, new String[]{"numFiles=1", "usedMetadataFile=true"}, null);
-
-    // read with a filter on the partition column
-    testBuilder()
-        .sqlQuery(query)
-        .unOrdered()
-        .baselineColumns("date_col")
-        .baselineValues(new DateTime(15334, 3, 17, 0, 0))
-        .go();
-  }
-
-  @Test
   public void testReadNewMetadataCacheFileOverOldAndNewFiles() throws Exception {
     File meta = dirTestWatcher.copyResourceToRoot(
        PARQUET_4203.resolve("mixed_version_partitioned_metadata.requires_replace.txt"),

http://git-wip-us.apache.org/repos/asf/drill/blob/0a525aab/exec/java-exec/src/test/resources/record/test_recorditerator.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/record/test_recorditerator.json b/exec/java-exec/src/test/resources/record/test_recorditerator.json
index 057466b..c822803 100644
--- a/exec/java-exec/src/test/resources/record/test_recorditerator.json
+++ b/exec/java-exec/src/test/resources/record/test_recorditerator.json
@@ -60,7 +60,7 @@
         @id:2,
         child:1,
         pop:"project",
-        exprs:[ { ref : "`*`", expr : "`*`"} ]
+        exprs:[ { ref : "`**`", expr : "`**`"} ]
       },
       {
         @id:3,


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

Posted by am...@apache.org.
DRILL-3993: Fix unit test failures connected with support Calcite 1.13

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

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

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

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

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

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


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

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

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


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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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


[14/18] drill git commit: DRILL-3993: Resolve conflicts

Posted by am...@apache.org.
DRILL-3993: Resolve conflicts


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

Branch: refs/heads/master
Commit: 3f0e517fb62a1ebad92fb473e787d343152920d6
Parents: 90b5e63
Author: Volodymyr Vysotskyi <vv...@gmail.com>
Authored: Tue Jan 9 12:44:12 2018 +0200
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Tue Jan 16 12:10:13 2018 +0200

----------------------------------------------------------------------
 .../org/apache/drill/exec/planner/physical/SubsetTransformer.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/3f0e517f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SubsetTransformer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SubsetTransformer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SubsetTransformer.java
index 3562d5c..a871c3a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SubsetTransformer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SubsetTransformer.java
@@ -70,7 +70,7 @@ public abstract class SubsetTransformer<T extends RelNode, E extends Exception>
 
     //2, convert the candidateSet to targeted taitSets
     for (RelTraitSet traitSet: traitSets) {
-      RelNode newRel = RelOptRule.convert(candidateSet, traitSet);
+      RelNode newRel = RelOptRule.convert(candidateSet, traitSet.simplify());
       if(transformedRels.contains(newRel)) {
         continue;
       }


[09/18] drill git commit: DRILL-3993: Move Drill-specific commits 'CALCITE-628' and 'Drill-specific change: Add back AbstractConverter in RelSet.java' from Calcite into DRILL

Posted by am...@apache.org.
DRILL-3993: Move Drill-specific commits 'CALCITE-628' and 'Drill-specific change: Add back AbstractConverter in RelSet.java' from Calcite into DRILL


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

Branch: refs/heads/master
Commit: 688e043b4546884d82422b266b419817d645acfc
Parents: d59f0cd
Author: Volodymyr Vysotskyi <vv...@gmail.com>
Authored: Wed Dec 13 14:53:47 2017 +0200
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Tue Jan 16 12:10:13 2018 +0200

----------------------------------------------------------------------
 .../exec/store/jdbc/DrillJdbcRuleBase.java      |  4 +--
 .../exec/store/jdbc/JdbcStoragePlugin.java      |  2 +-
 .../planner/logical/DrillAggregateRule.java     |  2 +-
 .../exec/planner/logical/DrillFilterRule.java   |  2 +-
 .../exec/planner/logical/DrillJoinRule.java     |  4 +--
 .../exec/planner/logical/DrillLimitRule.java    |  2 +-
 .../exec/planner/logical/DrillProjectRule.java  |  2 +-
 .../exec/planner/logical/DrillSortRule.java     |  2 +-
 .../exec/planner/logical/DrillUnionAllRule.java |  2 +-
 .../exec/planner/logical/DrillWindowRule.java   |  2 +-
 .../drill/exec/planner/physical/Prel.java       | 29 ++++++++++++--------
 .../drill/exec/planner/physical/Prule.java      |  2 +-
 .../exec/planner/physical/SortConvertPrule.java |  8 ++++--
 .../planner/physical/SubsetTransformer.java     |  2 +-
 14 files changed, 37 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/688e043b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/DrillJdbcRuleBase.java
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/DrillJdbcRuleBase.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/DrillJdbcRuleBase.java
index 963e80a..9d29a22 100644
--- a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/DrillJdbcRuleBase.java
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/DrillJdbcRuleBase.java
@@ -65,7 +65,7 @@ abstract class DrillJdbcRuleBase extends ConverterRule {
     public RelNode convert(RelNode rel) {
       LogicalProject project = (LogicalProject) rel;
       return new JdbcRules.JdbcProject(rel.getCluster(), rel.getTraitSet().replace(this.out), convert(
-          project.getInput(), project.getInput().getTraitSet().replace(this.out)), project.getProjects(),
+          project.getInput(), project.getInput().getTraitSet().replace(this.out).simplify()), project.getProjects(),
           project.getRowType());
     }
 
@@ -97,7 +97,7 @@ abstract class DrillJdbcRuleBase extends ConverterRule {
       LogicalFilter filter = (LogicalFilter) rel;
 
       return new JdbcRules.JdbcFilter(rel.getCluster(), rel.getTraitSet().replace(this.out), convert(filter.getInput(),
-          filter.getInput().getTraitSet().replace(this.out)), filter.getCondition());
+          filter.getInput().getTraitSet().replace(this.out).simplify()), filter.getCondition());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/688e043b/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 4b41696..47f0f21 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
@@ -230,7 +230,7 @@ public class JdbcStoragePlugin extends AbstractStoragePlugin {
     @Override
     public RelNode convert(RelNode in) {
       return new JdbcDrel(in.getCluster(), in.getTraitSet().replace(DrillRel.DRILL_LOGICAL),
-          convert(in, in.getTraitSet().replace(this.getInTrait())));
+          convert(in, in.getTraitSet().replace(this.getInTrait()).simplify()));
     }
 
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/688e043b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java
index 70f2215..2b998b2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java
@@ -54,7 +54,7 @@ public class DrillAggregateRule extends RelOptRule {
     }
 
     final RelTraitSet traits = aggregate.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
-    final RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL));
+    final RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL).simplify());
     try {
       call.transformTo(new DrillAggregateRel(aggregate.getCluster(), traits, convertedInput, aggregate.indicator,
           aggregate.getGroupSet(), aggregate.getGroupSets(), aggregate.getAggCallList()));

http://git-wip-us.apache.org/repos/asf/drill/blob/688e043b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRule.java
index f3769f0..5480830 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRule.java
@@ -38,7 +38,7 @@ public class DrillFilterRule extends RelOptRule {
   public void onMatch(RelOptRuleCall call) {
     final LogicalFilter filter = call.rel(0);
     final RelNode input = filter.getInput();
-    final RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL));
+    final RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL).simplify());
     call.transformTo(new DrillFilterRel(
         filter.getCluster(), convertedInput.getTraitSet(),
         convertedInput, filter.getCondition()));

http://git-wip-us.apache.org/repos/asf/drill/blob/688e043b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
index 766509e..1ad41a2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
@@ -58,8 +58,8 @@ public class DrillJoinRule extends RelOptRule {
     final RelNode right = join.getRight();
     final RelTraitSet traits = join.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
 
-    final RelNode convertedLeft = convert(left, left.getTraitSet().plus(DrillRel.DRILL_LOGICAL));
-    final RelNode convertedRight = convert(right, right.getTraitSet().plus(DrillRel.DRILL_LOGICAL));
+    final RelNode convertedLeft = convert(left, left.getTraitSet().plus(DrillRel.DRILL_LOGICAL).simplify());
+    final RelNode convertedRight = convert(right, right.getTraitSet().plus(DrillRel.DRILL_LOGICAL).simplify());
 
     List<Integer> leftKeys = Lists.newArrayList();
     List<Integer> rightKeys = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/drill/blob/688e043b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java
index 6d2da9d..cac24f0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java
@@ -54,7 +54,7 @@ public class DrillLimitRule extends RelOptRule {
       input = incomingSort.copy(incomingTraits, input, incomingSort.getCollation(), null, null);
     }
 
-    RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL));
+    RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL).simplify());
     call.transformTo(new DrillLimitRel(
         incomingSort.getCluster(), convertedInput.getTraitSet().plus(DrillRel.DRILL_LOGICAL),
         convertedInput, incomingSort.offset, incomingSort.fetch));

http://git-wip-us.apache.org/repos/asf/drill/blob/688e043b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRule.java
index 14c6533..c38a626 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRule.java
@@ -41,7 +41,7 @@ public class DrillProjectRule extends RelOptRule {
     final Project project = call.rel(0);
     final RelNode input = project.getInput();
     final RelTraitSet traits = project.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
-    final RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL));
+    final RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL).simplify());
     call.transformTo(new DrillProjectRel(
         project.getCluster(), traits, convertedInput, project.getProjects(), project.getRowType()));
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/688e043b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRule.java
index 1ab061b..390ae1e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRule.java
@@ -49,7 +49,7 @@ public class DrillSortRule extends RelOptRule {
     final RelNode input = sort.getInput();
     final RelTraitSet traits = sort.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
 
-    final RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL));
+    final RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL).simplify());
     call.transformTo(new DrillSortRel(sort.getCluster(), traits, convertedInput, sort.getCollation()));
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/688e043b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionAllRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionAllRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionAllRule.java
index 91697a2..df9d05a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionAllRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionAllRule.java
@@ -55,7 +55,7 @@ public class DrillUnionAllRule extends RelOptRule {
     final RelTraitSet traits = union.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
     final List<RelNode> convertedInputs = new ArrayList<>();
     for (RelNode input : union.getInputs()) {
-      final RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL));
+      final RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL).simplify());
       convertedInputs.add(convertedInput);
     }
     try {

http://git-wip-us.apache.org/repos/asf/drill/blob/688e043b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWindowRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWindowRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWindowRule.java
index 4461796..3c3ac6e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWindowRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWindowRule.java
@@ -37,7 +37,7 @@ public class DrillWindowRule extends RelOptRule {
   public void onMatch(RelOptRuleCall call) {
     final Window window = call.rel(0);
     final RelNode input = call.rel(1);
-    final RelTraitSet traits = window.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
+    final RelTraitSet traits = window.getTraitSet().plus(DrillRel.DRILL_LOGICAL).simplify();
     final RelNode convertedInput = convert(input, traits);
     call.transformTo(
         new DrillWindowRel(

http://git-wip-us.apache.org/repos/asf/drill/blob/688e043b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
index db37040..77794d0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.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,31 +20,38 @@ package org.apache.drill.exec.planner.physical;
 import java.io.IOException;
 
 import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelTraitSet;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.planner.common.DrillRelNode;
 import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
-public interface Prel extends DrillRelNode, Iterable<Prel>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Prel.class);
+public interface Prel extends DrillRelNode, Iterable<Prel> {
+  org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Prel.class);
 
-  final static Convention DRILL_PHYSICAL = new Convention.Impl("PHYSICAL", Prel.class);
+  Convention DRILL_PHYSICAL = new Convention.Impl("PHYSICAL", Prel.class) {
+    public boolean canConvertConvention(Convention toConvention) {
+      return true;
+    }
 
-  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException;
+    public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits,
+        RelTraitSet toTraits) {
+      return true;
+    }
+  };
 
-  public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E;
+  PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException;
+
+  <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E;
 
   /**
    * Supported 'encodings' of a Prel indicates what are the acceptable modes of SelectionVector
    * of its child Prel
    */
-  public SelectionVectorMode[] getSupportedEncodings();
+  SelectionVectorMode[] getSupportedEncodings();
   /**
    * A Prel's own SelectionVector mode - i.e whether it generates an SV2, SV4 or None
    */
-  public SelectionVectorMode getEncoding();
+  SelectionVectorMode getEncoding();
   boolean needsFinalColumnReordering();
-
-  // DRILL-3011
-  // public abstract Prel copy(RelTraitSet paramRelTraitSet, List<RelNode> paramList);
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/688e043b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prule.java
index 148e74b..28d8dab 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prule.java
@@ -41,7 +41,7 @@ public abstract class Prule extends RelOptRule {
       toTraits = toTraits.replace(DrillDistributionTrait.ANY);
     }
 
-    return RelOptRule.convert(rel, toTraits);
+    return RelOptRule.convert(rel, toTraits.simplify());
   }
 
   public static boolean isSingleMode(RelOptRuleCall call) {

http://git-wip-us.apache.org/repos/asf/drill/blob/688e043b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortConvertPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortConvertPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortConvertPrule.java
index e13c550..714262d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortConvertPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortConvertPrule.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,6 +17,7 @@
  */
 package org.apache.drill.exec.planner.physical;
 
+import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.convert.ConverterRule;
@@ -47,9 +48,10 @@ public class SortConvertPrule extends ConverterRule {
   @Override
   public RelNode convert(RelNode r) {
     Sort rel = (Sort) r;
+    RelTraitSet traits = rel.getInput().getTraitSet().replace(Prel.DRILL_PHYSICAL);
     return new SortPrel(rel.getCluster(),
-                        rel.getInput().getTraitSet().replace(Prel.DRILL_PHYSICAL).plus(rel.getCollation()),
-                        convert(rel.getInput(), rel.getInput().getTraitSet().replace(Prel.DRILL_PHYSICAL)),
+                        traits.plus(rel.getCollation()),
+                        convert(rel.getInput(), traits.simplify()),
                         rel.getCollation());
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/688e043b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SubsetTransformer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SubsetTransformer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SubsetTransformer.java
index 03283b1..3562d5c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SubsetTransformer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SubsetTransformer.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


[08/18] drill git commit: DRILL-3993: Fix failed tests after Calcite update

Posted by am...@apache.org.
DRILL-3993: Fix failed tests after Calcite update

- fix temporary table errors according to updated logic;
- fixed errors when we trying to make select from hbase table with schema name in query (example: "SELECT row_key FROM hbase.TestTableNullStr) from hbase schema (did "USE hbase" before). Added test for it;
- added fix for views which were created on Calcite 1.4 and test for it.


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

Branch: refs/heads/master
Commit: 9274cb9204c6ebf5bd2d13fe4d02af5cebb48fa5
Parents: 0a525aa
Author: Roman Kulyk <ro...@gmail.com>
Authored: Thu Nov 2 18:22:36 2017 +0000
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Tue Jan 16 12:10:13 2018 +0200

----------------------------------------------------------------------
 .../apache/drill/hbase/TestHBaseQueries.java    |  7 ++
 .../org/apache/drill/exec/dotdrill/View.java    |  7 +-
 .../drill/exec/planner/sql/SqlConverter.java    | 99 ++++++++++++++------
 .../apache/drill/exec/rpc/user/UserSession.java |  9 ++
 .../org/apache/drill/exec/sql/TestCTTAS.java    | 15 +++
 .../apache/drill/exec/sql/TestViewSupport.java  | 10 ++
 .../view/view_from_calcite_1_4.view.drill       | 10 ++
 7 files changed, 127 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/9274cb92/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseQueries.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseQueries.java b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseQueries.java
index c3ee7d9..ee839c5 100644
--- a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseQueries.java
+++ b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseQueries.java
@@ -102,4 +102,11 @@ public class TestHBaseQueries extends BaseHBaseTest {
     }
   }
 
+  @Test
+  public void testSelectFromSchema() throws Exception {
+    setColumnWidths(new int[] {8, 15});
+    test("USE hbase");
+    runHBaseSQLVerifyCount("SELECT row_key\n"
+        + " FROM hbase.TestTableNullStr t WHERE row_key='a1'", 1);
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/9274cb92/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/View.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/View.java b/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/View.java
index 2b69f00..3524d73 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/View.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/dotdrill/View.java
@@ -71,8 +71,11 @@ public class View {
         @JsonProperty("endUnit")                    TimeUnit endUnit,
         @JsonProperty("fractionalSecondPrecision")  Integer fractionalSecondPrecision,
         @JsonProperty("isNullable")                 Boolean isNullable) {
-      this.name = name;
-      this.type = type;
+      // Fix for views which were created on Calcite 1.4.
+      // After Calcite upgrade star "*" was changed on dynamic star "**"
+      // and type of star was changed to SqlTypeName.DYNAMIC_STAR
+      this.name = "*".equals(name) ? "**" : name;
+      this.type = "*".equals(name) && type == SqlTypeName.ANY ? SqlTypeName.DYNAMIC_STAR : type;
       this.precision = precision;
       this.scale = scale;
       this.intervalQualifier =

http://git-wip-us.apache.org/repos/asf/drill/blob/9274cb92/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 f900587..8224d97 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,8 +26,6 @@ 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.avatica.util.Casing;
-import org.apache.calcite.avatica.util.Quoting;
 import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 import org.apache.calcite.plan.ConventionTraitDef;
@@ -37,7 +35,6 @@ import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.volcano.VolcanoPlanner;
 import org.apache.calcite.prepare.CalciteCatalogReader;
 import org.apache.calcite.prepare.Prepare;
-import org.apache.calcite.prepare.RelOptTableImpl;
 import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.type.RelDataType;
@@ -49,19 +46,20 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParser;
-import org.apache.calcite.sql.parser.SqlParserImplFactory;
 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;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.sql2rel.RelDecorrelator;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
 import org.apache.calcite.util.Util;
@@ -237,6 +235,51 @@ public class SqlConverter {
         RelDataTypeFactory typeFactory, SqlConformance conformance) {
       super(opTab, catalogReader, typeFactory, conformance);
     }
+
+    @Override
+    protected void validateFrom(
+        SqlNode node,
+        RelDataType targetRowType,
+        SqlValidatorScope scope) {
+      switch (node.getKind()) {
+      case AS:
+        if (((SqlCall) node).operand(0) instanceof SqlIdentifier) {
+          SqlIdentifier tempNode = ((SqlCall) node).operand(0);
+          DrillCalciteCatalogReader catalogReader = (SqlConverter.DrillCalciteCatalogReader) getCatalogReader();
+
+          // Check the schema and throw a valid SchemaNotFound exception instead of TableNotFound exception.
+          if (catalogReader.getTable(Lists.newArrayList(tempNode.names)) == null) {
+            catalogReader.isValidSchema(tempNode.names);
+          }
+          changeNamesIfTableIsTemporary(tempNode);
+        }
+      default:
+        super.validateFrom(node, targetRowType, scope);
+      }
+    }
+
+    @Override
+    public String deriveAlias(
+        SqlNode node,
+        int ordinal) {
+      if (node instanceof SqlIdentifier) {
+        SqlIdentifier tempNode = ((SqlIdentifier) node);
+        changeNamesIfTableIsTemporary(tempNode);
+      }
+      return SqlValidatorUtil.getAlias(node, ordinal);
+    }
+
+    private void changeNamesIfTableIsTemporary(SqlIdentifier tempNode) {
+      List<String> temporaryTableNames = ((SqlConverter.DrillCalciteCatalogReader) getCatalogReader()).getTemporaryNames(tempNode.names);
+      if (temporaryTableNames != null) {
+        SqlParserPos pos = tempNode.getComponentParserPosition(0);
+        List<SqlParserPos> poses = Lists.newArrayList();
+        for (int i = 0; i < temporaryTableNames.size(); i++) {
+          poses.add(i, pos);
+        }
+        tempNode.setNames(temporaryTableNames, poses);
+      }
+    }
   }
 
   private static class DrillTypeSystem extends RelDataTypeSystemImpl {
@@ -508,6 +551,16 @@ public class SqlConverter {
       this.allowTemporaryTables = false;
     }
 
+    private List<String> getTemporaryNames(List<String> names) {
+      if (mightBeTemporaryTable(names, session.getDefaultSchemaPath(), drillConfig)) {
+        String temporaryTableName = session.resolveTemporaryTableName(names.get(names.size() - 1));
+        if (temporaryTableName != null) {
+          return Lists.newArrayList(temporarySchema, temporaryTableName);
+        }
+      }
+      return null;
+    }
+
     /**
      * If schema is not indicated (only one element in the list) or schema is default temporary workspace,
      * we need to check among session temporary tables in default temporary workspace first.
@@ -520,33 +573,23 @@ public class SqlConverter {
      */
     @Override
     public Prepare.PreparingTable getTable(final List<String> names) {
-      Prepare.PreparingTable temporaryTable = null;
-
-      if (mightBeTemporaryTable(names, session.getDefaultSchemaPath(), drillConfig)) {
-        String temporaryTableName = session.resolveTemporaryTableName(names.get(names.size() - 1));
-        if (temporaryTableName != null) {
-          List<String> temporaryNames = Lists.newArrayList(temporarySchema, temporaryTableName);
-          temporaryTable = super.getTable(temporaryNames);
+      String originalTableName = session.getOriginalTableNameFromTemporaryTable(names.get(names.size() - 1));
+      if (originalTableName != null) {
+        if (!allowTemporaryTables) {
+          throw UserException
+              .validationError()
+              .message("Temporary tables usage is disallowed. Used temporary table name: [%s].", originalTableName)
+              .build(logger);
         }
       }
-      if (temporaryTable != null) {
-        if (allowTemporaryTables) {
-          return temporaryTable;
+      // Fix for select from hbase table with schema name in query (example: "SELECT col FROM hbase.t)
+      // from hbase schema (did "USE hbase" before).
+      if (names.size() == getSchemaPaths().size() && getSchemaPaths().size() > 1) {
+        if (names.get(0).equals(getSchemaPaths().get(0).get(0))) {
+          useRootSchemaAsDefault(true);
         }
-        throw UserException
-            .validationError()
-            .message("Temporary tables usage is disallowed. Used temporary table name: %s.", names)
-            .build(logger);
       }
-
-      Prepare.PreparingTable table = super.getTable(names);
-
-      // Check the schema and throw a valid SchemaNotFound exception instead of TableNotFound exception.
-      if (table == null) {
-        isValidSchema(names);
-      }
-
-      return table;
+      return super.getTable(names);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/9274cb92/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
index 2e3ce3e..4edeadc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
@@ -287,6 +287,15 @@ public class UserSession implements AutoCloseable {
     return temporaryTables.get(tableName.toLowerCase());
   }
 
+  public String getOriginalTableNameFromTemporaryTable(String tableName) {
+    for (String originalTableName : temporaryTables.keySet()) {
+      if (temporaryTables.get(originalTableName).equals(tableName)) {
+        return originalTableName;
+      }
+    }
+    return null;
+  }
+
   /**
    * Checks if passed table is temporary, table name is case-insensitive.
    * Before looking for table checks if passed schema is temporary and returns false if not

http://git-wip-us.apache.org/repos/asf/drill/blob/9274cb92/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTTAS.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTTAS.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTTAS.java
index 1553227..b334049 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTTAS.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTTAS.java
@@ -282,6 +282,21 @@ public class TestCTTAS extends BaseTestQuery {
   }
 
   @Test
+  public void testSelectWithJoinOnTemporaryTables() throws Exception {
+    String temporaryLeftTableName = "temporary_left_table_for_Select_with_join";
+    String temporaryRightTableName = "temporary_right_table_for_Select_with_join";
+    test("create TEMPORARY table %s as select 'A' as c1, 'B' as c2 from (values(1))", temporaryLeftTableName);
+    test("create TEMPORARY table %s as select 'A' as c1, 'C' as c2 from (values(1))", temporaryRightTableName);
+
+    testBuilder()
+        .sqlQuery("select t1.c2 col1, t2.c2 col2 from %s t1 join %s t2 on t1.c1 = t2.c1", temporaryLeftTableName, temporaryRightTableName)
+        .unOrdered()
+        .baselineColumns("col1", "col2")
+        .baselineValues("B", "C")
+        .go();
+  }
+
+  @Test
   public void testTemporaryAndPersistentTablesPriority() throws Exception {
     String name = "temporary_and_persistent_table";
     test("use %s", temp2_schema);

http://git-wip-us.apache.org/repos/asf/drill/blob/9274cb92/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
index 6efa5ce..a3cb69a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
@@ -741,4 +741,14 @@ public class TestViewSupport extends TestBaseViewSupport {
       test("DROP TABLE IF EXISTS %s.%s ", DFS_TMP_SCHEMA, tableName);
     }
   }
+
+  @Test
+  public void selectFromViewCreatedOnCalcite1_4() throws Exception {
+    testBuilder()
+        .sqlQuery("select store_type from cp.`view/view_from_calcite_1_4.view.drill`")
+        .unOrdered()
+        .baselineColumns("store_type")
+        .baselineValues("HeadQuarters")
+        .go();
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/9274cb92/exec/java-exec/src/test/resources/view/view_from_calcite_1_4.view.drill
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/view/view_from_calcite_1_4.view.drill b/exec/java-exec/src/test/resources/view/view_from_calcite_1_4.view.drill
new file mode 100755
index 0000000..6c86d62
--- /dev/null
+++ b/exec/java-exec/src/test/resources/view/view_from_calcite_1_4.view.drill
@@ -0,0 +1,10 @@
+{
+  "name" : "view_from_calcite_1_4",
+  "sql" : "SELECT *\nFROM `cp`.`store.json`\nWHERE `store_id` = 0",
+  "fields" : [ {
+    "name" : "*",
+    "type" : "ANY",
+    "isNullable" : true
+  } ],
+  "workspaceSchemaPath" : [ "dfs", "tmp" ]
+}
\ No newline at end of file


[16/18] drill git commit: DRILL-3993: Changes for CALCITE-2018

Posted by am...@apache.org.
DRILL-3993: Changes for CALCITE-2018


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

Branch: refs/heads/master
Commit: 22d0f7e57b84fcb034624579f68b638ffdad6b7e
Parents: 6580674
Author: Volodymyr Vysotskyi <vv...@gmail.com>
Authored: Tue Dec 26 15:03:59 2017 +0200
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Tue Jan 16 12:10:13 2018 +0200

----------------------------------------------------------------------
 .../cost/DrillRelMdDistinctRowCount.java        | 31 ++++++++++++++------
 .../exec/planner/cost/DrillRelMdRowCount.java   |  2 +-
 .../drill/exec/planner/sql/SqlConverter.java    | 12 ++++++--
 3 files changed, 33 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/22d0f7e5/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 5d4e4f6..ab7fb87 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
@@ -17,7 +17,9 @@
  */
 package org.apache.drill.exec.planner.cost;
 
+import org.apache.calcite.plan.volcano.RelSubset;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMdDistinctRowCount;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
@@ -27,7 +29,7 @@ import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.drill.exec.planner.logical.DrillScanRel;
 
-public class DrillRelMdDistinctRowCount extends RelMdDistinctRowCount{
+public class DrillRelMdDistinctRowCount extends RelMdDistinctRowCount {
   private static final DrillRelMdDistinctRowCount INSTANCE =
       new DrillRelMdDistinctRowCount();
 
@@ -35,18 +37,29 @@ public class DrillRelMdDistinctRowCount extends RelMdDistinctRowCount{
       ReflectiveRelMetadataProvider.reflectiveSource(
           BuiltInMethod.DISTINCT_ROW_COUNT.method, INSTANCE);
 
+  /**
+   * We need to override this method since Calcite and Drill calculate
+   * joined row count in different ways. It helps avoid a case when
+   * at the first time was used Drill join row count but at the second time
+   * Calcite row count was used. It may happen when
+   * {@link RelMdDistinctRowCount#getDistinctRowCount(Join, RelMetadataQuery,
+   * ImmutableBitSet, RexNode)} method is used and after that used
+   * another getDistinctRowCount method for parent rel, which just uses
+   * row count of input rel node (our join rel).
+   * It causes cost increase of best rel node when
+   * {@link RelSubset#propagateCostImprovements} is called.
+   *
+   * This is a part of the fix for CALCITE-2018.
+   */
   @Override
-  public Double getDistinctRowCount(RelNode rel, RelMetadataQuery mq, ImmutableBitSet groupKey, RexNode predicate) {
-    if (rel instanceof DrillScanRel) {
-      return getDistinctRowCount((DrillScanRel) rel, groupKey, predicate);
-    } else {
-      return super.getDistinctRowCount(rel, mq, groupKey, predicate);
-    }
+  public Double getDistinctRowCount(Join rel, RelMetadataQuery mq,
+      ImmutableBitSet groupKey, RexNode predicate) {
+    return getDistinctRowCount((RelNode) rel, mq, groupKey, predicate);
   }
 
-  private Double getDistinctRowCount(DrillScanRel scan, ImmutableBitSet groupKey, RexNode predicate) {
+  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;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/22d0f7e5/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java
index 5cc2c6a..7f15fb3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/cost/DrillRelMdRowCount.java
@@ -32,7 +32,7 @@ import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.drill.exec.planner.common.DrillLimitRelBase;
 
-public class DrillRelMdRowCount extends RelMdRowCount{
+public class DrillRelMdRowCount extends RelMdRowCount {
   private static final DrillRelMdRowCount INSTANCE = new DrillRelMdRowCount();
 
   public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource(BuiltInMethod.ROW_COUNT.method, INSTANCE);

http://git-wip-us.apache.org/repos/asf/drill/blob/22d0f7e5/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 8ad4df7..466d2fe 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
@@ -109,6 +109,7 @@ public class SqlConverter {
   private final String temporarySchema;
   private final UserSession session;
   private final DrillConfig drillConfig;
+  private RelOptCluster cluster;
 
   private String sql;
   private VolcanoPlanner planner;
@@ -139,6 +140,7 @@ public class SqlConverter {
     this.costFactory = (settings.useDefaultCosting()) ? null : new DrillCostBase.DrillCostFactory();
     this.validator = new DrillValidator(opTab, catalog, typeFactory, SqlConformance.DEFAULT);
     validator.setIdentifierExpansion(true);
+    cluster = null;
   }
 
   private SqlConverter(SqlConverter parent, SchemaPlus defaultSchema, SchemaPlus rootSchema,
@@ -161,6 +163,7 @@ public class SqlConverter {
     this.session = parent.session;
     this.drillConfig = parent.drillConfig;
     validator.setIdentifierExpansion(true);
+    this.cluster = parent.cluster;
   }
 
 
@@ -320,7 +323,6 @@ public class SqlConverter {
   }
 
   public RelRoot toRel(final SqlNode validatedNode) {
-    final RexBuilder rexBuilder = new DrillRexBuilder(typeFactory);
     if (planner == null) {
       planner = new VolcanoPlanner(costFactory, settings);
       planner.setExecutor(new DrillConstExecutor(functions, util, settings));
@@ -330,7 +332,9 @@ public class SqlConverter {
       planner.addRelTraitDef(RelCollationTraitDef.INSTANCE);
     }
 
-    final RelOptCluster cluster = RelOptCluster.create(planner, rexBuilder);
+    if (cluster == null) {
+      initCluster();
+    }
     final SqlToRelConverter sqlToRelConverter =
         new SqlToRelConverter(new Expander(), validator, catalog, cluster, DrillConvertletTable.INSTANCE,
             sqlToRelConverterConfig);
@@ -488,6 +492,10 @@ public class SqlConverter {
     }
   }
 
+  private void initCluster() {
+    cluster = RelOptCluster.create(planner, new DrillRexBuilder(typeFactory));
+  }
+
   private static class DrillRexBuilder extends RexBuilder {
     private DrillRexBuilder(RelDataTypeFactory typeFactory) {
       super(typeFactory);


[15/18] drill git commit: DRILL-3993: Add unit tests for DRILL-4469 & DRILL-5768

Posted by am...@apache.org.
DRILL-3993: Add unit tests for DRILL-4469 & DRILL-5768


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

Branch: refs/heads/master
Commit: 90b5e6392d09db44ddef058b8c8caf88f52e23a2
Parents: d06a7cb
Author: Volodymyr Vysotskyi <vv...@gmail.com>
Authored: Fri Jan 5 15:35:45 2018 +0200
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Tue Jan 16 12:10:13 2018 +0200

----------------------------------------------------------------------
 .../apache/drill/exec/TestWindowFunctions.java  | 22 ++++++++++++++++++--
 .../exec/fn/impl/TestAggregateFunctions.java    | 15 +++++++++++++
 2 files changed, 35 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/90b5e639/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
index 3851228..db72ec2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestWindowFunctions.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,7 +20,6 @@ package org.apache.drill.exec;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.common.util.DrillFileUtils;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
 import org.apache.drill.exec.work.foreman.UnsupportedFunctionException;
@@ -935,4 +934,23 @@ public class TestWindowFunctions extends BaseTestQuery {
       assert(ex.getMessage().contains("Expression 'n_nationkey' is not being grouped"));
     }
   }
+
+  @Test // DRILL-4469
+  public void testWindowOnSubqueryWithStar() throws Exception {
+    String query = "SELECT SUM(n_nationkey) OVER w as s\n" +
+        "FROM (SELECT * FROM cp.`tpch/nation.parquet`) subQry\n" +
+        "WINDOW w AS (PARTITION BY REGION ORDER BY n_nationkey)\n" +
+        "limit 1";
+
+    final String[] expectedPlan = {"Project.*\\$0=\\[ITEM\\(\\$1, 'n_nationkey'\\)\\].*"};
+    PlanTestBase.testPlanMatchingPatterns(query, expectedPlan, new String[]{});
+
+    testBuilder()
+        .sqlQuery(query)
+        .ordered()
+        .baselineColumns("s")
+        .baselineValues(0L)
+        .build()
+        .run();
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/90b5e639/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
index ebad2f7..120f1e6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunctions.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.fn.impl;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.test.BaseTestQuery;
 import org.apache.drill.categories.OperatorTest;
 import org.apache.drill.PlanTestBase;
@@ -42,6 +43,9 @@ import java.nio.file.Paths;
 import java.util.List;
 import java.util.Map;
 
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 @Category({SqlFunctionTest.class, OperatorTest.class, PlannerTest.class})
 public class TestAggregateFunctions extends BaseTestQuery {
 
@@ -608,4 +612,15 @@ public class TestAggregateFunctions extends BaseTestQuery {
       .baselineValues(1L, 1L, 1L, 0L, 1L)
       .go();
   }
+
+  @Test // DRILL-5768
+  public void testGroupByWithoutAggregate() throws Exception {
+    try {
+      test("select * from cp.`tpch/nation.parquet` group by n_regionkey");
+      fail("Exception was not thrown");
+    } catch (UserRemoteException e) {
+      assertTrue("No expected current \"Expression 'tpch/nation.parquet.**' is not being grouped\"",
+          e.getMessage().matches(".*Expression 'tpch/nation\\.parquet\\.\\*\\*' is not being grouped(.*\\n)*"));
+    }
+  }
 }


[03/18] drill git commit: DRILL-3993: Fix aggregate exchange rules for the cases when aggregate rel node contains several calls

Posted by am...@apache.org.
DRILL-3993: Fix aggregate exchange rules for the cases when aggregate rel node contains several calls


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

Branch: refs/heads/master
Commit: 3c9093e32a095bd40832bcd8fe67ab20898537c4
Parents: 22d0f7e
Author: Volodymyr Vysotskyi <vv...@gmail.com>
Authored: Thu Jan 4 16:05:53 2018 +0200
Committer: Volodymyr Vysotskyi <vv...@gmail.com>
Committed: Tue Jan 16 12:10:13 2018 +0200

----------------------------------------------------------------------
 .../exec/planner/physical/AggPrelBase.java      | 24 ++++++++++--------
 .../exec/planner/physical/AggPruleBase.java     | 26 ++++++++++++++++----
 .../exec/planner/physical/HashAggPrule.java     | 22 +++++++++++------
 .../exec/planner/physical/StreamAggPrule.java   | 25 ++++++++++++++-----
 4 files changed, 69 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/3c9093e3/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 8c69930..232473b 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
@@ -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
@@ -31,7 +31,6 @@ import org.apache.drill.common.logical.data.NamedExpression;
 import org.apache.drill.exec.planner.common.DrillAggregateRelBase;
 import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.apache.calcite.rel.core.AggregateCall;
-import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.plan.RelOptCluster;
@@ -44,14 +43,13 @@ import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 
-import java.util.BitSet;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
 public abstract class AggPrelBase extends DrillAggregateRelBase implements Prel {
 
-  public static enum OperatorPhase {PHASE_1of1, PHASE_1of2, PHASE_2of2};
+  public enum OperatorPhase {PHASE_1of1, PHASE_1of2, PHASE_2of2}
 
   protected OperatorPhase operPhase = OperatorPhase.PHASE_1of1 ; // default phase
   protected List<NamedExpression> keys = Lists.newArrayList();
@@ -70,11 +68,14 @@ public abstract class AggPrelBase extends DrillAggregateRelBase implements Prel
 
     public SqlSumCountAggFunction(RelDataType type) {
       super("$SUM0",
+          null,
           SqlKind.OTHER_FUNCTION,
           ReturnTypes.BIGINT, // use the inferred return type of SqlCountAggFunction
           null,
           OperandTypes.NUMERIC,
-          SqlFunctionCategory.NUMERIC);
+          SqlFunctionCategory.NUMERIC,
+          false,
+          false);
 
       this.type = type;
     }
@@ -143,20 +144,24 @@ public abstract class AggPrelBase extends DrillAggregateRelBase implements Prel
           // If we are doing a COUNT aggregate in Phase1of2, then in Phase2of2 we should SUM the COUNTs,
           SqlAggFunction sumAggFun = new SqlSumCountAggFunction(aggCall.e.getType());
           AggregateCall newAggCall =
-              new AggregateCall(
+              AggregateCall.create(
                   sumAggFun,
                   aggCall.e.isDistinct(),
+                  aggCall.e.isApproximate(),
                   Collections.singletonList(aggExprOrdinal),
+                  aggCall.e.filterArg,
                   aggCall.e.getType(),
                   aggCall.e.getName());
 
           phase2AggCallList.add(newAggCall);
         } else {
           AggregateCall newAggCall =
-              new AggregateCall(
+              AggregateCall.create(
                   aggCall.e.getAggregation(),
                   aggCall.e.isDistinct(),
+                  aggCall.e.isApproximate(),
                   Collections.singletonList(aggExprOrdinal),
+                  aggCall.e.filterArg,
                   aggCall.e.getType(),
                   aggCall.e.getName());
 
@@ -174,10 +179,9 @@ public abstract class AggPrelBase extends DrillAggregateRelBase implements Prel
 
     // for count(1).
     if (args.isEmpty()) {
-      args.add(new ValueExpressions.LongExpression(1l));
+      args.add(new ValueExpressions.LongExpression(1L));
     }
-    LogicalExpression expr = new FunctionCall(call.getAggregation().getName().toLowerCase(), args, ExpressionPosition.UNKNOWN );
-    return expr;
+    return new FunctionCall(call.getAggregation().getName().toLowerCase(), args, ExpressionPosition.UNKNOWN);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/3c9093e3/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.java
index 84e37fc..6863967 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/AggPruleBase.java
@@ -21,8 +21,8 @@ package org.apache.drill.exec.planner.physical;
 import java.util.List;
 
 import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.util.BitSets;
 
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.drill.exec.planner.logical.DrillAggregateRel;
 import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
 import org.apache.calcite.rel.core.AggregateCall;
@@ -42,7 +42,7 @@ public abstract class AggPruleBase extends Prule {
   protected List<DistributionField> getDistributionField(DrillAggregateRel rel, boolean allFields) {
     List<DistributionField> groupByFields = Lists.newArrayList();
 
-    for (int group : BitSets.toIter(rel.getGroupSet())) {
+    for (int group : remapGroupSet(rel.getGroupSet())) {
       DistributionField field = new DistributionField(group);
       groupByFields.add(field);
 
@@ -63,10 +63,11 @@ public abstract class AggPruleBase extends Prule {
   protected boolean create2PhasePlan(RelOptRuleCall call, DrillAggregateRel aggregate) {
     PlannerSettings settings = PrelUtil.getPlannerSettings(call.getPlanner());
     RelNode child = call.rel(0).getInputs().get(0);
-    boolean smallInput = child.getRows() < settings.getSliceTarget();
-    if (! settings.isMultiPhaseAggEnabled() || settings.isSingleMode() ||
+    boolean smallInput =
+        child.estimateRowCount(child.getCluster().getMetadataQuery()) < settings.getSliceTarget();
+    if (!settings.isMultiPhaseAggEnabled() || settings.isSingleMode()
         // Can override a small child - e.g., for testing with a small table
-        ( smallInput && ! settings.isForce2phaseAggr() ) ) {
+        || (smallInput && !settings.isForce2phaseAggr())) {
       return false;
     }
 
@@ -82,4 +83,19 @@ public abstract class AggPruleBase extends Prule {
     }
     return true;
   }
+
+  /**
+   * Returns group-by keys with the remapped arguments for specified aggregate.
+   *
+   * @param groupSet ImmutableBitSet of aggregate rel node, whose group-by keys should be remapped.
+   * @return {@link ImmutableBitSet} instance with remapped keys.
+   */
+  public static ImmutableBitSet remapGroupSet(ImmutableBitSet groupSet) {
+    List<Integer> newGroupSet = Lists.newArrayList();
+    int groupSetToAdd = 0;
+    for (int ignored : groupSet) {
+      newGroupSet.add(groupSetToAdd++);
+    }
+    return ImmutableBitSet.of(newGroupSet);
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/3c9093e3/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrule.java
index f4cdf62..02dd4de 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrule.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.exec.planner.physical;
 
+import com.google.common.collect.Lists;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.drill.exec.planner.logical.DrillAggregateRel;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
 import org.apache.drill.exec.planner.physical.AggPrelBase.OperatorPhase;
@@ -31,6 +33,8 @@ import org.apache.calcite.util.trace.CalciteTrace;
 import com.google.common.collect.ImmutableList;
 import org.slf4j.Logger;
 
+import java.util.List;
+
 public class HashAggPrule extends AggPruleBase {
   public static final RelOptRule INSTANCE = new HashAggPrule();
   protected static final Logger tracer = CalciteTrace.getPlannerTracer();
@@ -51,7 +55,7 @@ public class HashAggPrule extends AggPruleBase {
       return;
     }
 
-    final DrillAggregateRel aggregate = (DrillAggregateRel) call.rel(0);
+    final DrillAggregateRel aggregate = call.rel(0);
     final RelNode input = call.rel(1);
 
     if (aggregate.containsDistinctCall() || aggregate.getGroupCount() == 0) {
@@ -60,7 +64,7 @@ public class HashAggPrule extends AggPruleBase {
       return;
     }
 
-    RelTraitSet traits = null;
+    RelTraitSet traits;
 
     try {
       if (aggregate.getGroupSet().isEmpty()) {
@@ -125,18 +129,22 @@ public class HashAggPrule extends AggPruleBase {
           new HashToRandomExchangePrel(phase1Agg.getCluster(), phase1Agg.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(distOnAllKeys),
               phase1Agg, ImmutableList.copyOf(getDistributionField(aggregate, true)));
 
-      HashAggPrel phase2Agg =  new HashAggPrel(
+      ImmutableBitSet newGroupSet = remapGroupSet(aggregate.getGroupSet());
+      List<ImmutableBitSet> newGroupSets = Lists.newArrayList();
+      for (ImmutableBitSet groupSet : aggregate.getGroupSets()) {
+        newGroupSets.add(remapGroupSet(groupSet));
+      }
+
+      return new HashAggPrel(
           aggregate.getCluster(),
           exch.getTraitSet(),
           exch,
           aggregate.indicator,
-          aggregate.getGroupSet(),
-          aggregate.getGroupSets(),
+          newGroupSet,
+          newGroupSets,
           phase1Agg.getPhase2AggCalls(),
           OperatorPhase.PHASE_2of2);
-      return phase2Agg;
     }
-
   }
 
   private void createTransformRequest(RelOptRuleCall call, DrillAggregateRel aggregate,

http://git-wip-us.apache.org/repos/asf/drill/blob/3c9093e3/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 a6a8f28..29fa750 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
@@ -21,6 +21,7 @@ import java.util.List;
 
 import org.apache.calcite.util.BitSets;
 
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.drill.exec.planner.logical.DrillAggregateRel;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
 import org.apache.drill.exec.planner.physical.AggPrelBase.OperatorPhase;
@@ -53,10 +54,10 @@ public class StreamAggPrule extends AggPruleBase {
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final DrillAggregateRel aggregate = (DrillAggregateRel) call.rel(0);
+    final DrillAggregateRel aggregate = call.rel(0);
     RelNode input = aggregate.getInput();
     final RelCollation collation = getCollation(aggregate);
-    RelTraitSet traits = null;
+    RelTraitSet traits;
 
     if (aggregate.containsDistinctCall()) {
       // currently, don't use StreamingAggregate if any of the logical aggrs contains DISTINCT
@@ -93,13 +94,19 @@ public class StreamAggPrule extends AggPruleBase {
               UnionExchangePrel exch =
                   new UnionExchangePrel(phase1Agg.getCluster(), singleDistTrait, phase1Agg);
 
+              ImmutableBitSet newGroupSet = remapGroupSet(aggregate.getGroupSet());
+              List<ImmutableBitSet> newGroupSets = Lists.newArrayList();
+              for (ImmutableBitSet groupSet : aggregate.getGroupSets()) {
+                newGroupSets.add(remapGroupSet(groupSet));
+              }
+
               return  new StreamAggPrel(
                   aggregate.getCluster(),
                   singleDistTrait,
                   exch,
                   aggregate.indicator,
-                  aggregate.getGroupSet(),
-                  aggregate.getGroupSets(),
+                  newGroupSet,
+                  newGroupSets,
                   phase1Agg.getPhase2AggCalls(),
                   OperatorPhase.PHASE_2of2);
             }
@@ -160,13 +167,19 @@ public class StreamAggPrule extends AggPruleBase {
                       collation,
                       numEndPoints);
 
+              ImmutableBitSet newGroupSet = remapGroupSet(aggregate.getGroupSet());
+              List<ImmutableBitSet> newGroupSets = Lists.newArrayList();
+              for (ImmutableBitSet groupSet : aggregate.getGroupSets()) {
+                newGroupSets.add(remapGroupSet(groupSet));
+              }
+
               return new StreamAggPrel(
                   aggregate.getCluster(),
                   exch.getTraitSet(),
                   exch,
                   aggregate.indicator,
-                  aggregate.getGroupSet(),
-                  aggregate.getGroupSets(),
+                  newGroupSet,
+                  newGroupSets,
                   phase1Agg.getPhase2AggCalls(),
                   OperatorPhase.PHASE_2of2);
             }