Browse Source

HADOOP-10433. Key Management Server based on KeyProvider API. (tucu)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1592637 13f79535-47bb-0310-9956-ffa450edef68
Alejandro Abdelnur 11 years ago
parent
commit
17d4fbbf0a
44 changed files with 5115 additions and 1 deletions
  1. 1 0
      .gitignore
  2. 52 0
      hadoop-assemblies/src/main/resources/assemblies/hadoop-kms-dist.xml
  3. 2 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  4. 6 0
      hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml
  5. 519 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
  6. 53 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSRESTConstants.java
  7. 1 0
      hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.crypto.key.KeyProviderFactory
  8. 41 0
      hadoop-common-project/hadoop-kms/dev-support/findbugsExcludeFile.xml
  9. 408 0
      hadoop-common-project/hadoop-kms/pom.xml
  10. 82 0
      hadoop-common-project/hadoop-kms/src/main/conf/kms-acls.xml
  11. 45 0
      hadoop-common-project/hadoop-kms/src/main/conf/kms-env.sh
  12. 38 0
      hadoop-common-project/hadoop-kms/src/main/conf/kms-log4j.properties
  13. 71 0
      hadoop-common-project/hadoop-kms/src/main/conf/kms-site.xml
  14. 305 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
  15. 133 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java
  16. 62 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java
  17. 123 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuthenticationFilter.java
  18. 180 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSCacheKeyProvider.java
  19. 94 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java
  20. 113 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSExceptionsProvider.java
  21. 54 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJSONReader.java
  22. 70 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJSONWriter.java
  23. 92 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSMDCFilter.java
  24. 80 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSServerJSONUtils.java
  25. 214 0
      hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java
  26. 181 0
      hadoop-common-project/hadoop-kms/src/main/libexec/kms-config.sh
  27. 60 0
      hadoop-common-project/hadoop-kms/src/main/sbin/kms.sh
  28. 16 0
      hadoop-common-project/hadoop-kms/src/main/tomcat/ROOT/WEB-INF/web.xml
  29. 27 0
      hadoop-common-project/hadoop-kms/src/main/tomcat/ROOT/index.html
  30. 67 0
      hadoop-common-project/hadoop-kms/src/main/tomcat/logging.properties
  31. 153 0
      hadoop-common-project/hadoop-kms/src/main/tomcat/server.xml
  32. 135 0
      hadoop-common-project/hadoop-kms/src/main/tomcat/ssl-server.xml
  33. 78 0
      hadoop-common-project/hadoop-kms/src/main/webapp/WEB-INF/web.xml
  34. 487 0
      hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm
  35. 29 0
      hadoop-common-project/hadoop-kms/src/site/resources/css/site.css
  36. 29 0
      hadoop-common-project/hadoop-kms/src/site/site.xml
  37. 806 0
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java
  38. 47 0
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSACLs.java
  39. 120 0
      hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSCacheKeyProvider.java
  40. 31 0
      hadoop-common-project/hadoop-kms/src/test/resources/log4j.properties
  41. 1 0
      hadoop-common-project/pom.xml
  42. 1 0
      hadoop-dist/pom.xml
  43. 7 1
      hadoop-project/pom.xml
  44. 1 0
      hadoop-project/src/site/site.xml

+ 1 - 0
.gitignore

@@ -7,5 +7,6 @@
 .project
 .settings
 target
+hadoop-common-project/hadoop-kms/downloads/
 hadoop-hdfs-project/hadoop-hdfs/downloads
 hadoop-hdfs-project/hadoop-hdfs-httpfs/downloads

+ 52 - 0
hadoop-assemblies/src/main/resources/assemblies/hadoop-kms-dist.xml

@@ -0,0 +1,52 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<assembly>
+  <id>hadoop-kms-dist</id>
+  <formats>
+    <format>dir</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <fileSets>
+    <!-- Configuration files -->
+    <fileSet>
+      <directory>${basedir}/src/main/conf</directory>
+      <outputDirectory>/etc/hadoop</outputDirectory>
+      <includes>
+        <include>*</include>
+      </includes>
+    </fileSet>
+    <fileSet>
+      <directory>${basedir}/src/main/sbin</directory>
+      <outputDirectory>/sbin</outputDirectory>
+      <includes>
+        <include>*</include>
+      </includes>
+      <fileMode>0755</fileMode>
+    </fileSet>
+    <fileSet>
+      <directory>${basedir}/src/main/libexec</directory>
+      <outputDirectory>/libexec</outputDirectory>
+      <includes>
+        <include>*</include>
+      </includes>
+      <fileMode>0755</fileMode>
+    </fileSet>
+    <!-- Documentation -->
+    <fileSet>
+      <directory>${project.build.directory}/site</directory>
+      <outputDirectory>/share/doc/hadoop/kms</outputDirectory>
+    </fileSet>
+  </fileSets>
+</assembly>

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

@@ -8,6 +8,8 @@ Trunk (Unreleased)
     FSDataOutputStream.sync() and Syncable.sync().  (szetszwo)
 
   NEW FEATURES
+
+    HADOOP-10433. Key Management Server based on KeyProvider API. (tucu)
     
   IMPROVEMENTS
 

+ 6 - 0
hadoop-common-project/hadoop-common/dev-support/findbugsExcludeFile.xml

@@ -357,4 +357,10 @@
        <Bug code="NP" />
      </Match>
 
+  <Match>
+    <Class name="org.apache.hadoop.crypto.key.kms.KMSClientProvider"/>
+    <Method name="validateResponse"/>
+    <Bug pattern="REC_CATCH_EXCEPTION"/>
+  </Match>
+
 </FindBugsFilter>

+ 519 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java

