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 2013/04/23 03:18:37 UTC

svn commit: r1470771 - in /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/ src/test/java/org/apache/hadoop/hdfs/ src/test/java/...

Author: szetszwo
Date: Tue Apr 23 01:18:36 2013
New Revision: 1470771

URL: http://svn.apache.org/r1470771
Log:
HDFS-4725. Fix HDFS file handle leaks in FSEditLog, NameNode, OfflineEditsBinaryLoader and some tests.  Contributed by Chris Nauroth

Modified:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsBinaryLoader.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1470771&r1=1470770&r2=1470771&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Tue Apr 23 01:18:36 2013
@@ -360,6 +360,9 @@ Trunk (Unreleased)
     HDFS-4674. TestBPOfferService fails on Windows due to failure parsing 
     datanode data directory as URI. (Chris Nauroth via suresh)
 
+    HDFS-4725. Fix HDFS file handle leaks in FSEditLog, NameNode,
+    OfflineEditsBinaryLoader and some tests.  (Chris Nauroth via szetszwo)
+
   BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
 
     HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java?rev=1470771&r1=1470770&r2=1470771&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java Tue Apr 23 01:18:36 2013
@@ -317,21 +317,23 @@ public class FSEditLog implements LogsPu
       LOG.debug("Closing log when already closed");
       return;
     }
-    if (state == State.IN_SEGMENT) {
-      assert editLogStream != null;
-      waitForSyncToFinish();
-      endCurrentLogSegment(true);
-    }
-    
-    if (journalSet != null && !journalSet.isEmpty()) {
-      try {
-        journalSet.close();
-      } catch (IOException ioe) {
-        LOG.warn("Error closing journalSet", ioe);
+
+    try {
+      if (state == State.IN_SEGMENT) {
+        assert editLogStream != null;
+        waitForSyncToFinish();
+        endCurrentLogSegment(true);
       }
+    } finally {
+      if (journalSet != null && !journalSet.isEmpty()) {
+        try {
+          journalSet.close();
+        } catch (IOException ioe) {
+          LOG.warn("Error closing journalSet", ioe);
+        }
+      }
+      state = State.CLOSED;
     }
-
-    state = State.CLOSED;
   }
 
 
@@ -583,6 +585,7 @@ public class FSEditLog implements LogsPu
                 "due to " + e.getMessage() + ". " +
                 "Unsynced transactions: " + (txid - synctxid);
             LOG.fatal(msg, new Exception());
+            IOUtils.cleanup(LOG, journalSet);
             terminate(1, msg);
           }
         } finally {
@@ -606,6 +609,7 @@ public class FSEditLog implements LogsPu
               "Could not sync enough journals to persistent storage. "
               + "Unsynced transactions: " + (txid - synctxid);
           LOG.fatal(msg, new Exception());
+          IOUtils.cleanup(LOG, journalSet);
           terminate(1, msg);
         }
       }

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java?rev=1470771&r1=1470770&r2=1470771&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java Tue Apr 23 01:18:36 2013
@@ -651,13 +651,14 @@ public class NameNode {
       }
     } catch (ServiceFailedException e) {
       LOG.warn("Encountered exception while exiting state ", e);
-    }
-    stopCommonServices();
-    if (metrics != null) {
-      metrics.shutdown();
-    }
-    if (namesystem != null) {
-      namesystem.shutdown();
+    } finally {
+      stopCommonServices();
+      if (metrics != null) {
+        metrics.shutdown();
+      }
+      if (namesystem != null) {
+        namesystem.shutdown();
+      }
     }
   }
 

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsBinaryLoader.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsBinaryLoader.java?rev=1470771&r1=1470770&r2=1470771&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsBinaryLoader.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsBinaryLoader.java Tue Apr 23 01:18:36 2013
@@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.tools.offl
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
 
 import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
