You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ji...@apache.org on 2014/08/12 00:27:52 UTC

svn commit: r1617377 - in /hadoop/common/branches/HDFS-6584/hadoop-common-project: hadoop-common/ hadoop-common/src/main/java/ hadoop-common/src/main/java/org/apache/hadoop/conf/ hadoop-common/src/main/java/org/apache/hadoop/crypto/key/ hadoop-common/s...

Author: jing9
Date: Mon Aug 11 22:27:50 2014
New Revision: 1617377

URL: http://svn.apache.org/r1617377
Log:
Merging r1616894 through r1617376 from trunk.

Modified:
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/CHANGES.txt   (contents, props changed)
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/   (props changed)
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSRESTConstants.java
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJMXServlet.java
    hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/CHANGES.txt?rev=1617377&r1=1617376&r2=1617377&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/CHANGES.txt (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/CHANGES.txt Mon Aug 11 22:27:50 2014
@@ -199,6 +199,9 @@ Trunk (Unreleased)
 
     HADOOP-10936. Change default KeyProvider bitlength to 128. (wang)
 
+    HADOOP-10224. JavaKeyStoreProvider has to protect against corrupting 
+    underlying store. (asuresh via tucu)
+
   BUG FIXES
 
     HADOOP-9451. Fault single-layer config if node group topology is enabled.
@@ -421,6 +424,9 @@ Trunk (Unreleased)
     HADOOP-10939. Fix TestKeyProviderFactory testcases to use default 128 bit
     length keys. (Arun Suresh via wang)
 
+    HADOOP-10862. Miscellaneous trivial corrections to KMS classes. 
+    (asuresh via tucu)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)
@@ -547,6 +553,9 @@ Release 2.6.0 - UNRELEASED
     HADOOP-10929. Typo in Configuration.getPasswordFromCredentialProviders
     (lmccay via brandonli)
 
+    HADOOP-10402. Configuration.getValByRegex does not substitute for
+    variables. (Robert Kanter via kasha)
+
 Release 2.5.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

Propchange: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/CHANGES.txt
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/CHANGES.txt:r1616894-1617376