@@ -0,0 +1,519 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProviderFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.security.authentication.client.PseudoAuthenticator;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.http.client.utils.URIBuilder;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import javax.net.ssl.HttpsURLConnection;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.lang.reflect.Constructor;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.security.GeneralSecurityException;
+import java.security.NoSuchAlgorithmException;
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * KMS client <code>KeyProvider</code> implementation.
+ */
+@InterfaceAudience.Private
+public class KMSClientProvider extends KeyProvider {
+
+  public static final String SCHEME_NAME = "kms";
+
+  private static final String UTF8 = "UTF-8";
+
+  private static final String CONTENT_TYPE = "Content-Type";
+  private static final String APPLICATION_JSON_MIME = "application/json";
+
+  private static final String HTTP_GET = "GET";
+  private static final String HTTP_POST = "POST";
+  private static final String HTTP_PUT = "PUT";
+  private static final String HTTP_DELETE = "DELETE";
+
+  private static KeyVersion parseJSONKeyVersion(Map valueMap) {
+    KeyVersion keyVersion = null;
+    if (!valueMap.isEmpty()) {
+      byte[] material = (valueMap.containsKey(KMSRESTConstants.MATERIAL_FIELD))
+          ? Base64.decodeBase64((String) valueMap.get(KMSRESTConstants.MATERIAL_FIELD))
+          : null;
+      keyVersion = new KMSKeyVersion((String)
+          valueMap.get(KMSRESTConstants.VERSION_NAME_FIELD), material);
+    }
+    return keyVersion;
+  }
+
+  private static Metadata parseJSONMetadata(Map valueMap) {
+    Metadata metadata = null;
+    if (!valueMap.isEmpty()) {
+      metadata = new KMSMetadata(
+          (String) valueMap.get(KMSRESTConstants.CIPHER_FIELD),
+          (Integer) valueMap.get(KMSRESTConstants.LENGTH_FIELD),
+          (String) valueMap.get(KMSRESTConstants.DESCRIPTION_FIELD),
+          new Date((Long) valueMap.get(KMSRESTConstants.CREATED_FIELD)),
+          (Integer) valueMap.get(KMSRESTConstants.VERSIONS_FIELD));
+    }
+    return metadata;
+  }
+
+  private static void writeJson(Map map, OutputStream os) throws IOException {
+    Writer writer = new OutputStreamWriter(os);
+    ObjectMapper jsonMapper = new ObjectMapper();
+    jsonMapper.writerWithDefaultPrettyPrinter().writeValue(writer, map);
+  }
+
+  /**
+   * The factory to create KMSClientProvider, which is used by the
+   * ServiceLoader.
+   */
+  public static class Factory extends KeyProviderFactory {
+
+    @Override
+    public KeyProvider createProvider(URI providerName, Configuration conf)
+        throws IOException {
+      if (SCHEME_NAME.equals(providerName.getScheme())) {
+        return new KMSClientProvider(providerName, conf);
+      }
+      return null;
+    }
+  }
+
+  public static <T> T checkNotNull(T o, String name)
+      throws IllegalArgumentException {
+    if (o == null) {
+      throw new IllegalArgumentException("Parameter '" + name +
+          "' cannot be null");
+    }
+    return o;
+  }
+
+
+  public static String checkNotEmpty(String s, String name)
+      throws IllegalArgumentException {
+    checkNotNull(s, name);
+    if (s.isEmpty()) {
+      throw new IllegalArgumentException("Parameter '" + name +
+          "' cannot be empty");
+    }
+    return s;
+  }
+
+  private String kmsUrl;
+  private SSLFactory sslFactory;
+
+  public KMSClientProvider(URI uri, Configuration conf) throws IOException {
+    Path path = unnestUri(uri);
+    URL url = path.toUri().toURL();
+    kmsUrl = createServiceURL(url);
+    if ("https".equalsIgnoreCase(url.getProtocol())) {
+      sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
+      try {
+        sslFactory.init();
+      } catch (GeneralSecurityException ex) {
+        throw new IOException(ex);
+      }
+    }
+  }
+
+  private String createServiceURL(URL url) throws IOException {
+    String str = url.toExternalForm();
+    if (str.endsWith("/")) {
+      str = str.substring(0, str.length() - 1);
+    }
+    return new URL(str + KMSRESTConstants.SERVICE_VERSION + "/").
+        toExternalForm();
+  }
+
+  private URL createURL(String collection, String resource, String subResource,
+      Map<String, ?> parameters) throws IOException {
+    try {
+      StringBuilder sb = new StringBuilder();
+      sb.append(kmsUrl);
+      sb.append(collection);
+      if (resource != null) {
+        sb.append("/").append(URLEncoder.encode(resource, UTF8));
+      }
+      if (subResource != null) {
+        sb.append("/").append(subResource);
+      }
+      URIBuilder uriBuilder = new URIBuilder(sb.toString());
+      if (parameters != null) {
+        for (Map.Entry<String, ?> param : parameters.entrySet()) {
+          Object value = param.getValue();
+          if (value instanceof String) {
+            uriBuilder.addParameter(param.getKey(), (String) value);
+          } else {
+            for (String s : (String[]) value) {
+              uriBuilder.addParameter(param.getKey(), s);
+            }
+          }
+        }
+      }
+      return uriBuilder.build().toURL();
+    } catch (URISyntaxException ex) {
+      throw new IOException(ex);
+    }
+  }
+
+  private HttpURLConnection configureConnection(HttpURLConnection conn)
+      throws IOException {
+    if (sslFactory != null) {
+      HttpsURLConnection httpsConn = (HttpsURLConnection) conn;
+      try {
+        httpsConn.setSSLSocketFactory(sslFactory.createSSLSocketFactory());
+      } catch (GeneralSecurityException ex) {
+        throw new IOException(ex);
+      }
+      httpsConn.setHostnameVerifier(sslFactory.getHostnameVerifier());
+    }
+    return conn;
+  }
+
+  private HttpURLConnection createConnection(URL url, String method)
+      throws IOException {
+    HttpURLConnection conn;
+    try {
+      AuthenticatedURL authUrl = new AuthenticatedURL(new PseudoAuthenticator(),
+          sslFactory);
+      conn = authUrl.openConnection(url, new AuthenticatedURL.Token());
+    } catch (AuthenticationException ex) {
+      throw new IOException(ex);
+    }
+    conn.setUseCaches(false);
+    conn.setRequestMethod(method);
+    if (method.equals(HTTP_POST) || method.equals(HTTP_PUT)) {
+      conn.setDoOutput(true);
+    }
+    conn = configureConnection(conn);
+    return conn;
+  }
+
+  // trick, riding on generics to throw an undeclared exception
+
+  private static void throwEx(Throwable ex) {
+    KMSClientProvider.<RuntimeException>throwException(ex);
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <E extends Throwable> void throwException(Throwable ex)
+      throws E {
+    throw (E) ex;
+  }
+
+  @SuppressWarnings("unchecked")
+  private static void validateResponse(HttpURLConnection conn, int expected)
+      throws IOException {
+    int status = conn.getResponseCode();
+    if (status != expected) {
+      InputStream es = null;
+      try {
+        es = conn.getErrorStream();
+        ObjectMapper mapper = new ObjectMapper();
+        Map json = mapper.readValue(es, Map.class);
+        String exClass = (String) json.get(
+            KMSRESTConstants.ERROR_EXCEPTION_JSON);
+        String exMsg = (String)
+            json.get(KMSRESTConstants.ERROR_MESSAGE_JSON);
+        Exception toThrow;
+        try {
+          ClassLoader cl = KMSClientProvider.class.getClassLoader();
+          Class klass = cl.loadClass(exClass);
+          Constructor constr = klass.getConstructor(String.class);
+          toThrow = (Exception) constr.newInstance(exMsg);
+        } catch (Exception ex) {
+          toThrow = new IOException(MessageFormat.format(
+              "HTTP status [{0}], {1}", status, conn.getResponseMessage()));
+        }
+        throwEx(toThrow);
+      } finally {
+        if (es != null) {
+          es.close();
+        }
+      }
+    }
+  }
+
+  private static <T> T call(HttpURLConnection conn, Map jsonOutput,
+      int expectedResponse, Class<T> klass)
+      throws IOException {
+    T ret = null;
+    try {
+      if (jsonOutput != null) {
+        writeJson(jsonOutput, conn.getOutputStream());
+      }
+    } catch (IOException ex) {
+      conn.getInputStream().close();
+      throw ex;
+    }
+    validateResponse(conn, expectedResponse);
+    if (APPLICATION_JSON_MIME.equalsIgnoreCase(conn.getContentType())
+        && klass != null) {
+      ObjectMapper mapper = new ObjectMapper();
+      InputStream is = null;
+      try {
+        is = conn.getInputStream();
+        ret = mapper.readValue(is, klass);
+      } catch (IOException ex) {
+        if (is != null) {
+          is.close();
+        }
+        throw ex;
+      } finally {
+        if (is != null) {
+          is.close();
+        }
+      }
+    }
+    return ret;
+  }
+
+  public static class KMSKeyVersion extends KeyVersion {
+    public KMSKeyVersion(String versionName, byte[] material) {
+      super(versionName, material);
+    }
+  }
+
+  @Override
+  public KeyVersion getKeyVersion(String versionName) throws IOException {
+    checkNotEmpty(versionName, "versionName");
+    URL url = createURL(KMSRESTConstants.KEY_VERSION_RESOURCE,
+        versionName, null, null);
+    HttpURLConnection conn = createConnection(url, HTTP_GET);
+    Map response = call(conn, null, HttpURLConnection.HTTP_OK, Map.class);
+    return parseJSONKeyVersion(response);
+  }
+
+  @Override
+  public KeyVersion getCurrentKey(String name) throws IOException {
+    checkNotEmpty(name, "name");
+    URL url = createURL(KMSRESTConstants.KEY_RESOURCE, name,
+        KMSRESTConstants.CURRENT_VERSION_SUB_RESOURCE, null);
+    HttpURLConnection conn = createConnection(url, HTTP_GET);
+    Map response = call(conn, null, HttpURLConnection.HTTP_OK, Map.class);
+    return parseJSONKeyVersion(response);
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public List<String> getKeys() throws IOException {
+    URL url = createURL(KMSRESTConstants.KEYS_NAMES_RESOURCE, null, null,
+        null);
+    HttpURLConnection conn = createConnection(url, HTTP_GET);
+    List response = call(conn, null, HttpURLConnection.HTTP_OK, List.class);
+    return (List<String>) response;
+  }
+
+  public static class KMSMetadata extends Metadata {
+    public KMSMetadata(String cipher, int bitLength, String description,
+        Date created, int versions) {
+      super(cipher, bitLength, description, created, versions);
+    }
+  }
+
+  // breaking keyNames into sets to keep resulting URL undler 2000 chars
+  private List<String[]> createKeySets(String[] keyNames) {
+    List<String[]> list = new ArrayList<String[]>();
+    List<String> batch = new ArrayList<String>();
+    int batchLen = 0;
+    for (String name : keyNames) {
+      int additionalLen = KMSRESTConstants.KEY_OP.length() + 1 + name.length();
+      batchLen += additionalLen;
+      // topping at 1500 to account for initial URL and encoded names
+      if (batchLen > 1500) {
+        list.add(batch.toArray(new String[batch.size()]));
+        batch = new ArrayList<String>();
+        batchLen = additionalLen;
+      }
+      batch.add(name);
+    }
+    if (!batch.isEmpty()) {
+      list.add(batch.toArray(new String[batch.size()]));
+    }
+    return list;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public Metadata[] getKeysMetadata(String ... keyNames) throws IOException {
+    List<Metadata> keysMetadata = new ArrayList<Metadata>();
+    List<String[]> keySets = createKeySets(keyNames);
+    for (String[] keySet : keySets) {
+      if (keyNames.length > 0) {
+        Map<String, Object> queryStr = new HashMap<String, Object>();
+        queryStr.put(KMSRESTConstants.KEY_OP, keySet);
+        URL url = createURL(KMSRESTConstants.KEYS_METADATA_RESOURCE, null,
+            null, queryStr);
+        HttpURLConnection conn = createConnection(url, HTTP_GET);
+        List<Map> list = call(conn, null, HttpURLConnection.HTTP_OK, List.class);
+        for (Map map : list) {
+          keysMetadata.add(parseJSONMetadata(map));
+        }
+      }
+    }
+    return keysMetadata.toArray(new Metadata[keysMetadata.size()]);
+  }
+
+  private KeyVersion createKeyInternal(String name, byte[] material,
+      Options options)
+      throws NoSuchAlgorithmException, IOException {
+    checkNotEmpty(name, "name");
+    checkNotNull(options, "options");
+    Map<String, Object> jsonKey = new HashMap<String, Object>();
+    jsonKey.put(KMSRESTConstants.NAME_FIELD, name);
+    jsonKey.put(KMSRESTConstants.CIPHER_FIELD, options.getCipher());
+    jsonKey.put(KMSRESTConstants.LENGTH_FIELD, options.getBitLength());
+    if (material != null) {
+      jsonKey.put(KMSRESTConstants.MATERIAL_FIELD,
+          Base64.encodeBase64String(material));
+    }
+    if (options.getDescription() != null) {
+      jsonKey.put(KMSRESTConstants.DESCRIPTION_FIELD,
+          options.getDescription());
+    }
+    URL url = createURL(KMSRESTConstants.KEYS_RESOURCE, null, null, null);
+    HttpURLConnection conn = createConnection(url, HTTP_POST);
+    conn.setRequestProperty(CONTENT_TYPE, APPLICATION_JSON_MIME);
+    Map response = call(conn, jsonKey, HttpURLConnection.HTTP_CREATED,
+        Map.class);
+    return parseJSONKeyVersion(response);
+  }
+
+  @Override
+  public KeyVersion createKey(String name, Options options)
+      throws NoSuchAlgorithmException, IOException {
+    return createKeyInternal(name, null, options);
+  }
+
+  @Override
+  public KeyVersion createKey(String name, byte[] material, Options options)
+      throws IOException {
+    checkNotNull(material, "material");
+    try {
+      return createKeyInternal(name, material, options);
+    } catch (NoSuchAlgorithmException ex) {
+      throw new RuntimeException("It should not happen", ex);
+    }
+  }
+
+  private KeyVersion rollNewVersionInternal(String name, byte[] material)
+      throws NoSuchAlgorithmException, IOException {
+    checkNotEmpty(name, "name");
+    Map<String, String> jsonMaterial = new HashMap<String, String>();
+    if (material != null) {
+      jsonMaterial.put(KMSRESTConstants.MATERIAL_FIELD,
+          Base64.encodeBase64String(material));
+    }
+    URL url = createURL(KMSRESTConstants.KEY_RESOURCE, name, null, null);
+    HttpURLConnection conn = createConnection(url, HTTP_POST);
+    conn.setRequestProperty(CONTENT_TYPE, APPLICATION_JSON_MIME);
+    Map response = call(conn, jsonMaterial,
+        HttpURLConnection.HTTP_OK, Map.class);
+    return parseJSONKeyVersion(response);
+  }
+
+
+  @Override
+  public KeyVersion rollNewVersion(String name)
+      throws NoSuchAlgorithmException, IOException {
+    return rollNewVersionInternal(name, null);
+  }
+
+  @Override
+  public KeyVersion rollNewVersion(String name, byte[] material)
+      throws IOException {
+    checkNotNull(material, "material");
+    try {
+      return rollNewVersionInternal(name, material);
+    } catch (NoSuchAlgorithmException ex) {
+      throw new RuntimeException("It should not happen", ex);
+    }
+  }
+
+  @Override
+  public List<KeyVersion> getKeyVersions(String name) throws IOException {
+    checkNotEmpty(name, "name");
+    URL url = createURL(KMSRESTConstants.KEY_RESOURCE, name,
+        KMSRESTConstants.VERSIONS_SUB_RESOURCE, null);
+    HttpURLConnection conn = createConnection(url, HTTP_GET);
+    List response = call(conn, null, HttpURLConnection.HTTP_OK, List.class);
+    List<KeyVersion> versions = null;
+    if (!response.isEmpty()) {
+      versions = new ArrayList<KeyVersion>();
+      for (Object obj : response) {
+        versions.add(parseJSONKeyVersion((Map) obj));
+      }
+    }
+    return versions;
+  }
+
+  @Override
+  public Metadata getMetadata(String name) throws IOException {
+    checkNotEmpty(name, "name");
+    URL url = createURL(KMSRESTConstants.KEY_RESOURCE, name,
+        KMSRESTConstants.METADATA_SUB_RESOURCE, null);
+    HttpURLConnection conn = createConnection(url, HTTP_GET);
+    Map response = call(conn, null, HttpURLConnection.HTTP_OK, Map.class);
+    return parseJSONMetadata(response);
+  }
+
+  @Override
+  public void deleteKey(String name) throws IOException {
+    checkNotEmpty(name, "name");
+    URL url = createURL(KMSRESTConstants.KEY_RESOURCE, name, null, null);
+    HttpURLConnection conn = createConnection(url, HTTP_DELETE);
+    call(conn, null, HttpURLConnection.HTTP_OK, null);
+  }
+
+  @Override
+  public void flush() throws IOException {
+    // NOP
+    // the client does not keep any local state, thus flushing is not required
+    // because of the client.
+    // the server should not keep in memory state on behalf of clients either.
+  }
+
+  @VisibleForTesting
+  public static String buildVersionName(String name, int version) {
+    return KeyProvider.buildVersionName(name, version);
+  }
+
+}

+ 53 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSRESTConstants.java

@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * KMS REST and JSON constants and utility methods for the KMSServer.
+ */
+@InterfaceAudience.Private
+public class KMSRESTConstants {
+
+  public static final String SERVICE_VERSION = "/v1";
+  public static final String KEY_RESOURCE = "key";
+  public static final String KEYS_RESOURCE = "keys";
+  public static final String KEYS_METADATA_RESOURCE = KEYS_RESOURCE +
+      "/metadata";
+  public static final String KEYS_NAMES_RESOURCE = KEYS_RESOURCE + "/names";
+  public static final String KEY_VERSION_RESOURCE = "keyversion";
+  public static final String METADATA_SUB_RESOURCE = "_metadata";
+  public static final String VERSIONS_SUB_RESOURCE = "_versions";
+  public static final String CURRENT_VERSION_SUB_RESOURCE = "_currentversion";
+
+  public static final String KEY_OP = "key";
+
+  public static final String NAME_FIELD = "name";
+  public static final String CIPHER_FIELD = "cipher";
+  public static final String LENGTH_FIELD = "length";
+  public static final String DESCRIPTION_FIELD = "description";
+  public static final String CREATED_FIELD = "created";
+  public static final String VERSIONS_FIELD = "versions";
+  public static final String MATERIAL_FIELD = "material";
+  public static final String VERSION_NAME_FIELD = "versionName";
+
+  public static final String ERROR_EXCEPTION_JSON = "exception";
+  public static final String ERROR_MESSAGE_JSON = "message";
+
+}

+ 1 - 0
hadoop-common-project/hadoop-common/src/main/resources/META-INF/services/org.apache.hadoop.crypto.key.KeyProviderFactory

@@ -15,3 +15,4 @@
 
 org.apache.hadoop.crypto.key.JavaKeyStoreProvider$Factory
 org.apache.hadoop.crypto.key.UserProvider$Factory
+org.apache.hadoop.crypto.key.kms.KMSClientProvider$Factory

+ 41 - 0
hadoop-common-project/hadoop-kms/dev-support/findbugsExcludeFile.xml

@@ -0,0 +1,41 @@
+<!--
+   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.
+-->
+<FindBugsFilter>
+  <!--
+    Findbug is complaining about System.out being NULL
+  -->
+  <Match>
+    <Class name="org.apache.hadoop.crypto.key.kms.server.KMSWebApp"/>
+    <Bug pattern="NP_ALWAYS_NULL"/>
+  </Match>
+  <!--
+    KMSWebApp is a webapp singleton managed by the servlet container via
+    ServletContextListener.
+  -->
+  <Match>
+    <Class name="org.apache.hadoop.crypto.key.kms.server.KMSWebApp"/>
+    <Bug pattern="ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD"/>
+  </Match>
+  <!--
+    KMSWebApp does an exit to kill the servlet container if the initialization
+    fails.
+  -->
+  <Match>
+    <Class name="org.apache.hadoop.crypto.key.kms.server.KMSWebApp"/>
+    <Bug pattern="DM_EXIT"/>
+  </Match>
+</FindBugsFilter>

+ 408 - 0
hadoop-common-project/hadoop-kms/pom.xml

@@ -0,0 +1,408 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0
+                      http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-project</artifactId>
+    <version>3.0.0-SNAPSHOT</version>
+    <relativePath>../../hadoop-project</relativePath>
+  </parent>
+  <groupId>org.apache.hadoop</groupId>
+  <artifactId>hadoop-kms</artifactId>
+  <version>3.0.0-SNAPSHOT</version>
+  <packaging>war</packaging>
+
+  <name>Apache Hadoop KMS</name>
+  <description>Apache Hadoop KMS</description>
+
+  <properties>
+    <tomcat.version>6.0.36</tomcat.version>
+    <kms.tomcat.dist.dir>
+      ${project.build.directory}/${project.artifactId}-${project.version}/share/hadoop/kms/tomcat
+    </kms.tomcat.dist.dir>
+    <tomcat.download.url>
+      http://archive.apache.org/dist/tomcat/tomcat-6/v${tomcat.version}/bin/apache-tomcat-${tomcat.version}.tar.gz
+    </tomcat.download.url>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minikdc</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-auth</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-core</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.sun.jersey</groupId>
+      <artifactId>jersey-server</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>javax.servlet</groupId>
+      <artifactId>servlet-api</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>compile</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.xml.stream</groupId>
+          <artifactId>stax-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-httpclient</groupId>
+          <artifactId>commons-httpclient</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-compiler</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>tomcat</groupId>
+          <artifactId>jasper-runtime</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>servlet-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>jsp-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet.jsp</groupId>
+          <artifactId>jsp-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jetty</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jetty-util</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>jsp-api-2.1</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>servlet-api-2.5</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>net.java.dev.jets3t</groupId>
+          <artifactId>jets3t</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.eclipse.jdt</groupId>
+          <artifactId>core</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>commons-el</groupId>
+          <artifactId>commons-el</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>jul-to-slf4j</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty-util</artifactId>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.codahale.metrics</groupId>
+      <artifactId>metrics-core</artifactId>
+      <scope>compile</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <forkMode>always</forkMode>
+          <threadCount>1</threadCount>
+          <forkedProcessTimeoutInSeconds>600</forkedProcessTimeoutInSeconds>
+          <properties>
+            <property>
+              <name>listener</name>
+              <value>org.apache.hadoop.test.TimedOutTestsListener</value>
+            </property>
+          </properties>
+          <excludes>
+            <exclude>**/${test.exclude}.java</exclude>
+            <exclude>${test.exclude.pattern}</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>create-web-xmls</id>
+            <phase>generate-test-resources</phase>
+            <goals>
+              <goal>run</goal>
+            </goals>
+            <configuration>
+              <target>
+                <mkdir dir="${project.build.directory}/test-classes/webapp"/>
+
+                <copy todir="${project.build.directory}/test-classes/webapp">
+                  <fileset dir="${basedir}/src/main/webapp"/>
+                </copy>
+              </target>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-war-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>default-war</id>
+            <phase>package</phase>
+            <goals>
+              <goal>war</goal>
+            </goals>
+            <configuration>
+              <warName>kms</warName>
+              <webappDirectory>${project.build.directory}/kms
+              </webappDirectory>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>findbugs-maven-plugin</artifactId>
+        <configuration>
+          <excludeFilterFile>${basedir}/dev-support/findbugsExcludeFile.xml
+          </excludeFilterFile>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
+  <profiles>
+    <profile>
+      <id>docs</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-site-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>docs</id>
+                <phase>prepare-package</phase>
+                <goals>
+                  <goal>site</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+
+    <profile>
+      <id>dist</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-assembly-plugin</artifactId>
+            <dependencies>
+              <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-assemblies</artifactId>
+                <version>${project.version}</version>
+              </dependency>
+            </dependencies>
+            <executions>
+              <execution>
+                <id>dist</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>single</goal>
+                </goals>
+                <configuration>
+                  <finalName>${project.artifactId}-${project.version}
+                  </finalName>
+                  <appendAssemblyId>false</appendAssemblyId>
+                  <attach>false</attach>
+                  <descriptorRefs>
+                    <descriptorRef>hadoop-kms-dist</descriptorRef>
+                  </descriptorRefs>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <!-- Downloading Tomcat TAR.GZ, using downloads/ dir to avoid downloading over an over -->
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>dist</id>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+                <phase>package</phase>
+                <configuration>
+                  <target>
+                    <mkdir dir="downloads"/>
+                    <get
+                      src="${tomcat.download.url}"
+                      dest="downloads/apache-tomcat-${tomcat.version}.tar.gz"
+                      verbose="true" skipexisting="true"/>
+                    <delete dir="${project.build.directory}/tomcat.exp"/>
+                    <mkdir dir="${project.build.directory}/tomcat.exp"/>
+
+                    <!-- Using Unix script to preserve file permissions -->
+                    <echo file="${project.build.directory}/tomcat-untar.sh">
+                      cd "${project.build.directory}/tomcat.exp"
+                      gzip -cd ../../downloads/apache-tomcat-${tomcat.version}.tar.gz | tar xf -
+                    </echo>
+                    <exec executable="sh" dir="${project.build.directory}"
+                          failonerror="true">
+                      <arg line="./tomcat-untar.sh"/>
+                    </exec>
+
+                    <move
+                      file="${project.build.directory}/tomcat.exp/apache-tomcat-${tomcat.version}"
+                      tofile="${kms.tomcat.dist.dir}"/>
+                    <delete dir="${project.build.directory}/tomcat.exp"/>
+                    <delete dir="${kms.tomcat.dist.dir}/webapps"/>
+                    <mkdir dir="${kms.tomcat.dist.dir}/webapps"/>
+                    <delete file="${kms.tomcat.dist.dir}/conf/server.xml"/>
+                    <copy file="${basedir}/src/main/tomcat/server.xml"
+                          toDir="${kms.tomcat.dist.dir}/conf"/>
+                    <delete file="${kms.tomcat.dist.dir}/conf/ssl-server.xml"/>
+                    <copy file="${basedir}/src/main/tomcat/ssl-server.xml"
+                          toDir="${kms.tomcat.dist.dir}/conf"/>
+                    <delete
+                      file="${kms.tomcat.dist.dir}/conf/logging.properties"/>
+                    <copy file="${basedir}/src/main/tomcat/logging.properties"
+                          toDir="${kms.tomcat.dist.dir}/conf"/>
+                    <copy toDir="${kms.tomcat.dist.dir}/webapps/ROOT">
+                      <fileset dir="${basedir}/src/main/tomcat/ROOT"/>
+                    </copy>
+                    <copy toDir="${kms.tomcat.dist.dir}/webapps/kms">
+                      <fileset dir="${project.build.directory}/kms"/>
+                    </copy>
+                  </target>
+                </configuration>
+              </execution>
+              <execution>
+                <id>tar</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+                <configuration>
+                  <target if="tar">
+                    <!-- Using Unix script to preserve symlinks -->
+                    <echo file="${project.build.directory}/dist-maketar.sh">
+                      cd "${project.build.directory}"
+                      tar cf - ${project.artifactId}-${project.version} | gzip > ${project.artifactId}-${project.version}.tar.gz
+                    </echo>
+                    <exec executable="sh" dir="${project.build.directory}"
+                          failonerror="true">
+                      <arg line="./dist-maketar.sh"/>
+                    </exec>
+                  </target>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+</project>

+ 82 - 0
hadoop-common-project/hadoop-kms/src/main/conf/kms-acls.xml

@@ -0,0 +1,82 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<configuration>
+
+  <!-- This file is hot-reloaded when it changes -->
+
+  <!-- KMS ACLs -->
+
+  <property>
+    <name>hadoop.kms.acl.CREATE</name>
+    <value>*</value>
+    <description>
+      ACL for create-key operations.
+      If the user does is not in the GET ACL, the key material is not returned
+      as part of the response.
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.kms.acl.DELETE</name>
+    <value>*</value>
+    <description>
+      ACL for delete-key operations.
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.kms.acl.ROLLOVER</name>
+    <value>*</value>
+    <description>
+      ACL for rollover-key operations.
+      If the user does is not in the GET ACL, the key material is not returned
+      as part of the response.
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.kms.acl.GET</name>
+    <value>*</value>
+    <description>
+      ACL for get-key-version and get-current-key operations.
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.kms.acl.GET_KEYS</name>
+    <value>*</value>
+    <description>
+      ACL for get-keys operation.
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.kms.acl.GET_METADATA</name>
+    <value>*</value>
+    <description>
+      ACL for get-key-metadata an get-keys-metadata operations.
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.kms.acl.SET_KEY_MATERIAL</name>
+    <value>*</value>
+    <description>
+      Complimentary ACL for CREATE and ROLLOVER operation to allow the client
+      to provide the key material when creating or rolling a key.
+    </description>
+  </property>
+
+</configuration>

+ 45 - 0
hadoop-common-project/hadoop-kms/src/main/conf/kms-env.sh

@@ -0,0 +1,45 @@
+#!/bin/bash
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License. See accompanying LICENSE file.
+#
+
+# Set kms specific environment variables here.
+
+# Settings for the Embedded Tomcat that runs KMS
+# Java System properties for KMS should be specified in this variable
+#
+# export CATALINA_OPTS=
+
+# KMS logs directory
+#
+# export KMS_LOG=${KMS_HOME}/logs
+
+# KMS temporary directory
+#
+# export KMS_TEMP=${KMS_HOME}/temp
+
+# The HTTP port used by KMS
+#
+# export KMS_HTTP_PORT=16000
+
+# The Admin port used by KMS
+#
+# export KMS_ADMIN_PORT=`expr ${KMS_HTTP_PORT} + 1`
+
+# The location of the SSL keystore if using SSL
+#
+# export KMS_SSL_KEYSTORE_FILE=${HOME}/.keystore
+
+# The password of the SSL keystore if using SSL
+#
+# export KMS_SSL_KEYSTORE_PASS=password

+ 38 - 0
hadoop-common-project/hadoop-kms/src/main/conf/kms-log4j.properties

@@ -0,0 +1,38 @@
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License. See accompanying LICENSE file.
+#
+
+# If the Java System property 'kms.log.dir' is not defined at KMS start up time
+# Setup sets its value to '${kms.home}/logs'
+
+log4j.appender.kms=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.kms.DatePattern='.'yyyy-MM-dd
+log4j.appender.kms.File=${kms.log.dir}/kms.log
+log4j.appender.kms.Append=true
+log4j.appender.kms.layout=org.apache.log4j.PatternLayout
+log4j.appender.kms.layout.ConversionPattern=%d{ISO8601} %-5p %c{1} - %m%n
+
+log4j.appender.kms-audit=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.kms-audit.DatePattern='.'yyyy-MM-dd
+log4j.appender.kms-audit.File=${kms.log.dir}/kms-audit.log
+log4j.appender.kms-audit.Append=true
+log4j.appender.kms-audit.layout=org.apache.log4j.PatternLayout
+log4j.appender.kms-audit.layout.ConversionPattern=%d{ISO8601} %m%n
+
+log4j.logger.kms-audit=INFO, kms-audit
+log4j.additivity.kms-audit=false
+
+log4j.rootLogger=ALL, kms
+log4j.logger.org.apache.hadoop.conf=ERROR
+log4j.logger.org.apache.hadoop=INFO
+log4j.logger.com.sun.jersey.server.wadl.generators.WadlGeneratorJAXBGrammarGenerator=OFF

+ 71 - 0
hadoop-common-project/hadoop-kms/src/main/conf/kms-site.xml

@@ -0,0 +1,71 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<configuration>
+
+  <!-- KMS Backend KeyProvider -->
+  <property>
+    <name>hadoop.security.key.provider.path</name>
+    <value>jceks://file@/${user.home}/kms.keystore</value>
+    <description>
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.security.keystore.JavaKeyStoreProvider.password</name>
+    <value>none</value>
+    <description>
+    </description>
+  </property>
+
+  <!-- KMS Cache -->
+  <property>
+    <name>hadoop.kms.cache.timeout.ms</name>
+    <value>10000</value>
+    <description>
+    </description>
+  </property>
+
+  <!-- KMS Security -->
+
+  <property>
+    <name>hadoop.kms.authentication.type</name>
+    <value>simple</value>
+    <description>
+      simple or kerberos
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.kms.authentication.kerberos.keytab</name>
+    <value>${user.home}/kms.keytab</value>
+    <description>
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.kms.authentication.kerberos.principal</name>
+    <value>HTTP/localhost</value>
+    <description>
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.kms.authentication.kerberos.name.rules</name>
+    <value>DEFAULT</value>
+    <description>
+    </description>
+  </property>
+
+</configuration>

+ 305 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java

@@ -0,0 +1,305 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms.server;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.crypto.key.KeyProvider;
+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.authorize.AuthorizationException;
+import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
+import org.apache.hadoop.util.StringUtils;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+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.net.URI;
+import java.net.URISyntaxException;
+import java.security.Principal;
+import java.text.MessageFormat;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Class providing the REST bindings, via Jersey, for the KMS.
+ */
+@Path(KMSRESTConstants.SERVICE_VERSION)
+@InterfaceAudience.Private
+public class KMS {
+  private static final String CREATE_KEY = "CREATE_KEY";
+  private static final String DELETE_KEY = "DELETE_KEY";
+  private static final String ROLL_NEW_VERSION = "ROLL_NEW_VERSION";
+  private static final String GET_KEYS = "GET_KEYS";
+  private static final String GET_KEYS_METADATA = "GET_KEYS_METADATA";
+  private static final String GET_KEY_VERSION = "GET_KEY_VERSION";
+  private static final String GET_CURRENT_KEY = "GET_CURRENT_KEY";
+  private static final String GET_KEY_VERSIONS = "GET_KEY_VERSIONS";
+  private static final String GET_METADATA = "GET_METADATA";
+
+  private KeyProvider provider;
+
+  public KMS() throws Exception {
+    provider = KMSWebApp.getKeyProvider();
+  }
+
+  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 void assertAccess(KMSACLs.Type aclType, Principal principal,
+      String operation, String key) throws AccessControlException {
+    if (!KMSWebApp.getACLs().hasAccess(aclType, principal.getName())) {
+      KMSWebApp.getUnauthorizedCallsMeter().mark();
+      KMSAudit.unauthorized(principal, operation, key);
+      throw new AuthorizationException(MessageFormat.format(
+          "User:{0} not allowed to do ''{1}'' on ''{2}''",
+          principal.getName(), operation, key));
+    }
+  }
+
+  private static KeyProvider.KeyVersion removeKeyMaterial(
+      KeyProvider.KeyVersion keyVersion) {
+    return new KMSClientProvider.KMSKeyVersion(keyVersion.getVersionName(),
+        null);
+  }
+
+  private static URI getKeyURI(String name) throws URISyntaxException {
+    return new URI(KMSRESTConstants.SERVICE_VERSION + "/" +
+        KMSRESTConstants.KEY_RESOURCE + "/" + name);
+  }
+
+  @POST
+  @Path(KMSRESTConstants.KEYS_RESOURCE)
+  @Consumes(MediaType.APPLICATION_JSON)
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response createKey(@Context SecurityContext securityContext,
+      Map jsonKey) throws Exception {
+    KMSWebApp.getAdminCallsMeter().mark();
+    Principal user = getPrincipal(securityContext);
+    String name = (String) jsonKey.get(KMSRESTConstants.NAME_FIELD);
+    KMSClientProvider.checkNotEmpty(name, KMSRESTConstants.NAME_FIELD);
+    assertAccess(KMSACLs.Type.CREATE, user, CREATE_KEY, name);
+    String cipher = (String) jsonKey.get(KMSRESTConstants.CIPHER_FIELD);
+    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)
+        jsonKey.get(KMSRESTConstants.DESCRIPTION_FIELD);
+
+    if (material != null) {
+      assertAccess(KMSACLs.Type.SET_KEY_MATERIAL, user,
+          CREATE_KEY + " with user provided material", name);
+    }
+    KeyProvider.Options options = new KeyProvider.Options(
+        KMSWebApp.getConfiguration());
+    if (cipher != null) {
+      options.setCipher(cipher);
+    }
+    if (length != 0) {
+      options.setBitLength(length);
+    }
+    options.setDescription(description);
+
+    KeyProvider.KeyVersion keyVersion = (material != null)
+        ? provider.createKey(name, Base64.decodeBase64(material), options)
+        : provider.createKey(name, options);
+
+    provider.flush();
+
+    KMSAudit.ok(user, CREATE_KEY, name, "UserProvidedMaterial:" +
+        (material != null) + " Description:" + description);
+
+    if (!KMSWebApp.getACLs().hasAccess(KMSACLs.Type.GET, user.getName())) {
+      keyVersion = removeKeyMaterial(keyVersion);
+    }
+    Map json = KMSServerJSONUtils.toJSON(keyVersion);
+    String requestURL = KMSMDCFilter.getURL();
+    int idx = requestURL.lastIndexOf(KMSRESTConstants.KEYS_RESOURCE);
+    requestURL = requestURL.substring(0, idx);
+    String keyURL = requestURL + KMSRESTConstants.KEY_RESOURCE + "/" + name;
+    return Response.created(getKeyURI(name)).type(MediaType.APPLICATION_JSON).
+        header("Location", keyURL).entity(json).build();
+  }
+
+  @DELETE
+  @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}")
+  public Response deleteKey(@Context SecurityContext securityContext,
+      @PathParam("name") String name) throws Exception {
+    KMSWebApp.getAdminCallsMeter().mark();
+    Principal user = getPrincipal(securityContext);
+    assertAccess(KMSACLs.Type.DELETE, user, DELETE_KEY, name);
+    KMSClientProvider.checkNotEmpty(name, "name");
+    provider.deleteKey(name);
+    provider.flush();
+
+    KMSAudit.ok(user, DELETE_KEY, name, "");
+
+    return Response.ok().build();
+  }
+
+  @POST
+  @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 {
+    KMSWebApp.getAdminCallsMeter().mark();
+    Principal user = getPrincipal(securityContext);
+    assertAccess(KMSACLs.Type.ROLLOVER, user, ROLL_NEW_VERSION, name);
+    KMSClientProvider.checkNotEmpty(name, "name");
+    String material = (String)
+        jsonMaterial.get(KMSRESTConstants.MATERIAL_FIELD);
+    if (material != null) {
+      assertAccess(KMSACLs.Type.SET_KEY_MATERIAL, user,
+          ROLL_NEW_VERSION + " with user provided material", name);
+    }
+    KeyProvider.KeyVersion keyVersion = (material != null)
+        ? provider.rollNewVersion(name, Base64.decodeBase64(material))
+        : provider.rollNewVersion(name);
+
+    provider.flush();
+
+    KMSAudit.ok(user, ROLL_NEW_VERSION, name, "UserProvidedMaterial:" +
+        (material != null) + " NewVersion:" + keyVersion.getVersionName());
+
+    if (!KMSWebApp.getACLs().hasAccess(KMSACLs.Type.GET, user.getName())) {
+      keyVersion = removeKeyMaterial(keyVersion);
+    }
+    Map json = KMSServerJSONUtils.toJSON(keyVersion);
+    return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
+  }
+
+  @GET
+  @Path(KMSRESTConstants.KEYS_METADATA_RESOURCE)
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response getKeysMetadata(@Context SecurityContext securityContext,
+      @QueryParam(KMSRESTConstants.KEY_OP) List<String> keyNamesList)
+      throws Exception {
+    KMSWebApp.getAdminCallsMeter().mark();
+    Principal user = getPrincipal(securityContext);
+    String[] keyNames = keyNamesList.toArray(new String[keyNamesList.size()]);
+    String names = StringUtils.arrayToString(keyNames);
+    assertAccess(KMSACLs.Type.GET_METADATA, user, GET_KEYS_METADATA, names);
+    KeyProvider.Metadata[] keysMeta = provider.getKeysMetadata(keyNames);
+    Object json = KMSServerJSONUtils.toJSON(keyNames, keysMeta);
+    KMSAudit.ok(user, GET_KEYS_METADATA, names, "");
+    return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
+  }
+
+  @GET
+  @Path(KMSRESTConstants.KEYS_NAMES_RESOURCE)
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response getKeyNames(@Context SecurityContext securityContext)
+      throws Exception {
+    KMSWebApp.getAdminCallsMeter().mark();
+    Principal user = getPrincipal(securityContext);
+    assertAccess(KMSACLs.Type.GET_KEYS, user, GET_KEYS, "*");
+    Object json = provider.getKeys();
+    KMSAudit.ok(user, 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)
+      throws Exception {
+    return getMetadata(securityContext, 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)
+      throws Exception {
+    Principal user = getPrincipal(securityContext);
+    KMSClientProvider.checkNotEmpty(name, "name");
+    KMSWebApp.getAdminCallsMeter().mark();
+    assertAccess(KMSACLs.Type.GET_METADATA, user, GET_METADATA, name);
+    Object json = KMSServerJSONUtils.toJSON(name, provider.getMetadata(name));
+    KMSAudit.ok(user, GET_METADATA, name, "");
+    return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
+  }
+
+  @GET
+  @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}/" +
+      KMSRESTConstants.CURRENT_VERSION_SUB_RESOURCE)
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response getCurrentVersion(@Context SecurityContext securityContext,
+      @PathParam("name") String name)
+      throws Exception {
+    Principal user = getPrincipal(securityContext);
+    KMSClientProvider.checkNotEmpty(name, "name");
+    KMSWebApp.getKeyCallsMeter().mark();
+    assertAccess(KMSACLs.Type.GET, user, GET_CURRENT_KEY, name);
+    Object json = KMSServerJSONUtils.toJSON(provider.getCurrentKey(name));
+    KMSAudit.ok(user, GET_CURRENT_KEY, name, "");
+    return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
+  }
+
+  @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);
+    KMSClientProvider.checkNotEmpty(versionName, "versionName");
+    KMSWebApp.getKeyCallsMeter().mark();
+    assertAccess(KMSACLs.Type.GET, user, GET_KEY_VERSION, versionName);
+    Object json = KMSServerJSONUtils.toJSON(provider.getKeyVersion(versionName));
+    KMSAudit.ok(user, GET_KEY_VERSION, versionName, "");
+    return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
+  }
+
+  @GET
+  @Path(KMSRESTConstants.KEY_RESOURCE + "/{name:.*}/" +
+      KMSRESTConstants.VERSIONS_SUB_RESOURCE)
+  @Produces(MediaType.APPLICATION_JSON)
+  public Response getKeyVersions(@Context SecurityContext securityContext,
+      @PathParam("name") String name)
+      throws Exception {
+    Principal user = getPrincipal(securityContext);
+    KMSClientProvider.checkNotEmpty(name, "name");
+    KMSWebApp.getKeyCallsMeter().mark();
+    assertAccess(KMSACLs.Type.GET, user, GET_KEY_VERSIONS, name);
+    Object json = KMSServerJSONUtils.toJSON(provider.getKeyVersions(name));
+    KMSAudit.ok(user, GET_KEY_VERSIONS, name, "");
+    return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
+  }
+
+}

