hadoop-hdfs-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From e..@apache.org
Subject svn commit: r1098888 - in /hadoop/hdfs/trunk: ./ src/java/org/apache/hadoop/hdfs/server/datanode/ src/java/org/apache/hadoop/hdfs/server/datanode/metrics/ src/java/org/apache/hadoop/hdfs/server/namenode/ src/java/org/apache/hadoop/hdfs/server/protocol/...
Date Tue, 03 May 2011 01:34:05 GMT
Author: eli
Date: Tue May  3 01:34:04 2011
New Revision: 1098888

URL: http://svn.apache.org/viewvc?rev=1098888&view=rev
Log:
HDFS-1850. DN should transmit absolute failed volume count rather than increments to the NN. Contributed by Eli Collins

Added:
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
Modified:
    hadoop/hdfs/trunk/CHANGES.txt
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/metrics/FSDatasetMBean.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java
    hadoop/hdfs/trunk/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java

Modified: hadoop/hdfs/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/CHANGES.txt?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/CHANGES.txt (original)
+++ hadoop/hdfs/trunk/CHANGES.txt Tue May  3 01:34:04 2011
@@ -873,6 +873,9 @@ Release 0.22.0 - Unreleased
 
     HDFS-1625. Ignore disk space values in TestDataNodeMXBean.  (szetszwo)
 
+    HDFS-1850. DN should transmit absolute failed volume count rather than 
+    increments to the NN. (eli)
+
 Release 0.21.1 - Unreleased
     HDFS-1466. TestFcHdfsSymlink relies on /tmp/test not existing. (eli)
 

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java Tue May  3 01:34:04 2011
@@ -958,7 +958,7 @@ public class DataNode extends Configured
           data.getRemaining(),
           data.getBlockPoolUsed(blockPoolId),
           xmitsInProgress.get(),
-          getXceiverCount());
+          getXceiverCount(), data.getNumFailedVolumes());
     }
     
     //This must be called only by blockPoolManager
@@ -1702,7 +1702,7 @@ public class DataNode extends Configured
   protected void checkDiskError( ) {
     try {
       data.checkDataDir();
-    } catch(DiskErrorException de) {
+    } catch (DiskErrorException de) {
       handleDiskError(de.getMessage());
     }
   }
@@ -1715,7 +1715,7 @@ public class DataNode extends Configured
     // shutdown the DN completely.
     int dpError = hasEnoughResources ? DatanodeProtocol.DISK_ERROR  
                                      : DatanodeProtocol.FATAL_DISK_ERROR;  