+import org.apache.hadoop.io.IOUtils;
 
 /**
  * OfflineEditsBinaryLoader loads edits from a binary edits file
@@ -59,43 +60,49 @@ class OfflineEditsBinaryLoader implement
    */
   @Override
   public void loadEdits() throws IOException {
-    visitor.start(inputStream.getVersion());
-    while (true) {
-      try {
-        FSEditLogOp op = inputStream.readOp();
-        if (op == null)
-          break;
-        if (fixTxIds) {
-          if (nextTxId <= 0) {
-            nextTxId = op.getTransactionId();
+    try {
+      visitor.start(inputStream.getVersion());
+      while (true) {
+        try {
+          FSEditLogOp op = inputStream.readOp();
+          if (op == null)
+            break;
+          if (fixTxIds) {
             if (nextTxId <= 0) {
-              nextTxId = 1;
+              nextTxId = op.getTransactionId();
+              if (nextTxId <= 0) {
+                nextTxId = 1;
+              }
             }
+            op.setTransactionId(nextTxId);
+            nextTxId++;
           }
-          op.setTransactionId(nextTxId);
-          nextTxId++;
-        }
-        visitor.visitOp(op);
-      } catch (IOException e) {
-        if (!recoveryMode) {
-          // Tell the visitor to clean up, then re-throw the exception
-          LOG.error("Got IOException at position " + inputStream.getPosition());
-          visitor.close(e);
-          throw e;
-        }
-        LOG.error("Got IOException while reading stream!  Resyncing.", e);
-        inputStream.resync();
-      } catch (RuntimeException e) {
-        if (!recoveryMode) {
-          // Tell the visitor to clean up, then re-throw the exception
-          LOG.error("Got RuntimeException at position " + inputStream.getPosition());
-          visitor.close(e);
-          throw e;
+          visitor.visitOp(op);
+        } catch (IOException e) {
+          if (!recoveryMode) {
+            // Tell the visitor to clean up, then re-throw the exception
+            LOG.error("Got IOException at position " +
+              inputStream.getPosition());
+            visitor.close(e);
+            throw e;
+          }
+          LOG.error("Got IOException while reading stream!  Resyncing.", e);
+          inputStream.resync();
+        } catch (RuntimeException e) {
+          if (!recoveryMode) {
+            // Tell the visitor to clean up, then re-throw the exception
+            LOG.error("Got RuntimeException at position " +
+              inputStream.getPosition());
+            visitor.close(e);
+            throw e;
+          }
+          LOG.error("Got RuntimeException while reading stream!  Resyncing.", e);
+          inputStream.resync();
         }
-        LOG.error("Got RuntimeException while reading stream!  Resyncing.", e);
-        inputStream.resync();
       }
+      visitor.close(null);
+    } finally {
+      IOUtils.cleanup(LOG, inputStream);
     }
-    visitor.close(null);
   }
-}
\ No newline at end of file
+}

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java?rev=1470771&r1=1470770&r2=1470771&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java Tue Apr 23 01:18:36 2013
@@ -48,6 +48,8 @@ import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -91,6 +93,8 @@ import com.google.common.base.Joiner;
 
 /** Utilities for HDFS tests */
 public class DFSTestUtil {
+
+  private static final Log LOG = LogFactory.getLog(DFSTestUtil.class);
   
   private static Random gen = new Random();
   private static String[] dirNames = {
@@ -723,7 +727,11 @@ public class DFSTestUtil {
     File file = new File(filename);
     DataInputStream in = new DataInputStream(new FileInputStream(file));
     byte[] content = new byte[(int)file.length()];
-    in.readFully(content);
+    try {
+      in.readFully(content);
+    } finally {
+      IOUtils.cleanup(LOG, in);
+    }
     return content;
   }
 

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java?rev=1470771&r1=1470770&r2=1470771&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java Tue Apr 23 01:18:36 2013
@@ -631,44 +631,48 @@ public class TestDistributedFileSystem {
         true);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(2).build();
-    DistributedFileSystem fs = cluster.getFileSystem();
-    // Create two files
-    Path tmpFile1 = new Path("/tmpfile1.dat");
-    Path tmpFile2 = new Path("/tmpfile2.dat");
-    DFSTestUtil.createFile(fs, tmpFile1, 1024, (short) 2, 0xDEADDEADl);
-    DFSTestUtil.createFile(fs, tmpFile2, 1024, (short) 2, 0xDEADDEADl);
-    // Get locations of blocks of both files and concat together
-    BlockLocation[] blockLocs1 = fs.getFileBlockLocations(tmpFile1, 0, 1024);
-    BlockLocation[] blockLocs2 = fs.getFileBlockLocations(tmpFile2, 0, 1024);
-    BlockLocation[] blockLocs = (BlockLocation[]) ArrayUtils.addAll(blockLocs1,
-        blockLocs2);
-    // Fetch VolumeBlockLocations in batch
-    BlockStorageLocation[] locs = fs.getFileBlockStorageLocations(Arrays
-        .asList(blockLocs));
-    int counter = 0;
-    // Print out the list of ids received for each block
-    for (BlockStorageLocation l : locs) {
-      for (int i = 0; i < l.getVolumeIds().length; i++) {
-        VolumeId id = l.getVolumeIds()[i];
-        String name = l.getNames()[i];
-        if (id != null) {
-          System.out.println("Datanode " + name + " has block " + counter
-              + " on volume id " + id.toString());
+    try {
+      DistributedFileSystem fs = cluster.getFileSystem();
+      // Create two files
+      Path tmpFile1 = new Path("/tmpfile1.dat");
+      Path tmpFile2 = new Path("/tmpfile2.dat");
+      DFSTestUtil.createFile(fs, tmpFile1, 1024, (short) 2, 0xDEADDEADl);
+      DFSTestUtil.createFile(fs, tmpFile2, 1024, (short) 2, 0xDEADDEADl);
+      // Get locations of blocks of both files and concat together
+      BlockLocation[] blockLocs1 = fs.getFileBlockLocations(tmpFile1, 0, 1024);
+      BlockLocation[] blockLocs2 = fs.getFileBlockLocations(tmpFile2, 0, 1024);
+      BlockLocation[] blockLocs = (BlockLocation[]) ArrayUtils.addAll(blockLocs1,
+          blockLocs2);
+      // Fetch VolumeBlockLocations in batch
+      BlockStorageLocation[] locs = fs.getFileBlockStorageLocations(Arrays
+          .asList(blockLocs));
+      int counter = 0;
+      // Print out the list of ids received for each block
+      for (BlockStorageLocation l : locs) {
+        for (int i = 0; i < l.getVolumeIds().length; i++) {
+          VolumeId id = l.getVolumeIds()[i];
+          String name = l.getNames()[i];
+          if (id != null) {
+            System.out.println("Datanode " + name + " has block " + counter
+                + " on volume id " + id.toString());
+          }
         }
+        counter++;
       }
-      counter++;
-    }
-    assertEquals("Expected two HdfsBlockLocations for two 1-block files", 2,
-        locs.length);
-    for (BlockStorageLocation l : locs) {
-      assertEquals("Expected two replicas for each block", 2,
-          l.getVolumeIds().length);
-      for (int i = 0; i < l.getVolumeIds().length; i++) {
-        VolumeId id = l.getVolumeIds()[i];
-        String name = l.getNames()[i];
-        assertTrue("Expected block to be valid on datanode " + name,
-            id.isValid());
+      assertEquals("Expected two HdfsBlockLocations for two 1-block files", 2,
+          locs.length);
+      for (BlockStorageLocation l : locs) {
+        assertEquals("Expected two replicas for each block", 2,
+            l.getVolumeIds().length);
+        for (int i = 0; i < l.getVolumeIds().length; i++) {
+          VolumeId id = l.getVolumeIds()[i];
+          String name = l.getNames()[i];
+          assertTrue("Expected block to be valid on datanode " + name,
+              id.isValid());
+        }
       }
+    } finally {
+      cluster.shutdown();
     }
   }
 
@@ -683,27 +687,31 @@ public class TestDistributedFileSystem {
         true);
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .numDataNodes(2).build();
-    cluster.getDataNodes();
-    DistributedFileSystem fs = cluster.getFileSystem();
-    // Create a file
-    Path tmpFile = new Path("/tmpfile1.dat");
-    DFSTestUtil.createFile(fs, tmpFile, 1024, (short) 2, 0xDEADDEADl);
-    // Get locations of blocks of the file
-    BlockLocation[] blockLocs = fs.getFileBlockLocations(tmpFile, 0, 1024);
-    // Stop a datanode to simulate a failure
-    cluster.stopDataNode(0);
-    // Fetch VolumeBlockLocations
-    BlockStorageLocation[] locs = fs.getFileBlockStorageLocations(Arrays
-        .asList(blockLocs));
-
-    assertEquals("Expected one HdfsBlockLocation for one 1-block file", 1,
-        locs.length);
-
-    for (BlockStorageLocation l : locs) {
-      assertEquals("Expected two replicas for each block", 2,
-          l.getVolumeIds().length);
-      assertTrue("Expected one valid and one invalid replica",
-          (l.getVolumeIds()[0].isValid()) ^ (l.getVolumeIds()[1].isValid()));
+    try {
+      cluster.getDataNodes();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      // Create a file
+      Path tmpFile = new Path("/tmpfile1.dat");
+      DFSTestUtil.createFile(fs, tmpFile, 1024, (short) 2, 0xDEADDEADl);
+      // Get locations of blocks of the file
+      BlockLocation[] blockLocs = fs.getFileBlockLocations(tmpFile, 0, 1024);
+      // Stop a datanode to simulate a failure
+      cluster.stopDataNode(0);
+      // Fetch VolumeBlockLocations
+      BlockStorageLocation[] locs = fs.getFileBlockStorageLocations(Arrays
+          .asList(blockLocs));
+
+      assertEquals("Expected one HdfsBlockLocation for one 1-block file", 1,
+          locs.length);
+
+      for (BlockStorageLocation l : locs) {
+        assertEquals("Expected two replicas for each block", 2,
+            l.getVolumeIds().length);
+        assertTrue("Expected one valid and one invalid replica",
+            (l.getVolumeIds()[0].isValid()) ^ (l.getVolumeIds()[1].isValid()));
+      }
+    } finally {
+      cluster.shutdown();
     }
   }
 

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java?rev=1470771&r1=1470770&r2=1470771&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java Tue Apr 23 01:18:36 2013
@@ -782,49 +782,53 @@ public class TestQuota {
   public void testMaxSpaceQuotas() throws Exception {
     final Configuration conf = new HdfsConfiguration();
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
-    final FileSystem fs = cluster.getFileSystem();
-    assertTrue("Not a HDFS: "+fs.getUri(),
-                fs instanceof DistributedFileSystem);
-    final DistributedFileSystem dfs = (DistributedFileSystem)fs;
-    
-    // create test directory
-    final Path testFolder = new Path("/testFolder");
-    assertTrue(dfs.mkdirs(testFolder));
-    
-    // setting namespace quota to Long.MAX_VALUE - 1 should work
-    dfs.setQuota(testFolder, Long.MAX_VALUE - 1, 10);
-    ContentSummary c = dfs.getContentSummary(testFolder);
-    assertTrue("Quota not set properly", c.getQuota() == Long.MAX_VALUE - 1);
-    
-    // setting diskspace quota to Long.MAX_VALUE - 1 should work
-    dfs.setQuota(testFolder, 10, Long.MAX_VALUE - 1);
-    c = dfs.getContentSummary(testFolder);
-    assertTrue("Quota not set properly", c.getSpaceQuota() == Long.MAX_VALUE - 1);
-    
-    // setting namespace quota to Long.MAX_VALUE should not work + no error
-    dfs.setQuota(testFolder, Long.MAX_VALUE, 10);
-    c = dfs.getContentSummary(testFolder);
-    assertTrue("Quota should not have changed", c.getQuota() == 10);
-    
-    // setting diskspace quota to Long.MAX_VALUE should not work + no error
-    dfs.setQuota(testFolder, 10, Long.MAX_VALUE);
-    c = dfs.getContentSummary(testFolder);
-    assertTrue("Quota should not have changed", c.getSpaceQuota() == 10);
-    
-    // setting namespace quota to Long.MAX_VALUE + 1 should not work + error
     try {
-      dfs.setQuota(testFolder, Long.MAX_VALUE + 1, 10);
-      fail("Exception not thrown");
-    } catch (IllegalArgumentException e) {
-      // Expected
-    }
+      final FileSystem fs = cluster.getFileSystem();
+      assertTrue("Not a HDFS: "+fs.getUri(),
+                  fs instanceof DistributedFileSystem);
+      final DistributedFileSystem dfs = (DistributedFileSystem)fs;
+    
+      // create test directory
+      final Path testFolder = new Path("/testFolder");
+      assertTrue(dfs.mkdirs(testFolder));
+    
+      // setting namespace quota to Long.MAX_VALUE - 1 should work
+      dfs.setQuota(testFolder, Long.MAX_VALUE - 1, 10);
+      ContentSummary c = dfs.getContentSummary(testFolder);
+      assertTrue("Quota not set properly", c.getQuota() == Long.MAX_VALUE - 1);
+    
+      // setting diskspace quota to Long.MAX_VALUE - 1 should work
+      dfs.setQuota(testFolder, 10, Long.MAX_VALUE - 1);
+      c = dfs.getContentSummary(testFolder);
+      assertTrue("Quota not set properly", c.getSpaceQuota() == Long.MAX_VALUE - 1);
+    
+      // setting namespace quota to Long.MAX_VALUE should not work + no error
+      dfs.setQuota(testFolder, Long.MAX_VALUE, 10);
+      c = dfs.getContentSummary(testFolder);
+      assertTrue("Quota should not have changed", c.getQuota() == 10);
+    
+      // setting diskspace quota to Long.MAX_VALUE should not work + no error
+      dfs.setQuota(testFolder, 10, Long.MAX_VALUE);
+      c = dfs.getContentSummary(testFolder);
+      assertTrue("Quota should not have changed", c.getSpaceQuota() == 10);
     
-    // setting diskspace quota to Long.MAX_VALUE + 1 should not work + error
-    try {
-      dfs.setQuota(testFolder, 10, Long.MAX_VALUE + 1);
-      fail("Exception not thrown");
-    } catch (IllegalArgumentException e) {
-      // Expected
+      // setting namespace quota to Long.MAX_VALUE + 1 should not work + error
+      try {
+        dfs.setQuota(testFolder, Long.MAX_VALUE + 1, 10);
+        fail("Exception not thrown");
+      } catch (IllegalArgumentException e) {
+        // Expected
+      }
+    
+      // setting diskspace quota to Long.MAX_VALUE + 1 should not work + error
+      try {
+        dfs.setQuota(testFolder, 10, Long.MAX_VALUE + 1);
+        fail("Exception not thrown");
+      } catch (IllegalArgumentException e) {
+        // Expected
+      }
+    } finally {
+      cluster.shutdown();
     }
   }
   

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java?rev=1470771&r1=1470770&r2=1470771&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLogJournalFailures.java Tue Apr 23 01:18:36 2013
@@ -255,7 +255,6 @@ public class TestEditLogJournalFailures 
       doThrow(new IOException("fail on setReadyToFlush()")).when(spyElos)
         .setReadyToFlush();
     }
-    doNothing().when(spyElos).abort();
   }
 
   private EditLogFileOutputStream spyOnStream(JournalAndStream jas) {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java?rev=1470771&r1=1470770&r2=1470771&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java Tue Apr 23 01:18:36 2013
@@ -530,7 +530,11 @@ public class TestStartup {
         .manageDataDfsDirs(false)
         .manageNameDfsDirs(false)
         .build();
-    cluster.waitActive();
+    try {
+      cluster.waitActive();
+    } finally {
+      cluster.shutdown();
+    }
 }
 
   /**