Просмотр исходного кода

HDFS-7359. NameNode in secured HA cluster fails to start if dfs.namenode.secondary.http-address cannot be interpreted as a network address. Contributed by Chris Nauroth.

(cherry picked from commit ba1d4ad25b301f7247f3f23df15e7f800e50feed)
cnauroth 10 лет назад
Родитель
Сommit
f172a08238

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

@@ -1239,6 +1239,10 @@ Release 2.6.0 - UNRELEASED
 
     HDFS-7328. TestTraceAdmin assumes Unix line endings. (cnauroth)
 
+    HDFS-7359. NameNode in secured HA cluster fails to start if
+    dfs.namenode.secondary.http-address cannot be interpreted as a network
+    address. (cnauroth)
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 17 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/GetJournalEditServlet.java

@@ -91,10 +91,23 @@ public class GetJournalEditServlet extends HttpServlet {
 
     Set<String> validRequestors = new HashSet<String>();
     validRequestors.addAll(DFSUtil.getAllNnPrincipals(conf));
-    validRequestors.add(
-        SecurityUtil.getServerPrincipal(conf
-            .get(DFSConfigKeys.DFS_SECONDARY_NAMENODE_KERBEROS_PRINCIPAL_KEY),
-            SecondaryNameNode.getHttpAddress(conf).getHostName()));
+    try {
+      validRequestors.add(
+          SecurityUtil.getServerPrincipal(conf
+              .get(DFSConfigKeys.DFS_SECONDARY_NAMENODE_KERBEROS_PRINCIPAL_KEY),
+              SecondaryNameNode.getHttpAddress(conf).getHostName()));
+    } catch (Exception e) {
+      // Don't halt if SecondaryNameNode principal could not be added.
+      LOG.debug("SecondaryNameNode principal could not be added", e);
+      String msg = String.format(
+        "SecondaryNameNode principal not considered, %s = %s, %s = %s",
+        DFSConfigKeys.DFS_SECONDARY_NAMENODE_KERBEROS_PRINCIPAL_KEY,
+        conf.get(DFSConfigKeys.DFS_SECONDARY_NAMENODE_KERBEROS_PRINCIPAL_KEY),
+        DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
+        conf.get(DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
+          DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_DEFAULT));
+      LOG.warn(msg);
+    }
 
     // Check the full principal name of all the configured valid requestors.
     for (String v : validRequestors) {

+ 16 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java

@@ -237,9 +237,23 @@ public class ImageServlet extends HttpServlet {
     validRequestors.add(SecurityUtil.getServerPrincipal(conf
         .get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY),
         NameNode.getAddress(conf).getHostName()));
-    validRequestors.add(SecurityUtil.getServerPrincipal(
+    try {
+      validRequestors.add(
+          SecurityUtil.getServerPrincipal(conf
+              .get(DFSConfigKeys.DFS_SECONDARY_NAMENODE_KERBEROS_PRINCIPAL_KEY),
+              SecondaryNameNode.getHttpAddress(conf).getHostName()));
+    } catch (Exception e) {
+      // Don't halt if SecondaryNameNode principal could not be added.
+      LOG.debug("SecondaryNameNode principal could not be added", e);
+      String msg = String.format(
+        "SecondaryNameNode principal not considered, %s = %s, %s = %s",
+        DFSConfigKeys.DFS_SECONDARY_NAMENODE_KERBEROS_PRINCIPAL_KEY,
         conf.get(DFSConfigKeys.DFS_SECONDARY_NAMENODE_KERBEROS_PRINCIPAL_KEY),
-        SecondaryNameNode.getHttpAddress(conf).getHostName()));
+        DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
+        conf.get(DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
+          DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_DEFAULT));
+      LOG.warn(msg);
+    }
 
     if (HAUtil.isHAEnabled(conf, DFSUtil.getNamenodeNameServiceId(conf))) {
       Configuration otherNnConf = HAUtil.getConfForOtherNode(conf);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java

@@ -45,7 +45,7 @@ public class TestNNWithQJM {
   final Configuration conf = new HdfsConfiguration();
   private MiniJournalCluster mjc = null;
   private final Path TEST_PATH = new Path("/test-dir");
-  private final Path TEST_PATH_2 = new Path("/test-dir");
+  private final Path TEST_PATH_2 = new Path("/test-dir-2");
 
   @Before
   public void resetSystemExit() {

+ 216 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestSecureNNWithQJM.java

@@ -0,0 +1,216 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.qjournal;
+
+import static org.junit.Assert.*;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HTTP_POLICY_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_JOURNALNODE_HTTPS_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_JOURNALNODE_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_JOURNALNODE_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_JOURNALNODE_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Properties;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+public class TestSecureNNWithQJM {
+
+  private static final Path TEST_PATH = new Path("/test-dir");
+  private static final Path TEST_PATH_2 = new Path("/test-dir-2");
+
+  private static HdfsConfiguration baseConf;
+  private static File baseDir;
+  private static MiniKdc kdc;
+
+  private MiniDFSCluster cluster;
+  private HdfsConfiguration conf;
+  private FileSystem fs;
+  private MiniJournalCluster mjc;
+
+  @Rule
+  public Timeout timeout = new Timeout(30000);
+
+  @BeforeClass
+  public static void init() throws Exception {
+    baseDir = new File(System.getProperty("test.build.dir", "target/test-dir"),
+      TestSecureNNWithQJM.class.getSimpleName());
+    FileUtil.fullyDelete(baseDir);
+    assertTrue(baseDir.mkdirs());
+
+    Properties kdcConf = MiniKdc.createConf();
+    kdc = new MiniKdc(kdcConf, baseDir);
+    kdc.start();
+
+    baseConf = new HdfsConfiguration();
+    SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS,
+      baseConf);
+    UserGroupInformation.setConfiguration(baseConf);
+    assertTrue("Expected configuration to enable security",
+      UserGroupInformation.isSecurityEnabled());
+
+    String userName = UserGroupInformation.getLoginUser().getShortUserName();
+    File keytabFile = new File(baseDir, userName + ".keytab");
+    String keytab = keytabFile.getAbsolutePath();
+    // Windows will not reverse name lookup "127.0.0.1" to "localhost".
+    String krbInstance = Path.WINDOWS ? "127.0.0.1" : "localhost";
+    kdc.createPrincipal(keytabFile,
+      userName + "/" + krbInstance,
+      "HTTP/" + krbInstance);
+    String hdfsPrincipal = userName + "/" + krbInstance + "@" + kdc.getRealm();
+    String spnegoPrincipal = "HTTP/" + krbInstance + "@" + kdc.getRealm();
+
+    baseConf.set(DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, hdfsPrincipal);
+    baseConf.set(DFS_NAMENODE_KEYTAB_FILE_KEY, keytab);
+    baseConf.set(DFS_DATANODE_KERBEROS_PRINCIPAL_KEY, hdfsPrincipal);
+    baseConf.set(DFS_DATANODE_KEYTAB_FILE_KEY, keytab);
+    baseConf.set(DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY, spnegoPrincipal);
+    baseConf.set(DFS_JOURNALNODE_KEYTAB_FILE_KEY, keytab);
+    baseConf.set(DFS_JOURNALNODE_KERBEROS_PRINCIPAL_KEY, hdfsPrincipal);
+    baseConf.set(DFS_JOURNALNODE_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY,
+      spnegoPrincipal);
+    baseConf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
+    baseConf.set(DFS_DATA_TRANSFER_PROTECTION_KEY, "authentication");
+    baseConf.set(DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTPS_ONLY.name());
+    baseConf.set(DFS_NAMENODE_HTTPS_ADDRESS_KEY, "localhost:0");
+    baseConf.set(DFS_DATANODE_HTTPS_ADDRESS_KEY, "localhost:0");
+    baseConf.set(DFS_JOURNALNODE_HTTPS_ADDRESS_KEY, "localhost:0");
+    baseConf.setInt(IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY, 10);
+
+    String keystoresDir = baseDir.getAbsolutePath();
+    String sslConfDir = KeyStoreTestUtil.getClasspathDir(
+      TestSecureNNWithQJM.class);
+    KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, baseConf, false);
+  }
+
+  @AfterClass
+  public static void destroy() {
+    if (kdc != null) {
+      kdc.stop();
+    }
+    FileUtil.fullyDelete(baseDir);
+  }
+
+  @Before
+  public void setup() throws Exception {
+    conf = new HdfsConfiguration(baseConf);
+  }
+
+  @After
+  public void shutdown() throws IOException {
+    IOUtils.cleanup(null, fs);
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+    if (mjc != null) {
+      mjc.shutdown();
+    }
+  }
+
+  @Test
+  public void testSecureMode() throws Exception {
+    doNNWithQJMTest();
+  }
+
+  @Test
+  public void testSecondaryNameNodeHttpAddressNotNeeded() throws Exception {
+    conf.set(DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY, "null");
+    doNNWithQJMTest();
+  }
+
+  /**
+   * Tests use of QJM with the defined cluster.
+   *
+   * @throws IOException if there is an I/O error
+   */
+  private void doNNWithQJMTest() throws IOException {
+    startCluster();
+    assertTrue(fs.mkdirs(TEST_PATH));
+
+    // Restart the NN and make sure the edit was persisted
+    // and loaded again
+    restartNameNode();
+
+    assertTrue(fs.exists(TEST_PATH));
+    assertTrue(fs.mkdirs(TEST_PATH_2));
+
+    // Restart the NN again and make sure both edits are persisted.
+    restartNameNode();
+    assertTrue(fs.exists(TEST_PATH));
+    assertTrue(fs.exists(TEST_PATH_2));
+  }
+
+  /**
+   * Restarts the NameNode and obtains a new FileSystem.
+   *
+   * @throws IOException if there is an I/O error
+   */
+  private void restartNameNode() throws IOException {
+    IOUtils.cleanup(null, fs);
+    cluster.restartNameNode();
+    fs = cluster.getFileSystem();
+  }
+
+  /**
+   * Starts a cluster using QJM with the defined configuration.
+   *
+   * @throws IOException if there is an I/O error
+   */
+  private void startCluster() throws IOException {
+    mjc = new MiniJournalCluster.Builder(conf)
+      .build();
+    conf.set(DFS_NAMENODE_EDITS_DIR_KEY,
+      mjc.getQuorumJournalURI("myjournal").toString());
+    cluster = new MiniDFSCluster.Builder(conf)
+      .build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+  }
+}