-    myMetrics.volumesFailed.inc(1);
+    myMetrics.volumeFailures.inc(1);
 
     //inform NameNodes
     for(BPOfferService bpos: blockPoolManager.getAllNamenodeThreads()) {
@@ -1741,7 +1741,7 @@ public class DataNode extends Configured
   int getXceiverCount() {
     return threadGroup == null ? 0 : threadGroup.activeCount();
   }
-
+    
   static UpgradeManagerDatanode getUpgradeManagerDatanode(String bpid) {
     DataNode dn = getDataNode();
     BPOfferService bpos = dn.blockPoolManager.get(bpid);

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java Tue May  3 01:34:04 2011
@@ -783,7 +783,8 @@ public class FSDataset implements FSCons
      */
     private volatile List<FSVolume> volumes = null;
     BlockVolumeChoosingPolicy blockChooser;
-      
+    int numFailedVolumes = 0;
+
     FSVolumeSet(FSVolume[] volumes, BlockVolumeChoosingPolicy blockChooser) {
       List<FSVolume> list = Arrays.asList(volumes);
       this.volumes = Collections.unmodifiableList(list);
@@ -793,7 +794,11 @@ public class FSDataset implements FSCons
     private int numberOfVolumes() {
       return volumes.size();
     }
-      
+
+    private int numberOfFailedVolumes() {
+      return numFailedVolumes;
+    }
+    
     /** 
      * Get next volume. Synchronized to ensure {@link #curVolume} is updated
      * by a single thread and next volume is chosen with no concurrent
@@ -876,6 +881,7 @@ public class FSDataset implements FSCons
           }
           removedVols.add(volumeList.get(idx));
           volumeList.set(idx, null); // Remove the volume
+          numFailedVolumes++;
         }
       }
       
@@ -1220,6 +1226,13 @@ public class FSDataset implements FSCons
   }
 
   /**
+   * Return the number of failed volumes in the FSDataset.
+   */
+  public int getNumFailedVolumes() {
+    return volumes.numberOfFailedVolumes();
+  }
+
+  /**
    * Find the block's on-disk length
    */
   @Override // FSDatasetInterface

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java Tue May  3 01:34:04 2011
@@ -77,8 +77,8 @@ public class DataNodeMetrics implements 
   public MetricsTimeVaryingInt writesFromRemoteClient = 
               new MetricsTimeVaryingInt("writes_from_remote_client", registry);
 
-  public MetricsTimeVaryingInt volumesFailed =
-    new MetricsTimeVaryingInt("volumes_failed", registry);
+  public MetricsTimeVaryingInt volumeFailures =
+    new MetricsTimeVaryingInt("volumeFailures", registry);
   
   public MetricsTimeVaryingRate readBlockOp = 
                 new MetricsTimeVaryingRate("readBlockOp", registry);

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/metrics/FSDatasetMBean.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/metrics/FSDatasetMBean.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/metrics/FSDatasetMBean.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/datanode/metrics/FSDatasetMBean.java Tue May  3 01:34:04 2011
@@ -72,4 +72,9 @@ public interface FSDatasetMBean {
    */
   public String getStorageInfo();
 
+  /**
+   * Returns the number of failed volumes in the datanode.
+   * @return The number of failed volumes in the datanode.
+   */
+  public int getNumFailedVolumes();
 }

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/DatanodeDescriptor.java Tue May  3 01:34:04 2011
@@ -130,7 +130,7 @@ public class DatanodeDescriptor extends 
    * @param nodeID id of the data node
    */
   public DatanodeDescriptor(DatanodeID nodeID) {
-    this(nodeID, 0L, 0L, 0L, 0L, 0);
+    this(nodeID, 0L, 0L, 0L, 0L, 0, 0);
   }
 
   /** DatanodeDescriptor constructor
@@ -152,7 +152,7 @@ public class DatanodeDescriptor extends 
   public DatanodeDescriptor(DatanodeID nodeID, 
                             String networkLocation,
                             String hostName) {
-    this(nodeID, networkLocation, hostName, 0L, 0L, 0L, 0L, 0);
+    this(nodeID, networkLocation, hostName, 0L, 0L, 0L, 0L, 0, 0);
   }
   
   /** DatanodeDescriptor constructor
@@ -169,9 +169,11 @@ public class DatanodeDescriptor extends 
                             long dfsUsed,
                             long remaining,
                             long bpused,
-                            int xceiverCount) {
+                            int xceiverCount,
+                            int failedVolumes) {
     super(nodeID);
-    updateHeartbeat(capacity, dfsUsed, remaining, bpused, xceiverCount);
+    updateHeartbeat(capacity, dfsUsed, remaining, bpused, xceiverCount, 
+        failedVolumes);
   }
 
   /** DatanodeDescriptor constructor
@@ -191,9 +193,11 @@ public class DatanodeDescriptor extends 
                             long dfsUsed,
                             long remaining,
                             long bpused,
-                            int xceiverCount) {
+                            int xceiverCount,
+                            int failedVolumes) {
     super(nodeID, networkLocation, hostName);
-    updateHeartbeat(capacity, dfsUsed, remaining, bpused, xceiverCount);
+    updateHeartbeat(capacity, dfsUsed, remaining, bpused, xceiverCount, 
+        failedVolumes);
   }
 
   /**
@@ -254,6 +258,7 @@ public class DatanodeDescriptor extends 
     this.xceiverCount = 0;
     this.blockList = null;
     this.invalidateBlocks.clear();
+    this.volumeFailures = 0;
   }
 
   public int numBlocks() {
@@ -261,15 +266,17 @@ public class DatanodeDescriptor extends 
   }
 
   /**
+   * Updates stats from datanode heartbeat.
    */
   void updateHeartbeat(long capacity, long dfsUsed, long remaining,
-      long blockPoolUsed, int xceiverCount) {
+      long blockPoolUsed, int xceiverCount, int volFailures) {
     this.capacity = capacity;
     this.dfsUsed = dfsUsed;
     this.remaining = remaining;
     this.blockPoolUsed = blockPoolUsed;
     this.lastUpdate = System.currentTimeMillis();
     this.xceiverCount = xceiverCount;
+    this.volumeFailures = volFailures;
     rollBlocksScheduled(lastUpdate);
   }
 
@@ -528,13 +535,6 @@ public class DatanodeDescriptor extends 
   }  // End of class DecommissioningStatus
 
   /**
-   * Increment the volume failure count.
-   */
-  public void incVolumeFailure() {
-    volumeFailures++;
-  }
-  
-  /**
    * Set the flag to indicate if this datanode is disallowed from communicating
    * with the namenode.
    */
@@ -554,11 +554,9 @@ public class DatanodeDescriptor extends 
   }
 
   /**
-   * Reset the volume failure count when a DN re-registers.
    * @param nodeReg DatanodeID to update registration for.
    */
   public void updateRegInfo(DatanodeID nodeReg) {
     super.updateRegInfo(nodeReg);
-    volumeFailures = 0;
   }
 }

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Tue May  3 01:34:04 2011
@@ -2674,7 +2674,7 @@ public class FSNamesystem implements FSC
         if( !heartbeats.contains(nodeS)) {
           heartbeats.add(nodeS);
           //update its timestamp
-          nodeS.updateHeartbeat(0L, 0L, 0L, 0L, 0);
+          nodeS.updateHeartbeat(0L, 0L, 0L, 0L, 0, 0);
           nodeS.isAlive = true;
         }
       }
@@ -2795,7 +2795,8 @@ public class FSNamesystem implements FSC
    */
   DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
       long capacity, long dfsUsed, long remaining, long blockPoolUsed,
