You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sqoop.apache.org by ja...@apache.org on 2016/03/29 23:14:33 UTC

[2/2] sqoop git commit: SQOOP-2889: Read and write from encrypted repository

SQOOP-2889: Read and write from encrypted repository

(Abraham Fine via Jarek Jarcec Cecho)


Project: http://git-wip-us.apache.org/repos/asf/sqoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/sqoop/commit/5b897a46
Tree: http://git-wip-us.apache.org/repos/asf/sqoop/tree/5b897a46
Diff: http://git-wip-us.apache.org/repos/asf/sqoop/diff/5b897a46

Branch: refs/heads/sqoop2
Commit: 5b897a46fcbbe8a83fdb7ce1ebe8d862c34e9736
Parents: bfcfedf
Author: Jarek Jarcec Cecho <ja...@apache.org>
Authored: Tue Mar 29 14:13:47 2016 -0700
Committer: Jarek Jarcec Cecho <ja...@apache.org>
Committed: Tue Mar 29 14:13:47 2016 -0700

----------------------------------------------------------------------
 .../sqoop/common/test/db/DerbyProvider.java     |  10 +-
 .../sqoop/error/code/CommonRepositoryError.java |   2 +
 .../apache/sqoop/error/code/PasswordError.java  |  44 ++
 .../java/org/apache/sqoop/model/MMasterKey.java |  54 +++
 .../org/apache/sqoop/utils/PasswordUtils.java   |  99 ++++
 .../org/apache/sqoop/utils/ProcessUtils.java    |  40 --
 .../apache/sqoop/utils/TestPasswordUtils.java   |  86 ++++
 .../oracle-jdbc-connector-config.properties     | 170 +++----
 .../java/org/apache/sqoop/core/SqoopServer.java |   3 +
 .../apache/sqoop/repository/JdbcRepository.java |  31 ++
 .../sqoop/repository/JdbcRepositoryHandler.java |  17 +
 .../sqoop/repository/MasterKeyManager.java      | 479 +++++++++++++++++++
 .../org/apache/sqoop/repository/Repository.java |  16 +
 .../sqoop/security/SecurityConstants.java       |  79 +++
 .../apache/sqoop/security/SecurityError.java    |  38 +-
 .../sqoop/repository/TestMasterKeyManager.java  | 208 ++++++++
 dist/src/main/conf/sqoop.properties             |  15 +
 .../common/CommonRepositoryHandler.java         | 107 ++++-
 ...RepositoryInsertUpdateDeleteSelectQuery.java |  64 ++-
 .../common/CommonRepositorySchemaConstants.java |  26 +
 .../derby/DerbyRepositoryHandler.java           |   8 +
 .../derby/DerbySchemaCreateQuery.java           |  42 ++
 .../derby/DerbySchemaUpgradeQuery.java          |  35 ++
 .../sqoop/repository/derby/DerbyTestCase.java   |   8 +
 .../mysql/MySqlRepositoryHandler.java           |   1 +
 .../mysql/MySqlSchemaCreateQuery.java           |  18 +-
 ...RepositoryInsertUpdateDeleteSelectQuery.java |   4 +
 .../postgresql/PostgresqlRepositoryHandler.java |   7 +
 .../postgresql/PostgresqlSchemaCreateQuery.java |  44 ++
 .../PostgresqlSchemaUpgradeQuery.java           |  48 ++
 .../apache/sqoop/server/SqoopJettyServer.java   |  38 +-
 .../sqoop/shell/SetTruststoreFunction.java      |   4 +-
 .../test/minicluster/SqoopMiniCluster.java      |  11 +
 33 files changed, 1657 insertions(+), 199 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/common-test/src/main/java/org/apache/sqoop/common/test/db/DerbyProvider.java
