Quellcode durchsuchen

HDFS-7132. hdfs namenode -metadataVersion command does not honor configured name dirs. Contributed by Charles Lamb.

(cherry picked from commit f48686a1ad81823000534665a76264bba51182f4)
Andrew Wang vor 10 Jahren
Ursprung
Commit
0d116b18de

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

@@ -453,6 +453,9 @@ Release 2.6.0 - UNRELEASED
     HDFS-7001. Tests in TestTracing depends on the order of execution
     (iwasakims via cmccabe)
 
+    HDFS-7132. hdfs namenode -metadataVersion command does not honor
+    configured name dirs. (Charles Lamb via wang)
+
     BREAKDOWN OF HDFS-6134 AND HADOOP-10150 SUBTASKS AND RELATED JIRAS
   
       HDFS-6387. HDFS CLI admin tool for creating & deleting an

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

@@ -1346,6 +1346,9 @@ public class NameNode implements NameNodeStatusMXBean {
    */
   private static boolean printMetadataVersion(Configuration conf)
     throws IOException {
+    final String nsId = DFSUtil.getNamenodeNameServiceId(conf);
+    final String namenodeId = HAUtil.getNameNodeId(conf, nsId);
+    NameNode.initializeGenericKeys(conf, nsId, namenodeId);
     final FSImage fsImage = new FSImage(conf);
     final FSNamesystem fs = new FSNamesystem(conf, fsImage, false);
     return fsImage.recoverTransitionRead(

+ 22 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestMetadataVersionOutput.java

@@ -25,27 +25,22 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 
 import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.PrintStream;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
+
 public class TestMetadataVersionOutput {
 
   private MiniDFSCluster dfsCluster = null;
   private final Configuration conf = new Configuration();
 
-  @Before
-  public void setUp() throws Exception {
-    dfsCluster = new MiniDFSCluster.Builder(conf).
-            numDataNodes(1).
-            checkExitOnShutdown(false).
-            build();
-    dfsCluster.waitClusterUp();
-  }
-
   @After
   public void tearDown() throws Exception {
     if (dfsCluster != null) {
@@ -54,9 +49,26 @@ public class TestMetadataVersionOutput {
     Thread.sleep(2000);
   }
 
+  private void initConfig() {
+    conf.set(DFS_NAMESERVICE_ID, "ns1");
+    conf.set(DFS_HA_NAMENODES_KEY_PREFIX + ".ns1", "nn1");
+    conf.set(DFS_HA_NAMENODE_ID_KEY, "nn1");
+    conf.set(DFS_NAMENODE_NAME_DIR_KEY + ".ns1.nn1", MiniDFSCluster.getBaseDirectory() + "1");
+    conf.unset(DFS_NAMENODE_NAME_DIR_KEY);
+  }
+
   @Test(timeout = 30000)
   public void testMetadataVersionOutput() throws IOException {
 
+    initConfig();
+    dfsCluster = new MiniDFSCluster.Builder(conf).
+        manageNameDfsDirs(false).
+        numDataNodes(1).
+        checkExitOnShutdown(false).
+        build();
+    dfsCluster.waitClusterUp();
+    dfsCluster.shutdown(false);
+    initConfig();
     final PrintStream origOut = System.out;
     final ByteArrayOutputStream baos = new ByteArrayOutputStream();
     final PrintStream stdOut = new PrintStream(baos);