+ 133 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSACLs.java

@@ -0,0 +1,133 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms.server;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+/**
+ * Provides access to the <code>AccessControlList</code>s used by KMS,
+ * hot-reloading them if the <code>kms-acls.xml</code> file where the ACLs
+ * are defined has been updated.
+ */
+public class KMSACLs implements Runnable {
+  private static final Logger LOG = LoggerFactory.getLogger(KMSACLs.class);
+
+
+  public enum Type {
+    CREATE, DELETE, ROLLOVER, GET, GET_KEYS, GET_METADATA, SET_KEY_MATERIAL;
+
+    public String getConfigKey() {
+      return KMSConfiguration.CONFIG_PREFIX + "acl." + this.toString();
+    }
+  }
+
+  public static final String ACL_DEFAULT = AccessControlList.WILDCARD_ACL_VALUE;
+
+  public static final int RELOADER_SLEEP_MILLIS = 1000;
+
+  Map<Type, AccessControlList> acls;
+  private ReadWriteLock lock;
+  private ScheduledExecutorService executorService;
+  private long lastReload;
+
+  KMSACLs(Configuration conf) {
+    lock = new ReentrantReadWriteLock();
+    if (conf == null) {
+      conf = loadACLs();
+    }
+    setACLs(conf);
+  }
+
+  public KMSACLs() {
+    this(null);
+  }
+
+  private void setACLs(Configuration conf) {
+    lock.writeLock().lock();
+    try {
+      acls = new HashMap<Type, AccessControlList>();
+      for (Type aclType : Type.values()) {
+        String aclStr = conf.get(aclType.getConfigKey(), ACL_DEFAULT);
+        acls.put(aclType, new AccessControlList(aclStr));
+        LOG.info("'{}' ACL '{}'", aclType, aclStr);
+      }
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  @Override
+  public void run() {
+    try {
+      if (KMSConfiguration.isACLsFileNewer(lastReload)) {
+        setACLs(loadACLs());
+      }
+    } catch (Exception ex) {
+      LOG.warn("Could not reload ACLs file: " + ex.toString(), ex);
+    }
+  }
+
+  public synchronized void startReloader() {
+    if (executorService == null) {
+      executorService = Executors.newScheduledThreadPool(1);
+      executorService.scheduleAtFixedRate(this, RELOADER_SLEEP_MILLIS,
+          RELOADER_SLEEP_MILLIS, TimeUnit.MILLISECONDS);
+    }
+  }
+
+  public synchronized void stopReloader() {
+    if (executorService != null) {
+      executorService.shutdownNow();
+      executorService = null;
+    }
+  }
+
+  private Configuration loadACLs() {
+    LOG.debug("Loading ACLs file");
+    lastReload = System.currentTimeMillis();
+    Configuration conf = KMSConfiguration.getACLsConf();
+    // triggering the resource loading.
+    conf.get(Type.CREATE.getConfigKey());
+    return conf;
+  }
+
+  public boolean hasAccess(Type type, String user) {
+    UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
+    AccessControlList acl = null;
+    lock.readLock().lock();
+    try {
+      acl = acls.get(type);
+    } finally {
+      lock.readLock().unlock();
+    }
+    return acl.isUserAllowed(ugi);
+  }
+
+}

+ 62 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java

@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms.server;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.security.Principal;
+
+/**
+ * Provides convenience methods for audit logging consistently the different
+ * types of events.
+ */
+public class KMSAudit {
+  public static final String KMS_LOGGER_NAME = "kms-audit";
+
+  private static Logger AUDIT_LOG = LoggerFactory.getLogger(KMS_LOGGER_NAME);
+
+  private static void op(String status, String op, Principal user, String key,
+      String extraMsg) {
+    AUDIT_LOG.info("Status:{} User:{} Op:{} Name:{}{}", status, user.getName(),
+        op, key, extraMsg);
+  }
+
+  public static void ok(Principal user, String op, String key,
+      String extraMsg) {
+    op("OK", op, user, key, extraMsg);
+  }
+
+  public static void unauthorized(Principal user, String op, String key) {
+    op("UNAUTHORIZED", op, user, key, "");
+  }
+
+  public static void error(Principal user, String method, String url,
+      String extraMsg) {
+    AUDIT_LOG.info("Status:ERROR User:{} Method:{} URL:{} Exception:'{}'",
+        user.getName(), method, url, extraMsg);
+  }
+
+  public static void unauthenticated(String remoteHost, String method,
+      String url, String extraMsg) {
+    AUDIT_LOG.info(
+        "Status:UNAUTHENTICATED RemoteHost:{} Method:{} URL:{} ErrorMsg:'{}'",
+        remoteHost, method, url, extraMsg);
+  }
+
+}

+ 123 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAuthenticationFilter.java

@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms.server;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+import javax.servlet.ServletRequest;
+import javax.servlet.ServletResponse;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import javax.servlet.http.HttpServletResponseWrapper;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * Authentication filter that takes the configuration from the KMS configuration
+ * file.
+ */
+@InterfaceAudience.Private
+public class KMSAuthenticationFilter extends AuthenticationFilter {
+  private static final String CONF_PREFIX = KMSConfiguration.CONFIG_PREFIX +
+      "authentication.";
+
+  @Override
+  protected Properties getConfiguration(String configPrefix,
+      FilterConfig filterConfig) {
+    Properties props = new Properties();
+    Configuration conf = KMSWebApp.getConfiguration();
+    for (Map.Entry<String, String> entry : conf) {
+      String name = entry.getKey();
+      if (name.startsWith(CONF_PREFIX)) {
+        String value = conf.get(name);
+        name = name.substring(CONF_PREFIX.length());
+        props.setProperty(name, value);
+      }
+    }
+    return props;
+  }
+
+  private static class KMSResponse extends HttpServletResponseWrapper {
+    public int statusCode;
+    public String msg;
+
+    public KMSResponse(ServletResponse response) {
+      super((HttpServletResponse)response);
+    }
+
+    @Override
+    public void setStatus(int sc) {
+      statusCode = sc;
+      super.setStatus(sc);
+    }
+
+    @Override
+    public void sendError(int sc, String msg) throws IOException {
+      statusCode = sc;
+      this.msg = msg;
+      super.sendError(sc, msg);
+    }
+
+    @Override
+    public void sendError(int sc) throws IOException {
+      statusCode = sc;
+      super.sendError(sc);
+    }
+
+    @Override
+    public void setStatus(int sc, String sm) {
+      statusCode = sc;
+      msg = sm;
+      super.setStatus(sc, sm);
+    }
+  }
+
+  @Override
+  public void doFilter(ServletRequest request, ServletResponse response,
+      FilterChain filterChain) throws IOException, ServletException {
+    KMSResponse kmsResponse = new KMSResponse(response);
+    super.doFilter(request, kmsResponse, filterChain);
+
+    if (kmsResponse.statusCode != HttpServletResponse.SC_OK &&
+        kmsResponse.statusCode != HttpServletResponse.SC_CREATED &&
+        kmsResponse.statusCode != HttpServletResponse.SC_UNAUTHORIZED) {
+      KMSWebApp.getInvalidCallsMeter().mark();
+    }
+
+    // HttpServletResponse.SC_UNAUTHORIZED is because the request does not
+    // belong to an authenticated user.
+    if (kmsResponse.statusCode == HttpServletResponse.SC_UNAUTHORIZED) {
+      KMSWebApp.getUnauthenticatedCallsMeter().mark();
+      String method = ((HttpServletRequest) request).getMethod();
+      StringBuffer requestURL = ((HttpServletRequest) request).getRequestURL();
+      String queryString = ((HttpServletRequest) request).getQueryString();
+      if (queryString != null) {
+        requestURL.append("?").append(queryString);
+      }
+      KMSAudit.unauthenticated(request.getRemoteHost(), method,
+          requestURL.toString(), kmsResponse.msg);
+    }
+  }
+
+}

+ 180 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSCacheKeyProvider.java

@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms.server;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import org.apache.hadoop.crypto.key.KeyProvider;
+
+import java.io.IOException;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A <code>KeyProvider</code> proxy implementation providing a short lived
+ * cache for <code>KeyVersions</code> to avoid burst of requests to hit the
+ * underlying <code>KeyProvider</code>.
+ */
+public class KMSCacheKeyProvider extends KeyProvider {
+  private final KeyProvider provider;
+  private LoadingCache<String, KeyVersion> keyVersionCache;
+  private LoadingCache<String, KeyVersion> currentKeyCache;
+
+  private static class KeyNotFoundException extends Exception {
+    private static final long serialVersionUID = 1L;
+  }
+
+  public KMSCacheKeyProvider(KeyProvider prov, long timeoutMillis) {
+    this.provider =  prov;
+    keyVersionCache = CacheBuilder.newBuilder().expireAfterAccess(timeoutMillis,
+        TimeUnit.MILLISECONDS).build(new CacheLoader<String, KeyVersion>() {
+      @Override
+      public KeyVersion load(String key) throws Exception {
+        KeyVersion kv = provider.getKeyVersion(key);
+        if (kv == null) {
+          throw new KeyNotFoundException();
+        }
+        return kv;
+      }
+    });
+    // for current key we don't want to go stale for more than 1 sec
+    currentKeyCache = CacheBuilder.newBuilder().expireAfterWrite(1000,
+        TimeUnit.MILLISECONDS).build(new CacheLoader<String, KeyVersion>() {
+      @Override
+      public KeyVersion load(String key) throws Exception {
+        KeyVersion kv =  provider.getCurrentKey(key);
+        if (kv == null) {
+          throw new KeyNotFoundException();
+        }
+        return kv;
+      }
+    });
+  }
+
+  @Override
+  public KeyVersion getCurrentKey(String name) throws IOException {
+    try {
+      return currentKeyCache.get(name);
+    } catch (ExecutionException ex) {
+      Throwable cause = ex.getCause();
+      if (cause instanceof KeyNotFoundException) {
+        return null;
+      } else if (cause instanceof IOException) {
+        throw (IOException) cause;
+      } else {
+        throw new IOException(cause);
+      }
+    }
+  }
+
+  @Override
+  public KeyVersion getKeyVersion(String versionName)
+      throws IOException {
+    try {
+      return keyVersionCache.get(versionName);
+    } catch (ExecutionException ex) {
+      Throwable cause = ex.getCause();
+      if (cause instanceof KeyNotFoundException) {
+        return null;
+      } else if (cause instanceof IOException) {
+        throw (IOException) cause;
+      } else {
+        throw new IOException(cause);
+      }
+    }
+  }
+
+  @Override
+  public List<String> getKeys() throws IOException {
+    return provider.getKeys();
+  }
+
+  @Override
+  public List<KeyVersion> getKeyVersions(String name)
+      throws IOException {
+    return provider.getKeyVersions(name);
+  }
+
+  @Override
+  public Metadata getMetadata(String name) throws IOException {
+    return provider.getMetadata(name);
+  }
+
+  @Override
+  public KeyVersion createKey(String name, byte[] material,
+      Options options) throws IOException {
+    return provider.createKey(name, material, options);
+  }
+
+  @Override
+  public KeyVersion createKey(String name,
+      Options options)
+      throws NoSuchAlgorithmException, IOException {
+    return provider.createKey(name, options);
+  }
+
+  @Override
+  public void deleteKey(String name) throws IOException {
+    Metadata metadata = provider.getMetadata(name);
+    List<String> versions = new ArrayList<String>(metadata.getVersions());
+    for (int i = 0; i < metadata.getVersions(); i++) {
+      versions.add(KeyProvider.buildVersionName(name, i));
+    }
+    provider.deleteKey(name);
+    currentKeyCache.invalidate(name);
+    keyVersionCache.invalidateAll(versions);
+  }
+
+  @Override
+  public KeyVersion rollNewVersion(String name, byte[] material)
+      throws IOException {
+    KeyVersion key = provider.rollNewVersion(name, material);
+    currentKeyCache.invalidate(name);
+    return key;
+  }
+
+  @Override
+  public KeyVersion rollNewVersion(String name)
+      throws NoSuchAlgorithmException, IOException {
+    KeyVersion key = provider.rollNewVersion(name);
+    currentKeyCache.invalidate(name);
+    return key;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    provider.flush();
+  }
+
+  @Override
+  public Metadata[] getKeysMetadata(String ... keyNames)
+      throws IOException {
+    return provider.getKeysMetadata(keyNames);
+  }
+
+  @Override
+  public boolean isTransient() {
+    return provider.isTransient();
+  }
+
+}

+ 94 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java

@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms.server;
+
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.File;
+import java.net.MalformedURLException;
+import java.net.URL;
+
+/**
+ * Utility class to load KMS configuration files.
+ */
+public class KMSConfiguration {
+
+  public static final String KMS_CONFIG_DIR = "kms.config.dir";
+  public static final String KMS_SITE_XML = "kms-site.xml";
+  public static final String KMS_ACLS_XML = "kms-acls.xml";
+
+  public static final String CONFIG_PREFIX = "hadoop.kms.";
+
+  public static final String KEY_CACHE_TIMEOUT_KEY = CONFIG_PREFIX +
+      "cache.timeout.ms";
+  public static final long KEY_CACHE_TIMEOUT_DEFAULT = 10 * 1000; // 10 secs
+
+  static Configuration getConfiguration(boolean loadHadoopDefaults,
+      String ... resources) {
+    Configuration conf = new Configuration(loadHadoopDefaults);
+    String confDir = System.getProperty(KMS_CONFIG_DIR);
+    if (confDir != null) {
+      try {
+        if (!confDir.startsWith("/")) {
+          throw new RuntimeException("System property '" + KMS_CONFIG_DIR +
+              "' must be an absolute path: " + confDir);
+        }
+        if (!confDir.endsWith("/")) {
+          confDir += "/";
+        }
+        for (String resource : resources) {
+          conf.addResource(new URL("file://" + confDir + resource));
+        }
+      } catch (MalformedURLException ex) {
+        throw new RuntimeException(ex);
+      }
+    } else {
+      for (String resource : resources) {
+        conf.addResource(resource);
+      }
+    }
+    return conf;
+  }
+
+  public static Configuration getKMSConf() {
+    return getConfiguration(true, "core-site.xml", KMS_SITE_XML);
+  }
+
+  public static Configuration getACLsConf() {
+    return getConfiguration(false, KMS_ACLS_XML);
+  }
+
+  public static boolean isACLsFileNewer(long time) {
+    boolean newer = false;
+    String confDir = System.getProperty(KMS_CONFIG_DIR);
+    if (confDir != null) {
+      if (!confDir.startsWith("/")) {
+        throw new RuntimeException("System property '" + KMS_CONFIG_DIR +
+            "' must be an absolute path: " + confDir);
+      }
+      if (!confDir.endsWith("/")) {
+        confDir += "/";
+      }
+      File f = new File(confDir, KMS_ACLS_XML);
+      // at least 100ms newer than time, we do this to ensure the file
+      // has been properly closed/flushed
+      newer = f.lastModified() - time > 100;
+    }
+    return newer;
+  }
+}

+ 113 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSExceptionsProvider.java

@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms.server;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import com.sun.jersey.api.container.ContainerException;
+import org.apache.hadoop.crypto.key.kms.KMSRESTConstants;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+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;
+
+/**
+ * Jersey provider that converts KMS exceptions into detailed HTTP errors.
+ */
+@Provider
+@InterfaceAudience.Private
+public class KMSExceptionsProvider implements ExceptionMapper<Exception> {
+  private static Logger LOG =
+      LoggerFactory.getLogger(KMSExceptionsProvider.class);
+
+  private static final String ENTER = System.getProperty("line.separator");
+
+  protected Response createResponse(Response.Status status, Throwable ex) {
+    Map<String, Object> json = new LinkedHashMap<String, Object>();
+    json.put(KMSRESTConstants.ERROR_EXCEPTION_JSON, ex.getClass().getName());
+    json.put(KMSRESTConstants.ERROR_MESSAGE_JSON, getOneLineMessage(ex));
+    log(status, ex);
+    return Response.status(status).type(MediaType.APPLICATION_JSON).
+        entity(json).build();
+  }
+
+  protected String getOneLineMessage(Throwable exception) {
+    String message = exception.getMessage();
+    if (message != null) {
+      int i = message.indexOf(ENTER);
+      if (i > -1) {
+        message = message.substring(0, i);
+      }
+    }
+    return message;
+  }
+
+  /**
+   * Maps different exceptions thrown by KMS to HTTP status codes.
+   */
+  @Override
+  public Response toResponse(Exception exception) {
+    Response.Status status;
+    boolean doAudit = true;
+    Throwable throwable = exception;
+    if (exception instanceof ContainerException) {
+      throwable = exception.getCause();
+    }
+    if (throwable instanceof SecurityException) {
+      status = Response.Status.FORBIDDEN;
+    } else if (throwable instanceof AuthenticationException) {
+      status = Response.Status.FORBIDDEN;
+      // we don't audit here because we did it already when checking access
+      doAudit = false;
+    } else if (throwable instanceof AccessControlException) {
+      status = Response.Status.FORBIDDEN;
+    } else if (exception instanceof IOException) {
+      status = Response.Status.INTERNAL_SERVER_ERROR;
+    } else if (exception instanceof UnsupportedOperationException) {
+      status = Response.Status.BAD_REQUEST;
+    } else if (exception instanceof IllegalArgumentException) {
+      status = Response.Status.BAD_REQUEST;
+    } else {
+      status = Response.Status.INTERNAL_SERVER_ERROR;
+    }
+    if (doAudit) {
+      KMSAudit.error(KMSMDCFilter.getPrincipal(), KMSMDCFilter.getMethod(),
+          KMSMDCFilter.getURL(), getOneLineMessage(exception));
+    }
+    return createResponse(status, throwable);
+  }
+
+  protected void log(Response.Status status, Throwable ex) {
+    Principal principal = KMSMDCFilter.getPrincipal();
+    String method = KMSMDCFilter.getMethod();
+    String url = KMSMDCFilter.getURL();
+    String msg = getOneLineMessage(ex);
+    LOG.warn("User:{} Method:{} URL:{} Response:{}-{}", principal, method, url,
+        status, msg, ex);
+  }
+
+}

+ 54 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJSONReader.java

@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms.server;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyReader;
+import javax.ws.rs.ext.Provider;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+import java.util.Map;
+
+@Provider
+@Consumes(MediaType.APPLICATION_JSON)
+@InterfaceAudience.Private
+public class KMSJSONReader implements MessageBodyReader<Map> {
+
+  @Override
+  public boolean isReadable(Class<?> type, Type genericType,
+      Annotation[] annotations, MediaType mediaType) {
+    return type.isAssignableFrom(Map.class);
+  }
+
+  @Override
+  public Map readFrom(Class<Map> type, Type genericType,
+      Annotation[] annotations, MediaType mediaType,
+      MultivaluedMap<String, String> httpHeaders, InputStream entityStream)
+      throws IOException, WebApplicationException {
+    ObjectMapper mapper = new ObjectMapper();
+    return mapper.readValue(entityStream, type);
+  }
+}

+ 70 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJSONWriter.java

@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms.server;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.ext.MessageBodyWriter;
+import javax.ws.rs.ext.Provider;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Type;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Jersey provider that converts <code>Map</code>s and <code>List</code>s
+ * to their JSON representation.
+ */
+@Provider
+@Produces(MediaType.APPLICATION_JSON)
+@InterfaceAudience.Private
+public class KMSJSONWriter implements MessageBodyWriter<Object> {
+
+  @Override
+  public boolean isWriteable(Class<?> aClass, Type type,
+      Annotation[] annotations, MediaType mediaType) {
+    return Map.class.isAssignableFrom(aClass) ||
+        List.class.isAssignableFrom(aClass);
+  }
+
+  @Override
+  public long getSize(Object obj, Class<?> aClass, Type type,
+      Annotation[] annotations, MediaType mediaType) {
+    return -1;
+  }
+
+  @Override
+  public void writeTo(Object obj, Class<?> aClass, Type type,
+      Annotation[] annotations, MediaType mediaType,
+      MultivaluedMap<String, Object> stringObjectMultivaluedMap,
+      OutputStream outputStream) throws IOException, WebApplicationException {
+    Writer writer = new OutputStreamWriter(outputStream);
+    ObjectMapper jsonMapper = new ObjectMapper();
+    jsonMapper.writerWithDefaultPrettyPrinter().writeValue(writer, obj);
+  }
+
+}

+ 92 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSMDCFilter.java

@@ -0,0 +1,92 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms.server;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import javax.servlet.Filter;
+import javax.servlet.FilterChain;
+import javax.servlet.FilterConfig;
+import javax.servlet.ServletException;
+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
+ * scope of KMS calls on the server side.
+ */
+@InterfaceAudience.Private
+public class KMSMDCFilter implements Filter {
+
+  private static class Data {
+    private Principal principal;
+    private String method;
+    private StringBuffer url;
+
+    private Data(Principal principal, String method, StringBuffer url) {
+      this.principal = principal;
+      this.method = method;
+      this.url = url;
+    }
+  }
+
+  private static ThreadLocal<Data> DATA_TL = new ThreadLocal<Data>();
+
+  public static Principal getPrincipal() {
+    return DATA_TL.get().principal;
+  }
+
+  public static String getMethod() {
+    return DATA_TL.get().method;
+  }
+
+  public static String getURL() {
+    return DATA_TL.get().url.toString();
+  }
+
+  @Override
+  public void init(FilterConfig config) throws ServletException {
+  }
+
+  @Override
+  public void doFilter(ServletRequest request, ServletResponse response,
+      FilterChain chain)
+      throws IOException, ServletException {
+    try {
+      DATA_TL.remove();
+      Principal principal = ((HttpServletRequest) request).getUserPrincipal();
+      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));
+      chain.doFilter(request, response);
+    } finally {
+      DATA_TL.remove();
+    }
+  }
+
+  @Override
+  public void destroy() {
+  }
+}

