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 to...@apache.org on 2011/07/06 20:32:07 UTC

svn commit: r1143523 [3/4] - in /hadoop/common/branches/HDFS-1073/hdfs: ./ src/c++/libhdfs/ src/contrib/hdfsproxy/ src/java/ src/java/org/apache/hadoop/hdfs/ src/java/org/apache/hadoop/hdfs/protocol/ src/java/org/apache/hadoop/hdfs/protocol/datatransfe...

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java Wed Jul  6 18:32:04 2011
@@ -17,15 +17,20 @@
  */
 package org.apache.hadoop.hdfs.server.protocol;
 
-import java.io.*;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor.BlockTargetPair;
-import org.apache.hadoop.io.*;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableFactories;
+import org.apache.hadoop.io.WritableFactory;
 
 
 /****************************************************

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataTransferProtocolAspects.aj
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataTransferProtocolAspects.aj?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataTransferProtocolAspects.aj (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/DataTransferProtocolAspects.aj Wed Jul  6 18:32:04 2011
@@ -46,7 +46,7 @@ public aspect DataTransferProtocolAspect
   */
 
   pointcut receiverOp(DataXceiver dataxceiver):
-    call(Op Receiver.readOp(DataInputStream)) && target(dataxceiver);
+    call(Op Receiver.readOp()) && target(dataxceiver);
 
   after(DataXceiver dataxceiver) returning(Op op): receiverOp(dataxceiver) {
     LOG.info("FI: receiverOp " + op + ", datanode="

Propchange: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed Jul  6 18:32:04 2011
@@ -1,4 +1,4 @@
-/hadoop/common/trunk/hdfs/src/test/hdfs:1134994-1138149
+/hadoop/common/trunk/hdfs/src/test/hdfs:1134994-1143516
 /hadoop/core/branches/branch-0.19/hdfs/src/test/hdfs:713112
 /hadoop/core/trunk/src/test/hdfs:776175-785643
 /hadoop/hdfs/branches/HDFS-1052/src/test/hdfs:987665-1095512

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/cli/testHDFSConf.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/cli/testHDFSConf.xml?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/cli/testHDFSConf.xml (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/cli/testHDFSConf.xml Wed Jul  6 18:32:04 2011
@@ -15440,7 +15440,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>put: The DiskSpace quota of /dir1 is exceeded: quota=1024 diskspace consumed=[0-9.]+[kmg]*</expected-output>
+          <expected-output>put: The DiskSpace quota of /dir1 is exceeded: quota=1.0k diskspace consumed=[0-9.]+[kmg]*</expected-output>
         </comparator>
       </comparators>
     </test>

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java Wed Jul  6 18:32:04 2011
@@ -61,10 +61,10 @@ import org.apache.hadoop.hdfs.protocol.L
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw;
-import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
@@ -683,8 +683,8 @@ public class DFSTestUtil {
     final DataInputStream in = new DataInputStream(NetUtils.getInputStream(s));
 
     // send the request
-    Sender.opTransferBlock(out, b, dfsClient.clientName,
-        new DatanodeInfo[]{datanodes[1]}, new Token<BlockTokenIdentifier>());
+    new Sender(out).transferBlock(b, new Token<BlockTokenIdentifier>(),
+        dfsClient.clientName, new DatanodeInfo[]{datanodes[1]});
     out.flush();
 
     return BlockOpResponseProto.parseDelimitedFrom(in);

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java Wed Jul  6 18:32:04 2011
@@ -320,7 +320,8 @@ public class MiniDFSCluster {
    * Servers will be started on free ports.
    * <p>
    * The caller must manage the creation of NameNode and DataNode directories
-   * and have already set dfs.namenode.name.dir and dfs.datanode.data.dir in the given conf.
+   * and have already set {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
+   * {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} in the given conf.
    * 
    * @param conf the base configuration to use in starting the servers.  This
    *          will be modified as necessary.
@@ -381,7 +382,8 @@ public class MiniDFSCluster {
   
   /**
    * NOTE: if possible, the other constructors that don't have nameNode port 
-   * parameter should be used as they will ensure that the servers use free ports.
+   * parameter should be used as they will ensure that the servers use free 
+   * ports.
    * <p>
    * Modify the config and start up the servers.  
    * 
@@ -390,9 +392,12 @@ public class MiniDFSCluster {
    * @param conf the base configuration to use in starting the servers.  This
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
-   * @param format if true, format the NameNode and DataNodes before starting up
+   * @param format if true, format the NameNode and DataNodes before starting 
+   *          up
    * @param manageDfsDirs if true, the data directories for servers will be
-   *          created and dfs.namenode.name.dir and dfs.datanode.data.dir will be set in the conf
+   *          created and {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
+   *          {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set in 
+   *          the conf
    * @param operation the operation with which to start the servers.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on
@@ -422,7 +427,9 @@ public class MiniDFSCluster {
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param format if true, format the NameNode and DataNodes before starting up
    * @param manageDfsDirs if true, the data directories for servers will be
-   *          created and dfs.namenode.name.dir and dfs.datanode.data.dir will be set in the conf
+   *          created and {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
+   *          {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set in 
+   *          the conf
    * @param operation the operation with which to start the servers.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on
@@ -454,9 +461,12 @@ public class MiniDFSCluster {
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param format if true, format the NameNode and DataNodes before starting up
    * @param manageNameDfsDirs if true, the data directories for servers will be
-   *          created and dfs.namenode.name.dir and dfs.datanode.data.dir will be set in the conf
+   *          created and {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
+   *          {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set in 
+   *          the conf
    * @param manageDataDfsDirs if true, the data directories for datanodes will
-   *          be created and dfs.datanode.data.dir set to same in the conf
+   *          be created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} 
+   *          set to same in the conf
    * @param operation the operation with which to start the servers.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on
@@ -723,7 +733,8 @@ public class MiniDFSCluster {
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param manageDfsDirs if true, the data directories for DataNodes will be
-   *          created and dfs.datanode.data.dir will be set in the conf
+   *          created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be set 
+   *          in the conf
    * @param operation the operation with which to start the DataNodes.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on
@@ -754,7 +765,8 @@ public class MiniDFSCluster {
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param manageDfsDirs if true, the data directories for DataNodes will be
-   *          created and dfs.datanode.data.dir will be set in the conf
+   *          created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be 
+   *          set in the conf
    * @param operation the operation with which to start the DataNodes.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on
@@ -787,7 +799,8 @@ public class MiniDFSCluster {
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param manageDfsDirs if true, the data directories for DataNodes will be
-   *          created and dfs.datanode.data.dir will be set in the conf
+   *          created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be 
+   *          set in the conf
    * @param operation the operation with which to start the DataNodes.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on
@@ -913,7 +926,8 @@ public class MiniDFSCluster {
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param manageDfsDirs if true, the data directories for DataNodes will be
-   *          created and dfs.datanode.data.dir will be set in the conf
+   *          created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be 
+   *          set in the conf
    * @param operation the operation with which to start the DataNodes.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on
@@ -943,7 +957,8 @@ public class MiniDFSCluster {
    *          will be modified as necessary.
    * @param numDataNodes Number of DataNodes to start; may be zero
    * @param manageDfsDirs if true, the data directories for DataNodes will be
-   *          created and dfs.datanode.data.dir will be set in the conf
+   *          created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will 
+   *          be set in the conf
    * @param operation the operation with which to start the DataNodes.  If null
    *          or StartupOption.FORMAT, then StartupOption.REGULAR will be used.
    * @param racks array of strings indicating the rack that each DataNode is on

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestBlocksScheduledCounter.java Wed Jul  6 18:32:04 2011
@@ -25,7 +25,7 @@ import junit.framework.TestCase;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 
 /**
  * This class tests DatanodeDescriptor.getBlocksScheduled() at the

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestByteRangeInputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestByteRangeInputStream.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestByteRangeInputStream.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestByteRangeInputStream.java Wed Jul  6 18:32:04 2011
@@ -23,12 +23,13 @@ import java.io.InputStream;
 import java.net.HttpURLConnection;
 import java.net.MalformedURLException;
 import java.net.URL;
-import junit.framework.TestCase;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+
 import org.apache.hadoop.hdfs.ByteRangeInputStream;
 import org.apache.hadoop.hdfs.ByteRangeInputStream.URLOpener;
 
+import org.junit.Test;
+import static org.junit.Assert.*;
+
 class MockHttpURLConnection extends HttpURLConnection {
   MockURL m;
   
@@ -101,13 +102,9 @@ class MockURL extends URLOpener {
   }
 }
 
-
-
-public class TestByteRangeInputStream extends TestCase {
-  
-  private static final Log LOG = 
-                           LogFactory.getLog(TestByteRangeInputStream.class);
+public class TestByteRangeInputStream {
   
+  @Test
   public void testByteRange() throws IOException, InterruptedException {
     MockURL o = new MockURL("http://test/");
     MockURL r =  new MockURL((URL)null);
@@ -149,7 +146,7 @@ public class TestByteRangeInputStream ex
     is.seek(101);
     is.read();
 
-    assertNull("Seek to 101 should not result in another request", null);
+    assertNull("Seek to 101 should not result in another request", r.getMsg());
 
     r.setMsg(null);
     is.seek(2500);
@@ -168,7 +165,7 @@ public class TestByteRangeInputStream ex
            + "but 206 is expected");
     } catch (IOException e) {
       assertEquals("Should fail because incorrect response code was sent",
-                   "206 expected, but received 200", e.getMessage());
+                   "HTTP_PARTIAL expected, received 200", e.getMessage());
     }
 
     r.responseCode = 206;
@@ -180,10 +177,7 @@ public class TestByteRangeInputStream ex
            + "but 200 is expected");
     } catch (IOException e) {
       assertEquals("Should fail because incorrect response code was sent",
-                   "200 expected, but received 206", e.getMessage());
+                   "HTTP_OK expected, received 206", e.getMessage());
     }
-
-
-
   }
 }

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSPermission.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSPermission.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSPermission.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSPermission.java Wed Jul  6 18:32:04 2011
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
-import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
@@ -31,7 +30,6 @@ import junit.framework.TestCase;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -68,7 +66,6 @@ public class TestDFSPermission extends T
   final static private int NUM_TEST_PERMISSIONS = 
     conf.getInt("test.dfs.permission.num", 10) * (MAX_PERMISSION + 1) / 100;
 
-
   final private static String PATH_NAME = "xx";
   final private static Path FILE_DIR_PATH = new Path("/", PATH_NAME);
   final private static Path NON_EXISTENT_PATH = new Path("/parent", PATH_NAME);
@@ -115,44 +112,66 @@ public class TestDFSPermission extends T
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     try {
       cluster.waitActive();
-      fs = FileSystem.get(conf);
-      LOG.info("ROOT=" + fs.getFileStatus(new Path("/")));
       testPermissionSetting(OpType.CREATE); // test file creation
       testPermissionSetting(OpType.MKDIRS); // test directory creation
     } finally {
-      fs.close();
       cluster.shutdown();
     }
   }
 
+  private void initFileSystem(short umask) throws Exception {
+    // set umask in configuration, converting to padded octal
+    conf.set(FsPermission.UMASK_LABEL, String.format("%1$03o", umask));
+    fs = FileSystem.get(conf);
+  }
+
+  private void closeFileSystem() throws Exception {
+    fs.close();
+  }
+  
   /* check permission setting works correctly for file or directory */
   private void testPermissionSetting(OpType op) throws Exception {
+    short uMask = DEFAULT_UMASK;
     // case 1: use default permission but all possible umasks
     PermissionGenerator generator = new PermissionGenerator(r);
+    FsPermission permission = new FsPermission(DEFAULT_PERMISSION);
     for (short i = 0; i < NUM_TEST_PERMISSIONS; i++) {
-      createAndCheckPermission(op, FILE_DIR_PATH, generator.next(),
-          new FsPermission(DEFAULT_PERMISSION), true);
+      uMask = generator.next();
+      initFileSystem(uMask);
+      createAndCheckPermission(op, FILE_DIR_PATH, uMask, permission, true);
+      closeFileSystem();
     }
-
     // case 2: use permission 0643 and the default umask
-    createAndCheckPermission(op, FILE_DIR_PATH, DEFAULT_UMASK,
-        new FsPermission((short) 0643), true);
+    uMask = DEFAULT_UMASK;
+    initFileSystem(uMask);
+    createAndCheckPermission(op, FILE_DIR_PATH, uMask, new FsPermission(
+        (short) 0643), true);
+    closeFileSystem();
 
     // case 3: use permission 0643 and umask 0222
-    createAndCheckPermission(op, FILE_DIR_PATH, (short) 0222, 
-        new FsPermission((short) 0643), false);
+    uMask = (short) 0222;
+    initFileSystem(uMask);
+    createAndCheckPermission(op, FILE_DIR_PATH, uMask, new FsPermission(
+        (short) 0643), false);
+    closeFileSystem();
 
     // case 4: set permission
-    fs.setPermission(FILE_DIR_PATH, new FsPermission((short) 0111));
+    uMask = (short) 0111;
+    initFileSystem(uMask);
+    fs.setPermission(FILE_DIR_PATH, new FsPermission(uMask));
     short expectedPermission = (short) ((op == OpType.CREATE) ? 0 : 0111);
     checkPermission(FILE_DIR_PATH, expectedPermission, true);
+    closeFileSystem();
 
     // case 5: test non-existent parent directory
-    assertFalse(fs.exists(NON_EXISTENT_PATH));
-    createAndCheckPermission(op, NON_EXISTENT_PATH, DEFAULT_UMASK,
-        new FsPermission(DEFAULT_PERMISSION), false);
+    uMask = DEFAULT_UMASK;
+    initFileSystem(uMask);
+    assertFalse("File shouldn't exists", fs.exists(NON_EXISTENT_PATH));
+    createAndCheckPermission(op, NON_EXISTENT_PATH, uMask, new FsPermission(
+        DEFAULT_PERMISSION), false);
     Path parent = NON_EXISTENT_PATH.getParent();
     checkPermission(parent, getPermission(parent.getParent()), true);
+    closeFileSystem();
   }
 
   /* get the permission of a file/directory */

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSStorageStateRecovery.java Wed Jul  6 18:32:04 2011
@@ -117,7 +117,8 @@ public class TestDFSStorageStateRecovery
   
   /**
    * Sets up the storage directories for namenode as defined by
-   * dfs.namenode.name.dir. For each element in dfs.namenode.name.dir, the subdirectories 
+   * {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY}. For each element 
+   * in {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY}, the subdirectories 
    * represented by the first four elements of the <code>state</code> array
    * will be created and populated.
    * 
@@ -145,7 +146,8 @@ public class TestDFSStorageStateRecovery
   
   /**
    * Sets up the storage directories for a datanode under
-   * dfs.datanode.data.dir. For each element in dfs.datanode.data.dir, the subdirectories 
+   * {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY}. For each element in 
+   * {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY}, the subdirectories 
    * represented by the first four elements of the <code>state</code> array 
    * will be created and populated. 
    * See {@link UpgradeUtilities#createDataNodeStorageDirs()}
@@ -172,7 +174,8 @@ public class TestDFSStorageStateRecovery
   
   /**
    * Sets up the storage directories for a block pool under
-   * dfs.datanode.data.dir. For each element in dfs.datanode.data.dir, the subdirectories 
+   * {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY}. For each element 
+   * in {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY}, the subdirectories 
    * represented by the first four elements of the <code>state</code> array 
    * will be created and populated. 
    * See {@link UpgradeUtilities#createBlockPoolStorageDirs()}

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgrade.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgrade.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgrade.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDFSUpgrade.java Wed Jul  6 18:32:04 2011
@@ -22,14 +22,12 @@ import static org.apache.hadoop.hdfs.ser
 
 import java.io.File;
 import java.io.IOException;
-
-import junit.framework.TestCase;
+import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
@@ -37,6 +35,9 @@ import org.apache.hadoop.hdfs.server.nam
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
 
+import org.apache.hadoop.util.StringUtils;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import com.google.common.base.Joiner;
@@ -51,8 +52,7 @@ import static org.junit.Assert.*;
 public class TestDFSUpgrade {
  
   private static final int EXPECTED_TXID = 17;
-  private static final Log LOG = LogFactory.getLog(
-                                                   "org.apache.hadoop.hdfs.TestDFSUpgrade");
+  private static final Log LOG = LogFactory.getLog(TestDFSUpgrade.class.getName());
   private Configuration conf;
   private int testCounter = 0;
   private MiniDFSCluster cluster = null;
@@ -125,11 +125,27 @@ public class TestDFSUpgrade {
       
     }
   }
+
   /**
    * Attempts to start a NameNode with the given operation.  Starting
    * the NameNode should throw an exception.
    */
   void startNameNodeShouldFail(StartupOption operation) {
+    startNameNodeShouldFail(operation, null, null);
+  }
+
+  /**
+   * Attempts to start a NameNode with the given operation.  Starting
+   * the NameNode should throw an exception.
+   * @param operation - NameNode startup operation
+   * @param exceptionClass - if non-null, will check that the caught exception
+   *     is assignment-compatible with exceptionClass
+   * @param messagePattern - if non-null, will check that a substring of the 
+   *     message from the caught exception matches this pattern, via the
+   *     {@link Matcher#find()} method.
+   */
+  void startNameNodeShouldFail(StartupOption operation,
+      Class<? extends Exception> exceptionClass, Pattern messagePattern) {
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
                                                 .startupOption(operation)
@@ -137,9 +153,23 @@ public class TestDFSUpgrade {
                                                 .manageDataDfsDirs(false)
                                                 .manageNameDfsDirs(false)
                                                 .build(); // should fail
-      throw new AssertionError("NameNode should have failed to start");
-    } catch (Exception expected) {
-      // expected
+      fail("NameNode should have failed to start");
+      
+    } catch (Exception e) {
+      // expect exception
+      if (exceptionClass != null) {
+        assertTrue("Caught exception is not of expected class "
+            + exceptionClass.getSimpleName() + ": "
+            + StringUtils.stringifyException(e), 
+            exceptionClass.isInstance(e));
+      }
+      if (messagePattern != null) {
+        assertTrue("Caught exception message string does not match expected pattern \""
+            + messagePattern.pattern() + "\" : "
+            + StringUtils.stringifyException(e), 
+            messagePattern.matcher(e.getMessage()).find());
+      }
+      LOG.info("Successfully detected expected NameNode startup failure.");
     }
   }
   
@@ -169,6 +199,11 @@ public class TestDFSUpgrade {
                                            .build();
   }
   
+  @BeforeClass
+  public static void initialize() throws Exception {
+    UpgradeUtilities.initialize();
+  }
+  
   /**
    * This test attempts to upgrade the NameNode and DataNode under
    * a number of valid and invalid conditions.
@@ -176,8 +211,6 @@ public class TestDFSUpgrade {
   @Test
   public void testUpgrade() throws Exception {
     File[] baseDirs;
-    UpgradeUtilities.initialize();
-    
     StorageInfo storageInfo = null;
     for (int numDirs = 1; numDirs <= 2; numDirs++) {
       conf = new HdfsConfiguration();
@@ -320,6 +353,30 @@ public class TestDFSUpgrade {
       UpgradeUtilities.createEmptyDirs(nameNodeDirs);
     }
   }
+  
+  /*
+   * Stand-alone test to detect failure of one SD during parallel upgrade.
+   * At this time, can only be done with manual hack of {@link FSImage.doUpgrade()}
+   */
+  @Ignore
+  public void testUpgrade4() throws Exception {
+    int numDirs = 4;
+    conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);      
+    conf = UpgradeUtilities.initializeStorageStateConf(numDirs, conf);
+    String[] nameNodeDirs = conf.getStrings(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY);
+
+    log("NameNode upgrade with one bad storage dir", numDirs);
+    UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
+    try {
+      // assert("storage dir has been prepared for failure before reaching this point");
+      startNameNodeShouldFail(StartupOption.UPGRADE, IOException.class,
+          Pattern.compile("failed in 1 storage"));
+    } finally {
+      // assert("storage dir shall be returned to normal state before exiting");
+      UpgradeUtilities.createEmptyDirs(nameNodeDirs);
+    }
+  }
  
   private void deleteStorageFilesWithPrefix(String[] nameNodeDirs, String prefix)
   throws Exception {
@@ -342,6 +399,7 @@ public class TestDFSUpgrade {
     fail("Expected IOException is not thrown");
   }
   
+  @Ignore
   public void test203LayoutVersion() {
     for (int lv : Storage.LAYOUT_VERSIONS_203) {
       assertTrue(Storage.is203LayoutVersion(lv));
@@ -349,7 +407,9 @@ public class TestDFSUpgrade {
   }
   
   public static void main(String[] args) throws Exception {
-    new TestDFSUpgrade().testUpgrade();
+    TestDFSUpgrade t = new TestDFSUpgrade();
+    TestDFSUpgrade.initialize();
+    t.testUpgrade();
   }
 }
 

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestDataTransferProtocol.java Wed Jul  6 18:32:04 2011
@@ -72,7 +72,8 @@ public class TestDataTransferProtocol ex
   DatanodeID datanode;
   InetSocketAddress dnAddr;
   ByteArrayOutputStream sendBuf = new ByteArrayOutputStream(128);
-  DataOutputStream sendOut = new DataOutputStream(sendBuf);
+  final DataOutputStream sendOut = new DataOutputStream(sendBuf);
+  final Sender sender = new Sender(sendOut);
   ByteArrayOutputStream recvBuf = new ByteArrayOutputStream(128);
   DataOutputStream recvOut = new DataOutputStream(recvBuf);
 
@@ -185,9 +186,9 @@ public class TestDataTransferProtocol ex
       String description, Boolean eofExcepted) throws IOException {
     sendBuf.reset();
     recvBuf.reset();
-    Sender.opWriteBlock(sendOut, block, 0,
-        stage, newGS, block.getNumBytes(), block.getNumBytes(), "cl", null,
-        new DatanodeInfo[1], BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.writeBlock(block, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        new DatanodeInfo[1], null, stage,
+        0, block.getNumBytes(), block.getNumBytes(), newGS);
     if (eofExcepted) {
       sendResponse(Status.ERROR, null, recvOut);
       sendRecvData(description, true);
@@ -372,10 +373,11 @@ public class TestDataTransferProtocol ex
     
     /* Test OP_WRITE_BLOCK */
     sendBuf.reset();
-    Sender.opWriteBlock(sendOut, 
-        new ExtendedBlock(poolId, newBlockId), 0,
-        BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L, 0L, 0L, "cl", null,
-        new DatanodeInfo[1], BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.writeBlock(new ExtendedBlock(poolId, newBlockId),
+        BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        new DatanodeInfo[1], null,
+        BlockConstructionStage.PIPELINE_SETUP_CREATE,
+        0, 0L, 0L, 0L);
     sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
     
     // bad bytes per checksum
@@ -386,10 +388,10 @@ public class TestDataTransferProtocol ex
 
     sendBuf.reset();
     recvBuf.reset();
-    Sender.opWriteBlock(sendOut,
-        new ExtendedBlock(poolId, ++newBlockId), 0,
-        BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L, 0L, 0L, "cl", null,
-        new DatanodeInfo[1], BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.writeBlock(new ExtendedBlock(poolId, ++newBlockId),
+        BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        new DatanodeInfo[1], null,
+        BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0L, 0L, 0L);
     sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
     sendOut.writeInt(512);
 
@@ -409,10 +411,10 @@ public class TestDataTransferProtocol ex
     // test for writing a valid zero size block
     sendBuf.reset();
     recvBuf.reset();
-    Sender.opWriteBlock(sendOut, 
-        new ExtendedBlock(poolId, ++newBlockId), 0,
-        BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L, 0L, 0L, "cl", null,
-        new DatanodeInfo[1], BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.writeBlock(new ExtendedBlock(poolId, ++newBlockId),
+        BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        new DatanodeInfo[1], null,
+        BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0L, 0L, 0L);
     sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
     sendOut.writeInt(512);         // checksum size
 
@@ -439,22 +441,22 @@ public class TestDataTransferProtocol ex
     sendBuf.reset();
     recvBuf.reset();
     blk.setBlockId(blkid-1);
-    Sender.opReadBlock(sendOut, blk, 0L, fileLen, "cl",
-          BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        0L, fileLen);
     sendRecvData("Wrong block ID " + newBlockId + " for read", false); 
 
     // negative block start offset -1L
     sendBuf.reset();
     blk.setBlockId(blkid);
-    Sender.opReadBlock(sendOut, blk, -1L, fileLen, "cl",
-          BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        -1L, fileLen);
     sendRecvData("Negative start-offset for read for block " + 
                  firstBlock.getBlockId(), false);
 
     // bad block start offset
     sendBuf.reset();
-    Sender.opReadBlock(sendOut, blk, fileLen, fileLen, "cl",
-          BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        fileLen, fileLen);
     sendRecvData("Wrong start-offset for reading block " +
                  firstBlock.getBlockId(), false);
     
@@ -462,8 +464,8 @@ public class TestDataTransferProtocol ex
     recvBuf.reset();
     sendResponse(Status.SUCCESS, null, recvOut);
     sendBuf.reset();
-    Sender.opReadBlock(sendOut, blk, 0L, 
-        -1 - random.nextInt(oneMil), "cl", BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        0L, -1L-random.nextInt(oneMil));
     sendRecvData("Negative length for reading block " +
                  firstBlock.getBlockId(), false);
     
@@ -471,15 +473,15 @@ public class TestDataTransferProtocol ex
     recvBuf.reset();
     sendResponse(Status.ERROR, null, recvOut);
     sendBuf.reset();
-    Sender.opReadBlock(sendOut, blk, 0L, 
-        fileLen + 1, "cl", BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        0L, fileLen+1);
     sendRecvData("Wrong length for reading block " +
                  firstBlock.getBlockId(), false);
     
     //At the end of all this, read the file to make sure that succeeds finally.
     sendBuf.reset();
-    Sender.opReadBlock(sendOut, blk, 0L, 
-        fileLen, "cl", BlockTokenSecretManager.DUMMY_TOKEN);
+    sender.readBlock(blk, BlockTokenSecretManager.DUMMY_TOKEN, "cl",
+        0L, fileLen);
     readFile(fileSys, file, fileLen);
     } finally {
       cluster.shutdown();

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestQuota.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestQuota.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestQuota.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/TestQuota.java Wed Jul  6 18:32:04 2011
@@ -54,6 +54,23 @@ public class TestQuota {
     }
   }
   
+  /**
+   * Tests to make sure we're getting human readable Quota exception messages
+   * Test for @link{ NSQuotaExceededException, DSQuotaExceededException}
+   * @throws Exception
+   */
+  @Test
+  public void testDSQuotaExceededExceptionIsHumanReadable() throws Exception {
+    Integer bytes = 1024;
+    try {
+      throw new DSQuotaExceededException(bytes, bytes);
+    } catch(DSQuotaExceededException e) {
+      
+      assertEquals("The DiskSpace quota is exceeded: quota=1.0k " +
+          "diskspace consumed=1.0k", e.getMessage());
+    }
+  }
+  
   /** Test quota related commands: 
    *    setQuota, clrQuota, setSpaceQuota, clrSpaceQuota, and count 
    */
@@ -695,7 +712,57 @@ public class TestQuota {
       // verify increase in space
       c = dfs.getContentSummary(dstPath);
       assertEquals(c.getSpaceConsumed(), 5 * fileSpace + file2Len);
-      
+
+      // Test HDFS-2053 :
+
+      // Create directory /hdfs-2053
+      final Path quotaDir2053 = new Path("/hdfs-2053");
+      assertTrue(dfs.mkdirs(quotaDir2053));
+
+      // Create subdirectories /hdfs-2053/{A,B,C}
+      final Path quotaDir2053_A = new Path(quotaDir2053, "A");
+      assertTrue(dfs.mkdirs(quotaDir2053_A));
+      final Path quotaDir2053_B = new Path(quotaDir2053, "B");
+      assertTrue(dfs.mkdirs(quotaDir2053_B));
+      final Path quotaDir2053_C = new Path(quotaDir2053, "C");
+      assertTrue(dfs.mkdirs(quotaDir2053_C));
+
+      // Factors to vary the sizes of test files created in each subdir.
+      // The actual factors are not really important but they allow us to create
+      // identifiable file sizes per subdir, which helps during debugging.
+      int sizeFactorA = 1;
+      int sizeFactorB = 2;
+      int sizeFactorC = 4;
+
+      // Set space quota for subdirectory C
+      dfs.setQuota(quotaDir2053_C, FSConstants.QUOTA_DONT_SET,
+          (sizeFactorC + 1) * fileSpace);
+      c = dfs.getContentSummary(quotaDir2053_C);
+      assertEquals(c.getSpaceQuota(), (sizeFactorC + 1) * fileSpace);
+
+      // Create a file under subdirectory A
+      DFSTestUtil.createFile(dfs, new Path(quotaDir2053_A, "fileA"),
+          sizeFactorA * fileLen, replication, 0);
+      c = dfs.getContentSummary(quotaDir2053_A);
+      assertEquals(c.getSpaceConsumed(), sizeFactorA * fileSpace);
+
+      // Create a file under subdirectory B
+      DFSTestUtil.createFile(dfs, new Path(quotaDir2053_B, "fileB"),
+          sizeFactorB * fileLen, replication, 0);
+      c = dfs.getContentSummary(quotaDir2053_B);
+      assertEquals(c.getSpaceConsumed(), sizeFactorB * fileSpace);
+
+      // Create a file under subdirectory C (which has a space quota)
+      DFSTestUtil.createFile(dfs, new Path(quotaDir2053_C, "fileC"),
+          sizeFactorC * fileLen, replication, 0);
+      c = dfs.getContentSummary(quotaDir2053_C);
+      assertEquals(c.getSpaceConsumed(), sizeFactorC * fileSpace);
+
+      // Check space consumed for /hdfs-2053
+      c = dfs.getContentSummary(quotaDir2053);
+      assertEquals(c.getSpaceConsumed(),
+          (sizeFactorA + sizeFactorB + sizeFactorC) * fileSpace);
+
     } finally {
       cluster.shutdown();
     }

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/UpgradeUtilities.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/UpgradeUtilities.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/UpgradeUtilities.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/UpgradeUtilities.java Wed Jul  6 18:32:04 2011
@@ -183,8 +183,9 @@ public class UpgradeUtilities {
   }
   
   /**
-   * Initialize dfs.namenode.name.dir and dfs.datanode.data.dir with the specified number of
-   * directory entries. Also initialize dfs.blockreport.intervalMsec.
+   * Initialize {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} and 
+   * {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} with the specified 
+   * number of directory entries. Also initialize dfs.blockreport.intervalMsec.
    */
   public static Configuration initializeStorageStateConf(int numDirs,
                                                          Configuration conf) {
@@ -308,7 +309,8 @@ public class UpgradeUtilities {
   }
   
   /**
-   * Simulate the <code>dfs.namenode.name.dir</code> of a populated DFS filesystem.
+   * Simulate the {@link DFSConfigKeys#DFS_NAMENODE_NAME_DIR_KEY} of a populated 
+   * DFS filesystem.
    * This method populates for each parent directory, <code>parent/dirName</code>
    * with the content of namenode storage directory that comes from a singleton
    * namenode master (that contains edits, fsimage, version and time files). 
@@ -335,7 +337,8 @@ public class UpgradeUtilities {
   }  
   
   /**
-   * Simulate the <code>dfs.datanode.data.dir</code> of a populated DFS filesystem.
+   * Simulate the {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} of a 
+   * populated DFS filesystem.
    * This method populates for each parent directory, <code>parent/dirName</code>
    * with the content of datanode storage directory that comes from a singleton
    * datanode master (that contains version and block files). If the destination
@@ -362,7 +365,8 @@ public class UpgradeUtilities {
   }
   
   /**
-   * Simulate the <code>dfs.datanode.data.dir</code> of a populated DFS filesystem.
+   * Simulate the {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} of a 
+   * populated DFS filesystem.
    * This method populates for each parent directory, <code>parent/dirName</code>
    * with the content of block pool storage directory that comes from a singleton
    * datanode master (that contains version and block files). If the destination

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java Wed Jul  6 18:32:04 2011
@@ -258,8 +258,8 @@ public class TestBlockReplacement extend
     sock.setKeepAlive(true);
     // sendRequest
     DataOutputStream out = new DataOutputStream(sock.getOutputStream());
-    Sender.opReplaceBlock(out, block, source
-        .getStorageID(), sourceProxy, BlockTokenSecretManager.DUMMY_TOKEN);
+    new Sender(out).replaceBlock(block, BlockTokenSecretManager.DUMMY_TOKEN,
+        source.getStorageID(), sourceProxy);
     out.flush();
     // receiveResponse
     DataInputStream reply = new DataInputStream(sock.getInputStream());

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java Wed Jul  6 18:32:04 2011
@@ -17,30 +17,31 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
+import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
+import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeTrue;
+
 import java.io.File;
 import java.util.ArrayList;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import static org.apache.hadoop.test.MetricsAsserts.*;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.log4j.Level;
-
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
-import static org.junit.Assert.*;
-import static org.junit.Assume.assumeTrue;
 
 /**
  * Test reporting of DN volume failure counts and metrics.

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java Wed Jul  6 18:32:04 2011
@@ -133,17 +133,17 @@ public class TestDataNodeVolumeFailureTo
   }
 
   /** 
-   * Restart the cluster with a new volume tolerated value.
-   * @param volTolerated
-   * @param manageCluster
+   * Restart the datanodes with a new volume tolerated value.
+   * @param volTolerated number of dfs data dir failures to tolerate
+   * @param manageDfsDirs whether the mini cluster should manage data dirs
    * @throws IOException
    */
-  private void restartCluster(int volTolerated, boolean manageCluster)
+  private void restartDatanodes(int volTolerated, boolean manageDfsDirs)
       throws IOException {
     //Make sure no datanode is running
     cluster.shutdownDataNodes();
     conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, volTolerated);
-    cluster.startDataNodes(conf, 1, manageCluster, null, null);
+    cluster.startDataNodes(conf, 1, manageDfsDirs, null, null);
     cluster.waitActive();
   }
 
@@ -174,19 +174,14 @@ public class TestDataNodeVolumeFailureTo
 
   /**
    * Tests for a given volumes to be tolerated and volumes failed.
-   * 
-   * @param volumesTolerated
-   * @param volumesFailed
-   * @param expectedBPServiceState
-   * @param clusterManaged
-   * @throws IOException
-   * @throws InterruptedException
    */
   private void testVolumeConfig(int volumesTolerated, int volumesFailed,
-      boolean expectedBPServiceState, boolean clusterManaged)
+      boolean expectedBPServiceState, boolean manageDfsDirs)
       throws IOException, InterruptedException {
     assumeTrue(!System.getProperty("os.name").startsWith("Windows"));
     final int dnIndex = 0;
+    // Fail the current directory since invalid storage directory perms
+    // get fixed up automatically on datanode startup.
     File[] dirs = {
         new File(MiniDFSCluster.getStorageDir(dnIndex, 0), "current"),
         new File(MiniDFSCluster.getStorageDir(dnIndex, 1), "current") };
@@ -195,11 +190,10 @@ public class TestDataNodeVolumeFailureTo
       for (int i = 0; i < volumesFailed; i++) {
         prepareDirToFail(dirs[i]);
       }
-      restartCluster(volumesTolerated, clusterManaged);
+      restartDatanodes(volumesTolerated, manageDfsDirs);
       assertEquals(expectedBPServiceState, cluster.getDataNodes().get(0)
           .isBPServiceAlive(cluster.getNamesystem().getBlockPoolId()));
     } finally {
-      // restore its old permission
       for (File dir : dirs) {
         FileUtil.chmod(dir.toString(), "755");
       }
@@ -215,8 +209,7 @@ public class TestDataNodeVolumeFailureTo
   private void prepareDirToFail(File dir) throws IOException,
       InterruptedException {
     dir.mkdirs();
-    assertTrue("Couldn't chmod local vol", FileUtil
-        .chmod(dir.toString(), "000") == 0);
+    assertEquals("Couldn't chmod local vol", 0,
+        FileUtil.chmod(dir.toString(), "000"));
   }
-
 }

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java Wed Jul  6 18:32:04 2011
@@ -140,10 +140,10 @@ public class TestDiskError {
     // write the header.
     DataOutputStream out = new DataOutputStream(s.getOutputStream());
 
-    Sender.opWriteBlock(out, block.getBlock(), 1,
-        BlockConstructionStage.PIPELINE_SETUP_CREATE,
-        0L, 0L, 0L, "", null, new DatanodeInfo[0],
-        BlockTokenSecretManager.DUMMY_TOKEN);
+    new Sender(out).writeBlock(block.getBlock(),
+        BlockTokenSecretManager.DUMMY_TOKEN, "",
+        new DatanodeInfo[0], null,
+        BlockConstructionStage.PIPELINE_SETUP_CREATE, 1, 0L, 0L, 0L);
 
     // write check header
     out.writeByte( 1 );

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java Wed Jul  6 18:32:04 2011
@@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.Storage;
 

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java Wed Jul  6 18:32:04 2011
@@ -19,10 +19,11 @@ package org.apache.hadoop.hdfs.server.na
 
 import java.io.IOException;
 
-import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.ipc.Server;
 
 /**
  * This is a utility class to expose NameNode functionality for unit tests.

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java Wed Jul  6 18:32:04 2011
@@ -34,6 +34,8 @@ import org.apache.hadoop.hdfs.TestFileCr
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java Wed Jul  6 18:32:04 2011
@@ -23,6 +23,7 @@ import java.net.InetSocketAddress;
 import java.io.File;
 import java.io.IOException;
 import java.net.URI;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Random;
@@ -173,6 +174,71 @@ public class TestCheckpoint extends Test
     resurrectNameDir(first); // put back namedir
   }
 
+  /**
+   * Tests EditLogFileOutputStream doesn't throw NullPointerException on being
+   * closed twice.
+   * See https://issues.apache.org/jira/browse/HDFS-2011
+   */
+  public void testEditLogFileOutputStreamCloses()
+    throws IOException,NullPointerException {
+    System.out.println("Testing EditLogFileOutputStream doesn't throw " +
+                       "NullPointerException on being closed twice");
+    File editLogStreamFile = null;
+    try {
+      editLogStreamFile = new File(System.getProperty("test.build.data","/tmp"),
+                                   "editLogStream.dat");
+      EditLogFileOutputStream editLogStream =
+                             new EditLogFileOutputStream(editLogStreamFile, 0);
+      editLogStream.close();
+      //Closing an twice should not throw a NullPointerException
+      editLogStream.close();
+    } finally {
+      if (editLogStreamFile != null)
+        // Cleanup the editLogStream.dat file we created
+          editLogStreamFile.delete();
+    }
+    System.out.println("Successfully tested EditLogFileOutputStream doesn't " +
+           "throw NullPointerException on being closed twice");
+  }
+
+  /**
+   * Checks that an IOException in NNStorage.writeTransactionIdFile is handled
+   * correctly (by removing the storage directory)
+   * See https://issues.apache.org/jira/browse/HDFS-2011
+   */
+  public void testWriteTransactionIdHandlesIOE() throws Exception {
+    System.out.println("Check IOException handled correctly by writeTransactionIdFile");
+    ArrayList<URI> fsImageDirs = new ArrayList<URI>();
+    ArrayList<URI> editsDirs = new ArrayList<URI>();
+    File filePath =
+      new File(System.getProperty("test.build.data","/tmp"), "storageDirToCheck");
+    assertTrue("Couldn't create directory storageDirToCheck",
+               filePath.exists() || filePath.mkdirs());
+    fsImageDirs.add(filePath.toURI());
+    editsDirs.add(filePath.toURI());
+    NNStorage nnStorage = new NNStorage(new HdfsConfiguration(),
+      fsImageDirs, editsDirs);
+    try {
+      assertTrue("List of storage directories didn't have storageDirToCheck.",
+                 nnStorage.getEditsDirectories().iterator().next().
+                 toString().indexOf("storageDirToCheck") != -1);
+      assertTrue("List of removed storage directories wasn't empty",
+                 nnStorage.getRemovedStorageDirs().isEmpty());
+    } finally {
+      // Delete storage directory to cause IOException in writeTransactionIdFile 
+      assertTrue("Couldn't remove directory " + filePath.getAbsolutePath(),
+                 filePath.delete());
+    }
+    // Just call writeTransactionIdFile using any random number
+    nnStorage.writeTransactionIdFileToStorage(1);
+    List<StorageDirectory> listRsd = nnStorage.getRemovedStorageDirs();
+    assertTrue("Removed directory wasn't what was expected",
+               listRsd.size() > 0 && listRsd.get(listRsd.size() - 1).getRoot().
+               toString().indexOf("storageDirToCheck") != -1);
+    System.out.println("Successfully checked IOException is handled correctly "
+                       + "by writeTransactionIdFile");
+  }
+
   /*
    * Simulate namenode crashing after rolling edit log.
    */

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestComputeInvalidateWork.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestComputeInvalidateWork.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestComputeInvalidateWork.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestComputeInvalidateWork.java Wed Jul  6 18:32:04 2011
@@ -17,14 +17,15 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import junit.framework.TestCase;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 
-import junit.framework.TestCase;
-
 /**
  * Test if FSNamesystem handles heartbeat right
  */

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java Wed Jul  6 18:32:04 2011
@@ -29,9 +29,9 @@ import org.apache.hadoop.hdfs.DFSConfigK
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java Wed Jul  6 18:32:04 2011
@@ -17,19 +17,15 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Random;
 
-import org.junit.BeforeClass;
-import org.junit.AfterClass;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -40,8 +36,10 @@ import org.apache.hadoop.hdfs.HdfsConfig
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
 
 /**
  * This class tests the decommissioning of nodes.

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHost2NodesMap.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHost2NodesMap.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHost2NodesMap.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHost2NodesMap.java Wed Jul  6 18:32:04 2011
@@ -18,10 +18,11 @@
 
 package org.apache.hadoop.hdfs.server.namenode;
 
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-
 import junit.framework.TestCase;
 
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+
 public class TestHost2NodesMap extends TestCase {
   static private Host2NodesMap map = new Host2NodesMap();
   private final static DatanodeDescriptor dataNodes[] = new DatanodeDescriptor[] {

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java Wed Jul  6 18:32:04 2011
@@ -21,17 +21,17 @@ package org.apache.hadoop.hdfs.server.na
 import java.io.File;
 import java.util.ArrayList;
 
+import junit.framework.TestCase;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DF;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import junit.framework.TestCase;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 
 
 

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNodeCount.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNodeCount.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNodeCount.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestNodeCount.java Wed Jul  6 18:32:04 2011
@@ -21,6 +21,8 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.concurrent.TimeoutException;
 
+import junit.framework.TestCase;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -30,9 +32,8 @@ import org.apache.hadoop.hdfs.MiniDFSClu
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NumberReplicas;
-
-import junit.framework.TestCase;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas;
 
 /**
  * Test if live nodes count per node is correct 

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java Wed Jul  6 18:32:04 2011
@@ -20,20 +20,21 @@ package org.apache.hadoop.hdfs.server.na
 import java.io.File;
 import java.io.IOException;
 
+import junit.framework.TestCase;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.TestDatanodeBlockScanner;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
+import org.apache.hadoop.hdfs.TestDatanodeBlockScanner;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-
-import junit.framework.TestCase;
 
 public class TestOverReplicatedBlocks extends TestCase {
   /** Test processOverReplicatedBlock can handle corrupt replicas fine.

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStartup.java Wed Jul  6 18:32:04 2011
@@ -25,7 +25,9 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.RandomAccessFile;
+import java.net.InetAddress;
 import java.net.URI;
+import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
@@ -44,10 +46,13 @@ import org.apache.hadoop.fs.RawLocalFile
 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.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.FSConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@@ -55,8 +60,8 @@ import org.apache.hadoop.hdfs.server.nam
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
-import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.junit.Assert;
 
 /**
  * Startup and checkpoint tests
@@ -478,4 +483,89 @@ public class TestStartup extends TestCas
       }
     }
   }
+  
+  /**
+   * This test tests hosts include list contains host names.  After namenode
+   * restarts, the still alive datanodes should not have any trouble in getting
+   * registrant again.
+   */
+  public void testNNRestart() throws IOException, InterruptedException {
+    MiniDFSCluster cluster = null;
+    FileSystem localFileSys;
+    Path hostsFile;
+    Path excludeFile;
+    Configuration conf = new HdfsConfiguration();
+    int HEARTBEAT_INTERVAL = 1; // heartbeat interval in seconds
+    // Set up the hosts/exclude files.
+    localFileSys = FileSystem.getLocal(conf);
+    Path workingDir = localFileSys.getWorkingDirectory();
+    Path dir = new Path(workingDir, "build/test/data/work-dir/restartnn");
+    hostsFile = new Path(dir, "hosts");
+    excludeFile = new Path(dir, "exclude");
+
+    // Setup conf
+    conf.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
+    writeConfigFile(localFileSys, excludeFile, null);    
+    conf.set(DFSConfigKeys.DFS_HOSTS, hostsFile.toUri().getPath());
+    // write into hosts file
+    ArrayList<String>list = new ArrayList<String>();
+    byte b[] = {127, 0, 0, 1};
+    InetAddress inetAddress = InetAddress.getByAddress(b);
+    list.add(inetAddress.getHostName());
+    writeConfigFile(localFileSys, hostsFile, list);
+    int numNameNodes = 1;
+    int numDatanodes = 1;
+    
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numNameNodes(numNameNodes)
+      .numDataNodes(numDatanodes).setupHostsFile(true).build();
+      cluster.waitActive();
+  
+      cluster.restartNameNode();
+      NameNode nn = cluster.getNameNode();
+      assertNotNull(nn);
+      Assert.assertTrue(cluster.isDataNodeUp());
+      
+      DatanodeInfo[] info = nn.getDatanodeReport(DatanodeReportType.LIVE);
+      for (int i = 0 ; i < 5 && info.length != numDatanodes; i++) {
+        Thread.sleep(HEARTBEAT_INTERVAL * 1000);
+        info = nn.getDatanodeReport(DatanodeReportType.LIVE);
+      }
+      assertEquals("Number of live nodes should be "+numDatanodes, numDatanodes, 
+          info.length);
+      
+    } catch (IOException e) {
+      fail(StringUtils.stringifyException(e));
+      throw e;
+    } finally {
+      cleanupFile(localFileSys, excludeFile.getParent());
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+  
+  private void writeConfigFile(FileSystem localFileSys, Path name,
+      ArrayList<String> nodes) throws IOException {
+    // delete if it already exists
+    if (localFileSys.exists(name)) {
+      localFileSys.delete(name, true);
+    }
+
+    if (nodes != null) {
+      FSDataOutputStream stm = localFileSys.create(name);
+      for (Iterator<String> it = nodes.iterator(); it.hasNext();) {
+        String node = it.next();
+        stm.writeBytes(node);
+        stm.writeBytes("\n");
+      }
+      stm.close();
+    }
+  }
+  
+  private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
+    assertTrue(fileSys.exists(name));
+    fileSys.delete(name, true);
+    assertTrue(!fileSys.exists(name));
+  }
 }

Modified: hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java?rev=1143523&r1=1143522&r2=1143523&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java (original)
+++ hadoop/common/branches/HDFS-1073/hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java Wed Jul  6 18:32:04 2011
@@ -18,6 +18,10 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import java.io.DataInputStream;
 import java.io.EOFException;
 import java.io.File;
@@ -28,22 +32,23 @@ import java.security.MessageDigest;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Random;
+import java.util.Set;
 
 import static org.mockito.Matchers.anyByte;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
 
-import junit.framework.TestCase;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.cli.CLITestCmdDFS;
-import org.apache.hadoop.cli.util.*;
+import org.apache.hadoop.cli.util.CLICommandDFSAdmin;
+import org.apache.hadoop.cli.util.CommandExecutor;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -55,12 +60,16 @@ import static org.apache.hadoop.hdfs.ser
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 
+import com.google.common.collect.ImmutableSet;
 /**
  * Startup and checkpoint tests
  * 
  */
-public class TestStorageRestore extends TestCase {
+public class TestStorageRestore {
   public static final String NAME_NODE_HOST = "localhost:";
   public static final String NAME_NODE_HTTP_HOST = "0.0.0.0:";
   private static final Log LOG =
@@ -72,8 +81,8 @@ public class TestStorageRestore extends 
   static final int fileSize = 8192;
   private File path1, path2, path3;
   private MiniDFSCluster cluster;  
- 
-  protected void setUp() throws Exception {
+  @Before
+  public void setUpNameDirs() throws Exception {
     config = new HdfsConfiguration();
     hdfsDir = new File(MiniDFSCluster.getBaseDirectory()).getCanonicalFile();
     if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) {
@@ -110,12 +119,13 @@ public class TestStorageRestore extends 
   /**
    * invalidate storage by removing the second and third storage directories
    */
-  public void invalidateStorage(FSImage fi) throws IOException {
+  public void invalidateStorage(FSImage fi, Set<File> filesToInvalidate) throws IOException {
     ArrayList<StorageDirectory> al = new ArrayList<StorageDirectory>(2);
     Iterator<StorageDirectory> it = fi.getStorage().dirIterator();
     while(it.hasNext()) {
       StorageDirectory sd = it.next();
-      if(sd.getRoot().equals(path2) || sd.getRoot().equals(path3)) {
+      if(filesToInvalidate.contains(sd.getRoot())) {
+        LOG.info("causing IO error on " + sd.getRoot());
         al.add(sd);
       }
     }
@@ -140,7 +150,7 @@ public class TestStorageRestore extends 
    * test
    */
   public void printStorages(FSImage fs) {
-    LOG.info("current storages and corresoponding sizes:");
+    LOG.info("current storages and corresponding sizes:");
     for(Iterator<StorageDirectory> it = fs.getStorage().dirIterator(); it.hasNext(); ) {
       StorageDirectory sd = it.next();
 
@@ -164,6 +174,7 @@ public class TestStorageRestore extends 
    * 8. verify that all the image and edits files are the same.
    */
   @SuppressWarnings("deprecation")
+  @Test
   public void testStorageRestore() throws Exception {
     int numDatanodes = 0;
     cluster = new MiniDFSCluster.Builder(config).numDataNodes(numDatanodes)
@@ -181,7 +192,7 @@ public class TestStorageRestore extends 
     
     System.out.println("****testStorageRestore: dir 'test' created, invalidating storage...");
   
-    invalidateStorage(cluster.getNameNode().getFSImage());
+    invalidateStorage(cluster.getNameNode().getFSImage(), ImmutableSet.of(path2, path3));
     printStorages(cluster.getNameNode().getFSImage());
     System.out.println("****testStorageRestore: storage invalidated");
 
@@ -265,6 +276,7 @@ public class TestStorageRestore extends 
    * Test dfsadmin -restoreFailedStorage command
    * @throws Exception
    */
+  @Test
   public void testDfsAdminCmd() throws Exception {
     cluster = new MiniDFSCluster.Builder(config).
                                  numDataNodes(2).
@@ -296,7 +308,7 @@ public class TestStorageRestore extends 
       restore = fsi.getStorage().getRestoreFailedStorage();
       assertTrue("After set false call restore is " + restore, restore);
       
-   // run one more time - no change in value
+      // run one more time - no change in value
       cmd = "-fs NAMENODE -restoreFailedStorage check";
       CommandExecutor.Result cmdResult = executor.executeCommand(cmd);
       restore = fsi.getStorage().getRestoreFailedStorage();
@@ -310,4 +322,67 @@ public class TestStorageRestore extends 
       cluster.shutdown();
     }
   }
+
+  /**
+   * Test to simulate interleaved checkpointing by 2 2NNs after a storage
+   * directory has been taken offline. The first will cause the directory to
+   * come back online, but it won't have any valid contents. The second 2NN will
+   * then try to perform a checkpoint. The NN should not serve up the image or
+   * edits from the restored (empty) dir.
+   */
+  @SuppressWarnings("deprecation")
+  @Test
+  public void testMultipleSecondaryCheckpoint() throws IOException {
+    
+    SecondaryNameNode secondary = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(config).numDataNodes(1)
+          .manageNameDfsDirs(false).build();
+      cluster.waitActive();
+      
+      secondary = new SecondaryNameNode(config);
+  
+      FSImage fsImage = cluster.getNameNode().getFSImage();
+      printStorages(fsImage);
+      
+      FileSystem fs = cluster.getFileSystem();
+      Path testPath = new Path("/", "test");
+      assertTrue(fs.mkdirs(testPath));
+      
+      printStorages(fsImage);
+  
+      // Take name1 offline
+      invalidateStorage(fsImage, ImmutableSet.of(path1));
+      
+      // Simulate a 2NN beginning a checkpoint, but not finishing. This will
+      // cause name1 to be restored.
+      cluster.getNameNode().rollEditLog();
+      
+      printStorages(fsImage);
+      
+      // Now another 2NN comes along to do a full checkpoint.
+      secondary.doCheckpoint();
+      
+      printStorages(fsImage);
+      
+      // The created file should still exist in the in-memory FS state after the
+      // checkpoint.
+      assertTrue("path exists before restart", fs.exists(testPath));
+      
+      secondary.shutdown();
+      
+      // Restart the NN so it reloads the edits from on-disk.
+      cluster.restartNameNode();
+  
+      // The created file should still exist after the restart.
+      assertTrue("path should still exist after restart", fs.exists(testPath));
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+      if (secondary != null) {
+        secondary.shutdown();
+      }
+    }
+  }
 }