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 ji...@apache.org on 2015/05/05 01:43:09 UTC

[05/33] hadoop git commit: HDFS-8276. LazyPersistFileScrubber should be disabled if scrubber interval configured zero. (Contributed by Surendra Singh Lilhore)

HDFS-8276. LazyPersistFileScrubber should be disabled if scrubber interval configured zero. (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/64d30a61
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/64d30a61
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/64d30a61

Branch: refs/heads/HDFS-7240
Commit: 64d30a61867e5cb0a2acaa7ae4fa4d3eb3be8edd
Parents: 3393461
Author: Arpit Agarwal <ar...@apache.org>
Authored: Fri May 1 11:11:48 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Fri May 1 14:43:09 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../hdfs/server/namenode/FSNamesystem.java      |  8 +++++--
 .../fsdataset/impl/LazyPersistTestCase.java     | 20 ++++++++++++----
 .../fsdataset/impl/TestLazyPersistFiles.java    | 25 ++++++++++++++++++++
 4 files changed, 50 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/64d30a61/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 5f6b283..b5c5e6b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -595,6 +595,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8300. Fix unit test failures and findbugs warning caused by HDFS-8283.
     (jing9)
 
+    HDFS-8276. LazyPersistFileScrubber should be disabled if scrubber interval
+    configured zero. (Surendra Singh Lilhore via Arpit Agarwal)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/64d30a61/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 b938263..d5ff80e 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
@@ -825,9 +825,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC,
           DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC_DEFAULT);
 
-      if (this.lazyPersistFileScrubIntervalSec == 0) {
+      if (this.lazyPersistFileScrubIntervalSec < 0) {
         throw new IllegalArgumentException(
-            DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC + " must be non-zero.");
+            DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC
+                + " must be zero (for disable) or greater than zero.");
       }
 
       // For testing purposes, allow the DT secret manager to be started regardless
@@ -1173,6 +1174,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         lazyPersistFileScrubber = new Daemon(new LazyPersistFileScrubber(
             lazyPersistFileScrubIntervalSec));
         lazyPersistFileScrubber.start();
+      } else {
+        LOG.warn("Lazy persist file scrubber is disabled,"
+            + " configured scrub interval is zero.");
       }
 
       cacheManager.startMonitorThread();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/64d30a61/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 d46964b..93cd23a 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
@@ -58,6 +58,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -239,12 +240,17 @@ public abstract class LazyPersistTestCase {
       long ramDiskStorageLimit,
       long evictionLowWatermarkReplicas,
       boolean useSCR,
-      boolean useLegacyBlockReaderLocal) throws IOException {
+      boolean useLegacyBlockReaderLocal,
+      boolean disableScrubber) throws IOException {
 
     Configuration conf = new Configuration();
     conf.setLong(DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    conf.setInt(DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC,
-                LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC);
+    if (disableScrubber) {
+      conf.setInt(DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC, 0);
+    } else {
+      conf.setInt(DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC,
+          LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC);
+    }
     conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, HEARTBEAT_INTERVAL_SEC);
     conf.setInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
                 HEARTBEAT_RECHECK_INTERVAL_MSEC);
@@ -357,11 +363,16 @@ public abstract class LazyPersistTestCase {
       return this;
     }
 
+    public ClusterWithRamDiskBuilder disableScrubber() {
+      this.disableScrubber = true;
+      return this;
+    }
+
     public void build() throws IOException {
       LazyPersistTestCase.this.startUpCluster(
           numDatanodes, hasTransientStorage, storageTypes, ramDiskReplicaCapacity,
           ramDiskStorageLimit, evictionLowWatermarkReplicas,
-          useScr, useLegacyBlockReaderLocal);
+          useScr, useLegacyBlockReaderLocal,disableScrubber);
     }
 
     private int numDatanodes = REPL_FACTOR;
@@ -372,6 +383,7 @@ public abstract class LazyPersistTestCase {
     private boolean useScr = false;
     private boolean useLegacyBlockReaderLocal = false;
     private long evictionLowWatermarkReplicas = EVICTION_LOW_WATERMARK;
+    private boolean disableScrubber=false;
   }
 
   protected final void triggerBlockReport()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/64d30a61/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 41398c8..60cc8fe 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
@@ -120,6 +120,31 @@ public class TestLazyPersistFiles extends LazyPersistTestCase {
                is(0L));
   }
 
+  @Test
+  public void testDisableLazyPersistFileScrubber()
+      throws IOException, InterruptedException {
+    getClusterBuilder().setRamDiskReplicaCapacity(2).disableScrubber().build();
+    final String METHOD_NAME = GenericTestUtils.getMethodName();
+    Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
+
+    makeTestFile(path1, BLOCK_SIZE, true);
+    ensureFileReplicasOnStorageType(path1, RAM_DISK);
+
+    // Stop the DataNode and sleep for the time it takes the NN to
+    // detect the DN as being dead.
+    cluster.shutdownDataNodes();
+    Thread.sleep(30000L);
+
+    // Next, wait for the replication monitor to mark the file as corrupt
+    Thread.sleep(2 * DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT * 1000);
+
+    // Wait for the LazyPersistFileScrubber to run
+    Thread.sleep(2 * LAZY_WRITE_FILE_SCRUBBER_INTERVAL_SEC * 1000);
+
+    // Ensure that path1 exist.
+    Assert.assertTrue(fs.exists(path1));
+
+  }
   /**
    * Concurrent read from the same node and verify the contents.
    */