Browse Source

HDFS-5185. DN fails to startup if one of the data dir is full. Contributed by Vinayakumar B.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1615504 13f79535-47bb-0310-9956-ffa450edef68
Vinayakumar B 10 years ago
parent
commit
33518e5613

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -433,6 +433,8 @@ Release 2.6.0 - UNRELEASED
     HDFS-5723. Append failed FINALIZED replica should not be accepted as valid
     when that block is underconstruction (vinayakumarb)
 
+    HDFS-5185. DN fails to startup if one of the data dir is full. (vinayakumarb)
+
 Release 2.5.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -253,7 +253,7 @@ class BlockReceiver implements Closeable {
       
       if (cause != null) { // possible disk error
         ioe = cause;
-        datanode.checkDiskError();
+        datanode.checkDiskErrorAsync();
       }
       
       throw ioe;
@@ -329,7 +329,7 @@ class BlockReceiver implements Closeable {
     }
     // disk check
     if(ioe != null) {
-      datanode.checkDiskError();
+      datanode.checkDiskErrorAsync();
       throw ioe;
     }
   }
@@ -639,7 +639,7 @@ class BlockReceiver implements Closeable {
           manageWriterOsCache(offsetInBlock);
         }
       } catch (IOException iex) {
-        datanode.checkDiskError();
+        datanode.checkDiskErrorAsync();
         throw iex;
       }
     }
@@ -1208,7 +1208,7 @@ class BlockReceiver implements Closeable {
         } catch (IOException e) {
           LOG.warn("IOException in BlockReceiver.run(): ", e);
           if (running) {
-            datanode.checkDiskError();
+            datanode.checkDiskErrorAsync();
             LOG.info(myString, e);
             running = false;
             if (!Thread.interrupted()) { // failure not caused by interruption

+ 21 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -1075,6 +1075,11 @@ public class DataNode extends Configured
     // In the case that this is the first block pool to connect, initialize
     // the dataset, block scanners, etc.
     initStorage(nsInfo);
+
+    // Exclude failed disks before initializing the block pools to avoid startup
+    // failures.
+    checkDiskError();
+
     initPeriodicScanners(conf);
     
     data.addBlockPool(nsInfo.getBlockPoolID(), conf);
@@ -1510,9 +1515,9 @@ public class DataNode extends Configured
   
   
   /**
-   *  Check if there is a disk failure and if so, handle the error
+   * Check if there is a disk failure asynchronously and if so, handle the error
    */
-  public void checkDiskError() {
+  public void checkDiskErrorAsync() {
     synchronized(checkDiskErrorMutex) {
       checkDiskErrorFlag = true;
       if(checkDiskErrorThread == null) {
@@ -1821,7 +1826,7 @@ public class DataNode extends Configured
         LOG.warn(bpReg + ":Failed to transfer " + b + " to " +
             targets[0] + " got ", ie);
         // check if there are any disk problem
-        checkDiskError();
+        checkDiskErrorAsync();
       } finally {
         xmitsInProgress.getAndDecrement();
         IOUtils.closeStream(blockSender);
@@ -2759,7 +2764,18 @@ public class DataNode extends Configured
   public ShortCircuitRegistry getShortCircuitRegistry() {
     return shortCircuitRegistry;
   }
-  
+
+  /**
+   * Check the disk error
+   */
+  private void checkDiskError() {
+    try {
+      data.checkDataDir();
+    } catch (DiskErrorException de) {
+      handleDiskError(de.getMessage());
+    }
+  }
+
   /**
    * Starts a new thread which will check for disk error check request 
    * every 5 sec
@@ -2776,9 +2792,7 @@ public class DataNode extends Configured
               }
               if(tempFlag) {
                 try {
-                  data.checkDataDir();
-                } catch (DiskErrorException de) {
-                  handleDiskError(de.getMessage());
+                  checkDiskError();
                 } catch (Exception e) {
                   LOG.warn("Unexpected exception occurred while checking disk error  " + e);
                   checkDiskErrorThread = null;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java

@@ -1151,7 +1151,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         return f;
    
       // if file is not null, but doesn't exist - possibly disk failed
-      datanode.checkDiskError();
+      datanode.checkDiskErrorAsync();
     }
     
     if (LOG.isDebugEnabled()) {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java

@@ -201,7 +201,7 @@ public class TestDiskError {
   }
   
   /**
-   * Checks whether {@link DataNode#checkDiskError()} is being called or not.
+   * Checks whether {@link DataNode#checkDiskErrorAsync()} is being called or not.
    * Before refactoring the code the above function was not getting called 
    * @throws IOException, InterruptedException
    */
@@ -214,7 +214,7 @@ public class TestDiskError {
     DataNode dataNode = cluster.getDataNodes().get(0);
     long slackTime = dataNode.checkDiskErrorInterval/2;
     //checking for disk error
-    dataNode.checkDiskError();
+    dataNode.checkDiskErrorAsync();
     Thread.sleep(dataNode.checkDiskErrorInterval);
     long lastDiskErrorCheck = dataNode.getLastDiskErrorCheck();
     assertTrue("Disk Error check is not performed within  " + dataNode.checkDiskErrorInterval +  "  ms", ((Time.monotonicNow()-lastDiskErrorCheck) < (dataNode.checkDiskErrorInterval + slackTime)));