+ 80 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSServerJSONUtils.java

@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms.server;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.kms.KMSRESTConstants;
+
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * JSON utility methods for the KMS.
+ */
+@InterfaceAudience.Private
+public class KMSServerJSONUtils {
+  @SuppressWarnings("unchecked")
+  public static Map toJSON(KeyProvider.KeyVersion keyVersion) {
+    Map json = new LinkedHashMap();
+    if (keyVersion != null) {
+      json.put(KMSRESTConstants.VERSION_NAME_FIELD,
+          keyVersion.getVersionName());
+      json.put(KMSRESTConstants.MATERIAL_FIELD, keyVersion.getMaterial());
+    }
+    return json;
+  }
+
+  @SuppressWarnings("unchecked")
+  public static List toJSON(List<KeyProvider.KeyVersion> keyVersions) {
+    List json = new ArrayList();
+    if (keyVersions != null) {
+      for (KeyProvider.KeyVersion version : keyVersions) {
+        json.add(toJSON(version));
+      }
+    }
+    return json;
+  }
+
+  @SuppressWarnings("unchecked")
+  public static Map toJSON(String keyName, KeyProvider.Metadata meta) {
+    Map json = new LinkedHashMap();
+    if (meta != null) {
+      json.put(KMSRESTConstants.NAME_FIELD, keyName);
+      json.put(KMSRESTConstants.CIPHER_FIELD, meta.getCipher());
+      json.put(KMSRESTConstants.LENGTH_FIELD, meta.getBitLength());
+      json.put(KMSRESTConstants.DESCRIPTION_FIELD, meta.getDescription());
+      json.put(KMSRESTConstants.CREATED_FIELD,
+          meta.getCreated().getTime());
+      json.put(KMSRESTConstants.VERSIONS_FIELD,
+          (long) meta.getVersions());
+    }
+    return json;
+  }
+
+  @SuppressWarnings("unchecked")
+  public static List toJSON(String[] keyNames, KeyProvider.Metadata[] metas) {
+    List json = new ArrayList();
+    for (int i = 0; i < keyNames.length; i++) {
+      json.add(toJSON(keyNames[i], metas[i]));
+    }
+    return json;
+  }
+}

+ 214 - 0
hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebApp.java

@@ -0,0 +1,214 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms.server;
+
+import com.codahale.metrics.JmxReporter;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.MetricRegistry;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProviderFactory;
+import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.util.VersionInfo;
+import org.apache.log4j.PropertyConfigurator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.bridge.SLF4JBridgeHandler;
+
+import javax.servlet.ServletContextEvent;
+import javax.servlet.ServletContextListener;
+import java.io.File;
+import java.net.URL;
+import java.util.List;
+
+@InterfaceAudience.Private
+public class KMSWebApp implements ServletContextListener {
+
+  private static final String LOG4J_PROPERTIES = "kms-log4j.properties";
+
+  private static final String METRICS_PREFIX = "hadoop.kms.";
+  private static final String ADMIN_CALLS_METER = METRICS_PREFIX +
+      "admin.calls.meter";
+  private static final String KEY_CALLS_METER = METRICS_PREFIX +
+      "key.calls.meter";
+  private static final String INVALID_CALLS_METER = METRICS_PREFIX +
+      "invalid.calls.meter";
+  private static final String UNAUTHORIZED_CALLS_METER = METRICS_PREFIX +
+      "unauthorized.calls.meter";
+  private static final String UNAUTHENTICATED_CALLS_METER = METRICS_PREFIX +
+      "unauthenticated.calls.meter";
+
+  private static Logger LOG;
+  private static MetricRegistry metricRegistry;
+
+  private JmxReporter jmxReporter;
+  private static Configuration kmsConf;
+  private static KMSACLs acls;
+  private static Meter adminCallsMeter;
+  private static Meter keyCallsMeter;
+  private static Meter unauthorizedCallsMeter;
+  private static Meter unauthenticatedCallsMeter;
+  private static Meter invalidCallsMeter;
+  private static KeyProvider keyProvider;
+
+  static {
+    SLF4JBridgeHandler.removeHandlersForRootLogger();
+    SLF4JBridgeHandler.install();
+  }
+
+  private void initLogging(String confDir) {
+    if (System.getProperty("log4j.configuration") == null) {
+      System.setProperty("log4j.defaultInitOverride", "true");
+      boolean fromClasspath = true;
+      File log4jConf = new File(confDir, LOG4J_PROPERTIES).getAbsoluteFile();
+      if (log4jConf.exists()) {
+        PropertyConfigurator.configureAndWatch(log4jConf.getPath(), 1000);
+        fromClasspath = false;
+      } else {
+        ClassLoader cl = Thread.currentThread().getContextClassLoader();
+        URL log4jUrl = cl.getResource(LOG4J_PROPERTIES);
+        if (log4jUrl != null) {
+          PropertyConfigurator.configure(log4jUrl);
+        }
+      }
+      LOG = LoggerFactory.getLogger(KMSWebApp.class);
+      LOG.debug("KMS log starting");
+      if (fromClasspath) {
+        LOG.warn("Log4j configuration file '{}' not found", LOG4J_PROPERTIES);
+        LOG.warn("Logging with INFO level to standard output");
+      }
+    } else {
+      LOG = LoggerFactory.getLogger(KMSWebApp.class);
+    }
+  }
+
+  @Override
+  public void contextInitialized(ServletContextEvent sce) {
+    try {
+      String confDir = System.getProperty(KMSConfiguration.KMS_CONFIG_DIR);
+      if (confDir == null) {
+        throw new RuntimeException("System property '" +
+            KMSConfiguration.KMS_CONFIG_DIR + "' not defined");
+      }
+      kmsConf = KMSConfiguration.getKMSConf();
+      initLogging(confDir);
+      LOG.info("-------------------------------------------------------------");
+      LOG.info("  Java runtime version : {}", System.getProperty(
+          "java.runtime.version"));
+      LOG.info("  KMS Hadoop Version: " + VersionInfo.getVersion());
+      LOG.info("-------------------------------------------------------------");
+
+      acls = new KMSACLs();
+      acls.startReloader();
+
+      metricRegistry = new MetricRegistry();
+      jmxReporter = JmxReporter.forRegistry(metricRegistry).build();
+      jmxReporter.start();
+      adminCallsMeter = metricRegistry.register(ADMIN_CALLS_METER, new Meter());
+      keyCallsMeter = metricRegistry.register(KEY_CALLS_METER, new Meter());
+      invalidCallsMeter = metricRegistry.register(INVALID_CALLS_METER,
+          new Meter());
+      unauthorizedCallsMeter = metricRegistry.register(UNAUTHORIZED_CALLS_METER,
+          new Meter());
+      unauthenticatedCallsMeter = metricRegistry.register(
+          UNAUTHENTICATED_CALLS_METER, new Meter());
+
+      // this is required for the the JMXJsonServlet to work properly.
+      // the JMXJsonServlet is behind the authentication filter,
+      // thus the '*' ACL.
+      sce.getServletContext().setAttribute(HttpServer2.CONF_CONTEXT_ATTRIBUTE,
+          kmsConf);
+      sce.getServletContext().setAttribute(HttpServer2.ADMINS_ACL,
+          new AccessControlList(AccessControlList.WILDCARD_ACL_VALUE));
+
+      // intializing the KeyProvider
+
+      List<KeyProvider> providers = KeyProviderFactory.getProviders(kmsConf);
+      if (providers.isEmpty()) {
+        throw new IllegalStateException("No KeyProvider has been defined");
+      }
+      if (providers.size() > 1) {
+        LOG.warn("There is more than one KeyProvider configured '{}', using " +
+            "the first provider",
+            kmsConf.get(KeyProviderFactory.KEY_PROVIDER_PATH));
+      }
+      keyProvider = providers.get(0);
+      long timeOutMillis =
+          kmsConf.getLong(KMSConfiguration.KEY_CACHE_TIMEOUT_KEY,
+              KMSConfiguration.KEY_CACHE_TIMEOUT_DEFAULT);
+      keyProvider = new KMSCacheKeyProvider(keyProvider, timeOutMillis);
+
+      LOG.info("KMS Started");
+    } catch (Throwable ex) {
+      System.out.println();
+      System.out.println("ERROR: Hadoop KMS could not be started");
+      System.out.println();
+      System.out.println("REASON: " + ex.toString());
+      System.out.println();
+      System.out.println("Stacktrace:");
+      System.out.println("---------------------------------------------------");
+      ex.printStackTrace(System.out);
+      System.out.println("---------------------------------------------------");
+      System.out.println();
+      System.exit(1);
+    }
+  }
+
+  @Override
+  public void contextDestroyed(ServletContextEvent sce) {
+    acls.stopReloader();
+    jmxReporter.stop();
+    jmxReporter.close();
+    metricRegistry = null;
+    LOG.info("KMS Stopped");
+  }
+
+  public static Configuration getConfiguration() {
+    return new Configuration(kmsConf);
+  }
+
+  public static KMSACLs getACLs() {
+    return acls;
+  }
+
+  public static Meter getAdminCallsMeter() {
+    return adminCallsMeter;
+  }
+
+  public static Meter getKeyCallsMeter() {
+    return keyCallsMeter;
+  }
+
+  public static Meter getInvalidCallsMeter() {
+    return invalidCallsMeter;
+  }
+
+  public static Meter getUnauthorizedCallsMeter() {
+    return unauthorizedCallsMeter;
+  }
+
+  public static Meter getUnauthenticatedCallsMeter() {
+    return unauthenticatedCallsMeter;
+  }
+
+  public static KeyProvider getKeyProvider() {
+    return keyProvider;
+  }
+}

