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 ba1d4ad25b)
This commit is contained in:
cnauroth 2014-11-05 15:09:22 -08:00
parent 1857623634
commit f172a08238
5 changed files with 254 additions and 7 deletions

View File

@ -1239,6 +1239,10 @@ Release 2.6.0 - UNRELEASED
HDFS-7328. TestTraceAdmin assumes Unix line endings. (cnauroth) 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 Release 2.5.2 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -91,10 +91,23 @@ public class GetJournalEditServlet extends HttpServlet {
Set<String> validRequestors = new HashSet<String>(); Set<String> validRequestors = new HashSet<String>();
validRequestors.addAll(DFSUtil.getAllNnPrincipals(conf)); validRequestors.addAll(DFSUtil.getAllNnPrincipals(conf));
validRequestors.add( try {
SecurityUtil.getServerPrincipal(conf validRequestors.add(
.get(DFSConfigKeys.DFS_SECONDARY_NAMENODE_KERBEROS_PRINCIPAL_KEY), SecurityUtil.getServerPrincipal(conf
SecondaryNameNode.getHttpAddress(conf).getHostName())); .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. // Check the full principal name of all the configured valid requestors.
for (String v : validRequestors) { for (String v : validRequestors) {

View File

@ -237,9 +237,23 @@ public class ImageServlet extends HttpServlet {
validRequestors.add(SecurityUtil.getServerPrincipal(conf validRequestors.add(SecurityUtil.getServerPrincipal(conf
.get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY), .get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY),
NameNode.getAddress(conf).getHostName())); 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), 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))) { if (HAUtil.isHAEnabled(conf, DFSUtil.getNamenodeNameServiceId(conf))) {
Configuration otherNnConf = HAUtil.getConfForOtherNode(conf); Configuration otherNnConf = HAUtil.getConfForOtherNode(conf);

View File

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

View File

@ -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();
}
}