-      int xceiverCount, int xmitsInProgress) throws IOException {
+      int xceiverCount, int xmitsInProgress, int failedVolumes) 
+      throws IOException {
     DatanodeCommand cmd = null;
     synchronized (heartbeats) {
       synchronized (datanodeMap) {
@@ -2818,7 +2819,7 @@ public class FSNamesystem implements FSC
 
         updateStats(nodeinfo, false);
         nodeinfo.updateHeartbeat(capacity, dfsUsed, remaining, blockPoolUsed,
-            xceiverCount);
+            xceiverCount, failedVolumes);
         updateStats(nodeinfo, true);
         
         //check lease recovery
@@ -3065,22 +3066,6 @@ public class FSNamesystem implements FSC
   }
 
   /**
-   * Update the descriptor for the datanode to reflect a volume failure.
-   * @param nodeID DatanodeID to update count for.
-   * @throws IOException
-   */
-  synchronized public void incVolumeFailure(DatanodeID nodeID)
-    throws IOException {
-    DatanodeDescriptor nodeInfo = getDatanode(nodeID);
-    if (nodeInfo != null) {
-      nodeInfo.incVolumeFailure();
-    } else {
-      NameNode.stateChangeLog.warn("BLOCK* NameSystem.incVolumeFailure: "
-                                   + nodeID.getName() + " does not exist");
-    }
-  }
-
-  /**
    * Remove a datanode descriptor.
    * @param nodeID datanode ID.
    * @throws IOException

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java Tue May  3 01:34:04 2011
@@ -1278,10 +1278,11 @@ public class NameNode implements Namenod
                                        long remaining,
                                        long blockPoolUsed,
                                        int xmitsInProgress,
-                                       int xceiverCount) throws IOException {
+                                       int xceiverCount,
+                                       int failedVolumes) throws IOException {
     verifyRequest(nodeReg);
     return namesystem.handleHeartbeat(nodeReg, capacity, dfsUsed, remaining,
-        blockPoolUsed, xceiverCount, xmitsInProgress);
+        blockPoolUsed, xceiverCount, xmitsInProgress, failedVolumes);
   }
 
   /**
@@ -1321,22 +1322,25 @@ public class NameNode implements Namenod
   }
 
   /**
+   * Handle an error report from a datanode.
    */
   public void errorReport(DatanodeRegistration nodeReg,
-                          int errorCode, 
-                          String msg) throws IOException {
-    // Log error message from datanode
+                          int errorCode, String msg) throws IOException { 
     String dnName = (nodeReg == null ? "unknown DataNode" : nodeReg.getName());
-    LOG.info("Error report from " + dnName + ": " + msg);
+
     if (errorCode == DatanodeProtocol.NOTIFY) {
+      LOG.info("Error report from " + dnName + ": " + msg);
       return;
     }
     verifyRequest(nodeReg);
-    namesystem.incVolumeFailure(nodeReg);
+
     if (errorCode == DatanodeProtocol.DISK_ERROR) {
-      LOG.warn("Volume failed on " + dnName); 
+      LOG.warn("Disk error on " + dnName + ": " + msg);
     } else if (errorCode == DatanodeProtocol.FATAL_DISK_ERROR) {
+      LOG.warn("Fatal disk error on " + dnName + ": " + msg);
       namesystem.removeDatanode(nodeReg);            
+    } else {
+      LOG.info("Error report from " + dnName + ": " + msg);
     }
   }
     

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java Tue May  3 01:34:04 2011
@@ -93,6 +93,7 @@ public interface DatanodeProtocol extend
    * @param blockPoolUsed storage used by the block pool
    * @param xmitsInProgress number of transfers from this datanode to others
    * @param xceiverCount number of active transceiver threads
+   * @param failedVolumes number of failed volumes
    * @throws IOException on error
    */
   @Nullable
@@ -101,7 +102,8 @@ public interface DatanodeProtocol extend
                                        long dfsUsed, long remaining,
                                        long blockPoolUsed,
                                        int xmitsInProgress,
-                                       int xceiverCount) throws IOException;
+                                       int xceiverCount,
+                                       int failedVolumes) throws IOException;
 
   /**
    * blockReport() tells the NameNode about all the locally-stored blocks.

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java Tue May  3 01:34:04 2011
@@ -58,6 +58,7 @@ import org.apache.hadoop.hdfs.protocol.L
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw;
+import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.io.IOUtils;
@@ -291,7 +292,7 @@ public class DFSTestUtil {
       Path file, ExtendedBlock b, int corruptRepls)
       throws IOException, TimeoutException {
     int count = 0;
-    final int ATTEMPTS = 20;
+    final int ATTEMPTS = 50;
     int repls = ns.numCorruptReplicas(b.getLocalBlock());
     while (repls != corruptRepls && count < ATTEMPTS) {
       try {
@@ -354,6 +355,88 @@ public class DFSTestUtil {
     return -1;
   }
 
+  /*
+   * Return the total capacity of all live DNs.
+   */
+  public static long getLiveDatanodeCapacity(FSNamesystem ns) {
+    ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
+    ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
+    ns.DFSNodesStatus(live, dead);
+    long capacity = 0;
+    for (final DatanodeDescriptor dn : live) {
+      capacity += dn.getCapacity();
+    }
+    return capacity;
+  }
+
+  /*
+   * Return the capacity of the given live DN.
+   */
+  public static long getDatanodeCapacity(FSNamesystem ns, int index) {
+    ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
+    ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
+    ns.DFSNodesStatus(live, dead);
+    return live.get(index).getCapacity();
+  }
+
+  /*
+   * Wait for the given # live/dead DNs, total capacity, and # vol failures. 
+   */
+  public static void waitForDatanodeStatus(FSNamesystem ns, int expectedLive, 
+      int expectedDead, long expectedVolFails, long expectedTotalCapacity, 
+      long timeout) throws InterruptedException, TimeoutException {
+    ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
+    ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
+    final int ATTEMPTS = 10;
+    int count = 0;
+    long currTotalCapacity = 0;
+    int volFails = 0;
+
+    do {
+      Thread.sleep(timeout);
+      live.clear();
+      dead.clear();
+      ns.DFSNodesStatus(live, dead);
+      currTotalCapacity = 0;
+      volFails = 0;
+      for (final DatanodeDescriptor dd : live) {
+        currTotalCapacity += dd.getCapacity();
+        volFails += dd.getVolumeFailures();
+      }
+      count++;
+    } while ((expectedLive != live.size() ||
+              expectedDead != dead.size() ||
+              expectedTotalCapacity != currTotalCapacity ||
+              expectedVolFails != volFails)
+             && count < ATTEMPTS);
+
+    if (count == ATTEMPTS) {
+      throw new TimeoutException("Timed out waiting for capacity."
+          + " Live = "+live.size()+" Expected = "+expectedLive
+          + " Dead = "+dead.size()+" Expected = "+expectedDead
+          + " Total capacity = "+currTotalCapacity
+          + " Expected = "+expectedTotalCapacity
+          + " Vol Fails = "+volFails+" Expected = "+expectedVolFails);
+    }
+  }
+
+  /*
+   * Wait for the given DN to consider itself dead.
+   */
+  public static void waitForDatanodeDeath(DataNode dn) 
+      throws InterruptedException, TimeoutException {
+    final int ATTEMPTS = 10;
+    int count = 0;
+    do {
+      Thread.sleep(1000);
+      count++;
+    } while (dn.isDatanodeUp() && count < ATTEMPTS);
+
+    if (count == ATTEMPTS) {
+      throw new TimeoutException("Timed out waiting for DN to die");
+    }
+  }
+  
   /** return list of filenames created as part of createFiles */
   public String[] getFileNames(String topDir) {
     if (nFiles == 0)

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java Tue May  3 01:34:04 2011
@@ -318,6 +318,10 @@ public class SimulatedFSDataset  impleme
       return getBPStorage(bpid).getUsed();
     }
     
