You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by me...@apache.org on 2015/04/11 06:47:08 UTC

[3/7] drill git commit: DRILL-2733: Make dfs_test.tmp schema location on local fs exclusive to test JVM fork.

DRILL-2733: Make dfs_test.tmp schema location on local fs exclusive to test JVM fork.


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

Branch: refs/heads/master
Commit: 451f500072e9c4bad4cbbc44b39e5766bf012182
Parents: 375f841
Author: vkorukanti <ve...@gmail.com>
Authored: Tue Apr 7 16:35:11 2015 -0700
Committer: Mehant Baid <me...@gmail.com>
Committed: Fri Apr 10 21:21:20 2015 -0700

----------------------------------------------------------------------
 .../apache/drill/exec/util/TestUtilities.java   |  89 +++++++++++++
 .../java/org/apache/drill/BaseTestQuery.java    |  31 ++++-
 .../org/apache/drill/TestExampleQueries.java    |  14 +-
 .../org/apache/drill/TestMergeFilterPlan.java   |   2 +-
 .../java/org/apache/drill/TestStarQueries.java  |   2 +-
 .../java/org/apache/drill/TestUnionAll.java     |  14 +-
 .../drill/exec/cache/TestWriteToDisk.java       |  15 ++-
 .../exec/compile/TestLargeFileCompilation.java  |   2 +-
 .../physical/impl/writer/TestParquetWriter.java | 127 ++++++++++---------
 .../exec/physical/impl/writer/TestWriter.java   |  84 +++++++-----
 .../security/TestCustomUserAuthenticator.java   |   2 +-
 .../apache/drill/jdbc/DrillConnectionImpl.java  |  33 +++++
 .../jdbc/DatabaseMetaDataGetColumnsTest.java    |  47 +++----
 .../apache/drill/jdbc/DrillResultSetTest.java   |   5 +-
 .../jdbc/test/Bug1735ConnectionCloseTest.java   |   4 +-
 ...39GetBooleanFailsSayingWrongTypeBugTest.java |   2 +-
 ...rill2461IntervalsBreakInfoSchemaBugTest.java |   4 +-
 ...2463GetNullsFailedWithAssertionsBugTest.java |   2 +-
 ...l2489CallsAfterCloseThrowExceptionsTest.java |  22 ++--
 .../org/apache/drill/jdbc/test/JdbcAssert.java  |   5 +-
 20 files changed, 343 insertions(+), 163 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/java-exec/src/main/java/org/apache/drill/exec/util/TestUtilities.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/TestUtilities.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/TestUtilities.java
