You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by am...@apache.org on 2017/09/03 05:52:36 UTC

[1/3] drill git commit: DRILL-5751: Fix unit tests to use local file system even if it is not set by default

Repository: drill
Updated Branches:
  refs/heads/master d1a6134b4 -> 406dd2a58


DRILL-5751: Fix unit tests to use local file system even if it is not set by default

DRILL-5751: Changes after code review.

close apache/drill#927


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

Branch: refs/heads/master
Commit: 75c35135da960d0fb8b995d0cc5c048a98d4954f
Parents: d1a6134
Author: Arina Ielchiieva <ar...@gmail.com>
Authored: Thu Aug 24 19:49:52 2017 +0300
Committer: Aman Sinha <as...@maprtech.com>
Committed: Sat Sep 2 13:37:43 2017 -0700

----------------------------------------------------------------------
 .../sql/handlers/CreateFunctionHandler.java     | 17 +++-
 .../java/org/apache/drill/BaseTestQuery.java    |  8 +-
 .../org/apache/drill/TestDynamicUDFSupport.java | 52 +++++++----
 .../java/org/apache/drill/exec/ExecTest.java    | 14 +++
 .../apache/drill/exec/TestRepeatedReaders.java  |  8 +-
 .../drill/exec/cache/TestWriteToDisk.java       | 11 +--
 .../physical/impl/writer/TestParquetWriter.java |  6 +-
 .../writer/TestParquetWriterEmptyFiles.java     |  9 +-
 .../exec/physical/impl/writer/TestWriter.java   |  8 +-
 .../exec/planner/TestDirectoryExplorerUDFs.java | 79 +++++++----------
 .../user/TemporaryTablesAutomaticDropTest.java  | 31 ++++---
 .../org/apache/drill/exec/sql/TestCTAS.java     |  4 +-
 .../org/apache/drill/exec/sql/TestCTTAS.java    | 45 +++++-----
 .../drill/exec/store/StorageStrategyTest.java   | 93 ++++++++------------
 .../parquet/TestParquetFilterPushDown.java      | 16 ++--
 .../exec/store/parquet/TestParquetScan.java     |  8 +-
 .../drill/exec/util/FileSystemUtilTestBase.java | 12 +--
 .../org/apache/drill/test/ClusterFixture.java   |  6 +-
 .../src/test/resources/drill-module.conf        |  2 +
 19 files changed, 206 insertions(+), 223 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateFunctionHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateFunctionHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateFunctionHandler.java
