You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by gq...@apache.org on 2015/02/10 06:49:17 UTC

incubator-sentry git commit: SENTRY-357: Not able to read policy files on HDFS for Solr (Reviewed by Gregory Chanan)

Repository: incubator-sentry
Updated Branches:
  refs/heads/master 2f579e9e5 -> 08adb2cbc


SENTRY-357: Not able to read policy files on HDFS for Solr (Reviewed by Gregory Chanan)


Project: http://git-wip-us.apache.org/repos/asf/incubator-sentry/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-sentry/commit/08adb2cb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-sentry/tree/08adb2cb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-sentry/diff/08adb2cb

Branch: refs/heads/master
Commit: 08adb2cbc16b2e49721a0eed23bd687490eb2b8f
Parents: 2f579e9
Author: Guoquan Shen <gu...@intel.com>
Authored: Tue Feb 10 13:24:05 2015 +0800
Committer: Guoquan Shen <gu...@intel.com>
Committed: Tue Feb 10 13:24:05 2015 +0800

----------------------------------------------------------------------
 pom.xml                                         |  6 ++
 sentry-binding/sentry-binding-solr/pom.xml      | 22 +++++
 .../binding/solr/authz/SolrAuthzBinding.java    |  4 +-
 .../sentry/binding/solr/HdfsTestUtil.java       | 85 ++++++++++++++++++
 .../binding/solr/TestSolrAuthzBinding.java      | 34 +++++++
 ...adoopGroupResourceAuthorizationProvider.java |  5 ++
 ...adoopGroupResourceAuthorizationProvider.java |  5 ++
 ...LocalGroupResourceAuthorizationProvider.java |  5 ++
 sentry-tests/sentry-tests-solr/pom.xml          | 20 ++---
 .../e2e/solr/AbstractSolrSentryTestBase.java    |  1 +
 .../sentry/tests/e2e/solr/HdfsTestUtil.java     | 95 --------------------
 11 files changed, 170 insertions(+), 112 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/08adb2cb/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 60a9f4a..65c2e75 100644
--- a/pom.xml
+++ b/pom.xml
@@ -349,6 +349,12 @@ limitations under the License.
       </dependency>
       <dependency>
         <groupId>org.apache.sentry</groupId>
+        <artifactId>sentry-binding-solr</artifactId>
+        <version>${project.version}</version>
+        <type>test-jar</type>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.sentry</groupId>
         <artifactId>sentry-provider-common</artifactId>
         <version>${project.version}</version>
       </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/08adb2cb/sentry-binding/sentry-binding-solr/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-solr/pom.xml b/sentry-binding/sentry-binding-solr/pom.xml
index 2dfc933..b3329e2 100644
--- a/sentry-binding/sentry-binding-solr/pom.xml
+++ b/sentry-binding/sentry-binding-solr/pom.xml
@@ -60,6 +60,28 @@ limitations under the License.
       <artifactId>hadoop-common</artifactId>
       <scope>provided</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