+ 181 - 0
hadoop-common-project/hadoop-kms/src/main/libexec/kms-config.sh

@@ -0,0 +1,181 @@
+#!/bin/bash
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#  http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+#
+
+# resolve links - $0 may be a softlink
+PRG="${0}"
+
+while [ -h "${PRG}" ]; do
+  ls=`ls -ld "${PRG}"`
+  link=`expr "$ls" : '.*-> \(.*\)$'`
+  if expr "$link" : '/.*' > /dev/null; then
+    PRG="$link"
+  else
+    PRG=`dirname "${PRG}"`/"$link"
+  fi
+done
+
+BASEDIR=`dirname ${PRG}`
+BASEDIR=`cd ${BASEDIR}/..;pwd`
+
+
+function print() {
+  if [ "${KMS_SILENT}" != "true" ]; then
+    echo "$@"
+  fi
+}
+
+# if KMS_HOME is already set warn it will be ignored
+#
+if [ "${KMS_HOME}" != "" ]; then
+  echo "WARNING: current setting of KMS_HOME ignored"
+fi
+
+print
+
+# setting KMS_HOME to the installation dir, it cannot be changed
+#
+export KMS_HOME=${BASEDIR}
+kms_home=${KMS_HOME}
+print "Setting KMS_HOME:          ${KMS_HOME}"
+
+# if the installation has a env file, source it
+# this is for native packages installations
+#
+if [ -e "${KMS_HOME}/bin/kms-env.sh" ]; then
+  print "Sourcing:                    ${KMS_HOME}/bin/kms-env.sh"
+  source ${KMS_HOME}/bin/kms-env.sh
+  grep "^ *export " ${KMS_HOME}/bin/kms-env.sh | sed 's/ *export/  setting/'
+fi
+
+# verify that the sourced env file didn't change KMS_HOME
+# if so, warn and revert
+#
+if [ "${KMS_HOME}" != "${kms_home}" ]; then
+  print "WARN: KMS_HOME resetting to ''${KMS_HOME}'' ignored"
+  export KMS_HOME=${kms_home}
+  print "  using KMS_HOME:        ${KMS_HOME}"
+fi
+
+if [ "${KMS_CONFIG}" = "" ]; then
+  export KMS_CONFIG=${KMS_HOME}/etc/hadoop
+  print "Setting KMS_CONFIG:        ${KMS_CONFIG}"
+else
+  print "Using   KMS_CONFIG:        ${KMS_CONFIG}"
+fi
+kms_config=${KMS_CONFIG}
+
+# if the configuration dir has a env file, source it
+#
+if [ -e "${KMS_CONFIG}/kms-env.sh" ]; then
+  print "Sourcing:                    ${KMS_CONFIG}/kms-env.sh"
+  source ${KMS_CONFIG}/kms-env.sh
+  grep "^ *export " ${KMS_CONFIG}/kms-env.sh | sed 's/ *export/  setting/'
+fi
+
+# verify that the sourced env file didn't change KMS_HOME
+# if so, warn and revert
+#
+if [ "${KMS_HOME}" != "${kms_home}" ]; then
+  echo "WARN: KMS_HOME resetting to ''${KMS_HOME}'' ignored"
+  export KMS_HOME=${kms_home}
+fi
+
+# verify that the sourced env file didn't change KMS_CONFIG
+# if so, warn and revert
+#
+if [ "${KMS_CONFIG}" != "${kms_config}" ]; then
+  echo "WARN: KMS_CONFIG resetting to ''${KMS_CONFIG}'' ignored"
+  export KMS_CONFIG=${kms_config}
+fi
+
+if [ "${KMS_LOG}" = "" ]; then
+  export KMS_LOG=${KMS_HOME}/logs
+  print "Setting KMS_LOG:           ${KMS_LOG}"
+else
+  print "Using   KMS_LOG:           ${KMS_LOG}"
+fi
+
+if [ ! -f ${KMS_LOG} ]; then
+  mkdir -p ${KMS_LOG}
+fi
+
+if [ "${KMS_TEMP}" = "" ]; then
+  export KMS_TEMP=${KMS_HOME}/temp
+  print "Setting KMS_TEMP:           ${KMS_TEMP}"
+else
+  print "Using   KMS_TEMP:           ${KMS_TEMP}"
+fi
+
+if [ ! -f ${KMS_TEMP} ]; then
+  mkdir -p ${KMS_TEMP}
+fi
+
+if [ "${KMS_HTTP_PORT}" = "" ]; then
+  export KMS_HTTP_PORT=16000
+  print "Setting KMS_HTTP_PORT:     ${KMS_HTTP_PORT}"
+else
+  print "Using   KMS_HTTP_PORT:     ${KMS_HTTP_PORT}"
+fi
+
+if [ "${KMS_ADMIN_PORT}" = "" ]; then
+  export KMS_ADMIN_PORT=`expr $KMS_HTTP_PORT +  1`
+  print "Setting KMS_ADMIN_PORT:     ${KMS_ADMIN_PORT}"
+else
+  print "Using   KMS_ADMIN_PORT:     ${KMS_ADMIN_PORT}"
+fi
+
+if [ "${KMS_SSL_KEYSTORE_FILE}" = "" ]; then
+  export KMS_SSL_KEYSTORE_FILE=${HOME}/.keystore
+  print "Setting KMS_SSL_KEYSTORE_FILE:     ${KMS_SSL_KEYSTORE_FILE}"
+else
+  print "Using   KMS_SSL_KEYSTORE_FILE:     ${KMS_SSL_KEYSTORE_FILE}"
+fi
+
+if [ "${KMS_SSL_KEYSTORE_PASS}" = "" ]; then
+  export KMS_SSL_KEYSTORE_PASS=password
+  print "Setting KMS_SSL_KEYSTORE_PASS:     ${KMS_SSL_KEYSTORE_PASS}"
+else
+  print "Using   KMS_SSL_KEYSTORE_PASS:     ${KMS_SSL_KEYSTORE_PASS}"
+fi
+
+if [ "${CATALINA_BASE}" = "" ]; then
+  export CATALINA_BASE=${KMS_HOME}/share/hadoop/kms/tomcat
+  print "Setting CATALINA_BASE:       ${CATALINA_BASE}"
+else
+  print "Using   CATALINA_BASE:       ${CATALINA_BASE}"
+fi
+
+if [ "${KMS_CATALINA_HOME}" = "" ]; then
+  export KMS_CATALINA_HOME=${CATALINA_BASE}
+  print "Setting KMS_CATALINA_HOME:       ${KMS_CATALINA_HOME}"
+else
+  print "Using   KMS_CATALINA_HOME:       ${KMS_CATALINA_HOME}"
+fi
+
+if [ "${CATALINA_OUT}" = "" ]; then
+  export CATALINA_OUT=${KMS_LOG}/kms-catalina.out
+  print "Setting CATALINA_OUT:        ${CATALINA_OUT}"
+else
+  print "Using   CATALINA_OUT:        ${CATALINA_OUT}"
+fi
+
+if [ "${CATALINA_PID}" = "" ]; then
+  export CATALINA_PID=/tmp/kms.pid
+  print "Setting CATALINA_PID:        ${CATALINA_PID}"
+else
+  print "Using   CATALINA_PID:        ${CATALINA_PID}"
+fi
+
+print

+ 60 - 0
hadoop-common-project/hadoop-kms/src/main/sbin/kms.sh

@@ -0,0 +1,60 @@
+#!/bin/bash
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#  http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+#
+
+# resolve links - $0 may be a softlink
+PRG="${0}"
+
+while [ -h "${PRG}" ]; do
+  ls=`ls -ld "${PRG}"`
+  link=`expr "$ls" : '.*-> \(.*\)$'`
+  if expr "$link" : '/.*' > /dev/null; then
+    PRG="$link"
+  else
+    PRG=`dirname "${PRG}"`/"$link"
+  fi
+done
+
+BASEDIR=`dirname ${PRG}`
+BASEDIR=`cd ${BASEDIR}/..;pwd`
+
+KMS_SILENT=${KMS_SILENT:-true}
+
+source ${HADOOP_LIBEXEC_DIR:-${BASEDIR}/libexec}/kms-config.sh
+
+# The Java System property 'kms.http.port' it is not used by Kms,
+# it is used in Tomcat's server.xml configuration file
+#
+print "Using   CATALINA_OPTS:       ${CATALINA_OPTS}"
+
+catalina_opts="-Dkms.home.dir=${KMS_HOME}";
+catalina_opts="${catalina_opts} -Dkms.config.dir=${KMS_CONFIG}";
+catalina_opts="${catalina_opts} -Dkms.log.dir=${KMS_LOG}";
+catalina_opts="${catalina_opts} -Dkms.temp.dir=${KMS_TEMP}";
+catalina_opts="${catalina_opts} -Dkms.admin.port=${KMS_ADMIN_PORT}";
+catalina_opts="${catalina_opts} -Dkms.http.port=${KMS_HTTP_PORT}";
+catalina_opts="${catalina_opts} -Dkms.ssl.keystore.file=${KMS_SSL_KEYSTORE_FILE}";
+catalina_opts="${catalina_opts} -Dkms.ssl.keystore.pass=${KMS_SSL_KEYSTORE_PASS}";
+
+print "Adding to CATALINA_OPTS:     ${catalina_opts}"
+
+export CATALINA_OPTS="${CATALINA_OPTS} ${catalina_opts}"
+
+# A bug in catalina.sh script does not use CATALINA_OPTS for stopping the server
+#
+if [ "${1}" = "stop" ]; then
+  export JAVA_OPTS=${CATALINA_OPTS}
+fi
+
+exec ${KMS_CATALINA_HOME}/bin/catalina.sh "$@"

+ 16 - 0
hadoop-common-project/hadoop-kms/src/main/tomcat/ROOT/WEB-INF/web.xml

@@ -0,0 +1,16 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<web-app version="2.4" xmlns="http://java.sun.com/xml/ns/j2ee">
+</web-app>

+ 27 - 0
hadoop-common-project/hadoop-kms/src/main/tomcat/ROOT/index.html

