Browse Source

HDFS-14390. Provide kerberos support for AliasMap service used by Provided storage. Contributed by Ashvin Agrawal

Virajith Jalaparti 6 years ago
parent
commit
77170e70d1

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/AliasMapProtocolPB.java

@@ -18,18 +18,21 @@ package org.apache.hadoop.hdfs.protocolPB;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.proto.AliasMapProtocolProtos;
 import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.security.KerberosInfo;
 
 /**
  * Protocol between the Namenode and the Datanode to read the AliasMap
  * used for Provided storage.
- * TODO add Kerberos support
  */
 @ProtocolInfo(
     protocolName =
         "org.apache.hadoop.hdfs.server.aliasmap.AliasMapProtocol",
     protocolVersion = 1)
+@KerberosInfo(
+    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public interface AliasMapProtocolPB extends

+ 0 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/aliasmap/InMemoryLevelDBAliasMapServer.java

@@ -20,7 +20,6 @@ import com.google.protobuf.BlockingService;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configurable;
@@ -71,10 +70,6 @@ public class InMemoryLevelDBAliasMapServer implements InMemoryAliasMapProtocol,
   }
 
   public void start() throws IOException {
-    if (UserGroupInformation.isSecurityEnabled()) {
-      throw new UnsupportedOperationException("Unable to start "
-          + "InMemoryLevelDBAliasMapServer as security is enabled");
-    }
     RPC.setProtocolEngine(getConf(), AliasMapProtocolPB.class,
         ProtobufRpcEngine.class);
     AliasMapProtocolServerSideTranslatorPB aliasMapProtocolXlator =

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ProvidedStorageMap.java

@@ -220,6 +220,11 @@ public class ProvidedStorageMap {
     return providedDescriptor.chooseRandom();
   }
 
+  @VisibleForTesting
+  public BlockAliasMap getAliasMap() {
+    return aliasMap;
+  }
+
   /**
    * Builder used for creating {@link LocatedBlocks} when a block is provided.
    */

+ 1 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/InMemoryLevelDBAliasMapClient.java

@@ -28,7 +28,6 @@ import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMapProtocol;
 import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
 import org.apache.hadoop.hdfs.server.common.FileRegion;
 import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -150,10 +149,6 @@ public class InMemoryLevelDBAliasMapClient extends BlockAliasMap<FileRegion>
   }
 
   InMemoryLevelDBAliasMapClient() {
-    if (UserGroupInformation.isSecurityEnabled()) {
-      throw new UnsupportedOperationException("Unable to start "
-          + "InMemoryLevelDBAliasMapClient as security is enabled");
-    }
     aliasMaps = new ArrayList<>();
   }
 
@@ -176,7 +171,7 @@ public class InMemoryLevelDBAliasMapClient extends BlockAliasMap<FileRegion>
       }
     }
     throw new IOException(
-        "Unable to retrive InMemoryAliasMap for block pool id " + blockPoolID);
+        "Unable to retrieve InMemoryAliasMap for block pool id " + blockPoolID);
   }
 
   @Override

+ 59 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -21,6 +21,17 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_DEFAULT;
 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_JOURNALNODE_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_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY;
@@ -69,6 +80,8 @@ import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Multimap;
 import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
 import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.InMemoryLevelDBAliasMapClient;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -3352,6 +3365,52 @@ public class MiniDFSCluster implements AutoCloseable {
         tempDirectory.getAbsolutePath());
   }
 
