You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by om...@apache.org on 2011/03/04 05:30:21 UTC

svn commit: r1077571 - in /hadoop/common/branches/branch-0.20-security-patches/src: hdfs/org/apache/hadoop/hdfs/protocol/ hdfs/org/apache/hadoop/hdfs/server/datanode/ hdfs/org/apache/hadoop/hdfs/server/namenode/ hdfs/org/apache/hadoop/hdfs/server/proto...

Author: omalley
Date: Fri Mar  4 04:30:20 2011
New Revision: 1077571

URL: http://svn.apache.org/viewvc?rev=1077571&view=rev
Log:
commit 976d6cfe056695befbcc8c6bf0963761faaf5e80
Author: Konstantin Shvachko <sh...@cdev6023.inktomisearch.com>
Date:   Wed Jul 21 02:00:38 2010 +0000

    HDFS:457 from https://issues.apache.org/jira/secure/attachment/12450007/HDFS-457-y20.patch
    
    +++ b/YAHOO-CHANGES.txt
    +
    +    HDFS-457. Reintroduce volume failure tolerance for DataNodes. (shv)
    +

Modified:
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/protocol/FSConstants.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java
    hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
    hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/protocol/FSConstants.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/protocol/FSConstants.java?rev=1077571&r1=1077570&r2=1077571&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/protocol/FSConstants.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/protocol/FSConstants.java Fri Mar  4 04:30:20 2011
