瀏覽代碼

HDFS-15961. standby namenode failed to start ordered snapshot deletion is enabled while having snapshottable directories (#2881)

bshashikant 4 年之前
父節點
當前提交
ef13f8ad6b

+ 28 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -124,7 +124,12 @@ import org.apache.hadoop.hdfs.server.common.ECTopologyVerifier;
 import org.apache.hadoop.hdfs.server.namenode.metrics.ReplicatedBlocksMBean;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 import org.apache.hadoop.ipc.ObserverRetryOnActiveException;
-import org.apache.hadoop.util.*;
+import org.apache.hadoop.util.Time;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.VersionInfo;
 
 import static org.apache.hadoop.util.Time.now;
 import static org.apache.hadoop.util.Time.monotonicNow;
@@ -8560,25 +8565,36 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /**
    * Check if snapshot roots are created for all existing snapshottable
    * directories. Create them if not.
+   * Only the active NameNode needs to execute this in HA setup once it is out
+   * of safe mode.
+   *
+   * The function gets called while exiting safe mode or post starting the
+   * services in Active NameNode, but comes into effect post whichever event
+   * happens later.
    */
   @Override
-  public void checkAndProvisionSnapshotTrashRoots() {
-    if (isSnapshotTrashRootEnabled) {
+  public synchronized void checkAndProvisionSnapshotTrashRoots() {
+    if (isSnapshotTrashRootEnabled && (haEnabled && inActiveState()
+        || !haEnabled) && !blockManager.isInSafeMode()) {
+      SnapshottableDirectoryStatus dirStatus = null;
       try {
         SnapshottableDirectoryStatus[] dirStatusList =
             getSnapshottableDirListing();
         if (dirStatusList == null) {
           return;
         }
-        for (SnapshottableDirectoryStatus dirStatus : dirStatusList) {
+        for (SnapshottableDirectoryStatus status : dirStatusList) {
+          dirStatus = status;
           String currDir = dirStatus.getFullPath().toString();
           if (!currDir.endsWith(Path.SEPARATOR)) {
             currDir += Path.SEPARATOR;
           }
           String trashPath = currDir + FileSystem.TRASH_PREFIX;
-          HdfsFileStatus fileStatus = getFileInfo(trashPath, false, false, false);
+          HdfsFileStatus fileStatus =
+              getFileInfo(trashPath, false, false, false);
           if (fileStatus == null) {
-            LOG.info("Trash doesn't exist for snapshottable directory {}. " + "Creating trash at {}", currDir, trashPath);
+            LOG.info("Trash doesn't exist for snapshottable directory {}. "
+                + "Creating trash at {}", currDir, trashPath);
             PermissionStatus permissionStatus =
                 new PermissionStatus(getRemoteUser().getShortUserName(), null,
                     SHARED_TRASH_PERMISSION);
@@ -8586,12 +8602,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           }
         }
       } catch (IOException e) {
-        final String msg =
-            "Could not provision Trash directory for existing "
-                + "snapshottable directories. Exiting Namenode.";
-        ExitUtil.terminate(1, msg);
+        if (dirStatus == null) {
+          LOG.error("Failed to get snapshottable directory list", e);
+        } else {
+          LOG.error("Could not provision Trash directory for existing "
+              + "snapshottable directory {}", dirStatus, e);
+        }
       }
-
     }
   }
 

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -2021,6 +2021,7 @@ public class NameNode extends ReconfigurableBase implements
     public void startActiveServices() throws IOException {
       try {
         namesystem.startActiveServices();
+        namesystem.checkAndProvisionSnapshotTrashRoots();
         startTrashEmptier(getConf());
       } catch (Throwable t) {
         doImmediateShutdown(t);

+ 37 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java

@@ -98,6 +98,7 @@ public class TestHASafeMode {
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+    conf.setBoolean("dfs.namenode.snapshot.trashroot.enabled", false);
 
     cluster = new MiniDFSCluster.Builder(conf)
       .nnTopology(MiniDFSNNTopology.simpleHATopology())
@@ -909,6 +910,42 @@ public class TestHASafeMode {
     assertSafeMode(nn1, 3, 3, 3, 0);
   }
 
+  @Test
+  public void testNameNodeCreateSnapshotTrashRootOnHASetup() throws Exception {
+    DistributedFileSystem dfs = cluster.getFileSystem(0);
+    final Path testDir = new Path("/disallowss/test2/");
+    final Path file0path = new Path(testDir, "file-0");
+    dfs.create(file0path).close();
+    dfs.allowSnapshot(testDir);
+    // .Trash won't be created right now since snapshot trash is disabled
+    final Path trashRoot = new Path(testDir, FileSystem.TRASH_PREFIX);
+    assertFalse(dfs.exists(trashRoot));
+    // Set dfs.namenode.snapshot.trashroot.enabled=true
+    cluster.getNameNode(0).getConf()
+        .setBoolean("dfs.namenode.snapshot.trashroot.enabled", true);
+    cluster.getNameNode(1).getConf()
+        .setBoolean("dfs.namenode.snapshot.trashroot.enabled", true);
+    restartActive();
+    cluster.transitionToActive(1);
+    dfs = cluster.getFileSystem(1);
+    // Make sure .Trash path does not exist yet as on NN1 trash root is not
+    // enabled
+    assertFalse(dfs.exists(trashRoot));
+    cluster.transitionToStandby(1);
+    cluster.transitionToActive(0);
+    dfs = cluster.getFileSystem(0);
+    // Check .Trash existence, should be created now
+    assertTrue(dfs.exists(trashRoot));
+    assertFalse(cluster.getNameNode(0).isInSafeMode());
+    restartStandby();
+    // Ensure Standby namenode is up and running
+    assertTrue(cluster.getNameNode(1).isStandbyState());
+    // Cleanup
+    dfs.delete(trashRoot, true);
+    dfs.disallowSnapshot(testDir);
+    dfs.delete(testDir, true);
+  }
+
   /**
    * Test transition to active when namenode in safemode.
    *