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 2012/04/02 09:28:49 UTC

svn commit: r1308260 [4/4] - in /hadoop/common/branches/HDFS-3042/hadoop-hdfs-project: hadoop-hdfs-httpfs/src/main/sbin/ hadoop-hdfs/ hadoop-hdfs/src/main/bin/ hadoop-hdfs/src/main/docs/src/documentation/content/xdocs/ hadoop-hdfs/src/main/java/ hadoop...

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDeleteBlockPool.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDeleteBlockPool.java?rev=1308260&r1=1308259&r2=1308260&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDeleteBlockPool.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDeleteBlockPool.java Mon Apr  2 07:28:42 2012
@@ -183,7 +183,7 @@ public class TestDeleteBlockPool {
       Assert.assertEquals(1, dn1.getAllBpOs().length);
       
       DFSAdmin admin = new DFSAdmin(nn1Conf);
-      String dn1Address = dn1.getSelfAddr().getHostName()+":"+dn1.getIpcPort();
+      String dn1Address = dn1.getDatanodeId().getIpAddr() + ":" + dn1.getIpcPort();
       String[] args = { "-deleteBlockPool", dn1Address, bpid2 };
       
       int ret = admin.run(args);

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java?rev=1308260&r1=1308259&r2=1308260&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java Mon Apr  2 07:28:42 2012
@@ -136,7 +136,7 @@ public class TestDiskError {
     DataNode datanode = cluster.getDataNodes().get(sndNode);
     
     // replicate the block to the second datanode
-    InetSocketAddress target = datanode.getSelfAddr();
+    InetSocketAddress target = datanode.getXferAddress();
     Socket s = new Socket(target.getAddress(), target.getPort());
     // write the header.
     DataOutputStream out = new DataOutputStream(s.getOutputStream());

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java?rev=1308260&r1=1308259&r2=1308260&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestInterDatanodeProtocol.java Mon Apr  2 07:28:42 2012
@@ -348,7 +348,7 @@ public class TestInterDatanodeProtocol {
 
     final InetSocketAddress addr = NetUtils.getConnectAddress(server);
     DatanodeID fakeDnId = new DatanodeID(
-        "localhost:" + addr.getPort(), "fake-storage", 0, addr.getPort());
+        "localhost", "localhost", "fake-storage", addr.getPort(), 0, addr.getPort());
     DatanodeInfo dInfo = new DatanodeInfo(fakeDnId);
     InterDatanodeProtocol proxy = null;
 

Modified: hadoop/common/branches/HDFS-3042/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-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java?rev=1308260&r1=1308259&r2=1308260&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java Mon Apr  2 07:28:42 2012
@@ -766,28 +766,33 @@ public class NNThroughputBenchmark {
     long[] blockReportList;
 
     /**
-     * Get data-node in the form 
-     * <host name> : <port>
-     * where port is a 6 digit integer.
+     * Return a a 6 digit integer port.
      * This is necessary in order to provide lexocographic ordering.
      * Host names are all the same, the ordering goes by port numbers.
      */
-    private static String getNodeName(int port) throws IOException {
-      String machineName = DNS.getDefaultHost("default", "default");
-      String sPort = String.valueOf(100000 + port);
-      if(sPort.length() > 6)
-        throw new IOException("Too many data-nodes.");
-      return machineName + ":" + sPort;
+    private static int getNodePort(int num) throws IOException {
+      int port = 100000 + num;
+      if (String.valueOf(port).length() > 6) {
+        throw new IOException("Too many data-nodes");
+      }
+      return port;
     }
 
     TinyDatanode(int dnIdx, int blockCapacity) throws IOException {
-      dnRegistration = new DatanodeRegistration(getNodeName(dnIdx));
+      String hostName = DNS.getDefaultHost("default", "default");
+      dnRegistration = new DatanodeRegistration(hostName);
+      dnRegistration.setXferPort(getNodePort(dnIdx));
+      dnRegistration.setHostName(hostName);
       this.blocks = new ArrayList<Block>(blockCapacity);
       this.nrBlocks = 0;
     }
 
-    String getName() {
-      return dnRegistration.getName();
+    public String toString() {
+      return dnRegistration.toString();
+    }
+
+    String getXferAddr() {
+      return dnRegistration.getXferAddr();
     }
 
     void register() throws IOException {
@@ -850,8 +855,8 @@ public class NNThroughputBenchmark {
       return blockReportList;
     }
 
-    public int compareTo(String name) {
-      return getName().compareTo(name);
+    public int compareTo(String xferAddr) {
+      return getXferAddr().compareTo(xferAddr);
     }
 
     /**
@@ -889,10 +894,12 @@ public class NNThroughputBenchmark {
         for(int t = 0; t < blockTargets.length; t++) {
           DatanodeInfo dnInfo = blockTargets[t];
           DatanodeRegistration receivedDNReg;
-          receivedDNReg = new DatanodeRegistration(dnInfo.getName());
+          receivedDNReg = new DatanodeRegistration(dnInfo.getIpAddr());
           receivedDNReg.setStorageInfo(
                           new DataStorage(nsInfo, dnInfo.getStorageID()));
+          receivedDNReg.setXferPort(dnInfo.getXferPort());
           receivedDNReg.setInfoPort(dnInfo.getInfoPort());
+          receivedDNReg.setIpcPort(dnInfo.getIpcPort());
           ReceivedDeletedBlockInfo[] rdBlocks = {
             new ReceivedDeletedBlockInfo(
                   blocks[i], ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK,
@@ -977,10 +984,10 @@ public class NNThroughputBenchmark {
       for(int idx=0; idx < nrDatanodes; idx++) {
         datanodes[idx] = new TinyDatanode(idx, blocksPerReport);
         datanodes[idx].register();
-        assert datanodes[idx].getName().compareTo(prevDNName) > 0
+        assert datanodes[idx].getXferAddr().compareTo(prevDNName) > 0
           : "Data-nodes must be sorted lexicographically.";
         datanodes[idx].sendHeartbeat();
-        prevDNName = datanodes[idx].getName();
+        prevDNName = datanodes[idx].getXferAddr();
       }
 
       // create files 
@@ -1010,7 +1017,7 @@ public class NNThroughputBenchmark {
         LocatedBlock loc = nameNodeProto.addBlock(fileName, clientName, prevBlock, null);
         prevBlock = loc.getBlock();
         for(DatanodeInfo dnInfo : loc.getLocations()) {
-          int dnIdx = Arrays.binarySearch(datanodes, dnInfo.getName());
+          int dnIdx = Arrays.binarySearch(datanodes, dnInfo.getXferAddr());
           datanodes[dnIdx].addBlock(loc.getBlock().getLocalBlock());
           ReceivedDeletedBlockInfo[] rdBlocks = { new ReceivedDeletedBlockInfo(
               loc.getBlock().getLocalBlock(),
@@ -1165,9 +1172,9 @@ public class NNThroughputBenchmark {
       for(int i=0; i < nodesToDecommission; i++) {
         TinyDatanode dn = blockReportObject.datanodes[nrDatanodes-1-i];
         numDecommissionedBlocks += dn.nrBlocks;
-        excludeFile.write(dn.getName().getBytes());
+        excludeFile.write(dn.getXferAddr().getBytes());
         excludeFile.write('\n');
-        LOG.info("Datanode " + dn.getName() + " is decommissioned.");
+        LOG.info("Datanode " + dn + " is decommissioned.");
       }
       excludeFile.close();
       nameNodeProto.refreshNodes();

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java?rev=1308260&r1=1308259&r2=1308260&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDecommissioningStatus.java Mon Apr  2 07:28:42 2012
@@ -156,7 +156,7 @@ public class TestDecommissioningStatus {
       throws IOException {
     DatanodeInfo[] info = client.datanodeReport(DatanodeReportType.LIVE);
 
-    String nodename = info[nodeIndex].getName();
+    String nodename = info[nodeIndex].getXferAddr();
     System.out.println("Decommissioning node: " + nodename);
 
     // write nodename into the exclude file.

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java?rev=1308260&r1=1308259&r2=1308260&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java Mon Apr  2 07:28:42 2012
@@ -93,6 +93,15 @@ public class TestNameNodeMXBean {
       // get attribute alivenodeinfo
       String alivenodeinfo = (String) (mbs.getAttribute(mxbeanName,
           "LiveNodes"));
+      Map<String, Map<String, Object>> liveNodes =
+          (Map<String, Map<String, Object>>) JSON.parse(alivenodeinfo);
+      assertTrue(liveNodes.size() > 0);
+      for (Map<String, Object> liveNode : liveNodes.values()) {
+        assertTrue(liveNode.containsKey("nonDfsUsedSpace"));
+        assertTrue(((Long)liveNode.get("nonDfsUsedSpace")) > 0);
+        assertTrue(liveNode.containsKey("capacity"));
+        assertTrue(((Long)liveNode.get("capacity")) > 0);
+      }
       Assert.assertEquals(fsn.getLiveNodes(), alivenodeinfo);
       // get attribute deadnodeinfo
       String deadnodeinfo = (String) (mbs.getAttribute(mxbeanName,

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyIsHot.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyIsHot.java?rev=1308260&r1=1308259&r2=1308260&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyIsHot.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyIsHot.java Mon Apr  2 07:28:42 2012
@@ -167,7 +167,7 @@ public class TestStandbyIsHot {
       
       // Stop the DN.
       DataNode dn = cluster.getDataNodes().get(0);
-      String dnName = dn.getDatanodeId().getName(); 
+      String dnName = dn.getDatanodeId().getXferAddr(); 
       DataNodeProperties dnProps = cluster.stopDataNode(0);
       
       // Make sure both NNs register it as dead.

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java?rev=1308260&r1=1308259&r2=1308260&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java Mon Apr  2 07:28:42 2012
@@ -65,12 +65,11 @@ public class TestOfflineEditsViewer {
    *
    * These are the opcodes that are not used anymore, some
    * are marked deprecated, we need to include them here to make
-   * sure we exclude them when checking for completness of testing,
+   * sure we exclude them when checking for completeness of testing,
    * that's why the "deprecation" warnings are suppressed.
    */
   @SuppressWarnings("deprecation")
   private static void initializeObsoleteOpCodes() {
-    // these are obsolete
     obsoleteOpCodes.put(FSEditLogOpCodes.OP_DATANODE_ADD, true);
     obsoleteOpCodes.put(FSEditLogOpCodes.OP_DATANODE_REMOVE, true);
     obsoleteOpCodes.put(FSEditLogOpCodes.OP_SET_NS_QUOTA, true);

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java?rev=1308260&r1=1308259&r2=1308260&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java Mon Apr  2 07:28:42 2012
@@ -30,16 +30,16 @@ import org.apache.hadoop.hdfs.server.blo
 public class TestNetworkTopology extends TestCase {
   private final static NetworkTopology cluster = new NetworkTopology();
   private final static DatanodeDescriptor dataNodes[] = new DatanodeDescriptor[] {
-    new DatanodeDescriptor(new DatanodeID("h1:5020"), "/d1/r1"),
-    new DatanodeDescriptor(new DatanodeID("h2:5020"), "/d1/r1"),
-    new DatanodeDescriptor(new DatanodeID("h3:5020"), "/d1/r2"),
-    new DatanodeDescriptor(new DatanodeID("h4:5020"), "/d1/r2"),
-    new DatanodeDescriptor(new DatanodeID("h5:5020"), "/d1/r2"),
-    new DatanodeDescriptor(new DatanodeID("h6:5020"), "/d2/r3"),
-    new DatanodeDescriptor(new DatanodeID("h7:5020"), "/d2/r3")
+    new DatanodeDescriptor(new DatanodeID("h1", 5020), "/d1/r1"),
+    new DatanodeDescriptor(new DatanodeID("h2", 5020), "/d1/r1"),
+    new DatanodeDescriptor(new DatanodeID("h3", 5020), "/d1/r2"),
+    new DatanodeDescriptor(new DatanodeID("h4", 5020), "/d1/r2"),
+    new DatanodeDescriptor(new DatanodeID("h5", 5020), "/d1/r2"),
+    new DatanodeDescriptor(new DatanodeID("h6", 5020), "/d2/r3"),
+    new DatanodeDescriptor(new DatanodeID("h7", 5020), "/d2/r3")
   };
   private final static DatanodeDescriptor NODE = 
-    new DatanodeDescriptor(new DatanodeID("h8:5020"), "/d2/r4");
+    new DatanodeDescriptor(new DatanodeID("h8", 5020), "/d2/r4");
   
   static {
     for(int i=0; i<dataNodes.length; i++) {
@@ -61,9 +61,9 @@ public class TestNetworkTopology extends
   public void testCreateInvalidTopology() throws Exception {
     NetworkTopology invalCluster = new NetworkTopology();
     DatanodeDescriptor invalDataNodes[] = new DatanodeDescriptor[] {
-      new DatanodeDescriptor(new DatanodeID("h1:5020"), "/d1/r1"),
-      new DatanodeDescriptor(new DatanodeID("h2:5020"), "/d1/r1"),
-      new DatanodeDescriptor(new DatanodeID("h3:5020"), "/d1")
+      new DatanodeDescriptor(new DatanodeID("h1", 5020), "/d1/r1"),
+      new DatanodeDescriptor(new DatanodeID("h2", 5020), "/d1/r1"),
+      new DatanodeDescriptor(new DatanodeID("h3", 5020), "/d1")
     };
     invalCluster.add(invalDataNodes[0]);
     invalCluster.add(invalDataNodes[1]);

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/HdfsTestDriver.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/HdfsTestDriver.java?rev=1308260&r1=1308259&r2=1308260&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/HdfsTestDriver.java (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/HdfsTestDriver.java Mon Apr  2 07:28:42 2012
@@ -36,7 +36,9 @@ public class HdfsTestDriver {
     this.pgd = pgd;
     try {
       pgd.addClass("dfsthroughput", BenchmarkThroughput.class, 
-      "measure hdfs throughput");
+          "measure hdfs throughput");
+      pgd.addClass("minidfscluster", MiniDFSClusterManager.class, 
+          "Run a single-process mini DFS cluster");
     } catch(Throwable e) {
       e.printStackTrace();
     }

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-dfs-dir.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-dfs-dir.txt?rev=1308260&r1=1308259&r2=1308260&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-dfs-dir.txt (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-dfs-dir.txt Mon Apr  2 07:28:42 2012
@@ -19,18 +19,6 @@
 # See HADOOP-1629 for more info if needed.
 # These two files are used by unit test TestDFSUpgradeFromImage.java 
 # 
-# hadoop-14-dfs-dir.tgz : 
-# ---------------------
-# This file contains the HDFS directory structure for one namenode and 4 datanodes.
-# The structure is setup similar to the structure used in MiniDFSCluster.
-# The directory was created with Hadoo-0.14.x.
-#
-# In the test, this directory is unpacked and MiniDFSCluster is run with 
-# "-upgrade" option. The test waits for the upgrade to complete 
-# (leave safe mode) and then all the files are read. The test checks that the
-# directory structure and file checksums exactly match the information
-# in this file.
-#
 # hadoop-dfs-dir.txt :
 # ---------------------
 # Along with this description this file contains the expected files and 
@@ -43,9 +31,6 @@
 # For e.g. "top-dir-1Mb-512" contains files created with dfs.blocksize of 1Mb 
 # and io.bytes.per.checksum of 512.
 #
-# In the future, when Hadoop project no longer supports upgrade from
-# Hadoop-0.12, then a new DFS directory image must be created.
-#
 # To generate checksum info for new files :
 # ---------------------------------------
 # Uncomment the last coment (starts with "printChecksums") and run the 

Modified: hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml?rev=1308260&r1=1308259&r2=1308260&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml (original)
+++ hadoop/common/branches/HDFS-3042/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml Mon Apr  2 07:28:42 2012
@@ -15420,7 +15420,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>setSpaceQuota: java.io.FileNotFoundException: Directory does not exist: /test1</expected-output>
+          <expected-output>setSpaceQuota: Directory does not exist: /test1</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -15486,7 +15486,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>clrQuota: java.io.FileNotFoundException: Directory does not exist: /test1</expected-output>
+          <expected-output>clrQuota: Directory does not exist: /test1</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -15506,7 +15506,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>put: org.apache.hadoop.hdfs.protocol.DSQuotaExceededException: The DiskSpace quota of /dir1 is exceeded: quota=1.0k 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>
@@ -15526,7 +15526,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>mkdir: org.apache.hadoop.hdfs.protocol.NSQuotaExceededException: The NameSpace quota (directories and files) of directory /dir1 is exceeded: quota=1 file count=2</expected-output>
+          <expected-output>mkdir: The NameSpace quota (directories and files) of directory /dir1 is exceeded: quota=1 file count=2</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -15741,6 +15741,10 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
+          <expected-output>Hostname: [-.a-zA-z0-9\.]+</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
           <expected-output>Decommission Status : [a-zA-Z]+</expected-output>
         </comparator>
         <comparator>
@@ -15792,7 +15796,7 @@
       <comparators>
         <comparator>
           <type>TokenComparator</type>
-          <expected-output>saveNamespace: java.io.IOException: Safe mode should be turned ON in order to create namespace image.</expected-output>
+          <expected-output>saveNamespace: Safe mode should be turned ON in order to create namespace image.</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -15838,6 +15842,10 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
+          <expected-output>Hostname: [-.a-zA-z0-9\.]+</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
           <expected-output>Decommission Status : [a-zA-Z]+</expected-output>
         </comparator>
         <comparator>
@@ -16168,7 +16176,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>setQuota: org.apache.hadoop.hdfs.server.namenode.SafeModeException: Cannot set quota on /test. Name node is in safe mode.</expected-output>
+          <expected-output>setQuota: Cannot set quota on /test. Name node is in safe mode.</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -16187,7 +16195,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>clrQuota: org.apache.hadoop.hdfs.server.namenode.SafeModeException: Cannot set quota on /test. Name node is in safe mode.</expected-output>
+          <expected-output>clrQuota: Cannot set quota on /test. Name node is in safe mode.</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -16207,7 +16215,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>setSpaceQuota: org.apache.hadoop.hdfs.server.namenode.SafeModeException: Cannot set quota on /test. Name node is in safe mode.</expected-output>
+          <expected-output>setSpaceQuota: Cannot set quota on /test. Name node is in safe mode.</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -16226,7 +16234,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-         <expected-output>clrSpaceQuota: org.apache.hadoop.hdfs.server.namenode.SafeModeException: Cannot set quota on /test. Name node is in safe mode.</expected-output>
+         <expected-output>clrSpaceQuota: Cannot set quota on /test. Name node is in safe mode.</expected-output>
         </comparator>
       </comparators>
     </test>