@@ -56,6 +56,7 @@ public interface FSConstants {
   public static final int DEFAULT_DATA_SOCKET_SIZE = 128 * 1024;
 
   public static final int SIZE_OF_INTEGER = Integer.SIZE / Byte.SIZE;
+  public static final int MIN_NUM_OF_VALID_VOLUMES = 1;// for a DN to run
 
   // SafeMode actions
   public enum SafeModeAction{ SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET; }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java?rev=1077571&r1=1077570&r2=1077571&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java Fri Mar  4 04:30:20 2011
@@ -117,6 +117,9 @@ class BlockReceiver implements java.io.C
       
       // check if there is a disk error
       IOException cause = FSDataset.getCauseIfDiskError(ioe);
+      DataNode.LOG.warn("IOException in BlockReceiver constructor. Cause is ",
+          cause);
+      
       if (cause != null) { // possible disk error
         ioe = cause;
         datanode.checkDiskError(ioe); // may throw an exception here
@@ -830,7 +833,14 @@ class BlockReceiver implements java.io.C
                 DataTransferProtocol.OP_STATUS_SUCCESS}).write(replyOut);
             replyOut.flush();
         } catch (Exception e) {
+          LOG.warn("IOException in BlockReceiver.lastNodeRun: ", e);
           if (running) {
+            try {
+              datanode.checkDiskError(e); // may throw an exception here
+            } catch (IOException ioe) {
+              LOG.warn("DataNode.chekDiskError failed in lastDataNodeRun with: ",
+                  ioe);
+            }
             LOG.info("PacketResponder " + block + " " + numTargets + 
                      " Exception " + StringUtils.stringifyException(e));
             running = false;

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java?rev=1077571&r1=1077570&r2=1077571&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java Fri Mar  4 04:30:20 2011
@@ -724,11 +724,14 @@ public class DataNode extends Configured
   }
   
   
-  /* Check if there is no space in disk or the disk is read-only
-   *  when IOException occurs. 
-   * If so, handle the error */
-  protected void checkDiskError( IOException e ) throws IOException {
-    if (e.getMessage() != null && 
+  /** Check if there is no space in disk 
+   *  @param e that caused this checkDiskError call
+   **/
+  protected void checkDiskError(Exception e ) throws IOException {
+    
+    LOG.warn("checkDiskError: exception: ", e);
+    
+    if (e.getMessage() != null &&
         e.getMessage().startsWith("No space left on device")) {
       throw new DiskOutOfSpaceException("No space left on device");
     } else {
@@ -736,8 +739,11 @@ public class DataNode extends Configured
     }
   }
   
-  /* Check if there is no disk space and if so, handle the error*/
-  protected void checkDiskError( ) throws IOException {
+  /**
+   *  Check if there is a disk failure and if so, handle the error
+   *
+   **/
+  protected void checkDiskError( ) {
     try {
       data.checkDataDir();
     } catch(DiskErrorException de) {
@@ -746,13 +752,31 @@ public class DataNode extends Configured
   }
   
   private void handleDiskError(String errMsgr) {
-    LOG.warn("DataNode is shutting down.\n" + errMsgr);
-    shouldRun = false;
+    boolean hasEnoughResource = data.hasEnoughResource();
+    LOG.warn("DataNode.handleDiskError: Keep Running: " + hasEnoughResource);
+    
+    //if hasEnoughtResource = true - more volumes are available, so we don't want 
+    // to shutdown DN completely and don't want NN to remove it.
+    int dp_error = DatanodeProtocol.DISK_ERROR;
+    if(hasEnoughResource == false) {
+      // DN will be shutdown and NN should remove it
+      dp_error = DatanodeProtocol.FATAL_DISK_ERROR;
+    }
+    //inform NameNode
     try {
       namenode.errorReport(
-                           dnRegistration, DatanodeProtocol.DISK_ERROR, errMsgr);
+                           dnRegistration, dp_error, errMsgr);
     } catch(IOException ignored) {              
     }
+    
+    
+    if(hasEnoughResource) {
+      scheduleBlockReport(0);
+      return; // do not shutdown
+    }
+    
+    LOG.warn("DataNode is shutting down.\n" + errMsgr);
+    shouldRun = false; 
   }
     
   /** Number of concurrent xceivers per node. */
@@ -1261,6 +1285,9 @@ public class DataNode extends Configured
       } catch (IOException ie) {
         LOG.warn(dnRegistration + ":Failed to transfer " + b + " to " + targets[0].getName()
             + " got " + StringUtils.stringifyException(ie));
+        // check if there are any disk problem
+        datanode.checkDiskError();
+        
       } finally {
         xmitsInProgress.getAndDecrement();
         IOUtils.closeStream(blockSender);

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDataset.java?rev=1077571&r1=1077570&r2=1077571&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDataset.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDataset.java Fri Mar  4 04:30:20 2011
@@ -17,24 +17,40 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import java.io.*;
-import java.util.*;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.RandomAccessFile;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.TreeSet;
 
 import javax.management.NotCompliantMBeanException;
 import javax.management.ObjectName;
 import javax.management.StandardMBean;
 
-import org.apache.hadoop.fs.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DF;
+import org.apache.hadoop.fs.DU;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.FSConstants;
+import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
+import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.metrics.util.MBeanUtil;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
-import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
+import org.mortbay.log.Log;
 
 /**************************************************
  * FSDataset manages a set of data blocks.  Each block
@@ -485,9 +501,25 @@ public class FSDataset implements FSCons
     FSVolumeSet(FSVolume[] volumes) {
       this.volumes = volumes;
     }
+    
+    private int numberOfVolumes() {
+      return volumes.length;
+    }
       
     synchronized FSVolume getNextVolume(long blockSize) throws IOException {
+      
+      if(volumes.length < 1) {
+        throw new DiskOutOfSpaceException("No more available volumes");
+      }
+      
+      // since volumes could've been removed because of the failure
+      // make sure we are not out of bounds
+      if(curVolume >= volumes.length) {
+        curVolume = 0;
+      }
+      
       int startVolume = curVolume;
+      
       while (true) {
         FSVolume volume = volumes[curVolume];
         curVolume = (curVolume + 1) % volumes.length;
@@ -534,10 +566,46 @@ public class FSDataset implements FSCons
       }
     }
       
-    synchronized void checkDirs() throws DiskErrorException {
+    /**
+     * goes over all the volumes and checkDir eachone of them
+     * if one throws DiskErrorException - removes from the list of active 
+     * volumes. 
+     * @return list of all the removed volumes
+     */
+    synchronized List<FSVolume> checkDirs() {
+      
+      ArrayList<FSVolume> removed_vols = null;  
+      
       for (int idx = 0; idx < volumes.length; idx++) {
-        volumes[idx].checkDirs();
+        FSVolume fsv = volumes[idx];
+        try {
+          fsv.checkDirs();
+        } catch (DiskErrorException e) {
+          DataNode.LOG.warn("Removing failed volume " + fsv + ": ",e);
+          if(removed_vols == null) {
+            removed_vols = new ArrayList<FSVolume>(1);
+          }
+          removed_vols.add(volumes[idx]);
+          volumes[idx] = null; //remove the volume
+        }
+      }
+      
+      // repair array - copy non null elements
+      int removed_size = (removed_vols==null)? 0 : removed_vols.size();
+      if(removed_size > 0) {
+        FSVolume fsvs[] = new FSVolume [volumes.length-removed_size];
+        for(int idx=0,idy=0; idx<volumes.length; idx++) {
+          if(volumes[idx] != null) {
+            fsvs[idy] = volumes[idx];
+            idy++;
+          }
+        }
+        volumes = fsvs; // replace array of volumes
       }
+      Log.info("Completed FSVolumeSet.checkDirs. Removed=" + removed_size + 
+          "volumes. List of current volumes: " +   toString());
+      
+      return removed_vols;
     }
       
     public String toString() {
@@ -700,7 +768,14 @@ public class FSDataset implements FSCons
   public long getDfsUsed() throws IOException {
     return volumes.getDfsUsed();
   }
-  
+  /**
+   * Return true - if there are still valid volumes 
+   * on the DataNode
+   */
+  public boolean hasEnoughResource(){
+    return volumes.numberOfVolumes() >= MIN_NUM_OF_VALID_VOLUMES;
+  }
+
   /**
    * Return total capacity, used and unused
    */
@@ -1226,17 +1301,32 @@ public class FSDataset implements FSCons
    * Check whether the given block is a valid one.
    */
   public boolean isValidBlock(Block b) {
-    return validateBlockFile(b) != null;
+    File f = null;;
+    try {
+      f = validateBlockFile(b);
+    } catch(IOException e) {
+      Log.warn("Block " + b + " is not valid:",e);
+    }
+    
+    return f != null;
   }
 
   /**
    * Find the file corresponding to the block and return it if it exists.
    */
-  File validateBlockFile(Block b) {
+  File validateBlockFile(Block b) throws IOException {
     //Should we check for metadata file too?
     File f = getFile(b);
-    if(f != null && f.exists())
-      return f;
+    
+    if(f != null ) {
+      if(f.exists())
+        return f;
+   
+      // if file is not null, but doesn't exist - possibly disk failed
+      DataNode datanode = DataNode.getDataNode();
+      datanode.checkDiskError();
+    }
+    
     if (InterDatanodeProtocol.LOG.isDebugEnabled()) {
       InterDatanodeProtocol.LOG.debug("b=" + b + ", f=" + f);
     }
@@ -1375,10 +1465,51 @@ public class FSDataset implements FSCons
 
   /**
    * check if a data directory is healthy
+   * if some volumes failed - make sure to remove all the blocks that belong
+   * to these volumes
    * @throws DiskErrorException
    */
   public void checkDataDir() throws DiskErrorException {
-    volumes.checkDirs();
+    long total_blocks=0, removed_blocks=0;
+    List<FSVolume> failed_vols =  volumes.checkDirs();
+    
+    //if there no failed volumes return
+    if(failed_vols == null) 
+      return;
+    
+    // else 
+    // remove related blocks
+    long mlsec = System.currentTimeMillis();
+    synchronized (this) {
+      Iterator<Block> ib = volumeMap.keySet().iterator();
+      while(ib.hasNext()) {
+        Block b = ib.next();
+        total_blocks ++;
+        // check if the volume block belongs to still valid
+        FSVolume vol = volumeMap.get(b).getVolume();
+        for(FSVolume fv: failed_vols) {
+          if(vol == fv) {
+            DataNode.LOG.warn("removing block " + b.getBlockId() + " from vol " 
+                + vol.dataDir.dir.getAbsolutePath());
+            ib.remove();
+            removed_blocks++;
+            break;
+          }
+        }
+      }
+    } // end of sync
+    mlsec = System.currentTimeMillis() - mlsec;
+    DataNode.LOG.warn(">>>>>>>>>>>>Removed " + removed_blocks + " out of " + total_blocks +
+        "(took " + mlsec + " millisecs)");
+
+    // report the error
+    StringBuilder sb = new StringBuilder();
+    for(FSVolume fv : failed_vols) {
+      sb.append(fv.dataDir.dir.getAbsolutePath() + ";");
+    }
+
+    throw  new DiskErrorException("DataNode failed volumes:" + sb);
+  
   }
     
 

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java?rev=1077571&r1=1077570&r2=1077571&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java Fri Mar  4 04:30:20 2011
@@ -264,4 +264,10 @@ public interface FSDatasetInterface exte
    * @throws IOException
    */
   public void validateBlockMetadata(Block b) throws IOException;
+
+  /**
+   * checks how many valid storage volumes are there in the DataNode
+   * @return true if more then minimum valid volumes left in the FSDataSet
+   */
+  public boolean hasEnoughResource();
 }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java?rev=1077571&r1=1077570&r2=1077571&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java Fri Mar  4 04:30:20 2011
@@ -934,6 +934,8 @@ public class NameNode implements ClientP
     }
     verifyRequest(nodeReg);
     if (errorCode == DatanodeProtocol.DISK_ERROR) {
+      LOG.warn("Volume failed on " + dnName); 
+    } else if (errorCode == DatanodeProtocol.FATAL_DISK_ERROR) {
       namesystem.removeDatanode(nodeReg);            
     }
   }

Modified: hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java?rev=1077571&r1=1077570&r2=1077571&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/hdfs/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java Fri Mar  4 04:30:20 2011
@@ -50,8 +50,9 @@ public interface DatanodeProtocol extend
   
   // error code
   final static int NOTIFY = 0;
-  final static int DISK_ERROR = 1;
+  final static int DISK_ERROR = 1; // there are still valid volumes on DN
   final static int INVALID_BLOCK = 2;
+  final static int FATAL_DISK_ERROR = 3; // no valid volumes left on DN
 
   /**
    * Determines actions that data node should perform 

Modified: hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java?rev=1077571&r1=1077570&r2=1077571&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java (original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/test/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java Fri Mar  4 04:30:20 2011
@@ -655,4 +655,8 @@ public class SimulatedFSDataset  impleme
   public String getStorageInfo() {
     return "Simulated FSDataset-" + storageId;
   }
+  
+  public boolean hasEnoughResource() {
+    return true;
+  }
 }