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 sz...@apache.org on 2012/10/19 04:28:07 UTC

svn commit: r1399950 [25/27] - in /hadoop/common/branches/HDFS-2802/hadoop-hdfs-project: ./ hadoop-hdfs-httpfs/ hadoop-hdfs-httpfs/dev-support/ hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/ hadoop-hdfs-httpfs/src/main/java/org/apac...

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java Fri Oct 19 02:25:55 2012
@@ -17,7 +17,12 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getFinalizedEditsFileName;
+import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
+import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
+
 import java.io.IOException;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -25,13 +30,9 @@ import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
-import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
 import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile;
+import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
-import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
-import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getFinalizedEditsFileName;
-import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
-
 import org.apache.hadoop.hdfs.server.namenode.NNStorageRetentionManager.StoragePurger;
 import org.junit.Assert;
 import org.junit.Before;
@@ -196,6 +197,35 @@ public class TestNNStorageRetentionManag
     runTest(tc);
   }
   
+  @Test
+  public void testRetainExtraLogsLimitedSegments() throws IOException {
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_KEY,
+        150);
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_MAX_EXTRA_EDITS_SEGMENTS_RETAINED_KEY, 2);
+    TestCaseDescription tc = new TestCaseDescription();
+    tc.addRoot("/foo1", NameNodeDirType.IMAGE);
+    tc.addRoot("/foo2", NameNodeDirType.EDITS);
+    tc.addImage("/foo1/current/" + getImageFileName(100), true);
+    tc.addImage("/foo1/current/" + getImageFileName(200), true);
+    tc.addImage("/foo1/current/" + getImageFileName(300), false);
+    tc.addImage("/foo1/current/" + getImageFileName(400), false);
+
+    tc.addLog("/foo2/current/" + getFinalizedEditsFileName(1, 100), true);
+    // Without lowering the max segments to retain, we'd retain all segments
+    // going back to txid 150 (300 - 150).
+    tc.addLog("/foo2/current/" + getFinalizedEditsFileName(101, 175), true);
+    tc.addLog("/foo2/current/" + getFinalizedEditsFileName(176, 200), true);
+    tc.addLog("/foo2/current/" + getFinalizedEditsFileName(201, 225), true);
+    tc.addLog("/foo2/current/" + getFinalizedEditsFileName(226, 240), true);
+    // Only retain 2 extra segments. The 301-400 segment is considered required,
+    // not extra.
+    tc.addLog("/foo2/current/" + getFinalizedEditsFileName(241, 275), false);
+    tc.addLog("/foo2/current/" + getFinalizedEditsFileName(276, 300), false);
+    tc.addLog("/foo2/current/" + getFinalizedEditsFileName(301, 400), false);
+    tc.addLog("/foo2/current/" + getInProgressEditsFileName(401), false);
+    runTest(tc);
+  }
+  
   private void runTest(TestCaseDescription tc) throws IOException {
     StoragePurger mockPurger =
       Mockito.mock(NNStorageRetentionManager.StoragePurger.class);
@@ -287,8 +317,10 @@ public class TestNNStorageRetentionManag
       return mockStorageForDirs(sds.toArray(new StorageDirectory[0]));
     }
     
+    @SuppressWarnings("unchecked")
     public FSEditLog mockEditLog(StoragePurger purger) {
       final List<JournalManager> jms = Lists.newArrayList();
+      final JournalSet journalSet = new JournalSet(0);
       for (FakeRoot root : dirRoots.values()) {
         if (!root.type.isOfType(NameNodeDirType.EDITS)) continue;
         
@@ -297,6 +329,7 @@ public class TestNNStorageRetentionManag
             root.mockStorageDir(), null);
         fjm.purger = purger;
         jms.add(fjm);
+        journalSet.add(fjm, false);
       }
 
       FSEditLog mockLog = Mockito.mock(FSEditLog.class);
@@ -314,6 +347,18 @@ public class TestNNStorageRetentionManag
           return null;
         }
       }).when(mockLog).purgeLogsOlderThan(Mockito.anyLong());
+      
+      Mockito.doAnswer(new Answer<Void>() {
+        
+        @Override
+        public Void answer(InvocationOnMock invocation) throws Throwable {
+          Object[] args = invocation.getArguments();
+          journalSet.selectInputStreams((Collection<EditLogInputStream>)args[0],
+              (long)((Long)args[1]), (boolean)((Boolean)args[2]));
+          return null;
+        }
+      }).when(mockLog).selectInputStreams(Mockito.anyCollection(),
+          Mockito.anyLong(), Mockito.anyBoolean());
       return mockLog;
     }
   }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameCache.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameCache.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameCache.java Fri Oct 19 02:25:55 2012
@@ -17,7 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 import org.junit.Test;
 

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java Fri Oct 19 02:25:55 2012
@@ -25,17 +25,15 @@ import static org.junit.Assert.fail;
 import java.io.File;
 import java.io.IOException;
 import java.util.List;
-import java.util.Random;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-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.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@@ -71,18 +69,6 @@ public class TestNameEditsConfigs {
     }
   }
 
-  private void writeFile(FileSystem fileSys, Path name, int repl)
-      throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short) repl, BLOCK_SIZE);
-    byte[] buffer = new byte[FILE_SIZE];
-    Random rand = new Random(SEED);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
-
   void checkImageAndEditsFilesExistence(File dir, 
                                         boolean shouldHaveImages,
                                         boolean shouldHaveEdits)
