|
@@ -17,17 +17,40 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.sps;
|
|
|
|
|
|
+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_CLIENT_HTTPS_KEYSTORE_RESOURCE_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_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_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SPS_ADDRESS_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SPS_KERBEROS_PRINCIPAL_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SPS_KEYTAB_FILE_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
|
|
|
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
|
|
|
+
|
|
|
+import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.net.URI;
|
|
|
+import java.security.PrivilegedExceptionAction;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collection;
|
|
|
import java.util.List;
|
|
|
+import java.util.Properties;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.StorageType;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
+import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.StoragePolicySatisfierMode;
|
|
@@ -39,8 +62,17 @@ import org.apache.hadoop.hdfs.server.namenode.sps.FileIdCollector;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.sps.SPSService;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.sps.TestStoragePolicySatisfier;
|
|
|
+import org.apache.hadoop.http.HttpConfig;
|
|
|
+import org.apache.hadoop.minikdc.MiniKdc;
|
|
|
+import org.apache.hadoop.security.SecurityUtil;
|
|
|
+import org.apache.hadoop.security.UserGroupInformation;
|
|
|
+import org.apache.hadoop.security.authentication.util.KerberosName;
|
|
|
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
+import org.junit.After;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Ignore;
|
|
|
+import org.junit.Test;
|
|
|
|
|
|
/**
|
|
|
* Tests the external sps service plugins.
|
|
@@ -52,6 +84,18 @@ public class TestExternalStoragePolicySatisfier
|
|
|
{StorageType.DISK, StorageType.DISK},
|
|
|
{StorageType.DISK, StorageType.DISK}};
|
|
|
private NameNodeConnector nnc;
|
|
|
+ private File keytabFile;
|
|
|
+ private String principal;
|
|
|
+ private MiniKdc kdc;
|
|
|
+ private File baseDir;
|
|
|
+
|
|
|
+ @After
|
|
|
+ public void destroy() throws Exception {
|
|
|
+ if (kdc != null) {
|
|
|
+ kdc.stop();
|
|
|
+ FileUtil.fullyDelete(baseDir);
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
@Override
|
|
|
public void setUp() {
|
|
@@ -178,20 +222,157 @@ public class TestExternalStoragePolicySatisfier
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private void initSecureConf(Configuration conf) throws Exception {
|
|
|
+ String username = "externalSPS";
|
|
|
+ baseDir = GenericTestUtils
|
|
|
+ .getTestDir(TestExternalStoragePolicySatisfier.class.getSimpleName());
|
|
|
+ FileUtil.fullyDelete(baseDir);
|
|
|
+ Assert.assertTrue(baseDir.mkdirs());
|
|
|
+
|
|
|
+ Properties kdcConf = MiniKdc.createConf();
|
|
|
+ kdc = new MiniKdc(kdcConf, baseDir);
|
|
|
+ kdc.start();
|
|
|
+
|
|
|
+ SecurityUtil.setAuthenticationMethod(
|
|
|
+ UserGroupInformation.AuthenticationMethod.KERBEROS, conf);
|
|
|
+ UserGroupInformation.setConfiguration(conf);
|
|
|
+ KerberosName.resetDefaultRealm();
|
|
|
+ Assert.assertTrue("Expected configuration to enable security",
|
|
|
+ UserGroupInformation.isSecurityEnabled());
|
|
|
+
|
|
|
+ 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";
|
|
|
+ principal = username + "/" + krbInstance + "@" + kdc.getRealm();
|
|
|
+ String spnegoPrincipal = "HTTP/" + krbInstance + "@" + kdc.getRealm();
|
|
|
+ kdc.createPrincipal(keytabFile, username, username + "/" + krbInstance,
|
|
|
+ "HTTP/" + krbInstance);
|
|
|
+
|
|
|
+ conf.set(DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, principal);
|
|
|
+ conf.set(DFS_NAMENODE_KEYTAB_FILE_KEY, keytab);
|
|
|
+ conf.set(DFS_DATANODE_KERBEROS_PRINCIPAL_KEY, principal);
|
|
|
+ conf.set(DFS_DATANODE_KEYTAB_FILE_KEY, keytab);
|
|
|
+ conf.set(DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY, spnegoPrincipal);
|
|
|
+ conf.setBoolean(DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
|
|
|
+ conf.set(DFS_DATA_TRANSFER_PROTECTION_KEY, "authentication");
|
|
|
+ conf.set(DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTPS_ONLY.name());
|
|
|
+ conf.set(DFS_NAMENODE_HTTPS_ADDRESS_KEY, "localhost:0");
|
|
|
+ conf.set(DFS_DATANODE_HTTPS_ADDRESS_KEY, "localhost:0");
|
|
|
+ conf.setInt(IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY, 10);
|
|
|
+
|
|
|
+ conf.set(DFS_SPS_ADDRESS_KEY, "localhost:0");
|
|
|
+ conf.set(DFS_SPS_KEYTAB_FILE_KEY, keytab);
|
|
|
+ conf.set(DFS_SPS_KERBEROS_PRINCIPAL_KEY, principal);
|
|
|
+
|
|
|
+ String keystoresDir = baseDir.getAbsolutePath();
|
|
|
+ String sslConfDir = KeyStoreTestUtil
|
|
|
+ .getClasspathDir(TestExternalStoragePolicySatisfier.class);
|
|
|
+ KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
|
|
|
+
|
|
|
+ conf.set(DFS_CLIENT_HTTPS_KEYSTORE_RESOURCE_KEY,
|
|
|
+ KeyStoreTestUtil.getClientSSLConfigFileName());
|
|
|
+ conf.set(DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY,
|
|
|
+ KeyStoreTestUtil.getServerSSLConfigFileName());
|
|
|
+
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, DEFAULT_BLOCK_SIZE);
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
- * This test need not run as external scan is not a batch based scanning right
|
|
|
- * now.
|
|
|
+ * Test SPS runs fine when logging in with a keytab in kerberized env. Reusing
|
|
|
+ * testWhenStoragePolicySetToALLSSD here for basic functionality testing.
|
|
|
*/
|
|
|
- @Ignore("ExternalFileIdCollector is not batch based right now."
|
|
|
- + " So, ignoring it.")
|
|
|
- public void testBatchProcessingForSPSDirectory() throws Exception {
|
|
|
+ @Test(timeout = 300000)
|
|
|
+ public void testWithKeytabs() throws Exception {
|
|
|
+ try {
|
|
|
+ initSecureConf(getConf());
|
|
|
+ final UserGroupInformation ugi = UserGroupInformation
|
|
|
+ .loginUserFromKeytabAndReturnUGI(principal,
|
|
|
+ keytabFile.getAbsolutePath());
|
|
|
+ ugi.doAs(new PrivilegedExceptionAction<Void>() {
|
|
|
+ @Override
|
|
|
+ public Void run() throws Exception {
|
|
|
+ // verify that sps runs Ok.
|
|
|
+ testWhenStoragePolicySetToALLSSD();
|
|
|
+ // verify that UGI was logged in using keytab.
|
|
|
+ Assert.assertTrue(UserGroupInformation.isLoginKeytabBased());
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ });
|
|
|
+ } finally {
|
|
|
+ // Reset UGI so that other tests are not affected.
|
|
|
+ UserGroupInformation.reset();
|
|
|
+ UserGroupInformation.setConfiguration(new Configuration());
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Status won't be supported for external SPS, now. So, ignoring it.
|
|
|
+ * Test verifies that SPS call will throw exception if the call Q exceeds
|
|
|
+ * OutstandingQueueLimit value.
|
|
|
+ *
|
|
|
+ * @throws Exception
|
|
|
*/
|
|
|
- @Ignore("Status is not supported for external SPS. So, ignoring it.")
|
|
|
+ @Test(timeout = 300000)
|
|
|
+ public void testOutstandingQueueLimitExceeds() throws Exception {
|
|
|
+ try {
|
|
|
+ getConf().setInt(DFS_SPS_MAX_OUTSTANDING_PATHS_KEY, 3);
|
|
|
+ createCluster();
|
|
|
+ List<String> files = new ArrayList<>();
|
|
|
+ files.add(FILE);
|
|
|
+ DistributedFileSystem fs = getFS();
|
|
|
+ BlockManager blkMgr = getCluster().getNameNode().getNamesystem()
|
|
|
+ .getBlockManager();
|
|
|
+ SPSService spsService = blkMgr.getSPSService();
|
|
|
+ spsService.stopGracefully(); // stops SPS
|
|
|
+
|
|
|
+ // Creates 4 more files. Send all of them for satisfying the storage
|
|
|
+ // policy together.
|
|
|
+ for (int i = 0; i < 3; i++) {
|
|
|
+ String file1 = "/testOutstandingQueueLimitExceeds_" + i;
|
|
|
+ files.add(file1);
|
|
|
+ writeContent(file1);
|
|
|
+ fs.satisfyStoragePolicy(new Path(file1));
|
|
|
+ }
|
|
|
+ String fileExceeds = "/testOutstandingQueueLimitExceeds_" + 4;
|
|
|
+ files.add(fileExceeds);
|
|
|
+ writeContent(fileExceeds);
|
|
|
+ try {
|
|
|
+ fs.satisfyStoragePolicy(new Path(fileExceeds));
|
|
|
+ Assert.fail("Should throw exception as it exceeds "
|
|
|
+ + "outstanding SPS call Q limit");
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ GenericTestUtils.assertExceptionContains(
|
|
|
+ "Outstanding satisfier queue limit: 3 exceeded, try later!", ioe);
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ shutdownCluster();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test verifies status check when Satisfier is not running inside namenode.
|
|
|
+ */
|
|
|
+ @Test(timeout = 90000)
|
|
|
public void testStoragePolicySatisfyPathStatus() throws Exception {
|
|
|
+ createCluster();
|
|
|
+ DistributedFileSystem fs = getFS();
|
|
|
+ try {
|
|
|
+ fs.getClient().checkStoragePolicySatisfyPathStatus(FILE);
|
|
|
+ Assert.fail("Should throw exception as SPS is not running inside NN!");
|
|
|
+ } catch (IOException e) {
|
|
|
+ GenericTestUtils.assertExceptionContains("Satisfier is not running"
|
|
|
+ + " inside namenode, so status can't be returned.", e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * This test need not run as external scan is not a batch based scanning right
|
|
|
+ * now.
|
|
|
+ */
|
|
|
+ @Ignore("ExternalFileIdCollector is not batch based right now."
|
|
|
+ + " So, ignoring it.")
|
|
|
+ public void testBatchProcessingForSPSDirectory() throws Exception {
|
|
|
}
|
|
|
|
|
|
/**
|