You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pa...@apache.org on 2017/02/04 03:58:57 UTC

[7/8] drill git commit: DRILL-5238: CTTAS: unable to resolve temporary table if workspace is indicated without schema

DRILL-5238: CTTAS: unable to resolve temporary table if workspace is indicated without schema

This closes #736


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

Branch: refs/heads/master
Commit: 1ec3edf01575f302b765b62317ca16c1547dbe10
Parents: c8fbc38
Author: Arina Ielchiieva <ar...@gmail.com>
Authored: Thu Feb 2 11:47:19 2017 +0000
Committer: Parth Chandra <pc...@maprtech.com>
Committed: Fri Feb 3 17:42:45 2017 -0800

----------------------------------------------------------------------
 .../org/apache/drill/exec/ops/QueryContext.java |  4 +-
 .../drill/exec/planner/sql/SqlConverter.java    | 46 ++++++++++++++++----
 .../apache/drill/exec/rpc/user/UserSession.java |  8 +---
 .../org/apache/drill/exec/sql/TestCTTAS.java    | 14 ++++++
 4 files changed, 55 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/1ec3edf0/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 101600c..4ee8a9d 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
@@ -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
@@ -93,7 +93,7 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext, Schem
     plannerSettings.setNumEndPoints(drillbitContext.getBits().size());
     table = new DrillOperatorTable(getFunctionRegistry(), drillbitContext.getOptionManager());
 
-    queryContextInfo = Utilities.createQueryContextInfo(session.getDefaultSchemaName(), session.getSessionId());
+    queryContextInfo = Utilities.createQueryContextInfo(session.getDefaultSchemaPath(), session.getSessionId());
     contextInformation = new ContextInformation(session.getCredentials(), queryContextInfo);
 
     allocator = drillbitContext.getAllocator().newChildAllocator(

http://git-wip-us.apache.org/repos/asf/drill/blob/1ec3edf0/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 d20fc06..3e3226d 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
@@ -480,25 +480,27 @@ public class SqlConverter {
     private boolean allowTemporaryTables;
 
     DrillCalciteCatalogReader(CalciteSchema rootSchema,
-                                     boolean caseSensitive,
-                                     List<String> defaultSchema,
-                                     JavaTypeFactory typeFactory,
-                                     DrillConfig drillConfig,
-                                     UserSession session) {
+                              boolean caseSensitive,
+                              List<String> defaultSchema,
+                              JavaTypeFactory typeFactory,
+                              DrillConfig drillConfig,
+                              UserSession session) {
       super(rootSchema, caseSensitive, defaultSchema, typeFactory);
       this.drillConfig = drillConfig;
       this.session = session;
       this.allowTemporaryTables = true;
     }
 
-    /** Disallow temporary tables presence in sql statement (ex: in view definitions) */
+    /**
+     * Disallow temporary tables presence in sql statement (ex: in view definitions)
+     */
     public void disallowTemporaryTables() {
       this.allowTemporaryTables = false;
     }
 
     /**
      * 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 first in default temporary workspace.
+     * we need to check among session temporary tables in default temporary workspace first.
      * If temporary table is found and temporary tables usage is allowed, its table instance will be returned,
      * otherwise search will be conducted in original workspace.
      *
@@ -509,8 +511,8 @@ public class SqlConverter {
     @Override
     public RelOptTableImpl getTable(final List<String> names) {
       RelOptTableImpl temporaryTable = null;
-      String schemaPath = SchemaUtilites.getSchemaPath(names.subList(0, names.size() - 1));
-      if (names.size() == 1 || SchemaUtilites.isTemporaryWorkspace(schemaPath, drillConfig)) {
+
+      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);
@@ -528,5 +530,31 @@ public class SqlConverter {
       }
       return super.getTable(names);
     }
+
+    /**
+     * We should check if passed table is temporary or not if:
+     * <li>schema is not indicated (only one element in the names list)<li/>
+     * <li>current schema or indicated schema is default temporary workspace<li/>
+     *
+     * Examples (where dfs.tmp is default temporary workspace):
+     * <li>select * from t<li/>
+     * <li>select * from dfs.tmp.t<li/>
+     * <li>use dfs; select * from tmp.t<li/>
+     *
+     * @param names             list of schema and table names, table name is always the last element
+     * @param defaultSchemaPath current schema path set using USE command
+     * @param drillConfig       drill config
+     * @return true if check for temporary table should be done, false otherwise
+     */
+    private boolean mightBeTemporaryTable(List<String> names, String defaultSchemaPath, DrillConfig drillConfig) {
+      if (names.size() == 1) {
+        return true;
+      }
+
+      String schemaPath = SchemaUtilites.getSchemaPath(names.subList(0, names.size() - 1));
+      return SchemaUtilites.isTemporaryWorkspace(schemaPath, drillConfig) ||
+          SchemaUtilites.isTemporaryWorkspace(
+              SchemaUtilites.SCHEMA_PATH_JOINER.join(defaultSchemaPath, schemaPath), drillConfig);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/1ec3edf0/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 27bf3e9..c1e577d 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
@@ -65,8 +65,8 @@ public class UserSession implements Closeable {
 
   private boolean supportComplexTypes = false;
   private UserCredentials credentials;
-  private Map<String, String> properties;
   private OptionManager sessionOptions;
+  private final Map<String, String> properties;
   private final AtomicInteger queryCount;
   private final String sessionId;
 
@@ -121,7 +121,6 @@ public class UserSession implements Closeable {
     }
 
     public Builder withUserProperties(UserProperties properties) {
-      userSession.properties = Maps.newHashMap();
       if (properties != null) {
         for (int i = 0; i < properties.getPropertiesCount(); i++) {
           final Property property = properties.getProperties(i);
@@ -157,6 +156,7 @@ public class UserSession implements Closeable {
     sessionId = UUID.randomUUID().toString();
     temporaryTables = Maps.newConcurrentMap();
     temporaryLocations = Maps.newConcurrentMap();
+    properties = Maps.newHashMap();
   }
 
   public boolean isSupportComplexTypes() {
@@ -189,10 +189,6 @@ public class UserSession implements Closeable {
     return properties.get(IMPERSONATION_TARGET);
   }
 
-  public String getDefaultSchemaName() {
-    return getProp(SCHEMA);
-  }
-
   public void incrementQueryCount(final QueryCountIncrementer incrementer) {
     assert incrementer != null;
     queryCount.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/drill/blob/1ec3edf0/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 5d83beb..b38af4a 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
@@ -143,6 +143,20 @@ public class TestCTTAS extends BaseTestQuery {
   }
 
   @Test
+  public void testResolveTemporaryTableWithPartialSchema() throws Exception {
+    String temporaryTableName = "temporary_table_with_partial_schema";
+    test("use %s", test_schema);
+    test("create temporary table tmp.%s as select 'A' as c1 from (values(1))", temporaryTableName);
+
+    testBuilder()
+        .sqlQuery("select * from tmp.%s", temporaryTableName)
+        .unOrdered()
+        .baselineColumns("c1")
+        .baselineValues("A")
+        .go();
+  }
+
+  @Test
   public void testPartitionByWithTemporaryTables() throws Exception {
     String temporaryTableName = "temporary_table_with_partitions";
     mockRandomUUID(UUID.nameUUIDFromBytes(temporaryTableName.getBytes()));