index 0902fb7..526a741 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateFunctionHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateFunctionHandler.java
@@ -255,8 +255,8 @@ public class CreateFunctionHandler extends DefaultSqlHandler {
      * @throws IOException in case of binary or source absence or problems during copying jars
      */
     void initRemoteBackup() throws IOException {
-      fs.getFileStatus(stagingBinary);
-      fs.getFileStatus(stagingSource);
+      checkPathExistence(stagingBinary);
+      checkPathExistence(stagingSource);
       fs.mkdirs(remoteTmpDir);
       FileUtil.copy(fs, stagingBinary, fs, tmpRemoteBinary, false, true, fs.getConf());
       FileUtil.copy(fs, stagingSource, fs, tmpRemoteSource, false, true, fs.getConf());
@@ -316,6 +316,19 @@ public class CreateFunctionHandler extends DefaultSqlHandler {
     }
 
     /**
+     * Checks if passed path exists on predefined file system.
+     *
+     * @param path path to be checked
+     * @throws IOException if path does not exist
+     */
+    private void checkPathExistence(Path path) throws IOException {
+      if (!fs.exists(path)) {
+        throw new IOException(String.format("File %s does not exist on file system %s",
+            path.toUri().getPath(), fs.getUri()));
+      }
+    }
+
+    /**
      * Deletes quietly file or directory, in case of errors, logs warning and proceeds.
      *
      * @param path path to file or directory

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/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 7e8bdab..4d49c7b 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
@@ -59,7 +59,6 @@ import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.util.TestUtilities;
 import org.apache.drill.exec.util.VectorUtil;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -83,7 +82,8 @@ import org.apache.drill.exec.vector.ValueVector;
 public class BaseTestQuery extends ExecTest {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseTestQuery.class);
 
-  public static final String TEMP_SCHEMA = "dfs_test.tmp";
+  public static final String TEST_SCHEMA = "dfs_test";
+  public static final String TEMP_SCHEMA = TEST_SCHEMA + ".tmp";
 
   private static final int MAX_WIDTH_PER_NODE = 2;
 
@@ -138,9 +138,7 @@ public class BaseTestQuery extends ExecTest {
     // turns on the verbose errors in tests
     // sever side stacktraces are added to the message before sending back to the client
     test("ALTER SESSION SET `exec.errors.verbose` = true");
-    Configuration conf = new Configuration();
-    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
-    fs = FileSystem.get(conf);
+    fs = getLocalFileSystem();
   }
 
   protected static void updateTestCluster(int newDrillbitCount, DrillConfig newConfig) {

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/exec/java-exec/src/test/java/org/apache/drill/TestDynamicUDFSupport.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestDynamicUDFSupport.java b/exec/java-exec/src/test/java/org/apache/drill/TestDynamicUDFSupport.java
index 25c01b8..7f5b116 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestDynamicUDFSupport.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestDynamicUDFSupport.java
@@ -23,6 +23,7 @@ import org.apache.drill.common.config.CommonConstants;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.common.util.TestTools;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.VersionMismatchException;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.expr.fn.registry.LocalFunctionRegistry;
@@ -35,6 +36,7 @@ import org.apache.drill.exec.util.JarUtil;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
@@ -43,7 +45,6 @@ import org.mockito.invocation.InvocationOnMock;
 import org.mockito.runners.MockitoJUnitRunner;
 import org.mockito.stubbing.Answer;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.List;
 import java.util.Properties;
@@ -67,18 +68,26 @@ import static org.mockito.Mockito.verify;
 @RunWith(MockitoJUnitRunner.class)
 public class TestDynamicUDFSupport extends BaseTestQuery {
 
-  private static final File jars = new File(TestTools.getWorkingPath() + "/src/test/resources/jars");
+  private static final Path jars = new Path(TestTools.getWorkingPath(), "src/test/resources/jars");
   private static final String default_binary_name = "DrillUDF-1.0.jar";
   private static final String default_source_name = JarUtil.getSourceName(default_binary_name);
 
   @Rule
   public final TemporaryFolder base = new TemporaryFolder();
 
+  private static FileSystem localFileSystem;
+
+  @BeforeClass
+  public static void init() throws IOException {
+    localFileSystem = getLocalFileSystem();
+  }
+
   @Before
   public void setup() {
     Properties overrideProps = new Properties();
-    overrideProps.setProperty("drill.exec.udf.directory.root", base.getRoot().getPath());
-    overrideProps.setProperty("drill.tmp-dir", base.getRoot().getPath());
+    overrideProps.setProperty(ExecConstants.UDF_DIRECTORY_ROOT, base.getRoot().getPath());
+    overrideProps.setProperty(ExecConstants.DRILL_TMP_DIR, base.getRoot().getPath());
+    overrideProps.setProperty(ExecConstants.UDF_DIRECTORY_FS, FileSystem.DEFAULT_FS);
     updateTestCluster(1, DrillConfig.create(overrideProps));
   }
 
@@ -120,8 +129,10 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
   @Test
   public void testAbsentBinaryInStaging() throws Exception {
     Path staging = getDrillbitContext().getRemoteFunctionRegistry().getStagingArea();
+    FileSystem fs = getDrillbitContext().getRemoteFunctionRegistry().getFs();
 
-    String summary = String.format("File %s does not exist", new Path(staging, default_binary_name).toUri().getPath());
+    String summary = String.format("File %s does not exist on file system %s",
+        new Path(staging, default_binary_name).toUri().getPath(), fs.getUri());
 
     testBuilder()
         .sqlQuery("create function using jar '%s'", default_binary_name)
@@ -134,10 +145,11 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
   @Test
   public void testAbsentSourceInStaging() throws Exception {
     Path staging = getDrillbitContext().getRemoteFunctionRegistry().getStagingArea();
-    copyJar(getDrillbitContext().getRemoteFunctionRegistry().getFs(), new Path(jars.toURI()),
-        staging, default_binary_name);
+    FileSystem fs = getDrillbitContext().getRemoteFunctionRegistry().getFs();
+    copyJar(fs, jars, staging, default_binary_name);
 
-    String summary = String.format("File %s does not exist", new Path(staging, default_source_name).toUri().getPath());
+    String summary = String.format("File %s does not exist on file system %s",
+        new Path(staging, default_source_name).toUri().getPath(), fs.getUri());
 
     testBuilder()
         .sqlQuery("create function using jar '%s'", default_binary_name)
@@ -432,10 +444,11 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
 
     Path localUdfDirPath = Deencapsulation.getField(
         getDrillbitContext().getFunctionImplementationRegistry(), "localUdfDir");
-    File localUdfDir = new File(localUdfDirPath.toUri().getPath());
 
-    assertTrue("Binary should exist in local udf directory", new File(localUdfDir, default_binary_name).exists());
-    assertTrue("Source should exist in local udf directory", new File(localUdfDir, default_source_name).exists());
+    assertTrue("Binary should exist in local udf directory",
+        localFileSystem.exists(new Path(localUdfDirPath, default_binary_name)));
+    assertTrue("Source should exist in local udf directory",
+        localFileSystem.exists(new Path(localUdfDirPath, default_source_name)));
   }
 
   @Test
@@ -498,10 +511,11 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
 
     Path localUdfDirPath = Deencapsulation.getField(
         getDrillbitContext().getFunctionImplementationRegistry(), "localUdfDir");
-    File localUdfDir = new File(localUdfDirPath.toUri().getPath());
 
-    assertTrue("Binary should exist in local udf directory", new File(localUdfDir, default_binary_name).exists());
-    assertTrue("Source should exist in local udf directory", new File(localUdfDir, default_source_name).exists());
+    assertTrue("Binary should exist in local udf directory",
+        localFileSystem.exists(new Path(localUdfDirPath, default_binary_name)));
+    assertTrue("Source should exist in local udf directory",
+        localFileSystem.exists(new Path(localUdfDirPath, default_source_name)));
 
     String summary = "The following UDFs in jar %s have been unregistered:\n" +
         "[custom_lower(VARCHAR-REQUIRED)]";
@@ -530,9 +544,9 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
         fs.exists(new Path(remoteFunctionRegistry.getRegistryArea(), default_source_name)));
 
     assertFalse("Binary should not be present in local udf directory",
-        new File(localUdfDir, default_binary_name).exists());
+        localFileSystem.exists(new Path(localUdfDirPath, default_binary_name)));
     assertFalse("Source should not be present in local udf directory",
-        new File(localUdfDir, default_source_name).exists());
+        localFileSystem.exists(new Path(localUdfDirPath, default_source_name)));
   }
 
   @Test
@@ -549,7 +563,7 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
 
     Thread.sleep(1000);
 
-    Path src = new Path(jars.toURI().getPath(), "v2");
+    Path src = new Path(jars, "v2");
     copyJarsToStagingArea(src, default_binary_name, default_source_name);
     test("create function using jar '%s'", default_binary_name);
     testBuilder()
@@ -887,11 +901,11 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
   }
 
   private void copyDefaultJarsToStagingArea() throws IOException {
-    copyJarsToStagingArea(new Path(jars.toURI()), default_binary_name, default_source_name);
+    copyJarsToStagingArea(jars, default_binary_name, default_source_name);
   }
 
   private void copyJarsToStagingArea(String binaryName, String sourceName) throws IOException  {
-    copyJarsToStagingArea(new Path(jars.toURI()), binaryName, sourceName);
+    copyJarsToStagingArea(jars, binaryName, sourceName);
   }
 
   private void copyJarsToStagingArea(Path src, String binaryName, String sourceName) throws IOException {

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
index 7dd7680..f96e3bd 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/ExecTest.java
@@ -39,10 +39,13 @@ import org.apache.drill.exec.server.options.SystemOptionManager;
 import org.apache.drill.exec.store.sys.store.provider.LocalPersistentStoreProvider;
 import org.apache.drill.exec.util.GuavaPatcher;
 import org.apache.drill.test.DrillTest;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.junit.After;
 import org.junit.BeforeClass;
 
 import java.io.File;
+import java.io.IOException;
 
 
 public class ExecTest extends DrillTest {
@@ -70,6 +73,17 @@ public class ExecTest extends DrillTest {
   }
 
   /**
+   * Creates instance of local file system.
+   *
+   * @return local file system
+   */
+  public static FileSystem getLocalFileSystem() throws IOException {
+    Configuration configuration = new Configuration();
+    configuration.set(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
+    return FileSystem.get(configuration);
+  }
+
+  /**
    * Create a temp directory to store the given <i>dirName</i>.
    * Directory will be deleted on exit.
    * @param dirName directory name

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/exec/java-exec/src/test/java/org/apache/drill/exec/TestRepeatedReaders.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestRepeatedReaders.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestRepeatedReaders.java
index be6fe79..df7b305 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/TestRepeatedReaders.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestRepeatedReaders.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -19,7 +19,6 @@
 package org.apache.drill.exec;
 
 import org.apache.drill.BaseTestQuery;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.BeforeClass;
@@ -31,10 +30,7 @@ public class TestRepeatedReaders extends BaseTestQuery {
 
   @BeforeClass
   public static void initFs() throws Exception {
-    Configuration conf = new Configuration();
-    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
-
-    fs = FileSystem.get(conf);
+    fs = getLocalFileSystem();
   }
 
   private static void deleteTableIfExists(String tableName) {

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/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 2473dc5..f94cf21 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
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -22,9 +22,6 @@ 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.scanner.ClassPathScanner;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.common.util.TestTools;
@@ -42,7 +39,6 @@ import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.IntVector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.VarBinaryVector;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -97,12 +93,9 @@ public class TestWriteToDisk extends ExecTest {
         VectorAccessibleSerializable wrap = new VectorAccessibleSerializable(
             batch, context.getAllocator());
 
-        Configuration conf = new Configuration();
-        conf.set(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
-
         final VectorAccessibleSerializable newWrap = new VectorAccessibleSerializable(
             context.getAllocator());
-        try (final FileSystem fs = FileSystem.get(conf)) {
+        try (final FileSystem fs = getLocalFileSystem()) {
           final File tempDir = Files.createTempDir();
           tempDir.deleteOnExit();
           final Path path = new Path(tempDir.getAbsolutePath(), "drillSerializable");

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/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 3c174ae..fd19378 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
@@ -119,10 +119,7 @@ public class TestParquetWriter extends BaseTestQuery {
 
   @BeforeClass
   public static void initFs() throws Exception {
-    Configuration conf = new Configuration();
-    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
-
-    fs = FileSystem.get(conf);
+    fs = getLocalFileSystem();
     test(String.format("alter session set `%s` = true", PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY));
   }
 
@@ -731,6 +728,7 @@ public class TestParquetWriter extends BaseTestQuery {
           .go();
 
       Configuration hadoopConf = new Configuration();
+      hadoopConf.set(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
       Path output = new Path(getDfsTestTmpSchemaLocation(), outputFile);
       FileSystem fs = output.getFileSystem(hadoopConf);
       for (FileStatus file : fs.listStatus(output)) {

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriterEmptyFiles.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriterEmptyFiles.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriterEmptyFiles.java
index d57605b..4f92c6f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriterEmptyFiles.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriterEmptyFiles.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -19,7 +19,6 @@ package org.apache.drill.exec.physical.impl.writer;
 
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.exec.ExecConstants;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.Assert;
@@ -32,11 +31,7 @@ public class TestParquetWriterEmptyFiles extends BaseTestQuery {
 
   @BeforeClass
   public static void initFs() throws Exception {
-    Configuration conf = new Configuration();
-    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
-
-    fs = FileSystem.get(conf);
-
+    fs = getLocalFileSystem();
     updateTestCluster(3, null);
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/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 5f306c6..9f93eed 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
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -30,7 +30,6 @@ import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.vector.BigIntVector;
 import org.apache.drill.exec.vector.VarCharVector;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -48,10 +47,7 @@ public class TestWriter extends BaseTestQuery {
 
   @BeforeClass
   public static void initFs() throws Exception {
-    Configuration conf = new Configuration();
-    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
-
-    fs = FileSystem.get(conf);
+    fs = getLocalFileSystem();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/exec/java-exec/src/test/java/org/apache/drill/exec/planner/TestDirectoryExplorerUDFs.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/TestDirectoryExplorerUDFs.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/TestDirectoryExplorerUDFs.java
index 4458d58..6b30de9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/TestDirectoryExplorerUDFs.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/TestDirectoryExplorerUDFs.java
@@ -30,7 +30,6 @@ import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.util.JsonStringArrayList;
 import org.apache.drill.exec.util.TestUtilities;
 import org.apache.drill.exec.util.Text;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
@@ -196,53 +195,43 @@ public class TestDirectoryExplorerUDFs extends PlanTestBase {
 
   @Test // DRILL-4720
   public void testDirectoryUDFsWithAndWithoutMetadataCache() throws Exception {
-    FileSystem fs = null;
-    try {
-      fs = FileSystem.get(new Configuration());
-
-      // prepare test table with partitions
-      Path table = new Path(getTempDir("table_with_partitions"));
-      String tablePath = table.toUri().getPath();
-      Path dataFile = new Path(TestTools.getWorkingPath(),"src/test/resources/parquet/alltypes_required.parquet");
-      createPartitions(fs, table, dataFile, 2);
-
-      Map<String, String> configurations = ImmutableMap.<String, String>builder()
-          .put("mindir", "part_1")
-          .put("imindir", "part_1")
-          .put("maxdir", "part_2")
-          .put("imaxdir", "part_2")
-          .build();
-
-      String query = "select dir0 from dfs.`%s` where dir0 = %s('dfs', '%s') limit 1";
-
-      // run tests without metadata cache
-      for (Map.Entry<String, String> entry : configurations.entrySet()) {
-        testBuilder()
-            .sqlQuery(query, tablePath, entry.getKey(), tablePath)
-            .unOrdered()
-            .baselineColumns("dir0")
-            .baselineValues(entry.getValue())
-            .go()
-        ;
-      }
+    FileSystem fs = getLocalFileSystem();
+    // prepare test table with partitions
+    Path table = new Path(getTempDir("table_with_partitions"));
+    String tablePath = table.toUri().getPath();
+    Path dataFile = new Path(TestTools.getWorkingPath(),"src/test/resources/parquet/alltypes_required.parquet");
+    createPartitions(fs, table, dataFile, 2);
+
+    Map<String, String> configurations = ImmutableMap.<String, String>builder()
+        .put("mindir", "part_1")
+        .put("imindir", "part_1")
+        .put("maxdir", "part_2")
+        .put("imaxdir", "part_2")
+        .build();
 
-      // generate metadata
-      test("refresh table metadata dfs.`%s`", tablePath);
+    String query = "select dir0 from dfs.`%s` where dir0 = %s('dfs', '%s') limit 1";
 
-      // run tests with metadata cache
-      for (Map.Entry<String, String> entry : configurations.entrySet()) {
-        testBuilder()
-            .sqlQuery(query, tablePath, entry.getKey(), tablePath)
-            .unOrdered()
-            .baselineColumns("dir0")
-            .baselineValues(entry.getValue())
-            .go();
-      }
+    // run tests without metadata cache
+    for (Map.Entry<String, String> entry : configurations.entrySet()) {
+      testBuilder()
+          .sqlQuery(query, tablePath, entry.getKey(), tablePath)
+          .unOrdered()
+          .baselineColumns("dir0")
+          .baselineValues(entry.getValue())
+          .go();
+    }
 
-    } finally {
-      if (fs != null) {
-        fs.close();
-      }
+    // generate metadata
+    test("refresh table metadata dfs.`%s`", tablePath);
+
+    // run tests with metadata cache
+    for (Map.Entry<String, String> entry : configurations.entrySet()) {
+      testBuilder()
+          .sqlQuery(query, tablePath, entry.getKey(), tablePath)
+          .unOrdered()
+          .baselineColumns("dir0")
+          .baselineValues(entry.getValue())
+          .go();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/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 df012be..9b551ce 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
@@ -25,11 +25,12 @@ 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.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
-import java.io.File;
 import java.util.Properties;
 import java.util.UUID;
 
@@ -41,8 +42,10 @@ public class TemporaryTablesAutomaticDropTest extends BaseTestQuery {
 
   private static final String session_id = "sessionId";
 
+  private FileSystem fs;
+
   @Before
-  public void init() throws Exception {
+  public void setup() throws Exception {
     new MockUp<UUID>() {
       @Mock
       public UUID randomUUID() {
@@ -52,47 +55,49 @@ public class TemporaryTablesAutomaticDropTest extends BaseTestQuery {
     Properties testConfigurations = cloneDefaultTestConfigProperties();
     testConfigurations.put(ExecConstants.DEFAULT_TEMPORARY_WORKSPACE, TEMP_SCHEMA);
     updateTestCluster(1, DrillConfig.create(testConfigurations));
+
+    fs = getLocalFileSystem();
   }
 
   @Test
   public void testAutomaticDropWhenClientIsClosed() throws Exception {
-    File sessionTemporaryLocation = createAndCheckSessionTemporaryLocation("client_closed",
+    Path sessionTemporaryLocation = createAndCheckSessionTemporaryLocation("client_closed",
             getDfsTestTmpSchemaLocation());
     updateClient("new_client");
-    assertFalse("Session temporary location should be absent", sessionTemporaryLocation.exists());
+    assertFalse("Session temporary location should be absent", fs.exists(sessionTemporaryLocation));
   }
 
   @Test
   public void testAutomaticDropWhenDrillbitIsClosed() throws Exception {
-    File sessionTemporaryLocation = createAndCheckSessionTemporaryLocation("drillbit_closed",
+    Path sessionTemporaryLocation = createAndCheckSessionTemporaryLocation("drillbit_closed",
             getDfsTestTmpSchemaLocation());
     bits[0].close();
-    assertFalse("Session temporary location should be absent", sessionTemporaryLocation.exists());
+    assertFalse("Session temporary location should be absent", fs.exists(sessionTemporaryLocation));
   }
 
   @Test
   public void testAutomaticDropOfSeveralSessionTemporaryLocations() throws Exception {
-    File firstSessionTemporaryLocation = createAndCheckSessionTemporaryLocation("first_location",
+    Path firstSessionTemporaryLocation = createAndCheckSessionTemporaryLocation("first_location",
             getDfsTestTmpSchemaLocation());
     StoragePluginRegistry pluginRegistry = getDrillbitContext().getStorage();
     String tempDir = TestUtilities.createTempDir();
     try {
       TestUtilities.updateDfsTestTmpSchemaLocation(pluginRegistry, tempDir);
-      File secondSessionTemporaryLocation = createAndCheckSessionTemporaryLocation("second_location", tempDir);
+      Path secondSessionTemporaryLocation = createAndCheckSessionTemporaryLocation("second_location", tempDir);
       updateClient("new_client");
-      assertFalse("First session temporary location should be absent", firstSessionTemporaryLocation.exists());
-      assertFalse("Second session temporary location should be absent", secondSessionTemporaryLocation.exists());
+      assertFalse("First session temporary location should be absent", fs.exists(firstSessionTemporaryLocation));
+      assertFalse("Second session temporary location should be absent", fs.exists(secondSessionTemporaryLocation));
     } finally {
       TestUtilities.updateDfsTestTmpSchemaLocation(pluginRegistry, getDfsTestTmpSchemaLocation());
     }
   }
 
-  private File createAndCheckSessionTemporaryLocation(String suffix, String schemaLocation) throws Exception {
+  private Path createAndCheckSessionTemporaryLocation(String suffix, String schemaLocation) throws Exception {
     String temporaryTableName = "temporary_table_automatic_drop_" + suffix;
     test("create TEMPORARY table %s.%s as select 'A' as c1 from (values(1))", TEMP_SCHEMA, temporaryTableName);
-    File sessionTemporaryLocation = new File(schemaLocation,
+    Path sessionTemporaryLocation = new Path(schemaLocation,
             UUID.nameUUIDFromBytes(session_id.getBytes()).toString());
-    assertTrue("Session temporary location should exist", sessionTemporaryLocation.exists());
+    assertTrue("Session temporary location should exist", fs.exists(sessionTemporaryLocation));
     return sessionTemporaryLocation;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTAS.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTAS.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTAS.java
index 88d23d3..26d81e2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTAS.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTAS.java
@@ -24,7 +24,6 @@ import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.store.StorageStrategy;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.Test;
@@ -288,7 +287,8 @@ public class TestCTAS extends BaseTestQuery {
     test("use %s", TEMP_SCHEMA);
     String tableName = "with_custom_permission";
     StorageStrategy storageStrategy = new StorageStrategy("000", false);
-    try (FileSystem fs = FileSystem.get(new Configuration())) {
+    FileSystem fs = getLocalFileSystem();
+    try {
       test("alter session set `%s` = '%s'", ExecConstants.PERSISTENT_TABLE_UMASK, storageStrategy.getUmask());
       test("create table %s as select 'A' from (values(1))", tableName);
       Path tableLocation = new Path(getDfsTestTmpSchemaLocation(), tableName);

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/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 8ff51be..0a51f6d 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
@@ -30,7 +30,6 @@ import org.apache.drill.exec.store.StorageStrategy;
 import org.apache.drill.exec.store.dfs.FileSystemConfig;
 import org.apache.drill.exec.store.dfs.WorkspaceConfig;
 import org.apache.drill.exec.util.TestUtilities;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
@@ -40,8 +39,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
-import java.io.File;
-import java.io.FileFilter;
 import java.io.IOException;
 import java.util.List;
 import java.util.Properties;
@@ -56,9 +53,8 @@ import static org.junit.Assert.assertTrue;
 public class TestCTTAS extends BaseTestQuery {
 
   private static final UUID session_id = UUID.nameUUIDFromBytes("sessionId".getBytes());
-  private static final String test_schema = "dfs_test";
   private static final String temp2_wk = "tmp2";
-  private static final String temp2_schema = String.format("%s.%s", test_schema, temp2_wk);
+  private static final String temp2_schema = String.format("%s.%s", TEST_SCHEMA, temp2_wk);
 
   private static FileSystem fs;
   private static FsPermission expectedFolderPermission;
@@ -73,11 +69,11 @@ public class TestCTTAS extends BaseTestQuery {
     uuidMockUp.tearDown();
 
     StoragePluginRegistry pluginRegistry = getDrillbitContext().getStorage();
-    FileSystemConfig pluginConfig = (FileSystemConfig) pluginRegistry.getPlugin(test_schema).getConfig();
+    FileSystemConfig pluginConfig = (FileSystemConfig) pluginRegistry.getPlugin(TEST_SCHEMA).getConfig();
     pluginConfig.workspaces.put(temp2_wk, new WorkspaceConfig(TestUtilities.createTempDir(), true, null));
-    pluginRegistry.createOrUpdate(test_schema, pluginConfig, true);
+    pluginRegistry.createOrUpdate(TEST_SCHEMA, pluginConfig, true);
 
-    fs = FileSystem.get(new Configuration());
+    fs = getLocalFileSystem();
     expectedFolderPermission = new FsPermission(StorageStrategy.TEMPORARY.getFolderPermission());
     expectedFilePermission = new FsPermission(StorageStrategy.TEMPORARY.getFilePermission());
   }
@@ -149,7 +145,7 @@ public class TestCTTAS extends BaseTestQuery {
   @Test
   public void testResolveTemporaryTableWithPartialSchema() throws Exception {
     String temporaryTableName = "temporary_table_with_partial_schema";
-    test("use %s", test_schema);
+    test("use %s", TEST_SCHEMA);
     test("create temporary table tmp.%s as select 'A' as c1 from (values(1))", temporaryTableName);
 
     testBuilder()
@@ -438,9 +434,9 @@ public class TestCTTAS extends BaseTestQuery {
   }
 
   private void checkPermission(String tmpTableName) throws IOException {
-    File[] files = findTemporaryTableLocation(tmpTableName);
-    assertEquals("Only one directory should match temporary table name " + tmpTableName, 1, files.length);
-    Path tmpTablePath = new Path(files[0].toURI().getPath());
+    List<Path> matchingPath = findTemporaryTableLocation(tmpTableName);
+    assertEquals("Only one directory should match temporary table name " + tmpTableName, 1, matchingPath.size());
+    Path tmpTablePath = matchingPath.get(0);
     assertEquals("Directory permission should match",
         expectedFolderPermission, fs.getFileStatus(tmpTablePath).getPermission());
     RemoteIterator<LocatedFileStatus> fileIterator = fs.listFiles(tmpTablePath, false);
@@ -449,19 +445,22 @@ public class TestCTTAS extends BaseTestQuery {
     }
   }
 
-  private File[] findTemporaryTableLocation(String tableName) throws IOException {
-    File sessionTempLocation = new File(getDfsTestTmpSchemaLocation(), session_id.toString());
-    Path sessionTempLocationPath = new Path(sessionTempLocation.toURI().getPath());
-    assertTrue("Session temporary location must exist", fs.exists(sessionTempLocationPath));
+  private List<Path> findTemporaryTableLocation(String tableName) throws IOException {
+    Path sessionTempLocation = new Path(getDfsTestTmpSchemaLocation(), session_id.toString());
+    assertTrue("Session temporary location must exist", fs.exists(sessionTempLocation));
     assertEquals("Session temporary location permission should match",
-        expectedFolderPermission, fs.getFileStatus(sessionTempLocationPath).getPermission());
-    final String tableUUID =  UUID.nameUUIDFromBytes(tableName.getBytes()).toString();
-    return sessionTempLocation.listFiles(new FileFilter() {
-      @Override
-      public boolean accept(File path) {
-        return path.isDirectory() && path.getName().equals(tableUUID);
+        expectedFolderPermission, fs.getFileStatus(sessionTempLocation).getPermission());
+    String tableUUID =  UUID.nameUUIDFromBytes(tableName.getBytes()).toString();
+
+    RemoteIterator<LocatedFileStatus> pathList = fs.listLocatedStatus(sessionTempLocation);
+    List<Path> matchingPath = Lists.newArrayList();
+    while (pathList.hasNext()) {
+      LocatedFileStatus path = pathList.next();
+      if (path.isDirectory() && path.getPath().getName().equals(tableUUID)) {
+        matchingPath.add(path.getPath());
       }
-    });
+    }
+    return matchingPath;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/exec/java-exec/src/test/java/org/apache/drill/exec/store/StorageStrategyTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/StorageStrategyTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/StorageStrategyTest.java
index 32eb234..8051d72 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/StorageStrategyTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/StorageStrategyTest.java
@@ -18,11 +18,11 @@ package org.apache.drill.exec.store;
 
 import com.google.common.collect.Lists;
 import com.google.common.io.Files;
-import org.apache.hadoop.conf.Configuration;
+import org.apache.drill.exec.ExecTest;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.File;
@@ -33,16 +33,14 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 public class StorageStrategyTest {
-
-  private static final Configuration CONFIGURATION = new Configuration();
   private static final FsPermission FULL_PERMISSION = FsPermission.getDirDefault();
   private static final StorageStrategy PERSISTENT_STRATEGY = new StorageStrategy("002", false);
   private static final StorageStrategy TEMPORARY_STRATEGY = new StorageStrategy("077", true);
-  private FileSystem FS;
+  private static FileSystem fs;
 
-  @Before
-  public void setup() throws Exception {
-    initFileSystem();
+  @BeforeClass
+  public static void setup() throws Exception {
+    fs = ExecTest.getLocalFileSystem();
   }
 
   @Test
@@ -51,11 +49,11 @@ public class StorageStrategyTest {
     Path file = addNLevelsAndFile(initialPath, 2, true);
     Path firstCreatedParentPath = addNLevelsAndFile(initialPath, 1, false);
 
-    Path createdParentPath = PERSISTENT_STRATEGY.createFileAndApply(FS, file);
+    Path createdParentPath = PERSISTENT_STRATEGY.createFileAndApply(fs, file);
 
     assertEquals("Path should match", firstCreatedParentPath, createdParentPath);
     checkPathAndPermission(initialPath, file, true, 2, PERSISTENT_STRATEGY);
-    checkDeleteOnExit(firstCreatedParentPath, true);
+    checkDeleteOnExit(firstCreatedParentPath, false);
   }
 
   @Test
@@ -64,11 +62,11 @@ public class StorageStrategyTest {
     Path file = addNLevelsAndFile(initialPath, 2, true);
     Path firstCreatedParentPath = addNLevelsAndFile(initialPath, 1, false);
 
-    Path createdParentPath = TEMPORARY_STRATEGY.createFileAndApply(FS, file);
+    Path createdParentPath = TEMPORARY_STRATEGY.createFileAndApply(fs, file);
 
     assertEquals("Path should match", firstCreatedParentPath, createdParentPath);
     checkPathAndPermission(initialPath, file, true, 2, TEMPORARY_STRATEGY);
-    checkDeleteOnExit(firstCreatedParentPath, false);
+    checkDeleteOnExit(firstCreatedParentPath, true);
   }
 
   @Test
@@ -76,11 +74,11 @@ public class StorageStrategyTest {
     Path initialPath = prepareStorageDirectory();
     Path file = addNLevelsAndFile(initialPath, 0, true);
 
-    Path createdFile = PERSISTENT_STRATEGY.createFileAndApply(FS, file);
+    Path createdFile = PERSISTENT_STRATEGY.createFileAndApply(fs, file);
 
     assertEquals("Path should match", file, createdFile);
     checkPathAndPermission(initialPath, file, true, 0, PERSISTENT_STRATEGY);
-    checkDeleteOnExit(file, true);
+    checkDeleteOnExit(file, false);
   }
 
   @Test
@@ -88,24 +86,24 @@ public class StorageStrategyTest {
     Path initialPath = prepareStorageDirectory();
     Path file = addNLevelsAndFile(initialPath, 0, true);
 
-    Path createdFile = TEMPORARY_STRATEGY.createFileAndApply(FS, file);
+    Path createdFile = TEMPORARY_STRATEGY.createFileAndApply(fs, file);
 
     assertEquals("Path should match", file, createdFile);
     checkPathAndPermission(initialPath, file, true, 0, TEMPORARY_STRATEGY);
-    checkDeleteOnExit(file, false);
+    checkDeleteOnExit(file, true);
   }
 
   @Test(expected = IOException.class)
   public void testFailureOnExistentFile() throws Exception {
     Path initialPath = prepareStorageDirectory();
     Path file = addNLevelsAndFile(initialPath, 0, true);
-    FS.createNewFile(file);
-    assertTrue("File should exist", FS.exists(file));
+    fs.createNewFile(file);
+    assertTrue("File should exist", fs.exists(file));
     try {
-      PERSISTENT_STRATEGY.createFileAndApply(FS, file);
+      PERSISTENT_STRATEGY.createFileAndApply(fs, file);
     } catch (IOException e) {
       assertEquals("Error message should match", String.format("File [%s] already exists on file system [%s].",
-          file.toUri().getPath(), FS.getUri()), e.getMessage());
+          file.toUri().getPath(), fs.getUri()), e.getMessage());
       throw e;
     }
   }
@@ -116,11 +114,11 @@ public class StorageStrategyTest {
     Path resultPath = addNLevelsAndFile(initialPath, 2, false);
     Path firstCreatedParentPath = addNLevelsAndFile(initialPath, 1, false);
 
-    Path createdParentPath = PERSISTENT_STRATEGY.createPathAndApply(FS, resultPath);
+    Path createdParentPath = PERSISTENT_STRATEGY.createPathAndApply(fs, resultPath);
 
     assertEquals("Path should match", firstCreatedParentPath, createdParentPath);
     checkPathAndPermission(initialPath, resultPath, false, 2, PERSISTENT_STRATEGY);
-    checkDeleteOnExit(firstCreatedParentPath, true);
+    checkDeleteOnExit(firstCreatedParentPath, false);
   }
 
   @Test
@@ -129,38 +127,38 @@ public class StorageStrategyTest {
     Path resultPath = addNLevelsAndFile(initialPath, 2, false);
     Path firstCreatedParentPath = addNLevelsAndFile(initialPath, 1, false);
 
-    Path createdParentPath = TEMPORARY_STRATEGY.createPathAndApply(FS, resultPath);
+    Path createdParentPath = TEMPORARY_STRATEGY.createPathAndApply(fs, resultPath);
 
     assertEquals("Path should match", firstCreatedParentPath, createdParentPath);
     checkPathAndPermission(initialPath, resultPath, false, 2, TEMPORARY_STRATEGY);
-    checkDeleteOnExit(firstCreatedParentPath, false);
+    checkDeleteOnExit(firstCreatedParentPath, true);
   }
 
   @Test
   public void testCreateNoPath() throws Exception {
     Path path = prepareStorageDirectory();
 
-    Path createdParentPath = TEMPORARY_STRATEGY.createPathAndApply(FS, path);
+    Path createdParentPath = TEMPORARY_STRATEGY.createPathAndApply(fs, path);
 
     assertNull("Path should be null", createdParentPath);
-    assertEquals("Permission should match", FULL_PERMISSION, FS.getFileStatus(path).getPermission());
+    assertEquals("Permission should match", FULL_PERMISSION, fs.getFileStatus(path).getPermission());
   }
 
   @Test
   public void testStrategyForExistingFile() throws Exception {
     Path initialPath = prepareStorageDirectory();
     Path file = addNLevelsAndFile(initialPath, 0, true);
-    FS.createNewFile(file);
-    FS.setPermission(file, FULL_PERMISSION);
+    fs.createNewFile(file);
+    fs.setPermission(file, FULL_PERMISSION);
 
-    assertTrue("File should exist", FS.exists(file));
-    assertEquals("Permission should match", FULL_PERMISSION, FS.getFileStatus(file).getPermission());
+    assertTrue("File should exist", fs.exists(file));
+    assertEquals("Permission should match", FULL_PERMISSION, fs.getFileStatus(file).getPermission());
 
-    TEMPORARY_STRATEGY.applyToFile(FS, file);
+    TEMPORARY_STRATEGY.applyToFile(fs, file);
 
     assertEquals("Permission should match", new FsPermission(TEMPORARY_STRATEGY.getFilePermission()),
-        FS.getFileStatus(file).getPermission());
-    checkDeleteOnExit(file, false);
+        fs.getFileStatus(file).getPermission());
+    checkDeleteOnExit(file, true);
   }
 
   @Test
@@ -176,21 +174,10 @@ public class StorageStrategyTest {
     File storageDirectory = Files.createTempDir();
     storageDirectory.deleteOnExit();
     Path path = new Path(storageDirectory.toURI().getPath());
-    FS.setPermission(path, FULL_PERMISSION);
+    fs.setPermission(path, FULL_PERMISSION);
     return path;
   }
 
-  private void initFileSystem() throws IOException {
-    if (FS != null) {
-      try {
-        FS.close();
-      } catch (Exception e) {
-        // do nothing
-      }
-    }
-    FS = FileSystem.get(CONFIGURATION);
-  }
-
   private Path addNLevelsAndFile(Path initialPath, int levels, boolean addFile) {
     Path resultPath = initialPath;
     for (int i = 1; i <= levels; i++) {
@@ -208,25 +195,23 @@ public class StorageStrategyTest {
                                       int levels,
                                       StorageStrategy storageStrategy) throws IOException {
 
-    assertEquals("Path type should match", isFile, FS.isFile(resultPath));
-    assertEquals("Permission should match", FULL_PERMISSION, FS.getFileStatus(initialPath).getPermission());
+    assertEquals("Path type should match", isFile, fs.isFile(resultPath));
+    assertEquals("Permission should match", FULL_PERMISSION, fs.getFileStatus(initialPath).getPermission());
 
     if (isFile) {
       assertEquals("Permission should match", new FsPermission(storageStrategy.getFilePermission()),
-          FS.getFileStatus(resultPath).getPermission());
+          fs.getFileStatus(resultPath).getPermission());
     }
     Path startingPath = initialPath;
     FsPermission folderPermission = new FsPermission(storageStrategy.getFolderPermission());
     for (int i = 1; i <= levels; i++) {
       startingPath = new Path(startingPath, "level" + i);
-      assertEquals("Permission should match", folderPermission, FS.getFileStatus(startingPath).getPermission());
+      assertEquals("Permission should match", folderPermission, fs.getFileStatus(startingPath).getPermission());
     }
   }
 
-  private void checkDeleteOnExit(Path path, boolean isPresent) throws IOException {
-    assertTrue("Path should be present", FS.exists(path));
-    // close and open file system to check for path presence
-    initFileSystem();
-    assertEquals("Path existence flag should match", isPresent, FS.exists(path));
+  private void checkDeleteOnExit(Path path, boolean isMarkedToBeDeleted) throws IOException {
+    assertTrue("Path should be present", fs.exists(path));
+    assertEquals("Path delete-on-exit status should match", isMarkedToBeDeleted, fs.cancelDeleteOnExit(path));
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
index 277e6f9..f9add51 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetFilterPushDown.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
@@ -35,7 +35,6 @@ import org.junit.Test;
 
 import java.io.IOException;
 
-import static org.apache.zookeeper.ZooDefs.OpCode.create;
 import static org.junit.Assert.assertEquals;
 
 public class TestParquetFilterPushDown extends PlanTestBase {
@@ -44,17 +43,16 @@ public class TestParquetFilterPushDown extends PlanTestBase {
   private static final String TEST_RES_PATH = WORKING_PATH + "/src/test/resources";
   private static FragmentContext fragContext;
 
-  static FileSystem fs;
+  private static Configuration conf;
+  private static FileSystem fs;
 
   @BeforeClass
   public static void initFSAndCreateFragContext() throws Exception {
     fragContext = new FragmentContext(bits[0].getContext(),
         BitControl.PlanFragment.getDefaultInstance(), null, bits[0].getContext().getFunctionImplementationRegistry());
 
-    Configuration conf = new Configuration();
-    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, FileSystem.DEFAULT_FS);
-
-    fs = FileSystem.get(conf);
+    fs = getLocalFileSystem();
+    conf = fs.getConf();
   }
 
   @AfterClass
@@ -408,9 +406,7 @@ public class TestParquetFilterPushDown extends PlanTestBase {
   }
 
   private ParquetMetadata getParquetMetaData(String filePathStr) throws IOException{
-    Configuration fsConf = new Configuration();
-    ParquetMetadata footer = ParquetFileReader.readFooter(fsConf, new Path(filePathStr));
-    return footer;
+    return ParquetFileReader.readFooter(new Configuration(conf), new Path(filePathStr));
   }
 
   private static void deleteTableIfExists(String tableName) {

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetScan.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetScan.java
index 69759e6..39df4e0 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetScan.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetScan.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -19,7 +19,6 @@ package org.apache.drill.exec.store.parquet;
 
 import com.google.common.io.Resources;
 import org.apache.drill.BaseTestQuery;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -32,10 +31,7 @@ public class TestParquetScan extends BaseTestQuery {
 
   @BeforeClass
   public static void initFs() throws Exception {
-    Configuration conf = new Configuration();
-    conf.set("fs.default.name", "local");
-
-    fs = FileSystem.get(conf);
+    fs = getLocalFileSystem();
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/exec/java-exec/src/test/java/org/apache/drill/exec/util/FileSystemUtilTestBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/util/FileSystemUtilTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/exec/util/FileSystemUtilTestBase.java
index 1df25ee..51e9204 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/util/FileSystemUtilTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/util/FileSystemUtilTestBase.java
@@ -19,10 +19,9 @@ package org.apache.drill.exec.util;
 import com.google.common.base.Strings;
 import com.google.common.io.Files;
 import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.conf.Configuration;
+import org.apache.drill.exec.ExecTest;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
 import java.io.File;
@@ -64,7 +63,7 @@ public class FileSystemUtilTestBase {
   @BeforeClass
   public static void setup() throws IOException {
     // initialize file system
-    fs = FileSystem.get(new Configuration());
+    fs = ExecTest.getLocalFileSystem();
 
     // create temporary directory with sub-folders and files
     final File tempDir = Files.createTempDir();
@@ -90,13 +89,6 @@ public class FileSystemUtilTestBase {
     fs.createNewFile(new Path(underscoreDirectory, "f.txt"));
   }
 
-  @AfterClass
-  public static void tearDown() throws Exception {
-    if (fs != null) {
-      fs.close();
-    }
-  }
-
   private static void createDefaultStructure(FileSystem fs, Path base, String name, int nesting) throws IOException {
     Path newBase = base;
     for (int i = 1; i <= nesting; i++) {

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
index 513fe3a..354b487 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
@@ -19,6 +19,7 @@ package org.apache.drill.test;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.URI;
 import java.net.URL;
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -118,7 +119,8 @@ public class ClusterFixture extends BaseFixture implements AutoCloseable {
       // storage. Profiles will go here when running in distributed
       // mode.
 
-      put(ZookeeperPersistentStoreProvider.DRILL_EXEC_SYS_STORE_PROVIDER_ZK_BLOBROOT, "/tmp/drill/log");
+      put(ZookeeperPersistentStoreProvider.DRILL_EXEC_SYS_STORE_PROVIDER_ZK_BLOBROOT, "/tmp/drill/tests");
+      put(ExecConstants.SYS_STORE_PROVIDER_LOCAL_PATH, "file:/tmp/drill/tests");
     }
   };
 
@@ -729,7 +731,7 @@ public class ClusterFixture extends BaseFixture implements AutoCloseable {
   }
 
   public File getDrillTempDir() {
-    return new File(config.getString(ExecConstants.SYS_STORE_PROVIDER_LOCAL_PATH));
+    return new File(URI.create(config.getString(ExecConstants.SYS_STORE_PROVIDER_LOCAL_PATH)).getPath());
   }
 
   public boolean usesZK() {

http://git-wip-us.apache.org/repos/asf/drill/blob/75c35135/exec/java-exec/src/test/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/drill-module.conf b/exec/java-exec/src/test/resources/drill-module.conf
index adeab56..d701bab 100644
--- a/exec/java-exec/src/test/resources/drill-module.conf
+++ b/exec/java-exec/src/test/resources/drill-module.conf
@@ -35,6 +35,8 @@ drill: {
     },
     use.ip : false
   },
+  sys.store.provider.local.path: "file:/tmp/drill/tests",
+  udf.directory.fs: "local",
   operator: {
     packages += "org.apache.drill.exec.physical.config"
   },


[2/3] drill git commit: DRILL-5729 explicitly made the travis container use openjdk7 to fix the build.

Posted by am...@apache.org.
DRILL-5729 explicitly made the travis container use openjdk7 to fix the build.

close apache/drill#913


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

Branch: refs/heads/master
Commit: 33f17aabadff5bf5f546762a9d0fb63edbd725f9
Parents: 75c3513
Author: Timothy Farkas <ti...@apache.org>
Authored: Thu Aug 17 16:37:24 2017 -0700
Committer: Aman Sinha <as...@maprtech.com>
Committed: Sat Sep 2 13:37:51 2017 -0700

----------------------------------------------------------------------
 .travis.yml | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/33f17aab/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 4032aab..a94ee96 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -13,8 +13,9 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-sudo: false
 before_install: git fetch --unshallow
 language: java
-install: mvn install --batch-mode -DskipTests=true -Dmaven.javadoc.skip=true -Dsource.skip=true > mvn_install.log || (cat mvn_install.log && false)
-script: mvn package -DskipTests=true
+jdk:
+  - openjdk7
+install: MAVEN_OPTS="-Xms1G -Xmx1G" mvn install --batch-mode -DskipTests=true -Dmaven.javadoc.skip=true -Dmaven.source.skip=true
+script: echo "Done"


[3/3] drill git commit: DRILL-1051: Casting timestamp as date gives wrong result for dates earlier than 1883

Posted by am...@apache.org.
DRILL-1051: Casting timestamp as date gives wrong result for dates earlier than 1883

- Fix DateAccessor's, TimestampAccessor's and TimeAccessor's converting joda time to java.sql

close apache/drill#915


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

Branch: refs/heads/master
Commit: 406dd2a58d2168faf47636fcd9829f29a93175bd
Parents: 33f17aa
Author: Vitalii Diravka <vi...@gmail.com>
Authored: Tue Aug 15 17:51:10 2017 +0000
Committer: Aman Sinha <as...@maprtech.com>
Committed: Sat Sep 2 13:38:02 2017 -0700

----------------------------------------------------------------------
 .../main/codegen/templates/SqlAccessors.java    | 28 +++++++++-----------
 .../apache/drill/jdbc/test/TestJdbcQuery.java   | 25 ++++++++++++++++-
 2 files changed, 36 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/406dd2a5/exec/java-exec/src/main/codegen/templates/SqlAccessors.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/SqlAccessors.java b/exec/java-exec/src/main/codegen/templates/SqlAccessors.java
index 49a91c2..888d1c6 100644
--- a/exec/java-exec/src/main/codegen/templates/SqlAccessors.java
+++ b/exec/java-exec/src/main/codegen/templates/SqlAccessors.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
@@ -264,9 +264,10 @@ public class ${name}Accessor extends AbstractSqlAccessor {
       return null;
     }
    </#if>
-    org.joda.time.DateTime date = new org.joda.time.DateTime(ac.get(index), org.joda.time.DateTimeZone.UTC);
-    date = date.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
-    return new Date(date.getMillis());
+    org.joda.time.LocalDate date = new org.joda.time.LocalDate(ac.get(index), org.joda.time.DateTimeZone.UTC);
+    // Use "toDate()" to get java.util.Date object with exactly the same year the same year, month and day as Joda date.
+    // See more in Javadoc for "LocalDate#toDate()"
+    return new Date(date.toDate().getTime());
   }
 
   <#elseif minor.class == "TimeStamp">
@@ -293,9 +294,10 @@ public class ${name}Accessor extends AbstractSqlAccessor {
       return null;
     }
    </#if>
-    org.joda.time.DateTime date = new org.joda.time.DateTime(ac.get(index), org.joda.time.DateTimeZone.UTC);
-    date = date.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
-    return new Timestamp(date.getMillis());
+    org.joda.time.LocalDateTime dateTime = new org.joda.time.LocalDateTime(ac.get(index), org.joda.time.DateTimeZone.UTC);
+    // use "toDate()" to get java.util.Date object with exactly the same fields as this Joda date-time.
+    // See more in Javadoc for "LocalDateTime#toDate()"
+    return new Timestamp(dateTime.toDate().getTime());
   }
 
   <#elseif minor.class == "Time">
@@ -317,9 +319,9 @@ public class ${name}Accessor extends AbstractSqlAccessor {
       return null;
     }
    </#if>
-    org.joda.time.DateTime time = new org.joda.time.DateTime(ac.get(index), org.joda.time.DateTimeZone.UTC);
-    time = time.withZoneRetainFields(org.joda.time.DateTimeZone.getDefault());
-    return new TimePrintMillis(time.getMillis());
+    org.joda.time.LocalTime time = new org.joda.time.LocalTime(ac.get(index), org.joda.time.DateTimeZone.UTC);
+    // use "toDateTimeToday()"  and "getMillis()" to get the local milliseconds from the Java epoch of 1970-01-01T00:00:00
+    return new TimePrintMillis(time.toDateTimeToday().getMillis());
   }
 
   <#else>
@@ -330,7 +332,6 @@ public class ${name}Accessor extends AbstractSqlAccessor {
   }
   </#if>
 
-
   <#if minor.class == "Bit" >
   public boolean getBoolean(int index) {
    <#if mode == "Nullable">
@@ -341,13 +342,8 @@ public class ${name}Accessor extends AbstractSqlAccessor {
    return 1 == ac.get(index);
   }
  </#if>
-
-
  </#if> <#-- not VarLen -->
-
 }
-
-
 </#list>
 </#list>
 </#list>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/406dd2a5/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java
index bff620e..1dd172f 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.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
@@ -367,4 +367,27 @@ public class TestJdbcQuery extends JdbcTestQueryBase {
             "TIME_INT_ADD=6\n"
         );
   }
+
+  @Test // DRILL-1051
+  public void testOldDateTimeJulianCalendar() throws Exception {
+    // Should be work with any timezone
+    JdbcAssert.withNoDefaultSchema()
+        .sql("select cast(to_timestamp('1581-12-01 23:32:01', 'yyyy-MM-dd HH:mm:ss') as date) as `DATE`, " +
+            "to_timestamp('1581-12-01 23:32:01', 'yyyy-MM-dd HH:mm:ss') as `TIMESTAMP`, " +
+            "cast(to_timestamp('1581-12-01 23:32:01', 'yyyy-MM-dd HH:mm:ss') as time) as `TIME` " +
+            "from (VALUES(1))")
+        .returns("DATE=1581-12-01; TIMESTAMP=1581-12-01 23:32:01.0; TIME=23:32:01");
+  }
+
+  @Test // DRILL-1051
+  public void testOldDateTimeLocalMeanTime() throws Exception {
+    // Should be work with any timezone
+    JdbcAssert.withNoDefaultSchema()
+        .sql("select cast(to_timestamp('1883-11-16 01:32:01', 'yyyy-MM-dd HH:mm:ss') as date) as `DATE`, " +
+            "to_timestamp('1883-11-16 01:32:01', 'yyyy-MM-dd HH:mm:ss') as `TIMESTAMP`, " +
+            "cast(to_timestamp('1883-11-16 01:32:01', 'yyyy-MM-dd HH:mm:ss') as time) as `TIME` " +
+            "from (VALUES(1))")
+        .returns("DATE=1883-11-16; TIMESTAMP=1883-11-16 01:32:01.0; TIME=01:32:01");
+  }
+
 }