You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2016/10/20 22:12:42 UTC

[05/23] hive git commit: HIVE-15008. Cleanup local workDir when MiniHS2 starts up in FS_ONLY mode. (Siddharth Seth, reviewed by Prasanth Jayachandran)

HIVE-15008. Cleanup local workDir when MiniHS2 starts up in FS_ONLY mode. (Siddharth Seth, reviewed by Prasanth Jayachandran)


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

Branch: refs/heads/hive-14535
Commit: 9072575eaa92290bf241d38ae490a96fe0822be6
Parents: 3252bed
Author: Siddharth Seth <ss...@apache.org>
Authored: Wed Oct 19 19:07:20 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Wed Oct 19 19:07:20 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hive/jdbc/miniHS2/MiniHS2.java   | 71 +++++++++++++++-----
 .../TestBeelineConnectionUsingHiveSite.java     |  7 --
 .../TestBeelineWithHS2ConnectionFile.java       | 12 ++--
 .../apache/hive/jdbc/TestJdbcWithMiniHS2.java   |  7 +-
 .../test/java/org/apache/hive/jdbc/TestSSL.java |  9 ++-
 .../jdbc/TestServiceDiscoveryWithMiniHS2.java   |  6 +-
 .../org/apache/hive/jdbc/TestXSRFFilter.java    | 14 +++-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |  2 +-
 8 files changed, 92 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9072575e/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
index ddc2690..f02f0e3 100644
--- a/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
+++ b/itests/hive-unit/src/main/java/org/apache/hive/jdbc/miniHS2/MiniHS2.java
@@ -19,11 +19,14 @@
 package org.apache.hive.jdbc.miniHS2;
 
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+
+import com.google.common.base.Preconditions;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -46,10 +49,13 @@ import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService;
 import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient;
 import org.apache.hive.service.cli.thrift.ThriftHttpCLIService;
 import org.apache.hive.service.server.HiveServer2;