@@ -0,0 +1,27 @@
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+
+
+-->
+<html>
+<head>
+  <title>Hadoop KMS</title>
+</head>
+<body>
+<h1>Hadoop KMS</h1>
+<ul>
+  <li>KMS REST API end-point <b>/kms/v1/*</b></li>
+  <li><a href="/kms/jmx">KMS JMX JSON end-point</a></li>
+</ul>
+</body>
+</html>

+ 67 - 0
hadoop-common-project/hadoop-kms/src/main/tomcat/logging.properties

@@ -0,0 +1,67 @@
+#
+#  All Rights Reserved.
+#
+# 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.
+
+handlers = 1catalina.org.apache.juli.FileHandler, 2localhost.org.apache.juli.FileHandler, 3manager.org.apache.juli.FileHandler, 4host-manager.org.apache.juli.FileHandler, java.util.logging.ConsoleHandler
+
+.handlers = 1catalina.org.apache.juli.FileHandler, java.util.logging.ConsoleHandler
+
+############################################################
+# Handler specific properties.
+# Describes specific configuration info for Handlers.
+############################################################
+
+1catalina.org.apache.juli.FileHandler.level = FINE
+1catalina.org.apache.juli.FileHandler.directory = ${kms.log.dir}
+1catalina.org.apache.juli.FileHandler.prefix = kms-catalina.
+
+2localhost.org.apache.juli.FileHandler.level = FINE
+2localhost.org.apache.juli.FileHandler.directory = ${kms.log.dir}
+2localhost.org.apache.juli.FileHandler.prefix = kms-localhost.
+
+3manager.org.apache.juli.FileHandler.level = FINE
+3manager.org.apache.juli.FileHandler.directory = ${kms.log.dir}
+3manager.org.apache.juli.FileHandler.prefix = kms-manager.
+
+4host-manager.org.apache.juli.FileHandler.level = FINE
+4host-manager.org.apache.juli.FileHandler.directory = ${kms.log.dir}
+4host-manager.org.apache.juli.FileHandler.prefix = kms-host-manager.
+
+java.util.logging.ConsoleHandler.level = FINE
+java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter
+
+
+############################################################
+# Facility specific properties.
+# Provides extra control for each logger.
+############################################################
+
+org.apache.catalina.core.ContainerBase.[Catalina].[localhost].level = INFO
+org.apache.catalina.core.ContainerBase.[Catalina].[localhost].handlers = 2localhost.org.apache.juli.FileHandler
+
+org.apache.catalina.core.ContainerBase.[Catalina].[localhost].[/manager].level = INFO
+org.apache.catalina.core.ContainerBase.[Catalina].[localhost].[/manager].handlers = 3manager.org.apache.juli.FileHandler
+
+org.apache.catalina.core.ContainerBase.[Catalina].[localhost].[/host-manager].level = INFO
+org.apache.catalina.core.ContainerBase.[Catalina].[localhost].[/host-manager].handlers = 4host-manager.org.apache.juli.FileHandler
+
+# For example, set the com.xyz.foo logger to only log SEVERE
+# messages:
+#org.apache.catalina.startup.ContextConfig.level = FINE
+#org.apache.catalina.startup.HostConfig.level = FINE
+#org.apache.catalina.session.ManagerBase.level = FINE
+#org.apache.catalina.core.AprLifecycleListener.level=FINE

+ 153 - 0
hadoop-common-project/hadoop-kms/src/main/tomcat/server.xml

@@ -0,0 +1,153 @@
+<?xml version='1.0' encoding='utf-8'?>
+<!--
+
+   All Rights Reserved.
+
+  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.
+-->
+<!-- Note:  A "Server" is not itself a "Container", so you may not
+     define subcomponents such as "Valves" at this level.
+     Documentation at /docs/config/server.html
+ -->
+<Server port="${kms.admin.port}" shutdown="SHUTDOWN">
+
+  <!--APR library loader. Documentation at /docs/apr.html -->
+  <Listener className="org.apache.catalina.core.AprLifecycleListener"
+            SSLEngine="on"/>
+  <!--Initialize Jasper prior to webapps are loaded. Documentation at /docs/jasper-howto.html -->
+  <Listener className="org.apache.catalina.core.JasperListener"/>
+  <!-- Prevent memory leaks due to use of particular java/javax APIs-->
+  <Listener
+    className="org.apache.catalina.core.JreMemoryLeakPreventionListener"/>
+  <!-- JMX Support for the Tomcat server. Documentation at /docs/non-existent.html -->
+  <Listener className="org.apache.catalina.mbeans.ServerLifecycleListener"/>
+  <Listener
+    className="org.apache.catalina.mbeans.GlobalResourcesLifecycleListener"/>
+
+  <!-- Global JNDI resources
+       Documentation at /docs/jndi-resources-howto.html
+  -->
+  <GlobalNamingResources>
+    <!-- Editable user database that can also be used by
+         UserDatabaseRealm to authenticate users
+    -->
+    <Resource name="UserDatabase" auth="Container"
+              type="org.apache.catalina.UserDatabase"
+              description="User database that can be updated and saved"
+              factory="org.apache.catalina.users.MemoryUserDatabaseFactory"
+              pathname="conf/tomcat-users.xml"/>
+  </GlobalNamingResources>
+
+  <!-- A "Service" is a collection of one or more "Connectors" that share
+       a single "Container" Note:  A "Service" is not itself a "Container",
+       so you may not define subcomponents such as "Valves" at this level.
+       Documentation at /docs/config/service.html
+   -->
+  <Service name="Catalina">
+
+    <!--The connectors can use a shared executor, you can define one or more named thread pools-->
+    <!--
+    <Executor name="tomcatThreadPool" namePrefix="catalina-exec-"
+        maxThreads="150" minSpareThreads="4"/>
+    -->
+
+
+    <!-- A "Connector" represents an endpoint by which requests are received
+         and responses are returned. Documentation at :
+         Java HTTP Connector: /docs/config/http.html (blocking & non-blocking)
+         Java AJP  Connector: /docs/config/ajp.html
+         APR (HTTP/AJP) Connector: /docs/apr.html
+         Define a non-SSL HTTP/1.1 Connector on port ${kms.http.port}
+    -->
+    <Connector port="${kms.http.port}" protocol="HTTP/1.1"
+               connectionTimeout="20000"
+               redirectPort="8443"/>
+    <!-- A "Connector" using the shared thread pool-->
+    <!--
+    <Connector executor="tomcatThreadPool"
+               port="${kms.http.port}" protocol="HTTP/1.1"
+               connectionTimeout="20000"
+               redirectPort="8443" />
+    -->
+    <!-- Define a SSL HTTP/1.1 Connector on port 8443
+         This connector uses the JSSE configuration, when using APR, the
+         connector should be using the OpenSSL style configuration
+         described in the APR documentation -->
+    <!--
+    <Connector port="8443" protocol="HTTP/1.1" SSLEnabled="true"
+               maxThreads="150" scheme="https" secure="true"
+               clientAuth="false" sslProtocol="TLS" />
+    -->
+
+    <!-- Define an AJP 1.3 Connector on port 8009 -->
+
+
+    <!-- An Engine represents the entry point (within Catalina) that processes
+ every request.  The Engine implementation for Tomcat stand alone
+ analyzes the HTTP headers included with the request, and passes them
+ on to the appropriate Host (virtual host).
+ Documentation at /docs/config/engine.html -->
+
+    <!-- You should set jvmRoute to support load-balancing via AJP ie :
+    <Engine name="Catalina" defaultHost="localhost" jvmRoute="jvm1">
+    -->
+    <Engine name="Catalina" defaultHost="localhost">
+
+      <!--For clustering, please take a look at documentation at:
+          /docs/cluster-howto.html  (simple how to)
+          /docs/config/cluster.html (reference documentation) -->
+      <!--
+      <Cluster className="org.apache.catalina.ha.tcp.SimpleTcpCluster"/>
+      -->
+
+      <!-- The request dumper valve dumps useful debugging information about
+           the request and response data received and sent by Tomcat.
+           Documentation at: /docs/config/valve.html -->
+      <!--
+      <Valve className="org.apache.catalina.valves.RequestDumperValve"/>
+      -->
+
+      <!-- This Realm uses the UserDatabase configured in the global JNDI
+           resources under the key "UserDatabase".  Any edits
+           that are performed against this UserDatabase are immediately
+           available for use by the Realm.  -->
+      <Realm className="org.apache.catalina.realm.UserDatabaseRealm"
+             resourceName="UserDatabase"/>
+
+      <!-- Define the default virtual host
+           Note: XML Schema validation will not work with Xerces 2.2.
+       -->
+      <Host name="localhost" appBase="webapps"
+            unpackWARs="true" autoDeploy="true"
+            xmlValidation="false" xmlNamespaceAware="false">
+
+        <!-- SingleSignOn valve, share authentication between web applications
+             Documentation at: /docs/config/valve.html -->
+        <!--
+        <Valve className="org.apache.catalina.authenticator.SingleSignOn" />
+        -->
+
+        <!-- Access log processes all example.
+             Documentation at: /docs/config/valve.html -->
+        <!--
+        <Valve className="org.apache.catalina.valves.AccessLogValve" directory="logs"
+               prefix="localhost_access_log." suffix=".txt" pattern="common" resolveHosts="false"/>
+        -->
+
+      </Host>
+    </Engine>
+  </Service>
+</Server>

+ 135 - 0
hadoop-common-project/hadoop-kms/src/main/tomcat/ssl-server.xml

@@ -0,0 +1,135 @@
+<?xml version='1.0' encoding='utf-8'?>
+<!--
+
+   All Rights Reserved.
+
+  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.
+-->
+<!-- Note:  A "Server" is not itself a "Container", so you may not
+     define subcomponents such as "Valves" at this level.
+     Documentation at /docs/config/server.html
+ -->
+<Server port="${kms.admin.port}" shutdown="SHUTDOWN">
+
+  <!--APR library loader. Documentation at /docs/apr.html -->
+  <Listener className="org.apache.catalina.core.AprLifecycleListener"
+            SSLEngine="on"/>
+  <!--Initialize Jasper prior to webapps are loaded. Documentation at /docs/jasper-howto.html -->
+  <Listener className="org.apache.catalina.core.JasperListener"/>
+  <!-- Prevent memory leaks due to use of particular java/javax APIs-->
+  <Listener
+    className="org.apache.catalina.core.JreMemoryLeakPreventionListener"/>
+  <!-- JMX Support for the Tomcat server. Documentation at /docs/non-existent.html -->
+  <Listener className="org.apache.catalina.mbeans.ServerLifecycleListener"/>
+  <Listener
+    className="org.apache.catalina.mbeans.GlobalResourcesLifecycleListener"/>
+
+  <!-- Global JNDI resources
+       Documentation at /docs/jndi-resources-howto.html
+  -->
+  <GlobalNamingResources>
+    <!-- Editable user database that can also be used by
+         UserDatabaseRealm to authenticate users
+    -->
+    <Resource name="UserDatabase" auth="Container"
+              type="org.apache.catalina.UserDatabase"
+              description="User database that can be updated and saved"
+              factory="org.apache.catalina.users.MemoryUserDatabaseFactory"
+              pathname="conf/tomcat-users.xml"/>
+  </GlobalNamingResources>
+
+  <!-- A "Service" is a collection of one or more "Connectors" that share
+       a single "Container" Note:  A "Service" is not itself a "Container",
+       so you may not define subcomponents such as "Valves" at this level.
+       Documentation at /docs/config/service.html
+   -->
+  <Service name="Catalina">
+
+    <!--The connectors can use a shared executor, you can define one or more named thread pools-->
+    <!--
+    <Executor name="tomcatThreadPool" namePrefix="catalina-exec-"
+        maxThreads="150" minSpareThreads="4"/>
+    -->
+
+    <!-- Define a SSL HTTP/1.1 Connector on port 8443
+         This connector uses the JSSE configuration, when using APR, the
+         connector should be using the OpenSSL style configuration
+         described in the APR documentation -->
+    <Connector port="${kms.http.port}" protocol="HTTP/1.1" SSLEnabled="true"
+               maxThreads="150" scheme="https" secure="true"
+               clientAuth="false" sslProtocol="TLS"
+               keystoreFile="${kms.ssl.keystore.file}"
+               keystorePass="${kms.ssl.keystore.pass}"/>
+
+    <!-- Define an AJP 1.3 Connector on port 8009 -->
+
+
+    <!-- An Engine represents the entry point (within Catalina) that processes
+ every request.  The Engine implementation for Tomcat stand alone
+ analyzes the HTTP headers included with the request, and passes them
+ on to the appropriate Host (virtual host).
+ Documentation at /docs/config/engine.html -->
+
+    <!-- You should set jvmRoute to support load-balancing via AJP ie :
+    <Engine name="Catalina" defaultHost="localhost" jvmRoute="jvm1">
+    -->
+    <Engine name="Catalina" defaultHost="localhost">
+
+      <!--For clustering, please take a look at documentation at:
+          /docs/cluster-howto.html  (simple how to)
+          /docs/config/cluster.html (reference documentation) -->
+      <!--
+      <Cluster className="org.apache.catalina.ha.tcp.SimpleTcpCluster"/>
+      -->
+
+      <!-- The request dumper valve dumps useful debugging information about
+           the request and response data received and sent by Tomcat.
+           Documentation at: /docs/config/valve.html -->
+      <!--
+      <Valve className="org.apache.catalina.valves.RequestDumperValve"/>
+      -->
+
+      <!-- This Realm uses the UserDatabase configured in the global JNDI
+           resources under the key "UserDatabase".  Any edits
+           that are performed against this UserDatabase are immediately
+           available for use by the Realm.  -->
+      <Realm className="org.apache.catalina.realm.UserDatabaseRealm"
+             resourceName="UserDatabase"/>
+
+      <!-- Define the default virtual host
+           Note: XML Schema validation will not work with Xerces 2.2.
+       -->
+      <Host name="localhost" appBase="webapps"
+            unpackWARs="true" autoDeploy="true"
+            xmlValidation="false" xmlNamespaceAware="false">
+
+        <!-- SingleSignOn valve, share authentication between web applications
+             Documentation at: /docs/config/valve.html -->
+        <!--
+        <Valve className="org.apache.catalina.authenticator.SingleSignOn" />
+        -->
+
+        <!-- Access log processes all example.
+             Documentation at: /docs/config/valve.html -->
+        <!--
+        <Valve className="org.apache.catalina.valves.AccessLogValve" directory="logs"
+               prefix="localhost_access_log." suffix=".txt" pattern="common" resolveHosts="false"/>
+        -->
+
+      </Host>
+    </Engine>
+  </Service>
+</Server>

+ 78 - 0
hadoop-common-project/hadoop-kms/src/main/webapp/WEB-INF/web.xml

@@ -0,0 +1,78 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<web-app version="2.4" xmlns="http://java.sun.com/xml/ns/j2ee">
+
+  <listener>
+    <listener-class>org.apache.hadoop.crypto.key.kms.server.KMSWebApp</listener-class>
+  </listener>
+
+  <servlet>
+    <servlet-name>webservices-driver</servlet-name>
+    <servlet-class>com.sun.jersey.spi.container.servlet.ServletContainer</servlet-class>
+    <init-param>
+      <param-name>com.sun.jersey.config.property.packages</param-name>
+      <param-value>org.apache.hadoop.crypto.key.kms.server</param-value>
+    </init-param>
+
+    <!-- Enables detailed Jersey request/response logging -->
+    <!--
+    <init-param>
+        <param-name>com.sun.jersey.spi.container.ContainerRequestFilters</param-name>
+        <param-value>com.sun.jersey.api.container.filter.LoggingFilter</param-value>
+    </init-param>
+    <init-param>
+        <param-name>com.sun.jersey.spi.container.ContainerResponseFilters</param-name>
+        <param-value>com.sun.jersey.api.container.filter.LoggingFilter</param-value>
+    </init-param>
+    -->
+    <load-on-startup>1</load-on-startup>
+  </servlet>
+
+  <servlet>
+    <servlet-name>jmx-servlet</servlet-name>
+    <servlet-class>org.apache.hadoop.jmx.JMXJsonServlet</servlet-class>
+  </servlet>
+
+  <servlet-mapping>
+    <servlet-name>webservices-driver</servlet-name>
+    <url-pattern>/*</url-pattern>
+  </servlet-mapping>
+
+  <servlet-mapping>
+    <servlet-name>jmx-servlet</servlet-name>
+    <url-pattern>/jmx</url-pattern>
+  </servlet-mapping>
+
+  <filter>
+    <filter-name>authFilter</filter-name>
+    <filter-class>org.apache.hadoop.crypto.key.kms.server.KMSAuthenticationFilter</filter-class>
+  </filter>
+
+  <filter>
+    <filter-name>MDCFilter</filter-name>
+    <filter-class>org.apache.hadoop.crypto.key.kms.server.KMSMDCFilter</filter-class>
+  </filter>
+
+  <filter-mapping>
+    <filter-name>authFilter</filter-name>
+    <url-pattern>/*</url-pattern>
+  </filter-mapping>
+
+  <filter-mapping>
+    <filter-name>MDCFilter</filter-name>
+    <url-pattern>/*</url-pattern>
+  </filter-mapping>
+
+</web-app>

+ 487 - 0
hadoop-common-project/hadoop-kms/src/site/apt/index.apt.vm

@@ -0,0 +1,487 @@
+~~ Licensed under the Apache License, Version 2.0 (the "License");
+~~ you may not use this file except in compliance with the License.
+~~ You may obtain a copy of the License at
+~~
+~~ http://www.apache.org/licenses/LICENSE-2.0
+~~
+~~ Unless required by applicable law or agreed to in writing, software
+~~ distributed under the License is distributed on an "AS IS" BASIS,
+~~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+~~ See the License for the specific language governing permissions and
+~~ limitations under the License.
+
+  ---
+  Hadoop KMS - Documentation Sets ${project.version}
+  ---
+  ---
+  ${maven.build.timestamp}
+
+Hadoop Key Management Server (KMS) - Documentation Sets ${project.version}
+
+  Hadoop KMS is a cryptographic key management server based on Hadoop's
+  <<KeyProvider>> API.
+
+  It provides a client and a server components which communicate over
+  HTTP using a REST API.
+
+  The client is a KeyProvider implementation interacts with the KMS
+  using the KMS HTTP REST API.
+
+  KMS and its client have built-in security and they support HTTP SPNEGO
+  Kerberos authentication and HTTPS secure transport.
+
+  KMS is a Java web-application and it runs using a pre-configured Tomcat
+  bundled with the Hadoop distribution.
+
+* KMS Client Configuration
+
+  The KMS client <<<KeyProvider>>> uses the <<kms>> scheme, and the embedded
+  URL must be the URL of the KMS. For example, for a KMS running
+  on <<<http://localhost:16000/kms>>>, the KeyProvider URI is
+  <<<kms://http@localhost:16000/kms>>>. And, for a KMS running on
+  <<<https://localhost:16000/kms>>>, the KeyProvider URI is
+  <<<kms://https@localhost:16000/kms>>>
+
+* KMS
+
+** KMS Configuration
+
+  Configure the KMS backing KeyProvider properties
+  in the <<<etc/hadoop/kms-site.xml>>> configuration file:
+
++---+
+  <property>
+    <name>hadoop.security.key.provider.path</name>
+    <value>jceks://file@/${user.home}/kms.keystore</value>
+  </property>
+
+  <property>
+    <name>hadoop.security.keystore.java-keystore-provider.password-file</name>
+    <value>kms.keystore.password</value>
+  </property>
++---+
+
+  The password file is looked up in the Hadoop's configuration directory via the
+  classpath.
+
+  NOTE: You need to restart the KMS for the configuration changes to take
+  effect.
+
+** KMS Cache
+
+  KMS caches keys for short period of time to avoid excessive hits to the
+  underlying key provider.
+
+  The cache is used with the following 2 methods only, <<<getCurrentKey()>>>
+  and <<<getKeyVersion()>>>.
+
+  For the <<<getCurrentKey()>>> method, cached entries are kept for a maximum
+  of 1000 millisecond regardless the number of times the key is being access
+  (to avoid stale keys to be considered current).
+
+  For the <<<getKeyVersion()>>> method, cached entries are kept with a default
+  inactivity timeout of 10000 milliseconds. This time out is configurable via
+  the following property in the <<<etc/hadoop/kms-site.xml>>> configuration
+  file:
+
++---+
+  <property>
+    <name>hadoop.kms.cache.timeout.ms</name>
+    <value>10000</value>
+  </property>
++---+
+
+** Start/Stop the KMS
+
+  To start/stop KMS use KMS's bin/kms.sh script. For example:
+
++---+
+hadoop-${project.version} $ sbin/kms.sh start
++---+
+
+  NOTE: Invoking the script without any parameters list all possible
+  parameters (start, stop, run, etc.). The <<<kms.sh>>> script is a wrapper
+  for Tomcat's <<<catalina.sh>>> script that sets the environment variables
+  and Java System properties required to run KMS.
+
+** Embedded Tomcat Configuration
+
+  To configure the embedded Tomcat go to the <<<share/hadoop/kms/tomcat/conf>>>.
+
+  KMS pre-configures the HTTP and Admin ports in Tomcat's <<<server.xml>>> to
+  16000 and 16001.
+
+  Tomcat logs are also preconfigured to go to Hadoop's <<<logs/>>> directory.
+
+  The following environment variables (which can be set in KMS's
+  <<<etc/hadoop/kms-env.sh>>> script) can be used to alter those values:
+
+  * KMS_HTTP_PORT
+
+  * KMS_ADMIN_PORT
+
+  * KMS_LOG
+
+  NOTE: You need to restart the KMS for the configuration changes to take
+  effect.
+
+** KMS Security Configuration
+
+*** Enabling Kerberos HTTP SPNEGO Authentication
+
+  Configure the Kerberos <<<etc/krb5.conf>>> file with the information of your
+  KDC server.
+
+  Create a service principal and its keytab for the KMS, it must be an
+  <<<HTTP>>> service principal.
+
+  Configure KMS <<<etc/hadoop/kms-site.xml>>> with the correct security values,
+  for example:
+
++---+
+  <property>
+    <name>hadoop.kms.authentication.type</name>
+    <value>kerberos</value>
+  </property>
+
+  <property>
+    <name>hadoop.kms.authentication.kerberos.keytab</name>
+    <value>${user.home}/kms.keytab</value>
+  </property>
+
+  <property>
+    <name>hadoop.kms.authentication.kerberos.principal</name>
+    <value>HTTP/localhost</value>
+  </property>
+
+  <property>
+    <name>hadoop.kms.authentication.kerberos.name.rules</name>
+    <value>DEFAULT</value>
+  </property>
++---+
+
+  NOTE: You need to restart the KMS for the configuration changes to take
+  effect.
+
+*** KMS over HTTPS (SSL)
+
+  To configure KMS to work over HTTPS the following 2 properties must be
+  set in the <<<etc/hadoop/kms_env.sh>>> script (shown with default values):
+
+    * KMS_SSL_KEYSTORE_FILE=${HOME}/.keystore
+
+    * KMS_SSL_KEYSTORE_PASS=password
+
+  In the KMS <<<tomcat/conf>>> directory, replace the <<<server.xml>>> file
+  with the provided <<<ssl-server.xml>>> file.
+
+  You need to create an SSL certificate for the KMS. As the
+  <<<kms>>> Unix user, using the Java <<<keytool>>> command to create the
+  SSL certificate:
+
++---+
+$ keytool -genkey -alias tomcat -keyalg RSA
++---+
+
+  You will be asked a series of questions in an interactive prompt.  It will
+  create the keystore file, which will be named <<.keystore>> and located in the
+  <<<kms>>> user home directory.
+
+  The password you enter for "keystore password" must match the  value of the
+  <<<KMS_SSL_KEYSTORE_PASS>>> environment variable set in the
+  <<<kms-env.sh>>> script in the configuration directory.
+
+  The answer to "What is your first and last name?" (i.e. "CN") must be the
+  hostname of the machine where the KMS will be running.
+
+  NOTE: You need to restart the KMS for the configuration changes to take
+  effect.
+
+*** KMS Access Control
+
+  KMS ACLs configuration are defined in the KMS <<<etc/hadoop/kms-acls.xml>>>
+  configuration file. This file is hot-reloaded when it changes.
+
+  KMS supports a fine grained access control via a set ACL
+  configuration properties:
+
++---+
+  <property>
+    <name>hadoop.kms.acl.CREATE</name>
+    <value>*</value>
+    <description>
+      ACL for create-key operations.
+      If the user does is not in the GET ACL, the key material is not returned
+      as part of the response.
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.kms.acl.DELETE</name>
+    <value>*</value>
+    <description>
+      ACL for delete-key operations.
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.kms.acl.ROLLOVER</name>
+    <value>*</value>
+    <description>
+      ACL for rollover-key operations.
+      If the user does is not in the GET ACL, the key material is not returned
+      as part of the response.
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.kms.acl.GET</name>
+    <value>*</value>
+    <description>
+      ACL for get-key-version and get-current-key operations.
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.kms.acl.GET_KEYS</name>
+    <value>*</value>
+    <description>
+      ACL for get-keys operation.
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.kms.acl.GET_METADATA</name>
+    <value>*</value>
+    <description>
+      ACL for get-key-metadata and get-keys-metadata operations.
+    </description>
+  </property>
+
+  <property>
+    <name>hadoop.kms.acl.SET_KEY_MATERIAL</name>
+    <value>*</value>
+    <description>
+        Complimentary ACL for CREATE and ROLLOVER operation to allow the client
+        to provide the key material when creating or rolling a key.
+    </description>
+  </property>
++---+
+
+** KMS HTTP REST API
+
+*** Create a Key
+
+  <REQUEST:>
+
++---+
+POST http://HOST:PORT/kms/v1/keys
+Content-Type: application/json
+
+{
+  "name"        : "<key-name>",
+  "cipher"      : "<cipher>",
+  "length"      : <length>,        //int
+  "material"    : "<material>",    //base64
+  "description" : "<description>"
+}
++---+
+  
+  <RESPONSE:>
+  
++---+
+201 CREATED
+LOCATION: http://HOST:PORT/kms/v1/key/<key-name>
+Content-Type: application/json
+
+{
+  "name"        : "versionName",
+  "material"    : "<material>",    //base64, not present without GET ACL
+}
++---+
+
+*** Rollover Key
+
+  <REQUEST:>
+
++---+
+POST http://HOST:PORT/kms/v1/key/<key-name>
+Content-Type: application/json
+
+{
+  "material"    : "<material>",
+}
++---+
+
+  <RESPONSE:>
+
++---+
+200 OK
+Content-Type: application/json
+
+{
+  "name"        : "versionName",
+  "material"    : "<material>",    //base64, not present without GET ACL
+}
++---+
+
+*** Delete Key
+
+  <REQUEST:>
+
++---+
+DELETE http://HOST:PORT/kms/v1/key/<key-name>
++---+
+
+  <RESPONSE:>
+
++---+
+200 OK
++---+
+
+*** Get Key Metadata
+
+  <REQUEST:>
+
++---+
+GET http://HOST:PORT/kms/v1/key/<key-name>/_metadata
++---+
+
+  <RESPONSE:>
+
++---+
+200 OK
+Content-Type: application/json
+
+{
+  "name"        : "<key-name>",
+  "cipher"      : "<cipher>",
+  "length"      : <length>,        //int
+  "description" : "<description>",
+  "created"     : <millis-epoc>,   //long
+  "versions"    : <versions>       //int
+}
++---+
+
+*** Get Current Key
+
+  <REQUEST:>
+
++---+
+GET http://HOST:PORT/kms/v1/key/<key-name>/_currentversion
++---+
+
+  <RESPONSE:>
+
++---+
+200 OK
+Content-Type: application/json
+
+{
+  "name"        : "versionName",
+  "material"    : "<material>",    //base64
+}
++---+
+
+*** Get Key Version
+
+  <REQUEST:>
+
++---+
+GET http://HOST:PORT/kms/v1/keyversion/<version-name>
++---+
+
+  <RESPONSE:>
+
++---+
+200 OK
+Content-Type: application/json
+
+{
+  "name"        : "versionName",
+  "material"    : "<material>",    //base64
+}
++---+
+
+*** Get Key Versions
+
+  <REQUEST:>
+
++---+
+GET http://HOST:PORT/kms/v1/key/<key-name>/_versions
++---+
+
+  <RESPONSE:>
+
++---+
+200 OK
+Content-Type: application/json
+
+[
+  {
+    "name"        : "versionName",
+    "material"    : "<material>",    //base64
+  },
+  {
+    "name"        : "versionName",
+    "material"    : "<material>",    //base64
+  },
+  ...
+]
++---+
+
+*** Get Key Names
+
+  <REQUEST:>
+
++---+
+GET http://HOST:PORT/kms/v1/keys/names
++---+
+
+  <RESPONSE:>
+
++---+
+200 OK
+Content-Type: application/json
+
+[
+  "<key-name>",
+  "<key-name>",
+  ...
+]
++---+
+
+*** Get Keys Metadata
+
++---+
+GET http://HOST:PORT/kms/v1/keys/metadata?key=<key-name>&key=<key-name>,...
++---+
+
+  <RESPONSE:>
+
++---+
+200 OK
+Content-Type: application/json
+
+[
+  {
+    "name"        : "<key-name>",
+    "cipher"      : "<cipher>",
+    "length"      : <length>,        //int
+    "description" : "<description>",
+    "created"     : <millis-epoc>,   //long
+    "versions"    : <versions>       //int
+  },
+  {
+    "name"        : "<key-name>",
+    "cipher"      : "<cipher>",
+    "length"      : <length>,        //int
+    "description" : "<description>",
+    "created"     : <millis-epoc>,   //long
+    "versions"    : <versions>       //int
+  },
+  ...
+]
++---+
+
+  \[ {{{./index.html}Go Back}} \]

+ 29 - 0
hadoop-common-project/hadoop-kms/src/site/resources/css/site.css

@@ -0,0 +1,29 @@
+/*
+* 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.
+*/
+#banner {
+  height: 93px;
+  background: none;
+}
+
+#bannerLeft img {
+  margin-left: 30px;
+  margin-top: 10px;
+}
+
+#bannerRight img {
+  margin: 17px;
+}

