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 zj...@apache.org on 2015/05/04 22:04:52 UTC

[33/50] [abbrv] hadoop git commit: HDFS-8229. LAZY_PERSIST file gets deleted after NameNode restart. (Contributed by Surendra Singh Lilhore)

HDFS-8229. LAZY_PERSIST file gets deleted after NameNode restart. (Contributed by Surendra Singh Lilhore)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/41ef2145
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/41ef2145
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/41ef2145

Branch: refs/heads/YARN-2928
Commit: 41ef21456d47f3872ddbf944504c127c46f02ea7
Parents: 3660b7f
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri May 1 16:30:51 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Mon May 4 12:58:58 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../hdfs/server/namenode/FSNamesystem.java      |  9 +++++-
 .../fsdataset/impl/LazyPersistTestCase.java     |  1 +
 .../fsdataset/impl/TestLazyPersistFiles.java    | 31 ++++++++++++++++++++
 4 files changed, 43 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/41ef2145/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 179fe7e..1882df5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -601,6 +601,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8276. LazyPersistFileScrubber should be disabled if scrubber interval
     configured zero. (Surendra Singh Lilhore via Arpit Agarwal)
 
+    HDFS-8229. LAZY_PERSIST file gets deleted after NameNode restart.
+    (Surendra Singh Lilhore via Arpit Agarwal) 
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41ef2145/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index d5ff80e..809d594 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -4767,7 +4767,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     public void run() {
       while (fsRunning && shouldRun) {
         try {
-          clearCorruptLazyPersistFiles();
+          if (!isInSafeMode()) {
+            clearCorruptLazyPersistFiles();
+          } else {
+            if (FSNamesystem.LOG.isDebugEnabled()) {
+              FSNamesystem.LOG
+                  .debug("Namenode is in safemode, skipping scrubbing of corrupted lazy-persist files.");
+            }
+          }
           Thread.sleep(scrubIntervalSec * 1000);
         } catch (InterruptedException e) {
           FSNamesystem.LOG.info(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41ef2145/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
index 93cd23a..7e1aa81 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
@@ -258,6 +258,7 @@ public abstract class LazyPersistTestCase {
                 LAZY_WRITER_INTERVAL_SEC);
     conf.setLong(DFS_DATANODE_RAM_DISK_LOW_WATERMARK_BYTES,
                 evictionLowWatermarkReplicas * BLOCK_SIZE);
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_SAFEMODE_MIN_DATANODES_KEY, 1);
 
     if (useSCR) {
       conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/41ef2145/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
index 60cc8fe..950e9dc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistFiles.java
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
+import com.google.common.collect.Iterators;
 import com.google.common.util.concurrent.Uninterruptibles;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.Path;
@@ -145,6 +146,36 @@ public class TestLazyPersistFiles extends LazyPersistTestCase {
     Assert.assertTrue(fs.exists(path1));
 
   }
+
+ /**
+  * If NN restarted then lazyPersist files should not deleted
+  */
+  @Test
+  public void testFileShouldNotDiscardedIfNNRestarted() throws IOException,
+      InterruptedException {
+    getClusterBuilder().setRamDiskReplicaCapacity(2).build();
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
+    makeTestFile(path1, BLOCK_SIZE, true);
+    ensureFileReplicasOnStorageType(path1, RAM_DISK);
+
+    cluster.shutdownDataNodes();
+
+    cluster.restartNameNodes();
+
+    // wait for the replication monitor to mark the file as corrupt
+    Thread.sleep(2 * DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT * 1000);
+
+    Long corruptBlkCount = (long) Iterators.size(cluster.getNameNode()
+        .getNamesystem().getBlockManager().getCorruptReplicaBlockIterator());
+
+    // Check block detected as corrupted
+    assertThat(corruptBlkCount, is(1L));
+
+    // Ensure path1 exist.
+    Assert.assertTrue(fs.exists(path1));
+  }
+
   /**
    * Concurrent read from the same node and verify the contents.
    */