-
-import com.google.common.io.Files;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class MiniHS2 extends AbstractHiveService {
+
+  private static final Logger LOG = LoggerFactory.getLogger(MiniHS2.class);
+
   public static final String HS2_BINARY_MODE = "binary";
   public static final String HS2_HTTP_MODE = "http";
   private static final String driverName = "org.apache.hive.jdbc.HiveDriver";
@@ -58,7 +64,7 @@ public class MiniHS2 extends AbstractHiveService {
   private static final String tmpDir = System.getProperty("test.tmp.dir");
   private HiveServer2 hiveServer2 = null;
   private final File baseDir;
-  private final Path baseDfsDir;
+  private final Path baseFsDir;
   private MiniMrShim mr;
   private MiniDFSShim dfs;
   private MiniLlapCluster llapCluster = null;
@@ -66,18 +72,19 @@ public class MiniHS2 extends AbstractHiveService {
   private boolean useMiniKdc = false;
   private final String serverPrincipal;
   private final boolean isMetastoreRemote;
-  private MiniClusterType miniClusterType = MiniClusterType.DFS_ONLY;
+  private final boolean cleanupLocalDirOnStartup;
+  private MiniClusterType miniClusterType = MiniClusterType.LOCALFS_ONLY;
 
   public enum MiniClusterType {
     MR,
     TEZ,
     LLAP,
-    DFS_ONLY;
+    LOCALFS_ONLY;
   }
 
   public static class Builder {
     private HiveConf hiveConf = new HiveConf();
-    private MiniClusterType miniClusterType = MiniClusterType.DFS_ONLY;
+    private MiniClusterType miniClusterType = MiniClusterType.LOCALFS_ONLY;
     private boolean useMiniKdc = false;
     private String serverPrincipal;
     private String serverKeytab;
@@ -86,6 +93,7 @@ public class MiniHS2 extends AbstractHiveService {
     private boolean usePortsFromConf = false;
     private String authType = "KERBEROS";
     private boolean isHA = false;
+    private boolean cleanupLocalDirOnStartup = true;
 
     public Builder() {
     }
@@ -131,6 +139,11 @@ public class MiniHS2 extends AbstractHiveService {
       return this;
     }
 
+    public Builder cleanupLocalDirOnStartup(boolean val) {
+      this.cleanupLocalDirOnStartup = val;
+      return this;
+    }
+
     public MiniHS2 build() throws Exception {
       if (miniClusterType == MiniClusterType.MR && useMiniKdc) {
         throw new IOException("Can't create secure miniMr ... yet");
@@ -141,7 +154,7 @@ public class MiniHS2 extends AbstractHiveService {
         hiveConf.setVar(ConfVars.HIVE_SERVER2_TRANSPORT_MODE, HS2_BINARY_MODE);
       }
       return new MiniHS2(hiveConf, miniClusterType, useMiniKdc, serverPrincipal, serverKeytab,
-          isMetastoreRemote, usePortsFromConf, authType, isHA);
+          isMetastoreRemote, usePortsFromConf, authType, isHA, cleanupLocalDirOnStartup);
     }
   }
 
@@ -179,7 +192,7 @@ public class MiniHS2 extends AbstractHiveService {
 
   private MiniHS2(HiveConf hiveConf, MiniClusterType miniClusterType, boolean useMiniKdc,
       String serverPrincipal, String serverKeytab, boolean isMetastoreRemote,
-      boolean usePortsFromConf, String authType, boolean isHA) throws Exception {
+      boolean usePortsFromConf, String authType, boolean isHA, boolean cleanupLocalDirOnStartup) throws Exception {
     // Always use localhost for hostname as some tests like SSL CN validation ones
     // are tied to localhost being present in the certificate name
     super(
@@ -196,11 +209,12 @@ public class MiniHS2 extends AbstractHiveService {
     this.useMiniKdc = useMiniKdc;
     this.serverPrincipal = serverPrincipal;
     this.isMetastoreRemote = isMetastoreRemote;
-    baseDir = new File(tmpDir + "/local_base");
+    this.cleanupLocalDirOnStartup = cleanupLocalDirOnStartup;
+    baseDir = getBaseDir();
     localFS = FileSystem.getLocal(hiveConf);
     FileSystem fs;
 
-    if (miniClusterType != MiniClusterType.DFS_ONLY) {
+    if (miniClusterType != MiniClusterType.LOCALFS_ONLY) {
       // Initialize dfs
       dfs = ShimLoader.getHadoopShims().getMiniDfs(hiveConf, 4, true, null, isHA);
       fs = dfs.getFileSystem();
@@ -227,11 +241,18 @@ public class MiniHS2 extends AbstractHiveService {
       }
       // store the config in system properties
       mr.setupConfiguration(getHiveConf());
-      baseDfsDir = new Path(new Path(fs.getUri()), "/base");
+      baseFsDir = new Path(new Path(fs.getUri()), "/base");
     } else {
-      // This is DFS only mode, just initialize the dfs root directory.
+      // This is FS only mode, just initialize the dfs root directory.
       fs = FileSystem.getLocal(hiveConf);
-      baseDfsDir = new Path("file://" + baseDir.toURI().getPath());
+      baseFsDir = new Path("file://" + baseDir.toURI().getPath());
+
+      if (cleanupLocalDirOnStartup) {
+        // Cleanup baseFsDir since it can be shared across tests.
+        LOG.info("Attempting to cleanup baseFsDir: {} while setting up MiniHS2", baseDir);
+        Preconditions.checkState(baseFsDir.depth() >= 3); // Avoid "/tmp", directories closer to "/"
+        fs.delete(baseFsDir, true);
+      }
     }
     if (useMiniKdc) {
       hiveConf.setVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL, serverPrincipal);
@@ -242,8 +263,8 @@ public class MiniHS2 extends AbstractHiveService {
         "jdbc:derby:;databaseName=" + baseDir.getAbsolutePath() + File.separator
             + "test_metastore;create=true";
 
-    fs.mkdirs(baseDfsDir);
-    Path wareHouseDir = new Path(baseDfsDir, "warehouse");
+    fs.mkdirs(baseFsDir);
+    Path wareHouseDir = new Path(baseFsDir, "warehouse");
     // Create warehouse with 777, so that user impersonation has no issues.
     FileSystem.mkdirs(fs, wareHouseDir, FULL_PERM);
 
@@ -259,7 +280,7 @@ public class MiniHS2 extends AbstractHiveService {
     hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT, getBinaryPort());
     hiveConf.setIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT, getHttpPort());
 
-    Path scratchDir = new Path(baseDfsDir, "scratch");
+    Path scratchDir = new Path(baseFsDir, "scratch");
     // Create root scratchdir with write all, so that user impersonation has no issues.
     Utilities.createDirsWithPermission(hiveConf, scratchDir, WRITE_ALL_PERM, true);
     System.setProperty(HiveConf.ConfVars.SCRATCHDIR.varname, scratchDir.toString());
@@ -271,7 +292,7 @@ public class MiniHS2 extends AbstractHiveService {
   }
 
   public MiniHS2(HiveConf hiveConf) throws Exception {
-    this(hiveConf, MiniClusterType.DFS_ONLY);
+    this(hiveConf, MiniClusterType.LOCALFS_ONLY);
   }
 
   public MiniHS2(HiveConf hiveConf, MiniClusterType clusterType) throws Exception {
@@ -281,7 +302,7 @@ public class MiniHS2 extends AbstractHiveService {
   public MiniHS2(HiveConf hiveConf, MiniClusterType clusterType,
       boolean usePortsFromConf) throws Exception {
     this(hiveConf, clusterType, false, null, null, false, usePortsFromConf,
-      "KERBEROS", false);
+      "KERBEROS", false, true);
   }
 
   public void start(Map<String, String> confOverlay) throws Exception {
@@ -516,4 +537,18 @@ public class MiniHS2 extends AbstractHiveService {
   public Service.STATE getState() {
     return hiveServer2.getServiceState();
   }
+
+  static File getBaseDir() {
+    File baseDir = new File(tmpDir + "/local_base");
+    return baseDir;
+  }
+
+  public static void cleanupLocalDir() throws IOException {
+    File baseDir = getBaseDir();
+    try {
+      org.apache.hadoop.hive.common.FileUtils.deleteDirectory(baseDir);
+    } catch (FileNotFoundException e) {
+      // Ignore. Safe if it does not exist.
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/9072575e/itests/hive-unit/src/test/java/org/apache/hive/beeline/hs2connection/TestBeelineConnectionUsingHiveSite.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/hs2connection/TestBeelineConnectionUsingHiveSite.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/hs2connection/TestBeelineConnectionUsingHiveSite.java
index fe77667..1ad764b 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/hs2connection/TestBeelineConnectionUsingHiveSite.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/hs2connection/TestBeelineConnectionUsingHiveSite.java
@@ -20,8 +20,6 @@ package org.apache.hive.beeline.hs2connection;
 import java.io.File;
 
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hive.beeline.hs2connection.HS2ConnectionFileParser;
-import org.apache.hive.jdbc.miniHS2.MiniHS2;
 import org.junit.Test;
 
 public class TestBeelineConnectionUsingHiveSite extends TestBeelineWithHS2ConnectionFile {
@@ -83,11 +81,6 @@ public class TestBeelineConnectionUsingHiveSite extends TestBeelineWithHS2Connec
     System.setProperty(JAVA_TRUST_STORE_PASS_PROP, KEY_STORE_TRUST_STORE_PASSWORD);
   }
 
-  @Override
-  protected MiniHS2 getNewMiniHS2() throws Exception {
-    return new MiniHS2(hiveConf, MiniHS2.MiniClusterType.DFS_ONLY, true);
-  }
-
   private void setupHs2() throws Exception {
     confOverlay.put(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname, HS2_HTTP_MODE);
     confOverlay.put(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH.varname, HS2_HTTP_ENDPOINT);

http://git-wip-us.apache.org/repos/asf/hive/blob/9072575e/itests/hive-unit/src/test/java/org/apache/hive/beeline/hs2connection/TestBeelineWithHS2ConnectionFile.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/hs2connection/TestBeelineWithHS2ConnectionFile.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/hs2connection/TestBeelineWithHS2ConnectionFile.java
index 32e9afd..7fc3d52 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/hs2connection/TestBeelineWithHS2ConnectionFile.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/hs2connection/TestBeelineWithHS2ConnectionFile.java
@@ -35,9 +35,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hive.beeline.BeeLine;
-import org.apache.hive.beeline.hs2connection.HS2ConnectionFileParser;
-import org.apache.hive.beeline.hs2connection.HiveSiteHS2ConnectionFileParser;
-import org.apache.hive.beeline.hs2connection.UserHS2ConnectionFileParser;
 import org.apache.hive.jdbc.miniHS2.MiniHS2;
 import org.apache.hive.service.cli.CLIServiceClient;
 import org.apache.hive.service.cli.HiveSQLException;
@@ -45,6 +42,7 @@ import org.apache.hive.service.cli.OperationHandle;
 import org.apache.hive.service.cli.RowSet;
 import org.apache.hive.service.cli.SessionHandle;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -144,9 +142,15 @@ public abstract class TestBeelineWithHS2ConnectionFile {
 
   @BeforeClass
   public static void beforeTest() throws Exception {
+    MiniHS2.cleanupLocalDir();
     Class.forName(MiniHS2.getJdbcDriverName());
   }
 
+  @AfterClass
+  public static void afterClass() throws IOException {
+    MiniHS2.cleanupLocalDir();
+  }
+
   @Before
   public void before() throws Exception {
     DriverManager.setLoginTimeout(0);
@@ -162,7 +166,7 @@ public abstract class TestBeelineWithHS2ConnectionFile {
   }
 
   protected MiniHS2 getNewMiniHS2() throws Exception {
-    return new MiniHS2(hiveConf);
+    return new MiniHS2.Builder().withConf(hiveConf).cleanupLocalDirOnStartup(false).build();
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hive/blob/9072575e/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java
index 34565e9..c84570b 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithMiniHS2.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
@@ -88,6 +89,7 @@ public class TestJdbcWithMiniHS2 {
 
   @BeforeClass
   public static void setupBeforeClass() throws Exception {
+    MiniHS2.cleanupLocalDir();
     HiveConf conf = new HiveConf();
     dataFileDir = conf.get("test.data.files").replace('\\', '/').replace("c:", "");
     kvDataFilePath = new Path(dataFileDir, "kv1.txt");
@@ -184,7 +186,7 @@ public class TestJdbcWithMiniHS2 {
   private static void startMiniHS2(HiveConf conf) throws Exception {
     conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
     conf.setBoolVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED, false);
-    miniHS2 = new MiniHS2(conf);
+    miniHS2 = new MiniHS2.Builder().withConf(conf).cleanupLocalDirOnStartup(false).build();
     Map<String, String> confOverlay = new HashMap<String, String>();
     miniHS2.start(confOverlay);
   }
@@ -195,10 +197,11 @@ public class TestJdbcWithMiniHS2 {
     }
   }
 
-  private static void cleanupMiniHS2() {
+  private static void cleanupMiniHS2() throws IOException {
     if (miniHS2 != null) {
       miniHS2.cleanup();
     }
+    MiniHS2.cleanupLocalDir();
   }
 
   private static void openDefaultConnections() throws Exception {

http://git-wip-us.apache.org/repos/asf/hive/blob/9072575e/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestSSL.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestSSL.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestSSL.java
index 2f4db0d..4036b53 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestSSL.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestSSL.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hive.jdbc.miniHS2.MiniHS2;
 import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Before;
@@ -70,9 +71,15 @@ public class TestSSL {
 
   @BeforeClass
   public static void beforeTest() throws Exception {
+    MiniHS2.cleanupLocalDir();
     Class.forName(MiniHS2.getJdbcDriverName());
   }
 
+  @AfterClass
+  public static void afterClass() throws Exception {
+    MiniHS2.cleanupLocalDir();
+  }
+
   @Before
   public void setUp() throws Exception {
     DriverManager.setLoginTimeout(0);
@@ -80,7 +87,7 @@ public class TestSSL {
       dataFileDir = System.getProperty("test.data.files");
     }
     dataFileDir = dataFileDir.replace('\\', '/').replace("c:", "");
-    miniHS2 = new MiniHS2(conf);
+    miniHS2 = new MiniHS2.Builder().withConf(conf).cleanupLocalDirOnStartup(false).build();
     confOverlay = new HashMap<String, String>();
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9072575e/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestServiceDiscoveryWithMiniHS2.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestServiceDiscoveryWithMiniHS2.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestServiceDiscoveryWithMiniHS2.java
index 907ccb0..e8051e4 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestServiceDiscoveryWithMiniHS2.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestServiceDiscoveryWithMiniHS2.java
@@ -41,7 +41,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestServiceDiscoveryWithMiniHS2 {
-  private static MiniHS2 miniHS2 = null;
+  private MiniHS2 miniHS2 = null;
   private static HiveConf hiveConf;
   private static TestingServer zkServer;
   private static String zkRootNamespace = "hs2test";
@@ -52,6 +52,7 @@ public class TestServiceDiscoveryWithMiniHS2 {
 
   @BeforeClass
   public static void beforeTest() throws Exception {
+    MiniHS2.cleanupLocalDir();
     zkServer = new TestingServer();
     Class.forName(MiniHS2.getJdbcDriverName());
     hiveConf = new HiveConf();
@@ -68,11 +69,12 @@ public class TestServiceDiscoveryWithMiniHS2 {
       zkServer.close();
       zkServer = null;
     }
+    MiniHS2.cleanupLocalDir();
   }
 
   @Before
   public void setUp() throws Exception {
-    miniHS2 = new MiniHS2(hiveConf);
+    miniHS2 = new MiniHS2.Builder().withConf(hiveConf).cleanupLocalDirOnStartup(false).build();
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hive/blob/9072575e/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestXSRFFilter.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestXSRFFilter.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestXSRFFilter.java
index 2b0ffbe..88a403a 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestXSRFFilter.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestXSRFFilter.java
@@ -18,6 +18,7 @@
 
 package org.apache.hive.jdbc;
 
+import java.io.IOException;
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
@@ -34,6 +35,8 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hive.jdbc.miniHS2.MiniHS2;
 import org.apache.hive.jdbc.XsrfHttpRequestInterceptor;
 import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -48,6 +51,15 @@ public class TestXSRFFilter {
 
   private Connection hs2Conn = null;
 
+  @BeforeClass
+  public static void beforeClass() throws IOException {
+    MiniHS2.cleanupLocalDir();
+  }
+
+  @AfterClass
+  public static void afterClass() throws IOException {
+    MiniHS2.cleanupLocalDir();
+  }
 
   // This is not modeled as a @Before, because it needs to be parameterized per-test.
   // If there is a better way to do this, we should do it.
@@ -55,7 +67,7 @@ public class TestXSRFFilter {
     Class.forName(MiniHS2.getJdbcDriverName());
     HiveConf conf = new HiveConf();
     conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
-    miniHS2 = new MiniHS2(conf);
+    miniHS2 = new MiniHS2.Builder().withConf(conf).cleanupLocalDirOnStartup(false).build();
     dataFileDir = conf.get("test.data.files").replace('\\', '/').replace("c:", "");
     kvDataFilePath = new Path(dataFileDir, "kv1.txt");
     Map<String,String> confOverlay = new HashMap<String, String>();

http://git-wip-us.apache.org/repos/asf/hive/blob/9072575e/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index ff640d3..0525335 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -509,7 +509,7 @@ public class QTestUtil {
   }
 
   public QTestUtil(String outDir, String logDir, MiniClusterType clusterType,
-      String confDir, String hadoopVer, String initScript, String cleanupScript,
+      String confDir, String hadzoopVer, String initScript, String cleanupScript,
       boolean useHBaseMetastore, boolean withLlapIo, FsType fsType)
     throws Exception {
     LOG.info("Setting up QTestUtil with outDir={}, logDir={}, clusterType={}, confDir={}," +