You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by wa...@apache.org on 2014/08/18 20:41:35 UTC

svn commit: r1618700 [2/2] - in /hadoop/common/branches/fs-encryption/hadoop-common-project: hadoop-common/ hadoop-common/src/main/java/ hadoop-common/src/main/java/org/apache/hadoop/crypto/key/ hadoop-common/src/main/java/org/apache/hadoop/crypto/key/...

Modified: hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/site/apt/ServiceLevelAuth.apt.vm
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/site/apt/ServiceLevelAuth.apt.vm?rev=1618700&r1=1618699&r2=1618700&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/site/apt/ServiceLevelAuth.apt.vm (original)
+++ hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/site/apt/ServiceLevelAuth.apt.vm Mon Aug 18 18:41:31 2014
@@ -110,6 +110,27 @@ security.ha.service.protocol.acl      | 
    <<<security.service.authorization.default.acl>>> is applied. If 
    <<<security.service.authorization.default.acl>>> is not defined, <<<*>>>  is applied.
 
+ ** Blocked Access Control Lists
+
+   In some cases, it is required to specify blocked access control list for a service. This specifies
+   the list of users and groups who are not authorized to access the service. The format of
+   the blocked access control list is same as that of access control list. The blocked access
+   control list can be specified via <<<${HADOOP_CONF_DIR}/hadoop-policy.xml>>>. The property name
+   is derived by suffixing with ".blocked".
+
+   Example: The property name of blocked access control list for <<<security.client.protocol.acl>>
+   will be <<<security.client.protocol.acl.blocked>>>
+
+   For a service, it is possible to specify both an access control list and a blocked control
+   list. A user is authorized to access the service if the user is in the access control and not in
+   the blocked access control list.
+
+   If blocked access control list is not defined for a service, the value of
+   <<<security.service.authorization.default.acl.blocked>>> is applied. If
+   <<<security.service.authorization.default.acl.blocked>>> is not defined,
+   empty blocked access control list is applied.
+
+
 ** Refreshing Service Level Authorization Configuration
 
    The service-level authorization configuration for the NameNode and

Modified: hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java?rev=1618700&r1=1618699&r2=1618700&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java Mon Aug 18 18:41:31 2014
@@ -583,14 +583,14 @@ public class TestRPC {
       }
       MetricsRecordBuilder rb = getMetrics(server.rpcMetrics.name());
       if (expectFailure) {
-        assertCounter("RpcAuthorizationFailures", 1, rb);
+        assertCounter("RpcAuthorizationFailures", 1L, rb);
       } else {
-        assertCounter("RpcAuthorizationSuccesses", 1, rb);
+        assertCounter("RpcAuthorizationSuccesses", 1L, rb);
       }
       //since we don't have authentication turned ON, we should see 
       // 0 for the authentication successes and 0 for failure
-      assertCounter("RpcAuthenticationFailures", 0, rb);
-      assertCounter("RpcAuthenticationSuccesses", 0, rb);
+      assertCounter("RpcAuthenticationFailures", 0L, rb);
+      assertCounter("RpcAuthenticationSuccesses", 0L, rb);
     }
   }
   

Modified: hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestServiceAuthorization.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestServiceAuthorization.java?rev=1618700&r1=1618699&r2=1618700&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestServiceAuthorization.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/authorize/TestServiceAuthorization.java Mon Aug 18 18:41:31 2014
@@ -18,16 +18,22 @@
 package org.apache.hadoop.security.authorize;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.ipc.TestRPC.TestProtocol;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.Test;
 
 public class TestServiceAuthorization {
 
   private static final String ACL_CONFIG = "test.protocol.acl";
   private static final String ACL_CONFIG1 = "test.protocol1.acl";
+  private static final String ADDRESS =  "0.0.0.0";
 
   public interface TestProtocol1 extends TestProtocol {};
 
@@ -64,4 +70,115 @@ public class TestServiceAuthorization {
     acl = serviceAuthorizationManager.getProtocolsAcls(TestProtocol1.class);
     assertEquals("user2 group2", acl.getAclString());
   }
+
+  @Test
+  public void testBlockedAcl() throws UnknownHostException {
+    UserGroupInformation drwho =
+        UserGroupInformation.createUserForTesting("drwho@EXAMPLE.COM",
+            new String[] { "group1", "group2" });
+
+    ServiceAuthorizationManager serviceAuthorizationManager =
+        new ServiceAuthorizationManager();
+    Configuration conf = new Configuration ();
+
+    //test without setting a blocked acl
+    conf.set(ACL_CONFIG, "user1 group1");
+    serviceAuthorizationManager.refresh(conf, new TestPolicyProvider());
+    try {
+      serviceAuthorizationManager.authorize(drwho, TestProtocol.class, conf,
+          InetAddress.getByName(ADDRESS));
+    } catch (AuthorizationException e) {
+      fail();
+    }
+    //now set a blocked acl with another user and another group
+    conf.set(ACL_CONFIG + ServiceAuthorizationManager.BLOCKED, "drwho2 group3");
+    serviceAuthorizationManager.refresh(conf, new TestPolicyProvider());
+    try {
+      serviceAuthorizationManager.authorize(drwho, TestProtocol.class, conf,
+          InetAddress.getByName(ADDRESS));
+    } catch (AuthorizationException e) {
+      fail();
+    }
+    //now set a blocked acl with the user and another group
+    conf.set(ACL_CONFIG + ServiceAuthorizationManager.BLOCKED, "drwho group3");
+    serviceAuthorizationManager.refresh(conf, new TestPolicyProvider());
+    try {
+      serviceAuthorizationManager.authorize(drwho, TestProtocol.class, conf,
+          InetAddress.getByName(ADDRESS));
+      fail();
+    } catch (AuthorizationException e) {
+
+    }
+    //now set a blocked acl with another user and another group
+    conf.set(ACL_CONFIG + ServiceAuthorizationManager.BLOCKED, "drwho2 group3");
+    serviceAuthorizationManager.refresh(conf, new TestPolicyProvider());
+    try {
+      serviceAuthorizationManager.authorize(drwho, TestProtocol.class, conf,
+          InetAddress.getByName(ADDRESS));
+    } catch (AuthorizationException e) {
+      fail();
+    }
+    //now set a blocked acl with another user and group that the user belongs to
+    conf.set(ACL_CONFIG + ServiceAuthorizationManager.BLOCKED, "drwho2 group2");
+    serviceAuthorizationManager.refresh(conf, new TestPolicyProvider());
+    try {
+      serviceAuthorizationManager.authorize(drwho, TestProtocol.class, conf,
+          InetAddress.getByName(ADDRESS));
+      fail();
+    } catch (AuthorizationException e) {
+      //expects Exception
+    }
+    //reset blocked acl so that there is no blocked ACL
+    conf.set(ACL_CONFIG + ServiceAuthorizationManager.BLOCKED, "");
+    serviceAuthorizationManager.refresh(conf, new TestPolicyProvider());
+    try {
+      serviceAuthorizationManager.authorize(drwho, TestProtocol.class, conf,
+          InetAddress.getByName(ADDRESS));
+    } catch (AuthorizationException e) {
+      fail();
+    }
+  }
+
+  @Test
+  public void testDefaultBlockedAcl() throws UnknownHostException {
+    UserGroupInformation drwho =
+        UserGroupInformation.createUserForTesting("drwho@EXAMPLE.COM",
+            new String[] { "group1", "group2" });
+
+    ServiceAuthorizationManager serviceAuthorizationManager =
+        new ServiceAuthorizationManager();
+    Configuration conf = new Configuration ();
+
+    //test without setting a default blocked acl
+    serviceAuthorizationManager.refresh(conf, new TestPolicyProvider());
+    try {
+      serviceAuthorizationManager.authorize(drwho, TestProtocol1.class, conf,
+          InetAddress.getByName(ADDRESS));
+    } catch (AuthorizationException e) {
+      fail();
+    }
+
+    //set a restrictive default blocked acl and an non-restricting blocked acl for TestProtocol
+    conf.set(
+        CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_DEFAULT_BLOCKED_ACL,
+        "user2 group2");
+    conf.set(ACL_CONFIG + ServiceAuthorizationManager.BLOCKED, "user2");
+    serviceAuthorizationManager.refresh(conf, new TestPolicyProvider());
+    //drwho is authorized to access TestProtocol
+    try {
+      serviceAuthorizationManager.authorize(drwho, TestProtocol.class, conf,
+          InetAddress.getByName(ADDRESS));
+    } catch (AuthorizationException e) {
+      fail();
+    }
+    //drwho is not authorized to access TestProtocol1 because it uses the default blocked acl.
+    try {
+      serviceAuthorizationManager.authorize(drwho, TestProtocol1.class, conf,
+          InetAddress.getByName(ADDRESS));
+      fail();
+    } catch (AuthorizationException e) {
+      //expects Exception
+    }
+  }
+
 }

Modified: hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDataChecksum.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDataChecksum.java?rev=1618700&r1=1618699&r2=1618700&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDataChecksum.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDataChecksum.java Mon Aug 18 18:41:31 2014
@@ -19,6 +19,9 @@ package org.apache.hadoop.util;
 
 import java.nio.ByteBuffer;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Stopwatch;
 
 import org.apache.hadoop.fs.ChecksumException;
 import org.junit.Test;
@@ -53,68 +56,113 @@ public class TestDataChecksum {
       }
     }
   }
