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 at...@apache.org on 2011/11/02 06:35:26 UTC

svn commit: r1196458 [9/9] - in /hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/bin/ src/main/java/ src/main/java/org/apache/hadoop/fs/ src/main/java/org/apache/hadoop/hdfs/ src/main/java/org/apache/hadoop/hdfs/protocol/ ...

Modified: hadoop/common/branches/HDFS-1623/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-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java Wed Nov  2 05:34:31 2011
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.protocol.d
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.util.DataChecksum;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -86,9 +87,9 @@ public class TestDiskError {
     cluster.waitActive();
     final int dnIndex = 0;
     String bpid = cluster.getNamesystem().getBlockPoolId();
-    File storageDir = MiniDFSCluster.getStorageDir(dnIndex, 0);
+    File storageDir = cluster.getInstanceStorageDir(dnIndex, 0);
     File dir1 = MiniDFSCluster.getRbwDir(storageDir, bpid);
-    storageDir = MiniDFSCluster.getStorageDir(dnIndex, 1);
+    storageDir = cluster.getInstanceStorageDir(dnIndex, 1);
     File dir2 = MiniDFSCluster.getRbwDir(storageDir, bpid);
     try {
       // make the data directory of the first datanode to be readonly
@@ -140,14 +141,13 @@ public class TestDiskError {
     // write the header.
     DataOutputStream out = new DataOutputStream(s.getOutputStream());
 
+    DataChecksum checksum = DataChecksum.newDataChecksum(
+        DataChecksum.CHECKSUM_CRC32, 512);
     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 );
-    out.writeInt( 512 );
+        BlockConstructionStage.PIPELINE_SETUP_CREATE, 1, 0L, 0L, 0L,
+        checksum);
     out.flush();
 
     // close the connection before sending the content of the block
@@ -155,9 +155,9 @@ public class TestDiskError {
 
     // the temporary block & meta files should be deleted
     String bpid = cluster.getNamesystem().getBlockPoolId();
-    File storageDir = MiniDFSCluster.getStorageDir(sndNode, 0);
+    File storageDir = cluster.getInstanceStorageDir(sndNode, 0);
     File dir1 = MiniDFSCluster.getRbwDir(storageDir, bpid);
-    storageDir = MiniDFSCluster.getStorageDir(sndNode, 1);
+    storageDir = cluster.getInstanceStorageDir(sndNode, 1);
     File dir2 = MiniDFSCluster.getRbwDir(storageDir, bpid);
     while (dir1.listFiles().length != 0 || dir2.listFiles().length != 0) {
       Thread.sleep(100);

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java Wed Nov  2 05:34:31 2011
@@ -29,6 +29,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
 
@@ -243,12 +244,50 @@ public abstract class FSImageTestUtil {
         // recurse
         assertParallelFilesAreIdentical(sameNameList, ignoredFileNames);
       } else {
-        assertFileContentsSame(sameNameList.toArray(new File[0]));
+        if ("VERSION".equals(sameNameList.get(0).getName())) {
+          assertPropertiesFilesSame(sameNameList.toArray(new File[0]));
+        } else {
+          assertFileContentsSame(sameNameList.toArray(new File[0]));
+        }
       }
     }  
   }
   
   /**
+   * Assert that a set of properties files all contain the same data.
+   * We cannot simply check the md5sums here, since Properties files
+   * contain timestamps -- thus, two properties files from the same
+   * saveNamespace operation may actually differ in md5sum.
+   * @param propFiles the files to compare
+   * @throws IOException if the files cannot be opened or read
+   * @throws AssertionError if the files differ
+   */
+  public static void assertPropertiesFilesSame(File[] propFiles)
+      throws IOException {
+    Set<Map.Entry<Object, Object>> prevProps = null;
+    
+    for (File f : propFiles) {
+      Properties props;
+      FileInputStream is = new FileInputStream(f);
+      try {
+        props = new Properties();
+        props.load(is);
+      } finally {
+        IOUtils.closeStream(is);
+      }
+      if (prevProps == null) {
+        prevProps = props.entrySet();
+      } else {
+        Set<Entry<Object,Object>> diff =
+          Sets.symmetricDifference(prevProps, props.entrySet());
+        if (!diff.isEmpty()) {
+          fail("Properties file " + f + " differs from " + propFiles[0]);
+        }
+      }
+    }
+  }
+
+  /**
    * Assert that all of the given paths have the exact same
    * contents 
    */

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java Wed Nov  2 05:34:31 2011
@@ -52,7 +52,7 @@ public class NameNodeAdapter {
    * @return rpc server
    */
   public static Server getRpcServer(NameNode namenode) {
-    return ((NameNodeRpcServer)namenode.getRpcServer()).server;
+    return ((NameNodeRpcServer)namenode.getRpcServer()).clientRpcServer;
   }
 
   public static DelegationTokenSecretManager getDtSecretManager(

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java Wed Nov  2 05:34:31 2011
@@ -203,11 +203,9 @@ public class OfflineEditsViewerHelper {
       "JobTracker/foo.com@FOO.COM");
     try {
       longUgi.doAs(new PrivilegedExceptionAction<Object>() {
-        public Object run() throws IOException {
-          final DistributedFileSystem dfs =
-            (DistributedFileSystem) cluster.getFileSystem();
-          dfs.renewDelegationToken(token);
-          dfs.cancelDelegationToken(token);
+        public Object run() throws IOException, InterruptedException {
+          token.renew(config);
+          token.cancel(config);
           return null;
         }
       });

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java Wed Nov  2 05:34:31 2011
@@ -19,9 +19,12 @@ package org.apache.hadoop.hdfs.server.na
 
 import java.io.File;
 import java.io.IOException;
+import java.net.InetSocketAddress;
 import java.util.Collections;
 import java.util.List;
 
+import junit.framework.TestCase;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
@@ -29,13 +32,13 @@ import org.apache.hadoop.conf.Configurat
 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.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
 
@@ -44,8 +47,6 @@ import com.google.common.collect.Immutab
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
-import junit.framework.TestCase;
-
 public class TestBackupNode extends TestCase {
   public static final Log LOG = LogFactory.getLog(TestBackupNode.class);
 
@@ -241,8 +242,11 @@ public class TestBackupNode extends Test
   void testCheckpoint(StartupOption op) throws Exception {
     Path file1 = new Path("checkpoint.dat");
     Path file2 = new Path("checkpoint2.dat");
+    Path file3 = new Path("backup.dat");
 
     Configuration conf = new HdfsConfiguration();
+    short replication = (short)conf.getInt("dfs.replication", 3);
+    int numDatanodes = Math.max(3, replication);
     conf.set(DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY, "0");
     conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1); // disable block scanner
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 1);
@@ -290,7 +294,7 @@ public class TestBackupNode extends Test
       //
       // Restart cluster and verify that file1 still exist.
       //
-      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes)
                                                 .format(false).build();
       fileSys = cluster.getFileSystem();
       // check that file1 still exists
@@ -319,6 +323,26 @@ public class TestBackupNode extends Test
       backup.doCheckpoint();
       waitCheckpointDone(cluster, backup, txid);
 
+      // Try BackupNode operations
+      InetSocketAddress add = backup.getNameNodeAddress();
+      // Write to BN
+      FileSystem bnFS = FileSystem.get(new Path("hdfs://"
+          + NameNode.getHostPortString(add)).toUri(), conf);
+      boolean canWrite = true;
+      try {
+        TestCheckpoint.writeFile(bnFS, file3, replication);
+      } catch (IOException eio) {
+        LOG.info("Write to BN failed as expected: ", eio);
+        canWrite = false;
+      }
+      assertFalse("Write to BackupNode must be prohibited.", canWrite);
+
+      TestCheckpoint.writeFile(fileSys, file3, replication);
+      TestCheckpoint.checkFile(fileSys, file3, replication);
+      // should also be on BN right away
+      assertTrue("file3 does not exist on BackupNode",
+          op != StartupOption.BACKUP || bnFS.exists(file3));
+
     } catch(IOException e) {
       LOG.error("Error in TestBackupNode:", e);
       assertTrue(e.getLocalizedMessage(), false);

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java Wed Nov  2 05:34:31 2011
@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
 import org.junit.After;
 import org.junit.Test;
 
@@ -129,7 +130,7 @@ public class TestDeadDatanode {
     // that asks datanode to register again
     DatanodeCommand[] cmd = dnp.sendHeartbeat(reg, 0, 0, 0, 0, 0, 0, 0);
     Assert.assertEquals(1, cmd.length);
-    Assert.assertEquals(cmd[0].getAction(), DatanodeCommand.REGISTER
+    Assert.assertEquals(cmd[0].getAction(), RegisterCommand.REGISTER
         .getAction());
   }
 }

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsLimits.java Wed Nov  2 05:34:31 2011
@@ -165,7 +165,7 @@ public class TestFsLimits {
     Class<?> generated = null;
     try {
       fs.verifyFsLimits(inodes, 1, child);
-      rootInode.addChild(child, false, false);
+      rootInode.addChild(child, false);
     } catch (QuotaExceededException e) {
       generated = e.getClass();
     }

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java Wed Nov  2 05:34:31 2011
@@ -466,7 +466,7 @@ public class TestFsck extends TestCase {
       final String bpid = cluster.getNamesystem().getBlockPoolId();
       for (int i=0; i<4; i++) {
         for (int j=0; j<=1; j++) {
-          File storageDir = MiniDFSCluster.getStorageDir(i, j);
+          File storageDir = cluster.getInstanceStorageDir(i, j);
           File data_dir = MiniDFSCluster.getFinalizedDir(storageDir, bpid);
           File[] blocks = data_dir.listFiles();
           if (blocks == null)

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestListCorruptFileBlocks.java Wed Nov  2 05:34:31 2011
@@ -80,7 +80,7 @@ public class TestListCorruptFileBlocks {
 
       // Now deliberately corrupt one block
       String bpid = cluster.getNamesystem().getBlockPoolId();
-      File storageDir = MiniDFSCluster.getStorageDir(0, 1);
+      File storageDir = cluster.getInstanceStorageDir(0, 1);
       File data_dir = MiniDFSCluster.getFinalizedDir(storageDir, bpid);
       assertTrue("data directory does not exist", data_dir.exists());
       File[] blocks = data_dir.listFiles();
@@ -163,7 +163,7 @@ public class TestListCorruptFileBlocks {
           + " corrupt files. Expecting None.", badFiles.size() == 0);
 
       // Now deliberately corrupt one block
-      File storageDir = MiniDFSCluster.getStorageDir(0, 0);
+      File storageDir = cluster.getInstanceStorageDir(0, 0);
       File data_dir = MiniDFSCluster.getFinalizedDir(storageDir, 
           cluster.getNamesystem().getBlockPoolId());
       assertTrue("data directory does not exist", data_dir.exists());
@@ -284,7 +284,7 @@ public class TestListCorruptFileBlocks {
       String bpid = cluster.getNamesystem().getBlockPoolId();
       for (int i = 0; i < 4; i++) {
         for (int j = 0; j <= 1; j++) {
-          File storageDir = MiniDFSCluster.getStorageDir(i, j);
+          File storageDir = cluster.getInstanceStorageDir(i, j);
           File data_dir = MiniDFSCluster.getFinalizedDir(storageDir, bpid);
           File[] blocks = data_dir.listFiles();
           if (blocks == null)
@@ -391,7 +391,7 @@ public class TestListCorruptFileBlocks {
       String bpid = cluster.getNamesystem().getBlockPoolId();
       // For loop through number of datadirectories per datanode (2)
       for (int i = 0; i < 2; i++) {
-        File storageDir = MiniDFSCluster.getStorageDir(0, i);
+        File storageDir = cluster.getInstanceStorageDir(0, i);
         File data_dir = MiniDFSCluster.getFinalizedDir(storageDir, bpid);
         File[] blocks = data_dir.listFiles();
         if (blocks == null)
@@ -466,7 +466,7 @@ public class TestListCorruptFileBlocks {
       final String bpid = cluster.getNamesystem().getBlockPoolId();
       for (int i=0; i<4; i++) {
         for (int j=0; j<=1; j++) {
-          File storageDir = MiniDFSCluster.getStorageDir(i, j);
+          File storageDir = cluster.getInstanceStorageDir(i, j);
           File data_dir = MiniDFSCluster.getFinalizedDir(storageDir, bpid);
           LOG.info("Removing files from " + data_dir);
           File[] blocks = data_dir.listFiles();

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourceChecker.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourceChecker.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourceChecker.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourceChecker.java Wed Nov  2 05:34:31 2011
@@ -19,9 +19,12 @@ package org.apache.hadoop.hdfs.server.na
 
 import java.io.File;
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NameNodeResourceMonitor;
@@ -29,6 +32,8 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import com.google.common.collect.Lists;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -41,7 +46,7 @@ public class TestNameNodeResourceChecker
   @Before
   public void setUp () throws IOException {
     conf = new Configuration();
-    baseDir = new File(conf.get("hadoop.tmp.dir"));
+    baseDir = new File(System.getProperty("test.build.data"));
     nameDir = new File(baseDir, "resource-check-name-dir");
     nameDir.mkdirs();
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nameDir.getAbsolutePath());
@@ -50,8 +55,6 @@ public class TestNameNodeResourceChecker
   /**
    * Tests that hasAvailableDiskSpace returns true if disk usage is below
    * threshold.
-   *
-   * @throws IOException in case of errors
    */
   @Test
   public void testCheckAvailability()
@@ -67,8 +70,6 @@ public class TestNameNodeResourceChecker
   /**
    * Tests that hasAvailableDiskSpace returns false if disk usage is above
    * threshold.
-   * 
-   * @throws IOException in case of errors
    */
   @Test
   public void testCheckAvailabilityNeg() throws IOException {
@@ -83,9 +84,6 @@ public class TestNameNodeResourceChecker
   /**
    * Tests that NameNode resource monitor causes the NN to enter safe mode when
    * resources are low.
-   * 
-   * @throws IOException in case of errors
-   * @throws InterruptedException 
    */
   @Test
   public void testCheckThatNameNodeResourceMonitorIsRunning()
@@ -139,14 +137,12 @@ public class TestNameNodeResourceChecker
   /**
    * Tests that only a single space check is performed if two name dirs are
    * supplied which are on the same volume.
-   * 
-   * @throws IOException
    */
   @Test
   public void testChecking2NameDirsOnOneVolume() throws IOException {
     Configuration conf = new Configuration();
-    File nameDir1 = new File(conf.get("hadoop.tmp.dir", "name-dir1"));
-    File nameDir2 = new File(conf.get("hadoop.tmp.dir", "name-dir2"));
+    File nameDir1 = new File(System.getProperty("test.build.data"), "name-dir1");
+    File nameDir2 = new File(System.getProperty("test.build.data"), "name-dir2");
     nameDir1.mkdirs();
     nameDir2.mkdirs();
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
@@ -162,13 +158,11 @@ public class TestNameNodeResourceChecker
   /**
    * Tests that only a single space check is performed if extra volumes are
    * configured manually which also coincide with a volume the name dir is on.
-   * 
-   * @throws IOException
    */
   @Test
   public void testCheckingExtraVolumes() throws IOException {
     Configuration conf = new Configuration();
-    File nameDir = new File(conf.get("hadoop.tmp.dir", "name-dir"));
+    File nameDir = new File(System.getProperty("test.build.data"), "name-dir");
     nameDir.mkdirs();
     conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nameDir.getAbsolutePath());
     conf.set(DFSConfigKeys.DFS_NAMENODE_CHECKED_VOLUMES_KEY, nameDir.getAbsolutePath());
@@ -179,4 +173,41 @@ public class TestNameNodeResourceChecker
     assertEquals("Should not check the same volume more than once.",
         1, nb.getVolumesLowOnSpace().size());
   }
+
+  /**
+   * Test that the NN is considered to be out of resources only once all
+   * configured volumes are low on resources.
+   */
+  @Test
+  public void testLowResourceVolumePolicy() throws IOException {
+    Configuration conf = new Configuration();
+    File nameDir1 = new File(System.getProperty("test.build.data"), "name-dir1");
+    File nameDir2 = new File(System.getProperty("test.build.data"), "name-dir2");
+    nameDir1.mkdirs();
+    nameDir2.mkdirs();
+    
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        nameDir1.getAbsolutePath() + "," + nameDir2.getAbsolutePath());
+    
+    NameNodeResourceChecker nnrc = new NameNodeResourceChecker(conf);
+    
+    // For the purpose of this test, we need to force the name dirs to appear to
+    // be on different volumes.
+    Map<String, DF> volumes = new HashMap<String, DF>();
+    volumes.put("volume1", new DF(nameDir1, conf));
+    volumes.put("volume2", new DF(nameDir2, conf));
+    nnrc.setVolumes(volumes);
+    
+    NameNodeResourceChecker spyNnrc = Mockito.spy(nnrc);
+    
+    Mockito.when(spyNnrc.getVolumesLowOnSpace()).thenReturn(
+        Lists.newArrayList("volume1"));
+    
+    assertTrue(spyNnrc.hasAvailableDiskSpace());
+    
+    Mockito.when(spyNnrc.getVolumesLowOnSpace()).thenReturn(
+        Lists.newArrayList("volume1", "volume2"));
+    
+    assertFalse(spyNnrc.hasAvailableDiskSpace());
+  }
 }

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSafeMode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSafeMode.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSafeMode.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSafeMode.java Wed Nov  2 05:34:31 2011
@@ -58,8 +58,9 @@ public class TestSafeMode {
 
       String tipMsg = cluster.getNamesystem().getSafeModeTip();
       assertTrue("Safemode tip message looks right",
-                 tipMsg.contains("The number of live datanodes 0 needs an " +
-                                 "additional 1 live"));
+                 tipMsg.contains("The number of live datanodes 0 needs an additional " +
+                                 "2 live datanodes to reach the minimum number 1. " +
+                                 "Safe mode will be turned off automatically."));
 
       // Start a datanode
       cluster.startDataNodes(conf, 1, true, null, null);

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java Wed Nov  2 05:34:31 2011
@@ -46,9 +46,9 @@ public class TestJsonUtil {
     final FileStatus fstatus = toFileStatus(status, parent);
     System.out.println("status  = " + status);
     System.out.println("fstatus = " + fstatus);
-    final String json = JsonUtil.toJsonString(status);
+    final String json = JsonUtil.toJsonString(status, true);
     System.out.println("json    = " + json.replace(",", ",\n  "));
-    final HdfsFileStatus s2 = JsonUtil.toFileStatus((Map<?, ?>)JSON.parse(json));
+    final HdfsFileStatus s2 = JsonUtil.toFileStatus((Map<?, ?>)JSON.parse(json), true);
     final FileStatus fs2 = toFileStatus(s2, parent);
     System.out.println("s2      = " + s2);
     System.out.println("fs2     = " + fs2);

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java Wed Nov  2 05:34:31 2011
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hdfs.web;
 
 import java.io.BufferedReader;
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.net.HttpURLConnection;
@@ -26,17 +27,25 @@ import java.net.URI;
 import java.net.URL;
 import java.security.PrivilegedExceptionAction;
 
+import javax.servlet.http.HttpServletResponse;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemContractBaseTest;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.web.resources.GetOpParam;
+import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
 import org.apache.hadoop.hdfs.web.resources.PutOpParam;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Assert;
 
 public class TestWebHdfsFileSystemContract extends FileSystemContractBaseTest {
   private static final Configuration conf = new Configuration();
@@ -121,6 +130,8 @@ public class TestWebHdfsFileSystemContra
     }    
   }
   
+  //the following are new tests (i.e. not over-riding the super class methods)
+
   public void testGetFileBlockLocations() throws IOException {
     final String f = "/test/testGetFileBlockLocations";
     createFile(path(f));
@@ -158,4 +169,141 @@ public class TestWebHdfsFileSystemContra
     //check if the command successes.
     assertTrue(fs.getFileStatus(p).isDirectory());
   }
+
+  public void testOpenNonExistFile() throws IOException {
+    final Path p = new Path("/test/testOpenNonExistFile");
+    //open it as a file, should get FileNotFoundException 
+    try {
+      final FSDataInputStream in = fs.open(p);
+      in.read();
+      fail();
+    } catch(FileNotFoundException fnfe) {
+      WebHdfsFileSystem.LOG.info("This is expected.", fnfe);
+    }
+  }
+
+  public void testSeek() throws IOException {
+    final Path p = new Path("/test/testSeek");
+    createFile(p);
+
+    final int one_third = data.length/3;
+    final int two_third = one_third*2;
+
+    { //test seek
+      final int offset = one_third; 
+      final int len = data.length - offset;
+      final byte[] buf = new byte[len];
+
+      final FSDataInputStream in = fs.open(p);
+      in.seek(offset);
+      
+      //read all remaining data
+      in.readFully(buf);
+      in.close();
+  
+      for (int i = 0; i < buf.length; i++) {
+        assertEquals("Position " + i + ", offset=" + offset + ", length=" + len,
+            data[i + offset], buf[i]);
+      }
+    }
+
+    { //test position read (read the data after the two_third location)
+      final int offset = two_third; 
+      final int len = data.length - offset;
+      final byte[] buf = new byte[len];
+
+      final FSDataInputStream in = fs.open(p);
+      in.readFully(offset, buf);
+      in.close();
+  
+      for (int i = 0; i < buf.length; i++) {
+        assertEquals("Position " + i + ", offset=" + offset + ", length=" + len,
+            data[i + offset], buf[i]);
+      }
+    }
+  }
+
+
+  public void testRootDir() throws IOException {
+    final Path root = new Path("/");
+
+    final WebHdfsFileSystem webhdfs = (WebHdfsFileSystem)fs;
+    final URL url = webhdfs.toUrl(GetOpParam.Op.NULL, root);
+    WebHdfsFileSystem.LOG.info("null url=" + url);
+    Assert.assertTrue(url.toString().contains("v1"));
+
+    //test root permission
+    final FileStatus status = fs.getFileStatus(root);
+    assertTrue(status != null);
+    assertEquals(0777, status.getPermission().toShort());
+
+    //delete root - disabled due to a sticky bit bug 
+    //assertFalse(fs.delete(root, true));
+
+    //create file using root path 
+    try {
+      final FSDataOutputStream out = fs.create(root);
+      out.write(1);
+      out.close();
+      fail();
+    } catch(IOException e) {
+      WebHdfsFileSystem.LOG.info("This is expected.", e);
+    }
+
+    //open file using root path 
+    try {
+      final FSDataInputStream in = fs.open(root);
+      in.read();
+      fail();
+      fail();
+    } catch(IOException e) {
+      WebHdfsFileSystem.LOG.info("This is expected.", e);
+    }
+  }
+
+  public void testResponseCode() throws IOException {
+    final WebHdfsFileSystem webhdfs = (WebHdfsFileSystem)fs;
+    final Path dir = new Path("/test/testUrl");
+    assertTrue(webhdfs.mkdirs(dir));
+
+    {//test set owner with empty parameters
+      final URL url = webhdfs.toUrl(PutOpParam.Op.SETOWNER, dir);
+      final HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+      conn.connect();
+      assertEquals(HttpServletResponse.SC_BAD_REQUEST, conn.getResponseCode());
+      conn.disconnect();
+    }
+
+    {//test set replication on a directory
+      final HttpOpParam.Op op = PutOpParam.Op.SETREPLICATION;
+      final URL url = webhdfs.toUrl(op, dir);
+      final HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+      conn.setRequestMethod(op.getType().toString());
+      conn.connect();
+      assertEquals(HttpServletResponse.SC_FORBIDDEN, conn.getResponseCode());
+      
+      assertFalse(webhdfs.setReplication(dir, (short)1));
+      conn.disconnect();
+    }
+
+    {//test get file status for a non-exist file.
+      final Path p = new Path(dir, "non-exist");
+      final URL url = webhdfs.toUrl(GetOpParam.Op.GETFILESTATUS, p);
+      final HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+      conn.connect();
+      assertEquals(HttpServletResponse.SC_NOT_FOUND, conn.getResponseCode());
+      conn.disconnect();
+    }
+
+    {//test set permission with empty parameters
+      final HttpOpParam.Op op = PutOpParam.Op.SETPERMISSION;
+      final URL url = webhdfs.toUrl(op, dir);
+      final HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+      conn.setRequestMethod(op.getType().toString());
+      conn.connect();
+      assertEquals(HttpServletResponse.SC_OK, conn.getResponseCode());
+      assertEquals((short)0755, webhdfs.getFileStatus(dir).getPermission().toShort());
+      conn.disconnect();
+    }
+  }
 }

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestPermission.java Wed Nov  2 05:34:31 2011
@@ -111,10 +111,18 @@ public class TestPermission extends Test
 
       FsPermission dirPerm = new FsPermission((short)0777);
       fs.mkdirs(new Path("/a1/a2/a3"), dirPerm);
-      checkPermission(fs, "/a1", inheritPerm);
-      checkPermission(fs, "/a1/a2", inheritPerm);
+      checkPermission(fs, "/a1", dirPerm);
+      checkPermission(fs, "/a1/a2", dirPerm);
       checkPermission(fs, "/a1/a2/a3", dirPerm);
 
+      dirPerm = new FsPermission((short)0123);
+      FsPermission permission = FsPermission.createImmutable(
+        (short)(dirPerm.toShort() | 0300));
+      fs.mkdirs(new Path("/aa/1/aa/2/aa/3"), dirPerm);
+      checkPermission(fs, "/aa/1", permission);
+      checkPermission(fs, "/aa/1/aa/2", permission);
+      checkPermission(fs, "/aa/1/aa/2/aa/3", dirPerm);
+
       FsPermission filePerm = new FsPermission((short)0444);
       FSDataOutputStream out = fs.create(new Path("/b1/b2/b3.txt"), filePerm,
           true, conf.getInt("io.file.buffer.size", 4096),
@@ -126,7 +134,7 @@ public class TestPermission extends Test
       checkPermission(fs, "/b1/b2/b3.txt", filePerm);
       
       conf.set(FsPermission.UMASK_LABEL, "022");
-      FsPermission permission = 
+      permission = 
         FsPermission.createImmutable((short)0666);
       FileSystem.mkdirs(fs, new Path("/c1"), new FsPermission(permission));
       FileSystem.create(fs, new Path("/c1/c2.txt"),

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/HdfsTestDriver.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/HdfsTestDriver.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/HdfsTestDriver.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/HdfsTestDriver.java Wed Nov  2 05:34:31 2011
@@ -43,14 +43,17 @@ public class HdfsTestDriver {
   }
 
   public void run(String argv[]) {
+    int exitCode = -1;
     try {
-      pgd.driver(argv);
+      exitCode = pgd.driver(argv);
     } catch(Throwable e) {
       e.printStackTrace();
     }
+
+    System.exit(exitCode);
   }
 
   public static void main(String argv[]){
     new HdfsTestDriver().run(argv);
   }
-}
\ No newline at end of file
+}

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenFetcher.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenFetcher.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenFetcher.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenFetcher.java Wed Nov  2 05:34:31 2011
@@ -20,7 +20,6 @@ import static org.junit.Assert.assertEqu
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
@@ -37,7 +36,9 @@ import org.apache.hadoop.hdfs.security.t
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenRenewer;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -46,6 +47,7 @@ public class TestDelegationTokenFetcher 
   private Configuration conf;
   private URI uri;
   private static final String SERVICE_VALUE = "localhost:2005";
+  private static final Text KIND = new Text("TESTING-TOKEN-KIND");
   private static String tokenFile = "file.dta";
 
   @Before 
@@ -56,25 +58,59 @@ public class TestDelegationTokenFetcher 
     FileSystemTestHelper.addFileSystemForTesting(uri, conf, dfs);
   }
   
+  public static class FakeRenewer extends TokenRenewer {
+    static Token<?> lastRenewed = null;
+    static Token<?> lastCanceled = null;
+
+    @Override
+    public boolean handleKind(Text kind) {
+      return KIND.equals(kind);
+    }
+
+    @Override
+    public boolean isManaged(Token<?> token) throws IOException {
+      return true;
+    }
+
+    @Override
+    public long renew(Token<?> token, Configuration conf) {
+      lastRenewed = token;
+      return 0;
+    }
+
+    @Override
+    public void cancel(Token<?> token, Configuration conf) {
+      lastCanceled = token;
+    }
+    
+    public static void reset() {
+      lastRenewed = null;
+      lastCanceled = null;
+    }
+  }
+
   /**
    * Verify that when the DelegationTokenFetcher runs, it talks to the Namenode,
    * pulls out the correct user's token and successfully serializes it to disk.
    */
+  @SuppressWarnings("deprecation")
   @Test
   public void expectedTokenIsRetrievedFromDFS() throws Exception {
     final byte[] ident = new DelegationTokenIdentifier(new Text("owner"),
         new Text("renewer"), new Text("realuser")).getBytes();
     final byte[] pw = new byte[] { 42 };
-    final Text kind = new Text("MY-KIND");
     final Text service = new Text(uri.toString());
+    final String user = 
+        UserGroupInformation.getCurrentUser().getShortUserName();
 
     // Create a token for the fetcher to fetch, wire NN to return it when asked
     // for this particular user.
-    Token<DelegationTokenIdentifier> t = new Token<DelegationTokenIdentifier>(
-        ident, pw, kind, service);
-    when(dfs.getDelegationToken((String) null)).thenReturn(t);
+    Token<DelegationTokenIdentifier> t = 
+      new Token<DelegationTokenIdentifier>(ident, pw, KIND, service);
+    when(dfs.getDelegationToken(eq((String) null))).thenReturn(t);
     when(dfs.renewDelegationToken(eq(t))).thenReturn(1000L);
     when(dfs.getUri()).thenReturn(uri);
+    FakeRenewer.reset();
 
     FileSystem fileSys = FileSystem.getLocal(conf);
     try {
@@ -88,14 +124,13 @@ public class TestDelegationTokenFetcher 
       assertEquals(t, itr.next());
       assertTrue(!itr.hasNext());
 
-      DelegationTokenFetcher.main(new String[] { "-fs", uri.toString(),
-          "--print", tokenFile });
-      DelegationTokenFetcher.main(new String[] { "-fs", uri.toString(),
-          "--renew", tokenFile });
-      DelegationTokenFetcher.main(new String[] { "-fs", uri.toString(),
-          "--cancel", tokenFile });
-      verify(dfs).renewDelegationToken(eq(t));
-      verify(dfs).cancelDelegationToken(eq(t));
+      DelegationTokenFetcher.main(new String[] { "--print", tokenFile });
+      DelegationTokenFetcher.main(new String[] { "--renew", tokenFile });
+      assertEquals(t, FakeRenewer.lastRenewed);
+      FakeRenewer.reset();
+
+      DelegationTokenFetcher.main(new String[] { "--cancel", tokenFile });
+      assertEquals(t, FakeRenewer.lastCanceled);
     } finally {
       fileSys.delete(new Path(tokenFile), true);
     }

Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java?rev=1196458&r1=1196457&r2=1196458&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/unit/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java Wed Nov  2 05:34:31 2011
@@ -139,11 +139,11 @@ public class TestINodeFile {
     assertEquals("f", inf.getFullPathName());
     assertEquals("", inf.getLocalParentDir());
 
-    dir.addChild(inf, false, false);
+    dir.addChild(inf, false);
     assertEquals("d"+Path.SEPARATOR+"f", inf.getFullPathName());
     assertEquals("d", inf.getLocalParentDir());
     
-    root.addChild(dir, false, false);
+    root.addChild(dir, false);
     assertEquals(Path.SEPARATOR+"d"+Path.SEPARATOR+"f", inf.getFullPathName());
     assertEquals(Path.SEPARATOR+"d", dir.getFullPathName());