You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gobblin.apache.org by hu...@apache.org on 2018/03/28 00:11:15 UTC

incubator-gobblin git commit: [GOBBLIN-444] add support for rotation of passwords

Repository: incubator-gobblin
Updated Branches:
  refs/heads/master 5d0e944c3 -> 57a6566ac


[GOBBLIN-444] add support for rotation of passwords

Closes #2318 from arjun4084346/rotatePasswords


Project: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/commit/57a6566a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/tree/57a6566a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gobblin/diff/57a6566a

Branch: refs/heads/master
Commit: 57a6566ac635053bb171ee35c0d00b9f1e2483c8
Parents: 5d0e944
Author: Arjun <ab...@linkedin.com>
Authored: Tue Mar 27 17:11:10 2018 -0700
Committer: Hung Tran <hu...@linkedin.com>
Committed: Tue Mar 27 17:11:10 2018 -0700

----------------------------------------------------------------------
 .../configuration/ConfigurationKeys.java        |   2 +
 .../gobblin/password/PasswordManager.java       | 158 +++++++++++++------
 .../gobblin/password/PasswordManagerTest.java   | 103 +++++++++++-
 3 files changed, 205 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/57a6566a/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java b/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
index 71d78a9..8e2d225 100644
--- a/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
+++ b/gobblin-api/src/main/java/org/apache/gobblin/configuration/ConfigurationKeys.java
@@ -796,6 +796,8 @@ public class ConfigurationKeys {
   public static final String ENCRYPT_KEY_LOC = "encrypt.key.loc";
   public static final String ENCRYPT_USE_STRONG_ENCRYPTOR = "encrypt.use.strong.encryptor";
   public static final boolean DEFAULT_ENCRYPT_USE_STRONG_ENCRYPTOR = false;
+  public static final String NUMBER_OF_ENCRYPT_KEYS = "num.encrypt.keys";
+  public static final int DEFAULT_NUMBER_OF_MASTER_PASSWORDS = 2;
 
   /**
    * Proxy Filesystem operation properties.

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/57a6566a/gobblin-api/src/main/java/org/apache/gobblin/password/PasswordManager.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/main/java/org/apache/gobblin/password/PasswordManager.java b/gobblin-api/src/main/java/org/apache/gobblin/password/PasswordManager.java
index 6df6d7e..10ee5ac 100644
--- a/gobblin-api/src/main/java/org/apache/gobblin/password/PasswordManager.java
+++ b/gobblin-api/src/main/java/org/apache/gobblin/password/PasswordManager.java
@@ -17,11 +17,14 @@
 
 package org.apache.gobblin.password;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.net.URI;
 import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.ExecutionException;
@@ -50,10 +53,16 @@ import com.google.common.io.LineReader;
 import org.apache.gobblin.configuration.ConfigurationKeys;
 import org.apache.gobblin.configuration.State;
 
+import lombok.EqualsAndHashCode;
 
 /**
  * A class for managing password encryption and decryption. To encrypt or decrypt a password, a master password
  * should be provided which is used as encryption or decryption key.
+ * Encryption is done with the single key provided.
+ * Decryption is tried with multiple keys to facilitate key rotation.
+ * If the master key file provided is /var/tmp/masterKey.txt, decryption is tried with keys at
+ * /var/tmp/masterKey.txt, /var/tmp/masterKey.txt.1, /var/tmp/masterKey.txt.2, and so on and so forth till
+ * either any such file does not exist or {@code this.NUMBER_OF_ENCRYPT_KEYS} attempts have been made.
  *
  * @author Ziyang Liu
  */
@@ -64,35 +73,73 @@ public class PasswordManager {
   private static final long CACHE_SIZE = 100;
   private static final long CACHE_EXPIRATION_MIN = 10;
   private static final Pattern PASSWORD_PATTERN = Pattern.compile("ENC\\((.*)\\)");
+  private final boolean useStrongEncryptor;
+  private FileSystem fs;
+  private List<TextEncryptor> encryptors;
 
-  private static final LoadingCache<Map.Entry<Optional<String>, Boolean>, PasswordManager> CACHED_INSTANCES =
+  private static final LoadingCache<CachedInstanceKey, PasswordManager> CACHED_INSTANCES =
       CacheBuilder.newBuilder().maximumSize(CACHE_SIZE).expireAfterAccess(CACHE_EXPIRATION_MIN, TimeUnit.MINUTES)
-          .build(new CacheLoader<Map.Entry<Optional<String>, Boolean>, PasswordManager>() {
-
+          .build(new CacheLoader<CachedInstanceKey, PasswordManager>() {
             @Override
-            public PasswordManager load(Map.Entry<Optional<String>, Boolean> cacheKey) {
-              return new PasswordManager(cacheKey.getKey(), cacheKey.getValue());
+            public PasswordManager load(CachedInstanceKey cacheKey) {
+              return new PasswordManager(cacheKey);
             }
           });
 
-  private Optional<TextEncryptor> encryptor;
+  private PasswordManager(CachedInstanceKey cacheKey) {
+    this.useStrongEncryptor = cacheKey.useStrongEncryptor;
 
-  private PasswordManager(Optional<String> masterPassword, boolean useStrongEncryptor) {
-    if (masterPassword.isPresent()) {
-      this.encryptor = useStrongEncryptor ? Optional.of((TextEncryptor) new StrongTextEncryptor())
-          : Optional.of((TextEncryptor) new BasicTextEncryptor());
-      try {
+    try {
+      this.fs = cacheKey.fsURI != null ? FileSystem.get(URI.create(cacheKey.fsURI), new Configuration())
+          : (cacheKey.masterPasswordFile != null ? new Path(cacheKey.masterPasswordFile).getFileSystem(new Configuration()) : null);
+    } catch (IOException e) {
+      LOG.warn("Failed to instantiate FileSystem.", e);
+    }
+    this.encryptors = getEncryptors(cacheKey);
+  }
 
+  private List<TextEncryptor> getEncryptors(CachedInstanceKey cacheKey) {
+    List<TextEncryptor> encryptors = new ArrayList<>();
+    int numOfEncryptionKeys = cacheKey.numOfEncryptionKeys;
+    String suffix = "";
+    int i = 1;
+
+    if (cacheKey.masterPasswordFile == null || numOfEncryptionKeys < 1) {
+      return encryptors;
+    }
+
+    Exception exception = null;
+
+    do {
+      Path currentMasterPasswordFile = new Path(cacheKey.masterPasswordFile + suffix);
+      try (Closer closer = Closer.create()) {
+        if (!fs.exists(currentMasterPasswordFile) ||
+            fs.getFileStatus(currentMasterPasswordFile).isDirectory()) {
+          continue;
+        }
+        InputStream in = closer.register(fs.open(currentMasterPasswordFile));
+        String masterPassword = new LineReader(new InputStreamReader(in, Charsets.UTF_8)).readLine();
+        TextEncryptor encryptor = useStrongEncryptor ? new StrongTextEncryptor() : new BasicTextEncryptor();
         // setPassword() needs to be called via reflection since the TextEncryptor interface doesn't have this method.
-        this.encryptor.get().getClass().getMethod("setPassword", String.class).invoke(this.encryptor.get(),
-            masterPassword.get());
+        encryptor.getClass().getMethod("setPassword", String.class).invoke(encryptor, masterPassword);
+        encryptors.add(encryptor);
+        suffix = "." + String.valueOf(i);
+      } catch (FileNotFoundException fnf) {
+        // It is ok for password files not being present
+        LOG.warn("Master password file " + currentMasterPasswordFile + " not found.");
+      } catch (IOException ioe) {
+        exception = ioe;
+        LOG.warn("Master password could not be read from file " + currentMasterPasswordFile);
       } catch (Exception e) {
-        LOG.error("Failed to set master password for encryptor", e);
-        this.encryptor = Optional.absent();
+        LOG.warn("Encryptor could not be instantiated.");
       }
-    } else {
-      this.encryptor = Optional.absent();
+    } while (i++ < numOfEncryptionKeys);
+
+    // Throw exception if could not read any existing password file
+    if (encryptors.size() < 1 && exception != null) {
+      throw new RuntimeException("Master Password could not be read from any master password file.", exception);
     }
+    return encryptors;
   }
 
   /**
@@ -100,8 +147,7 @@ public class PasswordManager {
    */
   public static PasswordManager getInstance() {
     try {
-      Optional<String> absent = Optional.absent();
-      return CACHED_INSTANCES.get(new AbstractMap.SimpleEntry<>(absent, shouldUseStrongEncryptor(new State())));
+      return CACHED_INSTANCES.get(new CachedInstanceKey());
     } catch (ExecutionException e) {
       throw new RuntimeException("Unable to get an instance of PasswordManager", e);
     }
@@ -113,7 +159,7 @@ public class PasswordManager {
   public static PasswordManager getInstance(State state) {
     try {
       return CACHED_INSTANCES
-          .get(new AbstractMap.SimpleEntry<>(getMasterPassword(state), shouldUseStrongEncryptor(state)));
+          .get(new CachedInstanceKey(state));
     } catch (ExecutionException e) {
       throw new RuntimeException("Unable to get an instance of PasswordManager", e);
     }
@@ -130,9 +176,12 @@ public class PasswordManager {
    * Get an instance. The master password file is given by masterPwdLoc.
    */
   public static PasswordManager getInstance(Path masterPwdLoc) {
+    State state = new State();
+    state.setProp(ConfigurationKeys.ENCRYPT_KEY_LOC, masterPwdLoc.toString());
+    state.setProp(ConfigurationKeys.ENCRYPT_KEY_FS_URI, masterPwdLoc.toUri());
     try {
       return CACHED_INSTANCES
-          .get(new AbstractMap.SimpleEntry<>(getMasterPassword(masterPwdLoc), shouldUseStrongEncryptor(new State())));
+          .get(new CachedInstanceKey(state));
     } catch (ExecutionException e) {
       throw new RuntimeException("Unable to get an instance of PasswordManager", e);
     }
@@ -149,38 +198,43 @@ public class PasswordManager {
    * @return The encrypted password.
    */
   public String encryptPassword(String plain) {
-    Preconditions.checkArgument(this.encryptor.isPresent(),
+    Preconditions.checkArgument(this.encryptors.size() > 0,
         "A master password needs to be provided for encrypting passwords.");
 
     try {
-      return this.encryptor.get().encrypt(plain);
+      return this.encryptors.get(0).encrypt(plain);
     } catch (Exception e) {
       throw new RuntimeException("Failed to encrypt password", e);
     }
   }
 
   /**
-   * Decrypt an encrypted password. A master password must have been provided in the constructor.
+   * Decrypt an encrypted password. A master password file must have been provided in the constructor.
    * @param encrypted An encrypted password.
    * @return The decrypted password.
    */
   public String decryptPassword(String encrypted) {
-    Preconditions.checkArgument(this.encryptor.isPresent(),
+    Preconditions.checkArgument(this.encryptors.size() > 0,
         "A master password needs to be provided for decrypting passwords.");
 
-    try {
-      return this.encryptor.get().decrypt(encrypted);
-    } catch (Exception e) {
-      throw new RuntimeException("Failed to decrypt password " + encrypted, e);
+    for (TextEncryptor encryptor : encryptors) {
+      try {
+        return encryptor.decrypt(encrypted);
+      } catch (Exception e) {
+        LOG.warn("Failed attempt to decrypt secret {}", encrypted, e);
+      }
     }
+    LOG.error("All {} decrypt attempt(s) failed.", encryptors.size());
+    throw new RuntimeException("Failed to decrypt password ENC(" + encrypted + ")");
   }
 
   /**
-   * Decrypt a password if it is an encrypted password (in the form of ENC(.*)), and a master password has been
-   * provided in the constructor. Otherwise, return the password as is.
+   * Decrypt a password if it is an encrypted password (in the form of ENC(.*))
+   * and a master password file has been provided in the constructor.
+   * Otherwise, return the password as is.
    */
   public String readPassword(String password) {
-    if (password == null || !this.encryptor.isPresent()) {
+    if (password == null || encryptors.size() < 1) {
       return password;
     }
     Matcher matcher = PASSWORD_PATTERN.matcher(password);
@@ -190,25 +244,6 @@ public class PasswordManager {
     return password;
   }
 
-  private static Optional<String> getMasterPassword(State state) {
-    if (!state.contains(ConfigurationKeys.ENCRYPT_KEY_LOC)) {
-      LOG.warn(String.format("Property %s not set. Cannot decrypt any encrypted password.",
-          ConfigurationKeys.ENCRYPT_KEY_LOC));
-      return Optional.absent();
-    }
-    try {
-      if (state.contains(ConfigurationKeys.ENCRYPT_KEY_FS_URI)) {
-        FileSystem fs =
-            FileSystem.get(URI.create(state.getProp(ConfigurationKeys.ENCRYPT_KEY_FS_URI)), new Configuration());
-        return getMasterPassword(fs, new Path(state.getProp(ConfigurationKeys.ENCRYPT_KEY_LOC)));
-      }
-      return getMasterPassword(new Path(state.getProp(ConfigurationKeys.ENCRYPT_KEY_LOC)));
-    } catch (Exception e) {
-      throw new RuntimeException(
-          "Failed to obtain master password from " + state.getProp(ConfigurationKeys.ENCRYPT_KEY_LOC), e);
-    }
-  }
-
   public static Optional<String> getMasterPassword(Path masterPasswordFile) {
     try {
       FileSystem fs = masterPasswordFile.getFileSystem(new Configuration());
@@ -230,4 +265,23 @@ public class PasswordManager {
       throw new RuntimeException("Failed to obtain master password from " + masterPasswordFile, e);
     }
   }
-}
+
+  @EqualsAndHashCode
+  private static class CachedInstanceKey {
+    int numOfEncryptionKeys;
+    String fsURI;
+    String masterPasswordFile;
+    boolean useStrongEncryptor;
+
+    public CachedInstanceKey(State state) {
+      this.numOfEncryptionKeys = state.getPropAsInt(ConfigurationKeys.NUMBER_OF_ENCRYPT_KEYS, ConfigurationKeys.DEFAULT_NUMBER_OF_MASTER_PASSWORDS);
+      this.useStrongEncryptor = shouldUseStrongEncryptor(state);
+      this.fsURI = state.getProp(ConfigurationKeys.ENCRYPT_KEY_LOC);
+      this.masterPasswordFile = state.getProp(ConfigurationKeys.ENCRYPT_KEY_LOC);
+    }
+
+    public CachedInstanceKey() {
+
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gobblin/blob/57a6566a/gobblin-api/src/test/java/org/apache/gobblin/password/PasswordManagerTest.java
----------------------------------------------------------------------
diff --git a/gobblin-api/src/test/java/org/apache/gobblin/password/PasswordManagerTest.java b/gobblin-api/src/test/java/org/apache/gobblin/password/PasswordManagerTest.java
index 24f3a42..2bf2c19 100644
--- a/gobblin-api/src/test/java/org/apache/gobblin/password/PasswordManagerTest.java
+++ b/gobblin-api/src/test/java/org/apache/gobblin/password/PasswordManagerTest.java
@@ -33,10 +33,9 @@ import org.testng.annotations.Test;
 
 import com.google.common.io.Files;
 
-@Test(enabled=false, groups = {"disabledOnTravis"} )
 public class PasswordManagerTest {
 
-  @Test (enabled=false)
+  @Test
   public void testReadNormalPassword() throws IOException {
     String password = UUID.randomUUID().toString();
     String masterPassword = UUID.randomUUID().toString();
@@ -47,7 +46,7 @@ public class PasswordManagerTest {
     masterPwdFile.delete();
   }
 
-  @Test (enabled=false)
+  @Test
   public void testMasterPasswordNotExist() {
     String password = "ENC(" + UUID.randomUUID().toString() + ")";
     State state = new State();
@@ -55,7 +54,7 @@ public class PasswordManagerTest {
     Assert.assertEquals(PasswordManager.getInstance(state).readPassword(password), password);
   }
 
-  @Test (enabled=false)
+  @Test
   public void testBasicEncryptionAndDecryption() throws IOException {
     String password = UUID.randomUUID().toString();
     String masterPassword = UUID.randomUUID().toString();
@@ -70,7 +69,7 @@ public class PasswordManagerTest {
     Assert.assertEquals(decrypted, password);
   }
 
-  @Test (enabled=false)
+  @Test
   public void testStrongEncryptionAndDecryption() throws IOException {
     String password = UUID.randomUUID().toString();
     String masterPassword = UUID.randomUUID().toString();
@@ -91,9 +90,101 @@ public class PasswordManagerTest {
     }
   }
 
+  @Test
+  public void testMultipleMasterPasswords() throws IOException {
+    String password = UUID.randomUUID().toString();
+
+    String masterPassword = UUID.randomUUID().toString();
+    String masterPassword1 = UUID.randomUUID().toString();
+    String masterPassword2 = UUID.randomUUID().toString();
+    String masterPassword3 = UUID.randomUUID().toString();
+
+    File masterPasswordFile = File.createTempFile("masterPassword", null);
+    Files.write(masterPassword, masterPasswordFile, Charset.defaultCharset());
+    Files.write(masterPassword1, new File(masterPasswordFile.toString()+".1"), Charset.defaultCharset());
+    Files.write(masterPassword2, new File(masterPasswordFile.toString()+".2"), Charset.defaultCharset());
+    Files.write(masterPassword3, new File(masterPasswordFile.toString()+".3"), Charset.defaultCharset());
+
+    State state = new State();
+    BasicTextEncryptor encryptor = new BasicTextEncryptor();
+
+    state.setProp(ConfigurationKeys.ENCRYPT_KEY_LOC, masterPasswordFile.toString());
+    state.setProp(ConfigurationKeys.NUMBER_OF_ENCRYPT_KEYS, 3);
+    PasswordManager passwordManager = PasswordManager.getInstance(state);
+
+    // Test current master password
+    encryptor.setPassword(masterPassword);
+    String encrypted = "ENC(" + encryptor.encrypt(password) + ")";
+    String decrypted = passwordManager.readPassword(encrypted);
+    Assert.assertEquals(decrypted, password);
+
+    // Test last master password using same passwordManager
+    encryptor = new BasicTextEncryptor();
+    encryptor.setPassword(masterPassword1);
+    encrypted = "ENC(" + encryptor.encrypt(password) + ")";
+    decrypted = passwordManager.readPassword(encrypted);
+    Assert.assertEquals(decrypted, password);
+
+    // Test second last master password using same passwordManager
+    encryptor = new BasicTextEncryptor();
+    encryptor.setPassword(masterPassword2);
+    encrypted = "ENC(" + encryptor.encrypt(password) + ")";
+    decrypted = passwordManager.readPassword(encrypted);
+    Assert.assertEquals(decrypted, password);
+
+    // Test third last master password using same passwordManager
+    // This one is not accepted because ConfigurationKeys.NUMBER_OF_ENCRYPT_KEYS = 3
+    encryptor = new BasicTextEncryptor();
+    encryptor.setPassword(masterPassword3);
+    encrypted = "ENC(" + encryptor.encrypt(password) + ")";
+    try {
+      passwordManager.readPassword(encrypted);
+    } catch (RuntimeException e) {
+      Assert.assertTrue(e.getMessage().startsWith( "Failed to decrypt password"));
+      return;
+    }
+    Assert.fail("Password Manager decrypted too old password.");
+  }
+
+  @Test
+  public void testMultipleMasterPasswordsWithoutPasswordFiles() throws IOException {
+    String password = UUID.randomUUID().toString();
+
+    String masterPassword = UUID.randomUUID().toString();
+    String masterPassword1 = UUID.randomUUID().toString();
+
+    File masterPasswordFile = File.createTempFile("masterPassword", null);
+    Files.write(masterPassword, masterPasswordFile, Charset.defaultCharset());
+
+    State state = new State();
+    BasicTextEncryptor encryptor = new BasicTextEncryptor();
+
+    state.setProp(ConfigurationKeys.ENCRYPT_KEY_LOC, masterPasswordFile.toString());
+    PasswordManager passwordManager = PasswordManager.getInstance(state);
+
+    // Test current master password
+    encryptor.setPassword(masterPassword);
+    String encrypted = "ENC(" + encryptor.encrypt(password) + ")";
+    String decrypted = passwordManager.readPassword(encrypted);
+    Assert.assertEquals(decrypted, password);
+
+    // Test last master password using same passwordManager
+    // This should throw FileNotFoundException as file for masterPassword1 is not created.
+    encryptor = new BasicTextEncryptor();
+    encryptor.setPassword(masterPassword1);
+    encrypted = "ENC(" + encryptor.encrypt(password) + ")";
+    try {
+      passwordManager.readPassword(encrypted);
+    } catch (RuntimeException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Failed to decrypt password"));
+      return;
+    }
+    Assert.fail("Password Manager decrypted password without correct master password.");
+  }
+
   private File getMasterPwdFile(String masterPwd) throws IOException {
     File masterPwdFile = File.createTempFile("masterPassword", null);
     Files.write(masterPwd, masterPwdFile, Charset.defaultCharset());
     return masterPwdFile;
   }
-}
+}
\ No newline at end of file