-  
-  private void doBulkTest(DataChecksum checksum, int dataLength,
-      boolean useDirect) throws Exception {
-    System.err.println("Testing bulk checksums of length " + 
-        dataLength + " with " +
-        (useDirect ? "direct" : "array-backed") + " buffers");
-    int numSums = (dataLength - 1)/checksum.getBytesPerChecksum() + 1;
-    int sumsLength = numSums * checksum.getChecksumSize();
-    
-    byte data[] = new byte[dataLength +
-                           DATA_OFFSET_IN_BUFFER +
-                           DATA_TRAILER_IN_BUFFER];
-    new Random().nextBytes(data);
-    ByteBuffer dataBuf = ByteBuffer.wrap(
+
+  private static class Harness {
+    final DataChecksum checksum;
+    final int dataLength, sumsLength, numSums;
+    ByteBuffer dataBuf, checksumBuf;
+
+    Harness(DataChecksum checksum, int dataLength, boolean useDirect) {
+      this.checksum = checksum;
+      this.dataLength = dataLength;
+
+      numSums = (dataLength - 1)/checksum.getBytesPerChecksum() + 1;
+      sumsLength = numSums * checksum.getChecksumSize();
+
+      byte data[] = new byte[dataLength +
+                             DATA_OFFSET_IN_BUFFER +
+                             DATA_TRAILER_IN_BUFFER];
+      new Random().nextBytes(data);
+      dataBuf = ByteBuffer.wrap(
         data, DATA_OFFSET_IN_BUFFER, dataLength);
 
-    byte checksums[] = new byte[SUMS_OFFSET_IN_BUFFER + sumsLength];
-    ByteBuffer checksumBuf = ByteBuffer.wrap(
+      byte checksums[] = new byte[SUMS_OFFSET_IN_BUFFER + sumsLength];
+      checksumBuf = ByteBuffer.wrap(
         checksums, SUMS_OFFSET_IN_BUFFER, sumsLength);
-    
-    // Swap out for direct buffers if requested.
-    if (useDirect) {
-      dataBuf = directify(dataBuf);
-      checksumBuf = directify(checksumBuf);
+
+      // Swap out for direct buffers if requested.
+      if (useDirect) {
+        dataBuf = directify(dataBuf);
+        checksumBuf = directify(checksumBuf);
+      }
     }
-    
-    // calculate real checksum, make sure it passes
-    checksum.calculateChunkedSums(dataBuf, checksumBuf);
-    checksum.verifyChunkedSums(dataBuf, checksumBuf, "fake file", 0);
-
-    // Change a byte in the header and in the trailer, make sure
-    // it doesn't affect checksum result
-    corruptBufferOffset(checksumBuf, 0);
-    checksum.verifyChunkedSums(dataBuf, checksumBuf, "fake file", 0);
-    corruptBufferOffset(dataBuf, 0);
-    dataBuf.limit(dataBuf.limit() + 1);
-    corruptBufferOffset(dataBuf, dataLength + DATA_OFFSET_IN_BUFFER);
-    dataBuf.limit(dataBuf.limit() - 1);
-    checksum.verifyChunkedSums(dataBuf, checksumBuf, "fake file", 0);    
-    
-    // Make sure bad checksums fail - error at beginning of array
-    corruptBufferOffset(checksumBuf, SUMS_OFFSET_IN_BUFFER);
-    try {
+
+    void testCorrectness() throws ChecksumException {
+      // calculate real checksum, make sure it passes
+      checksum.calculateChunkedSums(dataBuf, checksumBuf);
       checksum.verifyChunkedSums(dataBuf, checksumBuf, "fake file", 0);
-      fail("Did not throw on bad checksums");
-    } catch (ChecksumException ce) {
-      assertEquals(0, ce.getPos());
-    }
 
-    // Make sure bad checksums fail - error at end of array
-    uncorruptBufferOffset(checksumBuf, SUMS_OFFSET_IN_BUFFER);
-    corruptBufferOffset(checksumBuf, SUMS_OFFSET_IN_BUFFER + sumsLength - 1);
-    try {
+      // Change a byte in the header and in the trailer, make sure
+      // it doesn't affect checksum result
+      corruptBufferOffset(checksumBuf, 0);
+      checksum.verifyChunkedSums(dataBuf, checksumBuf, "fake file", 0);
+      corruptBufferOffset(dataBuf, 0);
+      dataBuf.limit(dataBuf.limit() + 1);
+      corruptBufferOffset(dataBuf, dataLength + DATA_OFFSET_IN_BUFFER);
+      dataBuf.limit(dataBuf.limit() - 1);
       checksum.verifyChunkedSums(dataBuf, checksumBuf, "fake file", 0);
-      fail("Did not throw on bad checksums");
-    } catch (ChecksumException ce) {
-      int expectedPos = checksum.getBytesPerChecksum() * (numSums - 1);
-      assertEquals(expectedPos, ce.getPos());
-      assertTrue(ce.getMessage().contains("fake file"));
+
+      // Make sure bad checksums fail - error at beginning of array
+      corruptBufferOffset(checksumBuf, SUMS_OFFSET_IN_BUFFER);
+      try {
+        checksum.verifyChunkedSums(dataBuf, checksumBuf, "fake file", 0);
+        fail("Did not throw on bad checksums");
+      } catch (ChecksumException ce) {
+        assertEquals(0, ce.getPos());
+      }
+
+      // Make sure bad checksums fail - error at end of array
+      uncorruptBufferOffset(checksumBuf, SUMS_OFFSET_IN_BUFFER);
+      corruptBufferOffset(checksumBuf, SUMS_OFFSET_IN_BUFFER + sumsLength - 1);
+      try {
+        checksum.verifyChunkedSums(dataBuf, checksumBuf, "fake file", 0);
+        fail("Did not throw on bad checksums");
+      } catch (ChecksumException ce) {
+        int expectedPos = checksum.getBytesPerChecksum() * (numSums - 1);
+        assertEquals(expectedPos, ce.getPos());
+        assertTrue(ce.getMessage().contains("fake file"));
+      }
     }
   }
-  
+
+  private void doBulkTest(DataChecksum checksum, int dataLength,
+      boolean useDirect) throws Exception {
+    System.err.println("Testing bulk checksums of length " +
+        dataLength + " with " +
+        (useDirect ? "direct" : "array-backed") + " buffers");
+
+    new Harness(checksum, dataLength, useDirect).testCorrectness();
+  }
+
+  /**
+   * Simple performance test for the "common case" checksum usage in HDFS:
+   * computing and verifying CRC32C with 512 byte chunking on native
+   * buffers.
+   */
+  @Test
+  public void commonUsagePerfTest() throws Exception {
+    final int NUM_RUNS = 5;
+    final DataChecksum checksum = DataChecksum.newDataChecksum(DataChecksum.Type.CRC32C, 512);
+    final int dataLength = 512 * 1024 * 1024;
+    Harness h = new Harness(checksum, dataLength, true);
+
+    for (int i = 0; i < NUM_RUNS; i++) {
+      Stopwatch s = new Stopwatch().start();
+      // calculate real checksum, make sure it passes
+      checksum.calculateChunkedSums(h.dataBuf, h.checksumBuf);
+      s.stop();
+      System.err.println("Calculate run #" + i + ": " +
+                         s.elapsedTime(TimeUnit.MICROSECONDS) + "us");
+
+      s = new Stopwatch().start();
+      // calculate real checksum, make sure it passes
+      checksum.verifyChunkedSums(h.dataBuf, h.checksumBuf, "fake file", 0);
+      s.stop();
+      System.err.println("Verify run #" + i + ": " +
+                         s.elapsedTime(TimeUnit.MICROSECONDS) + "us");
+    }
+  }
+
   @Test
   public void testEquality() {
     assertEquals(

Modified: hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java?rev=1618700&r1=1618699&r2=1618700&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestGenericOptionsParser.java Mon Aug 18 18:41:31 2014
@@ -21,11 +21,14 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.net.URI;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 import java.util.Map;
 
 import junit.framework.TestCase;
 
+import org.apache.commons.math3.util.Pair;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -34,12 +37,14 @@ import org.apache.hadoop.security.Creden
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.junit.Assert;
 
 import com.google.common.collect.Maps;
+import static org.junit.Assert.fail;
 
 public class TestGenericOptionsParser extends TestCase {
   File testDir;
@@ -93,6 +98,67 @@ public class TestGenericOptionsParser ex
   }
 
   /**
+   * Test the case where the libjars, files and archives arguments
+   * contains an empty token, which should create an IllegalArgumentException.
+   */
+  public void testEmptyFilenames() throws Exception {
+    List<Pair<String, String>> argsAndConfNames = new ArrayList<Pair<String, String>>();
+    argsAndConfNames.add(new Pair<String, String>("-libjars", "tmpjars"));
+    argsAndConfNames.add(new Pair<String, String>("-files", "tmpfiles"));
+    argsAndConfNames.add(new Pair<String, String>("-archives", "tmparchives"));
+    for (Pair<String, String> argAndConfName : argsAndConfNames) {
+      String arg = argAndConfName.getFirst();
+      String configName = argAndConfName.getSecond();
+
+      File tmpFileOne = new File(testDir, "tmpfile1");
+      Path tmpPathOne = new Path(tmpFileOne.toString());
+      File tmpFileTwo = new File(testDir, "tmpfile2");
+      Path tmpPathTwo = new Path(tmpFileTwo.toString());
+      localFs.create(tmpPathOne);
+      localFs.create(tmpPathTwo);
+      String[] args = new String[2];
+      args[0] = arg;
+      // create an empty path in between two valid files,
+      // which prior to HADOOP-10820 used to result in the
+      // working directory being added to "tmpjars" (or equivalent)
+      args[1] = String.format("%s,,%s",
+          tmpFileOne.toURI().toString(), tmpFileTwo.toURI().toString());
+      try {
+        new GenericOptionsParser(conf, args);
+        fail("Expected exception for empty filename");
+      } catch (IllegalArgumentException e) {
+        // expect to receive an IllegalArgumentException
+        GenericTestUtils.assertExceptionContains("File name can't be"
+            + " empty string", e);
+      }
+
+      // test zero file list length - it should create an exception
+      args[1] = ",,";
+      try {
+        new GenericOptionsParser(conf, args);
+        fail("Expected exception for zero file list length");
+      } catch (IllegalArgumentException e) {
+        // expect to receive an IllegalArgumentException
+        GenericTestUtils.assertExceptionContains("File name can't be"
+            + " empty string", e);
+      }
+
+      // test filename with space character
+      // it should create exception from parser in URI class
+      // due to URI syntax error
+      args[1] = String.format("%s, ,%s",
+          tmpFileOne.toURI().toString(), tmpFileTwo.toURI().toString());
+      try {
+        new GenericOptionsParser(conf, args);
+        fail("Expected exception for filename with space character");
+      } catch (IllegalArgumentException e) {
+        // expect to receive an IllegalArgumentException
+        GenericTestUtils.assertExceptionContains("URISyntaxException", e);
+      }
+    }
+  }
+
+  /**
    * Test that options passed to the constructor are used.
    */
   @SuppressWarnings("static-access")

Modified: hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml?rev=1618700&r1=1618699&r2=1618700&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml (original)
+++ hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml Mon Aug 18 18:41:31 2014
@@ -238,7 +238,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-count \[-q\] &lt;path&gt; \.\.\. :\s*</expected-output>
+          <expected-output>^-count \[-q\] \[-h\] &lt;path&gt; \.\.\. :( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -260,6 +260,10 @@
           <type>RegexpComparator</type>
           <expected-output>^( |\t)*DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME( )*</expected-output>
         </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*The -h option shows file sizes in human readable format.( )*</expected-output>
+        </comparator>
       </comparators>
     </test>
 

Modified: hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java?rev=1618700&r1=1618699&r2=1618700&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java Mon Aug 18 18:41:31 2014
@@ -25,9 +25,10 @@ import org.apache.hadoop.crypto.key.KeyP
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
 import org.apache.hadoop.crypto.key.kms.KMSRESTConstants;
 import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
+import org.apache.hadoop.security.token.delegation.web.HttpUserGroupInformation;
 
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
@@ -38,15 +39,13 @@ import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.QueryParam;
-import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import javax.ws.rs.core.SecurityContext;
 
 import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
-import java.security.Principal;
+import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
@@ -74,15 +73,6 @@ public class KMS {
     kmsAudit= KMSWebApp.getKMSAudit();
   }
 
-  private static Principal getPrincipal(SecurityContext securityContext)
-      throws AuthenticationException{
-    Principal user = securityContext.getUserPrincipal();
-    if (user == null) {
-      throw new AuthenticationException("User must be authenticated");
-    }
-    return user;
-  }
-
 
   private static final String UNAUTHORIZED_MSG_WITH_KEY = 
       "User:%s not allowed to do '%s' on '%s'";
@@ -90,20 +80,21 @@ public class KMS {
   private static final String UNAUTHORIZED_MSG_WITHOUT_KEY = 
       "User:%s not allowed to do '%s'";
 
-  private void assertAccess(KMSACLs.Type aclType, Principal principal,
+  private void assertAccess(KMSACLs.Type aclType, UserGroupInformation ugi,
       KMSOp operation) throws AccessControlException {
-    assertAccess(aclType, principal, operation, null);
+    assertAccess(aclType, ugi, operation, null);
   }
 
-  private void assertAccess(KMSACLs.Type aclType, Principal principal,
-      KMSOp operation, String key) throws AccessControlException {
-    if (!KMSWebApp.getACLs().hasAccess(aclType, principal.getName())) {
+  private void assertAccess(KMSACLs.Type aclType,
+      UserGroupInformation ugi, KMSOp operation, String key)
+      throws AccessControlException {
+    if (!KMSWebApp.getACLs().hasAccess(aclType, ugi)) {
       KMSWebApp.getUnauthorizedCallsMeter().mark();
-      kmsAudit.unauthorized(principal, operation, key);
+      kmsAudit.unauthorized(ugi, operation, key);
       throw new AuthorizationException(String.format(
           (key != null) ? UNAUTHORIZED_MSG_WITH_KEY 
                         : UNAUTHORIZED_MSG_WITHOUT_KEY,
-          principal.getName(), operation, key));
+          ugi.getShortUserName(), operation, key));
     }
   }
 
@@ -123,15 +114,14 @@ public class KMS {
   @Consumes(MediaType.APPLICATION_JSON)
   @Produces(MediaType.APPLICATION_JSON)
   @SuppressWarnings("unchecked")
-  public Response createKey(@Context SecurityContext securityContext,
-      Map jsonKey) throws Exception {
+  public Response createKey(Map jsonKey) throws Exception {
     KMSWebApp.getAdminCallsMeter().mark();
-    Principal user = getPrincipal(securityContext);
-    String name = (String) jsonKey.get(KMSRESTConstants.NAME_FIELD);
+    UserGroupInformation user = HttpUserGroupInformation.get();
+    final String name = (String) jsonKey.get(KMSRESTConstants.NAME_FIELD);
     KMSClientProvider.checkNotEmpty(name, KMSRESTConstants.NAME_FIELD);
     assertAccess(KMSACLs.Type.CREATE, user, KMSOp.CREATE_KEY, name);
     String cipher = (String) jsonKey.get(KMSRESTConstants.CIPHER_FIELD);
-    String material = (String) jsonKey.get(KMSRESTConstants.MATERIAL_FIELD);
+    final String material = (String) jsonKey.get(KMSRESTConstants.MATERIAL_FIELD);
     int length = (jsonKey.containsKey(KMSRESTConstants.LENGTH_FIELD))
                  ? (Integer) jsonKey.get(KMSRESTConstants.LENGTH_FIELD) : 0;
     String description = (String)
@@ -142,7 +132,7 @@ public class KMS {
       assertAccess(KMSACLs.Type.SET_KEY_MATERIAL, user,
           KMSOp.CREATE_KEY, name);
     }
-    KeyProvider.Options options = new KeyProvider.Options(
+    final KeyProvider.Options options = new KeyProvider.Options(
         KMSWebApp.getConfiguration());
     if (cipher != null) {
       options.setCipher(cipher);
@@ -153,16 +143,23 @@ public class KMS {
     options.setDescription(description);
     options.setAttributes(attributes);
 
-    KeyProvider.KeyVersion keyVersion = (material != null)
-        ? provider.createKey(name, Base64.decodeBase64(material), options)
-        : provider.createKey(name, options);
-
-    provider.flush();
+    KeyProvider.KeyVersion keyVersion = user.doAs(
+        new PrivilegedExceptionAction<KeyVersion>() {
+          @Override
+          public KeyVersion run() throws Exception {
+            KeyProvider.KeyVersion keyVersion = (material != null)
+              ? provider.createKey(name, Base64.decodeBase64(material), options)
+              : provider.createKey(name, options);
+            provider.flush();
+            return keyVersion;
+          }
+        }
+    );
 
     kmsAudit.ok(user, KMSOp.CREATE_KEY, name, "UserProvidedMaterial:" +
         (material != null) + " Description:" + description);
 
-    if (!KMSWebApp.getACLs().hasAccess(KMSACLs.Type.GET, user.getName())) {
+    if (!KMSWebApp.getACLs().hasAccess(KMSACLs.Type.GET, user)) {
       keyVersion = removeKeyMaterial(keyVersion);
     }
     Map json = KMSServerJSONUtils.toJSON(keyVersion);
@@ -176,14 +173,21 @@ public class KMS {
 
   @DELETE
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}")
-  public Response deleteKey(@Context SecurityContext securityContext,
-      @PathParam("name") String name) throws Exception {
+  public Response deleteKey(@PathParam("name") final String name)
+      throws Exception {
     KMSWebApp.getAdminCallsMeter().mark();
-    Principal user = getPrincipal(securityContext);
+    UserGroupInformation user = HttpUserGroupInformation.get();
     assertAccess(KMSACLs.Type.DELETE, user, KMSOp.DELETE_KEY, name);
     KMSClientProvider.checkNotEmpty(name, "name");
-    provider.deleteKey(name);
-    provider.flush();
+
+    user.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        provider.deleteKey(name);
+        provider.flush();
+        return null;
+      }
+    });
 
     kmsAudit.ok(user, KMSOp.DELETE_KEY, name, "");
 
@@ -194,29 +198,36 @@ public class KMS {
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}")
   @Consumes(MediaType.APPLICATION_JSON)
   @Produces(MediaType.APPLICATION_JSON)
-  public Response rolloverKey(@Context SecurityContext securityContext,
-      @PathParam("name") String name, Map jsonMaterial)
-      throws Exception {
+  public Response rolloverKey(@PathParam("name") final String name,
+      Map jsonMaterial) throws Exception {
     KMSWebApp.getAdminCallsMeter().mark();
-    Principal user = getPrincipal(securityContext);
+    UserGroupInformation user = HttpUserGroupInformation.get();
     assertAccess(KMSACLs.Type.ROLLOVER, user, KMSOp.ROLL_NEW_VERSION, name);
     KMSClientProvider.checkNotEmpty(name, "name");
-    String material = (String)
+    final String material = (String)
         jsonMaterial.get(KMSRESTConstants.MATERIAL_FIELD);
     if (material != null) {
       assertAccess(KMSACLs.Type.SET_KEY_MATERIAL, user,
           KMSOp.ROLL_NEW_VERSION, name);
     }
-    KeyProvider.KeyVersion keyVersion = (material != null)
-        ? provider.rollNewVersion(name, Base64.decodeBase64(material))
-        : provider.rollNewVersion(name);
 
-    provider.flush();
+    KeyProvider.KeyVersion keyVersion = user.doAs(
+        new PrivilegedExceptionAction<KeyVersion>() {
+          @Override
+          public KeyVersion run() throws Exception {
+            KeyVersion keyVersion = (material != null)
+              ? provider.rollNewVersion(name, Base64.decodeBase64(material))
+              : provider.rollNewVersion(name);
+            provider.flush();
+            return keyVersion;
+          }
+        }
+    );
 
     kmsAudit.ok(user, KMSOp.ROLL_NEW_VERSION, name, "UserProvidedMaterial:" +
         (material != null) + " NewVersion:" + keyVersion.getVersionName());
 
-    if (!KMSWebApp.getACLs().hasAccess(KMSACLs.Type.GET, user.getName())) {
+    if (!KMSWebApp.getACLs().hasAccess(KMSACLs.Type.GET, user)) {
       keyVersion = removeKeyMaterial(keyVersion);
     }
     Map json = KMSServerJSONUtils.toJSON(keyVersion);
@@ -226,14 +237,23 @@ public class KMS {
   @GET
   @Path(KMSRESTConstants.KEYS_METADATA_RESOURCE)
   @Produces(MediaType.APPLICATION_JSON)
-  public Response getKeysMetadata(@Context SecurityContext securityContext,
-      @QueryParam(KMSRESTConstants.KEY) List<String> keyNamesList)
-      throws Exception {
+  public Response getKeysMetadata(@QueryParam(KMSRESTConstants.KEY)
+      List<String> keyNamesList) throws Exception {
     KMSWebApp.getAdminCallsMeter().mark();
-    Principal user = getPrincipal(securityContext);
-    String[] keyNames = keyNamesList.toArray(new String[keyNamesList.size()]);
+    UserGroupInformation user = HttpUserGroupInformation.get();
+    final String[] keyNames = keyNamesList.toArray(
+        new String[keyNamesList.size()]);
     assertAccess(KMSACLs.Type.GET_METADATA, user, KMSOp.GET_KEYS_METADATA);
-    KeyProvider.Metadata[] keysMeta = provider.getKeysMetadata(keyNames);
+
+    KeyProvider.Metadata[] keysMeta = user.doAs(
+        new PrivilegedExceptionAction<KeyProvider.Metadata[]>() {
+          @Override
+          public KeyProvider.Metadata[] run() throws Exception {
+            return provider.getKeysMetadata(keyNames);
+          }
+        }
+    );
+
     Object json = KMSServerJSONUtils.toJSON(keyNames, keysMeta);
     kmsAudit.ok(user, KMSOp.GET_KEYS_METADATA, "");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
@@ -242,36 +262,52 @@ public class KMS {
   @GET
   @Path(KMSRESTConstants.KEYS_NAMES_RESOURCE)
   @Produces(MediaType.APPLICATION_JSON)
-  public Response getKeyNames(@Context SecurityContext securityContext)
-      throws Exception {
+  public Response getKeyNames() throws Exception {
     KMSWebApp.getAdminCallsMeter().mark();
-    Principal user = getPrincipal(securityContext);
+    UserGroupInformation user = HttpUserGroupInformation.get();
     assertAccess(KMSACLs.Type.GET_KEYS, user, KMSOp.GET_KEYS);
-    Object json = provider.getKeys();
+
+    List<String> json = user.doAs(
+        new PrivilegedExceptionAction<List<String>>() {
+          @Override
+          public List<String> run() throws Exception {
+            return provider.getKeys();
+          }
+        }
+    );
+
     kmsAudit.ok(user, KMSOp.GET_KEYS, "");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
   @GET
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}")
-  public Response getKey(@Context SecurityContext securityContext,
-      @PathParam("name") String name)
+  public Response getKey(@PathParam("name") String name)
       throws Exception {
-    return getMetadata(securityContext, name);
+    return getMetadata(name);
   }
 
   @GET
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}/" +
       KMSRESTConstants.METADATA_SUB_RESOURCE)
   @Produces(MediaType.APPLICATION_JSON)
-  public Response getMetadata(@Context SecurityContext securityContext,
-      @PathParam("name") String name)
+  public Response getMetadata(@PathParam("name") final String name)
       throws Exception {
-    Principal user = getPrincipal(securityContext);
+    UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(name, "name");
     KMSWebApp.getAdminCallsMeter().mark();
     assertAccess(KMSACLs.Type.GET_METADATA, user, KMSOp.GET_METADATA, name);
-    Object json = KMSServerJSONUtils.toJSON(name, provider.getMetadata(name));
+
+    KeyProvider.Metadata metadata = user.doAs(
+        new PrivilegedExceptionAction<KeyProvider.Metadata>() {
+          @Override
+          public KeyProvider.Metadata run() throws Exception {
+            return provider.getMetadata(name);
+          }
+        }
+    );
+
+    Object json = KMSServerJSONUtils.toJSON(name, metadata);
     kmsAudit.ok(user, KMSOp.GET_METADATA, name, "");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
@@ -280,14 +316,23 @@ public class KMS {
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}/" +
       KMSRESTConstants.CURRENT_VERSION_SUB_RESOURCE)
   @Produces(MediaType.APPLICATION_JSON)
-  public Response getCurrentVersion(@Context SecurityContext securityContext,
-      @PathParam("name") String name)
+  public Response getCurrentVersion(@PathParam("name") final String name)
       throws Exception {
-    Principal user = getPrincipal(securityContext);
+    UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(name, "name");
     KMSWebApp.getKeyCallsMeter().mark();
     assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_CURRENT_KEY, name);
-    Object json = KMSServerJSONUtils.toJSON(provider.getCurrentKey(name));
+
+    KeyVersion keyVersion = user.doAs(
+        new PrivilegedExceptionAction<KeyVersion>() {
+          @Override
+          public KeyVersion run() throws Exception {
+            return provider.getCurrentKey(name);
+          }
+        }
+    );
+
+    Object json = KMSServerJSONUtils.toJSON(keyVersion);
     kmsAudit.ok(user, KMSOp.GET_CURRENT_KEY, name, "");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
@@ -295,14 +340,22 @@ public class KMS {
   @GET
   @Path(KMSRESTConstants.KEY_VERSION_RESOURCE + "/{versionName:.*}")
   @Produces(MediaType.APPLICATION_JSON)
-  public Response getKeyVersion(@Context SecurityContext securityContext,
-      @PathParam("versionName") String versionName)
-      throws Exception {
-    Principal user = getPrincipal(securityContext);
+  public Response getKeyVersion(
+      @PathParam("versionName") final String versionName) throws Exception {
+    UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(versionName, "versionName");
     KMSWebApp.getKeyCallsMeter().mark();
-    KeyVersion keyVersion = provider.getKeyVersion(versionName);
     assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_KEY_VERSION);
+
+    KeyVersion keyVersion = user.doAs(
+        new PrivilegedExceptionAction<KeyVersion>() {
+          @Override
+          public KeyVersion run() throws Exception {
+            return provider.getKeyVersion(versionName);
+          }
+        }
+    );
+
     if (keyVersion != null) {
       kmsAudit.ok(user, KMSOp.GET_KEY_VERSION, keyVersion.getName(), "");
     }
@@ -316,13 +369,12 @@ public class KMS {
       KMSRESTConstants.EEK_SUB_RESOURCE)
   @Produces(MediaType.APPLICATION_JSON)
   public Response generateEncryptedKeys(
-          @Context SecurityContext securityContext,
-          @PathParam("name") String name,
+          @PathParam("name") final String name,
           @QueryParam(KMSRESTConstants.EEK_OP) String edekOp,
           @DefaultValue("1")
-          @QueryParam(KMSRESTConstants.EEK_NUM_KEYS) int numKeys)
+          @QueryParam(KMSRESTConstants.EEK_NUM_KEYS) final int numKeys)
           throws Exception {
-    Principal user = getPrincipal(securityContext);
+    UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(name, "name");
     KMSClientProvider.checkNotNull(edekOp, "eekOp");
 
@@ -330,12 +382,22 @@ public class KMS {
     if (edekOp.equals(KMSRESTConstants.EEK_GENERATE)) {
       assertAccess(KMSACLs.Type.GENERATE_EEK, user, KMSOp.GENERATE_EEK, name);
 
-      List<EncryptedKeyVersion> retEdeks =
+      final List<EncryptedKeyVersion> retEdeks =
           new LinkedList<EncryptedKeyVersion>();
       try {
-        for (int i = 0; i < numKeys; i ++) {
-          retEdeks.add(provider.generateEncryptedKey(name));
-        }
+
+        user.doAs(
+            new PrivilegedExceptionAction<Void>() {
+              @Override
+              public Void run() throws Exception {
+                for (int i = 0; i < numKeys; i++) {
+                  retEdeks.add(provider.generateEncryptedKey(name));
+                }
+                return null;
+              }
+            }
+        );
+
       } catch (Exception e) {
         throw new IOException(e);
       }
@@ -359,16 +421,17 @@ public class KMS {
   @Path(KMSRESTConstants.KEY_VERSION_RESOURCE + "/{versionName:.*}/" +
       KMSRESTConstants.EEK_SUB_RESOURCE)
   @Produces(MediaType.APPLICATION_JSON)
-  public Response decryptEncryptedKey(@Context SecurityContext securityContext,
-      @PathParam("versionName") String versionName,
+  public Response decryptEncryptedKey(
+      @PathParam("versionName") final String versionName,
       @QueryParam(KMSRESTConstants.EEK_OP) String eekOp,
       Map jsonPayload)
       throws Exception {
-    Principal user = getPrincipal(securityContext);
+    UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(versionName, "versionName");
     KMSClientProvider.checkNotNull(eekOp, "eekOp");
 
-    String keyName = (String) jsonPayload.get(KMSRESTConstants.NAME_FIELD);
+    final String keyName = (String) jsonPayload.get(
+        KMSRESTConstants.NAME_FIELD);
     String ivStr = (String) jsonPayload.get(KMSRESTConstants.IV_FIELD);
     String encMaterialStr = 
         (String) jsonPayload.get(KMSRESTConstants.MATERIAL_FIELD);
@@ -376,14 +439,24 @@ public class KMS {
     if (eekOp.equals(KMSRESTConstants.EEK_DECRYPT)) {
       assertAccess(KMSACLs.Type.DECRYPT_EEK, user, KMSOp.DECRYPT_EEK, keyName);
       KMSClientProvider.checkNotNull(ivStr, KMSRESTConstants.IV_FIELD);
-      byte[] iv = Base64.decodeBase64(ivStr);
+      final byte[] iv = Base64.decodeBase64(ivStr);
       KMSClientProvider.checkNotNull(encMaterialStr,
           KMSRESTConstants.MATERIAL_FIELD);
-      byte[] encMaterial = Base64.decodeBase64(encMaterialStr);
-      KeyProvider.KeyVersion retKeyVersion =
-          provider.decryptEncryptedKey(
-              new KMSClientProvider.KMSEncryptedKeyVersion(keyName, versionName,
-                  iv, KeyProviderCryptoExtension.EEK, encMaterial));
+      final byte[] encMaterial = Base64.decodeBase64(encMaterialStr);
+
+      KeyProvider.KeyVersion retKeyVersion = user.doAs(
+          new PrivilegedExceptionAction<KeyVersion>() {
+            @Override
+            public KeyVersion run() throws Exception {
+              return provider.decryptEncryptedKey(
+                  new KMSClientProvider.KMSEncryptedKeyVersion(keyName,
+                      versionName, iv, KeyProviderCryptoExtension.EEK,
+                      encMaterial)
+              );
+            }
+          }
+      );
+
       retJSON = KMSServerJSONUtils.toJSON(retKeyVersion);
       kmsAudit.ok(user, KMSOp.DECRYPT_EEK, keyName, "");
     } else {
@@ -400,14 +473,23 @@ public class KMS {
   @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}/" +
       KMSRESTConstants.VERSIONS_SUB_RESOURCE)
   @Produces(MediaType.APPLICATION_JSON)
-  public Response getKeyVersions(@Context SecurityContext securityContext,
-      @PathParam("name") String name)
+  public Response getKeyVersions(@PathParam("name") final String name)
       throws Exception {
-    Principal user = getPrincipal(securityContext);
+    UserGroupInformation user = HttpUserGroupInformation.get();
     KMSClientProvider.checkNotEmpty(name, "name");
     KMSWebApp.getKeyCallsMeter().mark();
     assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_KEY_VERSIONS, name);
-    Object json = KMSServerJSONUtils.toJSON(provider.getKeyVersions(name));
+
+    List<KeyVersion> ret = user.doAs(
+        new PrivilegedExceptionAction<List<KeyVersion>>() {
+          @Override
+          public List<KeyVersion> run() throws Exception {
+            return provider.getKeyVersions(name);
+          }
+        }
+    );
+
+    Object json = KMSServerJSONUtils.toJSON(ret);
     kmsAudit.ok(user, KMSOp.GET_KEY_VERSIONS, name, "");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }

Modified: hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java?rev=1618700&r1=1618699&r2=1618700&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java Mon Aug 18 18:41:31 2014
@@ -113,8 +113,7 @@ public class KMSACLs implements Runnable
     return conf;
   }
 
-  public boolean hasAccess(Type type, String user) {
-    UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
+  public boolean hasAccess(Type type, UserGroupInformation ugi) {
     return acls.get(type).isUserAllowed(ugi);
   }
 

Modified: hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java?rev=1618700&r1=1618699&r2=1618700&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java Mon Aug 18 18:41:31 2014
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.crypto.key.kms.server;
 
+import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -29,7 +30,6 @@ import com.google.common.cache.RemovalNo
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
-import java.security.Principal;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Set;
@@ -186,22 +186,22 @@ public class KMSAudit {
     }
   }
 
-  public void ok(Principal user, KMS.KMSOp op, String key,
+  public void ok(UserGroupInformation user, KMS.KMSOp op, String key,
       String extraMsg) {
-    op(OpStatus.OK, op, user.getName(), key, extraMsg);
+    op(OpStatus.OK, op, user.getShortUserName(), key, extraMsg);
   }
 
-  public void ok(Principal user, KMS.KMSOp op, String extraMsg) {
-    op(OpStatus.OK, op, user.getName(), null, extraMsg);
+  public void ok(UserGroupInformation user, KMS.KMSOp op, String extraMsg) {
+    op(OpStatus.OK, op, user.getShortUserName(), null, extraMsg);
   }
 
-  public void unauthorized(Principal user, KMS.KMSOp op, String key) {
-    op(OpStatus.UNAUTHORIZED, op, user.getName(), key, "");
+  public void unauthorized(UserGroupInformation user, KMS.KMSOp op, String key) {
+    op(OpStatus.UNAUTHORIZED, op, user.getShortUserName(), key, "");
   }
 
-  public void error(Principal user, String method, String url,
+  public void error(UserGroupInformation user, String method, String url,
       String extraMsg) {
-    op(OpStatus.ERROR, null, user.getName(), null, "Method:'" + method
+    op(OpStatus.ERROR, null, user.getShortUserName(), null, "Method:'" + method
         + "' Exception:'" + extraMsg + "'");
   }
 

Modified: hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuthenticationFilter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuthenticationFilter.java?rev=1618700&r1=1618699&r2=1618700&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuthenticationFilter.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuthenticationFilter.java Mon Aug 18 18:41:31 2014
@@ -19,7 +19,13 @@ package org.apache.hadoop.crypto.key.kms
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
+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;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationHandler;
+import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticationHandler;
+import org.apache.hadoop.security.token.delegation.web.PseudoDelegationTokenAuthenticationHandler;
 
 import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
@@ -38,7 +44,8 @@ import java.util.Properties;
  * file.
  */
 @InterfaceAudience.Private
-public class KMSAuthenticationFilter extends AuthenticationFilter {
+public class KMSAuthenticationFilter
+    extends DelegationTokenAuthenticationFilter {
   private static final String CONF_PREFIX = KMSConfiguration.CONFIG_PREFIX +
       "authentication.";
 
@@ -55,9 +62,30 @@ public class KMSAuthenticationFilter ext
         props.setProperty(name, value);
       }
     }
+    String authType = props.getProperty(AUTH_TYPE);
+    if (authType.equals(PseudoAuthenticationHandler.TYPE)) {
+      props.setProperty(AUTH_TYPE,
+          PseudoDelegationTokenAuthenticationHandler.class.getName());
+    } else if (authType.equals(KerberosAuthenticationHandler.TYPE)) {
+      props.setProperty(AUTH_TYPE,
+          KerberosDelegationTokenAuthenticationHandler.class.getName());
+    }
+    props.setProperty(DelegationTokenAuthenticationHandler.TOKEN_KIND,
+        KMSClientProvider.TOKEN_KIND);
     return props;
   }
 
+  protected Configuration getProxyuserConfiguration(FilterConfig filterConfig) {
+    Map<String, String> proxyuserConf = KMSWebApp.getConfiguration().
+        getValByRegex("hadoop\\.kms\\.proxyuser\\.");
+    Configuration conf = new Configuration(false);
+    for (Map.Entry<String, String> entry : proxyuserConf.entrySet()) {
+      conf.set(entry.getKey().substring("hadoop.kms.".length()),
+          entry.getValue());
+    }
+    return conf;
+  }
+
   private static class KMSResponse extends HttpServletResponseWrapper {
     public int statusCode;
     public String msg;

Modified: hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSExceptionsProvider.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSExceptionsProvider.java?rev=1618700&r1=1618699&r2=1618700&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSExceptionsProvider.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSExceptionsProvider.java Mon Aug 18 18:41:31 2014
@@ -23,6 +23,7 @@ import com.sun.jersey.api.container.Cont
 
 import org.apache.hadoop.crypto.key.kms.KMSRESTConstants;
 import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.slf4j.Logger;
@@ -34,7 +35,6 @@ import javax.ws.rs.ext.ExceptionMapper;
 import javax.ws.rs.ext.Provider;
 
 import java.io.IOException;
-import java.security.Principal;
 import java.util.LinkedHashMap;
 import java.util.Map;
 
@@ -102,7 +102,7 @@ public class KMSExceptionsProvider imple
       status = Response.Status.INTERNAL_SERVER_ERROR;
     }
     if (doAudit) {
-      KMSWebApp.getKMSAudit().error(KMSMDCFilter.getPrincipal(),
+      KMSWebApp.getKMSAudit().error(KMSMDCFilter.getUgi(),
           KMSMDCFilter.getMethod(),
           KMSMDCFilter.getURL(), getOneLineMessage(exception));
     }
@@ -110,11 +110,11 @@ public class KMSExceptionsProvider imple
   }
 
   protected void log(Response.Status status, Throwable ex) {
-    Principal principal = KMSMDCFilter.getPrincipal();
+    UserGroupInformation ugi = KMSMDCFilter.getUgi();
     String method = KMSMDCFilter.getMethod();
     String url = KMSMDCFilter.getURL();
     String msg = getOneLineMessage(ex);
-    LOG.warn("User:{} Method:{} URL:{} Response:{}-{}", principal, method, url,
+    LOG.warn("User:'{}' Method:{} URL:{} Response:{}-{}", ugi, method, url,
         status, msg, ex);
   }
 

Modified: hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSMDCFilter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSMDCFilter.java?rev=1618700&r1=1618699&r2=1618700&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSMDCFilter.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSMDCFilter.java Mon Aug 18 18:41:31 2014
@@ -18,6 +18,8 @@
 package org.apache.hadoop.crypto.key.kms.server;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.delegation.web.HttpUserGroupInformation;
 
 import javax.servlet.Filter;
 import javax.servlet.FilterChain;
@@ -27,7 +29,6 @@ import javax.servlet.ServletRequest;
 import javax.servlet.ServletResponse;
 import javax.servlet.http.HttpServletRequest;
 import java.io.IOException;
-import java.security.Principal;
 
 /**
  * Servlet filter that captures context of the HTTP request to be use in the
@@ -37,12 +38,12 @@ import java.security.Principal;
 public class KMSMDCFilter implements Filter {
 
   private static class Data {
-    private Principal principal;
+    private UserGroupInformation ugi;
     private String method;
     private StringBuffer url;
 
-    private Data(Principal principal, String method, StringBuffer url) {
-      this.principal = principal;
+    private Data(UserGroupInformation ugi, String method, StringBuffer url) {
+      this.ugi = ugi;
       this.method = method;
       this.url = url;
     }
@@ -50,8 +51,8 @@ public class KMSMDCFilter implements Fil
 
   private static ThreadLocal<Data> DATA_TL = new ThreadLocal<Data>();
 
-  public static Principal getPrincipal() {
-    return DATA_TL.get().principal;
+  public static UserGroupInformation getUgi() {
+    return DATA_TL.get().ugi;
   }
 
   public static String getMethod() {
@@ -72,14 +73,14 @@ public class KMSMDCFilter implements Fil
       throws IOException, ServletException {
     try {
       DATA_TL.remove();
-      Principal principal = ((HttpServletRequest) request).getUserPrincipal();
+      UserGroupInformation ugi = HttpUserGroupInformation.get();
       String method = ((HttpServletRequest) request).getMethod();
       StringBuffer requestURL = ((HttpServletRequest) request).getRequestURL();
       String queryString = ((HttpServletRequest) request).getQueryString();
       if (queryString != null) {
         requestURL.append("?").append(queryString);
       }
-      DATA_TL.set(new Data(principal, method, requestURL));
+      DATA_TL.set(new Data(ugi, method, requestURL));
       chain.doFilter(request, response);
     } finally {
       DATA_TL.remove();

Modified: hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm?rev=1618700&r1=1618699&r2=1618700&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm (original)
+++ hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm Mon Aug 18 18:41:31 2014
@@ -195,6 +195,46 @@ hadoop-${project.version} $ sbin/kms.sh 
   NOTE: You need to restart the KMS for the configuration changes to take
   effect.
 
+*** KMS Proxyuser Configuration
+
+  Each proxyusers must be configured in <<<etc/hadoop/kms-site.xml>>> using the
+  following properties:
+
++---+
+  <property>
+    <name>hadoop.kms.proxyusers.#USER#.users</name>
+    <value>*</value>
+  </property>
+
+  <property>
+    <name>hadoop.kms.proxyusers.#USER#.groups</name>
+    <value>*</value>
+  </property>
+
+  <property>
+    <name>hadoop.kms.proxyusers.#USER#.hosts</name>
+    <value>*</value>
+  </property>
++---+
+
+  <<<#USER#>>> is the username of the proxyuser to configure.
+
+  The <<<users>>> property indicates the users that can be impersonated.
+
+  The <<<groups>>> property indicates the groups users being impersonated must
+  belong to.
+
+  At least one of the <<<users>>> or <<<groups>>> properties must be defined.
+  If both are specified, then the configured proxyuser will be able to 
+  impersonate and user in the <<<users>>> list and any user belonging to one of 
+  the groups in the <<<groups>>> list.
+
+  The <<<hosts>>> property indicates from which host the proxyuser can make
+  impersonation requests.
+
+  If <<<users>>>, <<<groups>>> or <<<hosts>>> has a <<<*>>>, it means there are
+  no restrictions for the proxyuser regarding users, groups or hosts.
+  
 *** KMS over HTTPS (SSL)
 
   To configure KMS to work over HTTPS the following 2 properties must be
@@ -319,6 +359,46 @@ $ keytool -genkey -alias tomcat -keyalg 
 </configuration>
 +---+
 
+** KMS Delegation Token Configuration
+
+  KMS delegation token secret manager can be configured with the following
+  properties:
+
+  +---+
+    <property>
+      <name>hadoop.kms.authentication.delegation-token.update-interval.sec</name>
+      <value>86400</value>
+      <description>
+        How often the master key is rotated, in seconds. Default value 1 day.
+      </description>
+    </property>
+
+    <property>
+      <name>hadoop.kms.authentication.delegation-token.max-lifetime.sec</name>
+      <value>604800</value>
+      <description>
+        Maximum lifetime of a delagation token, in seconds. Default value 7 days.
+      </description>
+    </property>
+
+    <property>
+      <name>hadoop.kms.authentication.delegation-token.renew-interval.sec</name>
+      <value>86400</value>
+      <description>
+        Renewal interval of a delagation token, in seconds. Default value 1 day.
+      </description>
+    </property>
+
+    <property>
+      <name>hadoop.kms.authentication.delegation-token.removal-scan-interval.sec</name>
+      <value>3600</value>
+      <description>
+        Scan interval to remove expired delegation tokens.
+      </description>
+    </property>
+  +---+
+
+
 ** KMS HTTP REST API
 
 *** Create a Key

Modified: hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java?rev=1618700&r1=1618699&r2=1618700&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java Mon Aug 18 18:41:31 2014
@@ -22,12 +22,18 @@ import org.apache.hadoop.crypto.key.KeyP
 import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 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.io.Text;
 import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
 import org.junit.AfterClass;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mortbay.jetty.Connector;
@@ -45,6 +51,7 @@ import java.io.FileWriter;
 import java.io.IOException;
 import java.io.Writer;
 import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.net.MalformedURLException;
 import java.net.ServerSocket;
 import java.net.SocketTimeoutException;
@@ -65,6 +72,13 @@ import java.util.concurrent.Callable;
 
 public class TestKMS {
 
+  @Before
+  public void cleanUp() {
+    // resetting kerberos security
+    Configuration conf = new Configuration();
+    UserGroupInformation.setConfiguration(conf);
+  }
+
   public static File getTestDir() throws Exception {
     File file = new File("dummy");
     file = file.getAbsoluteFile();
@@ -255,6 +269,7 @@ public class TestKMS {
     principals.add("HTTP/localhost");
     principals.add("client");
     principals.add("client/host");
+    principals.add("client1");
     for (KMSACLs.Type type : KMSACLs.Type.values()) {
       principals.add(type.toString());
     }
@@ -284,7 +299,9 @@ public class TestKMS {
     try {
       loginContext.login();
       subject = loginContext.getSubject();
-      return Subject.doAs(subject, action);
+      UserGroupInformation ugi =
+          UserGroupInformation.getUGIFromSubject(subject);
+      return ugi.doAs(action);
     } finally {
       loginContext.logout();
     }
@@ -292,8 +309,13 @@ public class TestKMS {
 
   public void testStartStop(final boolean ssl, final boolean kerberos)
       throws Exception {
+    Configuration conf = new Configuration();
+    if (kerberos) {
+      conf.set("hadoop.security.authentication", "kerberos");
+    }
+    UserGroupInformation.setConfiguration(conf);
     File testDir = getTestDir();
-    Configuration conf = createBaseKMSConf(testDir);
+    conf = createBaseKMSConf(testDir);
 
     final String keystore;
     final String password;
@@ -321,18 +343,18 @@ public class TestKMS {
     runServer(keystore, password, testDir, new KMSCallable() {
       @Override
       public Void call() throws Exception {
-        Configuration conf = new Configuration();
+        final Configuration conf = new Configuration();
         URL url = getKMSUrl();
         Assert.assertEquals(keystore != null,
             url.getProtocol().equals("https"));
-        URI uri = createKMSUri(getKMSUrl());
-        final KeyProvider kp = new KMSClientProvider(uri, conf);
+        final URI uri = createKMSUri(getKMSUrl());
 
         if (kerberos) {
           for (String user : new String[]{"client", "client/host"}) {
             doAs(user, new PrivilegedExceptionAction<Void>() {
               @Override
               public Void run() throws Exception {
+                final KeyProvider kp = new KMSClientProvider(uri, conf);
                 // getKeys() empty
                 Assert.assertTrue(kp.getKeys().isEmpty());
                 return null;
@@ -340,6 +362,7 @@ public class TestKMS {
             });
           }
         } else {
+          KeyProvider kp = new KMSClientProvider(uri, conf);
           // getKeys() empty
           Assert.assertTrue(kp.getKeys().isEmpty());
         }
@@ -370,8 +393,11 @@ public class TestKMS {
 
   @Test
   public void testKMSProvider() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("hadoop.security.authentication", "kerberos");
+    UserGroupInformation.setConfiguration(conf);
     File confDir = getTestDir();
-    Configuration conf = createBaseKMSConf(confDir);
+    conf = createBaseKMSConf(confDir);
     writeConf(confDir, conf);
 
     runServer(null, null, confDir, new KMSCallable() {
@@ -565,6 +591,17 @@ public class TestKMS {
         Assert.assertEquals("d", meta.getDescription());
         Assert.assertEquals(attributes, meta.getAttributes());
 
+        KeyProviderDelegationTokenExtension kpdte =
+            KeyProviderDelegationTokenExtension.
+                createKeyProviderDelegationTokenExtension(kp);
+        Credentials credentials = new Credentials();
+        kpdte.addDelegationTokens("foo", credentials);
+        Assert.assertEquals(1, credentials.getAllTokens().size());
+        InetSocketAddress kmsAddr = new InetSocketAddress(getKMSUrl().getHost(),
+            getKMSUrl().getPort());
+
+        Assert.assertEquals(new Text("kms-dt"), credentials.getToken(
+            SecurityUtil.buildTokenService(kmsAddr)).getKind());
         return null;
       }
     });
@@ -572,8 +609,11 @@ public class TestKMS {
 
   @Test
   public void testACLs() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("hadoop.security.authentication", "kerberos");
+    UserGroupInformation.setConfiguration(conf);
     final File testDir = getTestDir();
-    Configuration conf = createBaseKMSConf(testDir);
+    conf = createBaseKMSConf(testDir);
     conf.set("hadoop.kms.authentication.type", "kerberos");
     conf.set("hadoop.kms.authentication.kerberos.keytab",
         keytab.getAbsolutePath());
@@ -596,20 +636,20 @@ public class TestKMS {
       public Void call() throws Exception {
         final Configuration conf = new Configuration();
         conf.setInt(KeyProvider.DEFAULT_BITLENGTH_NAME, 128);
-        URI uri = createKMSUri(getKMSUrl());
-        final KeyProvider kp = new KMSClientProvider(uri, conf);
+        final URI uri = createKMSUri(getKMSUrl());
 
         //nothing allowed
         doAs("client", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
+            KeyProvider kp = new KMSClientProvider(uri, conf);
             try {
               kp.createKey("k", new KeyProvider.Options(conf));
               Assert.fail();
             } catch (AuthorizationException ex) {
               //NOP
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             try {
               kp.createKey("k", new byte[16], new KeyProvider.Options(conf));
@@ -617,7 +657,7 @@ public class TestKMS {
             } catch (AuthorizationException ex) {
               //NOP
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             try {
               kp.rollNewVersion("k");
@@ -625,7 +665,7 @@ public class TestKMS {
             } catch (AuthorizationException ex) {
               //NOP
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             try {
               kp.rollNewVersion("k", new byte[16]);
@@ -633,7 +673,7 @@ public class TestKMS {
             } catch (AuthorizationException ex) {
               //NOP
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             try {
               kp.getKeys();
@@ -641,7 +681,7 @@ public class TestKMS {
             } catch (AuthorizationException ex) {
               //NOP
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             try {
               kp.getKeysMetadata("k");
@@ -649,7 +689,7 @@ public class TestKMS {
             } catch (AuthorizationException ex) {
               //NOP
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             try {
               // we are using JavaKeyStoreProvider for testing, so we know how
@@ -659,7 +699,7 @@ public class TestKMS {
             } catch (AuthorizationException ex) {
               //NOP
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             try {
               kp.getCurrentKey("k");
@@ -667,7 +707,7 @@ public class TestKMS {
             } catch (AuthorizationException ex) {
               //NOP
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             try {
               kp.getMetadata("k");
@@ -675,7 +715,7 @@ public class TestKMS {
             } catch (AuthorizationException ex) {
               //NOP
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             try {
               kp.getKeyVersions("k");
@@ -683,7 +723,7 @@ public class TestKMS {
             } catch (AuthorizationException ex) {
               //NOP
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
 
             return null;
@@ -693,12 +733,13 @@ public class TestKMS {
         doAs("CREATE", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
+            KeyProvider kp = new KMSClientProvider(uri, conf);
             try {
               KeyProvider.KeyVersion kv = kp.createKey("k0",
                   new KeyProvider.Options(conf));
               Assert.assertNull(kv.getMaterial());
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             return null;
           }
@@ -707,10 +748,11 @@ public class TestKMS {
         doAs("DELETE", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
+            KeyProvider kp = new KMSClientProvider(uri, conf);
             try {
               kp.deleteKey("k0");
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             return null;
           }
@@ -719,12 +761,13 @@ public class TestKMS {
         doAs("SET_KEY_MATERIAL", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
+            KeyProvider kp = new KMSClientProvider(uri, conf);
             try {
               KeyProvider.KeyVersion kv = kp.createKey("k1", new byte[16],
                   new KeyProvider.Options(conf));
               Assert.assertNull(kv.getMaterial());
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             return null;
           }
@@ -733,11 +776,12 @@ public class TestKMS {
         doAs("ROLLOVER", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
+            KeyProvider kp = new KMSClientProvider(uri, conf);
             try {
               KeyProvider.KeyVersion kv = kp.rollNewVersion("k1");
               Assert.assertNull(kv.getMaterial());
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             return null;
           }
@@ -746,12 +790,13 @@ public class TestKMS {
         doAs("SET_KEY_MATERIAL", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
+            KeyProvider kp = new KMSClientProvider(uri, conf);
             try {
               KeyProvider.KeyVersion kv =
                   kp.rollNewVersion("k1", new byte[16]);
               Assert.assertNull(kv.getMaterial());
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             return null;
           }
@@ -761,6 +806,7 @@ public class TestKMS {
             doAs("GET", new PrivilegedExceptionAction<KeyVersion>() {
           @Override
           public KeyVersion run() throws Exception {
+            KeyProvider kp = new KMSClientProvider(uri, conf);
             try {
               kp.getKeyVersion("k1@0");
               KeyVersion kv = kp.getCurrentKey("k1");
@@ -777,6 +823,7 @@ public class TestKMS {
                 new PrivilegedExceptionAction<EncryptedKeyVersion>() {
           @Override
           public EncryptedKeyVersion run() throws Exception {
+            KeyProvider kp = new KMSClientProvider(uri, conf);
             try {
               KeyProviderCryptoExtension kpCE = KeyProviderCryptoExtension.
                       createKeyProviderCryptoExtension(kp);
@@ -793,12 +840,13 @@ public class TestKMS {
         doAs("DECRYPT_EEK", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
+            KeyProvider kp = new KMSClientProvider(uri, conf);
             try {
               KeyProviderCryptoExtension kpCE = KeyProviderCryptoExtension.
                       createKeyProviderCryptoExtension(kp);
               kpCE.decryptEncryptedKey(encKv);
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             return null;
           }
@@ -807,10 +855,11 @@ public class TestKMS {
         doAs("GET_KEYS", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
+            KeyProvider kp = new KMSClientProvider(uri, conf);
             try {
               kp.getKeys();
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             return null;
           }
@@ -819,11 +868,12 @@ public class TestKMS {
         doAs("GET_METADATA", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
+            KeyProvider kp = new KMSClientProvider(uri, conf);
             try {
               kp.getMetadata("k1");
               kp.getKeysMetadata("k1");
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             return null;
           }
@@ -836,6 +886,7 @@ public class TestKMS {
         Thread.sleep(10); // to ensure the ACLs file modifiedTime is newer
         conf.set(KMSACLs.Type.CREATE.getConfigKey(), "foo");
         writeConf(testDir, conf);
+        Thread.sleep(1000);
 
         KMSWebApp.getACLs().run(); // forcing a reload by hand.
 
@@ -844,13 +895,14 @@ public class TestKMS {
           @Override
           public Void run() throws Exception {
             try {
+              KeyProvider kp = new KMSClientProvider(uri, conf);
               KeyProvider.KeyVersion kv = kp.createKey("k2",
                   new KeyProvider.Options(conf));
               Assert.fail();
             } catch (AuthorizationException ex) {
               //NOP
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
 
             return null;
@@ -864,8 +916,11 @@ public class TestKMS {
 
   @Test
   public void testServicePrincipalACLs() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("hadoop.security.authentication", "kerberos");
+    UserGroupInformation.setConfiguration(conf);
     File testDir = getTestDir();
-    Configuration conf = createBaseKMSConf(testDir);
+    conf = createBaseKMSConf(testDir);
     conf.set("hadoop.kms.authentication.type", "kerberos");
     conf.set("hadoop.kms.authentication.kerberos.keytab",
         keytab.getAbsolutePath());
@@ -883,18 +938,19 @@ public class TestKMS {
       public Void call() throws Exception {
         final Configuration conf = new Configuration();
         conf.setInt(KeyProvider.DEFAULT_BITLENGTH_NAME, 128);
-        URI uri = createKMSUri(getKMSUrl());
-        final KeyProvider kp = new KMSClientProvider(uri, conf);
+        conf.setInt(KeyProvider.DEFAULT_BITLENGTH_NAME, 64);
+        final URI uri = createKMSUri(getKMSUrl());
 
         doAs("client", new PrivilegedExceptionAction<Void>() {
           @Override
           public Void run() throws Exception {
             try {
+              KeyProvider kp = new KMSClientProvider(uri, conf);
               KeyProvider.KeyVersion kv = kp.createKey("ck0",
                   new KeyProvider.Options(conf));
               Assert.assertNull(kv.getMaterial());
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             return null;
           }
@@ -904,11 +960,12 @@ public class TestKMS {
           @Override
           public Void run() throws Exception {
             try {
+              KeyProvider kp = new KMSClientProvider(uri, conf);
               KeyProvider.KeyVersion kv = kp.createKey("ck1",
                   new KeyProvider.Options(conf));
               Assert.assertNull(kv.getMaterial());
             } catch (Exception ex) {
-              Assert.fail(ex.toString());
+              Assert.fail(ex.getMessage());
             }
             return null;
           }
@@ -982,4 +1039,142 @@ public class TestKMS {
 
     sock.close();
   }
+
+  @Test
+  public void testDelegationTokenAccess() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("hadoop.security.authentication", "kerberos");
+    UserGroupInformation.setConfiguration(conf);
+    final File testDir = getTestDir();
+    conf = createBaseKMSConf(testDir);
+    conf.set("hadoop.kms.authentication.type", "kerberos");
+    conf.set("hadoop.kms.authentication.kerberos.keytab",
+        keytab.getAbsolutePath());
+    conf.set("hadoop.kms.authentication.kerberos.principal", "HTTP/localhost");
+    conf.set("hadoop.kms.authentication.kerberos.name.rules", "DEFAULT");
+
+    writeConf(testDir, conf);
+
+    runServer(null, null, testDir, new KMSCallable() {
+      @Override
+      public Void call() throws Exception {
+        final Configuration conf = new Configuration();
+        conf.setInt(KeyProvider.DEFAULT_BITLENGTH_NAME, 64);
+        final URI uri = createKMSUri(getKMSUrl());
+        final Credentials credentials = new Credentials();
+        final UserGroupInformation nonKerberosUgi =
+            UserGroupInformation.getCurrentUser();
+
+        try {
+          KeyProvider kp = new KMSClientProvider(uri, conf);
+          kp.createKey("kA", new KeyProvider.Options(conf));
+        } catch (IOException ex) {
+          System.out.println(ex.getMessage());
+        }
+
+        doAs("client", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            KeyProvider kp = new KMSClientProvider(uri, conf);
+            KeyProviderDelegationTokenExtension kpdte =
+                KeyProviderDelegationTokenExtension.
+                    createKeyProviderDelegationTokenExtension(kp);
+            kpdte.addDelegationTokens("foo", credentials);
+            return null;
+          }
+        });
+
+        nonKerberosUgi.addCredentials(credentials);
+
+        try {
+          KeyProvider kp = new KMSClientProvider(uri, conf);
+          kp.createKey("kA", new KeyProvider.Options(conf));
+        } catch (IOException ex) {
+          System.out.println(ex.getMessage());
+        }
+
+        nonKerberosUgi.doAs(new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            KeyProvider kp = new KMSClientProvider(uri, conf);
+            kp.createKey("kD", new KeyProvider.Options(conf));
+            return null;
+          }
+        });
+
+        return null;
+      }
+    });
+  }
+
+  @Test
+  public void testProxyUser() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("hadoop.security.authentication", "kerberos");
+    UserGroupInformation.setConfiguration(conf);
+    final File testDir = getTestDir();
+    conf = createBaseKMSConf(testDir);
+    conf.set("hadoop.kms.authentication.type", "kerberos");
+    conf.set("hadoop.kms.authentication.kerberos.keytab",
+        keytab.getAbsolutePath());
+    conf.set("hadoop.kms.authentication.kerberos.principal", "HTTP/localhost");
+    conf.set("hadoop.kms.authentication.kerberos.name.rules", "DEFAULT");
+    conf.set("hadoop.kms.proxyuser.client.users", "foo");
+    conf.set("hadoop.kms.proxyuser.client.hosts", "*");
+    writeConf(testDir, conf);
+
+    runServer(null, null, testDir, new KMSCallable() {
+      @Override
+      public Void call() throws Exception {
+        final Configuration conf = new Configuration();
+        conf.setInt(KeyProvider.DEFAULT_BITLENGTH_NAME, 64);
+        final URI uri = createKMSUri(getKMSUrl());
+
+        // proxyuser client using kerberos credentials
+        UserGroupInformation clientUgi = UserGroupInformation.
+            loginUserFromKeytabAndReturnUGI("client", keytab.getAbsolutePath());
+        clientUgi.doAs(new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            final KeyProvider kp = new KMSClientProvider(uri, conf);
+            kp.createKey("kAA", new KeyProvider.Options(conf));
+
+            // authorized proxyuser
+            UserGroupInformation fooUgi =
+                UserGroupInformation.createRemoteUser("foo");
+            fooUgi.doAs(new PrivilegedExceptionAction<Void>() {
+              @Override
+              public Void run() throws Exception {
+                Assert.assertNotNull(kp.createKey("kBB",
+                    new KeyProvider.Options(conf)));
+                return null;
+              }
+            });
+
+            // unauthorized proxyuser
+            UserGroupInformation foo1Ugi =
+                UserGroupInformation.createRemoteUser("foo1");
+            foo1Ugi.doAs(new PrivilegedExceptionAction<Void>() {
+              @Override
+              public Void run() throws Exception {
+                try {
+                  kp.createKey("kCC", new KeyProvider.Options(conf));
+                  Assert.fail();
+                } catch (AuthorizationException ex) {
+                  // OK
+                } catch (Exception ex) {
+                  Assert.fail(ex.getMessage());
+                }
+                return null;
+              }
+            });
+            return null;
+          }
+        });
+
+        return null;
+      }
+    });
+  }
+
 }

Modified: hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSACLs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSACLs.java?rev=1618700&r1=1618699&r2=1618700&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSACLs.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSACLs.java Mon Aug 18 18:41:31 2014
@@ -18,6 +18,7 @@
 package org.apache.hadoop.crypto.key.kms.server;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -27,7 +28,8 @@ public class TestKMSACLs {
   public void testDefaults() {
     KMSACLs acls = new KMSACLs(new Configuration(false));
     for (KMSACLs.Type type : KMSACLs.Type.values()) {
-      Assert.assertTrue(acls.hasAccess(type, "foo"));
+      Assert.assertTrue(acls.hasAccess(type,
+          UserGroupInformation.createRemoteUser("foo")));
     }
   }
 
@@ -39,8 +41,10 @@ public class TestKMSACLs {
     }
     KMSACLs acls = new KMSACLs(conf);
     for (KMSACLs.Type type : KMSACLs.Type.values()) {
-      Assert.assertTrue(acls.hasAccess(type, type.toString()));
-      Assert.assertFalse(acls.hasAccess(type, "foo"));
+      Assert.assertTrue(acls.hasAccess(type,
+          UserGroupInformation.createRemoteUser(type.toString())));
+      Assert.assertFalse(acls.hasAccess(type,
+          UserGroupInformation.createRemoteUser("foo")));
     }
   }
 

Modified: hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java?rev=1618700&r1=1618699&r2=1618700&view=diff
==============================================================================
--- hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java (original)
+++ hadoop/common/branches/fs-encryption/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java Mon Aug 18 18:41:31 2014
@@ -21,9 +21,9 @@ import java.io.ByteArrayOutputStream;
 import java.io.FilterOutputStream;
 import java.io.OutputStream;
 import java.io.PrintStream;
-import java.security.Principal;
 
 import org.apache.hadoop.crypto.key.kms.server.KMS.KMSOp;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.PropertyConfigurator;
 import org.junit.After;
@@ -81,8 +81,8 @@ public class TestKMSAudit {
 
   @Test
   public void testAggregation() throws Exception {
-    Principal luser = Mockito.mock(Principal.class);
-    Mockito.when(luser.getName()).thenReturn("luser");
+    UserGroupInformation luser = Mockito.mock(UserGroupInformation.class);
+    Mockito.when(luser.getShortUserName()).thenReturn("luser");
     kmsAudit.ok(luser, KMSOp.DECRYPT_EEK, "k1", "testmsg");
     kmsAudit.ok(luser, KMSOp.DECRYPT_EEK, "k1", "testmsg");
     kmsAudit.ok(luser, KMSOp.DECRYPT_EEK, "k1", "testmsg");
@@ -109,8 +109,8 @@ public class TestKMSAudit {
 
   @Test
   public void testAggregationUnauth() throws Exception {
-    Principal luser = Mockito.mock(Principal.class);
-    Mockito.when(luser.getName()).thenReturn("luser");
+    UserGroupInformation luser = Mockito.mock(UserGroupInformation.class);
+    Mockito.when(luser.getShortUserName()).thenReturn("luser");
     kmsAudit.unauthorized(luser, KMSOp.GENERATE_EEK, "k2");
     Thread.sleep(1000);
     kmsAudit.ok(luser, KMSOp.GENERATE_EEK, "k3", "testmsg");