Prechádzať zdrojové kódy

HADOOP-10769. Create KeyProvider extension to handle delegation tokens. Contributed by Arun Suresh.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1608286 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 11 rokov pred
rodič
commit
ed642b5d8d

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -174,6 +174,9 @@ Trunk (Unreleased)
     HADOOP-10757. KeyProvider KeyVersion should provide the key name. 
     (asuresh via tucu)
 
+    HADOOP-10769. Create KeyProvider extension to handle delegation tokens.
+    (Arun Suresh via atm)
+
   BUG FIXES
 
     HADOOP-9451. Fault single-layer config if node group topology is enabled.

+ 111 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderDelegationTokenExtension.java

@@ -0,0 +1,111 @@
+/**
+ * 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;
+
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.token.Token;
+
+/**
+ * A KeyProvider extension with the ability to add a renewer's Delegation 
+ * Tokens to the provided Credentials.
+ */
+public class KeyProviderDelegationTokenExtension extends
+    KeyProviderExtension
+    <KeyProviderDelegationTokenExtension.DelegationTokenExtension> {
+  
+  private static DelegationTokenExtension DEFAULT_EXTENSION = 
+      new DefaultDelegationTokenExtension();
+
+  /**
+   * DelegationTokenExtension is a type of Extension that exposes methods to 
+   * needed to work with Delegation Tokens.
+   */  
+  public interface DelegationTokenExtension extends 
+    KeyProviderExtension.Extension {
+    
+    /**
+     * The implementer of this class will take a renewer and add all
+     * delegation tokens associated with the renewer to the 
+     * <code>Credentials</code> object if it is not already present, 
+     * @param renewer the user allowed to renew the delegation tokens
+     * @param credentials cache in which to add new delegation tokens
+     * @return list of new delegation tokens
+     */
+    public Token<?>[] addDelegationTokens(final String renewer, 
+        Credentials credentials);
+  }
+  
+  /**
+   * Default implementation of {@link DelegationTokenExtension} that
+   * implements the method as a no-op.
+   */
+  private static class DefaultDelegationTokenExtension implements 
+    DelegationTokenExtension {    
+    
+    @Override
+    public Token<?>[] addDelegationTokens(String renewer,
+        Credentials credentials) {
+      return null;
+    }
+    
+  }
+
+  private KeyProviderDelegationTokenExtension(KeyProvider keyProvider,
+      DelegationTokenExtension extensions) {
+    super(keyProvider, extensions);
+  }
+  
+  /**
+   * Passes the renewer and Credentials object to the underlying 
+   * {@link DelegationTokenExtension} 
+   * @param renewer the user allowed to renew the delegation tokens
+   * @param credentials cache in which to add new delegation tokens
+   * @return list of new delegation tokens
+   */
+  public Token<?>[] addDelegationTokens(final String renewer, 
+      Credentials credentials) {
+    return getExtension().addDelegationTokens(renewer, credentials);
+  }
+  
+  /**
+   * Creates a <code>KeyProviderDelegationTokenExtension</code> using a given 
+   * {@link KeyProvider}.
+   * <p/>
+   * If the given <code>KeyProvider</code> implements the 
+   * {@link DelegationTokenExtension} interface the <code>KeyProvider</code> 
+   * itself will provide the extension functionality, otherwise a default 
+   * extension implementation will be used.
+   * 
+   * @param keyProvider <code>KeyProvider</code> to use to create the 
+   * <code>KeyProviderDelegationTokenExtension</code> extension.
+   * @return a <code>KeyProviderDelegationTokenExtension</code> instance 
+   * using the given <code>KeyProvider</code>.
+   */  
+  public static KeyProviderDelegationTokenExtension
+      createKeyProviderDelegationTokenExtension(KeyProvider keyProvider) {
+
+    DelegationTokenExtension delTokExtension =
+        (keyProvider instanceof DelegationTokenExtension) ?
+            (DelegationTokenExtension) keyProvider :
+            DEFAULT_EXTENSION;
+    return new KeyProviderDelegationTokenExtension(
+        keyProvider, delTokExtension);
+
+  }
+
+}

+ 66 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderDelegationTokenExtension.java

@@ -0,0 +1,66 @@
+/**
+ * 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;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.net.URI;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension.DelegationTokenExtension;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.token.Token;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestKeyProviderDelegationTokenExtension {
+  
+  public static abstract class MockKeyProvider extends
+      KeyProvider implements DelegationTokenExtension {
+  }
+  
+  @Test
+  public void testCreateExtension() throws Exception {
+    Configuration conf = new Configuration();
+    Credentials credentials = new Credentials();    
+    KeyProvider kp = 
+        new UserProvider.Factory().createProvider(new URI("user:///"), conf);
+    KeyProviderDelegationTokenExtension kpDTE1 = 
+        KeyProviderDelegationTokenExtension
+        .createKeyProviderDelegationTokenExtension(kp);
+    Assert.assertNotNull(kpDTE1);
+    // Default implementation should be a no-op and return null
+    Assert.assertNull(kpDTE1.addDelegationTokens("user", credentials));
+    
+    MockKeyProvider mock = mock(MockKeyProvider.class);
+    when(mock.addDelegationTokens("renewer", credentials)).thenReturn(
+        new Token<?>[] { new Token(null, null, new Text("kind"), new Text(
+            "service")) });
+    KeyProviderDelegationTokenExtension kpDTE2 =
+        KeyProviderDelegationTokenExtension
+        .createKeyProviderDelegationTokenExtension(mock);
+    Token<?>[] tokens = 
+        kpDTE2.addDelegationTokens("renewer", credentials);
+    Assert.assertNotNull(tokens);
+    Assert.assertEquals("kind", tokens[0].getKind().toString());
+    
+  }
+
+}