HADOOP-12453. Support decoding KMS Delegation Token with its own Identifier. Contributed by Xiaoyu Yao.

This commit is contained in:
Xiao Chen 2016-11-03 13:09:03 -07:00
parent 20c4d8efab
commit 7154a20bcb
5 changed files with 79 additions and 12 deletions

View File

@ -98,8 +98,8 @@ public class KMSClientProvider extends KeyProvider implements CryptoExtension,
private static final String ANONYMOUS_REQUESTS_DISALLOWED = "Anonymous requests are disallowed";
public static final String TOKEN_KIND_STR = "kms-dt";
public static final Text TOKEN_KIND = new Text(TOKEN_KIND_STR);
public static final String TOKEN_KIND_STR = KMSDelegationToken.TOKEN_KIND_STR;
public static final Text TOKEN_KIND = KMSDelegationToken.TOKEN_KIND;
public static final String SCHEME_NAME = "kms";

View File

@ -0,0 +1,52 @@
/**
* 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.crypto.key.kms;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier;
/**
* Holder class for KMS delegation tokens.
*/
@InterfaceAudience.Private
public final class KMSDelegationToken {
public static final String TOKEN_KIND_STR = "kms-dt";
public static final Text TOKEN_KIND = new Text(TOKEN_KIND_STR);
// Utility class is not supposed to be instantiated.
private KMSDelegationToken() {
}
/**
* DelegationTokenIdentifier used for the KMS.
*/
public static class KMSDelegationTokenIdentifier
extends DelegationTokenIdentifier {
public KMSDelegationTokenIdentifier() {
super(TOKEN_KIND);
}
@Override
public Text getKind() {
return TOKEN_KIND;
}
}
}

View File

@ -0,0 +1,14 @@
#
# Licensed 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.
#
org.apache.hadoop.crypto.key.kms.KMSDelegationToken$KMSDelegationTokenIdentifier

View File

@ -19,7 +19,7 @@ package org.apache.hadoop.crypto.key.kms.server;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
import org.apache.hadoop.crypto.key.kms.KMSDelegationToken;
import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter;
@ -72,7 +72,7 @@ public class KMSAuthenticationFilter
KerberosDelegationTokenAuthenticationHandler.class.getName());
}
props.setProperty(DelegationTokenAuthenticationHandler.TOKEN_KIND,
KMSClientProvider.TOKEN_KIND_STR);
KMSDelegationToken.TOKEN_KIND_STR);
return props;
}

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.CryptoExtension;
import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
import org.apache.hadoop.crypto.key.kms.KMSDelegationToken;
import org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.Path;
@ -1804,13 +1805,13 @@ public class TestKMS {
InetSocketAddress kmsAddr =
new InetSocketAddress(getKMSUrl().getHost(),
getKMSUrl().getPort());
Assert.assertEquals(KMSClientProvider.TOKEN_KIND,
Assert.assertEquals(KMSDelegationToken.TOKEN_KIND,
credentials.getToken(SecurityUtil.buildTokenService(kmsAddr)).
getKind());
// Test non-renewer user cannot renew.
for (Token<?> token : tokens) {
if (!(token.getKind().equals(KMSClientProvider.TOKEN_KIND))) {
if (!(token.getKind().equals(KMSDelegationToken.TOKEN_KIND))) {
LOG.info("Skipping token {}", token);
continue;
}
@ -1843,7 +1844,7 @@ public class TestKMS {
boolean renewed = false;
for (Token<?> token : tokens) {
if (!(token.getKind()
.equals(KMSClientProvider.TOKEN_KIND))) {
.equals(KMSDelegationToken.TOKEN_KIND))) {
LOG.info("Skipping token {}", token);
continue;
}
@ -1863,7 +1864,7 @@ public class TestKMS {
// test delegation token cancellation
for (Token<?> token : tokens) {
if (!(token.getKind()
.equals(KMSClientProvider.TOKEN_KIND))) {
.equals(KMSDelegationToken.TOKEN_KIND))) {
LOG.info("Skipping token {}", token);
continue;
}
@ -1927,7 +1928,7 @@ public class TestKMS {
final Credentials credentials = new Credentials();
kpdte.addDelegationTokens("client", credentials);
Assert.assertEquals(1, credentials.getAllTokens().size());
Assert.assertEquals(KMSClientProvider.TOKEN_KIND, credentials.
Assert.assertEquals(KMSDelegationToken.TOKEN_KIND, credentials.
getToken(SecurityUtil.buildTokenService(kmsAddr)).getKind());
UserGroupInformation.getCurrentUser().addCredentials(credentials);
LOG.info("Added kms dt to credentials: {}", UserGroupInformation.
@ -1971,14 +1972,14 @@ public class TestKMS {
final Credentials newCreds = new Credentials();
kpdte.addDelegationTokens("client", newCreds);
Assert.assertEquals(1, newCreds.getAllTokens().size());
Assert.assertEquals(KMSClientProvider.TOKEN_KIND,
Assert.assertEquals(KMSDelegationToken.TOKEN_KIND,
newCreds.getToken(SecurityUtil.buildTokenService(kmsAddr)).
getKind());
// Using job 1's DT should fail.
final Credentials oldCreds = new Credentials();
for (Token<?> token : job1Token) {
if (token.getKind().equals(KMSClientProvider.TOKEN_KIND)) {
if (token.getKind().equals(KMSDelegationToken.TOKEN_KIND)) {
oldCreds
.addToken(SecurityUtil.buildTokenService(kmsAddr), token);
}
@ -1995,7 +1996,7 @@ public class TestKMS {
// Using the new DT should succeed.
Assert.assertEquals(1, newCreds.getAllTokens().size());
Assert.assertEquals(KMSClientProvider.TOKEN_KIND,
Assert.assertEquals(KMSDelegationToken.TOKEN_KIND,
newCreds.getToken(SecurityUtil.buildTokenService(kmsAddr)).
getKind());
UserGroupInformation.getCurrentUser().addCredentials(newCreds);