HDFS-14013. Skip any credentials stored in HDFS when starting ZKFC. Contributed by Stephen O'Donnell

(cherry picked from commit c3b3b36dee)

 Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSZKFailoverController.java
This commit is contained in:
S O'Donnell 2021-03-01 11:36:41 +00:00
parent e4c638eb21
commit 707806092f
2 changed files with 48 additions and 11 deletions

View File

@ -31,11 +31,14 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.ha.ActiveStandbyElector.ActiveNotFoundException; import org.apache.hadoop.ha.ActiveStandbyElector.ActiveNotFoundException;
import org.apache.hadoop.ha.ActiveStandbyElector.ActiveStandbyElectorCallback; import org.apache.hadoop.ha.ActiveStandbyElector.ActiveStandbyElectorCallback;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo; import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
import org.apache.hadoop.ha.HAServiceProtocol.RequestSource; import org.apache.hadoop.ha.HAServiceProtocol.RequestSource;
import org.apache.hadoop.security.ProviderUtils;
import org.apache.hadoop.util.ZKUtil; import org.apache.hadoop.util.ZKUtil;
import org.apache.hadoop.util.ZKUtil.ZKAuthInfo; import org.apache.hadoop.util.ZKUtil.ZKAuthInfo;
import org.apache.hadoop.ha.HealthMonitor.State; import org.apache.hadoop.ha.HealthMonitor.State;
@ -340,8 +343,19 @@ public abstract class ZKFailoverController {
zkAcls = Ids.CREATOR_ALL_ACL; zkAcls = Ids.CREATOR_ALL_ACL;
} }
// Parse authentication from configuration. // Parse authentication from configuration. Exclude any Credential providers
List<ZKAuthInfo> zkAuths = SecurityUtil.getZKAuthInfos(conf, ZK_AUTH_KEY); // using the hdfs scheme to avoid a circular dependency. As HDFS is likely
// not started when ZKFC is started, we cannot read the credentials from it.
Configuration c = conf;
try {
c = ProviderUtils.excludeIncompatibleCredentialProviders(
conf, FileSystem.getFileSystemClass("hdfs", conf));
} catch (UnsupportedFileSystemException e) {
// Should not happen in a real cluster, as the hdfs FS will always be
// present. Inside tests, the hdfs filesystem will not be present
LOG.debug("No filesystem found for the hdfs scheme", e);
}
List<ZKAuthInfo> zkAuths = SecurityUtil.getZKAuthInfos(c, ZK_AUTH_KEY);
// Sanity check configuration. // Sanity check configuration.
Preconditions.checkArgument(zkQuorum != null, Preconditions.checkArgument(zkQuorum != null,

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hdfs.server.namenode.MockNameNodeResourceChecker;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil; import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
import org.apache.hadoop.net.ServerSocketUtil; import org.apache.hadoop.net.ServerSocketUtil;
import org.apache.hadoop.security.alias.CredentialProviderFactory;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.MultithreadedTestUtil.TestContext; import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread; import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread;
@ -86,7 +87,9 @@ public class TestDFSZKFailoverController extends ClientBaseWithFixes {
ServerSocketUtil.getPort(10023, 100)); ServerSocketUtil.getPort(10023, 100));
conf.setInt(DFSConfigKeys.DFS_HA_ZKFC_PORT_KEY + ".ns1.nn2", conf.setInt(DFSConfigKeys.DFS_HA_ZKFC_PORT_KEY + ".ns1.nn2",
ServerSocketUtil.getPort(10024, 100)); ServerSocketUtil.getPort(10024, 100));
}
private void startCluster() throws Exception {
// prefer non-ephemeral port to avoid port collision on restartNameNode // prefer non-ephemeral port to avoid port collision on restartNameNode
MiniDFSNNTopology topology = new MiniDFSNNTopology() MiniDFSNNTopology topology = new MiniDFSNNTopology()
.addNameservice(new MiniDFSNNTopology.NSConf("ns1") .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
@ -140,11 +143,26 @@ public class TestDFSZKFailoverController extends ClientBaseWithFixes {
} }
} }
@Test(timeout=60000)
/**
* Ensure the cluster simply starts with a hdfs jceks credential provider
* configured. HDFS-14013.
*/
public void testZFFCStartsWithCredentialProviderReferencingHDFS()
throws Exception{
// Create a provider path on HDFS
conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
"jceks://hdfs/tmp/test.jceks");
//
startCluster();
}
/** /**
* Test that thread dump is captured after NN state changes. * Test that thread dump is captured after NN state changes.
*/ */
@Test(timeout=60000) @Test(timeout=60000)
public void testThreadDumpCaptureAfterNNStateChange() throws Exception { public void testThreadDumpCaptureAfterNNStateChange() throws Exception {
startCluster();
MockNameNodeResourceChecker mockResourceChecker = MockNameNodeResourceChecker mockResourceChecker =
new MockNameNodeResourceChecker(conf); new MockNameNodeResourceChecker(conf);
mockResourceChecker.setResourcesAvailable(false); mockResourceChecker.setResourcesAvailable(false);
@ -162,6 +180,7 @@ public class TestDFSZKFailoverController extends ClientBaseWithFixes {
*/ */
@Test(timeout=60000) @Test(timeout=60000)
public void testFailoverAndBackOnNNShutdown() throws Exception { public void testFailoverAndBackOnNNShutdown() throws Exception {
startCluster();
Path p1 = new Path("/dir1"); Path p1 = new Path("/dir1");
Path p2 = new Path("/dir2"); Path p2 = new Path("/dir2");
@ -194,6 +213,7 @@ public class TestDFSZKFailoverController extends ClientBaseWithFixes {
@Test(timeout=30000) @Test(timeout=30000)
public void testManualFailover() throws Exception { public void testManualFailover() throws Exception {
startCluster();
thr2.zkfc.getLocalTarget().getZKFCProxy(conf, 15000).gracefulFailover(); thr2.zkfc.getLocalTarget().getZKFCProxy(conf, 15000).gracefulFailover();
waitForHAState(0, HAServiceState.STANDBY); waitForHAState(0, HAServiceState.STANDBY);
waitForHAState(1, HAServiceState.ACTIVE); waitForHAState(1, HAServiceState.ACTIVE);
@ -205,6 +225,7 @@ public class TestDFSZKFailoverController extends ClientBaseWithFixes {
@Test(timeout=30000) @Test(timeout=30000)
public void testWithoutBindAddressSet() throws Exception { public void testWithoutBindAddressSet() throws Exception {
startCluster();
DFSZKFailoverController zkfc = DFSZKFailoverController.create( DFSZKFailoverController zkfc = DFSZKFailoverController.create(
conf); conf);
@ -215,6 +236,7 @@ public class TestDFSZKFailoverController extends ClientBaseWithFixes {
@Test(timeout=30000) @Test(timeout=30000)
public void testWithBindAddressSet() throws Exception { public void testWithBindAddressSet() throws Exception {
startCluster();
conf.set(DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY, WILDCARD_ADDRESS); conf.set(DFS_NAMENODE_SERVICE_RPC_BIND_HOST_KEY, WILDCARD_ADDRESS);
DFSZKFailoverController zkfc = DFSZKFailoverController.create( DFSZKFailoverController zkfc = DFSZKFailoverController.create(
conf); conf);
@ -226,6 +248,7 @@ public class TestDFSZKFailoverController extends ClientBaseWithFixes {
@Test(timeout=30000) @Test(timeout=30000)
public void testManualFailoverWithDFSHAAdmin() throws Exception { public void testManualFailoverWithDFSHAAdmin() throws Exception {
startCluster();
DFSHAAdmin tool = new DFSHAAdmin(); DFSHAAdmin tool = new DFSHAAdmin();
tool.setConf(conf); tool.setConf(conf);
assertEquals(0, assertEquals(0,