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

[10/29] drill git commit: DRILL-5255: Remove default temporary workspace check at drillbit start up

DRILL-5255: Remove default temporary workspace check at drillbit start up

closes #759


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

Branch: refs/heads/master
Commit: 5db557c66b3d3b1a01ff6a8d1c0081205c8b6ef3
Parents: 456e26c
Author: Arina Ielchiieva <ar...@gmail.com>
Authored: Wed Feb 22 18:13:14 2017 +0000
Committer: Sudheesh Katkam <su...@apache.org>
Committed: Fri Feb 24 19:01:40 2017 -0800

----------------------------------------------------------------------
 .../drill/exec/planner/sql/SchemaUtilites.java  | 90 ++++++++++++++------
 .../sql/handlers/CreateTableHandler.java        | 27 +++---
 .../planner/sql/handlers/DropTableHandler.java  |  5 +-
 .../apache/drill/exec/rpc/user/UserSession.java | 21 +++--
 .../org/apache/drill/exec/server/Drillbit.java  | 21 -----
 .../java/org/apache/drill/BaseTestQuery.java    |  1 -
 .../java/org/apache/drill/TestDropTable.java    |  2 +-
 .../user/TemporaryTablesAutomaticDropTest.java  |  6 +-
 .../org/apache/drill/exec/sql/TestCTTAS.java    | 11 ++-
 9 files changed, 102 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/5db557c6/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java
