HBASE-13244 Test delegation token generation with kerberos enabled

This commit is contained in:
zhangduo 2015-03-15 14:51:57 +08:00
parent 01bc979ea2
commit 0505b7941e
2 changed files with 162 additions and 12 deletions

View File

@ -17,19 +17,18 @@
*/
package org.apache.hadoop.hbase.security;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.hbase.HBaseConfiguration;
import com.google.common.base.Strings;
@InterfaceAudience.Private
class HBaseKerberosUtils {
public class HBaseKerberosUtils {
public static final String KRB_PRINCIPAL = "hbase.regionserver.kerberos.principal";
public static final String MASTER_KRB_PRINCIPAL = "hbase.master.kerberos.principal";
public static final String KRB_KEYTAB_FILE = "hbase.regionserver.keytab.file";
static boolean isKerberosPropertySetted() {
public static boolean isKerberosPropertySetted() {
String krbPrincipal = System.getProperty(KRB_PRINCIPAL);
String krbKeytab = System.getProperty(KRB_KEYTAB_FILE);
if (Strings.isNullOrEmpty(krbPrincipal) || Strings.isNullOrEmpty(krbKeytab)) {
@ -38,27 +37,27 @@ class HBaseKerberosUtils {
return true;
}
static void setPrincipalForTesting(String principal) {
public static void setPrincipalForTesting(String principal) {
setSystemProperty(KRB_PRINCIPAL, principal);
}
static void setKeytabFileForTesting(String keytabFile) {
public static void setKeytabFileForTesting(String keytabFile) {
setSystemProperty(KRB_KEYTAB_FILE, keytabFile);
}
static void setSystemProperty(String propertyName, String propertyValue) {
public static void setSystemProperty(String propertyName, String propertyValue) {
System.setProperty(propertyName, propertyValue);
}
static String getKeytabFileForTesting() {
public static String getKeytabFileForTesting() {
return System.getProperty(KRB_KEYTAB_FILE);
}
static String getPrincipalForTesting() {
public static String getPrincipalForTesting() {
return System.getProperty(KRB_PRINCIPAL);
}
static Configuration getConfigurationWoPrincipal() {
public static Configuration getConfigurationWoPrincipal() {
Configuration conf = HBaseConfiguration.create();
conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
conf.set("hbase.security.authentication", "kerberos");
@ -66,13 +65,18 @@ class HBaseKerberosUtils {
return conf;
}
static Configuration getSecuredConfiguration() {
public static Configuration getSecuredConfiguration() {
Configuration conf = HBaseConfiguration.create();
setSecuredConfiguration(conf);
return conf;
}
public static void setSecuredConfiguration(Configuration conf) {
conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
conf.set("hbase.security.authentication", "kerberos");
conf.setBoolean("hbase.security.authorization", true);
conf.set(KRB_KEYTAB_FILE, System.getProperty(KRB_KEYTAB_FILE));
conf.set(KRB_PRINCIPAL, System.getProperty(KRB_PRINCIPAL));
return conf;
conf.set(MASTER_KRB_PRINCIPAL, System.getProperty(KRB_PRINCIPAL));
}
}

View File

@ -0,0 +1,146 @@
/**
* 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.hbase.security.token;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.IOException;
import java.net.InetAddress;
import java.util.Properties;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.LocalHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.ipc.AsyncRpcClient;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.ipc.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcClientFactory;
import org.apache.hadoop.hbase.ipc.RpcClientImpl;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.GetAuthenticationTokenRequest;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.WhoAmIRequest;
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.WhoAmIResponse;
import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hbase.security.HBaseKerberosUtils;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.SecurityTests;
import org.apache.hadoop.minikdc.MiniKdc;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import com.google.protobuf.ServiceException;
@Category({ SecurityTests.class, MediumTests.class })
public class TestGenerateDelegationToken {
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static LocalHBaseCluster CLUSTER;
private static final File KEYTAB_FILE = new File(TEST_UTIL.getDataTestDir("keytab").toUri()
.getPath());
private static MiniKdc KDC;
private static String HOST;
private static String USERNAME = System.getProperty("user.name");
private static String PRINCIPAL;
@BeforeClass
public static void setUp() throws Exception {
Properties conf = MiniKdc.createConf();
conf.put(MiniKdc.DEBUG, true);
KDC = new MiniKdc(conf, new File(TEST_UTIL.getDataTestDir("kdc").toUri().getPath()));
KDC.start();
HOST = InetAddress.getLocalHost().getHostName();
PRINCIPAL = USERNAME + "/" + HOST;
KDC.createPrincipal(KEYTAB_FILE, PRINCIPAL);
TEST_UTIL.getConfiguration().setBoolean("ipc.client.fallback-to-simple-auth-allowed", true);
TEST_UTIL.startMiniZKCluster();
TEST_UTIL.startMiniDFSCluster(1);
HBaseKerberosUtils.setKeytabFileForTesting(KEYTAB_FILE.getAbsolutePath());
HBaseKerberosUtils.setPrincipalForTesting(PRINCIPAL + "@" + KDC.getRealm());
HBaseKerberosUtils.setSecuredConfiguration(TEST_UTIL.getConfiguration());
UserGroupInformation.setConfiguration(TEST_UTIL.getConfiguration());
TEST_UTIL.getConfiguration().setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
TokenProvider.class.getName());
CLUSTER = new LocalHBaseCluster(TEST_UTIL.getConfiguration(), 1);
CLUSTER.startup();
}
@AfterClass
public static void tearDown() throws Exception {
if (CLUSTER != null) {
CLUSTER.shutdown();
}
CLUSTER.join();
if (KDC != null) {
KDC.stop();
}
TEST_UTIL.shutdownMiniCluster();
}
private void testTokenAuth(Class<? extends RpcClient> rpcImplClass) throws IOException,
ServiceException {
TEST_UTIL.getConfiguration().set(RpcClientFactory.CUSTOM_RPC_CLIENT_IMPL_CONF_KEY,
rpcImplClass.getName());
try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
Table table = conn.getTable(TableName.META_TABLE_NAME)) {
CoprocessorRpcChannel rpcChannel = table.coprocessorService(HConstants.EMPTY_START_ROW);
AuthenticationProtos.AuthenticationService.BlockingInterface service =
AuthenticationProtos.AuthenticationService.newBlockingStub(rpcChannel);
WhoAmIResponse response = service.whoAmI(null, WhoAmIRequest.getDefaultInstance());
assertEquals(USERNAME, response.getUsername());
assertEquals(AuthenticationMethod.TOKEN.name(), response.getAuthMethod());
try {
service.getAuthenticationToken(null, GetAuthenticationTokenRequest.getDefaultInstance());
} catch (ServiceException e) {
AccessDeniedException exc = (AccessDeniedException) ProtobufUtil.getRemoteException(e);
assertTrue(exc.getMessage().contains(
"Token generation only allowed for Kerberos authenticated clients"));
}
}
}
@Test
public void test() throws Exception {
try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
Token<? extends TokenIdentifier> token = TokenUtil.obtainToken(conn);
UserGroupInformation.getCurrentUser().addToken(token);
testTokenAuth(RpcClientImpl.class);
testTokenAuth(AsyncRpcClient.class);
}
}
}