From 33518e561368c372bf9254b6b55a9b0c499fbd4d Mon Sep 17 00:00:00 2001 From: Vinayakumar B Date: Mon, 4 Aug 2014 08:43:51 +0000 Subject: [PATCH] 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 --- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 2 ++ .../hdfs/server/datanode/BlockReceiver.java | 8 +++--- .../hadoop/hdfs/server/datanode/DataNode.java | 28 ++++++++++++++----- .../fsdataset/impl/FsDatasetImpl.java | 2 +- .../hdfs/server/datanode/TestDiskError.java | 4 +-- 5 files changed, 30 insertions(+), 14 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 101a5dd8d2c..9514f326a6f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/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 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java index 2f3909ba564..fb7ecd69e9a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java +++ b/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 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index b55abed7e46..a3ace9b5125 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/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; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java index b133f60534a..a43ef849202 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java +++ b/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 { return f; // if file is not null, but doesn't exist - possibly disk failed - datanode.checkDiskError(); + datanode.checkDiskErrorAsync(); } if (LOG.isDebugEnabled()) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java index 798b7b7c705..4b5b6e1ec4f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java +++ b/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)));