index 20c92c7..51c3cb1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java
@@ -21,9 +21,7 @@ import com.google.common.base.Joiner;
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
 import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.tools.ValidationException;
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.store.AbstractSchema;
@@ -70,10 +68,9 @@ public class SchemaUtilites {
   /**
    * Same utility as {@link #findSchema(SchemaPlus, List)} except the search schema path given here is complete path
    * instead of list. Use "." separator to divided the schema into nested schema names.
-   * @param defaultSchema
-   * @param schemaPath
-   * @return
-   * @throws ValidationException
+   * @param defaultSchema default schema
+   * @param schemaPath current schema path
+   * @return found schema path
    */
   public static SchemaPlus findSchema(final SchemaPlus defaultSchema, final String schemaPath) {
     final List<String> schemaPathAsList = Lists.newArrayList(schemaPath.split("\\."));
@@ -92,9 +89,8 @@ public class SchemaUtilites {
   }
 
   /**
-   * Returns true if the given <i>schema</i> is root schema. False otherwise.
-   * @param schema
-   * @return
+   * @param schema current schema
+   * @return true if the given <i>schema</i> is root schema. False otherwise.
    */
   public static boolean isRootSchema(SchemaPlus schema) {
     return schema.getParentSchema() == null;
@@ -128,7 +124,7 @@ public class SchemaUtilites {
   /** Utility method to get the schema path as list for given schema instance. */
   public static List<String> getSchemaPathAsList(SchemaPlus schema) {
     if (isRootSchema(schema)) {
-      return Collections.EMPTY_LIST;
+      return Collections.emptyList();
     }
 
     List<String> path = Lists.newArrayListWithCapacity(5);
@@ -156,12 +152,13 @@ public class SchemaUtilites {
   /**
    * Given reference to default schema in schema tree, search for schema with given <i>schemaPath</i>. Once a schema is
    * found resolve it into a mutable <i>AbstractDrillSchema</i> instance. A {@link UserException} is throws when:
-   *   1. No schema for given <i>schemaPath</i> is found,
-   *   2. Schema found for given <i>schemaPath</i> is a root schema
-   *   3. Resolved schema is not a mutable schema.
-   * @param defaultSchema
-   * @param schemaPath
-   * @return
+   *   <li>No schema for given <i>schemaPath</i> is found.</li>
+   *   <li>Schema found for given <i>schemaPath</i> is a root schema.</li>
+   *   <li>Resolved schema is not a mutable schema.</li>
+   *
+   * @param defaultSchema default schema
+   * @param schemaPath current schema path
+   * @return mutable schema, exception otherwise
    */
   public static AbstractSchema resolveToMutableDrillSchema(final SchemaPlus defaultSchema, List<String> schemaPath) {
     final SchemaPlus schema = findSchema(defaultSchema, schemaPath);
@@ -171,7 +168,7 @@ public class SchemaUtilites {
     }
 
     if (isRootSchema(schema)) {
-      throw UserException.parseError()
+      throw UserException.validationError()
           .message("Root schema is immutable. Creating or dropping tables/views is not allowed in root schema." +
               "Select a schema using 'USE schema' command.")
           .build(logger);
@@ -179,7 +176,7 @@ public class SchemaUtilites {
 
     final AbstractSchema drillSchema = unwrapAsDrillSchemaInstance(schema);
     if (!drillSchema.isMutable()) {
-      throw UserException.parseError()
+      throw UserException.validationError()
           .message("Unable to create or drop tables/views. Schema [%s] is immutable.", getSchemaPath(schema))
           .build(logger);
     }
@@ -189,21 +186,16 @@ public class SchemaUtilites {
 
   /**
    * Looks in schema tree for default temporary workspace instance.
-   * Makes sure that temporary workspace is mutable and file-based
-   * (instance of {@link WorkspaceSchemaFactory.WorkspaceSchema}).
    *
    * @param defaultSchema default schema
    * @param config drill config
-   * @return default temporary workspace
+   * @return default temporary workspace, null if workspace was not found
    */
   public static AbstractSchema getTemporaryWorkspace(SchemaPlus defaultSchema, DrillConfig config) {
-    List<String> temporarySchemaPath = Lists.newArrayList(config.getString(ExecConstants.DEFAULT_TEMPORARY_WORKSPACE));
-    AbstractSchema temporarySchema = resolveToMutableDrillSchema(defaultSchema, temporarySchemaPath);
-    if (!(temporarySchema instanceof WorkspaceSchemaFactory.WorkspaceSchema)) {
-      DrillRuntimeException.format("Temporary workspace [%s] must be file-based, instance of " +
-          "WorkspaceSchemaFactory.WorkspaceSchema", temporarySchemaPath);
-    }
-    return temporarySchema;
+    String temporarySchema = config.getString(ExecConstants.DEFAULT_TEMPORARY_WORKSPACE);
+    List<String> temporarySchemaPath = Lists.newArrayList(temporarySchema);
+    SchemaPlus schema = findSchema(defaultSchema, temporarySchemaPath);
+    return schema == null ? null : unwrapAsDrillSchemaInstance(schema);
   }
 
   /**
@@ -217,4 +209,46 @@ public class SchemaUtilites {
   public static boolean isTemporaryWorkspace(String schemaPath, DrillConfig config) {
     return schemaPath.equals(config.getString(ExecConstants.DEFAULT_TEMPORARY_WORKSPACE));
   }
+
+  /**
+   * Makes sure that passed workspace exists, is default temporary workspace, mutable and file-based
+   * (instance of {@link WorkspaceSchemaFactory.WorkspaceSchema}).
+   *
+   * @param schema drill schema
+   * @param config drill config
+   * @return mutable & file-based workspace instance, otherwise throws validation error
+   */
+  public static WorkspaceSchemaFactory.WorkspaceSchema resolveToValidTemporaryWorkspace(AbstractSchema schema,
+                                                                                        DrillConfig config) {
+    if (schema == null) {
+      throw UserException.validationError()
+          .message("Default temporary workspace is not found")
+          .build(logger);
+    }
+
+    if (!isTemporaryWorkspace(schema.getFullSchemaName(), config)) {
+      throw UserException
+          .validationError()
+          .message(String.format("Temporary tables are not allowed to be created / dropped " +
+                  "outside of default temporary workspace [%s].",
+              config.getString(ExecConstants.DEFAULT_TEMPORARY_WORKSPACE)))
+          .build(logger);
+    }
+
+    if (!schema.isMutable()) {
+      throw UserException.validationError()
+          .message("Unable to create or drop temporary table. Schema [%s] is immutable.", schema.getFullSchemaName())
+          .build(logger);
+    }
+
+    if (schema instanceof WorkspaceSchemaFactory.WorkspaceSchema) {
+      return (WorkspaceSchemaFactory.WorkspaceSchema) schema;
+    } else {
+      throw UserException.validationError()
+          .message("Temporary workspace [%s] must be file-based, instance of " +
+              "WorkspaceSchemaFactory.WorkspaceSchema", schema)
+          .build(logger);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/5db557c6/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateTableHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateTableHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateTableHandler.java
index 07a3644..d50391c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateTableHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateTableHandler.java
@@ -97,7 +97,7 @@ public class CreateTableHandler extends DefaultSqlHandler {
     // Generated table name is unique, UUID.randomUUID() is used for its generation.
     // Original table name is stored in temporary tables cache, so it can be substituted to generated one during querying.
     String newTableName = sqlCreateTable.isTemporary() ?
-        context.getSession().registerTemporaryTable(drillSchema, originalTableName) : originalTableName;
+        context.getSession().registerTemporaryTable(drillSchema, originalTableName, drillConfig) : originalTableName;
 
     DrillRel drel = convertToDrel(newTblRelNodeWithPCol, drillSchema, newTableName,
         sqlCreateTable.getPartitionColumns(), newTblRelNode.getRowType(), storageStrategy);
@@ -263,8 +263,7 @@ public class CreateTableHandler extends DefaultSqlHandler {
    * returns temporary workspace.
    *
    * If schema path is indicated, resolves to mutable drill schema.
-   * Though if table to be created is temporary table, checks if resolved schema is temporary,
-   * since temporary table are allowed to be created only in temporary workspace.
+   * Though if table to be created is temporary table, checks if resolved schema is valid default temporary workspace.
    *
    * @param sqlCreateTable create table call
    * @param defaultSchema default schema
@@ -273,21 +272,19 @@ public class CreateTableHandler extends DefaultSqlHandler {
    * @throws UserException if attempted to create temporary table outside of temporary workspace
    */
   private AbstractSchema resolveSchema(SqlCreateTable sqlCreateTable, SchemaPlus defaultSchema, DrillConfig config) {
+    AbstractSchema resolvedSchema;
     if (sqlCreateTable.isTemporary() && sqlCreateTable.getSchemaPath().size() == 0) {
-      return SchemaUtilites.getTemporaryWorkspace(defaultSchema, config);
+      resolvedSchema = SchemaUtilites.getTemporaryWorkspace(defaultSchema, config);
     } else {
-      AbstractSchema resolvedSchema = SchemaUtilites.resolveToMutableDrillSchema(defaultSchema, sqlCreateTable.getSchemaPath());
-      boolean isTemporaryWorkspace = SchemaUtilites.isTemporaryWorkspace(resolvedSchema.getFullSchemaName(), config);
-
-      if (sqlCreateTable.isTemporary() && !isTemporaryWorkspace) {
-        throw UserException
-            .validationError()
-            .message(String.format("Temporary tables are not allowed to be created " +
-                "outside of default temporary workspace [%s].", config.getString(ExecConstants.DEFAULT_TEMPORARY_WORKSPACE)))
-            .build(logger);
-      }
-      return resolvedSchema;
+      resolvedSchema = SchemaUtilites.resolveToMutableDrillSchema(
+          defaultSchema, sqlCreateTable.getSchemaPath());
+    }
+
+    if (sqlCreateTable.isTemporary()) {
+      return SchemaUtilites.resolveToValidTemporaryWorkspace(resolvedSchema, config);
     }
+
+    return resolvedSchema;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/drill/blob/5db557c6/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropTableHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropTableHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropTableHandler.java
index a9895db..c17ac20 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropTableHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropTableHandler.java
@@ -52,9 +52,6 @@ public class DropTableHandler extends DefaultSqlHandler {
    * @param sqlNode - SqlDropTable (SQL parse tree of drop table [if exists] query)
    * @return - Single row indicating drop succeeded or table is not found while IF EXISTS statement is used,
    * raise exception otherwise
-   * @throws ValidationException
-   * @throws RelConversionException
-   * @throws IOException
    */
   @Override
   public PhysicalPlan getPlan(SqlNode sqlNode) throws ValidationException, RelConversionException, IOException {
@@ -69,7 +66,7 @@ public class DropTableHandler extends DefaultSqlHandler {
     boolean isTemporaryTable = session.isTemporaryTable(temporarySchema, drillConfig, originalTableName);
 
     if (isTemporaryTable) {
-      session.removeTemporaryTable(temporarySchema, originalTableName);
+      session.removeTemporaryTable(temporarySchema, originalTableName, drillConfig);
     } else {
       AbstractSchema drillSchema = SchemaUtilites.resolveToMutableDrillSchema(defaultSchema, tableSchema);
       Table tableToDrop = SqlHandlerUtil.getTableFromSchema(drillSchema, originalTableName);

http://git-wip-us.apache.org/repos/asf/drill/blob/5db557c6/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 c1e577d..61429f2 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
@@ -259,6 +259,7 @@ public class UserSession implements Closeable {
 
   /**
    * Creates and adds session temporary location if absent using schema configuration.
+   * Before any actions, checks if passed table schema is valid default temporary workspace.
    * Generates temporary table name and stores it's original name as key
    * and generated name as value in  session temporary tables cache.
    * Original temporary name is converted to lower case to achieve case-insensitivity.
@@ -268,14 +269,15 @@ public class UserSession implements Closeable {
    *
    * @param schema table schema
    * @param tableName original table name
+   * @param config drill config
    * @return generated temporary table name
    * @throws IOException if error during session temporary location creation
    */
-  public String registerTemporaryTable(AbstractSchema schema, String tableName) throws IOException {
-      addTemporaryLocation((WorkspaceSchemaFactory.WorkspaceSchema) schema);
-      String temporaryTableName = new Path(sessionId, UUID.randomUUID().toString()).toUri().getPath();
-      String oldTemporaryTableName = temporaryTables.putIfAbsent(tableName.toLowerCase(), temporaryTableName);
-      return oldTemporaryTableName == null ? temporaryTableName : oldTemporaryTableName;
+  public String registerTemporaryTable(AbstractSchema schema, String tableName, DrillConfig config) throws IOException {
+    addTemporaryLocation(SchemaUtilites.resolveToValidTemporaryWorkspace(schema, config));
+    String temporaryTableName = new Path(sessionId, UUID.randomUUID().toString()).toUri().getPath();
+    String oldTemporaryTableName = temporaryTables.putIfAbsent(tableName.toLowerCase(), temporaryTableName);
+    return oldTemporaryTableName == null ? temporaryTableName : oldTemporaryTableName;
   }
 
   /**
@@ -305,7 +307,7 @@ public class UserSession implements Closeable {
    * @return true if temporary table exists in schema, false otherwise
    */
   public boolean isTemporaryTable(AbstractSchema drillSchema, DrillConfig config, String tableName) {
-    if (!SchemaUtilites.isTemporaryWorkspace(drillSchema.getFullSchemaName(), config)) {
+    if (drillSchema == null || !SchemaUtilites.isTemporaryWorkspace(drillSchema.getFullSchemaName(), config)) {
       return false;
     }
     String temporaryTableName = resolveTemporaryTableName(tableName);
@@ -321,15 +323,18 @@ public class UserSession implements Closeable {
   /**
    * Removes temporary table name from the list of session temporary tables.
    * Original temporary name is converted to lower case to achieve case-insensitivity.
+   * Before temporary table drop, checks if passed table schema is valid default temporary workspace.
    *
+   * @param schema table schema
    * @param tableName original table name
+   * @param config drill config
    */
-  public void removeTemporaryTable(AbstractSchema drillSchema, String tableName) {
+  public void removeTemporaryTable(AbstractSchema schema, String tableName, DrillConfig config) {
     String temporaryTable = resolveTemporaryTableName(tableName);
     if (temporaryTable == null) {
       return;
     }
-    SqlHandlerUtil.dropTableFromSchema(drillSchema, temporaryTable);
+    SqlHandlerUtil.dropTableFromSchema(SchemaUtilites.resolveToValidTemporaryWorkspace(schema, config), temporaryTable);
     temporaryTables.remove(tableName.toLowerCase());
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/5db557c6/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
index 77532e1..b4300e0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
@@ -20,7 +20,6 @@ package org.apache.drill.exec.server;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.StackTrace;
 import org.apache.drill.common.config.DrillConfig;
@@ -31,17 +30,13 @@ import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.coord.ClusterCoordinator.RegistrationHandle;
 import org.apache.drill.exec.coord.zk.ZKClusterCoordinator;
 import org.apache.drill.exec.exception.DrillbitStartupException;
-import org.apache.drill.exec.planner.sql.SchemaUtilites;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.server.options.OptionValue.OptionType;
 import org.apache.drill.exec.server.rest.WebServer;
 import org.apache.drill.exec.service.ServiceEngine;
-import org.apache.drill.exec.store.AbstractSchema;
-import org.apache.drill.exec.store.SchemaTreeProvider;
 import org.apache.drill.exec.store.StoragePluginRegistry;
-import org.apache.drill.exec.store.dfs.WorkspaceSchemaFactory;
 import org.apache.drill.exec.store.sys.store.provider.CachingPersistentStoreProvider;
 import org.apache.drill.exec.store.sys.PersistentStoreProvider;
 import org.apache.drill.exec.store.sys.PersistentStoreRegistry;
@@ -128,7 +123,6 @@ public class Drillbit implements AutoCloseable {
     storageRegistry.init();
     drillbitContext.getOptionManager().init();
     javaPropertiesToSystemOptions();
-    validateTemporaryWorkspace(manager.getContext());
     manager.getContext().getRemoteFunctionRegistry().init(context.getConfig(), storeProvider, coord);
     registrationHandle = coord.register(md);
     webServer.start();
@@ -221,21 +215,6 @@ public class Drillbit implements AutoCloseable {
   }
 
   /**
-   * Validates that temporary workspace indicated in configuration is
-   * mutable and file-based (instance of {@link WorkspaceSchemaFactory.WorkspaceSchema}).
-   *
-   * @param context drillbit context
-   * @throws Exception in case when temporary table schema is not mutable or
-   *                   not file-based (instance of WorkspaceSchemaFactory.WorkspaceSchema)
-   */
-  private void validateTemporaryWorkspace(DrillbitContext context) throws Exception {
-    try (SchemaTreeProvider schemaTreeProvider = new SchemaTreeProvider(context)) {
-      final SchemaPlus rootSchema = schemaTreeProvider.createRootSchema(context.getOptionManager());
-      SchemaUtilites.getTemporaryWorkspace(rootSchema, context.getConfig());
-    }
-  }
-
-  /**
    * Shutdown hook for Drillbit. Closes the drillbit, and reports on errors that
    * occur during closure, as well as the location the drillbit was started from.
    */

http://git-wip-us.apache.org/repos/asf/drill/blob/5db557c6/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
index 2d5f0c7..1335d9d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
@@ -84,7 +84,6 @@ public class BaseTestQuery extends ExecTest {
     {
       put(ExecConstants.SYS_STORE_PROVIDER_LOCAL_ENABLE_WRITE, "false");
       put(ExecConstants.HTTP_ENABLE, "false");
-      put(ExecConstants.DEFAULT_TEMPORARY_WORKSPACE, TEMP_SCHEMA);
     }
   };
 

http://git-wip-us.apache.org/repos/asf/drill/blob/5db557c6/exec/java-exec/src/test/java/org/apache/drill/TestDropTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestDropTable.java b/exec/java-exec/src/test/java/org/apache/drill/TestDropTable.java
index acbf2e7..4b6dd5f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestDropTable.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestDropTable.java
@@ -168,7 +168,7 @@ public class TestDropTable extends PlanTestBase {
     try {
       test("drop table dfs.`/tmp`");
     } catch (UserException e) {
-      Assert.assertTrue(e.getMessage().contains("PARSE ERROR"));
+      Assert.assertTrue(e.getMessage().contains("VALIDATION ERROR"));
       dropFailed = true;
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/5db557c6/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/TemporaryTablesAutomaticDropTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/TemporaryTablesAutomaticDropTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/TemporaryTablesAutomaticDropTest.java
index f5d45b0..df012be 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/TemporaryTablesAutomaticDropTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/TemporaryTablesAutomaticDropTest.java
@@ -22,6 +22,7 @@ import mockit.MockUp;
 import mockit.integration.junit4.JMockit;
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.util.TestUtilities;
 import org.junit.Before;
@@ -29,6 +30,7 @@ import org.junit.Test;
 import org.junit.runner.RunWith;
 
 import java.io.File;
+import java.util.Properties;
 import java.util.UUID;
 
 import static org.junit.Assert.assertFalse;
@@ -47,7 +49,9 @@ public class TemporaryTablesAutomaticDropTest extends BaseTestQuery {
         return UUID.nameUUIDFromBytes(session_id.getBytes());
       }
     };
-    updateTestCluster(1, DrillConfig.create(cloneDefaultTestConfigProperties()));
+    Properties testConfigurations = cloneDefaultTestConfigProperties();
+    testConfigurations.put(ExecConstants.DEFAULT_TEMPORARY_WORKSPACE, TEMP_SCHEMA);
+    updateTestCluster(1, DrillConfig.create(testConfigurations));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/drill/blob/5db557c6/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 b38af4a..8ff51be 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
@@ -24,6 +24,7 @@ import mockit.integration.junit4.JMockit;
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.UserRemoteException;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.StorageStrategy;
 import org.apache.drill.exec.store.dfs.FileSystemConfig;
@@ -43,6 +44,7 @@ import java.io.File;
 import java.io.FileFilter;
 import java.io.IOException;
 import java.util.List;
+import java.util.Properties;
 import java.util.UUID;
 
 import static org.hamcrest.CoreMatchers.containsString;
@@ -65,7 +67,9 @@ public class TestCTTAS extends BaseTestQuery {
   @BeforeClass
   public static void init() throws Exception {
     MockUp<UUID> uuidMockUp = mockRandomUUID(session_id);
-    updateTestCluster(1, DrillConfig.create(cloneDefaultTestConfigProperties()));
+    Properties testConfigurations = cloneDefaultTestConfigProperties();
+    testConfigurations.put(ExecConstants.DEFAULT_TEMPORARY_WORKSPACE, TEMP_SCHEMA);
+    updateTestCluster(1, DrillConfig.create(testConfigurations));
     uuidMockUp.tearDown();
 
     StoragePluginRegistry pluginRegistry = getDrillbitContext().getStorage();
@@ -90,7 +94,7 @@ public class TestCTTAS extends BaseTestQuery {
   @Test
   public void testSyntax() throws Exception {
     test("create TEMPORARY table temporary_keyword as select 1 from (values(1))");
-    test("create TEMPORARY table temporary_keyword_with_wk as select 1 from (values(1))", TEMP_SCHEMA);
+    test("create TEMPORARY table %s.temporary_keyword_with_wk as select 1 from (values(1))", TEMP_SCHEMA);
   }
 
   @Test
@@ -172,7 +176,8 @@ public class TestCTTAS extends BaseTestQuery {
       test("create TEMPORARY table %s.%s as select 'A' as c1 from (values(1))", temp2_schema, temporaryTableName);
     } catch (UserRemoteException e) {
       assertThat(e.getMessage(), containsString(String.format(
-          "VALIDATION ERROR: Temporary tables are not allowed to be created outside of default temporary workspace [%s].",
+          "VALIDATION ERROR: Temporary tables are not allowed to be created / dropped " +
+              "outside of default temporary workspace [%s].",
           TEMP_SCHEMA)));
       throw e;
     }