You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by wa...@apache.org on 2013/09/20 01:42:22 UTC

svn commit: r1524865 [4/5] - in /hadoop/common/branches/HDFS-4949/hadoop-hdfs-project: hadoop-hdfs-nfs/ hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/mount/ hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/ hadoop-hdfs-nfs/src/t...

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java Thu Sep 19 23:42:10 2013
@@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configured
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -39,8 +40,7 @@ import org.junit.Test;
 public class TestLoadGenerator extends Configured implements Tool {
   private static final Configuration CONF = new HdfsConfiguration();
   private static final int DEFAULT_BLOCK_SIZE = 10;
-  private static final String OUT_DIR = 
-    System.getProperty("test.build.data","build/test/data");
+  private static final File OUT_DIR = PathUtils.getTestDir(TestLoadGenerator.class);
   private static final File DIR_STRUCTURE_FILE = 
     new File(OUT_DIR, StructureGenerator.DIR_STRUCTURE_FILE_NAME);
   private static final File FILE_STRUCTURE_FILE =
@@ -65,7 +65,7 @@ public class TestLoadGenerator extends C
     StructureGenerator sg = new StructureGenerator();
     String[] args = new String[]{"-maxDepth", "2", "-minWidth", "1",
         "-maxWidth", "2", "-numOfFiles", "2",
-        "-avgFileSize", "1", "-outDir", OUT_DIR, "-seed", "1"};
+        "-avgFileSize", "1", "-outDir", OUT_DIR.getAbsolutePath(), "-seed", "1"};
     
     final int MAX_DEPTH = 1;
     final int MIN_WIDTH = 3;