+ 29 - 0
hadoop-common-project/hadoop-kms/src/site/site.xml

@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project name="Hadoop KMS">
+
+    <skin>
+      <groupId>org.apache.maven.skins</groupId>
+      <artifactId>maven-stylus-skin</artifactId>
+      <version>1.2</version>
+    </skin>
+
+    <body>
+      <links>
+        <item name="Apache Hadoop" href="http://hadoop.apache.org/"/>
+      </links>
+    </body>
+
+</project>

+ 806 - 0
hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java

@@ -0,0 +1,806 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms.server;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
+import org.apache.hadoop.minikdc.MiniKdc;
+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.BeforeClass;
+import org.junit.Test;
+import org.mortbay.jetty.Connector;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.security.SslSocketConnector;
+import org.mortbay.jetty.webapp.WebAppContext;
+
+import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.LoginContext;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.Writer;
+import java.net.InetAddress;
+import java.net.MalformedURLException;
+import java.net.ServerSocket;
+import java.net.URI;
+import java.net.URL;
+import java.security.Principal;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+
+public class TestKMS {
+
+  public static File getTestDir() throws Exception {
+    File file = new File("dummy");
+    file = file.getAbsoluteFile();
+    file = file.getParentFile();
+    file = new File(file, "target");
+    file = new File(file, UUID.randomUUID().toString());
+    if (!file.mkdirs()) {
+      throw new RuntimeException("Could not create test directory: " + file);
+    }
+    return file;
+  }
+
+  public static Server createJettyServer(String keyStore, String password) {
+    try {
+      boolean ssl = keyStore != null;
+      InetAddress localhost = InetAddress.getByName("localhost");
+      String host = "localhost";
+      ServerSocket ss = new ServerSocket(0, 50, localhost);
+      int port = ss.getLocalPort();
+      ss.close();
+      Server server = new Server(0);
+      if (!ssl) {
+        server.getConnectors()[0].setHost(host);
+        server.getConnectors()[0].setPort(port);
+      } else {
+        SslSocketConnector c = new SslSocketConnector();
+        c.setHost(host);
+        c.setPort(port);
+        c.setNeedClientAuth(false);
+        c.setKeystore(keyStore);
+        c.setKeystoreType("jks");
+        c.setKeyPassword(password);
+        server.setConnectors(new Connector[]{c});
+      }
+      return server;
+    } catch (Exception ex) {
+      throw new RuntimeException("Could not start embedded servlet container, "
+          + ex.getMessage(), ex);
+    }
+  }
+
+  public static URL getJettyURL(Server server) {
+    boolean ssl = server.getConnectors()[0].getClass()
+        == SslSocketConnector.class;
+    try {
+      String scheme = (ssl) ? "https" : "http";
+      return new URL(scheme + "://" +
+          server.getConnectors()[0].getHost() + ":" +
+          server.getConnectors()[0].getPort());
+    } catch (MalformedURLException ex) {
+      throw new RuntimeException("It should never happen, " + ex.getMessage(),
+          ex);
+    }
+  }
+
+  public static abstract class KMSCallable implements Callable<Void> {
+    private URL kmsUrl;
+
+    protected URL getKMSUrl() {
+      return kmsUrl;
+    }
+  }
+
+  protected void runServer(String keystore, String password, File confDir,
+      KMSCallable callable) throws Exception {
+    System.setProperty(KMSConfiguration.KMS_CONFIG_DIR,
+        confDir.getAbsolutePath());
+    System.setProperty("log4j.configuration", "log4j.properties");
+    Server jetty = createJettyServer(keystore, password);
+    try {
+      ClassLoader cl = Thread.currentThread().getContextClassLoader();
+      URL url = cl.getResource("webapp");
+      if (url == null) {
+        throw new RuntimeException(
+            "Could not find webapp/ dir in test classpath");
+      }
+      WebAppContext context = new WebAppContext(url.getPath(), "/kms");
+      jetty.addHandler(context);
+      jetty.start();
+      url = new URL(getJettyURL(jetty), "kms");
+      System.out.println("Test KMS running at: " + url);
+      callable.kmsUrl = url;
+      callable.call();
+    } finally {
+      if (jetty != null && jetty.isRunning()) {
+        try {
+          jetty.stop();
+        } catch (Exception ex) {
+          throw new RuntimeException("Could not stop embedded Jetty, " +
+              ex.getMessage(), ex);
+        }
+      }
+    }
+  }
+
+  protected Configuration createBaseKMSConf(File keyStoreDir) throws Exception {
+    Configuration conf = new Configuration(false);
+    conf.set("hadoop.security.key.provider.path",
+        "jceks://file@/" + keyStoreDir.getAbsolutePath() + "/kms.keystore");
+    conf.set("hadoop.kms.authentication.type", "simple");
+    return conf;
+  }
+
+  protected void writeConf(File confDir, Configuration conf) throws Exception {
+    Writer writer = new FileWriter(new File(confDir,
+        KMSConfiguration.KMS_SITE_XML));
+    conf.writeXml(writer);
+    writer.close();
+
+    writer = new FileWriter(new File(confDir, KMSConfiguration.KMS_ACLS_XML));
+    conf.writeXml(writer);
+    writer.close();
+
+    //create empty core-site.xml
+    writer = new FileWriter(new File(confDir, "core-site.xml"));
+    new Configuration(false).writeXml(writer);
+    writer.close();
+  }
+
+  protected URI createKMSUri(URL kmsUrl) throws Exception {
+    String str = kmsUrl.toString();
+    str = str.replaceFirst("://", "@");
+    return new URI("kms://" + str);
+  }
+
+
+  private static class KerberosConfiguration
+      extends javax.security.auth.login.Configuration {
+    private String principal;
+    private String keytab;
+    private boolean isInitiator;
+
+    private KerberosConfiguration(String principal, File keytab,
+        boolean client) {
+      this.principal = principal;
+      this.keytab = keytab.getAbsolutePath();
+      this.isInitiator = client;
+    }
+
+    public static javax.security.auth.login.Configuration createClientConfig(
+        String principal,
+        File keytab) {
+      return new KerberosConfiguration(principal, keytab, true);
+    }
+
+    private static String getKrb5LoginModuleName() {
+      return System.getProperty("java.vendor").contains("IBM")
+             ? "com.ibm.security.auth.module.Krb5LoginModule"
+             : "com.sun.security.auth.module.Krb5LoginModule";
+    }
+
+    @Override
+    public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+      Map<String, String> options = new HashMap<String, String>();
+      options.put("keyTab", keytab);
+      options.put("principal", principal);
+      options.put("useKeyTab", "true");
+      options.put("storeKey", "true");
+      options.put("doNotPrompt", "true");
+      options.put("useTicketCache", "true");
+      options.put("renewTGT", "true");
+      options.put("refreshKrb5Config", "true");
+      options.put("isInitiator", Boolean.toString(isInitiator));
+      String ticketCache = System.getenv("KRB5CCNAME");
+      if (ticketCache != null) {
+        options.put("ticketCache", ticketCache);
+      }
+      options.put("debug", "true");
+
+      return new AppConfigurationEntry[]{
+          new AppConfigurationEntry(getKrb5LoginModuleName(),
+              AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
+              options)};
+    }
+  }
+
+  private static MiniKdc kdc;
+  private static File keytab;
+
+  @BeforeClass
+  public static void setUpMiniKdc() throws Exception {
+    File kdcDir = getTestDir();
+    Properties kdcConf = MiniKdc.createConf();
+    kdc = new MiniKdc(kdcConf, kdcDir);
+    kdc.start();
+    keytab = new File(kdcDir, "keytab");
+    List<String> principals = new ArrayList<String>();
+    principals.add("HTTP/localhost");
+    principals.add("client");
+    principals.add("client/host");
+    for (KMSACLs.Type type : KMSACLs.Type.values()) {
+      principals.add(type.toString());
+    }
+    principals.add("CREATE_MATERIAL");
+    principals.add("ROLLOVER_MATERIAL");
+    kdc.createPrincipal(keytab,
+        principals.toArray(new String[principals.size()]));
+  }
+
+  @AfterClass
+  public static void tearDownMiniKdc() throws Exception {
+    if (kdc != null) {
+      kdc.stop();
+    }
+  }
+
+  private void doAs(String user, final PrivilegedExceptionAction<Void> action)
+      throws Exception {
+    Set<Principal> principals = new HashSet<Principal>();
+    principals.add(new KerberosPrincipal(user));
+
+    //client login
+    Subject subject = new Subject(false, principals,
+        new HashSet<Object>(), new HashSet<Object>());
+    LoginContext loginContext = new LoginContext("", subject, null,
+        KerberosConfiguration.createClientConfig(user, keytab));
+    try {
+      loginContext.login();
+      subject = loginContext.getSubject();
+      Subject.doAs(subject, action);
+    } finally {
+      loginContext.logout();
+    }
+  }
+
+  public void testStartStop(final boolean ssl, final boolean kerberos)
+      throws Exception {
+    File testDir = getTestDir();
+    Configuration conf = createBaseKMSConf(testDir);
+
+    final String keystore;
+    final String password;
+    if (ssl) {
+      String sslConfDir = KeyStoreTestUtil.getClasspathDir(TestKMS.class);
+      KeyStoreTestUtil.setupSSLConfig(testDir.getAbsolutePath(), sslConfDir,
+          conf, false);
+      keystore = testDir.getAbsolutePath() + "/serverKS.jks";
+      password = "serverP";
+    } else {
+      keystore = null;
+      password = null;
+    }
+
+    if (kerberos) {
+      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(keystore, password, testDir, new KMSCallable() {
+      @Override
+      public Void call() throws Exception {
+        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);
+
+        if (kerberos) {
+          for (String user : new String[]{"client", "client/host"}) {
+            doAs(user, new PrivilegedExceptionAction<Void>() {
+              @Override
+              public Void run() throws Exception {
+                // getKeys() empty
+                Assert.assertTrue(kp.getKeys().isEmpty());
+                return null;
+              }
+            });
+          }
+        } else {
+          // getKeys() empty
+          Assert.assertTrue(kp.getKeys().isEmpty());
+        }
+        return null;
+      }
+    });
+  }
+
+  @Test
+  public void testStartStopHttpPseudo() throws Exception {
+    testStartStop(false, false);
+  }
+
+  @Test
+  public void testStartStopHttpsPseudo() throws Exception {
+    testStartStop(true, false);
+  }
+
+  @Test
+  public void testStartStopHttpKerberos() throws Exception {
+    testStartStop(false, true);
+  }
+
+  @Test
+  public void testStartStopHttpsKerberos() throws Exception {
+    testStartStop(true, true);
+  }
+
+  @Test
+  public void testKMSProvider() throws Exception {
+    File confDir = getTestDir();
+    Configuration conf = createBaseKMSConf(confDir);
+    writeConf(confDir, conf);
+
+    runServer(null, null, confDir, new KMSCallable() {
+      @Override
+      public Void call() throws Exception {
+        Date started = new Date();
+        Configuration conf = new Configuration();
+        URI uri = createKMSUri(getKMSUrl());
+        KeyProvider kp = new KMSClientProvider(uri, conf);
+
+        // getKeys() empty
+        Assert.assertTrue(kp.getKeys().isEmpty());
+
+        // getKeysMetadata() empty
+        Assert.assertEquals(0, kp.getKeysMetadata().length);
+
+        // createKey()
+        KeyProvider.Options options = new KeyProvider.Options(conf);
+        options.setCipher("AES/CTR/NoPadding");
+        options.setBitLength(128);
+        options.setDescription("l1");
+        KeyProvider.KeyVersion kv0 = kp.createKey("k1", options);
+        Assert.assertNotNull(kv0);
+        Assert.assertNotNull(kv0.getVersionName());
+        Assert.assertNotNull(kv0.getMaterial());
+
+        // getKeyVersion()
+        KeyProvider.KeyVersion kv1 = kp.getKeyVersion(kv0.getVersionName());
+        Assert.assertEquals(kv0.getVersionName(), kv1.getVersionName());
+        Assert.assertNotNull(kv1.getMaterial());
+
+        // getCurrent()
+        KeyProvider.KeyVersion cv1 = kp.getCurrentKey("k1");
+        Assert.assertEquals(kv0.getVersionName(), cv1.getVersionName());
+        Assert.assertNotNull(cv1.getMaterial());
+
+        // getKeyMetadata() 1 version
+        KeyProvider.Metadata m1 = kp.getMetadata("k1");
+        Assert.assertEquals("AES/CTR/NoPadding", m1.getCipher());
+        Assert.assertEquals("AES", m1.getAlgorithm());
+        Assert.assertEquals(128, m1.getBitLength());
+        Assert.assertEquals(1, m1.getVersions());
+        Assert.assertNotNull(m1.getCreated());
+        Assert.assertTrue(started.before(m1.getCreated()));
+
+        // getKeyVersions() 1 version
+        List<KeyProvider.KeyVersion> lkv1 = kp.getKeyVersions("k1");
+        Assert.assertEquals(1, lkv1.size());
+        Assert.assertEquals(kv0.getVersionName(), lkv1.get(0).getVersionName());
+        Assert.assertNotNull(kv1.getMaterial());
+
+        // rollNewVersion()
+        KeyProvider.KeyVersion kv2 = kp.rollNewVersion("k1");
+        Assert.assertNotSame(kv0.getVersionName(), kv2.getVersionName());
+        Assert.assertNotNull(kv2.getMaterial());
+
+        // getKeyVersion()
+        kv2 = kp.getKeyVersion(kv2.getVersionName());
+        boolean eq = true;
+        for (int i = 0; i < kv1.getMaterial().length; i++) {
+          eq = eq && kv1.getMaterial()[i] == kv2.getMaterial()[i];
+        }
+        Assert.assertFalse(eq);
+
+        // getCurrent()
+        KeyProvider.KeyVersion cv2 = kp.getCurrentKey("k1");
+        Assert.assertEquals(kv2.getVersionName(), cv2.getVersionName());
+        Assert.assertNotNull(cv2.getMaterial());
+        eq = true;
+        for (int i = 0; i < kv1.getMaterial().length; i++) {
+          eq = eq && cv2.getMaterial()[i] == kv2.getMaterial()[i];
+        }
+        Assert.assertTrue(eq);
+
+        // getKeyVersions() 2 versions
+        List<KeyProvider.KeyVersion> lkv2 = kp.getKeyVersions("k1");
+        Assert.assertEquals(2, lkv2.size());
+        Assert.assertEquals(kv1.getVersionName(), lkv2.get(0).getVersionName());
+        Assert.assertNotNull(lkv2.get(0).getMaterial());
+        Assert.assertEquals(kv2.getVersionName(), lkv2.get(1).getVersionName());
+        Assert.assertNotNull(lkv2.get(1).getMaterial());
+
+        // getKeyMetadata() 2 version
+        KeyProvider.Metadata m2 = kp.getMetadata("k1");
+        Assert.assertEquals("AES/CTR/NoPadding", m2.getCipher());
+        Assert.assertEquals("AES", m2.getAlgorithm());
+        Assert.assertEquals(128, m2.getBitLength());
+        Assert.assertEquals(2, m2.getVersions());
+        Assert.assertNotNull(m2.getCreated());
+        Assert.assertTrue(started.before(m2.getCreated()));
+
+        // getKeys() 1 key
+        List<String> ks1 = kp.getKeys();
+        Assert.assertEquals(1, ks1.size());
+        Assert.assertEquals("k1", ks1.get(0));
+
+        // getKeysMetadata() 1 key 2 versions
+        KeyProvider.Metadata[] kms1 = kp.getKeysMetadata("k1");
+        Assert.assertEquals(1, kms1.length);
+        Assert.assertEquals("AES/CTR/NoPadding", kms1[0].getCipher());
+        Assert.assertEquals("AES", kms1[0].getAlgorithm());
+        Assert.assertEquals(128, kms1[0].getBitLength());
+        Assert.assertEquals(2, kms1[0].getVersions());
+        Assert.assertNotNull(kms1[0].getCreated());
+        Assert.assertTrue(started.before(kms1[0].getCreated()));
+
+        // deleteKey()
+        kp.deleteKey("k1");
+
+        // getKey()
+        Assert.assertNull(kp.getKeyVersion("k1"));
+
+        // getKeyVersions()
+        Assert.assertNull(kp.getKeyVersions("k1"));
+
+        // getMetadata()
+        Assert.assertNull(kp.getMetadata("k1"));
+
+        // getKeys() empty
+        Assert.assertTrue(kp.getKeys().isEmpty());
+
+        // getKeysMetadata() empty
+        Assert.assertEquals(0, kp.getKeysMetadata().length);
+
+        return null;
+      }
+    });
+  }
+
+  @Test
+  public void testACLs() throws Exception {
+    final File testDir = getTestDir();
+    Configuration 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");
+
+    for (KMSACLs.Type type : KMSACLs.Type.values()) {
+      conf.set(type.getConfigKey(), type.toString());
+    }
+    conf.set(KMSACLs.Type.CREATE.getConfigKey(),
+        KMSACLs.Type.CREATE.toString() + ",SET_KEY_MATERIAL");
+
+    conf.set(KMSACLs.Type.ROLLOVER.getConfigKey(),
+        KMSACLs.Type.ROLLOVER.toString() + ",SET_KEY_MATERIAL");
+
+    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);
+        URI uri = createKMSUri(getKMSUrl());
+        final KeyProvider kp = new KMSClientProvider(uri, conf);
+
+        //nothing allowed
+        doAs("client", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            try {
+              kp.createKey("k", new KeyProvider.Options(conf));
+              Assert.fail();
+            } catch (AuthorizationException ex) {
+              //NOP
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            try {
+              kp.createKey("k", new byte[8], new KeyProvider.Options(conf));
+              Assert.fail();
+            } catch (AuthorizationException ex) {
+              //NOP
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            try {
+              kp.rollNewVersion("k");
+              Assert.fail();
+            } catch (AuthorizationException ex) {
+              //NOP
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            try {
+              kp.rollNewVersion("k", new byte[8]);
+              Assert.fail();
+            } catch (AuthorizationException ex) {
+              //NOP
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            try {
+              kp.getKeys();
+              Assert.fail();
+            } catch (AuthorizationException ex) {
+              //NOP
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            try {
+              kp.getKeysMetadata("k");
+              Assert.fail();
+            } catch (AuthorizationException ex) {
+              //NOP
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            try {
+              kp.getKeyVersion(KMSClientProvider.buildVersionName("k", 0));
+              Assert.fail();
+            } catch (AuthorizationException ex) {
+              //NOP
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            try {
+              kp.getCurrentKey("k");
+              Assert.fail();
+            } catch (AuthorizationException ex) {
+              //NOP
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            try {
+              kp.getMetadata("k");
+              Assert.fail();
+            } catch (AuthorizationException ex) {
+              //NOP
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            try {
+              kp.getKeyVersions("k");
+              Assert.fail();
+            } catch (AuthorizationException ex) {
+              //NOP
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+
+            return null;
+          }
+        });
+
+        doAs("CREATE", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            try {
+              KeyProvider.KeyVersion kv = kp.createKey("k0",
+                  new KeyProvider.Options(conf));
+              Assert.assertNull(kv.getMaterial());
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            return null;
+          }
+        });
+
+        doAs("DELETE", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            try {
+              kp.deleteKey("k0");
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            return null;
+          }
+        });
+
+        doAs("SET_KEY_MATERIAL", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            try {
+              KeyProvider.KeyVersion kv = kp.createKey("k1", new byte[8],
+                  new KeyProvider.Options(conf));
+              Assert.assertNull(kv.getMaterial());
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            return null;
+          }
+        });
+
+        doAs("ROLLOVER", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            try {
+              KeyProvider.KeyVersion kv = kp.rollNewVersion("k1");
+              Assert.assertNull(kv.getMaterial());
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            return null;
+          }
+        });
+
+        doAs("SET_KEY_MATERIAL", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            try {
+              KeyProvider.KeyVersion kv = kp.rollNewVersion("k1", new byte[8]);
+              Assert.assertNull(kv.getMaterial());
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            return null;
+          }
+        });
+
+        doAs("GET", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            try {
+              kp.getKeyVersion("k1@0");
+              kp.getCurrentKey("k1");
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            return null;
+          }
+        });
+
+        doAs("GET_KEYS", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            try {
+              kp.getKeys();
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            return null;
+          }
+        });
+
+        doAs("GET_METADATA", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            try {
+              kp.getMetadata("k1");
+              kp.getKeysMetadata("k1");
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            return null;
+          }
+        });
+
+        // test ACL reloading
+        Thread.sleep(10); // to ensure the ACLs file modifiedTime is newer
+        conf.set(KMSACLs.Type.CREATE.getConfigKey(), "foo");
+        writeConf(testDir, conf);
+
+        KMSWebApp.getACLs().run(); // forcing a reload by hand.
+
+        // should not be able to create a key now
+        doAs("CREATE", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            try {
+              KeyProvider.KeyVersion kv = kp.createKey("k2",
+                  new KeyProvider.Options(conf));
+              Assert.fail();
+            } catch (AuthorizationException ex) {
+              //NOP
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+
+            return null;
+          }
+        });
+
+        return null;
+      }
+    });
+  }
+
+  @Test
+  public void testServicePrincipalACLs() throws Exception {
+    File testDir = getTestDir();
+    Configuration 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");
+    for (KMSACLs.Type type : KMSACLs.Type.values()) {
+      conf.set(type.getConfigKey(), " ");
+    }
+    conf.set(KMSACLs.Type.CREATE.getConfigKey(), "client");
+
+    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);
+        URI uri = createKMSUri(getKMSUrl());
+        final KeyProvider kp = new KMSClientProvider(uri, conf);
+
+        doAs("client", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            try {
+              KeyProvider.KeyVersion kv = kp.createKey("ck0",
+                  new KeyProvider.Options(conf));
+              Assert.assertNull(kv.getMaterial());
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            return null;
+          }
+        });
+
+        doAs("client/host", new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws Exception {
+            try {
+              KeyProvider.KeyVersion kv = kp.createKey("ck1",
+                  new KeyProvider.Options(conf));
+              Assert.assertNull(kv.getMaterial());
+            } catch (Exception ex) {
+              Assert.fail(ex.toString());
+            }
+            return null;
+          }
+        });
+        return null;
+      }
+    });
+  }
+
+}