----------------------------------------------------------------------
diff --git a/common-test/src/main/java/org/apache/sqoop/common/test/db/DerbyProvider.java b/common-test/src/main/java/org/apache/sqoop/common/test/db/DerbyProvider.java
index 839e561..b879320 100644
--- a/common-test/src/main/java/org/apache/sqoop/common/test/db/DerbyProvider.java
+++ b/common-test/src/main/java/org/apache/sqoop/common/test/db/DerbyProvider.java
@@ -43,6 +43,10 @@ public class DerbyProvider extends DatabaseProvider {
 
   public static final String DRIVER = "org.apache.derby.jdbc.ClientDriver";
 
+  private static String DERBY_USERNAME = "sqoop2";
+
+  private static String DERBY_PASSWORD = "encryptme";
+
   // Used port for this instance
   int port;
 
@@ -62,7 +66,7 @@ public class DerbyProvider extends DatabaseProvider {
       port = NetworkUtils.findAvailablePort();
       LOG.info("Will bind to port " + port);
 
-      server = new NetworkServerControl(InetAddress.getByName("localhost"), port);
+      server = new NetworkServerControl(InetAddress.getByName("localhost"), port, DERBY_USERNAME, DERBY_PASSWORD);
       server.start(new LoggerWriter(LOG, Level.INFO));
 
       // Start won't thrown an exception in case that it fails to start, one
@@ -160,12 +164,12 @@ public class DerbyProvider extends DatabaseProvider {
 
   @Override
   public String getConnectionUsername() {
-    return null;
+    return DERBY_USERNAME;
   }
 
   @Override
   public String getConnectionPassword() {
-    return null;
+    return DERBY_PASSWORD;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/common/src/main/java/org/apache/sqoop/error/code/CommonRepositoryError.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/error/code/CommonRepositoryError.java b/common/src/main/java/org/apache/sqoop/error/code/CommonRepositoryError.java
index 37eb04a..dd52c82 100644
--- a/common/src/main/java/org/apache/sqoop/error/code/CommonRepositoryError.java
+++ b/common/src/main/java/org/apache/sqoop/error/code/CommonRepositoryError.java
@@ -216,6 +216,8 @@ public enum CommonRepositoryError implements ErrorCode {
   COMMON_0057("Unable to load specific connector"),
 
   COMMON_0058("Resource doesn't exist"),
+
+  COMMON_0059("Unable to retrieve master key"),
   ;
 
   private final String message;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/common/src/main/java/org/apache/sqoop/error/code/PasswordError.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/error/code/PasswordError.java b/common/src/main/java/org/apache/sqoop/error/code/PasswordError.java
new file mode 100644
index 0000000..c4f8e63
--- /dev/null
+++ b/common/src/main/java/org/apache/sqoop/error/code/PasswordError.java
@@ -0,0 +1,44 @@
+/**
+ * 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.sqoop.error.code;
+
+import org.apache.sqoop.common.ErrorCode;
+
+public enum PasswordError implements ErrorCode {
+
+  PASSWORD_0000("Failed to execute password generator"),
+
+  PASSWORD_0001("No password returned from generator"),
+
+  ;
+
+  private final String message;
+
+  private PasswordError(String message) {
+    this.message = message;
+  }
+
+  public String getCode() {
+    return name();
+  }
+
+  public String getMessage() {
+    return message;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/common/src/main/java/org/apache/sqoop/model/MMasterKey.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/MMasterKey.java b/common/src/main/java/org/apache/sqoop/model/MMasterKey.java
new file mode 100644
index 0000000..e87a8f9
--- /dev/null
+++ b/common/src/main/java/org/apache/sqoop/model/MMasterKey.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.sqoop.model;
+
+public class MMasterKey extends MPersistableEntity {
+
+  private String encryptedSecret;
+  private String hmac;
+  private String salt;
+  private String iv;
+
+  public MMasterKey(String encryptedSecret, String hmac, String salt, String iv) {
+    this.encryptedSecret = encryptedSecret;
+    this.hmac = hmac;
+    this.salt = salt;
+    this.iv = iv;
+  }
+
+  @Override
+  public String toString() {
+    return "hmac: " + getHmac() + " salt: " + getSalt() + " IV: " + getIv();
+  }
+
+  public String getEncryptedSecret() {
+    return encryptedSecret;
+  }
+
+  public String getHmac() {
+    return hmac;
+  }
+
+  public String getSalt() {
+    return salt;
+  }
+
+  public String getIv() {
+    return iv;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/common/src/main/java/org/apache/sqoop/utils/PasswordUtils.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/utils/PasswordUtils.java b/common/src/main/java/org/apache/sqoop/utils/PasswordUtils.java
new file mode 100644
index 0000000..f30ed6c
--- /dev/null
+++ b/common/src/main/java/org/apache/sqoop/utils/PasswordUtils.java
@@ -0,0 +1,99 @@
+/**
+ * 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.sqoop.utils;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.log4j.Logger;
+import org.apache.sqoop.common.MapContext;
+import org.apache.sqoop.common.SqoopException;
+import org.apache.sqoop.error.code.PasswordError;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.Charset;
+
+public class PasswordUtils {
+  private static final Logger LOG = Logger.getLogger(PasswordUtils.class);
+
+  /**
+   * Attempts to read a password from the configuration via plaintext or a generator.
+   *
+   * First, we attempt to read the plaintext password, if that value does not exist
+   * we will look for and try to run a configured password generator.
+   *
+   * If a password cannot be found and a password generator cannot be found, an
+   * exception is thrown.
+   *
+   * If both a password and a password generator are set, we prefer the password.
+   *
+   * @param configurationContext MapContext holding the sqoop configuration
+   * @param passwordProperty String containing the property that maps to the
+   *                         plaintext version of the password
+   * @param passwordGeneratorProperty String containing the property that maps to
+   *                                  the generator that prints the password to
+   *                                  standard out
+   * @return A String password value, null if neither password property is set
+   * @throws SqoopException
+   */
+  public static String readPassword(MapContext configurationContext, String passwordProperty, String passwordGeneratorProperty) {
+    String plaintextPassword = configurationContext.getString(passwordProperty);
+    String passwordGenerator = configurationContext.getString(passwordGeneratorProperty);
+
+    if (StringUtils.isNotBlank(plaintextPassword)) {
+      if (StringUtils.isNotBlank(passwordGenerator)) {
+        LOG.warn(passwordProperty + " and " + passwordGeneratorProperty + " are both set, using " + passwordProperty);
+      }
+      return plaintextPassword;
+    } else if (StringUtils.isNotBlank(passwordGenerator)) {
+      try {
+        String password = PasswordUtils.readOutputFromGenerator(passwordGenerator);
+
+        if (StringUtils.isNotBlank(password)) {
+          return password;
+        } else {
+          throw new SqoopException(PasswordError.PASSWORD_0001);
+        }
+      } catch (IOException exception) {
+        throw new SqoopException(PasswordError.PASSWORD_0000, exception);
+      }
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Executes the given command under /bin/sh and reads one line of standard output
+   *
+   * @param generatorCommand String containing the command to execute
+   * @return The first line of standard output from the generator command
+   * @throws IOException
+   */
+  public static String readOutputFromGenerator(String generatorCommand) throws IOException {
+    ProcessBuilder processBuilder = new ProcessBuilder("/bin/sh", "-c", generatorCommand);
+    Process process = processBuilder.start();
+    String output;
+    try (
+      InputStreamReader inputStreamReader = new InputStreamReader(process.getInputStream(), Charset.forName("UTF-8"));
+      BufferedReader bufferedReader = new BufferedReader(inputStreamReader);
+    ) {
+      output =  bufferedReader.readLine();
+    }
+    return output;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/common/src/main/java/org/apache/sqoop/utils/ProcessUtils.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/utils/ProcessUtils.java b/common/src/main/java/org/apache/sqoop/utils/ProcessUtils.java
deleted file mode 100644
index 0a32d68..0000000
--- a/common/src/main/java/org/apache/sqoop/utils/ProcessUtils.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * 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.sqoop.utils;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.nio.charset.Charset;
-
-public class ProcessUtils {
-  public static String readOutputFromGenerator(String generatorCommand) throws IOException {
-    ProcessBuilder processBuilder = new ProcessBuilder("/bin/sh", "-c", generatorCommand);
-    Process process = processBuilder.start();
-    String output;
-    try (
-      InputStreamReader inputStreamReader = new InputStreamReader(process.getInputStream(), Charset.forName("UTF-8"));
-      BufferedReader bufferedReader = new BufferedReader(inputStreamReader);
-    ) {
-      output =  bufferedReader.readLine();
-    } catch(IOException exception) {
-      throw exception;
-    }
-    return output;
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/common/src/test/java/org/apache/sqoop/utils/TestPasswordUtils.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/utils/TestPasswordUtils.java b/common/src/test/java/org/apache/sqoop/utils/TestPasswordUtils.java
new file mode 100644
index 0000000..18a096b
--- /dev/null
+++ b/common/src/test/java/org/apache/sqoop/utils/TestPasswordUtils.java
@@ -0,0 +1,86 @@
+/**
+ * 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.sqoop.utils;
+
+import org.apache.sqoop.common.MapContext;
+import org.apache.sqoop.common.SqoopException;
+import org.testng.annotations.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNull;
+
+public class TestPasswordUtils {
+
+  private static String PASSWORD_KEY = "PASSWORD_KEY";
+  private static String PASSWORD_GENERATOR_KEY = "PASSWORD_GENERATOR_KEY";
+
+  private static String PASSWORD = "password";
+  private static String PASSWORD_FROM_GENERATOR = "password_from_generator";
+  private static String PASSWORD_GENERATOR = "echo " + PASSWORD_FROM_GENERATOR;
+
+  @Test
+  public void passwordExistsAndPasswordGeneratorExists() {
+    Map<String, String> passwordMap = new HashMap<>();
+    passwordMap.put(PASSWORD_KEY, PASSWORD);
+    passwordMap.put(PASSWORD_GENERATOR_KEY, PASSWORD_GENERATOR);
+
+    assertEquals(
+      PasswordUtils.readPassword(new MapContext(passwordMap), PASSWORD_KEY, PASSWORD_GENERATOR_KEY),
+      PASSWORD);
+  }
+
+  @Test
+  public void passwordExistsAndPasswordGeneratorDoesNotExist() {
+    Map<String, String> passwordMap = new HashMap<>();
+    passwordMap.put(PASSWORD_KEY, PASSWORD);
+
+    assertEquals(
+      PasswordUtils.readPassword(new MapContext(passwordMap), PASSWORD_KEY, PASSWORD_GENERATOR_KEY),
+      PASSWORD);
+  }
+
+  @Test
+  public void passwordDoesNotExistAndPasswordGeneratorExists() {
+    Map<String, String> passwordMap = new HashMap<>();
+    passwordMap.put(PASSWORD_GENERATOR_KEY, PASSWORD_GENERATOR);
+
+    assertEquals(
+      PasswordUtils.readPassword(new MapContext(passwordMap), PASSWORD_KEY, PASSWORD_GENERATOR_KEY),
+      PASSWORD_FROM_GENERATOR);
+  }
+
+  @Test
+  public void passwordDoesNotExistAndPasswordGeneratorDoesNotExist() {
+    Map<String, String> passwordMap = new HashMap<>();
+
+    assertNull(PasswordUtils.readPassword(new MapContext(passwordMap), PASSWORD_KEY, PASSWORD_GENERATOR_KEY));
+  }
+
+  @Test(
+    expectedExceptions = {SqoopException.class},
+    expectedExceptionsMessageRegExp = ".*No password returned from generator")
+  public void passwordGeneratorFailsToExecute() {
+    Map<String, String> passwordMap = new HashMap<>();
+    passwordMap.put(PASSWORD_GENERATOR_KEY, "ISUREDOHOPEYOUDONTHAVESOMETHINGWITHTHISNAMEINYOURPATH");
+
+    PasswordUtils.readPassword(new MapContext(passwordMap), PASSWORD_KEY, PASSWORD_GENERATOR_KEY);
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/connector/connector-oracle-jdbc/src/main/resources/oracle-jdbc-connector-config.properties
----------------------------------------------------------------------
diff --git a/connector/connector-oracle-jdbc/src/main/resources/oracle-jdbc-connector-config.properties b/connector/connector-oracle-jdbc/src/main/resources/oracle-jdbc-connector-config.properties
index 1e36ae7..d99ab9b 100644
--- a/connector/connector-oracle-jdbc/src/main/resources/oracle-jdbc-connector-config.properties
+++ b/connector/connector-oracle-jdbc/src/main/resources/oracle-jdbc-connector-config.properties
@@ -13,160 +13,130 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-connector.name = Oracle connector
-
+# Oracle JDBC Connector Resources
+############################
+# Connection Config
+#
 connectionConfig.label = Oracle connection configuration
-connectionConfig.help = Information required to connect to an Oracle server.
+connectionConfig.help = You must supply the information requested in order to \
+                   create an Oracle connection object.
 
-connectionConfig.connectionString.label = Connection string
-connectionConfig.connectionString.example = jdbc:oracle:thin:@oracle.sqoop.org:1521:sqoop_db
-connectionConfig.connectionString.help = JDBC connection string associated with your Oracle database.
+# connect string
+connectionConfig.connectionString.label = JDBC connection string
+connectionConfig.connectionString.help = Enter the value of JDBC connection string to be \
+                   used by this connector for creating Oracle connections.
 
+# username string
 connectionConfig.username.label = Username
-connectionConfig.username.example = sqoop-user
-connectionConfig.username.help = Username to be used for connection to the Oracle server.
+connectionConfig.username.help = Enter the username to be used for connecting to the \
+                   database.
 
+# password string
 connectionConfig.password.label = Password
-connectionConfig.password.label = Sup3rS3rcr3t!
-connectionConfig.password.help = Password to be used for connection to the Oracle server.
+connectionConfig.password.help = Enter the password to be used for connecting to the \
+                   database.
 
-connectionConfig.jdbcProperties.label = Connection properties
-connectionConfig.jdbcProperties.example = defaultRowPrefetch=1000
-connectionConfig.jdbcProperties.help = Key-value pairs that should be passed down to JDBC driver when establishing connection.
+# jdbc properties
+connectionConfig.jdbcProperties.label = JDBC connection properties
+connectionConfig.jdbcProperties.help = Enter any JDBC properties that should be \
+                   supplied during the creation of connection.
 
-connectionConfig.timeZone.label = Time zone
-connectionConfig.timeZone.example = GMT
-connectionConfig.timeZone.help = Time zone that will be used for all created sessions.
+connectionConfig.timeZone.label = Session time zone
+connectionConfig.timeZone.help = timeZone
 
 connectionConfig.actionName.label = Session action name
-connectionConfig.actionName.example = sqoop-import
-connectionConfig.actionName.help = The connector will call DMS_APPLICATION_INFO.SET_MODULE procedure with action_name set to this value.
+connectionConfig.actionName.help = actionName
 
-connectionConfig.fetchSize.label = fetch size
-connectionConfig.fetchSize.example =  1000
-connectionConfig.fetchSize.help = Optional hint specifying requested JDBC fetch size.
+connectionConfig.fetchSize.label = JDBC fetch size
+connectionConfig.fetchSize.help = fetchSize
 
-connectionConfig.initializationStatements.label = Initialization statements
-connectionConfig.initializationStatements.example = alter session disable parallel query
-connectionConfig.initializationStatements.help = List of statements that will be executed on each connection immediately after opening and before any metadata/data retrieving queries.
+connectionConfig.initializationStatements.label = Session initialization statements
+connectionConfig.initializationStatements.help = initializationStatements
 
-connectionConfig.jdbcUrlVerbatim.label = Strict connection string
-connectionConfig.jdbcUrlVerbatim.example = false
-connectionConfig.jdbcUrlVerbatim.help = By default, OraOop will use specified connection string only to retrieve all instances of your RAC and \
-  then for actual data transfer jobs will generate connection strings to distribute the load across all nodes. This distribution can be turned \
-  off by setting this property to false.
+connectionConfig.jdbcUrlVerbatim.label = Use JDBC connection string verbatim
+connectionConfig.jdbcUrlVerbatim.help = jdbcUrlVerbatim
 
 connectionConfig.racServiceName.label = RAC service name
-connectionConfig.racServiceName.example = sales.sqoop.org
-connectionConfig.racServiceName.help = Service name that will be used for automatically generated connection strings when connecting to RAC.
-
+connectionConfig.racServiceName.help = racServiceName
 
-toJobConfig.label = Database target
-toJobConfig.help = Describes target destination and way how data should be persisted on the RDBMS system.
+# ToJob Config
+#
+toJobConfig.label = To database configuration
+toJobConfig.help = You must supply the information requested in order to create \
+                 the TO part of the job object.
 
+# To table name
 toJobConfig.tableName.label = Table name
-toJobConfig.tableName.example = target_table
-toJobConfig.tableName.help = Destination table name to store transfer results.
+toJobConfig.tableName.help = Table name to write data into
 
 toJobConfig.columns.label = Columns
-toJobConfig.columns.example = id,text,city
-toJobConfig.columns.help = Subset of columns that will will be written to. Omitted columns have to either allow \
-  NULL values or have defined default value.
+toJobConfig.columns.help = Columns
 
 toJobConfig.templateTable.label = Template table name
-toJobConfig.templateTable.example = existing_table
-toJobConfig.templateTable.help = If this field is not empty, then target table will get created with the same structure as the \
-  template table.
+toJobConfig.templateTable.help = templateTable
 
 toJobConfig.partitioned.label = Partitioned
-toJobConfig.partitioned.example = true
-toJobConfig.partitioned.help = If creating a new target table with structure from template table, this option controls whether \
-  the new table should be partitioned or not.
+toJobConfig.partitioned.help = partitioned
 
 toJobConfig.nologging.label = Nologging
-toJobConfig.nologging.example = Nologging
-toJobConfig.nologging.help = If creating a new target table with structure from template table, when set to true, this option will \
-   add NOLOGGING clause to the CREATE TABLE statement.
+toJobConfig.nologging.help = nologging
 
-toJobConfig.updateKey.label = Update columns
-toJobConfig.updateKey.example = id,date
-toJobConfig.updateKey.help = Specifying this option will switch OraOop to update mode. Instead of generating INSERT statements to \
-  insert data to Oracle, it will generate UPDATE statements. Configured options will then be used in WHERE clause.
+toJobConfig.updateKey.label = Update key columns
+toJobConfig.updateKey.help = updateKey
 
 toJobConfig.updateMerge.label = Merge updates
-toJobConfig.updateMerge.example = true
-toJobConfig.updateMerge.help = Instead of generating only UPDATE statements, OraOop will generate MERGE statement that will do so \
-  called upsert - will either update existing rows or insert new rows.
+toJobConfig.updateMerge.help = updateMerge
 
-toJobConfig.dropTableIfExists.label = Drop table
-toJobConfig.dropTableIfExists.example = true
-toJobConfig.dropTableIfExists.help = If set to true then, then OraOop will drop existing target table and re-create it using the \
-  template table structure.
+toJobConfig.dropTableIfExists.label = Drop table if exists
+toJobConfig.dropTableIfExists.help = dropTableIfExists
 
-toJobConfig.storageClause.label = Template storage clause
-toJobConfig.storageClause.example = STORAGE (INITIAL 100K NEXT 50K MINEXTENTS 1 MAXEXTENTS 50 PCTINCREASE 5)
-toJobConfig.storageClause.help = If target table will get created, insert specified storage clause to the CREATE TABLE statement.
+toJobConfig.storageClause.label = Template table storage clause
+toJobConfig.storageClause.help = storageClause
 
-toJobConfig.temporaryStorageClause.label = Temporary storage clause
-toJobConfig.temporaryStorageClause.example = STORAGE (INITIAL 100K NEXT 50K MINEXTENTS 1 MAXEXTENTS 50 PCTINCREASE 5)
-toJobConfig.temporaryStorageClause.help = Storage clause that will be used for all temporary tables that OraOop might need to create.
+toJobConfig.temporaryStorageClause.label = Temporary table storage clause
+toJobConfig.temporaryStorageClause.help = temporaryStorageClause
 
-toJobConfig.appendValuesHint.label = Append values hint
-toJobConfig.appendValuesHint.example = AUTO
-toJobConfig.appendValuesHint.help = Specifies whether generated INSERT queries should take advantage of Oracle's APPEND hint.
+toJobConfig.appendValuesHint.label = Append values hint usage
+toJobConfig.appendValuesHint.help = appendValuesHint
 
 toJobConfig.parallel.label = Parallel
-toJobConfig.parallel.example = true
-toJobConfig.parallel.help = When moving data from temporary tables, this option controls whether generated queries shoud include \
-  PARALLEL keyword.
-
+toJobConfig.parallel.help = parallel
 
-fromJobConfig.label = Database source
-fromJobConfig.help = Specifies source and way how the data should be fetched from source database.
+# FromJob Config
+#
+fromJobConfig.label = From Oracle configuration
+fromJobConfig.help = You must supply the information requested in order to create \
+                 the FROM part of the job object.
 
 fromJobConfig.tableName.label = Table name
-fromJobConfig.tableName.example = input_table
-fromJobConfig.tableName.help = Input table name from from which data will be retrieved.
+fromJobConfig.tableName.help = tableName
 
 fromJobConfig.columns.label = Columns
-fromJobConfig.columns.example = id,text,city
-fromJobConfig.columns.help = Subset of columns that should be retrieved from source table.
+fromJobConfig.columns.help = Columns
 
 fromJobConfig.consistentRead.label = Consistent read
-fromJobConfig.consistentRead.example = true
-fromJobConfig.consistentRead.help = If true, connector will use Oracle Flashback technology to achieve consistency across \
-  independent connections.
+fromJobConfig.consistentRead.help = consistentRead
 
 fromJobConfig.consistentReadScn.label = Consistent read SCN
-fromJobConfig.consistentReadScn.example = 10093466
-fromJobConfig.consistentReadScn.help = Optional SCN value that should be used to read the consistent point in time. Connector \
-  will use latest value during initialization if not specified.
+fromJobConfig.consistentReadScn.help = consistentReadScn
 
 fromJobConfig.partitionList.label = Partitions
-fromJobConfig.partitionList.example = PART1,PART2
-fromJobConfig.partitionList.help = Subset of partitions that should be retrieved from source table.
+fromJobConfig.partitionList.help = partitionList
 
 fromJobConfig.dataChunkMethod.label = Data chunk method
-fromJobConfig.dataChunkMethod.example = ROWID
-fromJobConfig.dataChunkMethod.help = Method that will be used to slice the data in source table to transfer them in parallel.
+fromJobConfig.dataChunkMethod.help = dataChunkMethod
 
 fromJobConfig.dataChunkAllocationMethod.label = Data chunk allocation method
-fromJobConfig.dataChunkAllocationMethod.example = ROUNDROBIN
-fromJobConfig.dataChunkAllocationMethod.help = Specific way in which various data chunks gets distributed to partitions.
+fromJobConfig.dataChunkAllocationMethod.help = dataChunkAllocationMethod
 
-fromJobConfig.whereClauseLocation.label = Where clause
-fromJobConfig.whereClauseLocation.example = SUBSPLIT
-fromJobConfig.whereClauseLocation.help = Determines whether the where clause should be used inside each data chunk separately \
-   or for union of all chunks that are selected in single partition.
+fromJobConfig.whereClauseLocation.label = Where clause location
+fromJobConfig.whereClauseLocation.help = whereClauseLocation
 
 fromJobConfig.omitLobColumns.label = Omit LOB columns
-fromJobConfig.omitLobColumns.example = true
-fromJobConfig.omitLobColumns.help = If set to true, then OraOop will ignore importing all LOB based columns (CLOB, BLOB).
+fromJobConfig.omitLobColumns.help = omitLobColumns
 
 fromJobConfig.queryHint.label = Query hint
-fromJobConfig.queryHint.example = ALL_ROWS
-fromJobConfig.queryHint.help = Hints that should be used for queries that are transferring data.
+fromJobConfig.queryHint.help = queryHint
 
 fromJobConfig.conditions.label = Conditions
-fromJobConfig.conditions.example = id > 100
-fromJobConfig.conditions.help = Additional conditions that should be used inside WHERE clause to limit imported data.
\ No newline at end of file
+fromJobConfig.conditions.help = conditions
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/core/src/main/java/org/apache/sqoop/core/SqoopServer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/sqoop/core/SqoopServer.java b/core/src/main/java/org/apache/sqoop/core/SqoopServer.java
index 80a7b88..0c983a0 100644
--- a/core/src/main/java/org/apache/sqoop/core/SqoopServer.java
+++ b/core/src/main/java/org/apache/sqoop/core/SqoopServer.java
@@ -22,6 +22,7 @@ import org.apache.sqoop.audit.AuditLoggerManager;
 import org.apache.sqoop.connector.ConnectorManager;
 import org.apache.sqoop.driver.Driver;
 import org.apache.sqoop.driver.JobManager;
+import org.apache.sqoop.repository.MasterKeyManager;
 import org.apache.sqoop.repository.RepositoryManager;
 import org.apache.sqoop.security.AuthenticationManager;
 import org.apache.sqoop.security.AuthorizationManager;
@@ -39,6 +40,7 @@ public class SqoopServer {
     JobManager.getInstance().destroy();
     Driver.getInstance().destroy();
     ConnectorManager.getInstance().destroy();
+    MasterKeyManager.getInstance().destroy();
     RepositoryManager.getInstance().destroy();
     AuditLoggerManager.getInstance().destroy();
     AuthorizationManager.getInstance().destroy();
@@ -56,6 +58,7 @@ public class SqoopServer {
       AuthorizationManager.getInstance().initialize();
       AuditLoggerManager.getInstance().initialize();
       RepositoryManager.getInstance().initialize();
+      MasterKeyManager.getInstance().initialize();
       ConnectorManager.getInstance().initialize();
       Driver.getInstance().initialize();
       JobManager.getInstance().initialize();

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/core/src/main/java/org/apache/sqoop/repository/JdbcRepository.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/sqoop/repository/JdbcRepository.java b/core/src/main/java/org/apache/sqoop/repository/JdbcRepository.java
index 5b70f95..44ab41b 100644
--- a/core/src/main/java/org/apache/sqoop/repository/JdbcRepository.java
+++ b/core/src/main/java/org/apache/sqoop/repository/JdbcRepository.java
@@ -27,6 +27,7 @@ import org.apache.sqoop.model.MConnector;
 import org.apache.sqoop.model.MDriver;
 import org.apache.sqoop.model.MJob;
 import org.apache.sqoop.model.MLink;
+import org.apache.sqoop.model.MMasterKey;
 import org.apache.sqoop.model.MSubmission;
 
 public class JdbcRepository extends Repository {
@@ -657,6 +658,36 @@ public class JdbcRepository extends Repository {
     });
   }
 
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public MMasterKey getMasterKey() {
+    return (MMasterKey) doWithConnection(new DoWithConnection() {
+      @Override
+      public Object doIt(Connection conn) throws Exception {
+        return handler.getMasterKey(conn);
+      }
+    });
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void createMasterKey(final MMasterKey mMasterKey) {
+    doWithConnection(new DoWithConnection() {
+      @Override
+      public Object doIt(Connection conn) {
+        if(mMasterKey.hasPersistenceId()) {
+          throw new SqoopException(RepositoryError.JDBCREPO_0023);
+        }
+        handler.createMasterKey(mMasterKey, conn);
+        return null;
+      }
+    });
+  }
+
   @Override
   protected void deleteJobInputs(final String jobName, RepositoryTransaction tx) {
     doWithConnection(new DoWithConnection() {

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryHandler.java b/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryHandler.java
index feab2ad..94f5e6f 100644
--- a/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryHandler.java
+++ b/core/src/main/java/org/apache/sqoop/repository/JdbcRepositoryHandler.java
@@ -25,6 +25,7 @@ import org.apache.sqoop.model.MConnector;
 import org.apache.sqoop.model.MDriver;
 import org.apache.sqoop.model.MJob;
 import org.apache.sqoop.model.MLink;
+import org.apache.sqoop.model.MMasterKey;
 import org.apache.sqoop.model.MSubmission;
 
 /**
@@ -465,4 +466,20 @@ public abstract class JdbcRepositoryHandler {
    */
   public abstract MSubmission findLastSubmissionForJob(String jobName, Connection conn);
 
+  /**
+   * Read the master key from the database
+   *
+   * @param conn Connection to the repository
+   * @return MMasterKey representing the Master Key
+   */
+  public abstract MMasterKey getMasterKey(Connection conn);
+
+  /**
+   * Create the master key in the database
+   *
+   * @param mMasterKey MMasterKey representing the Master Key
+   * @param conn Connection to the repository
+   */
+  public abstract void createMasterKey(MMasterKey mMasterKey, Connection conn);
+
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/core/src/main/java/org/apache/sqoop/repository/MasterKeyManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/sqoop/repository/MasterKeyManager.java b/core/src/main/java/org/apache/sqoop/repository/MasterKeyManager.java
new file mode 100644
index 0000000..df84d54
--- /dev/null
+++ b/core/src/main/java/org/apache/sqoop/repository/MasterKeyManager.java
@@ -0,0 +1,479 @@
+/**
+ * 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.sqoop.repository;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.sqoop.common.MapContext;
+import org.apache.sqoop.common.SqoopException;
+import org.apache.sqoop.core.SqoopConfiguration;
+import org.apache.sqoop.model.MMasterKey;
+import org.apache.sqoop.security.SecurityConstants;
+import org.apache.sqoop.security.SecurityError;
+import org.apache.sqoop.utils.PasswordUtils;
+import org.apache.commons.codec.binary.Base64;
+
+import javax.crypto.Cipher;
+import javax.crypto.Mac;
+import javax.crypto.SecretKey;
+import javax.crypto.SecretKeyFactory;
+import javax.crypto.spec.IvParameterSpec;
+import javax.crypto.spec.PBEKeySpec;
+import javax.crypto.spec.SecretKeySpec;
+import java.nio.charset.Charset;
+import java.security.GeneralSecurityException;
+import java.security.NoSuchAlgorithmException;
+import java.security.SecureRandom;
+import java.util.Arrays;
+import java.util.Random;
+
+public class MasterKeyManager {
+
+  private String hmacAlgorithm;
+  private int hmacKeySizeBytes;
+  private String cipherAlgorithm;
+  private int cipherKeySize;
+  private String cipherSpec;
+  private String pbkdf2Algorithm;
+  private int pbkdf2Rounds;
+  private int ivLength;
+
+  private SecretKey masterEncryptionKey;
+  private SecretKey masterHmacKey;
+
+  private Random random;
+
+  private static MasterKeyManager instance;
+
+  static {
+    instance = new MasterKeyManager();
+  }
+
+  private MasterKeyManager() {
+  }
+
+  public static MasterKeyManager getInstance() {
+    return instance;
+  }
+
+  public static void setInstance(MasterKeyManager newInstance) {
+    instance = newInstance;
+  }
+
+  public void initialize() throws SqoopException {
+    initialize(true);
+  }
+
+  public synchronized void initialize(boolean createMasterKey) throws SqoopException {
+    // This is used for the generation of random initialization vectors and salts
+    random = new SecureRandom();
+
+    MapContext configurationContext = SqoopConfiguration.getInstance().getContext();
+    if (configurationContext.getBoolean(SecurityConstants.REPO_ENCRYPTION_ENABLED, false)) {
+
+      // Grab configuration from the sqoop properties file. All of this configuration is required
+      // and an exception will be thrown if any of it is missing
+      hmacAlgorithm = populateStringConfiguration(configurationContext, SecurityConstants.REPO_ENCRYPTION_HMAC_ALGORITHM);
+      cipherAlgorithm = populateStringConfiguration(configurationContext, SecurityConstants.REPO_ENCRYPTION_CIPHER_ALGORITHM);
+      cipherSpec = populateStringConfiguration(configurationContext, SecurityConstants.REPO_ENCRYPTION_CIPHER_SPEC);
+      cipherKeySize = populateIntConfiguration(configurationContext, SecurityConstants.REPO_ENCRYPTION_CIPHER_KEY_SIZE);
+      ivLength = populateIntConfiguration(configurationContext, SecurityConstants.REPO_ENCRYPTION_INITIALIZATION_VECTOR_SIZE);
+      pbkdf2Algorithm = populateStringConfiguration(configurationContext, SecurityConstants.REPO_ENCRYPTION_PBKDF2_ALGORITHM);
+      pbkdf2Rounds = populateIntConfiguration(configurationContext, SecurityConstants.REPO_ENCRYPTION_PBKDF2_ROUNDS);
+
+      // The size of the hmac key can be derived from the provided HMAC algorithm
+      try {
+        hmacKeySizeBytes = Mac.getInstance(hmacAlgorithm).getMacLength();
+      } catch (NoSuchAlgorithmException e) {
+        throw new SqoopException(SecurityError.ENCRYPTION_0011, e);
+      }
+
+      Repository repository = RepositoryManager.getInstance().getRepository();
+      String password = PasswordUtils.readPassword(configurationContext, SecurityConstants.REPO_ENCRYPTION_PASSWORD,
+        SecurityConstants.REPO_ENCRYPTION_PASSWORD_GENERATOR);
+      if (StringUtils.isEmpty(password)) {
+        throw new SqoopException(SecurityError.ENCRYPTION_0008);
+      }
+
+      MMasterKey existingEncryptedMasterKey = repository.getMasterKey();
+      String salt;
+
+      if (existingEncryptedMasterKey == null) {
+        // Since the master key does not exist, we can generate a random salt that we will use
+        // for encryption of the Master Key
+        // We will use a salt that is the same size as the encryption key
+        salt = Base64.encodeBase64String(generateRandomByteArray(hmacKeySizeBytes));
+      } else {
+        // Since the master key already exists, we will read the salt from the repository
+        salt = existingEncryptedMasterKey.getSalt();
+      }
+
+      // Derive two keys (that we will be used to encrypt and verify the master key)
+      // from the configuration provided password and the salt we just read/created.
+      byte[] keyBytes = getKeysFromPassword(password, salt);
+      SecretKey passwordEncryptionKey = new SecretKeySpec(keyBytes, 0,
+        cipherKeySize, cipherAlgorithm);
+      SecretKey passwordHmacKey = new SecretKeySpec(keyBytes,
+        cipherKeySize, hmacKeySizeBytes, hmacAlgorithm);
+
+      byte[] masterEncryptionKeyBytes;
+      byte[] masterHmacKeyBytes;
+      if (existingEncryptedMasterKey == null) {
+        if (createMasterKey) {
+          // A master key does not exist so we must create one. We will simply
+          // use two random byte arrays for the encryption and hmac components.
+          // The sizes of these keys is determined by the values provided to
+          // configuration.
+          masterEncryptionKeyBytes = generateRandomByteArray(cipherKeySize);
+          masterHmacKeyBytes = generateRandomByteArray(hmacKeySizeBytes);
+
+          // The initialization vector for the encryption of the master key is
+          // randomly generated.
+          String iv = Base64.encodeBase64String(generateRandomByteArray(ivLength));
+
+          // We append our two keys together and encrypt the resulting byte array.
+          // This is the secret that all of the encryption in the repository depends upon
+          byte[] secret = ArrayUtils.addAll(masterEncryptionKeyBytes, masterHmacKeyBytes);
+          String encryptedSecret = encryptToString(passwordEncryptionKey, secret, iv);
+
+          // We store our new master key in the repository in its encrypted form
+          // along with an HMAC to verify the key when we read it, the salt needed to
+          // generate keys to decrypt it, and the initialization vector used
+          repository.createMasterKey(new MMasterKey(encryptedSecret, generateHmac(passwordHmacKey,
+            encryptedSecret), salt, iv));
+        } else {
+          // If a master key does not exist and we are trying to initialize the
+          // manager without allowing it to create a master key, we should fail
+          throw new SqoopException(SecurityError.ENCRYPTION_0002);
+        }
+      } else {
+        // A master key exists so we need to read it from the repository and
+        // decrypt it.
+        String iv = existingEncryptedMasterKey.getIv();
+        String encryptedSecret = existingEncryptedMasterKey.getEncryptedSecret();
+
+        // Before we go about decrypting the master key we should verify the hmac
+        // to ensure that it has not been tampered with
+        String hmac = existingEncryptedMasterKey.getHmac();
+        if (!validHmac(passwordHmacKey, encryptedSecret, hmac)) {
+          throw new SqoopException(SecurityError.ENCRYPTION_0001);
+        }
+
+        // The master key has not been tampered with, lets decrypt it using the key
+        // derived from the password and the initialization vector from the repository
+        byte[] decryptedKey = decryptToBytes(passwordEncryptionKey, encryptedSecret, iv);
+
+        // Since the master key is stored as the concatenation of an encryption
+        // key and an hmac key, we need to split it according to the sizes derived
+        // from the configuration
+        masterEncryptionKeyBytes = new byte[cipherKeySize];
+        masterHmacKeyBytes = new byte[hmacKeySizeBytes];
+        System.arraycopy(decryptedKey, 0, masterEncryptionKeyBytes, 0,
+          cipherKeySize);
+        System.arraycopy(decryptedKey, cipherKeySize,
+          masterHmacKeyBytes, 0, hmacKeySizeBytes);
+      }
+
+      // Place the master encryption and master hmac key in SecretKey objects
+      // so we can use them to encrypt and decrypt data
+      masterEncryptionKey = new SecretKeySpec(masterEncryptionKeyBytes, 0, cipherKeySize, cipherAlgorithm);
+      masterHmacKey = new SecretKeySpec(masterHmacKeyBytes, 0, hmacKeySizeBytes, hmacAlgorithm);
+    }
+  }
+
+  public synchronized void destroy() {
+    hmacAlgorithm = null;
+    cipherAlgorithm = null;
+    cipherSpec = null;
+    pbkdf2Algorithm = null;
+
+    masterEncryptionKey = null;
+    masterHmacKey = null;
+
+    random = null;
+  }
+
+  /**
+   * Returns a Base64 representation of the encrypted cleartext, using the Master Key
+   *
+   * @param clearText Text to encrypt
+   * @param iv Initialization vector for the cipher
+   * @return Base64 representation of the encrypted cleartext
+   * @throws SqoopException
+   */
+  public String encryptWithMasterKey(String clearText, String iv) throws SqoopException {
+    return encryptToString(masterEncryptionKey, clearText, iv);
+  }
+
+  /**
+   * Validates the HMAC against the cipher text and then returns a UTF-8 string
+   * of the data decrypted using the Master Key
+   *
+   * Throws an exception of the HMAC is incorrect
+   *
+   * @param cipherText Base64 representation of the encrypted text
+   * @param iv Initialization vector for the cipher
+   * @param hmac HMAC for tamper resistance
+   * @return UTF-8 string from the decrypted data
+   * @throws SqoopException
+   */
+  public String decryptWithMasterKey(String cipherText, String iv, String hmac) throws SqoopException {
+    if (!validWithMasterHmacKey(cipherText, hmac)) {
+      throw new SqoopException(SecurityError.ENCRYPTION_0010);
+    }
+    return decryptWithMasterKey(cipherText, iv);
+  }
+
+  /**
+   * Generates the hmac for the provided cipher text using the Master Key
+   *
+   * @param cipherText Base64 representation of the encrypted text
+   * @return Base64 representation of the HMAC
+   * @throws SqoopException
+   */
+  public String generateHmacWithMasterHmacKey(String cipherText) throws SqoopException {
+    return generateHmac(masterHmacKey, cipherText);
+  }
+
+  /**
+   * Generates a random initialization vector of the expected size
+   * @return Base64 representation of the initialization vector
+   */
+  public String generateRandomIv() {
+    return Base64.encodeBase64String(generateRandomByteArray(ivLength));
+  }
+
+  /**
+   * Determines if the provided HMAC matches the HMAC generated for the cipher text
+   * using the Master Key
+   *
+   * @param cipherText Base64 representation of the encrypted data
+   * @param expectedHmac Provided HMAC to compare against
+   * @return True if expectedHmac matches what is generated by the Master Key, false otherwise
+   * @throws SqoopException
+   */
+  private boolean validWithMasterHmacKey(String cipherText, String expectedHmac) throws SqoopException {
+    return validHmac(masterHmacKey, cipherText, expectedHmac);
+  }
+
+  /**
+   * Decrypts the provided ciphertext with the IV provided and the Master Key
+   *
+   * @param cipherText Base64 representation of the encrypted data
+   * @param iv Initialization vector for use by the cipher
+   * @return UTF-8 representation of the decrypted data
+   * @throws SqoopException
+   */
+  private String decryptWithMasterKey(String cipherText, String iv) throws SqoopException {
+    return decryptToString(masterEncryptionKey, cipherText, iv);
+  }
+
+  /**
+   * Encrypts the provided cleartext with the provided IV and SecretKey
+   *
+   * @param secretKey Key to use for the encryption
+   * @param clearText String that is to be encrypted
+   * @param iv Initialization vector for use by the cipher,
+   * @return byte array representing the encrypted data
+   * @throws SqoopException
+   */
+  private byte[] encryptToBytes(SecretKey secretKey, String clearText, String iv) throws SqoopException {
+    return encryptToBytes(secretKey, clearText.getBytes(Charset.forName("UTF-8")), iv);
+  }
+
+  /**
+   * Encrypts the provided cleartext with the provided IV and SecretKey
+   *
+   * @param secretKey Key to use for the encryption
+   * @param clearText Byte array that is to be encrypted
+   * @param iv Initialization vector for use by the cipher,
+   * @return byte array representing the encrypted data
+   * @throws SqoopException
+   */
+  private byte[] encryptToBytes(SecretKey secretKey, byte[] clearText, String iv) throws SqoopException {
+    try {
+      Cipher cipher = Cipher.getInstance(cipherSpec);
+      cipher.init(Cipher.ENCRYPT_MODE, secretKey, new IvParameterSpec(Base64.decodeBase64(iv)));
+      return cipher.doFinal(clearText);
+    } catch (GeneralSecurityException exception) {
+      throw new SqoopException(SecurityError.ENCRYPTION_0004, exception);
+    }
+  }
+
+  /**
+   * Encrypts the provided cleartext with the provided IV and SecretKey
+   *
+   * @param secretKey Key to use for the encryption
+   * @param clearText Byte array that is to be encrypted
+   * @param iv Initialization vector for use by the cipher,
+   * @return Base64 representation of the encrypted data
+   * @throws SqoopException
+   */
+  private String encryptToString(SecretKey secretKey, byte[] clearText, String iv) throws SqoopException {
+    return Base64.encodeBase64String(encryptToBytes(secretKey, clearText, iv));
+  }
+
+  /**
+   * Encrypts the provided cleartext with the provided IV and SecretKey
+   *
+   * @param secretKey Key to use for the encryption
+   * @param clearText String that is to be encrypted
+   * @param iv Initialization vector for use by the cipher,
+   * @return Base64 representation of the encrypted data
+   * @throws SqoopException
+   */
+  private String encryptToString(SecretKey secretKey, String clearText, String iv) throws SqoopException {
+    return Base64.encodeBase64String(encryptToBytes(secretKey, clearText, iv));
+  }
+
+  /**
+   * Validates an HMAC against some cipherText using the provided hmacKey
+   *
+   * @param hmacKey SecretKey which defines the HMAC key and the HMAC algorithm
+   * @param cipherText Encrypted text from which the HMAC is generated
+   * @param expectedHmac The expected value for the HMAC that we will be comparing against
+   * @return True if the generated HMAC matches the expectedHmac, false otherwise
+   * @throws SqoopException
+   */
+  private boolean validHmac(SecretKey hmacKey, String cipherText, String expectedHmac) throws SqoopException {
+    try {
+      Mac hmac = Mac.getInstance(hmacAlgorithm);
+      hmac.init(hmacKey);
+      byte[] calculatedHmac = hmac.doFinal(Base64.decodeBase64(cipherText));
+      return Arrays.equals(calculatedHmac, Base64.decodeBase64(expectedHmac));
+    } catch (GeneralSecurityException exception) {
+      throw new SqoopException(SecurityError.ENCRYPTION_0005, exception);
+    }
+  }
+
+  /**
+   * Generates an HMAC based on a SecretKey and some cipherText
+   *
+   * @param hmacKey SecretKey which defines the HMAC key and the HMAC algorithm
+   * @param cipherText Encrypted text from which the HMAC is generated
+   * @return Base64 representation of the HMAC value
+   * @throws SqoopException
+   */
+  private String generateHmac(SecretKey hmacKey, String cipherText) throws SqoopException {
+    try {
+      Mac hmac = Mac.getInstance(hmacAlgorithm);
+      hmac.init(hmacKey);
+      return Base64.encodeBase64String(hmac.doFinal(Base64.decodeBase64(cipherText)));
+    } catch (GeneralSecurityException exception) {
+      throw new SqoopException(SecurityError.ENCRYPTION_0005, exception);
+    }
+  }
+
+  /**
+   * Decrypts the provided cipherText using the provided encryptionKey and iv
+   *
+   * @param encryptionKey SecretKey which defines the encryption key and algorithm
+   * @param cipherText Base64 representation of the data we want to decrypt
+   * @param iv Base64 representation of the initialization vector used when the data was encrypted
+   * @return Byte array representing the decrypted data
+   * @throws SqoopException
+   */
+  private byte[] decryptToBytes(SecretKey encryptionKey, String cipherText, String iv) throws SqoopException {
+    try {
+      Cipher cipher = Cipher.getInstance(cipherSpec);
+      cipher.init(Cipher.DECRYPT_MODE, encryptionKey, new IvParameterSpec(Base64.decodeBase64(iv)));
+      return cipher.doFinal(Base64.decodeBase64(cipherText));
+    } catch (GeneralSecurityException exception) {
+      throw new SqoopException(SecurityError.ENCRYPTION_0006, exception);
+    }
+  }
+
+  /**
+   * Decrypts the provided cipherText using the provided encryptionKey and iv
+   *
+   * @param encryptionKey SecretKey which defines the encryption key and algorithm
+   * @param cipherText Base64 representation of the data we want to decrypt
+   * @param iv Base64 representation of the initialization vector used when the data was encrypted
+   * @return String representing the decrypted data using the UTF-8 character set
+   * @throws SqoopException
+   */
+  private String decryptToString(SecretKey encryptionKey, String cipherText, String iv) throws SqoopException {
+    return new String(decryptToBytes(encryptionKey, cipherText, iv), Charset.forName("UTF-8"));
+  }
+
+  /**
+   * Reads the specified String configuration value from the provided configurationContext,
+   * throws an exception if the value cannot be found
+   *
+   * @param configurationContext MapContext containing the sqoop configuration
+   * @param configuration Configuration value that we would like from the configurationContext
+   * @return String value from the configuration
+   * @throws SqoopException
+   */
+  private String populateStringConfiguration(MapContext configurationContext, String configuration) throws SqoopException {
+    String value = configurationContext.getString(configuration);
+    if (StringUtils.isEmpty(value)){
+      throw new SqoopException(SecurityError.ENCRYPTION_0009, configuration);
+    }
+    return value;
+  }
+
+  /**
+   * Reads the specified integer configuration value from the provided configurationContext,
+   * throws an exception if the value cannot be found
+   *
+   * @param configurationContext MapContext containing the sqoop configuration
+   * @param configuration Configuration value that we would like from the configurationContext
+   * @return int value from the configuration
+   * @throws SqoopException
+   */
+  private int populateIntConfiguration(MapContext configurationContext, String configuration) throws SqoopException {
+    int value = configurationContext.getInt(configuration, 0);
+    if (value < 1){
+      throw new SqoopException(SecurityError.ENCRYPTION_0009, configuration);
+    }
+    return value;
+  }
+
+  /**
+   * Generates a random byte array of the specified length
+   *
+   * @param size number of random bytes to return
+   * @return byte array containing random bytes of the specified size
+   */
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings("IS2_INCONSISTENT_SYNC")
+  private byte[] generateRandomByteArray(int size) {
+    byte[] randomBytes = new byte[size];
+    random.nextBytes(randomBytes);
+    return randomBytes;
+  }
+
+  /**
+   * Using the PBKDF2 algorithm, we will generate a Master Key.
+   *
+   * @param password The password that will be used for the generation of encryption keys
+   * @param salt Salt to be used for the generation of encryption keys
+   * @return byte[] containing the key generated by PDKDF2
+   * @throws SqoopException
+   */
+  private byte[] getKeysFromPassword(String password, String salt) throws SqoopException {
+    try {
+      PBEKeySpec spec = new PBEKeySpec(password.toCharArray(), Base64.decodeBase64(salt),
+        pbkdf2Rounds, (cipherKeySize + hmacKeySizeBytes) * 8);
+      SecretKeyFactory secretKeyFactory = SecretKeyFactory.getInstance(pbkdf2Algorithm);
+      return secretKeyFactory.generateSecret(spec).getEncoded();
+    } catch (GeneralSecurityException exception) {
+      throw new SqoopException(SecurityError.ENCRYPTION_0003, exception);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/core/src/main/java/org/apache/sqoop/repository/Repository.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/sqoop/repository/Repository.java b/core/src/main/java/org/apache/sqoop/repository/Repository.java
index 03989a3..c2e3c74 100644
--- a/core/src/main/java/org/apache/sqoop/repository/Repository.java
+++ b/core/src/main/java/org/apache/sqoop/repository/Repository.java
@@ -42,6 +42,7 @@ import org.apache.sqoop.model.MFromConfig;
 import org.apache.sqoop.model.MJob;
 import org.apache.sqoop.model.MLink;
 import org.apache.sqoop.model.MLinkConfig;
+import org.apache.sqoop.model.MMasterKey;
 import org.apache.sqoop.model.MPersistableEntity;
 import org.apache.sqoop.model.MSubmission;
 import org.apache.sqoop.model.MToConfig;
@@ -329,6 +330,21 @@ public abstract class Repository {
    */
   public abstract MSubmission findLastSubmissionForJob(String jobName);
 
+  /**
+   * Get the encrypted master key from the repository
+   *
+   * @return The encrypted master key, null if no master key exists
+   */
+  public abstract MMasterKey getMasterKey();
+
+
+  /**
+   * Create the encrypted master key in the repository
+   *
+   * @param mMasterKey The encrypted master key
+   */
+  public abstract void createMasterKey(MMasterKey mMasterKey);
+
 
   /*********************Configurable Upgrade APIs ******************************/
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/core/src/main/java/org/apache/sqoop/security/SecurityConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/sqoop/security/SecurityConstants.java b/core/src/main/java/org/apache/sqoop/security/SecurityConstants.java
index 0241c86..9d0974c 100644
--- a/core/src/main/java/org/apache/sqoop/security/SecurityConstants.java
+++ b/core/src/main/java/org/apache/sqoop/security/SecurityConstants.java
@@ -44,6 +44,13 @@ public final class SecurityConstants {
     PREFIX_SECURITY_CONFIG + "tls.";
 
   /**
+   * All repository encryption related configuration is prefixed with this:
+   * <tt>org.apache.sqoop.security.repo_encryption.</tt>
+   */
+  public static final String PREFIX_REPO_ENCRYPTION_CONFIG =
+    PREFIX_SECURITY_CONFIG + "repo_encryption.";
+
+  /**
    * The config specifies the sqoop authentication type (SIMPLE, KERBEROS).
    * The default type is SIMPLE
    * <tt>org.apache.sqoop.security.authentication.type</tt>.
@@ -217,6 +224,78 @@ public final class SecurityConstants {
     PREFIX_TLS_CONFIG + "keymanager_password_generator";
 
   /**
+   * The config specifies if repository encryption is enabled
+   * <tt>org.apache.sqoop.security.repo_encryption.enabled</tt>.
+   */
+  public static final String REPO_ENCRYPTION_ENABLED =
+    PREFIX_REPO_ENCRYPTION_CONFIG + "enabled";
+
+  /**
+   * The config specifies the password used to encrypt the repository
+   * <tt>org.apache.sqoop.security.repo_encryption.password</tt>.
+   */
+  public static final String REPO_ENCRYPTION_PASSWORD =
+    PREFIX_REPO_ENCRYPTION_CONFIG + "password";
+
+  /**
+   * The config specifies a command that prints the password used to encrypt
+   * the repository to standard out
+   * <tt>org.apache.sqoop.security.repo_encryption.password_generator</tt>.
+   */
+  public static final String REPO_ENCRYPTION_PASSWORD_GENERATOR=
+    PREFIX_REPO_ENCRYPTION_CONFIG + "password_generator";
+
+  /**
+   * The config specifies the algorithm to be used for hmac generation
+   * <tt>org.apache.sqoop.security.repo_encryption.hmac_algorithm</tt>.
+   */
+  public static final String REPO_ENCRYPTION_HMAC_ALGORITHM=
+    PREFIX_REPO_ENCRYPTION_CONFIG + "hmac_algorithm";
+
+  /**
+   * The config specifies the algorithm to be used for repository encryption
+   * <tt>org.apache.sqoop.security.repo_encryption.cipher_algorithm</tt>.
+   */
+  public static final String REPO_ENCRYPTION_CIPHER_ALGORITHM=
+    PREFIX_REPO_ENCRYPTION_CONFIG + "cipher_algorithm";
+
+  /**
+   * The config specifies the spec to be used for repository encryption
+   * <tt>org.apache.sqoop.security.repo_encryption.cipher_spec</tt>.
+   */
+  public static final String REPO_ENCRYPTION_CIPHER_SPEC=
+    PREFIX_REPO_ENCRYPTION_CONFIG + "cipher_spec";
+
+  /**
+   * The config specifies the size of the key used for repository encryption
+   * <tt>org.apache.sqoop.security.repo_encryption.cipher_key_size</tt>.
+   */
+  public static final String REPO_ENCRYPTION_CIPHER_KEY_SIZE=
+    PREFIX_REPO_ENCRYPTION_CONFIG + "cipher_key_size";
+
+  /**
+   * The config specifies the size of the initialization vector used for repository encryption
+   * <tt>org.apache.sqoop.security.repo_encryption.initialization_vector_size</tt>.
+   */
+  public static final String REPO_ENCRYPTION_INITIALIZATION_VECTOR_SIZE=
+    PREFIX_REPO_ENCRYPTION_CONFIG + "initialization_vector_size";
+
+  /**
+   * The config specifies the pbkdf2 algorithm to be used for master key generation
+   * <tt>org.apache.sqoop.security.repo_encryption.pbkdf2_algorithm</tt>.
+   */
+  public static final String REPO_ENCRYPTION_PBKDF2_ALGORITHM=
+    PREFIX_REPO_ENCRYPTION_CONFIG + "pbkdf2_algorithm";
+
+  /**
+   * The config specifies the number of rounds of the pbkdf2 algorithm
+   * to be used for master key generation
+   * <tt>org.apache.sqoop.security.repo_encryption.pbkdf2_algorithm</tt>.
+   */
+  public static final String REPO_ENCRYPTION_PBKDF2_ROUNDS=
+    PREFIX_REPO_ENCRYPTION_CONFIG + "pbkdf2_rounds";
+
+  /**
    * The config specifies the token kind in delegation token.
    */
   public static final String TOKEN_KIND = "sqoop_token_kind";

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/core/src/main/java/org/apache/sqoop/security/SecurityError.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/sqoop/security/SecurityError.java b/core/src/main/java/org/apache/sqoop/security/SecurityError.java
index 988e425..2ba849c 100644
--- a/core/src/main/java/org/apache/sqoop/security/SecurityError.java
+++ b/core/src/main/java/org/apache/sqoop/security/SecurityError.java
@@ -67,7 +67,43 @@ public enum SecurityError implements ErrorCode {
   AUTH_0014("Authorization exception"),
 
   /** Don't support to grant/remoke privileges for default user. */
-  AUTH_0015("Cannot grant/revoke privileges for default user");
+  AUTH_0015("Cannot grant/revoke privileges for default user"),
+
+  /** The HMAC calculation yielded a result different than what was stored in the database */
+  ENCRYPTION_0001("HMAC validation failed for Master Key"),
+
+  /** The Master Key is found in the database and the creation of one is disabled */
+  ENCRYPTION_0002("No Master Key found"),
+
+  /** Calculation of the Master Key from the provided password failed */
+  ENCRYPTION_0003("Failed to generate Master Key from password"),
+
+  /** Could not encrypt the provided plaintext */
+  ENCRYPTION_0004("Failed to perform encryption"),
+
+  /** HMAC calculation failed */
+  ENCRYPTION_0005("Failed to calculate HMAC"),
+
+  /** Could not decrypt the provided ciphertext */
+  ENCRYPTION_0006("Failed to perform decryption"),
+
+  /** Could not execute the generator script to create a password */
+  ENCRYPTION_0007("Failed to execute password generator"),
+
+  /** No password or password generator found in the configuration file */
+  ENCRYPTION_0008("No password or password generator set"),
+
+  /** One of the necessary configuration entries for encryption is missing */
+  ENCRYPTION_0009("Invalid configuration"),
+
+  /** HMAC validation failed for input, it may have been tampered with */
+  ENCRYPTION_0010("HMAC validation failed for input"),
+
+  /** The HMAC algorithm specified in configuration could not be found */
+  ENCRYPTION_0011("HMAC algorithm not found"),
+
+  ;
+
 
   private final String message;
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/core/src/test/java/org/apache/sqoop/repository/TestMasterKeyManager.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/sqoop/repository/TestMasterKeyManager.java b/core/src/test/java/org/apache/sqoop/repository/TestMasterKeyManager.java
new file mode 100644
index 0000000..f9579bf
--- /dev/null
+++ b/core/src/test/java/org/apache/sqoop/repository/TestMasterKeyManager.java
@@ -0,0 +1,208 @@
+ /**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.sqoop.repository;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.lang.StringUtils;
+import org.apache.sqoop.common.MapContext;
+import org.apache.sqoop.common.SqoopException;
+import org.apache.sqoop.core.SqoopConfiguration;
+import org.apache.sqoop.model.MMasterKey;
+import org.apache.sqoop.security.SecurityConstants;
+import org.mockito.ArgumentCaptor;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.assertEquals;
+
+public class TestMasterKeyManager {
+  private MasterKeyManager masterKeyManager;
+  private RepositoryManager repositoryManagerMock;
+  private Repository jdbcRepoMock;
+  private Map<String, String> configurationMap;
+
+  private static final int HMAC_KEY_SIZE_BYTES = 32;
+  private static final int CIPHER_KEY_SIZE_BYTES = 16;
+
+  @BeforeMethod(alwaysRun = true)
+  public void setUp() throws Exception {
+    SqoopConfiguration configurationMock = mock(SqoopConfiguration.class);
+    configurationMap = new HashMap<>();
+    configurationMap.put(SecurityConstants.REPO_ENCRYPTION_ENABLED, String
+      .valueOf(true));
+    configurationMap.put(SecurityConstants
+      .REPO_ENCRYPTION_PASSWORD_GENERATOR, "echo youwillnevergetthis");
+    configurationMap.put(SecurityConstants.REPO_ENCRYPTION_HMAC_ALGORITHM,
+      "HmacSHA256");
+    configurationMap.put(SecurityConstants.REPO_ENCRYPTION_CIPHER_ALGORITHM,
+      "AES");
+    configurationMap.put(SecurityConstants.REPO_ENCRYPTION_CIPHER_KEY_SIZE,
+      String.valueOf(CIPHER_KEY_SIZE_BYTES));
+    configurationMap.put(SecurityConstants.REPO_ENCRYPTION_INITIALIZATION_VECTOR_SIZE,
+      String.valueOf(CIPHER_KEY_SIZE_BYTES));
+    configurationMap.put(SecurityConstants.REPO_ENCRYPTION_CIPHER_SPEC,
+      "AES/CBC/PKCS5Padding");
+    configurationMap.put(SecurityConstants.REPO_ENCRYPTION_PBKDF2_ALGORITHM,
+      "PBKDF2WithHmacSHA1");
+    configurationMap.put(SecurityConstants.REPO_ENCRYPTION_PBKDF2_ROUNDS,
+      "4000");
+    doReturn(new MapContext(configurationMap)).when(configurationMock)
+      .getContext();
+    SqoopConfiguration.setInstance(configurationMock);
+
+    repositoryManagerMock = mock(RepositoryManager.class);
+    RepositoryManager.setInstance(repositoryManagerMock);
+
+
+    jdbcRepoMock = mock(JdbcRepository.class);
+    when(jdbcRepoMock.getMasterKey()).thenReturn(null);
+    when(repositoryManagerMock.getRepository()).thenReturn(jdbcRepoMock);
+
+    masterKeyManager = MasterKeyManager.getInstance();
+  }
+
+  @AfterMethod(alwaysRun = true)
+  public void tearDown() {
+    masterKeyManager.destroy();
+
+  }
+
+  @Test(
+    expectedExceptions = {SqoopException.class},
+    expectedExceptionsMessageRegExp = ".*No Master Key found")
+  public void testInitializeWithoutKeyCreationWithoutExistingKey() {
+    masterKeyManager.initialize(false);
+  }
+
+  @Test
+  public void testInitializeWithoutKeyCreationWithExistingKey() {
+    masterKeyManager.initialize();
+
+    ArgumentCaptor<MMasterKey> mMasterKeyArgumentCaptor = ArgumentCaptor
+      .forClass(MMasterKey.class);
+    verify(jdbcRepoMock, times(1)).createMasterKey(mMasterKeyArgumentCaptor
+      .capture());
+
+    // Encrypt something with that master key
+    String secret = "imasecret";
+    String iv = masterKeyManager.generateRandomIv();
+    String encrypted = masterKeyManager
+      .encryptWithMasterKey(secret, iv);
+
+    masterKeyManager.destroy();
+
+    // Create a new MasterKeyManager instance with existing master key
+    // coming from the "db"
+    jdbcRepoMock = mock(JdbcRepository.class);
+    when(jdbcRepoMock.getMasterKey()).thenReturn(mMasterKeyArgumentCaptor
+      .getValue());
+    when(repositoryManagerMock.getRepository()).thenReturn(jdbcRepoMock);
+
+    masterKeyManager.initialize();
+    verify(jdbcRepoMock, times(1)).getMasterKey();
+
+    // Try to decrypt
+    assertEquals(masterKeyManager.decryptWithMasterKey(encrypted, iv, masterKeyManager.generateHmacWithMasterHmacKey(encrypted)), secret);
+  }
+
+  @Test
+  public void testInitializeWithKeyCreationWithoutExistingKey() {
+    masterKeyManager.initialize();
+
+    verify(jdbcRepoMock, times(1)).createMasterKey(any(MMasterKey.class));
+  }
+
+  @Test(
+    expectedExceptions = {SqoopException.class},
+    expectedExceptionsMessageRegExp = ".*HMAC validation failed for Master Key"
+  )
+  public void testMasterKeyWithInvalidHmac() {
+    jdbcRepoMock = mock(JdbcRepository.class);
+    when(jdbcRepoMock.getMasterKey()).thenReturn(new MMasterKey(
+      Base64.encodeBase64String(generateRandomByteArray(CIPHER_KEY_SIZE_BYTES)),
+      Base64.encodeBase64String(generateRandomByteArray(HMAC_KEY_SIZE_BYTES)),
+      Base64.encodeBase64String(generateRandomByteArray(CIPHER_KEY_SIZE_BYTES)),
+      Base64.encodeBase64String(generateRandomByteArray(CIPHER_KEY_SIZE_BYTES))
+    ));
+    when(repositoryManagerMock.getRepository()).thenReturn(jdbcRepoMock);
+
+    masterKeyManager.initialize();
+  }
+
+  @Test(
+    expectedExceptions = {SqoopException.class},
+    expectedExceptionsMessageRegExp = ".*No password or password generator set")
+  public void testNoPasswordOrGenerator() {
+    configurationMap.put(SecurityConstants
+      .REPO_ENCRYPTION_PASSWORD_GENERATOR, StringUtils.EMPTY);
+
+    masterKeyManager.initialize();
+  }
+
+  @Test
+  public void testEncryptAndDecryptWithMasterKey() {
+    masterKeyManager.initialize();
+
+    String secret = "imasecret";
+    String iv = masterKeyManager.generateRandomIv();
+    String encrypted = masterKeyManager
+      .encryptWithMasterKey(secret, iv);
+
+    assertEquals(masterKeyManager.decryptWithMasterKey(encrypted, iv, masterKeyManager.generateHmacWithMasterHmacKey(encrypted)), secret);
+  }
+
+  @Test(
+    expectedExceptions = {SqoopException.class},
+    expectedExceptionsMessageRegExp = ".*HMAC validation failed for input")
+  public void testEncryptAndDecryptWithMasterKeyWithInvalidHmac() {
+    masterKeyManager.initialize();
+
+    String secret = "imasecret";
+    String iv = masterKeyManager.generateRandomIv();
+    String encrypted = masterKeyManager.encryptWithMasterKey(secret, iv);
+
+    String invalidHmac = Base64.encodeBase64String(generateRandomByteArray(HMAC_KEY_SIZE_BYTES));
+    masterKeyManager.decryptWithMasterKey(encrypted, iv, invalidHmac);
+  }
+
+  @Test(
+    expectedExceptions = {SqoopException.class},
+    expectedExceptionsMessageRegExp = ".*Invalid configuration.*" +
+      SecurityConstants.REPO_ENCRYPTION_PBKDF2_ALGORITHM)
+  public void testMissingConfiguration() {
+    configurationMap.put(SecurityConstants.REPO_ENCRYPTION_PBKDF2_ALGORITHM, "");
+    masterKeyManager.initialize();
+  }
+
+  private static byte[] generateRandomByteArray(int size) {
+    byte[] randomBytes = new byte[size];
+    new Random().nextBytes(randomBytes);
+    return randomBytes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/dist/src/main/conf/sqoop.properties
----------------------------------------------------------------------
diff --git a/dist/src/main/conf/sqoop.properties b/dist/src/main/conf/sqoop.properties
index 767d3f2..58b60fd 100755
--- a/dist/src/main/conf/sqoop.properties
+++ b/dist/src/main/conf/sqoop.properties
@@ -188,6 +188,21 @@ org.apache.sqoop.execution.engine=org.apache.sqoop.execution.mapreduce.Mapreduce
 #org.apache.sqoop.security.tls.keystore=
 #org.apache.sqoop.security.tls.keystore_password=
 
+#
+# Repository Encryption
+#
+
+#org.apache.sqoop.security.repo_encryption.enabled=true
+#org.apache.sqoop.security.repo_encryption.password=
+#org.apache.sqoop.security.repo_encryption.password_generator=
+#org.apache.sqoop.security.repo_encryption.hmac_algorithm=HmacSHA256
+#org.apache.sqoop.security.repo_encryption.cipher_algorithm=AES
+#org.apache.sqoop.security.repo_encryption.cipher_key_size=128
+#org.apache.sqoop.security.repo_encryption.cipher_spec=AES/CBC/PKCS5Padding
+#org.apache.sqoop.security.repo_encryption.initialization_vector_size=128
+#org.apache.sqoop.security.repo_encryption.pbkdf2_algorithm=PBKDF2WithHmacSHA1
+#org.apache.sqoop.security.repo_encryption.pbkdf2_rounds=4000
+
 
 # External connectors load path
 # "/path/to/external/connectors/": Add all the connector JARs in the specified folder

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5b897a46/repository/repository-common/src/main/java/org/apache/sqoop/repository/common/CommonRepositoryHandler.java
----------------------------------------------------------------------
diff --git a/repository/repository-common/src/main/java/org/apache/sqoop/repository/common/CommonRepositoryHandler.java b/repository/repository-common/src/main/java/org/apache/sqoop/repository/common/CommonRepositoryHandler.java
index 15cc41b..a9168ae 100644
--- a/repository/repository-common/src/main/java/org/apache/sqoop/repository/common/CommonRepositoryHandler.java
+++ b/repository/repository-common/src/main/java/org/apache/sqoop/repository/common/CommonRepositoryHandler.java
@@ -42,6 +42,7 @@ import org.apache.sqoop.common.MutableMapContext;
 import org.apache.sqoop.common.SqoopException;
 import org.apache.sqoop.common.SupportedDirections;
 import org.apache.sqoop.connector.ConnectorManager;
+import org.apache.sqoop.core.SqoopConfiguration;
 import org.apache.sqoop.driver.Driver;
 import org.apache.sqoop.error.code.CommonRepositoryError;
 import org.apache.sqoop.model.InputEditable;
@@ -65,11 +66,14 @@ import org.apache.sqoop.model.MLinkConfig;
 import org.apache.sqoop.model.MListInput;
 import org.apache.sqoop.model.MLongInput;
 import org.apache.sqoop.model.MMapInput;
+import org.apache.sqoop.model.MMasterKey;
 import org.apache.sqoop.model.MStringInput;
 import org.apache.sqoop.model.MSubmission;
 import org.apache.sqoop.model.MToConfig;
 import org.apache.sqoop.model.SubmissionError;
 import org.apache.sqoop.repository.JdbcRepositoryHandler;
+import org.apache.sqoop.repository.MasterKeyManager;
+import org.apache.sqoop.security.SecurityConstants;
 import org.apache.sqoop.submission.SubmissionStatus;
 import org.apache.sqoop.submission.counter.Counter;
 import org.apache.sqoop.submission.counter.CounterGroup;
@@ -1147,6 +1151,56 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
     }
   }
 
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public MMasterKey getMasterKey(Connection conn) {
+    try (PreparedStatement stmt = conn.prepareStatement(crudQueries.getStmtSelectSqMasterKey())) {
+      stmt.setMaxRows(1);
+
+      try (ResultSet rs = stmt.executeQuery()) {
+        if (!rs.next()) {
+          return null;
+        }
+
+        String aesKey = rs.getString(1);
+        String hmac = rs.getString(2);
+        String salt = rs.getString(3);
+        String iv = rs.getString(4);
+
+        return new MMasterKey(aesKey, hmac, salt, iv);
+      }
+    } catch (SQLException ex) {
+      logException(ex);
+      throw new SqoopException(CommonRepositoryError.COMMON_0059, ex);
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void createMasterKey(MMasterKey mMasterKey, Connection conn) {
+    int result;
+    try (PreparedStatement preparedStatement = conn.prepareStatement(crudQueries.getStmtInsertSqMasterKey(),
+      Statement.RETURN_GENERATED_KEYS)) {
+      preparedStatement.setString(1, mMasterKey.getEncryptedSecret());
+      preparedStatement.setString(2, mMasterKey.getHmac());
+      preparedStatement.setString(3, mMasterKey.getSalt());
+      preparedStatement.setString(4, mMasterKey.getIv());
+
+      result = preparedStatement.executeUpdate();
+      if (result != 1) {
+        throw new SqoopException(CommonRepositoryError.COMMON_0009,
+          Integer.toString(result));
+      }
+    } catch (SQLException ex) {
+      logException(ex, mMasterKey);
+      throw new SqoopException(CommonRepositoryError.COMMON_0031, ex);
+    }
+  }
+
   private void insertConnectorDirection(Long connectorId, Direction direction, Connection conn)
       throws SQLException {
     try (PreparedStatement stmt = conn.prepareStatement(crudQueries.getStmtInsertSqConnectorDirections())) {
@@ -2071,8 +2125,7 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
             // get the overrides value from the SQ_INPUT_RELATION table
             String overrides = getOverrides(inputId, conn);
             String inputEnumValues = rsetInput.getString(9);
-            String value = rsetInput.getString(10);
-
+            String value = readInputValue(rsetInput.getString(10), rsetInput.getBoolean(11), rsetInput.getString(12), rsetInput.getString(13));
             MInputType mit = MInputType.valueOf(inputType);
             MInput input = null;
             switch (mit) {
@@ -2184,7 +2237,7 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
         while (inputResults.next()) {
           long inputId = inputResults.getLong(1);
           String inputName = inputResults.getString(2);
-          String value = inputResults.getString(10);
+          String value = readInputValue(inputResults.getString(10), inputResults.getBoolean(11), inputResults.getString(12), inputResults.getString(13));
           if (mConfig.getName().equals(configName) && mConfig.getInputNames().contains(inputName)) {
             MInput mInput = mConfig.getInput(inputName);
             mInput.setPersistenceId(inputId);
@@ -2199,6 +2252,24 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
     }
   }
 
+
+  /**
+   * Reads the value of an input, while handling the possibility that the input is encrypted
+   *
+   * @param possiblyEncryptedValue Cleartext or base64 encoded ciphertext representing the input
+   * @param encrypted Is the input encrypted
+   * @param iv Encryption initialization vector
+   * @param hmac HMAC for tamper resistance
+   * @return The input value
+   */
+  private String readInputValue(String possiblyEncryptedValue, boolean encrypted, String iv, String hmac) throws SqoopException {
+    if (encrypted) {
+      return MasterKeyManager.getInstance().decryptWithMasterKey(possiblyEncryptedValue, iv, hmac);
+    } else {
+      return possiblyEncryptedValue;
+    }
+  }
+
   /**
    * Load configs and corresponding inputs related to a connector.
    *
@@ -2247,7 +2318,7 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
             // get the overrides value from the SQ_INPUT_RELATION table
             String overrides = getOverrides(inputId, conn);
             String inputEnumValues = rsetInput.getString(9);
-            String value = rsetInput.getString(10);
+            String value = readInputValue(rsetInput.getString(10), rsetInput.getBoolean(11), rsetInput.getString(12), rsetInput.getString(13));
 
             MInputType mit = MInputType.valueOf(inputType);
 
@@ -2401,24 +2472,40 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
       throws SQLException {
     int result;
 
-    try (PreparedStatement stmt = conn.prepareStatement(query)) {
+    boolean encryptionEnabled = SqoopConfiguration.getInstance().getContext().getBoolean(SecurityConstants.REPO_ENCRYPTION_ENABLED, false);
+    MasterKeyManager masterKeyManager = MasterKeyManager.getInstance();
       for (MConfig config : configs) {
         for (MInput<?> input : config.getInputs()) {
           // Skip empty values as we're not interested in storing those in db
           if (input.isEmpty()) {
             continue;
           }
-          stmt.setLong(1, id);
-          stmt.setLong(2, input.getPersistenceId());
-          stmt.setString(3, input.getUrlSafeValueString());
 
-          result = stmt.executeUpdate();
+          try (PreparedStatement stmt = conn.prepareStatement(query)) {
+            stmt.setLong(1, id);
+            stmt.setLong(2, input.getPersistenceId());
+            if (input.isSensitive() && encryptionEnabled) {
+              String iv = MasterKeyManager.getInstance().generateRandomIv();
+              String hmac = null;
+              String encryptedInput = masterKeyManager.encryptWithMasterKey(input.getUrlSafeValueString(), iv);
+              stmt.setString(3, encryptedInput);
+              hmac = masterKeyManager.generateHmacWithMasterHmacKey(encryptedInput);
+              stmt.setBoolean(4, true);
+              stmt.setString(5, iv);
+              stmt.setString(6, hmac);
+            } else {
+              stmt.setString(3, input.getUrlSafeValueString());
+              stmt.setBoolean(4, false);
+              stmt.setNull(5, Types.VARCHAR);
+              stmt.setNull(6, Types.VARCHAR);
+            }
+            result = stmt.executeUpdate();
+          }
           if (result != 1) {
             throw new SqoopException(CommonRepositoryError.COMMON_0017, Integer.toString(result));
           }
         }
       }
-    }
   }
 
   /**