@@ -133,8 +133,7 @@ public class TestLoadGenerator extends C
   public void testLoadGenerator() throws Exception {
     final String TEST_SPACE_ROOT = "/test";
 
-    final String SCRIPT_TEST_DIR = new File(System.getProperty("test.build.data",
-    "/tmp")).getAbsolutePath();
+    final String SCRIPT_TEST_DIR = OUT_DIR.getAbsolutePath();
     String script = SCRIPT_TEST_DIR + "/" + "loadgenscript";
     String script2 = SCRIPT_TEST_DIR + "/" + "loadgenscript2";
     File scriptFile1 = new File(script);
@@ -156,7 +155,7 @@ public class TestLoadGenerator extends C
     try {
       DataGenerator dg = new DataGenerator();
       dg.setConf(CONF);
-      String [] args = new String[] {"-inDir", OUT_DIR, "-root", TEST_SPACE_ROOT};
+      String [] args = new String[] {"-inDir", OUT_DIR.getAbsolutePath(), "-root", TEST_SPACE_ROOT};
       assertEquals(0, dg.run(args));
 
       final int READ_PROBABILITY = 1;

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java Thu Sep 19 23:42:10 2013
@@ -1407,6 +1407,13 @@ public class MiniDFSCluster {
    * Shutdown all the nodes in the cluster.
    */
   public void shutdown() {
+      shutdown(false);
+  }
+    
+  /**
+   * Shutdown all the nodes in the cluster.
+   */
+  public void shutdown(boolean deleteDfsDir) {
     LOG.info("Shutting down the Mini HDFS Cluster");
     if (checkExitOnShutdown)  {
       if (ExitUtil.terminateCalled()) {
@@ -1426,6 +1433,11 @@ public class MiniDFSCluster {
         nameNode = null;
       }
     }
+    if (deleteDfsDir) {
+        base_dir.delete();
+    } else {
+        base_dir.deleteOnExit();
+    }
   }
   
   /**
@@ -2118,7 +2130,7 @@ public class MiniDFSCluster {
    * <li><base directory>/data/data<2*dnIndex + 1></li>
    * <li><base directory>/data/data<2*dnIndex + 2></li>
    * </ol>
-   * 
+   *
    * @param dnIndex datanode index (starts from 0)
    * @param dirIndex directory index (0 or 1). Index 0 provides access to the
    *          first storage directory. Index 1 provides access to the second
@@ -2149,7 +2161,7 @@ public class MiniDFSCluster {
   public static String getDNCurrentDir(File storageDir) {
     return storageDir + "/" + Storage.STORAGE_DIR_CURRENT + "/";
   }
-  
+
   /**
    * Get directory corresponding to block pool directory in the datanode
    * @param storageDir the storage directory of a datanode.
@@ -2255,7 +2267,7 @@ public class MiniDFSCluster {
     }
     return null;
   }
-
+  
   /**
    * Get the block metadata file for a block from a given datanode
    * 
@@ -2343,14 +2355,17 @@ public class MiniDFSCluster {
     } else {
       if (checkDataNodeAddrConfig) {
         conf.setIfUnset(DFS_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
-        conf.setIfUnset(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");
-        conf.setIfUnset(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0");
       } else {
         conf.set(DFS_DATANODE_ADDRESS_KEY, "127.0.0.1:0");
-        conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");
-        conf.set(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0");
       }
     }
+    if (checkDataNodeAddrConfig) {
+      conf.setIfUnset(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");
+      conf.setIfUnset(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0");
+    } else {
+      conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0");
+      conf.set(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0");
+    }
   }
   
   private void addToFile(String p, String address) throws IOException {

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java Thu Sep 19 23:42:10 2013
@@ -71,10 +71,6 @@ public class TestClientReportBadBlock {
 
   @Before
   public void startUpCluster() throws IOException {
-    if (System.getProperty("test.build.data") == null) { // to allow test to be
-      // run outside of Ant
-      System.setProperty("test.build.data", "build/test/data");
-    }
     // disable block scanner
     conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1); 
     

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java Thu Sep 19 23:42:10 2013
@@ -20,9 +20,6 @@ package org.apache.hadoop.hdfs;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.DATA_NODE;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType.NAME_NODE;
 import static org.junit.Assert.*;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java Thu Sep 19 23:42:10 2013
@@ -44,6 +44,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.compress.BZip2Codec;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
@@ -61,9 +62,7 @@ public class TestDFSShell {
   private static final Log LOG = LogFactory.getLog(TestDFSShell.class);
   private static AtomicInteger counter = new AtomicInteger();
 
-  static final String TEST_ROOT_DIR =
-    new Path(System.getProperty("test.build.data","/tmp"))
-    .toString().replace(' ', '+');
+  static final String TEST_ROOT_DIR = PathUtils.getTestDirName(TestDFSShell.class);
 
   static Path writeFile(FileSystem fs, Path f) throws IOException {
     DataOutputStream out = fs.create(f);
@@ -482,12 +481,11 @@ public class TestDFSShell {
     Configuration dstConf = new HdfsConfiguration();
     MiniDFSCluster srcCluster =  null;
     MiniDFSCluster dstCluster = null;
-    String bak = System.getProperty("test.build.data");
+    File bak = new File(PathUtils.getTestDir(getClass()), "dfs_tmp_uri");
+    bak.mkdirs();
     try{
       srcCluster = new MiniDFSCluster.Builder(srcConf).numDataNodes(2).build();
-      File nameDir = new File(new File(bak), "dfs_tmp_uri/");
-      nameDir.mkdirs();
-      System.setProperty("test.build.data", nameDir.toString());
+      dstConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, bak.getAbsolutePath());
       dstCluster = new MiniDFSCluster.Builder(dstConf).numDataNodes(2).build();
       FileSystem srcFs = srcCluster.getFileSystem();
       FileSystem dstFs = dstCluster.getFileSystem();
@@ -559,7 +557,6 @@ public class TestDFSShell {
       ret = ToolRunner.run(shell, argv);
       assertEquals("default works for rm/rmr", 0, ret);
     } finally {
-      System.setProperty("test.build.data", bak);
       if (null != srcCluster) {
         srcCluster.shutdown();
       }

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java Thu Sep 19 23:42:10 2013
@@ -440,7 +440,8 @@ public class TestDatanodeBlockScanner {
     }
   }
   
-  private static final String BASE_PATH = "/data/current/finalized";
+  private static final String BASE_PATH = (new File("/data/current/finalized"))
+      .getAbsolutePath();
   
   @Test
   public void testReplicaInfoParsing() throws Exception {

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java Thu Sep 19 23:42:10 2013
@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.protocol.L
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -75,7 +76,7 @@ public class TestDecommission {
     // Set up the hosts/exclude files.
     localFileSys = FileSystem.getLocal(conf);
     Path workingDir = localFileSys.getWorkingDirectory();
-    Path dir = new Path(workingDir, System.getProperty("test.build.data", "target/test/data") + "/work-dir/decommission");
+    Path dir = new Path(workingDir, PathUtils.getTestDirName(getClass()) + "/work-dir/decommission");
     hostsFile = new Path(dir, "hosts");
     excludeFile = new Path(dir, "exclude");
     

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java Thu Sep 19 23:42:10 2013
@@ -25,6 +25,8 @@ import static org.junit.Assert.fail;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.inOrder;
 import static org.mockito.Mockito.mock;
+
+import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
@@ -73,6 +75,9 @@ public class TestDistributedFileSystem {
     HdfsConfiguration conf;
     if (noXmlDefaults) {
        conf = new HdfsConfiguration(false);
+       String namenodeDir = new File(MiniDFSCluster.getBaseDirectory(), "name").getAbsolutePath();
+       conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, namenodeDir);
+       conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, namenodeDir);     
     } else {
        conf = new HdfsConfiguration();
     }

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFSInputChecker.java Thu Sep 19 23:42:10 2013
@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.LocalFileSys
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.Test;
 
 /**
@@ -260,7 +261,7 @@ public class TestFSInputChecker {
     // create a file and verify that checksum corruption results in 
     // a checksum exception on LocalFS
     
-    String dir = System.getProperty("test.build.data", ".");
+    String dir = PathUtils.getTestDirName(getClass());
     Path file = new Path(dir + "/corruption-test.dat");
     Path crcFile = new Path(dir + "/.corruption-test.dat.crc");
     

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java Thu Sep 19 23:42:10 2013
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
@@ -337,4 +338,47 @@ public class TestFileAppend{
       cluster.shutdown();
     }
   }
+  
+  /** Tests appending after soft-limit expires. */
+  @Test
+  public void testAppendAfterSoftLimit() 
+      throws IOException, InterruptedException {
+    Configuration conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
+    conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
+    //Set small soft-limit for lease
+    final long softLimit = 1L;
+    final long hardLimit = 9999999L;
+
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
+        .build();
+    cluster.setLeasePeriod(softLimit, hardLimit);
+    cluster.waitActive();
+
+    FileSystem fs = cluster.getFileSystem();
+    FileSystem fs2 = new DistributedFileSystem();
+    fs2.initialize(fs.getUri(), conf);
+
+    final Path testPath = new Path("/testAppendAfterSoftLimit");
+    final byte[] fileContents = AppendTestUtil.initBuffer(32);
+
+    // create a new file without closing
+    FSDataOutputStream out = fs.create(testPath);
+    out.write(fileContents);
+
+    //Wait for > soft-limit
+    Thread.sleep(250);
+
+    try {
+      FSDataOutputStream appendStream2 = fs2.append(testPath);
+      appendStream2.write(fileContents);
+      appendStream2.close();
+      assertEquals(fileContents.length, fs.getFileStatus(testPath).getLen());
+    } finally {
+      fs.close();
+      fs2.close();
+      cluster.shutdown();
+    }
+  }
+
 }

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java Thu Sep 19 23:42:10 2013
@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.server.nam
 import org.apache.hadoop.hdfs.util.Holder;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.Test;
 
 /**
@@ -145,7 +146,7 @@ public class TestFileAppendRestart {
 
     String tarFile = System.getProperty("test.cache.data", "build/test/cache")
       + "/" + HADOOP_23_BROKEN_APPEND_TGZ;
-    String testDir = System.getProperty("test.build.data", "build/test/data");
+    String testDir = PathUtils.getTestDirName(getClass());
     File dfsDir = new File(testDir, "image-with-buggy-append");
     if (dfsDir.exists() && !FileUtil.fullyDelete(dfsDir)) {
       throw new IOException("Could not delete dfs directory '" + dfsDir + "'");

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java Thu Sep 19 23:42:10 2013
@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.server.dat
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.log4j.Level;
 import org.junit.Test;
 
@@ -95,7 +96,7 @@ public class TestFileCorruption {
   @Test
   public void testLocalFileCorruption() throws Exception {
     Configuration conf = new HdfsConfiguration();
-    Path file = new Path(System.getProperty("test.build.data"), "corruptFile");
+    Path file = new Path(PathUtils.getTestDirName(getClass()), "corruptFile");
     FileSystem fs = FileSystem.getLocal(conf);
     DataOutputStream dos = fs.create(file);
     dos.writeBytes("original bytes");

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSServerPorts.java Thu Sep 19 23:42:10 2013
@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.server.nam
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.DNS;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.Test;
 
 /**
@@ -53,6 +54,9 @@ public class TestHDFSServerPorts {
   
   // reset default 0.0.0.0 addresses in order to avoid IPv6 problem
   static final String THIS_HOST = getFullHostName() + ":0";
+  
+  private static final File TEST_DATA_DIR = PathUtils.getTestDir(TestHDFSServerPorts.class);
+  
   static {
     DefaultMetricsSystem.setMiniClusterMode(true);
   }
@@ -81,13 +85,6 @@ public class TestHDFSServerPorts {
     }
   }
   
-  /**
-   * Get base directory these tests should run in.
-   */
-  private String getTestingDir() {
-    return System.getProperty("test.build.data", "build/test/data");
-  }
-  
   public NameNode startNameNode() throws IOException {
     return startNameNode(false);
   }
@@ -95,8 +92,7 @@ public class TestHDFSServerPorts {
    * Start the namenode.
    */
   public NameNode startNameNode(boolean withService) throws IOException {
-    String dataDir = getTestingDir();
-    hdfsDir = new File(dataDir, "dfs");
+    hdfsDir = new File(TEST_DATA_DIR, "dfs");
     if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) {
       throw new IOException("Could not delete hdfs directory '" + hdfsDir + "'");
     }
@@ -119,9 +115,8 @@ public class TestHDFSServerPorts {
    * Start the BackupNode
    */
   public BackupNode startBackupNode(Configuration conf) throws IOException {
-    String dataDir = getTestingDir();
     // Set up testing environment directories
-    hdfsDir = new File(dataDir, "backupNode");
+    hdfsDir = new File(TEST_DATA_DIR, "backupNode");
     if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) {
       throw new IOException("Could not delete hdfs directory '" + hdfsDir + "'");
     }
@@ -150,8 +145,7 @@ public class TestHDFSServerPorts {
    */
   public DataNode startDataNode(int index, Configuration config) 
   throws IOException {
-    String dataDir = getTestingDir();
-    File dataNodeDir = new File(dataDir, "data-" + index);
+    File dataNodeDir = new File(TEST_DATA_DIR, "data-" + index);
     config.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataNodeDir.getPath());
 
     String[] args = new String[] {};

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHftpURLTimeouts.java Thu Sep 19 23:42:10 2013
@@ -33,16 +33,11 @@ import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hdfs.web.URLUtils;
-import org.junit.BeforeClass;
+import org.apache.hadoop.hdfs.web.URLConnectionFactory;
 import org.junit.Test;
 
 public class TestHftpURLTimeouts {
-  @BeforeClass
-  public static void setup() {
-    URLUtils.SOCKET_TIMEOUT = 5;
-  }
-  
+
   @Test
   public void testHftpSocketTimeout() throws Exception {
     Configuration conf = new Configuration();
@@ -51,9 +46,11 @@ public class TestHftpURLTimeouts {
         InetAddress.getByName(null).getHostAddress(),
         socket.getLocalPort(),
         null, null, null);
-    boolean timedout = false;
 
     HftpFileSystem fs = (HftpFileSystem)FileSystem.get(uri, conf);
+    fs.connectionFactory = new URLConnectionFactory(5);
+
+    boolean timedout = false;
     try {
       HttpURLConnection conn = fs.openConnection("/", "");
       timedout = false;
@@ -69,6 +66,7 @@ public class TestHftpURLTimeouts {
       assertTrue("read timedout", timedout);
       assertTrue("connect timedout", checkConnectTimeout(fs, false));
     } finally {
+      fs.connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
       fs.close();
     }
   }
@@ -84,6 +82,8 @@ public class TestHftpURLTimeouts {
     boolean timedout = false;
 
     HsftpFileSystem fs = (HsftpFileSystem)FileSystem.get(uri, conf);
+    fs.connectionFactory = new URLConnectionFactory(5);
+    
     try {
       HttpURLConnection conn = null;
       timedout = false;
@@ -100,6 +100,7 @@ public class TestHftpURLTimeouts {
       assertTrue("ssl read connect timedout", timedout);
       assertTrue("connect timedout", checkConnectTimeout(fs, true));
     } finally {
+      fs.connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
       fs.close();
     }
   }
@@ -121,7 +122,7 @@ public class TestHftpURLTimeouts {
           // https will get a read timeout due to SSL negotiation, but
           // a normal http will not, so need to ignore SSL read timeouts
           // until a connect timeout occurs
-          if (!(ignoreReadTimeout && message.equals("Read timed out"))) {
+          if (!(ignoreReadTimeout && "Read timed out".equals(message))) {
             timedout = true;
             assertEquals("connect timed out", message);
           }

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java Thu Sep 19 23:42:10 2013
@@ -25,8 +25,8 @@ import static org.junit.Assume.assumeTru
 import java.io.File;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.FSConstants;
-import org.junit.After;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.test.PathUtils;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -43,20 +43,10 @@ public class TestMiniDFSCluster {
   private static final String CLUSTER_3 = "cluster3";
   private static final String CLUSTER_4 = "cluster4";
   private static final String CLUSTER_5 = "cluster5";
-  protected String testDataPath;
-  protected File testDataDir;
+  protected File testDataPath;
   @Before
   public void setUp() {
-    testDataPath = System.getProperty(MiniDFSCluster.PROP_TEST_BUILD_DATA,
-        "build/test/data");
-    testDataDir = new File(new File(testDataPath).getParentFile(),
-                           "miniclusters");
-
-
-  }
-  @After
-  public void tearDown() {
-    System.setProperty(MiniDFSCluster.PROP_TEST_BUILD_DATA, testDataPath);
+    testDataPath = new File(PathUtils.getTestDir(getClass()), "miniclusters");
   }
 
   /**
@@ -120,7 +110,7 @@ public class TestMiniDFSCluster {
     MiniDFSCluster cluster4 = new MiniDFSCluster.Builder(conf).build();
     try {
       DistributedFileSystem dfs = (DistributedFileSystem) cluster4.getFileSystem();
-      dfs.setSafeMode(FSConstants.SafeModeAction.SAFEMODE_ENTER);
+      dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
       cluster4.shutdown();
     } finally {
       while(cluster4.isClusterUp()){

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java Thu Sep 19 23:42:10 2013
@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.server.nam
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.log4j.Level;
 import org.junit.Test;
 
@@ -322,7 +323,7 @@ public class TestPersistBlocks {
         
     String tarFile = System.getProperty("test.cache.data", "build/test/cache")
       + "/" + HADOOP_1_0_MULTIBLOCK_TGZ;
-    String testDir = System.getProperty("test.build.data", "build/test/data");
+    String testDir = PathUtils.getTestDirName(getClass());
     File dfsDir = new File(testDir, "image-1.0");
     if (dfsDir.exists() && !FileUtil.fullyDelete(dfsDir)) {
       throw new IOException("Could not delete dfs directory '" + dfsDir + "'");

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java Thu Sep 19 23:42:10 2013
@@ -174,6 +174,7 @@ public class TestNNWithQJM {
   public void testMismatchedNNIsRejected() throws Exception {
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
         MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
+    String defaultEditsDir = conf.get(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY);
     conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
         mjc.getQuorumJournalURI("myjournal").toString());
     
@@ -187,7 +188,7 @@ public class TestNNWithQJM {
     
     // Reformat just the on-disk portion
     Configuration onDiskOnly = new Configuration(conf);
-    onDiskOnly.unset(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY);
+    onDiskOnly.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY, defaultEditsDir);
     NameNode.format(onDiskOnly);
 
     // Start the NN - should fail because the JNs are still formatted

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java Thu Sep 19 23:42:10 2013
@@ -46,6 +46,7 @@ import org.apache.hadoop.metrics2.Metric
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.MetricsAsserts;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.Shell;
 import org.junit.After;
 import org.junit.Before;
@@ -61,13 +62,13 @@ public class TestJournalNode {
   private static final NamespaceInfo FAKE_NSINFO = new NamespaceInfo(
       12345, "mycluster", "my-bp", 0L);
 
+  private static File TEST_BUILD_DATA = PathUtils.getTestDir(TestJournalNode.class);
+
   private JournalNode jn;
   private Journal journal; 
   private Configuration conf = new Configuration();
   private IPCLoggerChannel ch;
   private String journalId;
-  private File TEST_BUILD_DATA =
-      new File(System.getProperty("test.build.data", "build/test/data"));
 
   static {
     // Avoid an error when we double-initialize JvmMetrics

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java Thu Sep 19 23:42:10 2013
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertEqu
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
-import java.io.IOException;
 import java.util.Collection;
 
 import org.apache.hadoop.conf.Configuration;

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java Thu Sep 19 23:42:10 2013
@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.server.nam
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -92,8 +93,7 @@ public class TestReplicationPolicy {
 
     FileSystem.setDefaultUri(conf, "hdfs://localhost:0");
     conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
-    File baseDir = new File(System.getProperty(
-        "test.build.data", "build/test/data"), "dfs/");
+    File baseDir = PathUtils.getTestDir(TestReplicationPolicy.class);
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
         new File(baseDir, "name").getPath());
 

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyWithNodeGroup.java Thu Sep 19 23:42:10 2013
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertFal
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -41,6 +40,9 @@ import org.apache.hadoop.hdfs.protocol.H
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
+import org.apache.hadoop.test.PathUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 public class TestReplicationPolicyWithNodeGroup {
@@ -48,10 +50,10 @@ public class TestReplicationPolicyWithNo
   private static final int NUM_OF_DATANODES = 8;
   private static final int NUM_OF_DATANODES_BOUNDARY = 6;
   private static final int NUM_OF_DATANODES_MORE_TARGETS = 12;
-  private static final Configuration CONF = new HdfsConfiguration();
-  private static final NetworkTopology cluster;
-  private static final NameNode namenode;
-  private static final BlockPlacementPolicy replicator;
+  private final Configuration CONF = new HdfsConfiguration();
+  private NetworkTopology cluster;
+  private NameNode namenode;
+  private BlockPlacementPolicy replicator;
   private static final String filename = "/dummyfile.txt";
 
   private final static DatanodeDescriptor dataNodes[] = new DatanodeDescriptor[] {
@@ -94,27 +96,23 @@ public class TestReplicationPolicyWithNo
   private final static DatanodeDescriptor NODE = 
       new DatanodeDescriptor(DFSTestUtil.getDatanodeDescriptor("9.9.9.9", "/d2/r4/n7"));
 
-  static {
-    try {
-      FileSystem.setDefaultUri(CONF, "hdfs://localhost:0");
-      CONF.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
-      // Set properties to make HDFS aware of NodeGroup.
-      CONF.set("dfs.block.replicator.classname", 
-          "org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyWithNodeGroup");
-      CONF.set(CommonConfigurationKeysPublic.NET_TOPOLOGY_IMPL_KEY, 
-          "org.apache.hadoop.net.NetworkTopologyWithNodeGroup");
-      
-      File baseDir = new File(System.getProperty(
-          "test.build.data", "build/test/data"), "dfs/");
-      CONF.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
-          new File(baseDir, "name").getPath());
-      
-      DFSTestUtil.formatNameNode(CONF);
-      namenode = new NameNode(CONF);
-    } catch (IOException e) {
-      e.printStackTrace();
-      throw (RuntimeException)new RuntimeException().initCause(e);
-    }
+  @Before
+  public void setUp() throws Exception {
+    FileSystem.setDefaultUri(CONF, "hdfs://localhost:0");
+    CONF.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
+    // Set properties to make HDFS aware of NodeGroup.
+    CONF.set("dfs.block.replicator.classname", 
+        "org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyWithNodeGroup");
+    CONF.set(CommonConfigurationKeysPublic.NET_TOPOLOGY_IMPL_KEY, 
+        "org.apache.hadoop.net.NetworkTopologyWithNodeGroup");
+    
+    File baseDir = PathUtils.getTestDir(TestReplicationPolicyWithNodeGroup.class);
+    
+    CONF.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        new File(baseDir, "name").getPath());
+    
+    DFSTestUtil.formatNameNode(CONF);
+    namenode = new NameNode(CONF);
     final BlockManager bm = namenode.getNamesystem().getBlockManager();
     replicator = bm.getBlockPlacementPolicy();
     cluster = bm.getDatanodeManager().getNetworkTopology();
@@ -125,6 +123,11 @@ public class TestReplicationPolicyWithNo
     setupDataNodeCapacity();
   }
 
+  @After
+  public void tearDown() throws Exception {
+    namenode.stop();
+  }
+  
   private static void setupDataNodeCapacity() {
     for(int i=0; i<NUM_OF_DATANODES; i++) {
       dataNodes[i].updateHeartbeat(
@@ -638,7 +641,9 @@ public class TestReplicationPolicyWithNo
    */
   @Test
   public void testChooseMoreTargetsThanNodeGroups() throws Exception {
-    // Cleanup nodes in previous tests
+    for(int i=0; i<NUM_OF_DATANODES; i++) {
+      cluster.remove(dataNodes[i]);
+    }
     for(int i=0; i<NUM_OF_DATANODES_BOUNDARY; i++) {
       DatanodeDescriptor node = dataNodesInBoundaryCase[i];
       if (cluster.contains(node)) {

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java Thu Sep 19 23:42:10 2013
@@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.log4j.Level;
 import org.junit.Before;
 import org.junit.Test;
@@ -71,8 +72,7 @@ public class TestBPOfferService {
       TestBPOfferService.class);
   private static final ExtendedBlock FAKE_BLOCK =
     new ExtendedBlock(FAKE_BPID, 12345L);
-  private static final String TEST_BUILD_DATA = System.getProperty(
-    "test.build.data", "build/test/data");
+  private static final File TEST_BUILD_DATA = PathUtils.getTestDir(TestBPOfferService.class);
 
   static {
     ((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java Thu Sep 19 23:42:10 2013
@@ -394,12 +394,12 @@ public class TestDirectoryScanner {
     
     @Override
     public String getBasePath() {
-      return "/base";
+      return (new File("/base")).getAbsolutePath();
     }
     
     @Override
     public String getPath(String bpid) throws IOException {
-      return "/base/current/" + bpid;
+      return (new File("/base/current/" + bpid)).getAbsolutePath();
     }
 
     @Override
@@ -416,8 +416,6 @@ public class TestDirectoryScanner {
       
   void testScanInfoObject(long blockId, File blockFile, File metaFile)
       throws Exception {
-    assertEquals("/base/current/" + BPID_1 + "/finalized",
-        TEST_VOLUME.getFinalizedDir(BPID_1).getAbsolutePath());
     DirectoryScanner.ScanInfo scanInfo =
         new DirectoryScanner.ScanInfo(blockId, blockFile, metaFile, TEST_VOLUME);
     assertEquals(blockId, scanInfo.getBlockId());

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java Thu Sep 19 23:42:10 2013
@@ -62,6 +62,7 @@ import org.mockito.Mockito;
 
 import com.google.common.base.Joiner;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
@@ -555,4 +556,16 @@ public abstract class FSImageTestUtil {
   public static long getNSQuota(FSNamesystem ns) {
     return ns.dir.rootDir.getNsQuota();
   }
+  
+  public static void assertNNFilesMatch(MiniDFSCluster cluster) throws Exception {
+    List<File> curDirs = Lists.newArrayList();
+    curDirs.addAll(FSImageTestUtil.getNameNodeCurrentDirs(cluster, 0));
+    curDirs.addAll(FSImageTestUtil.getNameNodeCurrentDirs(cluster, 1));
+    
+    // Ignore seen_txid file, since the newly bootstrapped standby
+    // will have a higher seen_txid than the one it bootstrapped from.
+    Set<String> ignoredFiles = ImmutableSet.of("seen_txid");
+    FSImageTestUtil.assertParallelFilesAreIdentical(curDirs,
+        ignoredFiles);
+  }
 }

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java Thu Sep 19 23:42:10 2013
@@ -62,6 +62,8 @@ import org.apache.hadoop.net.NetworkTopo
 import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
@@ -101,7 +103,7 @@ import org.apache.log4j.LogManager;
  * Then the benchmark executes the specified number of operations using 
  * the specified number of threads and outputs the resulting stats.
  */
-public class NNThroughputBenchmark {
+public class NNThroughputBenchmark implements Tool {
   private static final Log LOG = LogFactory.getLog(NNThroughputBenchmark.class);
   private static final int BLOCK_SIZE = 16;
   private static final String GENERAL_OPTIONS_USAGE = 
@@ -116,6 +118,8 @@ public class NNThroughputBenchmark {
     // We do not need many handlers, since each thread simulates a handler
     // by calling name-node methods directly
     config.setInt(DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY, 1);
+    // Turn off minimum block size verification
+    config.setInt(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, 0);
     // set exclude file
     config.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE,
       "${hadoop.tmp.dir}/dfs/hosts/exclude");
@@ -130,14 +134,11 @@ public class NNThroughputBenchmark {
     config.set(DFSConfigKeys.DFS_HOSTS, "${hadoop.tmp.dir}/dfs/hosts/include");
     File includeFile = new File(config.get(DFSConfigKeys.DFS_HOSTS, "include"));
     new FileOutputStream(includeFile).close();
-    // Start the NameNode
-    String[] argv = new String[] {};
-    nameNode = NameNode.createNameNode(argv, config);
-    nameNodeProto = nameNode.getRpcServer();
   }
 
   void close() {
-    nameNode.stop();
+    if(nameNode != null)
+      nameNode.stop();
   }
 
   static void setNameNodeLoggingLevel(Level logLevel) {
@@ -1293,52 +1294,69 @@ public class NNThroughputBenchmark {
     System.exit(-1);
   }
 
+  public static void runBenchmark(Configuration conf, List<String> args)
+      throws Exception {
+    NNThroughputBenchmark bench = null;
+    try {
+      bench = new NNThroughputBenchmark(conf);
+      bench.run(args.toArray(new String[]{}));
+    } finally {
+      if(bench != null)
+        bench.close();
+    }
+  }
+
   /**
    * Main method of the benchmark.
    * @param args command line parameters
    */
-  public static void runBenchmark(Configuration conf, List<String> args) throws Exception {
+  @Override // Tool
+  public int run(String[] aArgs) throws Exception {
+    List<String> args = new ArrayList<String>(Arrays.asList(aArgs));
     if(args.size() < 2 || ! args.get(0).startsWith("-op"))
       printUsage();
 
     String type = args.get(1);
     boolean runAll = OperationStatsBase.OP_ALL_NAME.equals(type);
 
-    NNThroughputBenchmark bench = null;
+    // Start the NameNode
+    String[] argv = new String[] {};
+    nameNode = NameNode.createNameNode(argv, config);
+    nameNodeProto = nameNode.getRpcServer();
+
     List<OperationStatsBase> ops = new ArrayList<OperationStatsBase>();
     OperationStatsBase opStat = null;
     try {
-      bench = new NNThroughputBenchmark(conf);
       if(runAll || CreateFileStats.OP_CREATE_NAME.equals(type)) {
-        opStat = bench.new CreateFileStats(args);
+        opStat = new CreateFileStats(args);
         ops.add(opStat);
       }
       if(runAll || OpenFileStats.OP_OPEN_NAME.equals(type)) {
-        opStat = bench.new OpenFileStats(args);
+        opStat = new OpenFileStats(args);
         ops.add(opStat);
       }
       if(runAll || DeleteFileStats.OP_DELETE_NAME.equals(type)) {
-        opStat = bench.new DeleteFileStats(args);
+        opStat = new DeleteFileStats(args);
         ops.add(opStat);
       }
       if(runAll || FileStatusStats.OP_FILE_STATUS_NAME.equals(type)) {
-        opStat = bench.new FileStatusStats(args);
+        opStat = new FileStatusStats(args);
         ops.add(opStat);
       }
       if(runAll || RenameFileStats.OP_RENAME_NAME.equals(type)) {
-        opStat = bench.new RenameFileStats(args);
+        opStat = new RenameFileStats(args);
         ops.add(opStat);
       }
       if(runAll || BlockReportStats.OP_BLOCK_REPORT_NAME.equals(type)) {
-        opStat = bench.new BlockReportStats(args);
+        opStat = new BlockReportStats(args);
         ops.add(opStat);
       }
       if(runAll || ReplicationStats.OP_REPLICATION_NAME.equals(type)) {
-        opStat = bench.new ReplicationStats(args);
+        opStat = new ReplicationStats(args);
         ops.add(opStat);
       }
       if(runAll || CleanAllStats.OP_CLEAN_NAME.equals(type)) {
-        opStat = bench.new CleanAllStats(args);
+        opStat = new CleanAllStats(args);
         ops.add(opStat);
       }
       if(ops.size() == 0)
@@ -1357,14 +1375,28 @@ public class NNThroughputBenchmark {
     } catch(Exception e) {
       LOG.error(StringUtils.stringifyException(e));
       throw e;
+    }
+    return 0;
+  }
+
+  public static void main(String[] args) throws Exception {
+    NNThroughputBenchmark bench = null;
+    try {
+      bench = new NNThroughputBenchmark(new HdfsConfiguration());
+      ToolRunner.run(bench, args);
     } finally {
       if(bench != null)
         bench.close();
     }
   }
 
-  public static void main(String[] args) throws Exception {
-    runBenchmark(new HdfsConfiguration(), 
-                  new ArrayList<String>(Arrays.asList(args)));
+  @Override // Configurable
+  public void setConf(Configuration conf) {
+    config = conf;
+  }
+
+  @Override // Configurable
+  public Configuration getConf() {
+    return config;
   }
 }

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAllowFormat.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAllowFormat.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAllowFormat.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAllowFormat.java Thu Sep 19 23:42:10 2013
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.HdfsConfig
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.TestGenericJournalConf.DummyJournalManager;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -54,37 +55,34 @@ public class TestAllowFormat {
   public static final String NAME_NODE_HTTP_HOST = "0.0.0.0:";
   private static final Log LOG =
     LogFactory.getLog(TestAllowFormat.class.getName());
+  private static final File DFS_BASE_DIR = new File(PathUtils.getTestDir(TestAllowFormat.class), "dfs");
   private static Configuration config;
   private static MiniDFSCluster cluster = null;
-  private static File hdfsDir=null;
 
   @BeforeClass
   public static void setUp() throws Exception {
     config = new Configuration();
-    String baseDir = System.getProperty("test.build.data", "build/test/data");
-
-    hdfsDir = new File(baseDir, "dfs");
-    if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) {
-      throw new IOException("Could not delete hdfs directory '" + hdfsDir +
+    if ( DFS_BASE_DIR.exists() && !FileUtil.fullyDelete(DFS_BASE_DIR) ) {
+      throw new IOException("Could not delete hdfs directory '" + DFS_BASE_DIR +
                             "'");
     }
-
+    
     // Test has multiple name directories.
     // Format should not really prompt us if one of the directories exist,
     // but is empty. So in case the test hangs on an input, it means something
     // could be wrong in the format prompting code. (HDFS-1636)
-    LOG.info("hdfsdir is " + hdfsDir.getAbsolutePath());
-    File nameDir1 = new File(hdfsDir, "name1");
-    File nameDir2 = new File(hdfsDir, "name2");
+    LOG.info("hdfsdir is " + DFS_BASE_DIR.getAbsolutePath());
+    File nameDir1 = new File(DFS_BASE_DIR, "name1");
+    File nameDir2 = new File(DFS_BASE_DIR, "name2");
 
     // To test multiple directory handling, we pre-create one of the name directories.
     nameDir1.mkdirs();
 
     // Set multiple name directories.
     config.set(DFS_NAMENODE_NAME_DIR_KEY, nameDir1.getPath() + "," + nameDir2.getPath());
-    config.set(DFS_DATANODE_DATA_DIR_KEY, new File(hdfsDir, "data").getPath());
+    config.set(DFS_DATANODE_DATA_DIR_KEY, new File(DFS_BASE_DIR, "data").getPath());
 
-    config.set(DFS_NAMENODE_CHECKPOINT_DIR_KEY,new File(hdfsDir, "secondary").getPath());
+    config.set(DFS_NAMENODE_CHECKPOINT_DIR_KEY,new File(DFS_BASE_DIR, "secondary").getPath());
 
     FileSystem.setDefaultUri(config, "hdfs://"+NAME_NODE_HOST + "0");
   }
@@ -99,9 +97,9 @@ public class TestAllowFormat {
       LOG.info("Stopping mini cluster");
     }
     
-    if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) {
+    if ( DFS_BASE_DIR.exists() && !FileUtil.fullyDelete(DFS_BASE_DIR) ) {
       throw new IOException("Could not delete hdfs directory in tearDown '"
-                            + hdfsDir + "'");
+                            + DFS_BASE_DIR + "'");
     }	
   }
 
@@ -170,7 +168,7 @@ public class TestAllowFormat {
     HATestUtil.setFailoverConfigurations(conf, logicalName, nnAddr1, nnAddr2);
 
     conf.set(DFS_NAMENODE_NAME_DIR_KEY,
-        new File(hdfsDir, "name").getAbsolutePath());
+        new File(DFS_BASE_DIR, "name").getAbsolutePath());
     conf.setBoolean(DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_KEY, true);
     conf.set(DFSUtil.addKeySuffixes(
         DFSConfigKeys.DFS_NAMENODE_EDITS_PLUGIN_PREFIX, "dummy"),

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java Thu Sep 19 23:42:10 2013
@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.web.WebHdf
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -58,8 +59,7 @@ import org.junit.Test;
  * A JUnit test that audit logs are generated
  */
 public class TestAuditLogs {
-  static final String auditLogFile = System.getProperty("test.build.dir",
-      "build/test") + "/audit.log";
+  static final String auditLogFile = PathUtils.getTestDirName(TestAuditLogs.class) + "/TestAuditLogs-audit.log";
   
   // Pattern for: 
   // allowed=(true|false) ugi=name ip=/address cmd={cmd} src={path} dst=null perm=null

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java Thu Sep 19 23:42:10 2013
@@ -100,6 +100,8 @@ public class TestBackupNode {
         "${" + DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY + "}");
     c.set(DFSConfigKeys.DFS_NAMENODE_BACKUP_ADDRESS_KEY,
         "127.0.0.1:0");
+    c.set(DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY,
+            "127.0.0.1:0");
 
     BackupNode bn = (BackupNode)NameNode.createNameNode(
         new String[]{startupOpt.getName()}, c);

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java Thu Sep 19 23:42:10 2013
@@ -39,7 +39,6 @@ import java.util.Collection;
 import java.util.List;
 
 import org.apache.commons.cli.ParseException;
-import org.apache.commons.io.filefilter.FileFilterUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
@@ -76,6 +75,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.DelayAnswer;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ExitUtil.ExitException;
 import org.apache.hadoop.util.StringUtils;
@@ -194,7 +194,7 @@ public class TestCheckpoint {
     ArrayList<URI> fsImageDirs = new ArrayList<URI>();
     ArrayList<URI> editsDirs = new ArrayList<URI>();
     File filePath =
-      new File(System.getProperty("test.build.data","/tmp"), "storageDirToCheck");
+      new File(PathUtils.getTestDir(getClass()), "storageDirToCheck");
     assertTrue("Couldn't create directory storageDirToCheck",
                filePath.exists() || filePath.mkdirs());
     fsImageDirs.add(filePath.toURI());
@@ -1224,7 +1224,6 @@ public class TestCheckpoint {
   }
   
   /* Test case to test CheckpointSignature */
-  @SuppressWarnings("deprecation")
   @Test
   public void testCheckpointSignature() throws IOException {
 
@@ -1562,12 +1561,65 @@ public class TestCheckpoint {
       Mockito.reset(faultInjector);
     }
   }
+  
+  /**
+   * Test that a fault while downloading edits the first time after the 2NN
+   * starts up does not prevent future checkpointing.
+   */
+  @Test(timeout = 30000)
+  public void testEditFailureOnFirstCheckpoint() throws IOException {
+    Configuration conf = new HdfsConfiguration();
+    SecondaryNameNode secondary = null;
+    MiniDFSCluster cluster = null;
+    FileSystem fs = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
+          .build();
+      cluster.waitActive();
+      fs = cluster.getFileSystem();
+      fs.mkdirs(new Path("test-file-1"));
+      
+      // Make sure the on-disk fsimage on the NN has txid > 0.
+      FSNamesystem fsns = cluster.getNamesystem();
+      fsns.enterSafeMode(false);
+      fsns.saveNamespace();
+      fsns.leaveSafeMode();
+      
+      secondary = startSecondaryNameNode(conf);
+
+      // Cause edit rename to fail during next checkpoint
+      Mockito.doThrow(new IOException("Injecting failure before edit rename"))
+          .when(faultInjector).beforeEditsRename();
+      
+      try {
+        secondary.doCheckpoint();
+        fail("Fault injection failed.");
+      } catch (IOException ioe) {
+        GenericTestUtils.assertExceptionContains(
+            "Injecting failure before edit rename", ioe);
+      }
+      Mockito.reset(faultInjector);
+      
+      // Next checkpoint should succeed
+      secondary.doCheckpoint();
+    } finally {
+      if (secondary != null) {
+        secondary.shutdown();
+      }
+      if (fs != null) {
+        fs.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+      Mockito.reset(faultInjector);
+    }
+  }
 
   /**
    * Test that the secondary namenode correctly deletes temporary edits
    * on startup.
    */
-
   @Test(timeout = 30000)
   public void testDeleteTemporaryEditsOnStartup() throws IOException {
     Configuration conf = new HdfsConfiguration();
@@ -1860,9 +1912,11 @@ public class TestCheckpoint {
       }
       
       // Start a new NN with the same host/port.
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
-          .nameNodePort(origPort).nameNodeHttpPort(origHttpPort).format(true)
-          .build();
+      cluster = new MiniDFSCluster.Builder(conf)
+          .numDataNodes(0)
+          .nameNodePort(origPort)
+          .nameNodeHttpPort(origHttpPort)
+          .format(true).build();
 
       try {
         secondary.doCheckpoint();
@@ -1943,7 +1997,6 @@ public class TestCheckpoint {
    * Test that, if a storage directory is failed when a checkpoint occurs,
    * the non-failed storage directory receives the checkpoint.
    */
-  @SuppressWarnings("deprecation")
   @Test
   public void testCheckpointWithFailedStorageDir() throws Exception {
     MiniDFSCluster cluster = null;
@@ -2006,7 +2059,6 @@ public class TestCheckpoint {
    * should function correctly.
    * @throws Exception
    */
-  @SuppressWarnings("deprecation")
   @Test
   public void testCheckpointWithSeparateDirsAfterNameFails() throws Exception {
     MiniDFSCluster cluster = null;
@@ -2086,7 +2138,8 @@ public class TestCheckpoint {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY, 1);
     
     try {
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+      cluster = new MiniDFSCluster.Builder(conf)
+          .numDataNodes(0)
           .format(true).build();
       FileSystem fs = cluster.getFileSystem();
       secondary = startSecondaryNameNode(conf);

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterId.java Thu Sep 19 23:42:10 2013
@@ -42,6 +42,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ExitUtil.ExitException;
 import org.junit.After;
@@ -72,7 +73,7 @@ public class TestClusterId {
   public void setUp() throws IOException {
     ExitUtil.disableSystemExit();
 
-    String baseDir = System.getProperty("test.build.data", "build/test/data");
+    String baseDir = PathUtils.getTestDirName(getClass());
 
     hdfsDir = new File(baseDir, "dfs/name");
     if (hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir)) {

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java Thu Sep 19 23:42:10 2013
@@ -169,4 +169,23 @@ public class TestCommitBlockSynchronizat
     namesystemSpy.commitBlockSynchronization(
         lastBlock, genStamp, length, true, false, newTargets, null);
   }
+
+  @Test
+  public void testCommitBlockSynchronizationWithCloseAndNonExistantTarget()
+      throws IOException {
+    INodeFileUnderConstruction file = mock(INodeFileUnderConstruction.class);
+    Block block = new Block(blockId, length, genStamp);
+    FSNamesystem namesystemSpy = makeNameSystemSpy(block, file);
+    DatanodeID[] newTargets = new DatanodeID[]{
+        new DatanodeID("0.0.0.0", "nonexistantHost", "1", 0, 0, 0)};
+
+    ExtendedBlock lastBlock = new ExtendedBlock();
+    namesystemSpy.commitBlockSynchronization(
+        lastBlock, genStamp, length, true,
+        false, newTargets, null);
+
+    // Repeat the call to make sure it returns true
+    namesystemSpy.commitBlockSynchronization(
+        lastBlock, genStamp, length, true, false, newTargets, null);
+  }
 }

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java Thu Sep 19 23:42:10 2013
@@ -69,6 +69,7 @@ import org.apache.hadoop.hdfs.server.nam
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
@@ -96,9 +97,8 @@ public class TestEditLog {
   static final int NUM_TRANSACTIONS = 100;
   static final int NUM_THREADS = 100;
   
-  static final File TEST_DIR = new File(
-    System.getProperty("test.build.data","build/test/data"));
-
+  static final File TEST_DIR = PathUtils.getTestDir(TestEditLog.class);
+  
   /** An edits log with 3 edits from 0.20 - the result of
    * a fresh namesystem followed by hadoop fs -touchz /myfile */
   static final byte[] HADOOP20_SOME_EDITS =
@@ -569,6 +569,7 @@ public class TestEditLog {
       fail("should not be able to start");
     } catch (IOException e) {
       // expected
+      assertNotNull("Cause of exception should be ChecksumException", e.getCause());
       assertEquals("Cause of exception should be ChecksumException",
           ChecksumException.class, e.getCause().getClass());
     }

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileInputStream.java Thu Sep 19 23:42:10 2013
@@ -42,7 +42,8 @@ public class TestEditLogFileInputStream 
   @Test
   public void testReadURL() throws Exception {
     // Start a simple web server which hosts the log data.
-    HttpServer server = new HttpServer("test", "0.0.0.0", 0, true);
+    HttpServer server = new HttpServer.Builder().setName("test")
+        .setBindAddress("0.0.0.0").setPort(0).setFindPort(true).build();
     server.start();
     try {
       server.addServlet("fakeLog", "/fakeLog", FakeLogServlet.class);

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogFileOutputStream.java Thu Sep 19 23:42:10 2013
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertTru
 import java.io.File;
 import java.io.IOException;
 
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.After;
@@ -34,8 +35,7 @@ import org.junit.Test;
  * Test the EditLogFileOutputStream
  */
 public class TestEditLogFileOutputStream {
-  private final static File TEST_DIR =
-      new File(System.getProperty("test.build.data", "/tmp"));
+  private final static File TEST_DIR = PathUtils.getTestDir(TestEditLogFileOutputStream.class);
   private static final File TEST_EDITS =
       new File(TEST_DIR, "testEditLogFileOutput.log");
   final static int MIN_PREALLOCATION_LENGTH =

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java Thu Sep 19 23:42:10 2013
@@ -38,6 +38,7 @@ import org.apache.hadoop.hdfs.MiniDFSClu
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.ExitUtil.ExitException;
 import org.junit.After;
 import org.junit.Before;
@@ -192,8 +193,7 @@ public class TestEditLogJournalFailures 
     Configuration conf = new HdfsConfiguration();
     String[] nameDirs = new String[4];
     for (int i = 0; i < nameDirs.length; i++) {
-      File nameDir = new File(System.getProperty("test.build.data"),
-          "name-dir" + i);
+      File nameDir = new File(PathUtils.getTestDir(getClass()), "name-dir" + i);
       nameDir.mkdirs();
       nameDirs[i] = nameDir.getAbsolutePath();
     }

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java Thu Sep 19 23:42:10 2013
@@ -47,6 +47,7 @@ import org.apache.hadoop.hdfs.server.com
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.test.PathUtils;
 import org.apache.log4j.Level;
 import org.junit.Test;
 
@@ -60,8 +61,7 @@ public class TestFSEditLogLoader {
     ((Log4JLogger)FSEditLogLoader.LOG).getLogger().setLevel(Level.ALL);
   }
   
-  private static final File TEST_DIR = new File(
-      System.getProperty("test.build.data","build/test/data"));
+  private static final File TEST_DIR = PathUtils.getTestDir(TestFSEditLogLoader.class);
 
   private static final int NUM_DATA_NODES = 0;
   

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileJournalManager.java Thu Sep 19 23:42:10 2013
@@ -479,6 +479,6 @@ public class TestFileJournalManager {
 
   private static String getLogsAsString(
       FileJournalManager fjm, long firstTxId) throws IOException {
-    return Joiner.on(",").join(fjm.getRemoteEditLogs(firstTxId, true));
+    return Joiner.on(",").join(fjm.getRemoteEditLogs(firstTxId, true, false));
   }
 }

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java Thu Sep 19 23:42:10 2013
@@ -91,7 +91,7 @@ import static org.mockito.Mockito.*;
  */
 public class TestFsck {
   static final String auditLogFile = System.getProperty("test.build.dir",
-      "build/test") + "/audit.log";
+      "build/test") + "/TestFsck-audit.log";
   
   // Pattern for: 
   // allowed=true ugi=name ip=/address cmd=FSCK src=/ dst=null perm=null
@@ -153,13 +153,14 @@ public class TestFsck {
       String outStr = runFsck(conf, 0, true, "/");
       verifyAuditLogs();
       assertEquals(aTime, fs.getFileStatus(file).getAccessTime());
-      assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
       System.out.println(outStr);
+      assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
       if (fs != null) {try{fs.close();} catch(Exception e){}}
       cluster.shutdown();
       
       // restart the cluster; bring up namenode but not the data nodes
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).format(false).build();
+      cluster = new MiniDFSCluster.Builder(conf)
+          .numDataNodes(0).format(false).build();
       outStr = runFsck(conf, 1, true, "/");
       // expect the result is corrupt
       assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS));
@@ -194,18 +195,30 @@ public class TestFsck {
     // Turn off the logs
     Logger logger = ((Log4JLogger) FSNamesystem.auditLog).getLogger();
     logger.setLevel(Level.OFF);
-    
-    // Audit log should contain one getfileinfo and one fsck
-    BufferedReader reader = new BufferedReader(new FileReader(auditLogFile));
-    String line = reader.readLine();
-    assertNotNull(line);
-    assertTrue("Expected getfileinfo event not found in audit log",
-        getfileinfoPattern.matcher(line).matches());
-    line = reader.readLine();
-    assertNotNull(line);
-    assertTrue("Expected fsck event not found in audit log",
-        fsckPattern.matcher(line).matches());
-    assertNull("Unexpected event in audit log", reader.readLine());
+
+    BufferedReader reader = null;
+    try {
+      // Audit log should contain one getfileinfo and one fsck
+      reader = new BufferedReader(new FileReader(auditLogFile));
+      String line = reader.readLine();
+      assertNotNull(line);
+      assertTrue("Expected getfileinfo event not found in audit log",
+          getfileinfoPattern.matcher(line).matches());
+      line = reader.readLine();
+      assertNotNull(line);
+      assertTrue("Expected fsck event not found in audit log", fsckPattern
+          .matcher(line).matches());
+      assertNull("Unexpected event in audit log", reader.readLine());
+    } finally {
+      // Close the reader and remove the appender to release the audit log file
+      // handle after verifying the content of the file.
+      if (reader != null) {
+        reader.close();
+      }
+      if (logger != null) {
+        logger.removeAllAppenders();
+      }
+    }
   }
   
   @Test
@@ -963,9 +976,9 @@ public class TestFsck {
       String outStr = runFsck(conf, 0, true, "/");
       verifyAuditLogs();
       assertEquals(aTime, fc.getFileStatus(symlink).getAccessTime());
-      assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
-      assertTrue(outStr.contains("Total symlinks:\t\t1\n"));
       System.out.println(outStr);
+      assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
+      assertTrue(outStr.contains("Total symlinks:\t\t1"));
       util.cleanup(fs, fileName);
     } finally {
       if (fs != null) {try{fs.close();} catch(Exception e){}}

Modified: hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java?rev=1524865&r1=1524864&r2=1524865&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java Thu Sep 19 23:42:10 2013
@@ -45,6 +45,7 @@ import org.apache.hadoop.fs.PathIsNotDir
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -901,31 +902,65 @@ public class TestINodeFile {
   @Test
   public void testInodeReplacement() throws Exception {
     final Configuration conf = new Configuration();
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).
-        numDataNodes(1).build();
-    cluster.waitActive();
-    final DistributedFileSystem hdfs = cluster.getFileSystem();
-    final FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
-    
-    final Path dir = new Path("/dir");
-    hdfs.mkdirs(dir);
-    INode dirNode = fsdir.getINode(dir.toString());
-    INode dirNodeFromNode = fsdir.getInode(dirNode.getId());
-    assertSame(dirNode, dirNodeFromNode);
-    
-    // set quota to dir, which leads to node replacement
-    hdfs.setQuota(dir, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
-    dirNode = fsdir.getINode(dir.toString());
-    assertTrue(dirNode instanceof INodeDirectoryWithQuota);
-    // the inode in inodeMap should also be replaced
-    dirNodeFromNode = fsdir.getInode(dirNode.getId());
-    assertSame(dirNode, dirNodeFromNode);
-    
-    hdfs.setQuota(dir, -1, -1);
-    dirNode = fsdir.getINode(dir.toString());
-    assertTrue(dirNode instanceof INodeDirectory);
-    // the inode in inodeMap should also be replaced
-    dirNodeFromNode = fsdir.getInode(dirNode.getId());
-    assertSame(dirNode, dirNodeFromNode);
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      cluster.waitActive();
+      final DistributedFileSystem hdfs = cluster.getFileSystem();
+      final FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+
+      final Path dir = new Path("/dir");
+      hdfs.mkdirs(dir);
+      INode dirNode = fsdir.getINode(dir.toString());
+      INode dirNodeFromNode = fsdir.getInode(dirNode.getId());
+      assertSame(dirNode, dirNodeFromNode);
+
+      // set quota to dir, which leads to node replacement
+      hdfs.setQuota(dir, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
+      dirNode = fsdir.getINode(dir.toString());
+      assertTrue(dirNode instanceof INodeDirectoryWithQuota);
+      // the inode in inodeMap should also be replaced
+      dirNodeFromNode = fsdir.getInode(dirNode.getId());
+      assertSame(dirNode, dirNodeFromNode);
+
+      hdfs.setQuota(dir, -1, -1);
+      dirNode = fsdir.getINode(dir.toString());
+      assertTrue(dirNode instanceof INodeDirectory);
+      // the inode in inodeMap should also be replaced
+      dirNodeFromNode = fsdir.getInode(dirNode.getId());
+      assertSame(dirNode, dirNodeFromNode);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+  
+  @Test
+  public void testDotdotInodePath() throws Exception {
+    final Configuration conf = new Configuration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+      cluster.waitActive();
+      final DistributedFileSystem hdfs = cluster.getFileSystem();
+      final FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+
+      final Path dir = new Path("/dir");
+      hdfs.mkdirs(dir);
+      long dirId = fsdir.getINode(dir.toString()).getId();
+      long parentId = fsdir.getINode("/").getId();
+      String testPath = "/.reserved/.inodes/" + dirId + "/..";
+
+      DFSClient client = new DFSClient(NameNode.getAddress(conf), conf);
+      HdfsFileStatus status = client.getFileInfo(testPath);
+      assertTrue(parentId == status.getFileId());
+      
+      // Test root's parent is still root
+      testPath = "/.reserved/.inodes/" + parentId + "/..";
+      status = client.getFileInfo(testPath);
+      assertTrue(parentId == status.getFileId());
+      
+    } finally {
+      cluster.shutdown();
+    }
   }
 }