+    int getNumFailedVolumes() {
+      return 0;
+    }
+
     synchronized boolean alloc(String bpid, long amount) throws IOException {
       if (getFree() >= amount) {
         getBPStorage(bpid).alloc(amount);
@@ -477,6 +481,11 @@ public class SimulatedFSDataset  impleme
     return storage.getFree();
   }
 
+  @Override // FSDatasetMBean
+  public int getNumFailedVolumes() {
+    return storage.getNumFailedVolumes();
+  }
+
   @Override // FSDatasetInterface
   public synchronized long getLength(ExtendedBlock b) throws IOException {
     final Map<Block, BInfo> map = getMap(b.getBlockPoolId());

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java Tue May  3 01:34:04 2011
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.da
 
 import java.io.File;
 import java.util.ArrayList;
+import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -40,9 +41,10 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import static org.junit.Assert.*;
+import static org.junit.Assume.assumeTrue;
 
 /**
- * Test successive volume failures, failure metrics and capacity reporting.
+ * Test reporting of DN volume failure counts and metrics.
  */
 public class TestDataNodeVolumeFailureReporting {
 
@@ -56,6 +58,14 @@ public class TestDataNodeVolumeFailureRe
   private Configuration conf;
   private String dataDir;
 
+  // Sleep at least 3 seconds (a 1s heartbeat plus padding) to allow
+  // for heartbeats to propagate from the datanodes to the namenode.
+  final int WAIT_FOR_HEARTBEATS = 3000;
+
+  // Wait at least (2 * re-check + 10 * heartbeat) seconds for
+  // a datanode to be considered dead by the namenode.  
+  final int WAIT_FOR_DEATH = 15000;
+
   @Before
   public void setUp() throws Exception {
     conf = new HdfsConfiguration();
@@ -77,6 +87,10 @@ public class TestDataNodeVolumeFailureRe
 
   @After
   public void tearDown() throws Exception {
+    for (int i = 0; i < 3; i++) {
+      new File(dataDir, "data"+(2*i+1)).setExecutable(true);
+      new File(dataDir, "data"+(2*i+2)).setExecutable(true);
+    }
     cluster.shutdown();
   }
 
@@ -87,41 +101,22 @@ public class TestDataNodeVolumeFailureRe
    */
   @Test
   public void testSuccessiveVolumeFailures() throws Exception {
-    if (System.getProperty("os.name").startsWith("Windows")) {
-      // See above
-      return;
-    }
+    assumeTrue(!System.getProperty("os.name").startsWith("Windows"));
+
     // Bring up two more datanodes
     cluster.startDataNodes(conf, 2, true, null, null);
     cluster.waitActive();
 
     /*
-     * Sleep at least 3 seconds (a 1s heartbeat plus padding) to allow
-     * for heartbeats to propagate from the datanodes to the namenode.
-     * Sleep  at least (2 * re-check + 10 * heartbeat) 12 seconds for
-     * a datanode  to be called dead by the namenode.
-     */
-    final int WAIT_FOR_HEARTBEATS = 3000;
-    final int WAIT_FOR_DEATH = 15000;
-
-    /*
      * Calculate the total capacity of all the datanodes. Sleep for
      * three seconds to be sure the datanodes have had a chance to
      * heartbeat their capacities.
      */
     Thread.sleep(WAIT_FOR_HEARTBEATS);
-    FSNamesystem namesystem = cluster.getNamesystem();
-    ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
-    ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
-    namesystem.DFSNodesStatus(live, dead);
-    assertEquals("All DNs should be live", 3, live.size());
-    assertEquals("All DNs should be live", 0, dead.size());
-    long origCapacity = 0;
-    for (final DatanodeDescriptor dn : live) {
-      origCapacity += dn.getCapacity();
-      assertEquals("DN "+dn+" vols should be healthy",
-          0, dn.getVolumeFailures());
-    }
+    FSNamesystem ns = cluster.getNamesystem();
+
+    long origCapacity = DFSTestUtil.getLiveDatanodeCapacity(ns);
+    long dnCapacity = DFSTestUtil.getDatanodeCapacity(ns, 0);
 
     File dn1Vol1 = new File(dataDir, "data"+(2*0+1));
     File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
@@ -158,27 +153,18 @@ public class TestDataNodeVolumeFailureRe
     DataNodeMetrics metrics2 = dns.get(1).getMetrics();
     DataNodeMetrics metrics3 = dns.get(2).getMetrics();
     assertEquals("Vol1 should report 1 failure",
-        1, metrics1.volumesFailed.getCurrentIntervalValue());
+        1, metrics1.volumeFailures.getCurrentIntervalValue());
     assertEquals("Vol2 should report 1 failure",
-        1, metrics2.volumesFailed.getCurrentIntervalValue());
+        1, metrics2.volumeFailures.getCurrentIntervalValue());
     assertEquals("Vol3 should have no failures",
-        0, metrics3.volumesFailed.getCurrentIntervalValue());
+        0, metrics3.volumeFailures.getCurrentIntervalValue());
 
-    // Eventually the NN should report two volume failures as well
-    while (true) {
-      Thread.sleep(WAIT_FOR_HEARTBEATS);
-      live.clear();
-      dead.clear();
-      namesystem.DFSNodesStatus(live, dead);
-      int volumeFailures = 0;
-      for (final DatanodeDescriptor dn : live) {
-        volumeFailures += dn.getVolumeFailures();
-      }
-      if (2 == volumeFailures) {
-        break;
-      }
-      LOG.warn("Still waiting for volume failures: "+volumeFailures);
-    }
+    // Ensure we wait a sufficient amount of time
+    assert (WAIT_FOR_HEARTBEATS * 10) > WAIT_FOR_DEATH;
+
+    // Eventually the NN should report two volume failures
+    DFSTestUtil.waitForDatanodeStatus(ns, 3, 0, 2, 
+        origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS);
 
     /*
      * Now fail a volume on the third datanode. We should be able to get
@@ -190,10 +176,14 @@ public class TestDataNodeVolumeFailureRe
     DFSTestUtil.waitReplication(fs, file2, (short)3);
     assertTrue("DN3 should still be up", dns.get(2).isDatanodeUp());
     assertEquals("Vol3 should report 1 failure",
-        1, metrics3.volumesFailed.getCurrentIntervalValue());
+        1, metrics3.volumeFailures.getCurrentIntervalValue());
+
+    ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
+    ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
+    ns.DFSNodesStatus(live, dead);
     live.clear();
     dead.clear();
-    namesystem.DFSNodesStatus(live, dead);
+    ns.DFSNodesStatus(live, dead);
     assertEquals("DN3 should have 1 failed volume",
         1, live.get(2).getVolumeFailures());
 
@@ -202,25 +192,9 @@ public class TestDataNodeVolumeFailureRe
      * total capacity should be down by three volumes (assuming the host
      * did not grow or shrink the data volume while the test was running).
      */
-    while (true) {
-      Thread.sleep(WAIT_FOR_HEARTBEATS);
-      live.clear();
-      dead.clear();
-      namesystem.DFSNodesStatus(live, dead);
-      long currCapacity = 0;
-      long singleVolCapacity = live.get(0).getCapacity();
-      for (final DatanodeDescriptor dn : live) {
-        currCapacity += dn.getCapacity();
-      }
-      LOG.info("Live: "+live.size()+" Dead: "+dead.size());
-      LOG.info("Original capacity: "+origCapacity);
-      LOG.info("Current capacity: "+currCapacity);
-      LOG.info("Volume capacity: "+singleVolCapacity);
-      if (3 == live.size() && 0 == dead.size() &&
-          origCapacity == (currCapacity + (3 * singleVolCapacity))) {
-        break;
-      }
-    }
+    dnCapacity = DFSTestUtil.getDatanodeCapacity(ns, 0);
+    DFSTestUtil.waitForDatanodeStatus(ns, 3, 0, 3, 
+        origCapacity - (3*dnCapacity), WAIT_FOR_HEARTBEATS);
 
     /*
      * Now fail the 2nd volume on the 3rd datanode. All its volumes
@@ -232,25 +206,18 @@ public class TestDataNodeVolumeFailureRe
     Path file3 = new Path("/test3");
     DFSTestUtil.createFile(fs, file3, 1024, (short)3, 1L);
     DFSTestUtil.waitReplication(fs, file3, (short)2);
-    // Eventually the DN should go down
-    while (dns.get(2).isDatanodeUp()) {
-      Thread.sleep(1000);
-    }
-    // and report two failed volumes
+
+    // The DN should consider itself dead
+    DFSTestUtil.waitForDatanodeDeath(dns.get(2));
+
+    // And report two failed volumes
     metrics3 = dns.get(2).getMetrics();
     assertEquals("DN3 should report 2 vol failures",
-        2, metrics3.volumesFailed.getCurrentIntervalValue());
-    // and eventually be seen as dead by the NN.
-    while (true) {
-      Thread.sleep(WAIT_FOR_DEATH);
-      live.clear();
-      dead.clear();
-      namesystem.DFSNodesStatus(live, dead);
-      if (1 == dead.size() && 2 == live.size()) {
-        break;
-      }
-      LOG.warn("Still waiting for dn to die: "+dead.size());
-    }
+        2, metrics3.volumeFailures.getCurrentIntervalValue());
+
+    // The NN considers the DN dead
+    DFSTestUtil.waitForDatanodeStatus(ns, 2, 1, 2, 
+        origCapacity - (4*dnCapacity), WAIT_FOR_HEARTBEATS);
 
     /*
      * The datanode never tries to restore the failed volume, even if
@@ -273,105 +240,44 @@ public class TestDataNodeVolumeFailureRe
      * and that the volume failure count should be reported as zero by
      * both the metrics and the NN.
      */
-    while (true) {
-      Thread.sleep(WAIT_FOR_DEATH);
-      live.clear();
-      dead.clear();
-      namesystem.DFSNodesStatus(live, dead);
-      assertEquals("All DNs should be live", 3, live.size());
-      assertEquals("All DNs should be live", 0, dead.size());
-      long currCapacity = 0;
-      long volFailures = 0;
-      for (final DatanodeDescriptor dn : live) {
-        currCapacity += dn.getCapacity();
-        volFailures += dn.getVolumeFailures();
-      }
-      if (3 == live.size() && 0 == dead.size() && 0 == volFailures &&
-          origCapacity == currCapacity) {
-        break;
-      }
-      LOG.warn("Waiting for capacity: original="+origCapacity+" current="+
-          currCapacity+" live="+live.size()+" dead="+dead.size()+
-          " vols="+volFailures);
-    }
+    DFSTestUtil.waitForDatanodeStatus(ns, 3, 0, 0, origCapacity, 
+        WAIT_FOR_HEARTBEATS);
   }
 
   /**
-   * Test the DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY configuration
-   * option, ie the DN shuts itself down when the number of failures
-   * experienced drops below the tolerated amount.
+   * Test that the NN re-learns of volume failures after restart.
    */
   @Test
-  public void testConfigureMinValidVolumes() throws Exception {
-    if (System.getProperty("os.name").startsWith("Windows")) {
-      // See above
-      return;
-    }
+  public void testVolFailureStatsPreservedOnNNRestart() throws Exception {
+    assumeTrue(!System.getProperty("os.name").startsWith("Windows"));
 
-    // Bring up two additional datanodes that need both of their volumes
-    // functioning in order to stay up.
-    conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 0);
+    // Bring up two more datanodes that can tolerate 1 failure
     cluster.startDataNodes(conf, 2, true, null, null);
     cluster.waitActive();
 
-    // Fail a volume on the 2nd DN
+    FSNamesystem ns = cluster.getNamesystem();
+    long origCapacity = DFSTestUtil.getLiveDatanodeCapacity(ns);
+    long dnCapacity = DFSTestUtil.getDatanodeCapacity(ns, 0);
+
+    // Fail the first volume on both datanodes (we have to keep the 
+    // third healthy so one node in the pipeline will not fail). 
+    File dn1Vol1 = new File(dataDir, "data"+(2*0+1));
     File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
+    assertTrue("Couldn't chmod local vol", dn1Vol1.setExecutable(false));
     assertTrue("Couldn't chmod local vol", dn2Vol1.setExecutable(false));
 
-    // Should only get two replicas (the first DN and the 3rd)
     Path file1 = new Path("/test1");
-    DFSTestUtil.createFile(fs, file1, 1024, (short)3, 1L);
+    DFSTestUtil.createFile(fs, file1, 1024, (short)2, 1L);
     DFSTestUtil.waitReplication(fs, file1, (short)2);
 
-    // Check that this single failure caused a DN to die.
-    while (true) {
-      final int WAIT_FOR_DEATH = 15000;
-      Thread.sleep(WAIT_FOR_DEATH);
-      FSNamesystem namesystem = cluster.getNamesystem();
-      ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
-      ArrayList<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
-      namesystem.DFSNodesStatus(live, dead);
-      if (1 == dead.size()) {
-        break;
-      }
-      LOG.warn("Waiting for datanode to die: "+dead.size());
-    }
-
-    // If we restore the volume we should still only be able to get
-    // two replicas since the DN is still considered dead.
-    assertTrue("Couldn't chmod local vol", dn2Vol1.setExecutable(true));
-    Path file2 = new Path("/test2");
-    DFSTestUtil.createFile(fs, file2, 1024, (short)3, 1L);
-    DFSTestUtil.waitReplication(fs, file2, (short)2);
-  }
+    // The NN reports two volumes failures
+    DFSTestUtil.waitForDatanodeStatus(ns, 3, 0, 2, 
+        origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS);
 
-  /**
-   * Test invalid DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY values.
-   */
-  @Test
-  public void testInvalidFailedVolumesConfig() throws Exception {
-    if (System.getProperty("os.name").startsWith("Windows")) {
-      // See above
-      return;
-    }
-    /*
-     * Bring up another datanode that has an invalid value set.
-     * We should still be able to create a file with two replicas
-     * since the minimum valid volume parameter is only checked
-     * when we experience a disk error.
-     */
-    conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, -1);
-    cluster.startDataNodes(conf, 1, true, null, null);
-    cluster.waitActive();
-    Path file1 = new Path("/test1");
-    DFSTestUtil.createFile(fs, file1, 1024, (short)2, 1L);
-    DFSTestUtil.waitReplication(fs, file1, (short)2);
-    // Ditto if the value is too big.
-    conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 100);
-    cluster.startDataNodes(conf, 1, true, null, null);
+    // After restarting the NN it still see the two failures
+    cluster.restartNameNode(0);
     cluster.waitActive();