Propchange: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-common-project/hadoop-common/src/main/java:r1616894-1617376

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java?rev=1617377&r1=1617376&r2=1617377&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java Mon Aug 11 22:27:50 2014
@@ -2755,7 +2755,8 @@ public class Configuration implements It
           item.getValue() instanceof String) {
         m = p.matcher((String)item.getKey());
         if(m.find()) { // match
-          result.put((String) item.getKey(), (String) item.getValue());
+          result.put((String) item.getKey(),
+              substituteVars(getProps().getProperty((String) item.getKey())));
         }
       }
     }

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java?rev=1617377&r1=1617376&r2=1617377&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java Mon Aug 11 22:27:50 2014
@@ -27,8 +27,11 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.ProviderUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import javax.crypto.spec.SecretKeySpec;
+
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.ObjectInputStream;
@@ -80,6 +83,9 @@ import java.util.concurrent.locks.Reentr
 @InterfaceAudience.Private
 public class JavaKeyStoreProvider extends KeyProvider {
   private static final String KEY_METADATA = "KeyMetadata";
+  private static Logger LOG =
+      LoggerFactory.getLogger(JavaKeyStoreProvider.class);
+
   public static final String SCHEME_NAME = "jceks";
 
   public static final String KEYSTORE_PASSWORD_FILE_KEY =
@@ -115,6 +121,10 @@ public class JavaKeyStoreProvider extend
       if (pwFile != null) {
         ClassLoader cl = Thread.currentThread().getContextClassLoader();
         URL pwdFile = cl.getResource(pwFile);
+        if (pwdFile == null) {
+          // Provided Password file does not exist
+          throw new IOException("Password file does not exists");
+        }
         if (pwdFile != null) {
           InputStream is = pwdFile.openStream();
           try {
@@ -129,19 +139,25 @@ public class JavaKeyStoreProvider extend
       password = KEYSTORE_PASSWORD_DEFAULT;
     }
     try {
+      Path oldPath = constructOldPath(path);
+      Path newPath = constructNewPath(path);
       keyStore = KeyStore.getInstance(SCHEME_NAME);
+      FsPermission perm = null;
       if (fs.exists(path)) {
-        // save off permissions in case we need to
-        // rewrite the keystore in flush()
-        FileStatus s = fs.getFileStatus(path);
-        permissions = s.getPermission();
-
-        keyStore.load(fs.open(path), password);
+        // flush did not proceed to completion
+        // _NEW should not exist
+        if (fs.exists(newPath)) {
+          throw new IOException(
+              String.format("Keystore not loaded due to some inconsistency "
+              + "('%s' and '%s' should not exist together)!!", path, newPath));
+        }
+        perm = tryLoadFromPath(path, oldPath);
       } else {
-        permissions = new FsPermission("700");
-        // required to create an empty keystore. *sigh*
-        keyStore.load(null, password);
+        perm = tryLoadIncompleteFlush(oldPath, newPath);
       }
+      // Need to save off permissions in case we need to
+      // rewrite the keystore in flush()
+      permissions = perm;
     } catch (KeyStoreException e) {
       throw new IOException("Can't create keystore", e);
     } catch (NoSuchAlgorithmException e) {
@@ -154,6 +170,136 @@ public class JavaKeyStoreProvider extend
     writeLock = lock.writeLock();
   }
 
+  /**
+   * Try loading from the user specified path, else load from the backup
+   * path in case Exception is not due to bad/wrong password
+   * @param path Actual path to load from
+   * @param backupPath Backup path (_OLD)
+   * @return The permissions of the loaded file
+   * @throws NoSuchAlgorithmException
+   * @throws CertificateException
+   * @throws IOException
+   */
+  private FsPermission tryLoadFromPath(Path path, Path backupPath)
+      throws NoSuchAlgorithmException, CertificateException,
+      IOException {
+    FsPermission perm = null;
+    try {
+      perm = loadFromPath(path, password);
+      // Remove _OLD if exists
+      if (fs.exists(backupPath)) {
+        fs.delete(backupPath, true);
+      }
+      LOG.debug("KeyStore loaded successfully !!");
+    } catch (IOException ioe) {
+      // If file is corrupted for some reason other than
+      // wrong password try the _OLD file if exits
+      if (!isBadorWrongPassword(ioe)) {
+        perm = loadFromPath(backupPath, password);
+        // Rename CURRENT to CORRUPTED
+        renameOrFail(path, new Path(path.toString() + "_CORRUPTED_"
+            + System.currentTimeMillis()));
+        renameOrFail(backupPath, path);
+        LOG.debug(String.format(
+            "KeyStore loaded successfully from '%s' since '%s'"
+                + "was corrupted !!", backupPath, path));
+      } else {
+        throw ioe;
+      }
+    }
+    return perm;
+  }
+
+  /**
+   * The KeyStore might have gone down during a flush, In which case either the
+   * _NEW or _OLD files might exists. This method tries to load the KeyStore
+   * from one of these intermediate files.
+   * @param oldPath the _OLD file created during flush
+   * @param newPath the _NEW file created during flush
+   * @return The permissions of the loaded file
+   * @throws IOException
+   * @throws NoSuchAlgorithmException
+   * @throws CertificateException
+   */
+  private FsPermission tryLoadIncompleteFlush(Path oldPath, Path newPath)
+      throws IOException, NoSuchAlgorithmException, CertificateException {
+    FsPermission perm = null;
+    // Check if _NEW exists (in case flush had finished writing but not
+    // completed the re-naming)
+    if (fs.exists(newPath)) {
+      perm = loadAndReturnPerm(newPath, oldPath);
+    }
+    // try loading from _OLD (An earlier Flushing MIGHT not have completed
+    // writing completely)
+    if ((perm == null) && fs.exists(oldPath)) {
+      perm = loadAndReturnPerm(oldPath, newPath);
+    }
+    // If not loaded yet,
+    // required to create an empty keystore. *sigh*
+    if (perm == null) {
+      keyStore.load(null, password);
+      LOG.debug("KeyStore initialized anew successfully !!");
+      perm = new FsPermission("700");
+    }
+    return perm;
+  }
+
+  private FsPermission loadAndReturnPerm(Path pathToLoad, Path pathToDelete)
+      throws NoSuchAlgorithmException, CertificateException,
+      IOException {
+    FsPermission perm = null;
+    try {
+      perm = loadFromPath(pathToLoad, password);
+      renameOrFail(pathToLoad, path);
+      LOG.debug(String.format("KeyStore loaded successfully from '%s'!!",
+          pathToLoad));
+      if (fs.exists(pathToDelete)) {
+        fs.delete(pathToDelete, true);
+      }
+    } catch (IOException e) {
+      // Check for password issue : don't want to trash file due
+      // to wrong password
+      if (isBadorWrongPassword(e)) {
+        throw e;
+      }
+    }
+    return perm;
+  }
+
+  private boolean isBadorWrongPassword(IOException ioe) {
+    // As per documentation this is supposed to be the way to figure
+    // if password was correct
+    if (ioe.getCause() instanceof UnrecoverableKeyException) {
+      return true;
+    }
+    // Unfortunately that doesn't seem to work..
+    // Workaround :
+    if ((ioe.getCause() == null)
+        && (ioe.getMessage() != null)
+        && ((ioe.getMessage().contains("Keystore was tampered")) || (ioe
+            .getMessage().contains("password was incorrect")))) {
+      return true;
+    }
+    return false;
+  }
+
+  private FsPermission loadFromPath(Path p, char[] password)
+      throws IOException, NoSuchAlgorithmException, CertificateException {
+    FileStatus s = fs.getFileStatus(p);
+    keyStore.load(fs.open(p), password);
+    return s.getPermission();
+  }
+
+  private Path constructNewPath(Path path) {
+    Path newPath = new Path(path.toString() + "_NEW");
+    return newPath;
+  }
+
+  private Path constructOldPath(Path path) {
+    Path oldPath = new Path(path.toString() + "_OLD");
+    return oldPath;
+  }
+
   @Override
   public KeyVersion getKeyVersion(String versionName) throws IOException {
     readLock.lock();
@@ -352,11 +498,22 @@ public class JavaKeyStoreProvider extend
 
   @Override
   public void flush() throws IOException {
+    Path newPath = constructNewPath(path);
+    Path oldPath = constructOldPath(path);
     writeLock.lock();
     try {
       if (!changed) {
         return;
       }
+      // Might exist if a backup has been restored etc.
+      if (fs.exists(newPath)) {
+        renameOrFail(newPath, new Path(newPath.toString()
+            + "_ORPHANED_" + System.currentTimeMillis()));
+      }
+      if (fs.exists(oldPath)) {
+        renameOrFail(oldPath, new Path(oldPath.toString()
+            + "_ORPHANED_" + System.currentTimeMillis()));
+      }
       // put all of the updates into the keystore
       for(Map.Entry<String, Metadata> entry: cache.entrySet()) {
         try {
@@ -366,25 +523,77 @@ public class JavaKeyStoreProvider extend
           throw new IOException("Can't set metadata key " + entry.getKey(),e );
         }
       }
+
+      // Save old File first
+      boolean fileExisted = backupToOld(oldPath);
       // write out the keystore
-      FSDataOutputStream out = FileSystem.create(fs, path, permissions);
+      // Write to _NEW path first :
       try {
-        keyStore.store(out, password);
-      } catch (KeyStoreException e) {
-        throw new IOException("Can't store keystore " + this, e);
-      } catch (NoSuchAlgorithmException e) {
-        throw new IOException("No such algorithm storing keystore " + this, e);
-      } catch (CertificateException e) {
-        throw new IOException("Certificate exception storing keystore " + this,
-            e);
+        writeToNew(newPath);
+      } catch (IOException ioe) {
+        // rename _OLD back to curent and throw Exception
+        revertFromOld(oldPath, fileExisted);
+        throw ioe;
       }
-      out.close();
+      // Rename _NEW to CURRENT and delete _OLD
+      cleanupNewAndOld(newPath, oldPath);
       changed = false;
     } finally {
       writeLock.unlock();
     }
   }
 
+  private void cleanupNewAndOld(Path newPath, Path oldPath) throws IOException {
+    // Rename _NEW to CURRENT
+    renameOrFail(newPath, path);
+    // Delete _OLD
+    if (fs.exists(oldPath)) {
+      fs.delete(oldPath, true);
+    }
+  }
+
+  private void writeToNew(Path newPath) throws IOException {
+    FSDataOutputStream out =
+        FileSystem.create(fs, newPath, permissions);
+    try {
+      keyStore.store(out, password);
+    } catch (KeyStoreException e) {
+      throw new IOException("Can't store keystore " + this, e);
+    } catch (NoSuchAlgorithmException e) {
+      throw new IOException(
+          "No such algorithm storing keystore " + this, e);
+    } catch (CertificateException e) {
+      throw new IOException(
+          "Certificate exception storing keystore " + this, e);
+    }
+    out.close();
+  }
+
+  private void revertFromOld(Path oldPath, boolean fileExisted)
+      throws IOException {
+    if (fileExisted) {
+      renameOrFail(oldPath, path);
+    }
+  }
+
+  private boolean backupToOld(Path oldPath)
+      throws IOException {
+    boolean fileExisted = false;
+    if (fs.exists(path)) {
+      renameOrFail(path, oldPath);
+      fileExisted = true;
+    }
+    return fileExisted;
+  }
+
+  private void renameOrFail(Path src, Path dest)
+      throws IOException {
+    if (!fs.rename(src, dest)) {
+      throw new IOException("Rename unsuccessful : "
+          + String.format("'%s' to '%s'", src, dest));
+    }
+  }
+
   @Override
   public String toString() {
     return uri.toString();

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java?rev=1617377&r1=1617376&r2=1617377&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSClientProvider.java Mon Aug 11 22:27:50 2014
@@ -512,7 +512,7 @@ public class KMSClientProvider extends K
     List<String> batch = new ArrayList<String>();
     int batchLen = 0;
     for (String name : keyNames) {
-      int additionalLen = KMSRESTConstants.KEY_OP.length() + 1 + name.length();
+      int additionalLen = KMSRESTConstants.KEY.length() + 1 + name.length();
       batchLen += additionalLen;
       // topping at 1500 to account for initial URL and encoded names
       if (batchLen > 1500) {
@@ -536,7 +536,7 @@ public class KMSClientProvider extends K
     for (String[] keySet : keySets) {
       if (keyNames.length > 0) {
         Map<String, Object> queryStr = new HashMap<String, Object>();
-        queryStr.put(KMSRESTConstants.KEY_OP, keySet);
+        queryStr.put(KMSRESTConstants.KEY, keySet);
         URL url = createURL(KMSRESTConstants.KEYS_METADATA_RESOURCE, null,
             null, queryStr);
         HttpURLConnection conn = createConnection(url, HTTP_GET);

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSRESTConstants.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSRESTConstants.java?rev=1617377&r1=1617376&r2=1617377&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSRESTConstants.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/KMSRESTConstants.java Mon Aug 11 22:27:50 2014
@@ -37,7 +37,7 @@ public class KMSRESTConstants {
   public static final String EEK_SUB_RESOURCE = "_eek";
   public static final String CURRENT_VERSION_SUB_RESOURCE = "_currentversion";
 
-  public static final String KEY_OP = "key";
+  public static final String KEY = "key";
   public static final String EEK_OP = "eek_op";
   public static final String EEK_GENERATE = "generate";
   public static final String EEK_DECRYPT = "decrypt";

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java?rev=1617377&r1=1617376&r2=1617377&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfiguration.java Mon Aug 11 22:27:50 2014
@@ -178,6 +178,14 @@ public class TestConfiguration extends T
     // check that expansion also occurs for getInt()
     assertTrue(conf.getInt("intvar", -1) == 42);
     assertTrue(conf.getInt("my.int", -1) == 42);
+
+    Map<String, String> results = conf.getValByRegex("^my.*file$");
+    assertTrue(results.keySet().contains("my.relfile"));
+    assertTrue(results.keySet().contains("my.fullfile"));
+    assertTrue(results.keySet().contains("my.file"));
+    assertEquals(-1, results.get("my.relfile").indexOf("${"));
+    assertEquals(-1, results.get("my.fullfile").indexOf("${"));
+    assertEquals(-1, results.get("my.file").indexOf("${"));
   }
 
   public void testFinalParam() throws IOException {

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java?rev=1617377&r1=1617376&r2=1617377&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java Mon Aug 11 22:27:50 2014
@@ -220,11 +220,76 @@ public class TestKeyProviderFactory {
     assertTrue(s.getPermission().toString().equals("rwx------"));
     assertTrue(file + " should exist", file.isFile());
 
+    // Corrupt file and Check if JKS can reload from _OLD file
+    File oldFile = new File(file.getPath() + "_OLD");
+    file.renameTo(oldFile);
+    file.delete();
+    file.createNewFile();
+    assertTrue(oldFile.exists());
+    KeyProvider provider = KeyProviderFactory.getProviders(conf).get(0);
+    assertTrue(file.exists());
+    assertTrue(oldFile + "should be deleted", !oldFile.exists());
+    verifyAfterReload(file, provider);
+    assertTrue(!oldFile.exists());
+
+    // _NEW and current file should not exist together
+    File newFile = new File(file.getPath() + "_NEW");
+    newFile.createNewFile();
+    try {
+      provider = KeyProviderFactory.getProviders(conf).get(0);
+      Assert.fail("_NEW and current file should not exist together !!");
+    } catch (Exception e) {
+      // Ignore
+    } finally {
+      if (newFile.exists()) {
+        newFile.delete();
+      }
+    }
+
+    // Load from _NEW file
+    file.renameTo(newFile);
+    file.delete();
+    try {
+      provider = KeyProviderFactory.getProviders(conf).get(0);
+      Assert.assertFalse(newFile.exists());
+      Assert.assertFalse(oldFile.exists());
+    } catch (Exception e) {
+      Assert.fail("JKS should load from _NEW file !!");
+      // Ignore
+    }
+    verifyAfterReload(file, provider);
+
+    // _NEW exists but corrupt.. must load from _OLD
+    newFile.createNewFile();
+    file.renameTo(oldFile);
+    file.delete();
+    try {
+      provider = KeyProviderFactory.getProviders(conf).get(0);
+      Assert.assertFalse(newFile.exists());
+      Assert.assertFalse(oldFile.exists());
+    } catch (Exception e) {
+      Assert.fail("JKS should load from _OLD file !!");
+      // Ignore
+    } finally {
+      if (newFile.exists()) {
+        newFile.delete();
+      }
+    }
+    verifyAfterReload(file, provider);
+
     // check permission retention after explicit change
     fs.setPermission(path, new FsPermission("777"));
     checkPermissionRetention(conf, ourUrl, path);
   }
 
+  private void verifyAfterReload(File file, KeyProvider provider)
+      throws IOException {
+    List<String> existingKeys = provider.getKeys();
+    assertTrue(existingKeys.contains("key4"));
+    assertTrue(existingKeys.contains("key3"));
+    assertTrue(file.exists());
+  }
+
   public void checkPermissionRetention(Configuration conf, String ourUrl, Path path) throws Exception {
     KeyProvider provider = KeyProviderFactory.getProviders(conf).get(0);
     // let's add a new key and flush and check that permissions are still set to 777

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java?rev=1617377&r1=1617376&r2=1617377&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMS.java Mon Aug 11 22:27:50 2014
@@ -47,7 +47,6 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.security.Principal;
-import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
@@ -59,19 +58,14 @@ import java.util.Map;
 @Path(KMSRESTConstants.SERVICE_VERSION)
 @InterfaceAudience.Private
 public class KMS {
-  public static final String CREATE_KEY = "CREATE_KEY";
-  public static final String DELETE_KEY = "DELETE_KEY";
-  public static final String ROLL_NEW_VERSION = "ROLL_NEW_VERSION";
-  public static final String GET_KEYS = "GET_KEYS";
-  public static final String GET_KEYS_METADATA = "GET_KEYS_METADATA";
-  public static final String GET_KEY_VERSIONS = "GET_KEY_VERSIONS";
-  public static final String GET_METADATA = "GET_METADATA";
-
-  public static final String GET_KEY_VERSION = "GET_KEY_VERSION";
-  public static final String GET_CURRENT_KEY = "GET_CURRENT_KEY";
-  public static final String GENERATE_EEK = "GENERATE_EEK";
-  public static final String DECRYPT_EEK = "DECRYPT_EEK";
-  
+
+  public static enum KMSOp {
+    CREATE_KEY, DELETE_KEY, ROLL_NEW_VERSION,
+    GET_KEYS, GET_KEYS_METADATA,
+    GET_KEY_VERSIONS, GET_METADATA, GET_KEY_VERSION, GET_CURRENT_KEY,
+    GENERATE_EEK, DECRYPT_EEK
+  }
+
   private KeyProviderCryptoExtension provider;
   private KMSAudit kmsAudit;
 
@@ -91,22 +85,22 @@ public class KMS {
 
 
   private static final String UNAUTHORIZED_MSG_WITH_KEY = 
-      "User:{0} not allowed to do ''{1}'' on ''{2}''";
+      "User:%s not allowed to do '%s' on '%s'";
   
   private static final String UNAUTHORIZED_MSG_WITHOUT_KEY = 
-      "User:{0} not allowed to do ''{1}''";
+      "User:%s not allowed to do '%s'";
 
   private void assertAccess(KMSACLs.Type aclType, Principal principal,
-      String operation) throws AccessControlException {
+      KMSOp operation) throws AccessControlException {
     assertAccess(aclType, principal, operation, null);
   }
 
   private void assertAccess(KMSACLs.Type aclType, Principal principal,
-      String operation, String key) throws AccessControlException {
+      KMSOp 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(
+      throw new AuthorizationException(String.format(
           (key != null) ? UNAUTHORIZED_MSG_WITH_KEY 
                         : UNAUTHORIZED_MSG_WITHOUT_KEY,
           principal.getName(), operation, key));
@@ -135,7 +129,7 @@ public class KMS {
     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);
+    assertAccess(KMSACLs.Type.CREATE, user, KMSOp.CREATE_KEY, name);
     String cipher = (String) jsonKey.get(KMSRESTConstants.CIPHER_FIELD);
     String material = (String) jsonKey.get(KMSRESTConstants.MATERIAL_FIELD);
     int length = (jsonKey.containsKey(KMSRESTConstants.LENGTH_FIELD))
@@ -146,7 +140,7 @@ public class KMS {
         jsonKey.get(KMSRESTConstants.ATTRIBUTES_FIELD);
     if (material != null) {
       assertAccess(KMSACLs.Type.SET_KEY_MATERIAL, user,
-          CREATE_KEY + " with user provided material", name);
+          KMSOp.CREATE_KEY, name);
     }
     KeyProvider.Options options = new KeyProvider.Options(
         KMSWebApp.getConfiguration());
@@ -165,7 +159,7 @@ public class KMS {
 
     provider.flush();
 
-    kmsAudit.ok(user, CREATE_KEY, name, "UserProvidedMaterial:" +
+    kmsAudit.ok(user, KMSOp.CREATE_KEY, name, "UserProvidedMaterial:" +
         (material != null) + " Description:" + description);
 
     if (!KMSWebApp.getACLs().hasAccess(KMSACLs.Type.GET, user.getName())) {
@@ -186,12 +180,12 @@ public class KMS {
       @PathParam("name") String name) throws Exception {
     KMSWebApp.getAdminCallsMeter().mark();
     Principal user = getPrincipal(securityContext);
-    assertAccess(KMSACLs.Type.DELETE, user, DELETE_KEY, name);
+    assertAccess(KMSACLs.Type.DELETE, user, KMSOp.DELETE_KEY, name);
     KMSClientProvider.checkNotEmpty(name, "name");
     provider.deleteKey(name);
     provider.flush();
 
-    kmsAudit.ok(user, DELETE_KEY, name, "");
+    kmsAudit.ok(user, KMSOp.DELETE_KEY, name, "");
 
     return Response.ok().build();
   }
@@ -205,13 +199,13 @@ public class KMS {
       throws Exception {
     KMSWebApp.getAdminCallsMeter().mark();
     Principal user = getPrincipal(securityContext);
-    assertAccess(KMSACLs.Type.ROLLOVER, user, ROLL_NEW_VERSION, name);
+    assertAccess(KMSACLs.Type.ROLLOVER, user, KMSOp.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);
+          KMSOp.ROLL_NEW_VERSION, name);
     }
     KeyProvider.KeyVersion keyVersion = (material != null)
         ? provider.rollNewVersion(name, Base64.decodeBase64(material))
@@ -219,7 +213,7 @@ public class KMS {
 
     provider.flush();
 
-    kmsAudit.ok(user, ROLL_NEW_VERSION, name, "UserProvidedMaterial:" +
+    kmsAudit.ok(user, KMSOp.ROLL_NEW_VERSION, name, "UserProvidedMaterial:" +
         (material != null) + " NewVersion:" + keyVersion.getVersionName());
 
     if (!KMSWebApp.getACLs().hasAccess(KMSACLs.Type.GET, user.getName())) {
@@ -233,15 +227,15 @@ public class KMS {
   @Path(KMSRESTConstants.KEYS_METADATA_RESOURCE)
   @Produces(MediaType.APPLICATION_JSON)
   public Response getKeysMetadata(@Context SecurityContext securityContext,
-      @QueryParam(KMSRESTConstants.KEY_OP) List<String> keyNamesList)
+      @QueryParam(KMSRESTConstants.KEY) List<String> keyNamesList)
       throws Exception {
     KMSWebApp.getAdminCallsMeter().mark();
     Principal user = getPrincipal(securityContext);
     String[] keyNames = keyNamesList.toArray(new String[keyNamesList.size()]);
-    assertAccess(KMSACLs.Type.GET_METADATA, user, GET_KEYS_METADATA);
+    assertAccess(KMSACLs.Type.GET_METADATA, user, KMSOp.GET_KEYS_METADATA);
     KeyProvider.Metadata[] keysMeta = provider.getKeysMetadata(keyNames);
     Object json = KMSServerJSONUtils.toJSON(keyNames, keysMeta);
-    kmsAudit.ok(user, GET_KEYS_METADATA, "");
+    kmsAudit.ok(user, KMSOp.GET_KEYS_METADATA, "");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
@@ -252,9 +246,9 @@ public class KMS {
       throws Exception {
     KMSWebApp.getAdminCallsMeter().mark();
     Principal user = getPrincipal(securityContext);
-    assertAccess(KMSACLs.Type.GET_KEYS, user, GET_KEYS);
+    assertAccess(KMSACLs.Type.GET_KEYS, user, KMSOp.GET_KEYS);
     Object json = provider.getKeys();
-    kmsAudit.ok(user, GET_KEYS, "");
+    kmsAudit.ok(user, KMSOp.GET_KEYS, "");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
@@ -276,9 +270,9 @@ public class KMS {
     Principal user = getPrincipal(securityContext);
     KMSClientProvider.checkNotEmpty(name, "name");
     KMSWebApp.getAdminCallsMeter().mark();
-    assertAccess(KMSACLs.Type.GET_METADATA, user, GET_METADATA, name);
+    assertAccess(KMSACLs.Type.GET_METADATA, user, KMSOp.GET_METADATA, name);
     Object json = KMSServerJSONUtils.toJSON(name, provider.getMetadata(name));
-    kmsAudit.ok(user, GET_METADATA, name, "");
+    kmsAudit.ok(user, KMSOp.GET_METADATA, name, "");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
@@ -292,9 +286,9 @@ public class KMS {
     Principal user = getPrincipal(securityContext);
     KMSClientProvider.checkNotEmpty(name, "name");
     KMSWebApp.getKeyCallsMeter().mark();
-    assertAccess(KMSACLs.Type.GET, user, GET_CURRENT_KEY, name);
+    assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_CURRENT_KEY, name);
     Object json = KMSServerJSONUtils.toJSON(provider.getCurrentKey(name));
-    kmsAudit.ok(user, GET_CURRENT_KEY, name, "");
+    kmsAudit.ok(user, KMSOp.GET_CURRENT_KEY, name, "");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 
@@ -308,9 +302,9 @@ public class KMS {
     KMSClientProvider.checkNotEmpty(versionName, "versionName");
     KMSWebApp.getKeyCallsMeter().mark();
     KeyVersion keyVersion = provider.getKeyVersion(versionName);
-    assertAccess(KMSACLs.Type.GET, user, GET_KEY_VERSION);
+    assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_KEY_VERSION);
     if (keyVersion != null) {
-      kmsAudit.ok(user, GET_KEY_VERSION, keyVersion.getName(), "");
+      kmsAudit.ok(user, KMSOp.GET_KEY_VERSION, keyVersion.getName(), "");
     }
     Object json = KMSServerJSONUtils.toJSON(keyVersion);
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
@@ -334,7 +328,7 @@ public class KMS {
 
     Object retJSON;
     if (edekOp.equals(KMSRESTConstants.EEK_GENERATE)) {
-      assertAccess(KMSACLs.Type.GENERATE_EEK, user, GENERATE_EEK, name);
+      assertAccess(KMSACLs.Type.GENERATE_EEK, user, KMSOp.GENERATE_EEK, name);
 
       List<EncryptedKeyVersion> retEdeks =
           new LinkedList<EncryptedKeyVersion>();
@@ -345,7 +339,7 @@ public class KMS {
       } catch (Exception e) {
         throw new IOException(e);
       }
-      kmsAudit.ok(user, GENERATE_EEK, name, "");
+      kmsAudit.ok(user, KMSOp.GENERATE_EEK, name, "");
       retJSON = new ArrayList();
       for (EncryptedKeyVersion edek : retEdeks) {
         ((ArrayList)retJSON).add(KMSServerJSONUtils.toJSON(edek));
@@ -380,7 +374,7 @@ public class KMS {
         (String) jsonPayload.get(KMSRESTConstants.MATERIAL_FIELD);
     Object retJSON;
     if (eekOp.equals(KMSRESTConstants.EEK_DECRYPT)) {
-      assertAccess(KMSACLs.Type.DECRYPT_EEK, user, DECRYPT_EEK, keyName);
+      assertAccess(KMSACLs.Type.DECRYPT_EEK, user, KMSOp.DECRYPT_EEK, keyName);
       KMSClientProvider.checkNotNull(ivStr, KMSRESTConstants.IV_FIELD);
       byte[] iv = Base64.decodeBase64(ivStr);
       KMSClientProvider.checkNotNull(encMaterialStr,
@@ -391,7 +385,7 @@ public class KMS {
               new KMSClientProvider.KMSEncryptedKeyVersion(keyName, versionName,
                   iv, KeyProviderCryptoExtension.EEK, encMaterial));
       retJSON = KMSServerJSONUtils.toJSON(retKeyVersion);
-      kmsAudit.ok(user, DECRYPT_EEK, keyName, "");
+      kmsAudit.ok(user, KMSOp.DECRYPT_EEK, keyName, "");
     } else {
       throw new IllegalArgumentException("Wrong " + KMSRESTConstants.EEK_OP +
           " value, it must be " + KMSRESTConstants.EEK_GENERATE + " or " +
@@ -412,9 +406,9 @@ public class KMS {
     Principal user = getPrincipal(securityContext);
     KMSClientProvider.checkNotEmpty(name, "name");
     KMSWebApp.getKeyCallsMeter().mark();
-    assertAccess(KMSACLs.Type.GET, user, GET_KEY_VERSIONS, name);
+    assertAccess(KMSACLs.Type.GET, user, KMSOp.GET_KEY_VERSIONS, name);
     Object json = KMSServerJSONUtils.toJSON(provider.getKeyVersions(name));
-    kmsAudit.ok(user, GET_KEY_VERSIONS, name, "");
+    kmsAudit.ok(user, KMSOp.GET_KEY_VERSIONS, name, "");
     return Response.ok().type(MediaType.APPLICATION_JSON).entity(json).build();
   }
 

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java?rev=1617377&r1=1617376&r2=1617377&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java Mon Aug 11 22:27:50 2014
@@ -50,11 +50,11 @@ public class KMSAudit {
     private final AtomicLong accessCount = new AtomicLong(-1);
     private final String keyName;
     private final String user;
-    private final String op;
+    private final KMS.KMSOp op;
     private final String extraMsg;
     private final long startTime = System.currentTimeMillis();
 
-    private AuditEvent(String keyName, String user, String op, String msg) {
+    private AuditEvent(String keyName, String user, KMS.KMSOp op, String msg) {
       this.keyName = keyName;
       this.user = user;
       this.op = op;
@@ -77,7 +77,7 @@ public class KMSAudit {
       return user;
     }
 
-    public String getOp() {
+    public KMS.KMSOp getOp() {
       return op;
     }
 
@@ -90,8 +90,9 @@ public class KMSAudit {
     OK, UNAUTHORIZED, UNAUTHENTICATED, ERROR;
   }
 
-  private static Set<String> AGGREGATE_OPS_WHITELIST = Sets.newHashSet(
-    KMS.GET_KEY_VERSION, KMS.GET_CURRENT_KEY, KMS.DECRYPT_EEK, KMS.GENERATE_EEK
+  private static Set<KMS.KMSOp> AGGREGATE_OPS_WHITELIST = Sets.newHashSet(
+    KMS.KMSOp.GET_KEY_VERSION, KMS.KMSOp.GET_CURRENT_KEY,
+    KMS.KMSOp.DECRYPT_EEK, KMS.KMSOp.GENERATE_EEK
   );
 
   private Cache<String, AuditEvent> cache;
@@ -137,10 +138,10 @@ public class KMSAudit {
         event.getExtraMsg());
   }
 
-  private void op(OpStatus opStatus, final String op, final String user,
+  private void op(OpStatus opStatus, final KMS.KMSOp op, final String user,
       final String key, final String extraMsg) {
     if (!Strings.isNullOrEmpty(user) && !Strings.isNullOrEmpty(key)
-        && !Strings.isNullOrEmpty(op)
+        && (op != null)
         && AGGREGATE_OPS_WHITELIST.contains(op)) {
       String cacheKey = createCacheKey(user, key, op);
       if (opStatus == OpStatus.UNAUTHORIZED) {
@@ -167,7 +168,7 @@ public class KMSAudit {
       }
     } else {
       List<String> kvs = new LinkedList<String>();
-      if (!Strings.isNullOrEmpty(op)) {
+      if (op != null) {
         kvs.add("op=" + op);
       }
       if (!Strings.isNullOrEmpty(key)) {
@@ -185,16 +186,16 @@ public class KMSAudit {
     }
   }
 
-  public void ok(Principal user, String op, String key,
+  public void ok(Principal user, KMS.KMSOp op, String key,
       String extraMsg) {
     op(OpStatus.OK, op, user.getName(), key, extraMsg);
   }
 
-  public void ok(Principal user, String op, String extraMsg) {
+  public void ok(Principal user, KMS.KMSOp op, String extraMsg) {
     op(OpStatus.OK, op, user.getName(), null, extraMsg);
   }
 
-  public void unauthorized(Principal user, String op, String key) {
+  public void unauthorized(Principal user, KMS.KMSOp op, String key) {
     op(OpStatus.UNAUTHORIZED, op, user.getName(), key, "");
   }
 
@@ -211,7 +212,7 @@ public class KMSAudit {
         + " URL:" + url + " ErrorMsg:'" + extraMsg + "'");
   }
 
-  private static String createCacheKey(String user, String key, String op) {
+  private static String createCacheKey(String user, String key, KMS.KMSOp op) {
     return user + "#" + key + "#" + op;
   }
 

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java?rev=1617377&r1=1617376&r2=1617377&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java Mon Aug 11 22:27:50 2014
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.crypto.key.kms.server;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 
 import java.io.File;
@@ -26,6 +27,7 @@ import java.net.URL;
 /**
  * Utility class to load KMS configuration files.
  */
+@InterfaceAudience.Private
 public class KMSConfiguration {
 
   public static final String KMS_CONFIG_DIR = "kms.config.dir";

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJMXServlet.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJMXServlet.java?rev=1617377&r1=1617376&r2=1617377&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJMXServlet.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSJMXServlet.java Mon Aug 11 22:27:50 2014
@@ -17,12 +17,15 @@
  */
 package org.apache.hadoop.crypto.key.kms.server;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.jmx.JMXJsonServlet;
 
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
+
 import java.io.IOException;
 
+@InterfaceAudience.Private
 public class KMSJMXServlet extends JMXJsonServlet {
 
   @Override

Modified: hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java?rev=1617377&r1=1617376&r2=1617377&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java (original)
+++ hadoop/common/branches/HDFS-6584/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java Mon Aug 11 22:27:50 2014
@@ -23,6 +23,7 @@ import java.io.OutputStream;
 import java.io.PrintStream;
 import java.security.Principal;
 
+import org.apache.hadoop.crypto.key.kms.server.KMS.KMSOp;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.PropertyConfigurator;
 import org.junit.After;
@@ -82,16 +83,16 @@ public class TestKMSAudit {
   public void testAggregation() throws Exception {
     Principal luser = Mockito.mock(Principal.class);
     Mockito.when(luser.getName()).thenReturn("luser");
-    kmsAudit.ok(luser, KMS.DECRYPT_EEK, "k1", "testmsg");
-    kmsAudit.ok(luser, KMS.DECRYPT_EEK, "k1", "testmsg");
-    kmsAudit.ok(luser, KMS.DECRYPT_EEK, "k1", "testmsg");
-    kmsAudit.ok(luser, KMS.DELETE_KEY, "k1", "testmsg");
-    kmsAudit.ok(luser, KMS.ROLL_NEW_VERSION, "k1", "testmsg");
-    kmsAudit.ok(luser, KMS.DECRYPT_EEK, "k1", "testmsg");
-    kmsAudit.ok(luser, KMS.DECRYPT_EEK, "k1", "testmsg");
-    kmsAudit.ok(luser, KMS.DECRYPT_EEK, "k1", "testmsg");
+    kmsAudit.ok(luser, KMSOp.DECRYPT_EEK, "k1", "testmsg");
+    kmsAudit.ok(luser, KMSOp.DECRYPT_EEK, "k1", "testmsg");
+    kmsAudit.ok(luser, KMSOp.DECRYPT_EEK, "k1", "testmsg");
+    kmsAudit.ok(luser, KMSOp.DELETE_KEY, "k1", "testmsg");
+    kmsAudit.ok(luser, KMSOp.ROLL_NEW_VERSION, "k1", "testmsg");
+    kmsAudit.ok(luser, KMSOp.DECRYPT_EEK, "k1", "testmsg");
+    kmsAudit.ok(luser, KMSOp.DECRYPT_EEK, "k1", "testmsg");
+    kmsAudit.ok(luser, KMSOp.DECRYPT_EEK, "k1", "testmsg");
     Thread.sleep(1500);
-    kmsAudit.ok(luser, KMS.DECRYPT_EEK, "k1", "testmsg");
+    kmsAudit.ok(luser, KMSOp.DECRYPT_EEK, "k1", "testmsg");
     Thread.sleep(1500);
     String out = getAndResetLogOutput();
     System.out.println(out);
@@ -110,15 +111,15 @@ public class TestKMSAudit {
   public void testAggregationUnauth() throws Exception {
     Principal luser = Mockito.mock(Principal.class);
     Mockito.when(luser.getName()).thenReturn("luser");
-    kmsAudit.unauthorized(luser, KMS.GENERATE_EEK, "k2");
+    kmsAudit.unauthorized(luser, KMSOp.GENERATE_EEK, "k2");
     Thread.sleep(1000);
-    kmsAudit.ok(luser, KMS.GENERATE_EEK, "k3", "testmsg");
-    kmsAudit.ok(luser, KMS.GENERATE_EEK, "k3", "testmsg");
-    kmsAudit.ok(luser, KMS.GENERATE_EEK, "k3", "testmsg");
-    kmsAudit.ok(luser, KMS.GENERATE_EEK, "k3", "testmsg");
-    kmsAudit.ok(luser, KMS.GENERATE_EEK, "k3", "testmsg");
-    kmsAudit.unauthorized(luser, KMS.GENERATE_EEK, "k3");
-    kmsAudit.ok(luser, KMS.GENERATE_EEK, "k3", "testmsg");
+    kmsAudit.ok(luser, KMSOp.GENERATE_EEK, "k3", "testmsg");
+    kmsAudit.ok(luser, KMSOp.GENERATE_EEK, "k3", "testmsg");
+    kmsAudit.ok(luser, KMSOp.GENERATE_EEK, "k3", "testmsg");
+    kmsAudit.ok(luser, KMSOp.GENERATE_EEK, "k3", "testmsg");
+    kmsAudit.ok(luser, KMSOp.GENERATE_EEK, "k3", "testmsg");
+    kmsAudit.unauthorized(luser, KMSOp.GENERATE_EEK, "k3");
+    kmsAudit.ok(luser, KMSOp.GENERATE_EEK, "k3", "testmsg");
     Thread.sleep(2000);
     String out = getAndResetLogOutput();
     System.out.println(out);