@@ -187,7 +173,8 @@ public class TestNameEditsConfigs {
 
     try {
       assertTrue(!fileSys.exists(file1));
-      writeFile(fileSys, file1, replication);
+      DFSTestUtil.createFile(fileSys, file1, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file1, replication);
       secondary.doCheckpoint();
     } finally {
@@ -224,7 +211,8 @@ public class TestNameEditsConfigs {
       assertTrue(fileSys.exists(file1));
       checkFile(fileSys, file1, replication);
       cleanupFile(fileSys, file1);
-      writeFile(fileSys, file2, replication);
+      DFSTestUtil.createFile(fileSys, file2, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file2, replication);
       secondary.doCheckpoint();
     } finally {
@@ -260,7 +248,8 @@ public class TestNameEditsConfigs {
       assertTrue(fileSys.exists(file2));
       checkFile(fileSys, file2, replication);
       cleanupFile(fileSys, file2);
-      writeFile(fileSys, file3, replication);
+      DFSTestUtil.createFile(fileSys, file3, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file3, replication);
       secondary.doCheckpoint();
     } finally {
@@ -364,7 +353,8 @@ public class TestNameEditsConfigs {
       fileSys = cluster.getFileSystem();
 
       assertTrue(!fileSys.exists(file1));
-      writeFile(fileSys, file1, replication);
+      DFSTestUtil.createFile(fileSys, file1, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file1, replication);
     } finally  {
       fileSys.close();
@@ -402,7 +392,8 @@ public class TestNameEditsConfigs {
       assertTrue(fileSys.exists(file1));
       checkFile(fileSys, file1, replication);
       cleanupFile(fileSys, file1);
-      writeFile(fileSys, file2, replication);
+      DFSTestUtil.createFile(fileSys, file2, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file2, replication);
     } finally {
       fileSys.close();
@@ -429,7 +420,8 @@ public class TestNameEditsConfigs {
       assertTrue(fileSys.exists(file2));
       checkFile(fileSys, file2, replication);
       cleanupFile(fileSys, file2);
-      writeFile(fileSys, file3, replication);
+      DFSTestUtil.createFile(fileSys, file3, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file3, replication);
     } finally {
       fileSys.close();
@@ -483,7 +475,8 @@ public class TestNameEditsConfigs {
       assertTrue(fileSys.exists(file3));
       checkFile(fileSys, file3, replication);
       cleanupFile(fileSys, file3);
-      writeFile(fileSys, file3, replication);
+      DFSTestUtil.createFile(fileSys, file3, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
+          (short) replication, SEED);
       checkFile(fileSys, file3, replication);
     } finally {
       fileSys.close();

Modified: hadoop/common/branches/HDFS-2802/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-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java Fri Oct 19 02:25:55 2012
@@ -17,7 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.File;
 import java.lang.management.ManagementFactory;
@@ -31,18 +32,20 @@ import javax.management.ObjectName;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.util.VersionInfo;
-
 import org.junit.Test;
 import org.mortbay.util.ajax.JSON;
 
-import junit.framework.Assert;
-
 /**
  * Class for testing {@link NameNodeMXBean} implementation
  */
 public class TestNameNodeMXBean {
+
+  /**
+   * Used to assert equality between doubles
+   */
+  private static final double DELTA = 0.000001;
+
   @SuppressWarnings({ "unchecked", "deprecation" })
   @Test
   public void testNameNodeMXBeanInfo() throws Exception {
@@ -60,36 +63,36 @@ public class TestNameNodeMXBean {
           "Hadoop:service=NameNode,name=NameNodeInfo");
       // get attribute "ClusterId"
       String clusterId = (String) mbs.getAttribute(mxbeanName, "ClusterId");
-      Assert.assertEquals(fsn.getClusterId(), clusterId);
+      assertEquals(fsn.getClusterId(), clusterId);
       // get attribute "BlockPoolId"
       String blockpoolId = (String) mbs.getAttribute(mxbeanName, 
           "BlockPoolId");
-      Assert.assertEquals(fsn.getBlockPoolId(), blockpoolId);
+      assertEquals(fsn.getBlockPoolId(), blockpoolId);
       // get attribute "Version"
       String version = (String) mbs.getAttribute(mxbeanName, "Version");
-      Assert.assertEquals(fsn.getVersion(), version);
-      Assert.assertTrue(version.equals(VersionInfo.getVersion()
+      assertEquals(fsn.getVersion(), version);
+      assertTrue(version.equals(VersionInfo.getVersion()
           + ", r" + VersionInfo.getRevision()));
       // get attribute "Used"
       Long used = (Long) mbs.getAttribute(mxbeanName, "Used");
-      Assert.assertEquals(fsn.getUsed(), used.longValue());
+      assertEquals(fsn.getUsed(), used.longValue());
       // get attribute "Total"
       Long total = (Long) mbs.getAttribute(mxbeanName, "Total");
-      Assert.assertEquals(fsn.getTotal(), total.longValue());
+      assertEquals(fsn.getTotal(), total.longValue());
       // get attribute "safemode"
       String safemode = (String) mbs.getAttribute(mxbeanName, "Safemode");
-      Assert.assertEquals(fsn.getSafemode(), safemode);
+      assertEquals(fsn.getSafemode(), safemode);
       // get attribute nondfs
       Long nondfs = (Long) (mbs.getAttribute(mxbeanName, "NonDfsUsedSpace"));
-      Assert.assertEquals(fsn.getNonDfsUsedSpace(), nondfs.longValue());
+      assertEquals(fsn.getNonDfsUsedSpace(), nondfs.longValue());
       // get attribute percentremaining
       Float percentremaining = (Float) (mbs.getAttribute(mxbeanName,
           "PercentRemaining"));
-      Assert.assertEquals(fsn.getPercentRemaining(), percentremaining
-          .floatValue());
+      assertEquals(fsn.getPercentRemaining(), percentremaining
+          .floatValue(), DELTA);
       // get attribute Totalblocks
       Long totalblocks = (Long) (mbs.getAttribute(mxbeanName, "TotalBlocks"));
-      Assert.assertEquals(fsn.getTotalBlocks(), totalblocks.longValue());
+      assertEquals(fsn.getTotalBlocks(), totalblocks.longValue());
       // get attribute alivenodeinfo
       String alivenodeinfo = (String) (mbs.getAttribute(mxbeanName,
           "LiveNodes"));
@@ -104,15 +107,15 @@ public class TestNameNodeMXBean {
         assertTrue(liveNode.containsKey("numBlocks"));
         assertTrue(((Long)liveNode.get("numBlocks")) == 0);
       }
-      Assert.assertEquals(fsn.getLiveNodes(), alivenodeinfo);
+      assertEquals(fsn.getLiveNodes(), alivenodeinfo);
       // get attribute deadnodeinfo
       String deadnodeinfo = (String) (mbs.getAttribute(mxbeanName,
           "DeadNodes"));
-      Assert.assertEquals(fsn.getDeadNodes(), deadnodeinfo);
+      assertEquals(fsn.getDeadNodes(), deadnodeinfo);
       // get attribute NameDirStatuses
       String nameDirStatuses = (String) (mbs.getAttribute(mxbeanName,
           "NameDirStatuses"));
-      Assert.assertEquals(fsn.getNameDirStatuses(), nameDirStatuses);
+      assertEquals(fsn.getNameDirStatuses(), nameDirStatuses);
       Map<String, Map<String, String>> statusMap =
         (Map<String, Map<String, String>>) JSON.parse(nameDirStatuses);
       Collection<URI> nameDirUris = cluster.getNameDirs(0);

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java Fri Oct 19 02:25:55 2012
@@ -18,16 +18,18 @@
 
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.spy;
+
 import java.io.File;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.util.HashSet;
 import java.util.Set;
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.doNothing;
-import static org.mockito.Mockito.spy;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -37,10 +39,8 @@ import org.apache.hadoop.hdfs.HdfsConfig
 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.FSEditLogOp.OpInstanceCache;
-import org.apache.hadoop.hdfs.server.namenode.FSImage;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteOp;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.OpInstanceCache;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -57,6 +57,7 @@ public class TestNameNodeRecovery {
 
   static {
     recoverStartOpt.setForce(MetaRecoveryContext.FORCE_ALL);
+    EditLogFileOutputStream.setShouldSkipFsyncForTesting(true);
   }
 
   static void runEditLogTest(EditLogTestSetup elts) throws IOException {
@@ -73,7 +74,7 @@ public class TestNameNodeRecovery {
 
       elts.addTransactionsToLog(elfos, cache);
       elfos.setReadyToFlush();
-      elfos.flushAndSync();
+      elfos.flushAndSync(true);
       elfos.close();
       elfos = null;
       file = new File(TEST_LOG_NAME);
@@ -200,15 +201,18 @@ public class TestNameNodeRecovery {
       this.paddingLength = paddingLength;
     }
 
+    @Override
     public void addTransactionsToLog(EditLogOutputStream elos,
         OpInstanceCache cache) throws IOException {
       padEditLog(elos, paddingLength);
     }
 
+    @Override
     public long getLastValidTxId() {
       return -1;
     }
 
+    @Override
     public Set<Long> getValidTxIds() {
       return new HashSet<Long>();
     } 
@@ -248,16 +252,19 @@ public class TestNameNodeRecovery {
       this.paddingLength = paddingLength;
     }
 
+    @Override
     public void addTransactionsToLog(EditLogOutputStream elos,
         OpInstanceCache cache) throws IOException {
       padEditLog(elos, paddingLength);
       addDeleteOpcode(elos, cache);
     }
 
+    @Override
     public long getLastValidTxId() {
       return 0;
     }
 
+    @Override
     public Set<Long> getValidTxIds() {
       return Sets.newHashSet(0L);
     } 
@@ -279,6 +286,7 @@ public class TestNameNodeRecovery {
     final private long BAD_TXID = 4;
     final private long MAX_TXID = 10;
     
+    @Override
     public void addTransactionsToLog(EditLogOutputStream elos,
         OpInstanceCache cache) throws IOException {
       for (long txid = 1; txid <= MAX_TXID; txid++) {
@@ -297,10 +305,12 @@ public class TestNameNodeRecovery {
       }
     }
 
+    @Override
     public long getLastValidTxId() {
       return BAD_TXID - 1;
     }
 
+    @Override
     public Set<Long> getValidTxIds() {
       return Sets.newHashSet(1L , 2L, 3L, 5L, 6L, 7L, 8L, 9L, 10L);
     }

Modified: hadoop/common/branches/HDFS-2802/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-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourceChecker.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourceChecker.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourceChecker.java Fri Oct 19 02:25:55 2012
@@ -17,6 +17,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.File;
 import java.io.IOException;
 import java.net.URISyntaxException;
@@ -29,14 +33,11 @@ import org.apache.hadoop.hdfs.DFSConfigK
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NameNodeResourceMonitor;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeResourceChecker.CheckedVolume;
+import org.apache.hadoop.util.Time;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 public class TestNameNodeResourceChecker {
   private Configuration conf;
   private File baseDir;
@@ -119,9 +120,9 @@ public class TestNameNodeResourceChecker
       Mockito.when(mockResourceChecker.hasAvailableDiskSpace()).thenReturn(false);
 
       // Make sure the NNRM thread has a chance to run.
-      long startMillis = System.currentTimeMillis();
+      long startMillis = Time.now();
       while (!cluster.getNameNode().isInSafeMode() &&
-          System.currentTimeMillis() < startMillis + (60 * 1000)) {
+          Time.now() < startMillis + (60 * 1000)) {
         Thread.sleep(1000);
       }
 

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourcePolicy.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourcePolicy.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourcePolicy.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeResourcePolicy.java Fri Oct 19 02:25:55 2012
@@ -17,7 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java Fri Oct 19 02:25:55 2012
@@ -18,12 +18,12 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 
+import static org.junit.Assert.assertTrue;
+
 import java.io.File;
 import java.util.ArrayList;
 import java.util.List;
 
-import junit.framework.TestCase;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -34,13 +34,14 @@ import org.apache.hadoop.hdfs.HdfsConfig
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+import org.junit.Test;
 
 
 
 /**
  * This tests InterDataNodeProtocol for block handling. 
  */
-public class TestNamenodeCapacityReport extends TestCase {
+public class TestNamenodeCapacityReport {
   private static final Log LOG = LogFactory.getLog(TestNamenodeCapacityReport.class);
 
   /**
@@ -48,6 +49,7 @@ public class TestNamenodeCapacityReport 
    * It verifies the block information from a datanode.
    * Then, it updates the block with new information and verifies again. 
    */
+  @Test
   public void testVolumeSize() throws Exception {
     Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestParallelImageWrite.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestParallelImageWrite.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestParallelImageWrite.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestParallelImageWrite.java Fri Oct 19 02:25:55 2012
@@ -18,32 +18,35 @@
 
 package org.apache.hadoop.hdfs.server.namenode;
 
-import junit.framework.TestCase;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.util.Collections;
+import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
-
-import java.util.Collections;
-import java.util.List;
-
-import java.io.File;
+import org.junit.Test;
 
 /**
  * A JUnit test for checking if restarting DFS preserves integrity.
  * Specifically with FSImage being written in parallel
  */
-public class TestParallelImageWrite extends TestCase {
+public class TestParallelImageWrite {
   private static final int NUM_DATANODES = 4;
   /** check if DFS remains in proper condition after a restart */
+  @Test
   public void testRestartDFS() throws Exception {
     final Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;
@@ -127,7 +130,7 @@ public class TestParallelImageWrite exte
    * @param fsn - the FSNamesystem being checked.
    * @param numImageDirs - the configured number of StorageDirectory of type IMAGE. 
    * @return - the md5 hash of the most recent FSImage files, which must all be the same.
-   * @throws AssertionFailedError if image files are empty or different,
+   * @throws AssertionError if image files are empty or different,
    *     if less than two StorageDirectory are provided, or if the
    *     actual number of StorageDirectory is less than configured.
    */

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathComponents.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathComponents.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathComponents.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestPathComponents.java Fri Oct 19 02:25:55 2012
@@ -17,15 +17,13 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.fs.Path;
-
-import org.junit.Test;
+import static org.junit.Assert.assertTrue;
 
 import java.util.Arrays;
 
-import static org.junit.Assert.*;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.junit.Test;
 
 
 /**

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java Fri Oct 19 02:25:55 2012
@@ -53,7 +53,7 @@ public class TestProcessCorruptBlocks {
    *      replicas (2) is equal to replication factor (2))
    */
   @Test
-  public void testWhenDecreasingReplication() throws IOException {
+  public void testWhenDecreasingReplication() throws Exception {
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L);
     conf.set(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, Integer.toString(2));
@@ -108,7 +108,7 @@ public class TestProcessCorruptBlocks {
    * 
    */
   @Test
-  public void testByAddingAnExtraDataNode() throws IOException {
+  public void testByAddingAnExtraDataNode() throws Exception {
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L);
     conf.set(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, Integer.toString(2));
@@ -159,7 +159,7 @@ public class TestProcessCorruptBlocks {
    *      replicas (1) is equal to replication factor (1))
    */
   @Test
-  public void testWithReplicationFactorAsOne() throws IOException {
+  public void testWithReplicationFactorAsOne() throws Exception {
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L);
     conf.set(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, Integer.toString(2));
@@ -208,7 +208,7 @@ public class TestProcessCorruptBlocks {
    *    Verify that all replicas are corrupt and 3 replicas are present.
    */
   @Test
-  public void testWithAllCorruptReplicas() throws IOException {
+  public void testWithAllCorruptReplicas() throws Exception {
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L);
     conf.set(DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, Integer.toString(2));

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java Fri Oct 19 02:25:55 2012
@@ -89,6 +89,7 @@ public class TestSaveNamespace {
       this.throwRTE = throwRTE;
     }
 
+    @Override
     public Void answer(InvocationOnMock invocation) throws Throwable {
       Object[] args = invocation.getArguments();
       StorageDirectory sd = (StorageDirectory)args[1];
@@ -535,6 +536,7 @@ public class TestSaveNamespace {
         delayer.waitForCall();
         // then cancel the saveNamespace
         Future<Void> cancelFuture = pool.submit(new Callable<Void>() {
+          @Override
           public Void call() throws Exception {
             canceler.cancel("cancelled");
             return null;

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNode.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNode.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNode.java Fri Oct 19 02:25:55 2012
@@ -17,10 +17,13 @@
 
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 
-import junit.framework.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystem;
@@ -79,14 +82,14 @@ public class TestSecureNameNode {
       try {
         Path p = new Path("/users");
         fs.mkdirs(p);
-        Assert.fail("user1 must not be allowed to write in /");
+        fail("user1 must not be allowed to write in /");
       } catch (IOException expected) {
       }
 
       Path p = new Path("/tmp/alpha");
       fs.mkdirs(p);
-      Assert.assertNotNull(fs.listStatus(p));
-      Assert.assertEquals(AuthenticationMethod.KERBEROS,
+      assertNotNull(fs.listStatus(p));
+      assertEquals(AuthenticationMethod.KERBEROS,
           ugi.getAuthenticationMethod());
     } finally {
       if (cluster != null) {

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java Fri Oct 19 02:25:55 2012
@@ -17,28 +17,30 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import junit.framework.TestCase;
-import java.io.*;
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.io.IOException;
 import java.net.URI;
 import java.util.Iterator;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
-
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
+import org.junit.Test;
 
 /**
  * This class tests the creation and validation of a checkpoint.
  */
-public class TestSecurityTokenEditLog extends TestCase {
+public class TestSecurityTokenEditLog {
   static final int NUM_DATA_NODES = 1;
 
   // This test creates NUM_THREADS threads and each thread does
@@ -47,6 +49,12 @@ public class TestSecurityTokenEditLog ex
   static final int NUM_THREADS = 100;
   static final int opsPerTrans = 3;
 
+  static {
+    // No need to fsync for the purposes of tests. This makes
+    // the tests run much faster.
+    EditLogFileOutputStream.setShouldSkipFsyncForTesting(true);
+  }
+
   //
   // an object that does a bunch of transactions
   //
@@ -62,6 +70,7 @@ public class TestSecurityTokenEditLog ex
     }
 
     // add a bunch of transactions.
+    @Override
     public void run() {
       FSEditLog editLog = namesystem.getEditLog();
 
@@ -84,6 +93,7 @@ public class TestSecurityTokenEditLog ex
   /**
    * Tests transaction logging in dfs.
    */
+  @Test
   public void testEditLog() throws IOException {
 
     // start a cluster 

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java Fri Oct 19 02:25:55 2012
@@ -19,6 +19,10 @@ package org.apache.hadoop.hdfs.server.na
 
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption.IMPORT;
 import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;
@@ -29,8 +33,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
 
-import junit.framework.TestCase;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -41,7 +43,6 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
-
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -56,13 +57,15 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.util.StringUtils;
-import org.junit.Assert;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 
 /**
  * Startup and checkpoint tests
  * 
  */
-public class TestStartup extends TestCase {
+public class TestStartup {
   public static final String NAME_NODE_HOST = "localhost:";
   public static final String WILDCARD_HTTP_HOST = "0.0.0.0:";
   private static final Log LOG =
@@ -74,21 +77,8 @@ public class TestStartup extends TestCas
   static final int fileSize = 8192;
   private long editsLength=0, fsimageLength=0;
 
-
-  private void writeFile(FileSystem fileSys, Path name, int repl)
-  throws IOException {
-    FSDataOutputStream stm = fileSys.create(name, true, fileSys.getConf()
-        .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096),
-        (short) repl, blockSize);
-    byte[] buffer = new byte[fileSize];
-    Random rand = new Random(seed);
-    rand.nextBytes(buffer);
-    stm.write(buffer);
-    stm.close();
-  }
-
-
-  protected void setUp() throws Exception {
+  @Before
+  public void setUp() throws Exception {
     config = new HdfsConfiguration();
     hdfsDir = new File(MiniDFSCluster.getBaseDirectory());
 
@@ -114,6 +104,7 @@ public class TestStartup extends TestCas
   /**
    * clean up
    */
+  @After
   public void tearDown() throws Exception {
     if ( hdfsDir.exists() && !FileUtil.fullyDelete(hdfsDir) ) {
       throw new IOException("Could not delete hdfs directory in tearDown '" + hdfsDir + "'");
@@ -145,7 +136,8 @@ public class TestStartup extends TestCas
       // create a file
       FileSystem fileSys = cluster.getFileSystem();
       Path file1 = new Path("t1");
-      this.writeFile(fileSys, file1, 1);
+      DFSTestUtil.createFile(fileSys, file1, fileSize, fileSize, blockSize, 
+          (short) 1, seed);
 
       LOG.info("--doing checkpoint");
       sn.doCheckpoint();  // this shouldn't fail
@@ -256,6 +248,7 @@ public class TestStartup extends TestCas
    * checkpoint for edits and image is the same directory
    * @throws IOException
    */
+  @Test
   public void testChkpointStartup2() throws IOException{
     LOG.info("--starting checkpointStartup2 - same directory for checkpoint");
     // different name dirs
@@ -281,6 +274,7 @@ public class TestStartup extends TestCas
    * checkpoint for edits and image are different directories 
    * @throws IOException
    */
+  @Test
   public void testChkpointStartup1() throws IOException{
     //setUpConfig();
     LOG.info("--starting testStartup Recovery");
@@ -305,6 +299,7 @@ public class TestStartup extends TestCas
    * secondary node copies fsimage and edits into correct separate directories.
    * @throws IOException
    */
+  @Test
   public void testSNNStartup() throws IOException{
     //setUpConfig();
     LOG.info("--starting SecondNN startup test");
@@ -368,6 +363,7 @@ public class TestStartup extends TestCas
     }
   }
   
+  @Test
   public void testCompression() throws IOException {
     LOG.info("Test compressing image.");
     Configuration conf = new Configuration();
@@ -424,6 +420,7 @@ public class TestStartup extends TestCas
     namenode.join();
   }
   
+  @Test
   public void testImageChecksum() throws Exception {
     LOG.info("Test uncompressed image checksum");
     testImageChecksum(false);
@@ -433,16 +430,15 @@ public class TestStartup extends TestCas
 
   private void testImageChecksum(boolean compress) throws Exception {
     MiniDFSCluster cluster = null;
-    Configuration conf = new HdfsConfiguration();
     if (compress) {
-      conf.setBoolean(DFSConfigKeys.DFS_IMAGE_COMPRESSION_CODEC_KEY, true);
+      config.setBoolean(DFSConfigKeys.DFS_IMAGE_COMPRESSION_CODEC_KEY, true);
     }
 
     try {
         LOG.info("\n===========================================\n" +
                  "Starting empty cluster");
         
-        cluster = new MiniDFSCluster.Builder(conf)
+        cluster = new MiniDFSCluster.Builder(config)
           .numDataNodes(0)
           .format(true)
           .build();
@@ -469,7 +465,7 @@ public class TestStartup extends TestCas
         LOG.info("\n===========================================\n" +
         "Starting same cluster after simulated crash");
         try {
-          cluster = new MiniDFSCluster.Builder(conf)
+          cluster = new MiniDFSCluster.Builder(config)
             .numDataNodes(0)
             .format(false)
             .build();
@@ -491,24 +487,24 @@ public class TestStartup extends TestCas
    * restarts, the still alive datanodes should not have any trouble in getting
    * registrant again.
    */
+  @Test
   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);
+    localFileSys = FileSystem.getLocal(config);
     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());
+    config.set(DFSConfigKeys.DFS_HOSTS_EXCLUDE, excludeFile.toUri().getPath());
     writeConfigFile(localFileSys, excludeFile, null);
-    conf.set(DFSConfigKeys.DFS_HOSTS, hostsFile.toUri().getPath());
+    config.set(DFSConfigKeys.DFS_HOSTS, hostsFile.toUri().getPath());
     // write into hosts file
     ArrayList<String>list = new ArrayList<String>();
     byte b[] = {127, 0, 0, 1};
@@ -518,14 +514,14 @@ public class TestStartup extends TestCas
     int numDatanodes = 1;
     
     try {
-      cluster = new MiniDFSCluster.Builder(conf)
+      cluster = new MiniDFSCluster.Builder(config)
       .numDataNodes(numDatanodes).setupHostsFile(true).build();
       cluster.waitActive();
   
       cluster.restartNameNode();
       NamenodeProtocols nn = cluster.getNameNodeRpc();
       assertNotNull(nn);
-      Assert.assertTrue(cluster.isDataNodeUp());
+      assertTrue(cluster.isDataNodeUp());
       
       DatanodeInfo[] info = nn.getDatanodeReport(DatanodeReportType.LIVE);
       for (int i = 0 ; i < 5 && info.length != numDatanodes; i++) {

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupOptionUpgrade.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupOptionUpgrade.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupOptionUpgrade.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupOptionUpgrade.java Fri Oct 19 02:25:55 2012
@@ -17,9 +17,11 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.net.URI;
 import java.util.Collections;
-import junit.framework.Assert;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -67,7 +69,7 @@ public class TestStartupOptionUpgrade {
   public void testStartupOptUpgradeFrom204() throws Exception {
     layoutVersion = Feature.RESERVED_REL20_204.getLayoutVersion();
     storage.processStartupOptionsForUpgrade(startOpt, layoutVersion);
-    Assert.assertTrue("Clusterid should start with CID", storage.getClusterID()
+    assertTrue("Clusterid should start with CID", storage.getClusterID()
         .startsWith("CID"));
   }
 
@@ -83,7 +85,7 @@ public class TestStartupOptionUpgrade {
     startOpt.setClusterId("cid");
     layoutVersion = Feature.RESERVED_REL22.getLayoutVersion();
     storage.processStartupOptionsForUpgrade(startOpt, layoutVersion);
-    Assert.assertEquals("Clusterid should match with the given clusterid",
+    assertEquals("Clusterid should match with the given clusterid",
         "cid", storage.getClusterID());
   }
 
@@ -101,7 +103,7 @@ public class TestStartupOptionUpgrade {
     storage.setClusterID("currentcid");
     layoutVersion = Feature.FEDERATION.getLayoutVersion();
     storage.processStartupOptionsForUpgrade(startOpt, layoutVersion);
-    Assert.assertEquals("Clusterid should match with the existing one",
+    assertEquals("Clusterid should match with the existing one",
         "currentcid", storage.getClusterID());
   }
 
@@ -119,7 +121,7 @@ public class TestStartupOptionUpgrade {
     storage.setClusterID("currentcid");
     layoutVersion = Feature.FEDERATION.getLayoutVersion();
     storage.processStartupOptionsForUpgrade(startOpt, layoutVersion);
-    Assert.assertEquals("Clusterid should match with the existing one",
+    assertEquals("Clusterid should match with the existing one",
         "currentcid", storage.getClusterID());
   }
 
@@ -137,7 +139,7 @@ public class TestStartupOptionUpgrade {
     storage.setClusterID("currentcid");
     layoutVersion = Feature.FEDERATION.getLayoutVersion();
     storage.processStartupOptionsForUpgrade(startOpt, layoutVersion);
-    Assert.assertEquals("Clusterid should match with the existing one",
+    assertEquals("Clusterid should match with the existing one",
         "currentcid", storage.getClusterID());
   }
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java Fri Oct 19 02:25:55 2012
@@ -18,9 +18,14 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 
+import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getFinalizedEditsFileName;
+import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
+import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.spy;
 
 import java.io.File;
 import java.io.IOException;
@@ -28,9 +33,6 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Set;
 
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.spy;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.cli.CLITestCmdDFS;
@@ -45,11 +47,6 @@ import org.apache.hadoop.hdfs.HdfsConfig
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
-
-import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
-import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getFinalizedEditsFileName;
-import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
-
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStreamFile.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStreamFile.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStreamFile.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStreamFile.java Fri Oct 19 02:25:55 2012
@@ -53,21 +53,26 @@ import org.mortbay.jetty.InclusiveByteRa
  */
 class MockFSInputStream extends FSInputStream {
   long currentPos = 0;
+  @Override
   public int read() throws IOException {
     return (int)(currentPos++);
   }
 
+  @Override
   public void close() throws IOException {
   }
 
+  @Override
   public void seek(long pos) throws IOException {
     currentPos = pos;
   }
   
+  @Override
   public long getPos() throws IOException {
     return currentPos;
   }
 
+  @Override
   public boolean seekToNewSource(long targetPos) throws IOException {
     return false;
   }
@@ -87,6 +92,7 @@ public class TestStreamFile {
   StreamFile sfile = new StreamFile() {
     private static final long serialVersionUID = -5513776238875189473L;
   
+    @Override
     public ServletContext getServletContext() {
       return mockServletContext;
     }
@@ -216,6 +222,7 @@ public class TestStreamFile {
 
         private static final long serialVersionUID = 7715590481809562722L;
 
+        @Override
         public ServletContext getServletContext() {
           return mockServletContext;
         }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java Fri Oct 19 02:25:55 2012
@@ -17,16 +17,27 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.SocketTimeoutException;
+import java.net.URL;
 import java.util.Collections;
 import java.util.List;
 
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.http.HttpServerFunctionalTest;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.Test;
@@ -99,4 +110,45 @@ public class TestTransferFsImage {
       cluster.shutdown();      
     }
   }
+
+  /**
+   * Test to verify the read timeout
+   */
+  @Test(timeout = 5000)
+  public void testImageTransferTimeout() throws Exception {
+    HttpServer testServer = HttpServerFunctionalTest.createServer("hdfs");
+    try {
+      testServer.addServlet("GetImage", "/getimage", TestGetImageServlet.class);
+      testServer.start();
+      URL serverURL = HttpServerFunctionalTest.getServerURL(testServer);
+      TransferFsImage.timeout = 2000;
+      try {
+        TransferFsImage.getFileClient(serverURL.getAuthority(), "txid=1", null,
+            null, false);
+        fail("TransferImage Should fail with timeout");
+      } catch (SocketTimeoutException e) {
+        assertEquals("Read should timeout", "Read timed out", e.getMessage());
+      }
+    } finally {
+      if (testServer != null) {
+        testServer.stop();
+      }
+    }
+  }
+
+  public static class TestGetImageServlet extends HttpServlet {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    protected void doGet(HttpServletRequest req, HttpServletResponse resp)
+        throws ServletException, IOException {
+      synchronized (this) {
+        try {
+          wait(5000);
+        } catch (InterruptedException e) {
+          // Ignore
+        }
+      }
+    }
+  }
 }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java Fri Oct 19 02:25:55 2012
@@ -17,19 +17,21 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.junit.Assert.*;
-import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.io.File;
 import java.io.IOException;
-import junit.framework.Assert;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
 
 /**
  * This class tests the validation of the configuration object when passed 
@@ -53,7 +55,7 @@ public class TestValidateConfigurationSe
     DFSTestUtil.formatNameNode(conf);
     try {
       NameNode nameNode = new NameNode(conf);
-      Assert.fail("Should have throw the exception since the ports match");
+      fail("Should have throw the exception since the ports match");
     } catch (IOException e) {
       // verify we're getting the right IOException
       assertTrue(e.toString().contains("dfs.namenode.rpc-address (")); 

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java Fri Oct 19 02:25:55 2012
@@ -41,6 +41,7 @@ import org.apache.hadoop.hdfs.server.dat
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Time;
 
 import com.google.common.base.Supplier;
 
@@ -64,7 +65,7 @@ public abstract class HATestUtil {
    * @throws CouldNotCatchUpException if the standby doesn't catch up to the
    *         active in NN_LAG_TIMEOUT milliseconds
    */
-  static void waitForStandbyToCatchUp(NameNode active,
+  public static void waitForStandbyToCatchUp(NameNode active,
       NameNode standby) throws InterruptedException, IOException, CouldNotCatchUpException {
     
     long activeTxId = active.getNamesystem().getFSImage().getEditLog()
@@ -72,8 +73,8 @@ public abstract class HATestUtil {
     
     active.getRpcServer().rollEditLog();
     
-    long start = System.currentTimeMillis();
-    while (System.currentTimeMillis() - start < TestEditLogTailer.NN_LAG_TIMEOUT) {
+    long start = Time.now();
+    while (Time.now() - start < TestEditLogTailer.NN_LAG_TIMEOUT) {
       long nn2HighestTxId = standby.getNamesystem().getFSImage()
         .getLastAppliedTxId();
       if (nn2HighestTxId >= activeTxId) {
@@ -206,13 +207,13 @@ public abstract class HATestUtil {
   
   public static void waitForCheckpoint(MiniDFSCluster cluster, int nnIdx,
       List<Integer> txids) throws InterruptedException {
-    long start = System.currentTimeMillis();
+    long start = Time.now();
     while (true) {
       try {
         FSImageTestUtil.assertNNHasCheckpoints(cluster, nnIdx, txids);
         return;
       } catch (AssertionError err) {
-        if (System.currentTimeMillis() - start > 10000) {
+        if (Time.now() - start > 10000) {
           throw err;
         } else {
           Thread.sleep(300);

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java Fri Oct 19 02:25:55 2012
@@ -17,11 +17,13 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
-import java.io.ByteArrayInputStream;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.io.File;
 import java.io.IOException;
 import java.net.URI;
-import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
@@ -36,20 +38,16 @@ import org.apache.hadoop.hdfs.server.nam
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
-import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
-import static org.junit.Assert.*;
-
 public class TestBootstrapStandby {
   private static final Log LOG = LogFactory.getLog(TestBootstrapStandby.class);
   
@@ -77,7 +75,7 @@ public class TestBootstrapStandby {
   }
   
   @After
-  public void shutdownCluster() throws IOException {
+  public void shutdownCluster() {
     if (cluster != null) {
       cluster.shutdown();
     }
@@ -127,7 +125,7 @@ public class TestBootstrapStandby {
     // Make checkpoint
     NameNodeAdapter.enterSafeMode(nn0, false);
     NameNodeAdapter.saveNamespace(nn0);
-    NameNodeAdapter.leaveSafeMode(nn0, false);
+    NameNodeAdapter.leaveSafeMode(nn0);
     long expectedCheckpointTxId = NameNodeAdapter.getNamesystem(nn0)
       .getFSImage().getMostRecentCheckpointTxId();
     assertEquals(6, expectedCheckpointTxId);

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSUpgradeWithHA.java Fri Oct 19 02:25:55 2012
@@ -17,18 +17,20 @@
 */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;
 import java.net.URI;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSZKFailoverController.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSZKFailoverController.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSZKFailoverController.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDFSZKFailoverController.java Fri Oct 19 02:25:55 2012
@@ -17,7 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.util.concurrent.TimeoutException;
 
@@ -26,11 +27,11 @@ import org.apache.hadoop.fs.CommonConfig
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.ha.ClientBaseWithFixes;
+import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HealthMonitor;
+import org.apache.hadoop.ha.TestNodeFencer.AlwaysSucceedFencer;
 import org.apache.hadoop.ha.ZKFCTestUtil;
 import org.apache.hadoop.ha.ZKFailoverController;
-import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
-import org.apache.hadoop.ha.TestNodeFencer.AlwaysSucceedFencer;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
@@ -44,7 +45,6 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-
 import com.google.common.base.Supplier;
 
 public class TestDFSZKFailoverController extends ClientBaseWithFixes {

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java Fri Oct 19 02:25:55 2012
@@ -41,12 +41,12 @@ import org.apache.hadoop.hdfs.MiniDFSNNT
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java Fri Oct 19 02:25:55 2012
@@ -98,6 +98,7 @@ public class TestDNFencingWithReplicatio
       }
     }
     
+    @Override
     public String toString() {
       return "Toggler for " + path;
     }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java Fri Oct 19 02:25:55 2012
@@ -17,7 +17,12 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
@@ -32,6 +37,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.AbstractFileSystem;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -56,7 +62,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.base.Joiner;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 
 /**
  * Test case for client support of delegation tokens in an HA cluster.
@@ -111,7 +116,8 @@ public class TestDelegationTokensWithHA 
   
   @Test
   public void testDelegationTokenDFSApi() throws Exception {
-    Token<DelegationTokenIdentifier> token = dfs.getDelegationToken("JobTracker");
+    final Token<DelegationTokenIdentifier> token =
+        getDelegationToken(fs, "JobTracker");
     DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();
     byte[] tokenId = token.getIdentifier();
     identifier.readFields(new DataInputStream(
@@ -152,13 +158,14 @@ public class TestDelegationTokensWithHA 
   @SuppressWarnings("deprecation")
   @Test
   public void testDelegationTokenWithDoAs() throws Exception {
-    final Token<DelegationTokenIdentifier> token = 
-        dfs.getDelegationToken("JobTracker");
+    final Token<DelegationTokenIdentifier> token =
+        getDelegationToken(fs, "JobTracker");
     final UserGroupInformation longUgi = UserGroupInformation
         .createRemoteUser("JobTracker/foo.com@FOO.COM");
     final UserGroupInformation shortUgi = UserGroupInformation
         .createRemoteUser("JobTracker");
     longUgi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
       public Void run() throws Exception {
         DistributedFileSystem dfs = (DistributedFileSystem)
             HATestUtil.configureFailoverFs(cluster, conf);
@@ -168,6 +175,7 @@ public class TestDelegationTokensWithHA 
       }
     });
     shortUgi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
       public Void run() throws Exception {
         DistributedFileSystem dfs = (DistributedFileSystem)
             HATestUtil.configureFailoverFs(cluster, conf);
@@ -176,6 +184,7 @@ public class TestDelegationTokensWithHA 
       }
     });
     longUgi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
       public Void run() throws Exception {
         DistributedFileSystem dfs = (DistributedFileSystem)
             HATestUtil.configureFailoverFs(cluster, conf);
@@ -188,8 +197,8 @@ public class TestDelegationTokensWithHA 
   
   @Test
   public void testHAUtilClonesDelegationTokens() throws Exception {
-    final Token<DelegationTokenIdentifier> token = 
-      dfs.getDelegationToken("test");
+    final Token<DelegationTokenIdentifier> token =
+        getDelegationToken(fs, "JobTracker");
 
     UserGroupInformation ugi = UserGroupInformation.createRemoteUser("test");
     
@@ -250,8 +259,9 @@ public class TestDelegationTokensWithHA 
     URI hAUri = HATestUtil.getLogicalUri(cluster);
     String haService = HAUtil.buildTokenServiceForLogicalUri(hAUri).toString();
     assertEquals(haService, dfs.getCanonicalServiceName());
-    Token<?> token = dfs.getDelegationToken(
-        UserGroupInformation.getCurrentUser().getShortUserName());
+    final String renewer = UserGroupInformation.getCurrentUser().getShortUserName();
+    final Token<DelegationTokenIdentifier> token =
+        getDelegationToken(dfs, renewer);
     assertEquals(haService, token.getService().toString());
     // make sure the logical uri is handled correctly
     token.renew(dfs.getConf());
@@ -273,6 +283,13 @@ public class TestDelegationTokensWithHA 
     token.cancel(conf);
   }
   
+  @SuppressWarnings("unchecked")
+  private Token<DelegationTokenIdentifier> getDelegationToken(FileSystem fs,
+      String renewer) throws IOException {
+    final Token<?> tokens[] = fs.addDelegationTokens(renewer, null);
+    assertEquals(1, tokens.length);
+    return (Token<DelegationTokenIdentifier>) tokens[0];
+  }
   enum TokenTestAction {
     RENEW, CANCEL;
   }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java Fri Oct 19 02:25:55 2012
@@ -22,8 +22,6 @@ import static org.junit.Assert.assertTru
 import java.io.File;
 import java.io.IOException;
 import java.net.URI;
-import java.util.List;
-import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java Fri Oct 19 02:25:55 2012
@@ -17,7 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 
 import java.io.File;
 import java.io.FileOutputStream;
@@ -33,6 +34,7 @@ import org.apache.hadoop.fs.permission.F
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
@@ -51,6 +53,12 @@ public class TestEditLogsDuringFailover 
   private static final Log LOG =
     LogFactory.getLog(TestEditLogsDuringFailover.class);
   private static final int NUM_DIRS_IN_LOG = 5;
+
+  static {
+    // No need to fsync for the purposes of tests. This makes
+    // the tests run much faster.
+    EditLogFileOutputStream.setShouldSkipFsyncForTesting(true);
+  }
   
   @Test
   public void testStartup() throws Exception {

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailoverWithBlockTokensEnabled.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailoverWithBlockTokensEnabled.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailoverWithBlockTokensEnabled.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailoverWithBlockTokensEnabled.java Fri Oct 19 02:25:55 2012
@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.server.dat
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Time;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -124,7 +125,7 @@ public class TestFailoverWithBlockTokens
               BlockTokenIdentifier id = lb.getBlockToken().decodeIdentifier();
               // This will make the token invalid, since the password
               // won't match anymore
-              id.setExpiryDate(System.currentTimeMillis() + 10);
+              id.setExpiryDate(Time.now() + 10);
               Token<BlockTokenIdentifier> newToken =
                   new Token<BlockTokenIdentifier>(id.getBytes(),
                       token.getPassword(), token.getKind(), token.getService());

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureOfSharedDir.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureOfSharedDir.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureOfSharedDir.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureOfSharedDir.java Fri Oct 19 02:25:55 2012
@@ -19,7 +19,10 @@ package org.apache.hadoop.hdfs.server.na
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java Fri Oct 19 02:25:55 2012
@@ -17,8 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
-import static org.junit.Assert.*;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.net.URI;

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAFsck.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAFsck.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAFsck.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAFsck.java Fri Oct 19 02:25:55 2012
@@ -17,13 +17,12 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
 
-import junit.framework.Assert;
-
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
@@ -96,7 +95,7 @@ public class TestHAFsck {
         new String[]{"/", "-files"});
     String result = bStream.toString();
     System.out.println("output from fsck:\n" + result);
-    Assert.assertEquals(0, errCode);
+    assertEquals(0, errCode);
     assertTrue(result.contains("/test1"));
     assertTrue(result.contains("/test2"));
   }

Modified: hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java?rev=1399950&r1=1399949&r2=1399950&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java (original)
+++ hadoop/common/branches/HDFS-2802/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java Fri Oct 19 02:25:55 2012
@@ -17,7 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.ha;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.List;
@@ -90,7 +91,7 @@ public class TestHASafeMode {
   }
   
   @After
-  public void shutdownCluster() throws IOException {
+  public void shutdownCluster() {
     if (cluster != null) {
       cluster.shutdown();
     }
@@ -407,7 +408,7 @@ public class TestHASafeMode {
         4*BLOCK_SIZE, (short) 3, 1L);
     NameNodeAdapter.enterSafeMode(nn0, false);
     NameNodeAdapter.saveNamespace(nn0);
-    NameNodeAdapter.leaveSafeMode(nn0, false);
+    NameNodeAdapter.leaveSafeMode(nn0);
     
     // OP_ADD for 2 blocks
     DFSTestUtil.createFile(fs, new Path("/test2"),
@@ -419,8 +420,8 @@ public class TestHASafeMode {
     restartActive();
   }
   
-  private void assertSafeMode(NameNode nn, int safe, int total) {
-    String status = nn1.getNamesystem().getSafemode();
+  private static void assertSafeMode(NameNode nn, int safe, int total) {
+    String status = nn.getNamesystem().getSafemode();
     if (safe == total) {
       assertTrue("Bad safemode status: '" + status + "'",
           status.startsWith(