-    Path file2 = new Path("/test1");
-    DFSTestUtil.createFile(fs, file2, 1024, (short)2, 1L);
-    DFSTestUtil.waitReplication(fs, file2, (short)2);
+    DFSTestUtil.waitForDatanodeStatus(ns, 3, 0, 2,
+        origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS);
   }
 }

Added: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java?rev=1098888&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java (added)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java Tue May  3 01:34:04 2011
@@ -0,0 +1,160 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.File;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.log4j.Level;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.junit.Assume.assumeTrue;
+
+/**
+ * Test the ability of a DN to tolerate volume failures.
+ */
+public class TestDataNodeVolumeFailureToleration {
+
+  private static final Log LOG = LogFactory.getLog(TestDataNodeVolumeFailureToleration.class);
+  {
+    ((Log4JLogger)TestDataNodeVolumeFailureToleration.LOG).getLogger().setLevel(Level.ALL);
+  }
+
+  private FileSystem fs;
+  private MiniDFSCluster cluster;
+  private Configuration conf;
+  private String dataDir;
+
+  // Sleep at least 3 seconds (a 1s heartbeat plus padding) to allow
+  // for heartbeats to propagate from the datanodes to the namenode.
+  final int WAIT_FOR_HEARTBEATS = 3000;
+
+  // Wait at least (2 * re-check + 10 * heartbeat) seconds for
+  // a datanode to be considered dead by the namenode.  
+  final int WAIT_FOR_DEATH = 15000;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 512L);
+    /*
+     * Lower the DN heartbeat, DF rate, and recheck interval to one second
+     * so state about failures and datanode death propagates faster.
+     */
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
+    conf.setInt(DFSConfigKeys.DFS_DF_INTERVAL_KEY, 1000);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 1000);
+    // Allow a single volume failure (there are two volumes)
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 1);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+    dataDir = cluster.getDataDirectory();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    for (int i = 0; i < 3; i++) {
+      new File(dataDir, "data"+(2*i+1)).setExecutable(true);
+      new File(dataDir, "data"+(2*i+2)).setExecutable(true);
+    }
+    cluster.shutdown();
+  }
+
+  /**
+   * Test the DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY configuration
+   * option, ie the DN shuts itself down when the number of failures
+   * experienced drops below the tolerated amount.
+   */
+  @Test
+  public void testConfigureMinValidVolumes() throws Exception {
+    assumeTrue(!System.getProperty("os.name").startsWith("Windows"));
+
+    // Bring up two additional datanodes that need both of their volumes
+    // functioning in order to stay up.
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 0);
+    cluster.startDataNodes(conf, 2, true, null, null);
+    cluster.waitActive();
+    FSNamesystem ns = cluster.getNamesystem();
+    long origCapacity = DFSTestUtil.getLiveDatanodeCapacity(ns);
+    long dnCapacity = DFSTestUtil.getDatanodeCapacity(ns, 0);
+
+    // Fail a volume on the 2nd DN
+    File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
+    assertTrue("Couldn't chmod local vol", dn2Vol1.setExecutable(false));
+
+    // Should only get two replicas (the first DN and the 3rd)
+    Path file1 = new Path("/test1");
+    DFSTestUtil.createFile(fs, file1, 1024, (short)3, 1L);
+    DFSTestUtil.waitReplication(fs, file1, (short)2);
+
+    // Check that this single failure caused a DN to die.
+    DFSTestUtil.waitForDatanodeStatus(ns, 2, 1, 0, 
+        origCapacity - (1*dnCapacity), WAIT_FOR_HEARTBEATS);
+
+    // If we restore the volume we should still only be able to get
+    // two replicas since the DN is still considered dead.
+    assertTrue("Couldn't chmod local vol", dn2Vol1.setExecutable(true));
+    Path file2 = new Path("/test2");
+    DFSTestUtil.createFile(fs, file2, 1024, (short)3, 1L);
+    DFSTestUtil.waitReplication(fs, file2, (short)2);
+  }
+
+  /**
+   * Test invalid DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY values.
+   */
+  @Test
+  public void testInvalidFailedVolumesConfig() throws Exception {
+    assumeTrue(!System.getProperty("os.name").startsWith("Windows"));
+
+    /*
+     * Bring up another datanode that has an invalid value set.
+     * We should still be able to create a file with two replicas
+     * since the minimum valid volume parameter is only checked
+     * when we experience a disk error.
+     */
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, -1);
+    cluster.startDataNodes(conf, 1, true, null, null);
+    cluster.waitActive();
+    Path file1 = new Path("/test1");
+    DFSTestUtil.createFile(fs, file1, 1024, (short)2, 1L);
+    DFSTestUtil.waitReplication(fs, file1, (short)2);
+
+    // Ditto if the value is too big.
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 100);
+    cluster.startDataNodes(conf, 1, true, null, null);
+    cluster.waitActive();
+    Path file2 = new Path("/test1");
+    DFSTestUtil.createFile(fs, file2, 1024, (short)2, 1L);
+    DFSTestUtil.waitReplication(fs, file2, (short)2);
+  }
+}

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java Tue May  3 01:34:04 2011
@@ -801,7 +801,7 @@ public class NNThroughputBenchmark {
       // register datanode
       // TODO:FEDERATION currently a single block pool is supported
       DatanodeCommand[] cmds = nameNode.sendHeartbeat(dnRegistration,
-          DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED, 0, 0);
+          DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED, 0, 0, 0);
       if(cmds != null) {
         for (DatanodeCommand cmd : cmds ) {
           if(LOG.isDebugEnabled()) {
@@ -846,7 +846,7 @@ public class NNThroughputBenchmark {
       // register datanode
       // TODO:FEDERATION currently a single block pool is supported
       DatanodeCommand[] cmds = nameNode.sendHeartbeat(dnRegistration,
-          DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED, 0, 0);
+          DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED, 0, 0, 0);
       if (cmds != null) {
         for (DatanodeCommand cmd : cmds) {
           if (cmd.getAction() == DatanodeProtocol.DNA_TRANSFER) {

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java Tue May  3 01:34:04 2011
@@ -123,7 +123,7 @@ public class TestDeadDatanode {
 
     // Ensure heartbeat from dead datanode is rejected with a command
     // that asks datanode to register again
-    DatanodeCommand[] cmd = dnp.sendHeartbeat(reg, 0, 0, 0, 0, 0, 0);
+    DatanodeCommand[] cmd = dnp.sendHeartbeat(reg, 0, 0, 0, 0, 0, 0, 0);
     Assert.assertEquals(1, cmd.length);
     Assert.assertEquals(cmd[0].getAction(), DatanodeCommand.REGISTER
         .getAction());

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestHeartbeatHandling.java Tue May  3 01:34:04 2011
@@ -81,7 +81,6 @@ public class TestHeartbeatHandling exten
             blockList.add(new Block(i, 0, GenerationStamp.FIRST_VALID_STAMP));
           }
           dd.addBlocksToBeInvalidated(blockList);
-               
           cmds = sendHeartBeat(nodeReg, dd, namesystem);
           assertEquals(2, cmds.length);
           assertEquals(DatanodeProtocol.DNA_TRANSFER, cmds[0].getAction());
@@ -115,6 +114,6 @@ public class TestHeartbeatHandling exten
   private static DatanodeCommand[] sendHeartBeat(DatanodeRegistration nodeReg,
       DatanodeDescriptor dd, FSNamesystem namesystem) throws IOException {
     return namesystem.handleHeartbeat(nodeReg, dd.getCapacity(), 
-        dd.getDfsUsed(), dd.getRemaining(), dd.getBlockPoolUsed(), 0, 0);
+        dd.getDfsUsed(), dd.getRemaining(), dd.getBlockPoolUsed(), 0, 0, 0);
   }
 }

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestOverReplicatedBlocks.java Tue May  3 01:34:04 2011
@@ -90,7 +90,7 @@ public class TestOverReplicatedBlocks ex
           String corruptMachineName = corruptDataNode.getName();
           for (DatanodeDescriptor datanode : namesystem.heartbeats) {
             if (!corruptMachineName.equals(datanode.getName())) {
-              datanode.updateHeartbeat(100L, 100L, 0L, 100L, 0);
+              datanode.updateHeartbeat(100L, 100L, 0L, 100L, 0, 0);
             }
           }
 

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java (original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestReplicationPolicy.java Tue May  3 01:34:04 2011
@@ -77,7 +77,7 @@ public class TestReplicationPolicy exten
     for(int i=0; i<NUM_OF_DATANODES; i++) {
       dataNodes[i].updateHeartbeat(
           2*FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-          2*FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0);
+          2*FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
     }
   }
   
@@ -93,7 +93,7 @@ public class TestReplicationPolicy exten
   public void testChooseTarget1() throws Exception {
     dataNodes[0].updateHeartbeat(
         2*FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 
-        FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 4); // overloaded
+        FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 4, 0); // overloaded
 
     DatanodeDescriptor[] targets;
     targets = replicator.chooseTarget(filename,
@@ -128,7 +128,7 @@ public class TestReplicationPolicy exten
     
     dataNodes[0].updateHeartbeat(
         2*FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-        FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0); 
+        FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0); 
   }
 
   private static DatanodeDescriptor[] chooseTarget(
@@ -229,7 +229,7 @@ public class TestReplicationPolicy exten
     // make data node 0 to be not qualified to choose
     dataNodes[0].updateHeartbeat(
         2*FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-        (FSConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0); // no space
+        (FSConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0); // no space
         
     DatanodeDescriptor[] targets;
     targets = replicator.chooseTarget(filename,
@@ -267,7 +267,7 @@ public class TestReplicationPolicy exten
 
     dataNodes[0].updateHeartbeat(
         2*FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-        FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0); 
+        FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0); 
   }
   
   /**
@@ -283,7 +283,7 @@ public class TestReplicationPolicy exten
     for(int i=0; i<2; i++) {
       dataNodes[i].updateHeartbeat(
           2*FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-          (FSConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0);
+          (FSConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0);
     }
       
     DatanodeDescriptor[] targets;
@@ -315,7 +315,7 @@ public class TestReplicationPolicy exten
     for(int i=0; i<2; i++) {
       dataNodes[i].updateHeartbeat(
           2*FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
-          FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0);
+          FSConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
     }
   }
   /**

Modified: hadoop/hdfs/trunk/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java?rev=1098888&r1=1098887&r2=1098888&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java (original)
+++ hadoop/hdfs/trunk/src/test/unit/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java Tue May  3 01:34:04 2011
@@ -109,8 +109,8 @@ public class TestBlockRecovery {
     when(namenode.versionRequest()).thenReturn(new NamespaceInfo
         (1, CLUSTER_ID, POOL_ID, 1L, 1));
     when(namenode.sendHeartbeat(any(DatanodeRegistration.class), anyLong(), 
-        anyLong(), anyLong(), anyLong(), anyInt(), anyInt())).thenReturn(
-            new DatanodeCommand[0]);
+        anyLong(), anyLong(), anyLong(), anyInt(), anyInt(), anyInt()))
+        .thenReturn(new DatanodeCommand[0]);
     dn = new DataNode(conf, dirs, null);
     
     DataNodeTestUtils.setBPNamenodeByIndex(dn, nsifno, POOL_ID, namenode);



Mime
View raw message