+ 47 - 0
hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSACLs.java

@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms.server;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestKMSACLs {
+
+  @Test
+  public void testDefaults() {
+    KMSACLs acls = new KMSACLs(new Configuration(false));
+    for (KMSACLs.Type type : KMSACLs.Type.values()) {
+      Assert.assertTrue(acls.hasAccess(type, "foo"));
+    }
+  }
+
+  @Test
+  public void testCustom() {
+    Configuration conf = new Configuration(false);
+    for (KMSACLs.Type type : KMSACLs.Type.values()) {
+      conf.set(type.getConfigKey(), type.toString() + " ");
+    }
+    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"));
+    }
+  }
+
+}

+ 120 - 0
hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSCacheKeyProvider.java

@@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto.key.kms.server;
+
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.kms.KMSClientProvider;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.Date;
+
+public class TestKMSCacheKeyProvider {
+
+  @Test
+  public void testCurrentKey() throws Exception {
+    KeyProvider.KeyVersion mockKey = Mockito.mock(KeyProvider.KeyVersion.class);
+    KeyProvider mockProv = Mockito.mock(KeyProvider.class);
+    Mockito.when(mockProv.getCurrentKey(Mockito.eq("k1"))).thenReturn(mockKey);
+    Mockito.when(mockProv.getCurrentKey(Mockito.eq("k2"))).thenReturn(null);
+    KeyProvider cache = new KMSCacheKeyProvider(mockProv, 100);
+
+    // asserting caching
+    Assert.assertEquals(mockKey, cache.getCurrentKey("k1"));
+    Mockito.verify(mockProv, Mockito.times(1)).getCurrentKey(Mockito.eq("k1"));
+    Assert.assertEquals(mockKey, cache.getCurrentKey("k1"));
+    Mockito.verify(mockProv, Mockito.times(1)).getCurrentKey(Mockito.eq("k1"));
+    Thread.sleep(1200);
+    Assert.assertEquals(mockKey, cache.getCurrentKey("k1"));
+    Mockito.verify(mockProv, Mockito.times(2)).getCurrentKey(Mockito.eq("k1"));
+
+    // asserting no caching when key is not known
+    cache = new KMSCacheKeyProvider(mockProv, 100);
+    Assert.assertEquals(null, cache.getCurrentKey("k2"));
+    Mockito.verify(mockProv, Mockito.times(1)).getCurrentKey(Mockito.eq("k2"));
+    Assert.assertEquals(null, cache.getCurrentKey("k2"));
+    Mockito.verify(mockProv, Mockito.times(2)).getCurrentKey(Mockito.eq("k2"));
+  }
+
+  @Test
+  public void testKeyVersion() throws Exception {
+    KeyProvider.KeyVersion mockKey = Mockito.mock(KeyProvider.KeyVersion.class);
+    KeyProvider mockProv = Mockito.mock(KeyProvider.class);
+    Mockito.when(mockProv.getKeyVersion(Mockito.eq("k1@0"))).thenReturn(mockKey);
+    Mockito.when(mockProv.getKeyVersion(Mockito.eq("k2@0"))).thenReturn(null);
+    KeyProvider cache = new KMSCacheKeyProvider(mockProv, 100);
+
+    // asserting caching
+    Assert.assertEquals(mockKey, cache.getKeyVersion("k1@0"));
+    Mockito.verify(mockProv, Mockito.times(1)).getKeyVersion(Mockito.eq("k1@0"));
+    Assert.assertEquals(mockKey, cache.getKeyVersion("k1@0"));
+    Mockito.verify(mockProv, Mockito.times(1)).getKeyVersion(Mockito.eq("k1@0"));
+    Thread.sleep(200);
+    Assert.assertEquals(mockKey, cache.getKeyVersion("k1@0"));
+    Mockito.verify(mockProv, Mockito.times(2)).getKeyVersion(Mockito.eq("k1@0"));
+
+    // asserting no caching when key is not known
+    cache = new KMSCacheKeyProvider(mockProv, 100);
+    Assert.assertEquals(null, cache.getKeyVersion("k2@0"));
+    Mockito.verify(mockProv, Mockito.times(1)).getKeyVersion(Mockito.eq("k2@0"));
+    Assert.assertEquals(null, cache.getKeyVersion("k2@0"));
+    Mockito.verify(mockProv, Mockito.times(2)).getKeyVersion(Mockito.eq("k2@0"));
+  }
+
+  @Test
+  public void testRollNewVersion() throws Exception {
+    KeyProvider.KeyVersion mockKey = Mockito.mock(KeyProvider.KeyVersion.class);
+    KeyProvider mockProv = Mockito.mock(KeyProvider.class);
+    Mockito.when(mockProv.getCurrentKey(Mockito.eq("k1"))).thenReturn(mockKey);
+    KeyProvider cache = new KMSCacheKeyProvider(mockProv, 100);
+    Assert.assertEquals(mockKey, cache.getCurrentKey("k1"));
+    Mockito.verify(mockProv, Mockito.times(1)).getCurrentKey(Mockito.eq("k1"));
+    cache.rollNewVersion("k1");
+
+    // asserting the cache is purged
+    Assert.assertEquals(mockKey, cache.getCurrentKey("k1"));
+    Mockito.verify(mockProv, Mockito.times(2)).getCurrentKey(Mockito.eq("k1"));
+    cache.rollNewVersion("k1", new byte[0]);
+    Assert.assertEquals(mockKey, cache.getCurrentKey("k1"));
+    Mockito.verify(mockProv, Mockito.times(3)).getCurrentKey(Mockito.eq("k1"));
+  }
+
+  @Test
+  public void testDeleteKey() throws Exception {
+    KeyProvider.KeyVersion mockKey = Mockito.mock(KeyProvider.KeyVersion.class);
+    KeyProvider mockProv = Mockito.mock(KeyProvider.class);
+    Mockito.when(mockProv.getCurrentKey(Mockito.eq("k1"))).thenReturn(mockKey);
+    Mockito.when(mockProv.getKeyVersion(Mockito.eq("k1@0"))).thenReturn(mockKey);
+    Mockito.when(mockProv.getMetadata(Mockito.eq("k1"))).thenReturn(
+        new KMSClientProvider.KMSMetadata("c", 0, "l", new Date(), 1));
+    KeyProvider cache = new KMSCacheKeyProvider(mockProv, 100);
+    Assert.assertEquals(mockKey, cache.getCurrentKey("k1"));
+    Mockito.verify(mockProv, Mockito.times(1)).getCurrentKey(Mockito.eq("k1"));
+    Assert.assertEquals(mockKey, cache.getKeyVersion("k1@0"));
+    Mockito.verify(mockProv, Mockito.times(1)).getKeyVersion(Mockito.eq("k1@0"));
+    cache.deleteKey("k1");
+
+    // asserting the cache is purged
+    Assert.assertEquals(mockKey, cache.getCurrentKey("k1"));
+    Mockito.verify(mockProv, Mockito.times(2)).getCurrentKey(Mockito.eq("k1"));
+    Assert.assertEquals(mockKey, cache.getKeyVersion("k1@0"));
+    Mockito.verify(mockProv, Mockito.times(2)).getKeyVersion(Mockito.eq("k1@0"));
+  }
+
+}

+ 31 - 0
hadoop-common-project/hadoop-kms/src/test/resources/log4j.properties

@@ -0,0 +1,31 @@
+#
+# 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.
+#
+
+# STDOUT Appender
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.Target=System.out
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{1} - %m%n
+
+log4j.rootLogger=WARN, stdout
+log4j.logger.org.apache.hadoop.conf=ERROR
+log4j.logger.org.apache.hadoop.crytpo.key.kms.server=ALL
+log4j.logger.com.sun.jersey.server.wadl.generators.WadlGeneratorJAXBGrammarGenerator=OFF
+log4j.logger.org.apache.hadoop.security=OFF
+log4j.logger.org.apache.directory.server.core=OFF
+log4j.logger.org.apache.hadoop.util.NativeCodeLoader=OFF

+ 1 - 0
hadoop-common-project/pom.xml

@@ -37,6 +37,7 @@
     <module>hadoop-annotations</module>
     <module>hadoop-nfs</module>
     <module>hadoop-minikdc</module>
+    <module>hadoop-kms</module>
   </modules>
 
   <build>

+ 1 - 0
hadoop-dist/pom.xml

@@ -118,6 +118,7 @@
                       run cp -r $ROOT/hadoop-common-project/hadoop-nfs/target/hadoop-nfs-${project.version}/* .
                       run cp -r $ROOT/hadoop-hdfs-project/hadoop-hdfs/target/hadoop-hdfs-${project.version}/* .
                       run cp -r $ROOT/hadoop-hdfs-project/hadoop-hdfs-httpfs/target/hadoop-hdfs-httpfs-${project.version}/* .
+                      run cp -r $ROOT/hadoop-common-project/hadoop-kms/target/hadoop-kms-${project.version}/* .
                       run cp -r $ROOT/hadoop-hdfs-project/hadoop-hdfs-nfs/target/hadoop-hdfs-nfs-${project.version}/* .
                       run cp -r $ROOT/hadoop-yarn-project/target/hadoop-yarn-project-${project.version}/* .
                       run cp -r $ROOT/hadoop-mapreduce-project/target/hadoop-mapreduce-${project.version}/* .

+ 7 - 1
hadoop-project/pom.xml

@@ -588,6 +588,11 @@
         <artifactId>slf4j-log4j12</artifactId>
         <version>1.7.5</version>
       </dependency>
+      <dependency>
+        <groupId>org.slf4j</groupId>
+        <artifactId>jul-to-slf4j</artifactId>
+        <version>1.7.5</version>
+      </dependency>
       <dependency>
         <groupId>org.eclipse.jdt</groupId>
         <artifactId>core</artifactId>
@@ -715,7 +720,7 @@
       <dependency>
         <groupId>com.codahale.metrics</groupId>
         <artifactId>metrics-core</artifactId>
-        <version>3.0.0</version>
+        <version>3.0.1</version>
       </dependency>
       <dependency>
         <groupId>org.apache.hadoop</groupId>
@@ -760,6 +765,7 @@
         <artifactId>leveldbjni-all</artifactId>
         <version>1.8</version>
       </dependency>
+
     </dependencies>
   </dependencyManagement>
 

+ 1 - 0
hadoop-project/src/site/site.xml

@@ -62,6 +62,7 @@
       <item name="Secure Mode" href="hadoop-project-dist/hadoop-common/SecureMode.html"/>
       <item name="Service Level Authorization" href="hadoop-project-dist/hadoop-common/ServiceLevelAuth.html"/>
       <item name="HTTP Authentication" href="hadoop-project-dist/hadoop-common/HttpAuthentication.html"/>
+      <item name="Hadoop KMS" href="hadoop-kms/index.html"/>
     </menu>
     
     <menu name="HDFS" inherit="top">