new file mode 100644
index 0000000..a1fcc2a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/TestUtilities.java
@@ -0,0 +1,89 @@
+/**
+ * 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.util;
+
+import com.google.common.io.Files;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.dfs.FileSystemConfig;
+import org.apache.drill.exec.store.dfs.FileSystemPlugin;
+import org.apache.drill.exec.store.dfs.WorkspaceConfig;
+
+import java.io.File;
+
+/**
+ * This class contains utility methods to speed up tests. Some of the production code currently calls this method
+ * when the production code is executed as part of the test runs. That's the reason why this code has to be in
+ * production module.
+ */
+public class TestUtilities {
+  // Below two variable values are derived from
+  // <DRILL_SRC_HOME>/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json.
+  private static final String dfsPluginName = "dfs";
+  private static final String dfsTmpSchema = "tmp";
+
+  // Below two variable values are derived from
+  // <DRILL_SRC_HOME>/exec/java-exec/src/test/resources/bootstrap-storage-plugins.json.
+  private static final String dfsTestPluginName = "dfs_test";
+  private static final String dfsTestTmpSchema = "tmp";
+
+  /**
+   * Update the location of dfs_test.tmp location. Get the "dfs_test.tmp" workspace and update the location with an
+   * exclusive temp directory just for use in the current test jvm.
+   *
+   * @param pluginRegistry
+   * @return JVM exclusive temporary directory location.
+   */
+  public static String updateDfsTestTmpSchemaLocation(final StoragePluginRegistry pluginRegistry)
+      throws ExecutionSetupException {
+    final FileSystemPlugin plugin = (FileSystemPlugin) pluginRegistry.getPlugin(dfsTestPluginName);
+    final FileSystemConfig pluginConfig = (FileSystemConfig) plugin.getConfig();
+    final WorkspaceConfig tmpWSConfig = pluginConfig.workspaces.get(dfsTestTmpSchema);
+
+    final File tmpDir = Files.createTempDir();
+    tmpDir.deleteOnExit();
+    final WorkspaceConfig newTmpWSConfig = new WorkspaceConfig(tmpDir.getAbsolutePath(),
+        true, tmpWSConfig.getDefaultInputFormat());
+
+    pluginConfig.workspaces.remove(dfsTestTmpSchema);
+    pluginConfig.workspaces.put(dfsTestTmpSchema, newTmpWSConfig);
+
+    pluginRegistry.createOrUpdate(dfsTestPluginName, pluginConfig, true);
+
+    return tmpDir.getAbsolutePath();
+  }
+
+  /**
+   * Make the dfs.tmp schema immutable, so that tests writers don't use the dfs.tmp to create views.
+   * Schema "dfs.tmp" added as part of the default bootstrap plugins file that comes with drill-java-exec jar
+   */
+  public static void makeDfsTmpSchemaImmutable(final StoragePluginRegistry pluginRegistry) throws ExecutionSetupException {
+    final FileSystemPlugin dfsPlugin = (FileSystemPlugin) pluginRegistry.getPlugin(dfsPluginName);
+    final FileSystemConfig dfsPluginConfig = (FileSystemConfig) dfsPlugin.getConfig();
+    final WorkspaceConfig tmpWSConfig = dfsPluginConfig.workspaces.get(dfsTmpSchema);
+
+    final WorkspaceConfig newTmpWSConfig = new WorkspaceConfig(tmpWSConfig.getLocation(), false,
+        tmpWSConfig.getDefaultInputFormat());
+
+    dfsPluginConfig.workspaces.remove(dfsTmpSchema);
+    dfsPluginConfig.workspaces.put(dfsTmpSchema, newTmpWSConfig);
+
+    pluginRegistry.createOrUpdate(dfsPluginName, dfsPluginConfig, true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/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 725594a..0c2f0e5 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
@@ -25,11 +25,12 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.base.Preconditions;
+import com.google.common.io.Files;
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.exec.client.QuerySubmitter;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.TopLevelAllocator;
@@ -43,6 +44,11 @@ import org.apache.drill.exec.rpc.user.UserResultsListener;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.dfs.FileSystemConfig;
+import org.apache.drill.exec.store.dfs.FileSystemPlugin;
+import org.apache.drill.exec.store.dfs.WorkspaceConfig;
+import org.apache.drill.exec.util.TestUtilities;
 import org.apache.drill.exec.util.VectorUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -82,7 +88,6 @@ public class BaseTestQuery extends ExecTest {
   protected static Drillbit[] bits;
   protected static RemoteServiceSet serviceSet;
   protected static DrillConfig config;
-  protected static QuerySubmitter submitter = new QuerySubmitter();
   protected static BufferAllocator allocator;
 
   /**
@@ -92,6 +97,11 @@ public class BaseTestQuery extends ExecTest {
    */
   private static int drillbitCount = 1;
 
+  /**
+   * Location of the dfs_test.tmp schema on local filesystem.
+   */
+  private static String dfsTestTmpSchemaLocation;
+
   private int[] columnWidths = new int[] { 8 };
 
   @BeforeClass
@@ -130,6 +140,19 @@ public class BaseTestQuery extends ExecTest {
     return bits[0].getContext();
   }
 
+  protected static Properties cloneDefaultTestConfigProperties() {
+    final Properties props = new Properties();
+    for(String propName : TEST_CONFIGURATIONS.stringPropertyNames()) {
+      props.put(propName, TEST_CONFIGURATIONS.getProperty(propName));
+    }
+
+    return props;
+  }
+
+  protected static String getDfsTestTmpSchemaLocation() {
+    return dfsTestTmpSchemaLocation;
+  }
+
   private static void resetClientAndBit() throws Exception{
     closeClient();
     openClient();
@@ -149,6 +172,10 @@ public class BaseTestQuery extends ExecTest {
       bits[i].run();
     }
 
+    final StoragePluginRegistry pluginRegistry = getDrillbitContext().getStorage();
+    dfsTestTmpSchemaLocation = TestUtilities.updateDfsTestTmpSchemaLocation(pluginRegistry);
+    TestUtilities.makeDfsTmpSchemaImmutable(pluginRegistry);
+
     client = QueryTestUtil.createClient(config,  serviceSet, MAX_WIDTH_PER_NODE, null);
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java b/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
index 921dacc..ec1fb58 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
@@ -34,7 +34,7 @@ public class TestExampleQueries extends BaseTestQuery{
   @Test // see DRILL-2328
   public void testConcatOnNull() throws Exception {
     try {
-      test("use dfs.tmp");
+      test("use dfs_test.tmp");
       test("create view concatNull as (select * from cp.`customer.json` where customer_id < 5);");
 
       // Test Left Null
@@ -123,9 +123,9 @@ public class TestExampleQueries extends BaseTestQuery{
 
   @Test // see DRILL-985
   public void testViewFileName() throws Exception {
-    test("use dfs.tmp");
+    test("use dfs_test.tmp");
     test("create view nation_view_testexamplequeries as select * from cp.`tpch/nation.parquet`;");
-    test("select * from dfs.tmp.`nation_view_testexamplequeries.view.drill`");
+    test("select * from dfs_test.tmp.`nation_view_testexamplequeries.view.drill`");
     test("drop view nation_view_testexamplequeries");
   }
 
@@ -376,7 +376,7 @@ public class TestExampleQueries extends BaseTestQuery{
 
   @Test // DRILL-811
   public void testDRILL_811View() throws Exception {
-    test("use dfs.tmp");
+    test("use dfs_test.tmp");
     test("create view nation_view_testexamplequeries as select * from cp.`tpch/nation.parquet`;");
 
     test("select n.n_nationkey, n.n_name, n.n_regionkey from nation_view_testexamplequeries n where n.n_nationkey > 8 order by n.n_regionkey");
@@ -388,7 +388,7 @@ public class TestExampleQueries extends BaseTestQuery{
 
   @Test  // DRILL-811
   public void testDRILL_811ViewJoin() throws Exception {
-    test("use dfs.tmp");
+    test("use dfs_test.tmp");
     test("create view nation_view_testexamplequeries as select * from cp.`tpch/nation.parquet`;");
     test("create view region_view_testexamplequeries as select * from cp.`tpch/region.parquet`;");
 
@@ -404,7 +404,7 @@ public class TestExampleQueries extends BaseTestQuery{
 
   @Test  // DRILL-811
   public void testDRILL_811Json() throws Exception {
-    test("use dfs.tmp");
+    test("use dfs_test.tmp");
     test("create view region_view_testexamplequeries as select * from cp.`region.json`;");
     test("select sales_city, sales_region from region_view_testexamplequeries where region_id > 50 order by sales_country; ");
     test("drop view region_view_testexamplequeries ");
@@ -699,7 +699,7 @@ public class TestExampleQueries extends BaseTestQuery{
         "select cast(r_regionkey as BIGINT) BIGINT_col, cast(r_regionkey as DECIMAL) bigint_col \n" +
         "FROM cp.`tpch/region.parquet`;\n";
 
-    test("USE dfs.tmp");
+    test("USE dfs_test.tmp");
     test(creatTable);
 
     testBuilder()

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/java-exec/src/test/java/org/apache/drill/TestMergeFilterPlan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestMergeFilterPlan.java b/exec/java-exec/src/test/java/org/apache/drill/TestMergeFilterPlan.java
index 0b20b7a..2de4ea2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestMergeFilterPlan.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestMergeFilterPlan.java
@@ -59,7 +59,7 @@ public class TestMergeFilterPlan extends PlanTestBase {
     String expectedPattern2 = "Filter(condition=[AND(OR(=($0, 1), =($0, 2), =($0, 3)), LIKE($1, '%VP%'))])";
     String excludedPattern = "Filter(condition=[OR(=($0, 1), =($0, 2), =($0, 3))])";
 
-    test("use dfs.tmp");
+    test("use dfs_test.tmp");
 
     test(viewDDL);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/java-exec/src/test/java/org/apache/drill/TestStarQueries.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestStarQueries.java b/exec/java-exec/src/test/java/org/apache/drill/TestStarQueries.java
index 89cc496..effef9b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestStarQueries.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestStarQueries.java
@@ -233,7 +233,7 @@ public class TestStarQueries extends BaseTestQuery{
 
   @Test // DRILL-1293
   public void testStarView1() throws Exception {
-    test("use dfs.tmp");
+    test("use dfs_test.tmp");
     test("create view vt1 as select * from cp.`tpch/region.parquet` r, cp.`tpch/nation.parquet` n where r.r_regionkey = n.n_regionkey");
     test("select * from vt1");
     test("drop view vt1");

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java b/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java
index bcffe0c..11d83f9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestUnionAll.java
@@ -184,17 +184,17 @@ public class TestUnionAll extends BaseTestQuery{
 
   @Test
   public void testUnionAllViewExpandableStar() throws Exception {
-    test("use dfs.tmp");
+    test("use dfs_test.tmp");
     test("create view nation_view_testunionall as select n_name, n_nationkey from cp.`tpch/nation.parquet`;");
     test("create view region_view_testunionall as select r_name, r_regionkey from cp.`tpch/region.parquet`;");
 
-    String query1 = "(select * from dfs.tmp.`nation_view_testunionall`) " +
+    String query1 = "(select * from dfs_test.tmp.`nation_view_testunionall`) " +
                     "union all " +
-                    "(select * from dfs.tmp.`region_view_testunionall`) ";
+                    "(select * from dfs_test.tmp.`region_view_testunionall`) ";
 
     String query2 =  "(select r_name, r_regionkey from cp.`tpch/region.parquet`) " +
                      "union all " +
-                     "(select * from dfs.tmp.`nation_view_testunionall`)";
+                     "(select * from dfs_test.tmp.`nation_view_testunionall`)";
 
     try {
       testBuilder()
@@ -220,11 +220,11 @@ public class TestUnionAll extends BaseTestQuery{
 
   @Test(expected = UnsupportedRelOperatorException.class) // see DRILL-2002
   public void testUnionAllViewUnExpandableStar() throws Exception {
-    test("use dfs.tmp");
+    test("use dfs_test.tmp");
     test("create view nation_view_testunionall as select * from cp.`tpch/nation.parquet`;");
 
     try {
-      String query = "(select * from dfs.tmp.`nation_view_testunionall`) " +
+      String query = "(select * from dfs_test.tmp.`nation_view_testunionall`) " +
                      "union all (select * from cp.`tpch/region.parquet`)";
       test(query);
     } catch(Exception ex) {
@@ -237,7 +237,7 @@ public class TestUnionAll extends BaseTestQuery{
 
   @Test
   public void testDiffDataTypesAndModes() throws Exception {
-    test("use dfs.tmp");
+    test("use dfs_test.tmp");
     test("create view nation_view_testunionall as select n_name, n_nationkey from cp.`tpch/nation.parquet`;");
     test("create view region_view_testunionall as select r_name, r_regionkey from cp.`tpch/region.parquet`;");
 

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
index d2a324f..0f437e9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
@@ -17,13 +17,16 @@
  */
 package org.apache.drill.exec.cache;
 
+import java.io.File;
 import java.util.List;
 
+import com.google.common.io.Files;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
+import org.apache.drill.common.util.TestTools;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.record.MaterializedField;
@@ -43,11 +46,15 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.junit.Rule;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;
+import org.junit.rules.TestRule;
 
 public class TestWriteToDisk extends ExecTest {
+  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(90000); // 90secs
+
   @Test
   @SuppressWarnings("static-method")
   public void test() throws Exception {
@@ -98,11 +105,9 @@ public class TestWriteToDisk extends ExecTest {
         final VectorAccessibleSerializable newWrap = new VectorAccessibleSerializable(
             context.getAllocator());
         try (final FileSystem fs = FileSystem.get(conf)) {
-          final Path path = new Path("/tmp/drillSerializable");
-          if (fs.exists(path)) {
-            fs.delete(path, false);
-          }
-
+          final File tempDir = Files.createTempDir();
+          tempDir.deleteOnExit();
+          final Path path = new Path(tempDir.getAbsolutePath(), "drillSerializable");
           try (final FSDataOutputStream out = fs.create(path)) {
             wrap.writeToStream(out);
             out.close();

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestLargeFileCompilation.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestLargeFileCompilation.java b/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestLargeFileCompilation.java
index a2562d7..e63bdc0 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestLargeFileCompilation.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestLargeFileCompilation.java
@@ -26,7 +26,7 @@ import org.junit.Test;
 import org.junit.rules.TestRule;
 
 public class TestLargeFileCompilation extends BaseTestQuery {
-  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(120000);
+  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(150000); // 150secs
 
   private static final String LARGE_QUERY_GROUP_BY;
 

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
index eae65ec..325f770 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.writer;
 
+import java.io.File;
 import java.math.BigDecimal;
 
 import org.apache.drill.BaseTestQuery;
@@ -37,7 +38,7 @@ public class TestParquetWriter extends BaseTestQuery {
   @BeforeClass
   public static void initFs() throws Exception {
     Configuration conf = new Configuration();
-    conf.set("fs.name.default", "local");
+    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "local");
 
     fs = FileSystem.get(conf);
   }
@@ -254,18 +255,17 @@ public class TestParquetWriter extends BaseTestQuery {
     String query = "select r_regionkey, r_name, r_comment, r_regionkey +1 as key1 from cp.`tpch/region.parquet` order by r_name";
     String queryFromWriteOut = "select * from " + outputFile;
 
-    Path path = new Path("/tmp/" + outputFile);
-    if (fs.exists(path)) {
-      fs.delete(path, true);
+    try {
+      test("use dfs_test.tmp");
+      test(ctasStmt);
+      testBuilder()
+          .ordered()
+          .sqlQuery(queryFromWriteOut)
+          .sqlBaselineQuery(query)
+          .build().run();
+    } finally {
+      deleteTableIfExists(outputFile);
     }
-
-    test("use dfs.tmp");
-    test(ctasStmt);
-    testBuilder()
-        .ordered()
-        .sqlQuery(queryFromWriteOut)
-        .sqlBaselineQuery(query)
-        .build().run();
   }
 
   public void compareParquetReadersColumnar(String selection, String table) throws Exception {
@@ -383,90 +383,97 @@ public class TestParquetWriter extends BaseTestQuery {
   @Test
   public void testWriteDecimal() throws Exception {
     String outputTable = "decimal_test";
-    Path path = new Path("/tmp/" + outputTable);
-    if (fs.exists(path)) {
-      fs.delete(path, true);
-    }
-    String ctas = String.format("use dfs.tmp; " +
-        "create table %s as select " +
-        "cast('1.2' as decimal(38, 2)) col1, cast('1.2' as decimal(28, 2)) col2 " +
-        "from cp.`employee.json` limit 1", outputTable);
 
-    test(ctas);
+    try {
+      String ctas = String.format("use dfs_test.tmp; " +
+          "create table %s as select " +
+          "cast('1.2' as decimal(38, 2)) col1, cast('1.2' as decimal(28, 2)) col2 " +
+          "from cp.`employee.json` limit 1", outputTable);
 
-    BigDecimal result = new BigDecimal("1.20");
+      test(ctas);
 
-    testBuilder()
-        .unOrdered()
-        .sqlQuery(String.format("select col1, col2 from %s ", outputTable))
-        .baselineColumns("col1", "col2")
-        .baselineValues(result, result)
-        .go();
+      BigDecimal result = new BigDecimal("1.20");
+
+      testBuilder()
+          .unOrdered()
+          .sqlQuery(String.format("select col1, col2 from %s ", outputTable))
+          .baselineColumns("col1", "col2")
+          .baselineValues(result, result)
+          .go();
+    } finally {
+      deleteTableIfExists(outputTable);
+    }
   }
 
+
   @Test // see DRILL-2408
   public void testWriteEmptyFile() throws Exception {
     String outputFile = "testparquetwriter_test_write_empty_file";
 
-    Path path = new Path("/tmp/" + outputFile);
-    if (fs.exists(path)) {
-      fs.delete(path, true);
-    }
-
-//    test("ALTER SESSION SET `planner.add_producer_consumer` = false");
-    test("CREATE TABLE dfs.tmp.%s AS SELECT * FROM cp.`employee.json` WHERE 1=0", outputFile);
+    try {
+      Path path = new Path(getDfsTestTmpSchemaLocation(), outputFile);
+      //    test("ALTER SESSION SET `planner.add_producer_consumer` = false");
+      test("CREATE TABLE dfs_test.tmp.%s AS SELECT * FROM cp.`employee.json` WHERE 1=0", outputFile);
 
-    Assert.assertEquals(fs.listStatus(path).length, 0);
+      Assert.assertEquals(fs.listStatus(path).length, 0);
+    } finally {
+      deleteTableIfExists(outputFile);
+    }
   }
 
   @Test // see DRILL-2408
   public void testWriteEmptyFileAfterFlush() throws Exception {
     String outputFile = "testparquetwriter_test_write_empty_file_after_flush";
 
-    Path path = new Path("/tmp/" + outputFile);
-    if (fs.exists(path)) {
-      fs.delete(path, true);
-    }
-
     try {
       // this specific value will force a flush just after the final row is written
       // this will cause the creation of a new "empty" parquet file
       test("ALTER SESSION SET `store.parquet.block-size` = 19926");
 
       String query = "SELECT * FROM cp.`employee.json` LIMIT 100";
-      test("CREATE TABLE dfs.tmp.%s AS %s", outputFile, query);
+      test("CREATE TABLE dfs_test.tmp.%s AS %s", outputFile, query);
 
       // this query will fail if the "empty" file wasn't deleted
       testBuilder()
         .unOrdered()
-        .sqlQuery("SELECT * FROM dfs.tmp.%s", outputFile)
+        .sqlQuery("SELECT * FROM dfs_test.tmp.%s", outputFile)
         .sqlBaselineQuery(query)
         .go();
     } finally {
       // restore the session option
       test("ALTER SESSION SET `store.parquet.block-size` = %d", ExecConstants.PARQUET_BLOCK_SIZE_VALIDATOR.getDefault().num_val);
+      deleteTableIfExists(outputFile);
     }
   }
 
-  public void runTestAndValidate(String selection, String validationSelection, String inputTable, String outputFile) throws Exception {
-
-    Path path = new Path("/tmp/" + outputFile);
-    if (fs.exists(path)) {
-      fs.delete(path, true);
+  private static void deleteTableIfExists(String tableName) {
+    try {
+      Path path = new Path(getDfsTestTmpSchemaLocation(), tableName);
+      if (fs.exists(path)) {
+        fs.delete(path, true);
+      }
+    } catch (Exception e) {
+      // ignore exceptions.
     }
+  }
+
+  public void runTestAndValidate(String selection, String validationSelection, String inputTable, String outputFile) throws Exception {
+    try {
+      test("use dfs_test.tmp");
+  //    test("ALTER SESSION SET `planner.add_producer_consumer` = false");
+      String query = String.format("SELECT %s FROM %s", selection, inputTable);
+      String create = "CREATE TABLE " + outputFile + " AS " + query;
+      String validateQuery = String.format("SELECT %s FROM " + outputFile, validationSelection);
+      test(create);
 
-    test("use dfs.tmp");
-//    test("ALTER SESSION SET `planner.add_producer_consumer` = false");
-    String query = String.format("SELECT %s FROM %s", selection, inputTable);
-    String create = "CREATE TABLE " + outputFile + " AS " + query;
-    String validateQuery = String.format("SELECT %s FROM " + outputFile, validationSelection);
-    test(create);
+      testBuilder()
+          .unOrdered()
+          .sqlQuery(query)
+          .sqlBaselineQuery(validateQuery)
+          .go();
 
-    testBuilder()
-        .unOrdered()
-        .sqlQuery(query)
-        .sqlBaselineQuery(validateQuery)
-        .go();
+    } finally {
+      deleteTableIfExists(outputFile);
+    }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestWriter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestWriter.java
index 783abc7..5991046 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestWriter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestWriter.java
@@ -40,6 +40,7 @@ import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 
 public class TestWriter extends BaseTestQuery {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestWriter.class);
 
   static FileSystem fs;
   static String ALTER_SESSION = String.format("ALTER SESSION SET `%s` = 'csv'", ExecConstants.OUTPUT_FORMAT_OPTION);
@@ -47,7 +48,7 @@ public class TestWriter extends BaseTestQuery {
   @BeforeClass
   public static void initFs() throws Exception {
     Configuration conf = new Configuration();
-    conf.set("fs.name.default", "local");
+    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "local");
 
     fs = FileSystem.get(conf);
   }
@@ -93,78 +94,91 @@ public class TestWriter extends BaseTestQuery {
 
   @Test
   public void simpleCTAS() throws Exception {
+    final String tableName = "simplectas";
     runSQL("Use dfs_test.tmp");
     runSQL(ALTER_SESSION);
 
-    String testQuery = "CREATE TABLE simplectas AS SELECT * FROM cp.`employee.json`";
+    final String testQuery = String.format("CREATE TABLE %s AS SELECT * FROM cp.`employee.json`", tableName);
 
-    ctasHelper("/tmp/drilltest/simplectas", testQuery, 1155);
+    testCTASQueryHelper(tableName, testQuery, 1155);
   }
 
   @Test
   public void complex1CTAS() throws Exception {
+    final String tableName = "complex1ctas";
     runSQL("Use dfs_test.tmp");
     runSQL(ALTER_SESSION);
-    String testQuery = "CREATE TABLE complex1ctas AS SELECT first_name, last_name, position_id FROM cp.`employee.json`";
+    final String testQuery = String.format("CREATE TABLE %s AS SELECT first_name, last_name, " +
+        "position_id FROM cp.`employee.json`", tableName);
 
-    ctasHelper("/tmp/drilltest/complex1ctas", testQuery, 1155);
+    testCTASQueryHelper(tableName, testQuery, 1155);
   }
 
   @Test
   public void complex2CTAS() throws Exception {
+    final String tableName = "complex1ctas";
     runSQL("Use dfs_test.tmp");
     runSQL(ALTER_SESSION);
-    String testQuery = "CREATE TABLE complex2ctas AS SELECT CAST(`birth_date` as Timestamp) FROM cp.`employee.json` GROUP BY birth_date";
+    final String testQuery = String.format("CREATE TABLE %s AS SELECT CAST(`birth_date` as Timestamp) FROM " +
+        "cp.`employee.json` GROUP BY birth_date", tableName);
 
-    ctasHelper("/tmp/drilltest/complex2ctas", testQuery, 52);
+    testCTASQueryHelper(tableName, testQuery, 52);
   }
 
   @Test
   public void simpleCTASWithSchemaInTableName() throws Exception {
+    final String tableName = "/test/simplectas2";
     runSQL(ALTER_SESSION);
-    String testQuery = "CREATE TABLE dfs_test.tmp.`/test/simplectas2` AS SELECT * FROM cp.`employee.json`";
+    final String testQuery =
+        String.format("CREATE TABLE dfs_test.tmp.`%s` AS SELECT * FROM cp.`employee.json`",tableName);
 
-    ctasHelper("/tmp/drilltest/test/simplectas2", testQuery, 1155);
+    testCTASQueryHelper(tableName, testQuery, 1155);
   }
 
   @Test
   public void simpleParquetDecimal() throws Exception {
-//    String testQuery = "CREATE TABLE dfs_test.tmp.`simpleparquetdecimal` AS SELECT full_name FROM cp.`employee.json`";
-    String testQuery = "CREATE TABLE dfs_test.tmp.`simpleparquetdecimal` AS SELECT cast(salary as decimal(30,2)) * -1 as salary FROM cp.`employee.json`";
-//    String testQuery = "select * from dfs_test.tmp.`simpleparquetdecimal`";
-    ctasHelper("/tmp/drilltest/simpleparquetdecimal", testQuery, 1155);
+    final String tableName = "simpleparquetdecimal";
+    final String testQuery = String.format("CREATE TABLE dfs_test.tmp.`%s` AS SELECT cast(salary as " +
+        "decimal(30,2)) * -1 as salary FROM cp.`employee.json`", tableName);
+    testCTASQueryHelper(tableName, testQuery, 1155);
   }
 
-  private void ctasHelper(String tableDir, String testQuery, int expectedOutputCount) throws Exception {
-    Path tableLocation = new Path(tableDir);
-    if (fs.exists(tableLocation)) {
-      fs.delete(tableLocation, true);
-    }
+  private void testCTASQueryHelper(String tableName, String testQuery, int expectedOutputCount) throws Exception {
+    try {
+      List<QueryDataBatch> results = testSqlWithResults(testQuery);
 
-    List<QueryDataBatch> results = testSqlWithResults(testQuery);
+      RecordBatchLoader batchLoader = new RecordBatchLoader(getAllocator());
 
-    RecordBatchLoader batchLoader = new RecordBatchLoader(getAllocator());
+      int recordsWritten = 0;
+      for (QueryDataBatch batch : results) {
+        batchLoader.load(batch.getHeader().getDef(), batch.getData());
 
-    int recordsWritten = 0;
-    for (QueryDataBatch batch : results) {
-      batchLoader.load(batch.getHeader().getDef(), batch.getData());
+        if (batchLoader.getRecordCount() <= 0) {
+          continue;
+        }
 
-      if (batchLoader.getRecordCount() <= 0) {
-        continue;
-      }
+        BigIntVector recordWrittenV = (BigIntVector) batchLoader.getValueAccessorById(BigIntVector.class, 1).getValueVector();
 
-      BigIntVector recordWrittenV = (BigIntVector) batchLoader.getValueAccessorById(BigIntVector.class, 1).getValueVector();
+        for (int i = 0; i < batchLoader.getRecordCount(); i++) {
+          recordsWritten += recordWrittenV.getAccessor().get(i);
+        }
 
-      for (int i = 0; i < batchLoader.getRecordCount(); i++) {
-        recordsWritten += recordWrittenV.getAccessor().get(i);
+        batchLoader.clear();
+        batch.release();
       }
 
-      batchLoader.clear();
-      batch.release();
+      assertEquals(expectedOutputCount, recordsWritten);
+    } finally {
+      try {
+        Path path = new Path(getDfsTestTmpSchemaLocation(), tableName);
+        if (fs.exists(path)) {
+          fs.delete(path, true);
+        }
+      } catch (Exception e) {
+        // ignore exceptions.
+        logger.warn("Failed to delete the table [{}, {}] created as part of the test",
+            getDfsTestTmpSchemaLocation(), tableName);
+      }
     }
-
-//    assertTrue(fs.exists(tableLocation));
-    assertEquals(expectedOutputCount, recordsWritten);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java
index 06a2d1a..70d43b6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java
@@ -41,7 +41,7 @@ public class TestCustomUserAuthenticator extends BaseTestQuery {
   public static void setupCluster() {
     // Create a new DrillConfig which has user authentication enabled and authenticator set to
     // UserAuthenticatorTestImpl.
-    final Properties props = new Properties();
+    final Properties props = cloneDefaultTestConfigProperties();
     props.setProperty(ExecConstants.USER_AUTHENTICATION_ENABLED, "true");
     props.setProperty(ExecConstants.USER_AUTHENTICATOR_IMPL, UserAuthenticatorTestImpl.TYPE);
     final DrillConfig newConfig = DrillConfig.create(props);

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
index 170495e..b18cb75 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
@@ -23,6 +23,7 @@ import java.sql.SQLException;
 import java.util.Properties;
 import java.util.TimeZone;
 
+import com.google.common.io.Files;
 import net.hydromatic.avatica.AvaticaConnection;
 import net.hydromatic.avatica.AvaticaFactory;
 import net.hydromatic.avatica.Helper;
@@ -30,12 +31,19 @@ import net.hydromatic.avatica.Meta;
 import net.hydromatic.avatica.UnregisteredDriver;
 
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.TopLevelAllocator;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.dfs.FileSystemConfig;
+import org.apache.drill.exec.store.dfs.FileSystemPlugin;
+import org.apache.drill.exec.store.dfs.WorkspaceConfig;
+import org.apache.drill.exec.util.TestUtilities;
 
 /**
  * Implementation of JDBC connection in Drill.
@@ -96,6 +104,9 @@ abstract class DrillConnectionImpl extends AvaticaConnection implements DrillCon
           serviceSet = null;
           bit = null;
         }
+
+        makeTmpSchemaLocationsUnique(bit.getContext().getStorage(), info);
+
         this.client = new DrillClient(dConfig, set.getCoordinator());
         this.client.connect(null, info);
       } else {
@@ -200,4 +211,26 @@ abstract class DrillConnectionImpl extends AvaticaConnection implements DrillCon
     }
   }
 
+  /**
+   * Test only code to make JDBC tests run concurrently. If the property <i>drillJDBCUnitTests</i> is set to
+   * <i>true</i> in connection properties:
+   *   - Update dfs_test.tmp workspace location with a temp directory. This temp is for exclusive use for test jvm.
+   *   - Update dfs.tmp workspace to immutable, so that test writer don't try to create views in dfs.tmp
+   * @param pluginRegistry
+   */
+  private static void makeTmpSchemaLocationsUnique(StoragePluginRegistry pluginRegistry, Properties props) {
+    try {
+      if (props != null && "true".equalsIgnoreCase(props.getProperty("drillJDBCUnitTests"))) {
+        TestUtilities.updateDfsTestTmpSchemaLocation(pluginRegistry);
+        TestUtilities.makeDfsTmpSchemaImmutable(pluginRegistry);
+      }
+    } catch(Throwable e) {
+      // Reason for catching Throwable is to capture NoSuchMethodError etc which depend on certain classed to be
+      // present in classpath which may not be available when just using the standalone JDBC. This is unlikely to
+      // happen, but just a safeguard to avoid failing user applications.
+      logger.warn("Failed to update tmp schema locations. This step is purely for testing purpose. " +
+          "Shouldn't be seen in production code.");
+      // Ignore the error and go with defaults
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/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 a5bd35b..42e343f 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
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertThat;
 import static org.hamcrest.CoreMatchers.*;
 
 import org.apache.drill.jdbc.Driver;
+import org.apache.drill.jdbc.test.JdbcAssert;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
@@ -168,7 +169,7 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTest {
   public static void setUpConnectionAndMetadataToCheck() throws Exception {
 
     // Get JDBC connection to Drill:
-    connection = new Driver().connect( "jdbc:drill:zk=local", null );
+    connection = new Driver().connect( "jdbc:drill:zk=local", JdbcAssert.getDefaultProperties());
     dbMetadata = connection.getMetaData();
     Statement stmt = connection.createStatement();
 
@@ -205,7 +206,7 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTest {
     // Note: Assertions must be enabled (as they have been so far in tests).
 
     // Create temporary test-columns view:
-    util = stmt.executeQuery( "USE dfs.tmp" );
+    util = stmt.executeQuery( "USE dfs_test.tmp" );
     assert util.next();
     assert util.getBoolean( 1 )
         : "Error setting schema for test: " + util.getString( 2 );
@@ -248,31 +249,31 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTest {
 
     // Set up result rows for temporary test view and Hivetest columns:
 
-    mdrOptBOOLEAN        = setUpRow( "dfs.tmp", VIEW_NAME, "optBOOLEAN" );
+    mdrOptBOOLEAN        = setUpRow( "dfs_test.tmp", VIEW_NAME, "optBOOLEAN" );
 
-    mdrReqTINYINT        = setUpRow( "dfs.tmp", VIEW_NAME, "reqTINYINT" );
-    mdrOptSMALLINT       = setUpRow( "dfs.tmp", VIEW_NAME, "optSMALLINT" );
-    mdrReqINTEGER        = setUpRow( "dfs.tmp", VIEW_NAME, "reqINTEGER" );
-    mdrOptBIGINT         = setUpRow( "dfs.tmp", VIEW_NAME, "optBIGINT" );
+    mdrReqTINYINT        = setUpRow( "dfs_test.tmp", VIEW_NAME, "reqTINYINT" );
+    mdrOptSMALLINT       = setUpRow( "dfs_test.tmp", VIEW_NAME, "optSMALLINT" );
+    mdrReqINTEGER        = setUpRow( "dfs_test.tmp", VIEW_NAME, "reqINTEGER" );
+    mdrOptBIGINT         = setUpRow( "dfs_test.tmp", VIEW_NAME, "optBIGINT" );
 
-    mdrOptFLOAT          = setUpRow( "dfs.tmp", VIEW_NAME, "optFLOAT" );
-    mdrReqDOUBLE         = setUpRow( "dfs.tmp", VIEW_NAME, "reqDOUBLE" );
-    mdrOptREAL           = setUpRow( "dfs.tmp", VIEW_NAME, "optREAL" );
+    mdrOptFLOAT          = setUpRow( "dfs_test.tmp", VIEW_NAME, "optFLOAT" );
+    mdrReqDOUBLE         = setUpRow( "dfs_test.tmp", VIEW_NAME, "reqDOUBLE" );
+    mdrOptREAL           = setUpRow( "dfs_test.tmp", VIEW_NAME, "optREAL" );
 
-    mdrReqDECIMAL_5_3    = setUpRow( "dfs.tmp", VIEW_NAME, "reqDECIMAL_5_3" );
+    mdrReqDECIMAL_5_3    = setUpRow( "dfs_test.tmp", VIEW_NAME, "reqDECIMAL_5_3" );
 
-    mdrReqVARCHAR_10     = setUpRow( "dfs.tmp", VIEW_NAME, "reqVARCHAR_10" );
-    mdrOptVARCHAR        = setUpRow( "dfs.tmp", VIEW_NAME, "optVARCHAR" );
-    mdrReqCHAR_5         = setUpRow( "dfs.tmp", VIEW_NAME, "reqCHAR_5" );
-    mdrOptVARBINARY_16   = setUpRow( "dfs.tmp", VIEW_NAME, "optVARBINARY_16" );
-    mdrOptBINARY_1048576 = setUpRow( "dfs.tmp", VIEW_NAME, "optBINARY_1048576" );
+    mdrReqVARCHAR_10     = setUpRow( "dfs_test.tmp", VIEW_NAME, "reqVARCHAR_10" );
+    mdrOptVARCHAR        = setUpRow( "dfs_test.tmp", VIEW_NAME, "optVARCHAR" );
+    mdrReqCHAR_5         = setUpRow( "dfs_test.tmp", VIEW_NAME, "reqCHAR_5" );
+    mdrOptVARBINARY_16   = setUpRow( "dfs_test.tmp", VIEW_NAME, "optVARBINARY_16" );
+    mdrOptBINARY_1048576 = setUpRow( "dfs_test.tmp", VIEW_NAME, "optBINARY_1048576" );
 
-    mdrReqDATE           = setUpRow( "dfs.tmp", VIEW_NAME, "reqDATE" );
-    mdrOptTIME           = setUpRow( "dfs.tmp", VIEW_NAME, "optTIME" );
-    mdrOptTIME_7         = setUpRow( "dfs.tmp", VIEW_NAME, "optTIME_7" );
-    mdrOptTIMESTAMP      = setUpRow( "dfs.tmp", VIEW_NAME, "optTIMESTAMP" );
-    mdrOptINTERVAL_H_S3  = setUpRow( "dfs.tmp", VIEW_NAME, "optINTERVAL_H_S3" );
-    mdrOptINTERVAL_Y4    = setUpRow( "dfs.tmp", VIEW_NAME, "optINTERVAL_Y4" );
+    mdrReqDATE           = setUpRow( "dfs_test.tmp", VIEW_NAME, "reqDATE" );
+    mdrOptTIME           = setUpRow( "dfs_test.tmp", VIEW_NAME, "optTIME" );
+    mdrOptTIME_7         = setUpRow( "dfs_test.tmp", VIEW_NAME, "optTIME_7" );
+    mdrOptTIMESTAMP      = setUpRow( "dfs_test.tmp", VIEW_NAME, "optTIMESTAMP" );
+    mdrOptINTERVAL_H_S3  = setUpRow( "dfs_test.tmp", VIEW_NAME, "optINTERVAL_H_S3" );
+    mdrOptINTERVAL_Y4    = setUpRow( "dfs_test.tmp", VIEW_NAME, "optINTERVAL_Y4" );
 
     /* TODO(start): Uncomment this block once we have a test plugin which supports all the needed types.
     mdrReqARRAY   = setUpRow( "hive_test.default", "infoschematest", "listtype" );
@@ -380,7 +381,7 @@ public class DatabaseMetaDataGetColumnsTest extends JdbcTest {
 
   @Test
   public void test_TABLE_SCHEM_hasRightValue_optBOOLEAN() throws SQLException {
-    assertThat( mdrOptBOOLEAN.getString( "TABLE_SCHEM" ), equalTo( "dfs.tmp" ) );
+    assertThat( mdrOptBOOLEAN.getString( "TABLE_SCHEM" ), equalTo( "dfs_test.tmp" ) );
   }
 
   // Not bothering with other _local_view_ test columns for TABLE_SCHEM.

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/jdbc/src/test/java/org/apache/drill/jdbc/DrillResultSetTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/DrillResultSetTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/DrillResultSetTest.java
index de19615..64be408 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/DrillResultSetTest.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/DrillResultSetTest.java
@@ -39,6 +39,7 @@ import org.apache.drill.common.logical.LogicalPlan;
 import org.apache.drill.common.logical.data.LogicalOperator;
 import org.apache.drill.common.util.Hook;
 import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.jdbc.test.JdbcAssert;
 import org.apache.drill.test.DrillTest;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -84,7 +85,7 @@ public class DrillResultSetTest extends DrillTest {
   public void test_next_blocksFurtherAccessAfterEnd()
       throws SQLException
   {
-    Connection connection = new Driver().connect( "jdbc:drill:zk=local", null );
+    Connection connection = new Driver().connect( "jdbc:drill:zk=local", JdbcAssert.getDefaultProperties() );
     Statement statement = connection.createStatement();
     ResultSet resultSet =
         statement.executeQuery( "SELECT 1 AS x \n" +
@@ -124,7 +125,7 @@ public class DrillResultSetTest extends DrillTest {
   public void test_next_blocksFurtherAccessWhenNoRows()
     throws Exception
   {
-    Connection connection = new Driver().connect( "jdbc:drill:zk=local", null );
+    Connection connection = new Driver().connect( "jdbc:drill:zk=local", JdbcAssert.getDefaultProperties() );
     Statement statement = connection.createStatement();
     ResultSet resultSet =
         statement.executeQuery( "SELECT 'Hi' AS x \n" +

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Bug1735ConnectionCloseTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Bug1735ConnectionCloseTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Bug1735ConnectionCloseTest.java
index 32240d6..78a8af2 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Bug1735ConnectionCloseTest.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Bug1735ConnectionCloseTest.java
@@ -77,7 +77,7 @@ public class Bug1735ConnectionCloseTest extends JdbcTestQueryBase {
     for ( int i = 1; i <= SMALL_ITERATION_COUNT; i++ ) {
       logger.info( "iteration " + i + ":" );
       System.out.println( "iteration " + i + ":" );
-      Connection connection = new Driver().connect( "jdbc:drill:zk=local", null );
+      Connection connection = new Driver().connect( "jdbc:drill:zk=local", JdbcAssert.getDefaultProperties() );
       connection.close();
     }
   }
@@ -94,7 +94,7 @@ public class Bug1735ConnectionCloseTest extends JdbcTestQueryBase {
     for ( int i = 1; i <= LARGE_ITERATION_COUNT; i++ ) {
       logger.info( "iteration " + i + ":" );
       System.out.println( "iteration " + i + ":" );
-      Connection connection = new Driver().connect( "jdbc:drill:zk=local", null );
+      Connection connection = new Driver().connect( "jdbc:drill:zk=local", JdbcAssert.getDefaultProperties() );
       connection.close();
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2439GetBooleanFailsSayingWrongTypeBugTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2439GetBooleanFailsSayingWrongTypeBugTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2439GetBooleanFailsSayingWrongTypeBugTest.java
index 69f194a..bd993fd 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2439GetBooleanFailsSayingWrongTypeBugTest.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2439GetBooleanFailsSayingWrongTypeBugTest.java
@@ -41,7 +41,7 @@ public class Drill2439GetBooleanFailsSayingWrongTypeBugTest extends JdbcTest {
 
   @BeforeClass
   public static void setUpConnection() throws SQLException {
-    connection = new Driver().connect( "jdbc:drill:zk=local", null );
+    connection = new Driver().connect( "jdbc:drill:zk=local", JdbcAssert.getDefaultProperties() );
     statement = connection.createStatement();
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2461IntervalsBreakInfoSchemaBugTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2461IntervalsBreakInfoSchemaBugTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2461IntervalsBreakInfoSchemaBugTest.java
index fce7923..d9ac3c7 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2461IntervalsBreakInfoSchemaBugTest.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2461IntervalsBreakInfoSchemaBugTest.java
@@ -62,10 +62,10 @@ public class Drill2461IntervalsBreakInfoSchemaBugTest extends JdbcTest {
     ResultSet util;
 
     // Create a view using an INTERVAL type:
-    util = stmt.executeQuery( "USE dfs.tmp" );
+    util = stmt.executeQuery( "USE dfs_test.tmp" );
     assert util.next();
     assert util.getBoolean( 1 )
-        : "Error setting schema to dfs.tmp: " + util.getString( 2 );
+        : "Error setting schema to dfs_test.tmp: " + util.getString( 2 );
     util = stmt.executeQuery(
         "CREATE OR REPLACE VIEW " + VIEW_NAME + " AS "
       + "\n  SELECT CAST( NULL AS INTERVAL HOUR(4) TO MINUTE ) AS optINTERVAL_HM "

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2463GetNullsFailedWithAssertionsBugTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2463GetNullsFailedWithAssertionsBugTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2463GetNullsFailedWithAssertionsBugTest.java
index 536a020..1d1361a 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2463GetNullsFailedWithAssertionsBugTest.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/Drill2463GetNullsFailedWithAssertionsBugTest.java
@@ -41,7 +41,7 @@ public class Drill2463GetNullsFailedWithAssertionsBugTest extends JdbcTest {
 
   @BeforeClass
   public static void setUpConnection() throws SQLException {
-    connection = new Driver().connect( "jdbc:drill:zk=local", null );
+    connection = new Driver().connect( "jdbc:drill:zk=local", JdbcAssert.getDefaultProperties() );
     statement = connection.createStatement();
 
     boolean assertionsEnabled = false;

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/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 9446d5f..3282945 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
@@ -77,7 +77,7 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTest {
 
   @BeforeClass
   public static void setUpConnection() throws Exception {
-    final Connection connection = new Driver().connect( "jdbc:drill:zk=local", null );
+    final Connection connection = new Driver().connect( "jdbc:drill:zk=local", JdbcAssert.getDefaultProperties() );
     final Statement stmt = connection.createStatement();
     final ResultSet result =
         stmt.executeQuery( "SELECT * FROM INFORMATION_SCHEMA.CATALOGS" );
@@ -422,7 +422,7 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTest {
   @Ignore( "until DRILL-2489 addressed" )
   @Test( expected = AlreadyClosedSqlException.class )
   public void testClosedStatement_addBatch_throws() throws SQLException {
-    closedStatement.addBatch( "USE dfs.tmp" );
+    closedStatement.addBatch( "USE dfs_test.tmp" );
   }
 
   @Ignore( "until DRILL-2489 addressed" )
@@ -446,24 +446,24 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTest {
   @Ignore( "until DRILL-2489 addressed" )
   @Test( expected = AlreadyClosedSqlException.class )
   public void testClosedStatement_execute1_throws() throws SQLException {
-    closedStatement.execute( "USE dfs.tmp", Statement.RETURN_GENERATED_KEYS );
+    closedStatement.execute( "USE dfs_test.tmp", Statement.RETURN_GENERATED_KEYS );
   }
 
   @Test( expected = AlreadyClosedSqlException.class )
   public void testClosedStatement_execute2_throws() throws SQLException {
-    closedStatement.execute( "USE dfs.tmp" );
+    closedStatement.execute( "USE dfs_test.tmp" );
   }
 
   @Ignore( "until DRILL-2489 addressed" )
   @Test( expected = AlreadyClosedSqlException.class )
   public void testClosedStatement_execute3_throws() throws SQLException {
-    closedStatement.execute( "USE dfs.tmp", (int[]) null );
+    closedStatement.execute( "USE dfs_test.tmp", (int[]) null );
   }
 
   @Ignore( "until DRILL-2489 addressed" )
   @Test( expected = AlreadyClosedSqlException.class )
   public void testClosedStatement_execute4_throws() throws SQLException {
-    closedStatement.execute( "USE dfs.tmp", (String[]) null );
+    closedStatement.execute( "USE dfs_test.tmp", (String[]) null );
   }
 
   @Ignore( "until DRILL-2489 addressed" )
@@ -474,28 +474,28 @@ public class Drill2489CallsAfterCloseThrowExceptionsTest extends JdbcTest {
 
   @Test( expected = AlreadyClosedSqlException.class )
   public void testClosedStatement_executeQuery_throws() throws SQLException {
-    closedStatement.executeQuery( "USE dfs.tmp" );
+    closedStatement.executeQuery( "USE dfs_test.tmp" );
   }
 
   @Ignore( "until DRILL-2489 addressed" )
   @Test( expected = AlreadyClosedSqlException.class )
   public void testClosedStatement_executeUpdate1_throws() throws SQLException {
-    closedStatement.executeUpdate( "USE dfs.tmp", Statement.RETURN_GENERATED_KEYS );
+    closedStatement.executeUpdate( "USE dfs_test.tmp", Statement.RETURN_GENERATED_KEYS );
   }
 
   @Test( expected = AlreadyClosedSqlException.class )
   public void testClosedStatement_executeUpdate2_throws() throws SQLException {
-    closedStatement.executeUpdate( "USE dfs.tmp", (int[]) null );
+    closedStatement.executeUpdate( "USE dfs_test.tmp", (int[]) null );
   }
 
   @Test( expected = AlreadyClosedSqlException.class )
   public void testClosedStatement_executeUpdate3_throws() throws SQLException {
-    closedStatement.executeUpdate( "USE dfs.tmp", (String[]) null );
+    closedStatement.executeUpdate( "USE dfs_test.tmp", (String[]) null );
   }
 
   @Test( expected = AlreadyClosedSqlException.class )
   public void testClosedStatement_executeUpdate4_throws() throws SQLException {
-    closedStatement.executeUpdate( "USE dfs.tmp" );
+    closedStatement.executeUpdate( "USE dfs_test.tmp" );
   }
 
   @Ignore( "until DRILL-2489 addressed" )

http://git-wip-us.apache.org/repos/asf/drill/blob/451f5000/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/JdbcAssert.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/JdbcAssert.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/JdbcAssert.java
index 84e05e6..4979dff 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/JdbcAssert.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/JdbcAssert.java
@@ -60,10 +60,13 @@ public class JdbcAssert {
 
   /**
    * Returns default bag of properties that is passed to JDBC connection.
-   * By default, includes an option to turn off the web server.
+   * By default, includes options to:
+   *   - turn off the web server
+   *   - indicate DrillConnectionImpl to set up dfs_test.tmp schema location to an exclusive dir just for this test jvm
    */
   public static Properties getDefaultProperties() {
     final Properties properties = new Properties();
+    properties.setProperty("drillJDBCUnitTests", "true");
     properties.setProperty(ExecConstants.HTTP_ENABLE, "false");
     return properties;
   }