+  /**
+   * Updates configuration objects with keys required to setup a secure
+   * {@link org.apache.hadoop.minikdc.MiniKdc} based DFS cluster.
+   * @param conf the configuration object to be updated
+   * @param userName username to be used in kerberos principal
+   * @param realm realm to be used in the kerberos principal
+   * @param keytab absolute path of the the keytab file
+   * @param keystoresDir absolute path of the keystore
+   * @param sslConfDir absolute path of the ssl conf dir
+   * @throws Exception
+   */
+  public static void setupKerberosConfiguration(Configuration conf,
+      String userName, String realm, String keytab, String keystoresDir,
+      String sslConfDir) throws Exception {
+    // Windows will not reverse name lookup "127.0.0.1" to "localhost".
+    String krbInstance = Path.WINDOWS ? "127.0.0.1" : "localhost";
+    String hdfsPrincipal = userName + "/" + krbInstance + "@" + realm;
+    String spnegoPrincipal = "HTTP/" + krbInstance + "@" + realm;
+
+    conf.set(DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY, hdfsPrincipal);
+    conf.set(DFS_NAMENODE_KEYTAB_FILE_KEY, keytab);
+    conf.set(DFS_DATANODE_KERBEROS_PRINCIPAL_KEY, hdfsPrincipal);
+    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.set(DFS_JOURNALNODE_HTTPS_ADDRESS_KEY, "localhost:0");
+    conf.setInt(IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY, 10);
+
+    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());
+
+    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());
+  }
+
   @Override
   public void close() {
     shutdown();

+ 155 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/aliasmap/TestSecureAliasMap.java

@@ -0,0 +1,155 @@
+/**
+ * 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.server.aliasmap;
+
+import org.apache.hadoop.fs.FileSystem;
+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.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.qjournal.TestSecureNNWithQJM;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Properties;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test DN & NN communication in secured hdfs with alias map.
+ */
+public class TestSecureAliasMap {
+  private static HdfsConfiguration baseConf;
+  private static File baseDir;
+  private static MiniKdc kdc;
+
+  private static String keystoresDir;
+  private static String sslConfDir;
+  private MiniDFSCluster cluster;
+  private HdfsConfiguration conf;
+  private FileSystem fs;
+
+  @BeforeClass
+  public static void init() throws Exception {
+    baseDir =
+        GenericTestUtils.getTestDir(TestSecureAliasMap.class.getSimpleName());
+    FileUtil.fullyDelete(baseDir);
+    assertTrue(baseDir.mkdirs());
+
+    Properties kdcConf = MiniKdc.createConf();
+    kdc = new MiniKdc(kdcConf, baseDir);
+    kdc.start();
+
+    baseConf = new HdfsConfiguration();
+    SecurityUtil.setAuthenticationMethod(
+        UserGroupInformation.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);
+
+    keystoresDir = baseDir.getAbsolutePath();
+    sslConfDir = KeyStoreTestUtil.getClasspathDir(TestSecureNNWithQJM.class);
+    MiniDFSCluster.setupKerberosConfiguration(baseConf, userName,
+        kdc.getRealm(), keytab, keystoresDir, sslConfDir);
+  }
+
+  @AfterClass
+  public static void destroy() throws Exception {
+    if (kdc != null) {
+      kdc.stop();
+    }
+    FileUtil.fullyDelete(baseDir);
+    KeyStoreTestUtil.cleanupSSLConfig(keystoresDir, sslConfDir);
+  }
+
+  @After
+  public void shutdown() throws IOException {
+    IOUtils.cleanupWithLogger(null, fs);
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }
+  }
+
+  @Test
+  public void testSecureConnectionToAliasMap() throws Exception {
+    conf = new HdfsConfiguration(baseConf);
+    MiniDFSCluster.setupNamenodeProvidedConfiguration(conf);
+    conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
+        "127.0.0.1:" + NetUtils.getFreeSocketPort());
+
+    int numNodes = 1;
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numNodes)
+        .storageTypes(
+            new StorageType[] {StorageType.DISK, StorageType.PROVIDED})
+        .build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+
+    FSNamesystem namesystem = cluster.getNamesystem();
+    BlockManager blockManager = namesystem.getBlockManager();
+    DataNode dn = cluster.getDataNodes().get(0);
+
+    FsDatasetSpi.FsVolumeReferences volumes =
+        dn.getFSDataset().getFsVolumeReferences();
+    FsVolumeSpi providedVolume = null;
+    for (FsVolumeSpi volume : volumes) {
+      if (volume.getStorageType().equals(StorageType.PROVIDED)) {
+        providedVolume = volume;
+        break;
+      }
+    }
+
+    String[] bps = providedVolume.getBlockPoolList();
+    assertEquals("Missing provided volume", 1, bps.length);
+
+    BlockAliasMap aliasMap = blockManager.getProvidedStorageMap().getAliasMap();
+    BlockAliasMap.Reader reader = aliasMap.getReader(null, bps[0]);
+    assertNotNull("Failed to create blockAliasMap reader", reader);
+  }
+}