+  <!-- build a test jar -->
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>2.2</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/08adb2cb/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SolrAuthzBinding.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SolrAuthzBinding.java b/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SolrAuthzBinding.java
index faf862f..a43a1e0 100644
--- a/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SolrAuthzBinding.java
+++ b/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/authz/SolrAuthzBinding.java
@@ -99,9 +99,9 @@ public class SolrAuthzBinding {
 
     // load the authz provider class
     Constructor<?> constrctor =
-      Class.forName(authProviderName).getDeclaredConstructor(String.class, PolicyEngine.class);
+      Class.forName(authProviderName).getDeclaredConstructor(Configuration.class, String.class, PolicyEngine.class);
     constrctor.setAccessible(true);
-    return (AuthorizationProvider) constrctor.newInstance(new Object[] {resourceName, policyEngine});
+    return (AuthorizationProvider) constrctor.newInstance(new Object[] {authzConf, resourceName, policyEngine});
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/08adb2cb/sentry-binding/sentry-binding-solr/src/test/java/org/apache/sentry/binding/solr/HdfsTestUtil.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-solr/src/test/java/org/apache/sentry/binding/solr/HdfsTestUtil.java b/sentry-binding/sentry-binding-solr/src/test/java/org/apache/sentry/binding/solr/HdfsTestUtil.java
new file mode 100644
index 0000000..859c793
--- /dev/null
+++ b/sentry-binding/sentry-binding-solr/src/test/java/org/apache/sentry/binding/solr/HdfsTestUtil.java
@@ -0,0 +1,85 @@
+package org.apache.sentry.binding.solr;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.Locale;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+/*
+ * 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.
+ */
+
+
+/**
+ * Copied from Apache Solr since the solr-core test jar currently isn't
+ * published
+ */
+public class HdfsTestUtil {
+  private static Locale savedLocale;
+
+  public static MiniDFSCluster setupClass(String dataDir) throws Exception {
+    File dir = new File(dataDir);
+    new File(dataDir).mkdirs();
+
+    savedLocale = Locale.getDefault();
+    // TODO: we HACK around HADOOP-9643
+    Locale.setDefault(Locale.ENGLISH);
+
+    int dataNodes = 2;
+
+    Configuration conf = new Configuration();
+    conf.set("dfs.block.access.token.enable", "false");
+    conf.set("dfs.permissions.enabled", "false");
+    conf.set("hadoop.security.authentication", "simple");
+    conf.set("hdfs.minidfs.basedir", dir.getAbsolutePath() + File.separator + "hdfsBaseDir");
+    conf.set("dfs.namenode.name.dir", dir.getAbsolutePath() + File.separator + "nameNodeNameDir");
+
+    System.setProperty("test.build.data", dir.getAbsolutePath() + File.separator + "hdfs" + File.separator + "build");
+    System.setProperty("test.cache.data", dir.getAbsolutePath() + File.separator + "hdfs" + File.separator + "cache");
+    System.setProperty("solr.lock.type", "hdfs");
+
+    MiniDFSCluster dfsCluster = new MiniDFSCluster(conf, dataNodes, true, null);
+
+    return dfsCluster;
+  }
+
+  public static void teardownClass(MiniDFSCluster dfsCluster) throws Exception {
+    System.clearProperty("solr.lock.type");
+    System.clearProperty("test.build.data");
+    System.clearProperty("test.cache.data");
+    if (dfsCluster != null) {
+      dfsCluster.shutdown();
+    }
+
+    // TODO: we HACK around HADOOP-9643
+    if (savedLocale != null) {
+      Locale.setDefault(savedLocale);
+    }
+  }
+
+  public static String getDataDir(MiniDFSCluster dfsCluster, String dataDir)
+      throws IOException {
+    URI uri = dfsCluster.getURI();
+    String dir = uri.toString()
+        + "/"
+        + new File(dataDir).toString().replaceAll(":", "_")
+            .replaceAll("/", "_");
+    return dir;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/08adb2cb/sentry-binding/sentry-binding-solr/src/test/java/org/apache/sentry/binding/solr/TestSolrAuthzBinding.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-solr/src/test/java/org/apache/sentry/binding/solr/TestSolrAuthzBinding.java b/sentry-binding/sentry-binding-solr/src/test/java/org/apache/sentry/binding/solr/TestSolrAuthzBinding.java
index db5ae29..1bc01a2 100644
--- a/sentry-binding/sentry-binding-solr/src/test/java/org/apache/sentry/binding/solr/TestSolrAuthzBinding.java
+++ b/sentry-binding/sentry-binding-solr/src/test/java/org/apache/sentry/binding/solr/TestSolrAuthzBinding.java
@@ -25,10 +25,13 @@ import java.lang.reflect.InvocationTargetException;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.Set;
+import java.util.UUID;
 
 import junit.framework.Assert;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.sentry.binding.solr.authz.SentrySolrAuthorizationException;
 import org.apache.sentry.binding.solr.authz.SolrAuthzBinding;
 import org.apache.sentry.binding.solr.conf.SolrAuthzConf;
@@ -325,4 +328,35 @@ public class TestSolrAuthzBinding {
      binding.authorizeCollection(general1, generalInfoCollection, allSet);
      binding.authorizeCollection(general1, generalInfoCollection, allOfSet);
   }
+
+  /**
+   * Test that when the resource is put on  HDFS and the scheme of the resource is not set,
+   * the resouce can be found if fs.defaultFS is specified
+   */
+  @Test
+  public void testResourceWithSchemeNotSet() throws Exception {
+    SolrAuthzConf solrAuthzConf =
+        new SolrAuthzConf(Resources.getResource("sentry-site.xml"));
+    setUsableAuthzConf(solrAuthzConf);
+
+    MiniDFSCluster dfsCluster =  HdfsTestUtil.setupClass(new File(Files.createTempDir(),
+      TestSolrAuthzBinding.class.getName() + "_"
+        + System.currentTimeMillis()).getAbsolutePath());
+    String resourceOnHDFS = "/hdfs" + File.separator + UUID.randomUUID() + File.separator + "test-authz-provider.ini";
+    try {
+      Path src = new Path(baseDir.getPath(), RESOURCE_PATH);
+      // Copy resource to HDFSS
+      dfsCluster.getFileSystem().copyFromLocalFile(false,
+          new Path(baseDir.getPath(), RESOURCE_PATH),
+          new Path(resourceOnHDFS));
+      solrAuthzConf.set(AuthzConfVars.AUTHZ_PROVIDER_RESOURCE.getVar(), resourceOnHDFS);
+      // set HDFS as the defaultFS so the resource will be found
+      solrAuthzConf.set("fs.defaultFS", dfsCluster.getFileSystem().getConf().get("fs.defaultFS"));
+      new SolrAuthzBinding(solrAuthzConf);
+    } finally {
+      if (dfsCluster != null) {
+        HdfsTestUtil.teardownClass(dfsCluster);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/08adb2cb/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/HadoopGroupResourceAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/HadoopGroupResourceAuthorizationProvider.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/HadoopGroupResourceAuthorizationProvider.java
index 47ba77a..626fd90 100644
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/HadoopGroupResourceAuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/HadoopGroupResourceAuthorizationProvider.java
@@ -37,6 +37,11 @@ public class HadoopGroupResourceAuthorizationProvider extends
         Groups.getUserToGroupsMappingService(new Configuration())));
   }
 
+  public HadoopGroupResourceAuthorizationProvider(Configuration conf, String resource, PolicyEngine policy) throws IOException {
+    this(policy, new HadoopGroupMappingService(
+        Groups.getUserToGroupsMappingService(conf)));
+  }
+
   @VisibleForTesting
   public HadoopGroupResourceAuthorizationProvider(PolicyEngine policy,
       GroupMappingService groupService) {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/08adb2cb/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/file/HadoopGroupResourceAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/file/HadoopGroupResourceAuthorizationProvider.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/file/HadoopGroupResourceAuthorizationProvider.java
index 1cbc70c..8674700 100644
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/file/HadoopGroupResourceAuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/file/HadoopGroupResourceAuthorizationProvider.java
@@ -19,6 +19,7 @@ package org.apache.sentry.provider.file;
 
 import java.io.IOException;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.sentry.policy.common.PolicyEngine;
 import org.apache.sentry.provider.common.GroupMappingService;
 
@@ -35,6 +36,10 @@ public class HadoopGroupResourceAuthorizationProvider extends
     super(resource, policy);
   }
 
+  public HadoopGroupResourceAuthorizationProvider(Configuration conf, String resource, PolicyEngine policy) throws IOException {
+    super(conf, resource, policy);
+  }
+
   @VisibleForTesting
   public HadoopGroupResourceAuthorizationProvider(PolicyEngine policy,
       GroupMappingService groupService) {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/08adb2cb/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/LocalGroupResourceAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/LocalGroupResourceAuthorizationProvider.java b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/LocalGroupResourceAuthorizationProvider.java
index e66361b..489daf4 100644
--- a/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/LocalGroupResourceAuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-file/src/main/java/org/apache/sentry/provider/file/LocalGroupResourceAuthorizationProvider.java
@@ -19,6 +19,7 @@ package org.apache.sentry.provider.file;
 
 import java.io.IOException;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.sentry.policy.common.PolicyEngine;
 import org.apache.sentry.provider.common.ResourceAuthorizationProvider;
@@ -30,4 +31,8 @@ public class LocalGroupResourceAuthorizationProvider extends
   public LocalGroupResourceAuthorizationProvider(String resource, PolicyEngine policy) throws IOException {
     super(policy, new LocalGroupMappingService(new Path(resource)));
   }
+
+  public LocalGroupResourceAuthorizationProvider(Configuration conf, String resource, PolicyEngine policy) throws IOException {
+    super(policy, new LocalGroupMappingService(conf, new Path(resource)));
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/08adb2cb/sentry-tests/sentry-tests-solr/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-solr/pom.xml b/sentry-tests/sentry-tests-solr/pom.xml
index dfc3792..12fea7c 100644
--- a/sentry-tests/sentry-tests-solr/pom.xml
+++ b/sentry-tests/sentry-tests-solr/pom.xml
@@ -29,13 +29,13 @@ limitations under the License.
   <name>Sentry Solr Tests</name>
   <description>end to end tests for sentry-solr integration</description>
 
-  <properties>
-    <lucene-junit.version>4.10</lucene-junit.version>
-    <carrot-search.version>2.0.10</carrot-search.version>
-  </properties>
-
   <dependencies>
     <dependency>
+      <groupId>org.apache.sentry</groupId>
+      <artifactId>sentry-binding-solr</artifactId>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
       <groupId>org.apache.solr</groupId>
       <artifactId>solr-test-framework</artifactId>
     </dependency>
@@ -67,15 +67,5 @@ limitations under the License.
         </exclusion>
       </exclusions>
     </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <version>${lucene-junit.version}</version> <!-- required for LuceneTestCase -->
-    </dependency>
-    <dependency>
-      <groupId>com.carrotsearch.randomizedtesting</groupId>
-      <artifactId>randomizedtesting-runner</artifactId>
-      <version>${carrot-search.version}</version>
-    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/08adb2cb/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/AbstractSolrSentryTestBase.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/AbstractSolrSentryTestBase.java b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/AbstractSolrSentryTestBase.java
index 6d2550b..ea2b12f 100644
--- a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/AbstractSolrSentryTestBase.java
+++ b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/AbstractSolrSentryTestBase.java
@@ -54,6 +54,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.sentry.binding.solr.HdfsTestUtil;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrServer;
 import org.apache.solr.client.solrj.impl.CloudSolrServer;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/08adb2cb/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/HdfsTestUtil.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/HdfsTestUtil.java b/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/HdfsTestUtil.java
deleted file mode 100644
index bb566bb..0000000
--- a/sentry-tests/sentry-tests-solr/src/test/java/org/apache/sentry/tests/e2e/solr/HdfsTestUtil.java
+++ /dev/null
@@ -1,95 +0,0 @@
-package org.apache.sentry.tests.e2e.solr;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.util.Locale;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.solr.SolrTestCaseJ4;
-
-/*
- * 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.
- */
-
-
-/**
- * Copied from Apache Solr since the solr-core test jar currently isn't
- * published
- */
-public class HdfsTestUtil {
-  
-  private static Locale savedLocale;
-
-  public static MiniDFSCluster setupClass(String dataDir) throws Exception {
-    LuceneTestCase.assumeFalse("HDFS tests were disabled by -Dtests.disableHdfs",
-      Boolean.parseBoolean(System.getProperty("tests.disableHdfs", "false")));
-    File dir = new File(dataDir);
-    new File(dataDir).mkdirs();
-
-    savedLocale = Locale.getDefault();
-    // TODO: we HACK around HADOOP-9643
-    Locale.setDefault(Locale.ENGLISH);
-    
-    int dataNodes = 2;
-    
-    Configuration conf = new Configuration();
-    conf.set("dfs.block.access.token.enable", "false");
-    conf.set("dfs.permissions.enabled", "false");
-    conf.set("hadoop.security.authentication", "simple");
-    conf.set("hdfs.minidfs.basedir", dir.getAbsolutePath() + File.separator + "hdfsBaseDir");
-    conf.set("dfs.namenode.name.dir", dir.getAbsolutePath() + File.separator + "nameNodeNameDir");
-    
-    
-    System.setProperty("test.build.data", dir.getAbsolutePath() + File.separator + "hdfs" + File.separator + "build");
-    System.setProperty("test.cache.data", dir.getAbsolutePath() + File.separator + "hdfs" + File.separator + "cache");
-    System.setProperty("solr.lock.type", "hdfs");
-    
-    MiniDFSCluster dfsCluster = new MiniDFSCluster(conf, dataNodes, true, null);
-    
-    SolrTestCaseJ4.useFactory("org.apache.solr.core.HdfsDirectoryFactory");
-    
-    return dfsCluster;
-  }
-  
-  public static void teardownClass(MiniDFSCluster dfsCluster) throws Exception {
-    SolrTestCaseJ4.resetFactory();
-    System.clearProperty("solr.lock.type");
-    System.clearProperty("test.build.data");
-    System.clearProperty("test.cache.data");
-    if (dfsCluster != null) {
-      dfsCluster.shutdown();
-    }
-    
-    // TODO: we HACK around HADOOP-9643
-    if (savedLocale != null) {
-      Locale.setDefault(savedLocale);
-    }
-  }
-  
-  public static String getDataDir(MiniDFSCluster dfsCluster, String dataDir)
-      throws IOException {
-    URI uri = dfsCluster.getURI();
-    String dir = uri.toString()
-        + "/"
-        + new File(dataDir).toString().replaceAll(":", "_")
-            .replaceAll("/", "_");
-    return dir;
-  }
-
-}