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 2015/02/05 17:36:57 UTC

[1/2] sqoop git commit: SQOOP-1804: Add editable and override attribute to inputs

Repository: sqoop
Updated Branches:
  refs/heads/sqoop2 433a42d7a -> 6fc50b08b


http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestGenericJdbcConnectorUpgrader.java
----------------------------------------------------------------------
diff --git a/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestGenericJdbcConnectorUpgrader.java b/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestGenericJdbcConnectorUpgrader.java
index 1b1fa7f..220e42e 100644
--- a/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestGenericJdbcConnectorUpgrader.java
+++ b/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestGenericJdbcConnectorUpgrader.java
@@ -17,10 +17,12 @@
  */
 package org.apache.sqoop.connector.jdbc;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
 import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
 import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
 import org.apache.sqoop.model.ConfigUtils;
+import org.apache.sqoop.model.InputEditable;
 import org.apache.sqoop.model.MBooleanInput;
 import org.apache.sqoop.model.MConfig;
 import org.apache.sqoop.model.MFromConfig;
@@ -71,13 +73,13 @@ public class TestGenericJdbcConnectorUpgrader {
     originalConfigs = new MFromConfig(new LinkedList<MConfig>());
     newConfigs = new MFromConfig(ConfigUtils.toConfigs(FromJobConfiguration.class));
     originalConfigs.getConfigs().add(new MConfig("table", new LinkedList<MInput<?>>()));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.schemaName", false, (short)50));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.tableName", false, (short)50));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.sql", false, (short)2000));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.columns", false, (short)50));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.partitionColumn", false, (short)50));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MBooleanInput("table.partitionColumnNull", false));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.boundaryQuery", false, (short)50));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.schemaName", false, InputEditable.ANY, StringUtils.EMPTY, (short)50));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.tableName", false, InputEditable.ANY, StringUtils.EMPTY, (short)50));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.sql", false, InputEditable.ANY, StringUtils.EMPTY, (short)2000));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.columns", false, InputEditable.ANY, StringUtils.EMPTY, (short)50));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.partitionColumn", false, InputEditable.ANY, StringUtils.EMPTY, (short)50));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MBooleanInput("table.partitionColumnNull", false, InputEditable.ANY, StringUtils.EMPTY));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.boundaryQuery", false, InputEditable.ANY, StringUtils.EMPTY, (short)50));
     originalConfigs.getInput("table.schemaName").setValue("test-schema");
     originalConfigs.getInput("table.tableName").setValue("test-tableName");
     originalConfigs.getInput("table.sql").setValue("test-sql");
@@ -117,12 +119,13 @@ public class TestGenericJdbcConnectorUpgrader {
     originalConfigs = new MToConfig(new LinkedList<MConfig>());
     newConfigs = new MToConfig(ConfigUtils.toConfigs(ToJobConfiguration.class));
     originalConfigs.getConfigs().add(new MConfig("table", new LinkedList<MInput<?>>()));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.schemaName", false, (short)50));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.tableName", false, (short)50));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.sql", false, (short)2000));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.columns", false, (short)50));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.stageTableName", false, (short)50));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MBooleanInput("table.clearStageTable", false));
+
+    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.schemaName", false, InputEditable.ANY, StringUtils.EMPTY, (short)50));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.tableName", false, InputEditable.ANY, StringUtils.EMPTY, (short)50));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.sql", false, InputEditable.ANY, StringUtils.EMPTY,(short)2000));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.columns", false, InputEditable.ANY, StringUtils.EMPTY, (short)50));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("table.stageTableName", false, InputEditable.ANY, StringUtils.EMPTY, (short)50));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MBooleanInput("table.clearStageTable", false, InputEditable.ANY, StringUtils.EMPTY));
     originalConfigs.getInput("table.schemaName").setValue("test-schema");
     originalConfigs.getInput("table.tableName").setValue("test-tableName");
     originalConfigs.getInput("table.sql").setValue("test-sql");
@@ -160,11 +163,11 @@ public class TestGenericJdbcConnectorUpgrader {
     originalConfigs = new MLinkConfig(new LinkedList<MConfig>());
     newConfigs = new MLinkConfig(ConfigUtils.toConfigs(LinkConfiguration.class));
     originalConfigs.getConfigs().add(new MConfig("connection", new LinkedList<MInput<?>>()));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("connection.jdbcDriver", false, (short)50));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("connection.connectionString", false, (short)50));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("connection.username", false, (short)2000));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("connection.password", false, (short)50));
-    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("connection.jdbcProperties", false, (short)50));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("connection.jdbcDriver", false, InputEditable.ANY, StringUtils.EMPTY, (short)50));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("connection.connectionString", false,InputEditable.ANY, StringUtils.EMPTY, (short)50));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("connection.username", false, InputEditable.ANY, StringUtils.EMPTY, (short)2000));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("connection.password", false, InputEditable.ANY, StringUtils.EMPTY, (short)50));
+    originalConfigs.getConfigs().get(0).getInputs().add(new MStringInput("connection.jdbcProperties", false, InputEditable.ANY, StringUtils.EMPTY, (short)50));
     originalConfigs.getInput("connection.jdbcDriver").setValue("test-jdbcDriver");
     originalConfigs.getInput("connection.connectionString").setValue("test-connectionString");
     originalConfigs.getInput("connection.username").setValue("test-username");

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/core/src/test/java/org/apache/sqoop/driver/TestDriverConfigUpgrader.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/sqoop/driver/TestDriverConfigUpgrader.java b/core/src/test/java/org/apache/sqoop/driver/TestDriverConfigUpgrader.java
index d4522ae..819a7d0 100644
--- a/core/src/test/java/org/apache/sqoop/driver/TestDriverConfigUpgrader.java
+++ b/core/src/test/java/org/apache/sqoop/driver/TestDriverConfigUpgrader.java
@@ -24,6 +24,8 @@ import static org.testng.Assert.assertNull;
 import java.util.LinkedList;
 import java.util.List;
 
+import org.apache.commons.lang.StringUtils;
+import org.apache.sqoop.model.InputEditable;
 import org.apache.sqoop.model.MConfig;
 import org.apache.sqoop.model.MConfigList;
 import org.apache.sqoop.model.MDriverConfig;
@@ -127,9 +129,11 @@ public class TestDriverConfigUpgrader {
 
   List<MInput<?>> inputs1(String formName) {
     List<MInput<?>> list = new LinkedList<MInput<?>>();
-    list.add(new MStringInput(formName + ".s1", false, (short) 30));
-    list.add(new MStringInput(formName + ".s2", false, (short) 30));
-    list.add(new MIntegerInput(formName + ".i", false));
+    list.add(new MStringInput(formName + ".s1", false, InputEditable.ANY, StringUtils.EMPTY,
+        (short) 30));
+    list.add(new MStringInput(formName + ".s2", false, InputEditable.ANY, StringUtils.EMPTY,
+        (short) 30));
+    list.add(new MIntegerInput(formName + ".i", false, InputEditable.ANY, StringUtils.EMPTY));
     return list;
   }
 
@@ -141,9 +145,11 @@ public class TestDriverConfigUpgrader {
 
   List<MInput<?>> inputs2(String formName) {
     List<MInput<?>> list = new LinkedList<MInput<?>>();
-    list.add(new MStringInput(formName + ".s1", false, (short) 30));
-    list.add(new MStringInput(formName + ".s2_", false, (short) 30));
-    list.add(new MIntegerInput(formName + ".i", false));
+    list.add(new MStringInput(formName + ".s1", false, InputEditable.ANY, StringUtils.EMPTY,
+        (short) 30));
+    list.add(new MStringInput(formName + ".s2_", false, InputEditable.ANY, StringUtils.EMPTY,
+        (short) 30));
+    list.add(new MIntegerInput(formName + ".i", false, InputEditable.ANY, StringUtils.EMPTY));
     return list;
   }
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/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 4feaee6..b029ab2 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
@@ -25,6 +25,7 @@ import org.apache.sqoop.common.SqoopException;
 import org.apache.sqoop.common.SupportedDirections;
 import org.apache.sqoop.driver.Driver;
 import org.apache.sqoop.error.code.CommonRepositoryError;
+import org.apache.sqoop.model.InputEditable;
 import org.apache.sqoop.model.SubmissionError;
 import org.apache.sqoop.model.MBooleanInput;
 import org.apache.sqoop.model.MConfig;
@@ -60,8 +61,10 @@ import java.sql.Timestamp;
 import java.sql.Types;
 import java.util.ArrayList;
 import java.util.Date;
+import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 
 /**
  * Set of methods required from each JDBC based repository.
@@ -184,8 +187,11 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
     PreparedStatement deleteConfig = null;
     PreparedStatement deleteConfigDirection = null;
     PreparedStatement deleteInput = null;
+    PreparedStatement deleteInputRelation = null;
+
     try {
       updateConnectorStatement = conn.prepareStatement(crudQueries.getStmtUpdateConfigurable());
+      deleteInputRelation = conn.prepareStatement(CommonRepositoryInsertUpdateDeleteSelectQuery.STMT_DELETE_INPUT_RELATIONS_FOR_INPUT);
       deleteInput = conn.prepareStatement(crudQueries.getStmtDeleteInputsForConfigurable());
       deleteConfigDirection = conn.prepareStatement(crudQueries.getStmtDeleteDirectionsForConfigurable());
       deleteConfig = conn.prepareStatement(crudQueries.getStmtDeleteConfigsForConfigurable());
@@ -199,9 +205,11 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
       if (updateConnectorStatement.executeUpdate() != 1) {
         throw new SqoopException(CommonRepositoryError.COMMON_0035);
       }
+      deleteInputRelation.setLong(1, mConnector.getPersistenceId());
       deleteInput.setLong(1, mConnector.getPersistenceId());
       deleteConfigDirection.setLong(1, mConnector.getPersistenceId());
       deleteConfig.setLong(1, mConnector.getPersistenceId());
+      deleteInputRelation.executeUpdate();
       deleteInput.executeUpdate();
       deleteConfigDirection.executeUpdate();
       deleteConfig.executeUpdate();
@@ -227,8 +235,10 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
     PreparedStatement updateDriverStatement = null;
     PreparedStatement deleteConfig = null;
     PreparedStatement deleteInput = null;
+    PreparedStatement deleteInputRelation = null;
     try {
       updateDriverStatement = conn.prepareStatement(crudQueries.getStmtUpdateConfigurable());
+      deleteInputRelation = conn.prepareStatement(CommonRepositoryInsertUpdateDeleteSelectQuery.STMT_DELETE_INPUT_RELATIONS_FOR_INPUT);
       deleteInput = conn.prepareStatement(crudQueries.getStmtDeleteInputsForConfigurable());
       deleteConfig = conn.prepareStatement(crudQueries.getStmtDeleteConfigsForConfigurable());
       updateDriverStatement.setString(1, mDriver.getUniqueName());
@@ -240,8 +250,10 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
       if (updateDriverStatement.executeUpdate() != 1) {
         throw new SqoopException(CommonRepositoryError.COMMON_0035);
       }
+      deleteInputRelation.setLong(1, mDriver.getPersistenceId());
       deleteInput.setLong(1, mDriver.getPersistenceId());
       deleteConfig.setLong(1, mDriver.getPersistenceId());
+      deleteInputRelation.executeUpdate();
       deleteInput.executeUpdate();
       deleteConfig.executeUpdate();
 
@@ -261,6 +273,7 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
    * @param conn    JDBC link to use for updating the configs
    */
   private void insertConfigsForDriver(MDriver mDriver, Connection conn) {
+    long driverId = mDriver.getPersistenceId();
     PreparedStatement baseConfigStmt = null;
     PreparedStatement baseInputStmt = null;
     try {
@@ -271,7 +284,7 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
           Statement.RETURN_GENERATED_KEYS);
 
       // Register a driver config as a job type with no direction
-      registerConfigs(null, null, mDriver.getDriverConfig().getConfigs(),
+      registerConfigs(driverId, null, mDriver.getDriverConfig().getConfigs(),
           MConfigType.JOB.name(), baseConfigStmt, baseInputStmt, conn);
 
     } catch (SQLException ex) {
@@ -308,11 +321,8 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
 
       driverConfigInputFetchStmt = conn.prepareStatement(crudQueries.getStmtSelectInput());
       List<MConfig> driverConfigs = new ArrayList<MConfig>();
-      loadDriverConfigs(driverConfigs, driverConfigFetchStmt, driverConfigInputFetchStmt, 1);
+      loadDriverConfigs(driverConfigs, driverConfigFetchStmt, driverConfigInputFetchStmt, 1, conn);
 
-      if (driverConfigs.isEmpty()) {
-        return null;
-      }
       mDriver = new MDriver(new MDriverConfig(driverConfigs), driverVersion);
       mDriver.setPersistenceId(driverId);
 
@@ -355,7 +365,7 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
       throw new SqoopException(CommonRepositoryError.COMMON_0008, mDriver.getUniqueName());
     }
     mDriver.setPersistenceId(insertAndGetDriverId(mDriver, conn));
-    insertConfigsforDriver(mDriver, conn);
+    insertConfigsForDriver(mDriver, conn);
   }
 
   /**
@@ -1295,27 +1305,6 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
     }
   }
 
-  private void insertConfigsforDriver(MDriver mDriver, Connection conn) {
-    PreparedStatement baseConfigStmt = null;
-    PreparedStatement baseInputStmt = null;
-    try {
-      baseConfigStmt = conn.prepareStatement(crudQueries.getStmtInsertIntoConfig(),
-          Statement.RETURN_GENERATED_KEYS);
-      baseInputStmt = conn.prepareStatement(crudQueries.getStmtInsertIntoInput(),
-          Statement.RETURN_GENERATED_KEYS);
-
-      // Register a driver config as a job type with no owner/connector and direction
-      registerConfigs(mDriver.getPersistenceId(), null /* no direction*/, mDriver.getDriverConfig().getConfigs(),
-          MConfigType.JOB.name(), baseConfigStmt, baseInputStmt, conn);
-
-    } catch (SQLException ex) {
-      logException(ex, mDriver);
-      throw new SqoopException(CommonRepositoryError.COMMON_0011, ex);
-    } finally {
-      closeStatements(baseConfigStmt, baseInputStmt);
-    }
-  }
-
   /**
    * Stores counters for given submission in repository.
    *
@@ -1595,7 +1584,7 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
         List<MConfig> fromConfig = new ArrayList<MConfig>();
         List<MConfig> toConfig = new ArrayList<MConfig>();
 
-        loadConnectorConfigTypes(linkConfig, fromConfig, toConfig, connectorConfigFetchStmt,
+        loadConnectorConfigs(linkConfig, fromConfig, toConfig, connectorConfigFetchStmt,
             connectorConfigInputFetchStmt, 1, conn);
 
         SupportedDirections supportedDirections
@@ -1654,7 +1643,7 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
         List<MConfig> fromConfig = new ArrayList<MConfig>();
         List<MConfig> toConfig = new ArrayList<MConfig>();
 
-        loadConnectorConfigTypes(connectorLinkConfig, fromConfig, toConfig, connectorConfigFetchStatement,
+        loadConnectorConfigs(connectorLinkConfig, fromConfig, toConfig, connectorConfigFetchStatement,
             connectorConfigInputStatement, 2, conn);
         MLink link = new MLink(connectorId, new MLinkConfig(connectorLinkConfig));
 
@@ -1721,7 +1710,7 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
         List<MConfig> fromConnectorFromJobConfig = new ArrayList<MConfig>();
         List<MConfig> fromConnectorToJobConfig = new ArrayList<MConfig>();
 
-        loadConnectorConfigTypes(fromConnectorLinkConfig, fromConnectorFromJobConfig, fromConnectorToJobConfig,
+        loadConnectorConfigs(fromConnectorLinkConfig, fromConnectorFromJobConfig, fromConnectorToJobConfig,
             fromConfigFetchStmt, jobInputFetchStmt, 2, conn);
 
         // TO entity configs
@@ -1732,10 +1721,10 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
         // ?? dont we need 2 different driver configs for the from/to?
         List<MConfig> driverConfig = new ArrayList<MConfig>();
 
-        loadConnectorConfigTypes(toConnectorLinkConfig, toConnectorFromJobConfig, toConnectorToJobConfig,
+        loadConnectorConfigs(toConnectorLinkConfig, toConnectorFromJobConfig, toConnectorToJobConfig,
             toConfigFetchStmt, jobInputFetchStmt, 2, conn);
 
-        loadDriverConfigs(driverConfig, driverConfigfetchStmt, jobInputFetchStmt, 2);
+        loadDriverConfigs(driverConfig, driverConfigfetchStmt, jobInputFetchStmt, 2, conn);
 
         MJob job = new MJob(
             fromConnectorId, toConnectorId,
@@ -1792,18 +1781,13 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
    * @return short number of configs registered.
    * @throws java.sql.SQLException
    */
-  private short registerConfigs(Long configurableId, Direction direction,
-                                List<MConfig> configs, String type, PreparedStatement baseConfigStmt,
-                                PreparedStatement baseInputStmt, Connection conn)
-      throws SQLException {
+  private short registerConfigs(Long configurableId, Direction direction, List<MConfig> configs,
+      String type, PreparedStatement baseConfigStmt, PreparedStatement baseInputStmt,
+      Connection conn) throws SQLException {
     short configIndex = 0;
 
     for (MConfig config : configs) {
-      if (configurableId == null) {
-        baseConfigStmt.setNull(1, Types.BIGINT);
-      } else {
-        baseConfigStmt.setLong(1, configurableId);
-      }
+      baseConfigStmt.setLong(1, configurableId);
 
       baseConfigStmt.setString(2, config.getName());
       baseConfigStmt.setString(3, type);
@@ -1828,7 +1812,26 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
 
       // Insert all the inputs
       List<MInput<?>> inputs = config.getInputs();
-      registerConfigInputs(configId, inputs, baseInputStmt);
+      registerConfigInputs(config, inputs, baseInputStmt);
+      // validate all the input relations
+      Map<Long, List<String>> inputRelationships = new HashMap<Long, List<String>>();
+      for (MInput<?> input : inputs) {
+        List<String> inputOverrides = validateAndGetOverridesAttribute(input, config);
+        if (inputOverrides != null && inputOverrides.size() > 0) {
+          inputRelationships.put(input.getPersistenceId(), inputOverrides);
+        }
+      }
+
+      // Insert all input relations
+      if (inputRelationships != null && inputRelationships.size() > 0) {
+        for (Map.Entry<Long, List<String>> entry : inputRelationships.entrySet()) {
+          List<String> children = entry.getValue();
+          for (String child : children) {
+            Long childId = config.getInput(child).getPersistenceId();
+            insertConfigInputRelationship(entry.getKey(), childId, conn);
+          }
+        }
+      }
     }
     return configIndex;
   }
@@ -1838,17 +1841,22 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
    *
    * Use given prepare statement to save all inputs into repository.
    *
-   * @param configId Identifier for corresponding config
-   * @param inputs List of inputs that needs to be saved
-   * @param baseInputStmt Statement that we can utilize
-   * @throws java.sql.SQLException In case of any failure on Derby side
+   * @param config
+   *          corresponding config
+   * @param inputs
+   *          List of inputs that needs to be saved
+   * @param baseInputStmt
+   *          Statement that we can utilize
+   * @throws java.sql.SQLException
+   *           In case of any failure on Derby side
    */
-  private void registerConfigInputs(long configId, List<MInput<?>> inputs,
-                                    PreparedStatement baseInputStmt) throws SQLException {
+  private void registerConfigInputs(MConfig config, List<MInput<?>> inputs,
+      PreparedStatement baseInputStmt) throws SQLException {
+
     short inputIndex = 0;
     for (MInput<?> input : inputs) {
       baseInputStmt.setString(1, input.getName());
-      baseInputStmt.setLong(2, configId);
+      baseInputStmt.setLong(2, config.getPersistenceId());
       baseInputStmt.setShort(3, inputIndex++);
       baseInputStmt.setString(4, input.getType().name());
       baseInputStmt.setBoolean(5, input.isSensitive());
@@ -1859,11 +1867,14 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
       } else {
         baseInputStmt.setNull(6, Types.INTEGER);
       }
+
+      baseInputStmt.setString(7, input.getEditable().name());
+
       // Enum specific column(s)
-      if(input.getType() == MInputType.ENUM) {
-        baseInputStmt.setString(7, StringUtils.join(((MEnumInput) input).getValues(), ","));
+      if (input.getType() == MInputType.ENUM) {
+        baseInputStmt.setString(8, StringUtils.join(((MEnumInput) input).getValues(), ","));
       } else {
-        baseInputStmt.setNull(7, Types.VARCHAR);
+        baseInputStmt.setNull(8, Types.VARCHAR);
       }
 
       int baseInputCount = baseInputStmt.executeUpdate();
@@ -1882,6 +1893,59 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
     }
   }
 
+  private void insertConfigInputRelationship(Long parent, Long child, Connection conn) {
+    PreparedStatement baseInputRelationStmt = null;
+    try {
+      baseInputRelationStmt = conn
+          .prepareStatement(CommonRepositoryInsertUpdateDeleteSelectQuery.STMT_INSERT_INTO_INPUT_RELATION);
+      baseInputRelationStmt.setLong(1, parent);
+      baseInputRelationStmt.setLong(2, child);
+      baseInputRelationStmt.executeUpdate();
+
+    } catch (SQLException ex) {
+      throw new SqoopException(CommonRepositoryError.COMMON_0047, ex);
+    } finally {
+      closeStatements(baseInputRelationStmt);
+    }
+  }
+
+  /**
+   * Validate that the input override attribute adheres to the rules imposed
+   * NOTE: all input names in a config class will and must be unique
+   * Rule #1.
+   * If editable == USER_ONLY ( cannot override itself ) can override other  CONNECTOR_ONLY and ANY inputs,
+   * but cannot overriding other USER_ONLY attributes
+   * Rule #2.
+   * If editable == CONNECTOR_ONLY or ANY ( cannot override itself ) can override any other attribute in the config object
+   * @param currentInput
+   *
+   */
+  private List<String> validateAndGetOverridesAttribute(MInput<?> currentInput, MConfig config) {
+
+    // split the overrides string into comma separated list
+    String overrides = currentInput.getOverrides();
+    if (StringUtils.isEmpty(overrides)) {
+      return null;
+    }
+    String[] overrideInputs = overrides.split("\\,");
+    List<String> children = new ArrayList<String>();
+
+    for (String override : overrideInputs) {
+      if (override.equals(currentInput.getName())) {
+        throw new SqoopException(CommonRepositoryError.COMMON_0046, "for input :"
+            + currentInput.toString());
+      }
+      if (currentInput.getEditable().equals(InputEditable.USER_ONLY)) {
+        if (config.getUserOnlyEditableInputNames().contains(override)) {
+          throw new SqoopException(CommonRepositoryError.COMMON_0045, "for input :"
+              + currentInput.toString());
+        }
+      }
+      children.add(override);
+    }
+    return children;
+  }
+
   /**
    * Load configs and corresponding inputs from Derby database.
    *
@@ -1894,10 +1958,10 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
    * @param configPosition position of the config
    * @throws java.sql.SQLException In case of any failure on Derby side
    */
-  public void loadDriverConfigs(List<MConfig> driverConfig,
+  private void loadDriverConfigs(List<MConfig> driverConfig,
                                 PreparedStatement configFetchStatement,
                                 PreparedStatement inputFetchStmt,
-                                int configPosition) throws SQLException {
+                                int configPosition, Connection conn) throws SQLException {
 
     // Get list of structures from database
     ResultSet rsetConfig = configFetchStatement.executeQuery();
@@ -1923,27 +1987,31 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
         String inputType = rsetInput.getString(5);
         boolean inputSensitivity = rsetInput.getBoolean(6);
         short inputStrLength = rsetInput.getShort(7);
-        String inputEnumValues = rsetInput.getString(8);
-        String value = rsetInput.getString(9);
+        String editable = rsetInput.getString(8);
+        InputEditable editableEnum = editable != null ? InputEditable.valueOf(editable)
+            : InputEditable.ANY;
+        // 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);
 
         MInputType mit = MInputType.valueOf(inputType);
-
         MInput input = null;
         switch (mit) {
-          case STRING:
-            input = new MStringInput(inputName, inputSensitivity, inputStrLength);
+     case STRING:
+            input = new MStringInput(inputName, inputSensitivity, editableEnum,  overrides, inputStrLength);
             break;
           case MAP:
-            input = new MMapInput(inputName, inputSensitivity);
+            input = new MMapInput(inputName, inputSensitivity, editableEnum, overrides);
             break;
           case BOOLEAN:
-            input = new MBooleanInput(inputName, inputSensitivity);
+            input = new MBooleanInput(inputName, inputSensitivity, editableEnum, overrides);
             break;
           case INTEGER:
-            input = new MIntegerInput(inputName, inputSensitivity);
+            input = new MIntegerInput(inputName, inputSensitivity, editableEnum, overrides);
             break;
           case ENUM:
-            input = new MEnumInput(inputName, inputSensitivity, inputEnumValues.split(","));
+            input = new MEnumInput(inputName, inputSensitivity, editableEnum, overrides, inputEnumValues.split(","));
             break;
           default:
             throw new SqoopException(CommonRepositoryError.COMMON_0003,
@@ -2037,7 +2105,7 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
    * @param conn Connection object that is used to find config direction.
    * @throws java.sql.SQLException In case of any failure on Derby side
    */
-  public void loadConnectorConfigTypes(List<MConfig> linkConfig, List<MConfig> fromConfig, List<MConfig> toConfig,
+  public void loadConnectorConfigs(List<MConfig> linkConfig, List<MConfig> fromConfig, List<MConfig> toConfig,
                                        PreparedStatement configFetchStmt, PreparedStatement inputFetchStmt,
                                        int configPosition, Connection conn) throws SQLException {
 
@@ -2065,32 +2133,38 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
         String inputType = rsetInput.getString(5);
         boolean inputSensitivity = rsetInput.getBoolean(6);
         short inputStrLength = rsetInput.getShort(7);
-        String inputEnumValues = rsetInput.getString(8);
-        String value = rsetInput.getString(9);
+        String editable = rsetInput.getString(8);
+        InputEditable editableEnum = editable != null ? InputEditable.valueOf(editable)
+            : InputEditable.ANY;
+        // 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);
 
         MInputType mit = MInputType.valueOf(inputType);
 
         MInput<?> input = null;
         switch (mit) {
-          case STRING:
-            input = new MStringInput(inputName, inputSensitivity, inputStrLength);
-            break;
-          case MAP:
-            input = new MMapInput(inputName, inputSensitivity);
-            break;
-          case BOOLEAN:
-            input = new MBooleanInput(inputName, inputSensitivity);
-            break;
-          case INTEGER:
-            input = new MIntegerInput(inputName, inputSensitivity);
-            break;
-          case ENUM:
-            input = new MEnumInput(inputName, inputSensitivity, inputEnumValues.split(","));
-            break;
-          default:
-            throw new SqoopException(CommonRepositoryError.COMMON_0003,
-                "input-" + inputName + ":" + inputId + ":"
-                    + "config-" + inputConfig + ":" + mit.name());
+        case STRING:
+          input = new MStringInput(inputName, inputSensitivity, editableEnum, overrides,
+              inputStrLength);
+          break;
+        case MAP:
+          input = new MMapInput(inputName, inputSensitivity, editableEnum, overrides);
+          break;
+        case BOOLEAN:
+          input = new MBooleanInput(inputName, inputSensitivity, editableEnum, overrides);
+          break;
+        case INTEGER:
+          input = new MIntegerInput(inputName, inputSensitivity, editableEnum, overrides);
+          break;
+        case ENUM:
+          input = new MEnumInput(inputName, inputSensitivity, editableEnum, overrides,
+              inputEnumValues.split(","));
+          break;
+        default:
+          throw new SqoopException(CommonRepositoryError.COMMON_0003, "input-" + inputName + ":"
+              + inputId + ":" + "config-" + inputConfig + ":" + mit.name());
         }
 
         // Set persistent ID
@@ -2170,6 +2244,60 @@ public abstract class CommonRepositoryHandler extends JdbcRepositoryHandler {
     }
   }
 
+  /**
+   * @param inputId
+   * @param conn
+   * @return
+   */
+  private String getOverrides(long inputId, Connection conn) {
+
+    PreparedStatement overridesStmt = null;
+    PreparedStatement inputStmt = null;
+
+    ResultSet rsOverride = null;
+    ResultSet rsInput = null;
+
+    List<String> overrides = new ArrayList<String>();
+    try {
+      overridesStmt = conn.prepareStatement(crudQueries.getStmtSelectInputOverrides());
+      inputStmt = conn.prepareStatement(crudQueries.getStmtSelectInputById());
+      overridesStmt.setLong(1, inputId);
+      rsOverride = overridesStmt.executeQuery();
+
+      while (rsOverride.next()) {
+        long overrideId = rsOverride.getLong(1);
+        inputStmt.setLong(1, overrideId);
+        rsInput = inputStmt.executeQuery();
+        if(rsInput.next()) {
+         overrides.add(rsInput.getString(2));
+        }
+      }
+      if (overrides != null && overrides.size() > 0) {
+        return StringUtils.join(overrides, ",");
+      } else {
+        return StringUtils.EMPTY;
+
+      }
+    } catch (SQLException ex) {
+      logException(ex, inputId);
+      throw new SqoopException(CommonRepositoryError.COMMON_0048, ex);
+    } finally {
+      if (rsOverride != null) {
+        closeResultSets(rsOverride);
+      }
+      if (rsInput != null) {
+        closeResultSets(rsInput);
+      }
+
+      if (overridesStmt != null) {
+        closeStatements(overridesStmt);
+      }
+      if (inputStmt != null) {
+        closeStatements(inputStmt);
+      }
+    }
+  }
+
   private void createInputValues(String query,
                                  long id,
                                  List<MConfig> configs,

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/repository/repository-common/src/main/java/org/apache/sqoop/repository/common/CommonRepositoryInsertUpdateDeleteSelectQuery.java
----------------------------------------------------------------------
diff --git a/repository/repository-common/src/main/java/org/apache/sqoop/repository/common/CommonRepositoryInsertUpdateDeleteSelectQuery.java b/repository/repository-common/src/main/java/org/apache/sqoop/repository/common/CommonRepositoryInsertUpdateDeleteSelectQuery.java
index 8626b31..d61ff0b 100644
--- a/repository/repository-common/src/main/java/org/apache/sqoop/repository/common/CommonRepositoryInsertUpdateDeleteSelectQuery.java
+++ b/repository/repository-common/src/main/java/org/apache/sqoop/repository/common/CommonRepositoryInsertUpdateDeleteSelectQuery.java
@@ -81,6 +81,21 @@ public class CommonRepositoryInsertUpdateDeleteSelectQuery {
           + " WHERE "
           + CommonRepoUtils.escapeColumnName(COLUMN_SQ_CFG_CONFIGURABLE) + " = ?)";
 
+  public static final String STMT_DELETE_INPUT_RELATIONS_FOR_INPUT =
+      "DELETE FROM " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_INPUT_RELATION_NAME)
+          + " WHERE "
+          + CommonRepoUtils.escapeColumnName(COLUMN_SQIR_PARENT)
+          + " IN (SELECT "
+          + CommonRepoUtils.escapeColumnName(COLUMN_SQI_ID)
+          + " FROM " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_INPUT_NAME)
+          + " WHERE "
+          + CommonRepoUtils.escapeColumnName(COLUMN_SQI_CONFIG)
+          + " IN (SELECT "
+          + CommonRepoUtils.escapeColumnName(COLUMN_SQ_CFG_ID)
+          + " FROM " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_CONFIG_NAME)
+          + " WHERE "
+          + CommonRepoUtils.escapeColumnName(COLUMN_SQ_CFG_CONFIGURABLE) + " = ?))";
+
   //Update the configurable
   public static final String STMT_UPDATE_CONFIGURABLE =
       "UPDATE " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_CONFIGURABLE_NAME)
@@ -128,12 +143,29 @@ public class CommonRepositoryInsertUpdateDeleteSelectQuery {
           + CommonRepoUtils.escapeColumnName(COLUMN_SQI_TYPE) + ", "
           + CommonRepoUtils.escapeColumnName(COLUMN_SQI_STRMASK) + ", "
           + CommonRepoUtils.escapeColumnName(COLUMN_SQI_STRLENGTH) + ", "
+          + CommonRepoUtils.escapeColumnName(COLUMN_SQI_EDITABLE) + ", "
           + CommonRepoUtils.escapeColumnName(COLUMN_SQI_ENUMVALS) + ", "
           + "cast(null as varchar(100))"
           + " FROM " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_INPUT_NAME)
           + " WHERE " + CommonRepoUtils.escapeColumnName(COLUMN_SQI_CONFIG) + " = ?"
           + " ORDER BY " + CommonRepoUtils.escapeColumnName(COLUMN_SQI_INDEX);
 
+   // DML get Input by Id
+  public static final String STMT_SELECT_INPUT_BY_ID =
+      "SELECT "
+          + CommonRepoUtils.escapeColumnName(COLUMN_SQI_ID) + ", "
+          + CommonRepoUtils.escapeColumnName(COLUMN_SQI_NAME)
+          + " FROM " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_INPUT_NAME)
+          + " WHERE " + CommonRepoUtils.escapeColumnName(COLUMN_SQI_ID) + " = ?";
+
+  // DML get Input by name
+  public static final String STMT_SELECT_INPUT_BY_NAME =
+      "SELECT "
+          + CommonRepoUtils.escapeColumnName(COLUMN_SQI_ID) + ", "
+          + CommonRepoUtils.escapeColumnName(COLUMN_SQI_NAME)
+          + " FROM " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_INPUT_NAME)
+          + " WHERE " + CommonRepoUtils.escapeColumnName(COLUMN_SQI_NAME) + " = ?";
+
   // DML: Insert into config input
   public static final String STMT_INSERT_INTO_INPUT =
       "INSERT INTO " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_INPUT_NAME) + " ("
@@ -143,8 +175,22 @@ public class CommonRepositoryInsertUpdateDeleteSelectQuery {
           + CommonRepoUtils.escapeColumnName(COLUMN_SQI_TYPE) + ", "
           + CommonRepoUtils.escapeColumnName(COLUMN_SQI_STRMASK) + ", "
           + CommonRepoUtils.escapeColumnName(COLUMN_SQI_STRLENGTH) + ", "
+          + CommonRepoUtils.escapeColumnName(COLUMN_SQI_EDITABLE) + ", "
           + CommonRepoUtils.escapeColumnName(COLUMN_SQI_ENUMVALS)
-          + ") VALUES (?, ?, ?, ?, ?, ?, ?)";
+          + ") VALUES (?, ?, ?, ?, ?, ?, ?, ?)";
+
+/********** INPUT-RELATIONSHIP TABLE **************/
+  public static final String STMT_INSERT_INTO_INPUT_RELATION =
+     "INSERT INTO " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_INPUT_RELATION_NAME) + " ("
+         + CommonRepoUtils.escapeColumnName(COLUMN_SQIR_PARENT) + ", "
+         + CommonRepoUtils.escapeColumnName(COLUMN_SQIR_CHILD)
+         + ") VALUES (?, ?)";
+
+  public static final String STMT_FETCH_SQ_INPUT_OVERRIDES =
+      "SELECT "
+          + CommonRepoUtils.escapeColumnName(COLUMN_SQIR_CHILD)
+          + " FROM " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_INPUT_RELATION_NAME)
+          + " WHERE " + CommonRepoUtils.escapeColumnName(COLUMN_SQIR_PARENT) + " = ?";
 
   /**
    * *******LINK INPUT TABLE *************
@@ -159,6 +205,7 @@ public class CommonRepositoryInsertUpdateDeleteSelectQuery {
           + CommonRepoUtils.escapeColumnName(COLUMN_SQI_TYPE) + ", "
           + CommonRepoUtils.escapeColumnName(COLUMN_SQI_STRMASK) + ", "
           + CommonRepoUtils.escapeColumnName(COLUMN_SQI_STRLENGTH) + ","
+          + CommonRepoUtils.escapeColumnName(COLUMN_SQI_EDITABLE) + ", "
           + CommonRepoUtils.escapeColumnName(COLUMN_SQI_ENUMVALS) + ", "
           + CommonRepoUtils.escapeColumnName(COLUMN_SQ_LNKI_VALUE)
           + " FROM " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_INPUT_NAME)
@@ -182,6 +229,7 @@ public class CommonRepositoryInsertUpdateDeleteSelectQuery {
           + CommonRepoUtils.escapeColumnName(COLUMN_SQI_TYPE) + ", "
           + CommonRepoUtils.escapeColumnName(COLUMN_SQI_STRMASK) + ", "
           + CommonRepoUtils.escapeColumnName(COLUMN_SQI_STRLENGTH) + ", "
+          + CommonRepoUtils.escapeColumnName(COLUMN_SQI_EDITABLE) + ", "
           + CommonRepoUtils.escapeColumnName(COLUMN_SQI_ENUMVALS) + ", "
           + CommonRepoUtils.escapeColumnName(COLUMN_SQBI_VALUE)
           + " FROM " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_INPUT_NAME)
@@ -607,6 +655,18 @@ public class CommonRepositoryInsertUpdateDeleteSelectQuery {
     return STMT_SELECT_INPUT;
   }
 
+  public String getStmtSelectInputById() {
+    return STMT_SELECT_INPUT_BY_ID;
+  }
+
+  public String getStmtSelectInputByName() {
+    return STMT_SELECT_INPUT_BY_NAME;
+  }
+
+  public String getStmtSelectInputOverrides() {
+    return STMT_FETCH_SQ_INPUT_OVERRIDES;
+  }
+
   public String getStmtInsertIntoInput() {
     return STMT_INSERT_INTO_INPUT;
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/repository/repository-common/src/main/java/org/apache/sqoop/repository/common/CommonRepositorySchemaConstants.java
----------------------------------------------------------------------
diff --git a/repository/repository-common/src/main/java/org/apache/sqoop/repository/common/CommonRepositorySchemaConstants.java b/repository/repository-common/src/main/java/org/apache/sqoop/repository/common/CommonRepositorySchemaConstants.java
index 4ab07b2..5ce9488 100644
--- a/repository/repository-common/src/main/java/org/apache/sqoop/repository/common/CommonRepositorySchemaConstants.java
+++ b/repository/repository-common/src/main/java/org/apache/sqoop/repository/common/CommonRepositorySchemaConstants.java
@@ -117,6 +117,19 @@ public final class CommonRepositorySchemaConstants {
 
   public static final String COLUMN_SQI_ENUMVALS = "SQI_ENUMVALS";
 
+  public static final String COLUMN_SQI_EDITABLE = "SQI_EDITABLE";
+
+  // SQ_INPUT_RELATION
+
+  public static final String TABLE_SQ_INPUT_RELATION_NAME = "SQ_INPUT_RELATION";
+
+  public static final String COLUMN_SQIR_ID = "SQIR_ID";
+
+  public static final String COLUMN_SQIR_PARENT = "SQIR_PARENT_ID";
+
+  public static final String COLUMN_SQIR_CHILD = "SQIR_CHILD_ID";
+
+
   public static final String TABLE_SQ_LINK_NAME = "SQ_LINK";
 
   public static final String TABLE_SQ_LINK = SCHEMA_PREFIX + TABLE_SQ_LINK_NAME;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbyRepositoryHandler.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbyRepositoryHandler.java b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbyRepositoryHandler.java
index 2f05fcb..c991e5c 100644
--- a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbyRepositoryHandler.java
+++ b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbyRepositoryHandler.java
@@ -293,10 +293,13 @@ public class DerbyRepositoryHandler extends CommonRepositoryHandler {
     if (repositoryVersion < 5) {
       runQuery(QUERY_UPGRADE_TABLE_SQ_CONFIG_ADD_UNIQUE_CONSTRAINT_NAME_TYPE_AND_CONFIGURABLE_ID, conn);
       runQuery(QUERY_UPGRADE_TABLE_SQ_INPUT_ADD_UNIQUE_CONSTRAINT_NAME_TYPE_AND_CONFIG_ID, conn);
-      // table column rename
+      // submission table column rename
       runQuery(QUERY_UPGRADE_RENAME_TABLE_SQ_JOB_SUBMISSION_COLUMN_1, conn);
       runQuery(QUERY_UPGRADE_RENAME_TABLE_SQ_JOB_SUBMISSION_COLUMN_2, conn);
-
+      // SQOOP-1804
+      runQuery(QUERY_UPGRADE_TABLE_SQ_INPUT_ADD_COLUMN_SQI_EDITABLE, conn);
+      // create a new table for SQ_INPUT relationships
+      runQuery(QUERY_CREATE_TABLE_SQ_INPUT_RELATION, conn);
     }
 
     // last step upgrade the repository version to the latest value in the code

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaConstants.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaConstants.java b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaConstants.java
index a551094..8504091 100644
--- a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaConstants.java
+++ b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaConstants.java
@@ -160,6 +160,15 @@ public final class DerbySchemaConstants {
   public static final String CONSTRAINT_SQ_INPUT_UNIQUE = SCHEMA_PREFIX
       + CONSTRAINT_SQ_INPUT_UNIQUE_NAME_TYPE_CONFIG;
 
+  // SQ_INPUT_RELATION FK
+
+  public static final String CONSTRAINT_SQIR_PARENT_NAME = CONSTRAINT_PREFIX + "SQIR_PARENT_ID";
+  public static final String CONSTRAINT_SQIR_PARENT = SCHEMA_PREFIX + CONSTRAINT_SQIR_PARENT_NAME;
+
+  public static final String CONSTRAINT_SQIR_CHILD_NAME = CONSTRAINT_PREFIX + "SQIR_CHILD_ID";
+  public static final String CONSTRAINT_SQIR_CHILD = SCHEMA_PREFIX + CONSTRAINT_SQIR_CHILD_NAME;
+
+
   // SQ_LINK
   @Deprecated
   // used only for upgrade

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaCreateQuery.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaCreateQuery.java b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaCreateQuery.java
index ad80797..d501a5d 100644
--- a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaCreateQuery.java
+++ b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaCreateQuery.java
@@ -122,6 +122,19 @@ import static org.apache.sqoop.repository.derby.DerbySchemaConstants.*;
  *    | SQI_STRMASK: BOOLEAN       |
  *    | SQI_STRLENGTH: SMALLINT    |
  *    | SQI_ENUMVALS: VARCHAR(100) |
+ *    | SQI_EDITABLE: VARCHAR(32)  |
+ *    +----------------------------+
+ * </pre>
+* <p>
+ * <strong>SQ_INPUT_RELATION</strong>: Input to Input relationship
+ *
+ * <pre>
+ *    +----------------------------+
+ *    | SQ_INPUT_RELATION           |
+ *    +----------------------------+
+ *    | SQIR_ID: BIGINT PK AUTO-GEN |
+ *    | SQIR_PARENT_ID: BIGINT      |FK SQ_INPUT(SQI_ID)
+ *    | SQIR_CHILD_ID: BIGINT       |FK SQ_INPUT(SQI_ID)
  *    +----------------------------+
  * </pre>
  *
@@ -369,6 +382,20 @@ public final class DerbySchemaCreateQuery {
           + "REFERENCES " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_FORM_NAME) + " (" + CommonRepoUtils.escapeColumnName(COLUMN_SQF_ID) + ")"
       + ")";
 
+    // DDL : Create table SQ_INPUT_RELATION
+    public static final String QUERY_CREATE_TABLE_SQ_INPUT_RELATION =
+        "CREATE TABLE " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_INPUT_RELATION_NAME) + " ("
+        + CommonRepoUtils.escapeColumnName(COLUMN_SQIR_ID) + " BIGINT GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1) PRIMARY KEY, "
+        + CommonRepoUtils.escapeColumnName(COLUMN_SQIR_PARENT) + " BIGINT, "
+        + CommonRepoUtils.escapeColumnName(COLUMN_SQIR_CHILD) + " BIGINT, "
+        + "CONSTRAINT " + CONSTRAINT_SQIR_PARENT + " "
+          + "FOREIGN KEY (" + CommonRepoUtils.escapeColumnName(COLUMN_SQIR_PARENT) + ") "
+            + "REFERENCES " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_INPUT_NAME) + " (" + CommonRepoUtils.escapeColumnName(COLUMN_SQI_ID) + "),"
+        + "CONSTRAINT " + CONSTRAINT_SQIR_CHILD + " "
+          + "FOREIGN KEY (" + CommonRepoUtils.escapeColumnName(COLUMN_SQIR_CHILD) + ") "
+            + "REFERENCES " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_INPUT_NAME) + " (" + CommonRepoUtils.escapeColumnName(COLUMN_SQI_ID) + ")"
+        + ")";
+
   // DDL: Create table SQ_CONNECTION
   public static final String QUERY_CREATE_TABLE_SQ_CONNECTION =
       "CREATE TABLE " + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_CONNECTION_NAME) + " ("

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaUpgradeQuery.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaUpgradeQuery.java b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaUpgradeQuery.java
index fa6710b..606fc68 100644
--- a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaUpgradeQuery.java
+++ b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaUpgradeQuery.java
@@ -510,7 +510,12 @@ public final class DerbySchemaUpgradeQuery {
   public static final String QUERY_UPGRADE_RENAME_TABLE_SQ_JOB_SUBMISSION_COLUMN_2 = "RENAME COLUMN "
       + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_SUBMISSION_NAME) + "." + CommonRepoUtils.escapeColumnName(COLUMN_SQS_EXCEPTION_TRACE) + " TO " + CommonRepoUtils.escapeColumnName(COLUMN_SQS_ERROR_DETAILS);
 
-  private DerbySchemaUpgradeQuery() {
+  // SQOOP-1804, column add for SQ_INPUT
+  public static final String QUERY_UPGRADE_TABLE_SQ_INPUT_ADD_COLUMN_SQI_EDITABLE = "ALTER TABLE "
+      + CommonRepoUtils.getTableName(SCHEMA_SQOOP, TABLE_SQ_INPUT_NAME) + " ADD COLUMN "
+      + CommonRepoUtils.escapeColumnName(COLUMN_SQI_EDITABLE) + " VARCHAR(32)";
+
+    private DerbySchemaUpgradeQuery() {
     // Disable explicit object creation
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/DerbyTestCase.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/DerbyTestCase.java b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/DerbyTestCase.java
index be8c23e..bea5cd7 100644
--- a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/DerbyTestCase.java
+++ b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/DerbyTestCase.java
@@ -32,8 +32,10 @@ import java.sql.Statement;
 import java.util.LinkedList;
 import java.util.List;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.sqoop.common.Direction;
 import org.apache.sqoop.json.DriverBean;
+import org.apache.sqoop.model.InputEditable;
 import org.apache.sqoop.model.MConfig;
 import org.apache.sqoop.model.MConnector;
 import org.apache.sqoop.model.MDriver;
@@ -150,10 +152,10 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * Create derby schema. FIX(SQOOP-1583): This code needs heavy refactoring.
-   * Details are in the ticket.
+   *Create derby schema. FIX(SQOOP-1583): This code needs heavy refactoring.
+   *Details are in the ticket.
    *
-   * @throws Exception
+   *@throws Exception
    */
   protected void createOrUpgradeSchema(int version) throws Exception {
     if (version > 0) {
@@ -215,6 +217,9 @@ abstract public class DerbyTestCase {
       // add submission table column name renames
       runQuery(QUERY_UPGRADE_RENAME_TABLE_SQ_JOB_SUBMISSION_COLUMN_1);
       runQuery(QUERY_UPGRADE_RENAME_TABLE_SQ_JOB_SUBMISSION_COLUMN_2);
+      // SQOOP-1804
+      runQuery(QUERY_UPGRADE_TABLE_SQ_INPUT_ADD_COLUMN_SQI_EDITABLE);
+      runQuery(QUERY_CREATE_TABLE_SQ_INPUT_RELATION);
     }
 
     // deprecated repository version
@@ -230,9 +235,11 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * Run arbitrary query on derby memory repository.
-   * @param query Query to execute
-   * @throws Exception
+   *Run arbitrary query on derby memory repository.
+   *
+   *@param query
+   *         Query to execute
+   *@throws Exception
    */
   protected void runQuery(String query, Object... args) throws Exception {
     PreparedStatement stmt = null;
@@ -258,11 +265,12 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * Run single, arbitrary insert query on derby memory repository.
+   *Run single, arbitrary insert query on derby memory repository.
    *
-   * @param query Query to execute
-   * @return Long id of newly inserted row (-1 if none).
-   * @throws Exception
+   *@param query
+   *         Query to execute
+   *@return Long id of newly inserted row (-1 if none).
+   *@throws Exception
    */
   protected Long runInsertQuery(String query, Object... args) throws Exception {
     PreparedStatement stmt = null;
@@ -431,19 +439,19 @@ abstract public class DerbyTestCase {
     for (int i = 0; i < 3; i++) {
       // First config
       runInsertQuery("INSERT INTO SQOOP.SQ_INPUT"
-          + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH)"
-          + " VALUES('I1', " + (i * 2 + 1) + ", 0, 'STRING', false, 30)");
+          + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH, SQI_EDITABLE)"
+          + " VALUES('I1', " + (i * 2 + 1) + ", 0, 'STRING', false, 30, 'CONNECTOR_ONLY')");
       runInsertQuery("INSERT INTO SQOOP.SQ_INPUT"
-          + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH)"
-          + " VALUES('I2', " + (i * 2 + 1) + ", 1, 'MAP', false, 30)");
+          + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH, SQI_EDITABLE)"
+          + " VALUES('I2', " + (i * 2 + 1) + ", 1, 'MAP', false, 30, 'CONNECTOR_ONLY')");
 
       // Second config
       runInsertQuery("INSERT INTO SQOOP.SQ_INPUT"
-          + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH)"
-          + " VALUES('I3', " + (i * 2 + 2) + ", 0, 'STRING', false, 30)");
+          + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH, SQI_EDITABLE)"
+          + " VALUES('I3', " + (i * 2 + 2) + ", 0, 'STRING', false, 30, 'CONNECTOR_ONLY')");
       runInsertQuery("INSERT INTO SQOOP.SQ_INPUT"
-          + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH)"
-          + " VALUES('I4', " + (i * 2 + 2) + ", 1, 'MAP', false, 30)");
+          + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH, SQI_EDITABLE)"
+          + " VALUES('I4', " + (i * 2 + 2) + ", 1, 'MAP', false, 30, 'CONNECTOR_ONLY')");
     }
 
   }
@@ -517,28 +525,28 @@ abstract public class DerbyTestCase {
     for (int i = 0; i < 4; i++) {
       // First config
       runInsertQuery("INSERT INTO SQOOP.SQ_INPUT"
-          + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH)"
-          + " VALUES('I1', " + (i * 2 + 1) + ", 0, 'STRING', false, 30)");
+          + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH, SQI_EDITABLE)"
+          + " VALUES('I1', " + (i * 2 + 1) + ", 0, 'STRING', false, 30, 'CONNECTOR_ONLY')");
       runInsertQuery("INSERT INTO SQOOP.SQ_INPUT"
-          + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH)"
-          + " VALUES('I2', " + (i * 2 + 1) + ", 1, 'MAP', false, 30)");
+          + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH, SQI_EDITABLE)"
+          + " VALUES('I2', " + (i * 2 + 1) + ", 1, 'MAP', false, 30, 'CONNECTOR_ONLY')");
 
       // Second config
       runInsertQuery("INSERT INTO SQOOP.SQ_INPUT"
-          + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH)"
-          + " VALUES('I3', " + (i * 2 + 2) + ", 0, 'STRING', false, 30)");
+          + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH, SQI_EDITABLE)"
+          + " VALUES('I3', " + (i * 2 + 2) + ", 0, 'STRING', false, 30, 'CONNECTOR_ONLY')");
       runInsertQuery("INSERT INTO SQOOP.SQ_INPUT"
-          + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH)"
-          + " VALUES('I4', " + (i * 2 + 2) + ", 1, 'MAP', false, 30)");
+          + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH, SQI_EDITABLE)"
+          + " VALUES('I4', " + (i * 2 + 2) + ", 1, 'MAP', false, 30, 'CONNECTOR_ONLY')");
     }
   }
 
   /**
-   * Load testing connector and driver config into repository.
+   *Load testing connector and driver config into repository.
    *
-   * @param version
-   *          system version (2 or 4)
-   * @throws Exception
+   *@param version
+   *         system version (2 or 4)
+   *@throws Exception
    */
   protected void loadConnectorAndDriverConfig(int version) throws Exception {
     switch (version) {
@@ -561,9 +569,11 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * Load testing link objects into repository.
-   * @param version system version (2 or 4)
-   * @throws Exception
+   *Load testing link objects into repository.
+   *
+   *@param version
+   *         system version (2 or 4)
+   *@throws Exception
    */
   public void loadConnectionsOrLinks(int version) throws Exception {
     switch (version) {
@@ -606,9 +616,11 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * Load testing job objects into repository.
-   * @param version system version (2 or 4)
-   * @throws Exception
+   *Load testing job objects into repository.
+   *
+   *@param version
+   *         system version (2 or 4)
+   *@throws Exception
    */
   public void loadJobs(int version) throws Exception {
     int index = 0;
@@ -665,8 +677,9 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * testing job with non unique name objects into repository.
-   * @throws Exception
+   *testing job with non unique name objects into repository.
+   *
+   *@throws Exception
    */
   public void loadNonUniqueJobsInVersion4() throws Exception {
     int index = 0;
@@ -678,8 +691,9 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * testing link with non unique name objects into repository.
-   * @throws Exception
+   *testing link with non unique name objects into repository.
+   *
+   *@throws Exception
    */
   public void loadNonUniqueLinksInVersion4() throws Exception {
 
@@ -690,8 +704,9 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * testing configurable with non unique name objects into repository.
-   * @throws Exception
+   *testing configurable with non unique name objects into repository.
+   *
+   *@throws Exception
    */
   public void loadNonUniqueConfigurablesInVersion4() throws Exception {
 
@@ -704,8 +719,9 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * testing config with non unique name/type objects into repository.
-   * @throws Exception
+   *testing config with non unique name/type objects into repository.
+   *
+   *@throws Exception
    */
   public void loadNonUniqueConfigNameTypeInVersion4() throws Exception {
 
@@ -718,8 +734,9 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * testing input with non unique name/type objects into repository.
-   * @throws Exception
+   *testing input with non unique name/type objects into repository.
+   *
+   *@throws Exception
    */
   public void loadNonUniqueInputNameTypeInVersion4() throws Exception {
 
@@ -727,16 +744,17 @@ abstract public class DerbyTestCase {
         + "(SQ_CFG_CONFIGURABLE, SQ_CFG_NAME, SQ_CFG_TYPE, SQ_CFG_INDEX) "
         + "VALUES(1, 'C1', 'LINK', 0)");
     runInsertQuery("INSERT INTO SQOOP.SQ_INPUT"
-        + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH)"
-        + " VALUES('I1', 1, 0, 'STRING', false, 30)");
+        + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH,SQI_EDITABLE)"
+        + " VALUES('I1', 1, 0, 'STRING', false, 30, 'CONNECTOR_ONLY')");
     runInsertQuery("INSERT INTO SQOOP.SQ_INPUT"
-        + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH)"
-        + " VALUES('I1', 1, 0, 'STRING', false, 30)");
+        + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH, SQI_EDITABLE)"
+        + " VALUES('I1', 1, 0, 'STRING', false, 30, 'CONNECTOR_ONLY')");
   }
 
   /**
-   * testing config with non unique name/type objects into repository.
-   * @throws Exception
+   *testing config with non unique name/type objects into repository.
+   *
+   *@throws Exception
    */
   public void loadNonUniqueConfigNameButUniqueTypeInVersion4() throws Exception {
 
@@ -749,8 +767,9 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * testing config with non unique name/type objects into repository.
-   * @throws Exception
+   *testing config with non unique name/type objects into repository.
+   *
+   *@throws Exception
    */
   public void loadNonUniqueConfigNameAndTypeButUniqueConfigurableInVersion4() throws Exception {
 
@@ -763,8 +782,9 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * testing input with non unique name/type objects into repository.
-   * @throws Exception
+   *testing input with non unique name/type objects into repository.
+   *
+   *@throws Exception
    */
   public void loadNonUniqueInputNameAndTypeButUniqueConfigInVersion4() throws Exception {
 
@@ -776,11 +796,11 @@ abstract public class DerbyTestCase {
         + "VALUES(2, 'C2', 'LINK', 0)");
 
     runInsertQuery("INSERT INTO SQOOP.SQ_INPUT"
-        + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH)"
-        + " VALUES('C1.A', 1, 0, 'STRING', false, 30)");
+        + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH, SQI_EDITABLE)"
+        + " VALUES('C1.A', 1, 0, 'STRING', false, 30, 'ANY')");
     runInsertQuery("INSERT INTO SQOOP.SQ_INPUT"
-        + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH)"
-        + " VALUES('C1.A', 2, 1, 'STRING', false, 30)");
+        + "(SQI_NAME, SQI_CONFIG, SQI_INDEX, SQI_TYPE, SQI_STRMASK, SQI_STRLENGTH, SQI_EDITABLE)"
+        + " VALUES('C1.A', 2, 1, 'STRING', false, 30, 'ANY')");
   }
 
   public void loadJobsForLatestVersion() throws Exception {
@@ -788,7 +808,7 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * Add a second connector for testing with multiple connectors
+   *Add a second connector for testing with multiple connectors
    */
   public void addConnectorB() throws Exception {
     // Connector entry
@@ -801,8 +821,9 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * Load testing submissions into the repository.
-   * @throws Exception
+   *Load testing submissions into the repository.
+   *
+   *@throws Exception
    */
   public void loadSubmissions() throws Exception {
     runQuery("INSERT INTO SQOOP.SQ_COUNTER_GROUP " + "(SQG_NAME) " + "VALUES" + "('gA'), ('gB')");
@@ -832,6 +853,57 @@ abstract public class DerbyTestCase {
     return getConnector(true, true);
   }
 
+  protected MConnector getConnectorWithIncorrectOverridesAttribute() {
+    return getBadConnector(true, true);
+  }
+
+  protected MConnector getConnectorWithIncorrectOverridesAttribute2() {
+    return getBadConnector2(true, true);
+  }
+
+  protected MConnector getConnectorWithMultipleOverridesAttribute() {
+    return getConnectorWithMultipleOverrides(true, true);
+  }
+
+  protected MConnector getConnectorWithMultipleOverrides(boolean from, boolean to) {
+    MFromConfig fromConfig = null;
+    MToConfig toConfig = null;
+    if (from) {
+      fromConfig = getMultipleOverridesFromConfig();
+    }
+    if (to) {
+      toConfig = getToConfig();
+    }
+    return new MConnector("A", "org.apache.sqoop.test.A", "1.0-test", getLinkConfig(), fromConfig,
+        toConfig);
+  }
+
+  protected MConnector getBadConnector(boolean from, boolean to) {
+    MFromConfig fromConfig = null;
+    MToConfig toConfig = null;
+    if (from) {
+      fromConfig = getBadFromConfig();
+    }
+    if (to) {
+      toConfig = getToConfig();
+    }
+    return new MConnector("A", "org.apache.sqoop.test.A", "1.0-test", getLinkConfig(), fromConfig,
+        toConfig);
+  }
+
+  protected MConnector getBadConnector2(boolean from, boolean to) {
+    MFromConfig fromConfig = null;
+    MToConfig toConfig = null;
+    if (from) {
+      fromConfig = getNonExistentOverridesFromConfig();
+    }
+    if (to) {
+      toConfig = getToConfig();
+    }
+    return new MConnector("A", "org.apache.sqoop.test.A", "1.0-test", getLinkConfig(), fromConfig,
+        toConfig);
+  }
+
   protected MConnector getConnector(boolean from, boolean to) {
     MFromConfig fromConfig = null;
     MToConfig toConfig = null;
@@ -849,6 +921,10 @@ abstract public class DerbyTestCase {
     return new MDriver(getDriverConfig(), DriverBean.CURRENT_DRIVER_VERSION);
   }
 
+  protected MDriver getBadDriver() {
+    return new MDriver(getBadDriverConfig(), DriverBean.CURRENT_DRIVER_VERSION);
+  }
+
   protected void fillLink(MLink link) {
     List<MConfig> configs = link.getConnectorLinkConfig().getConfigs();
     ((MStringInput) configs.get(0).getInputs().get(0)).setValue("Value1");
@@ -877,6 +953,18 @@ abstract public class DerbyTestCase {
     return new MFromConfig(getConfigs("from1", "from2"));
   }
 
+  protected MFromConfig getBadFromConfig() {
+    return new MFromConfig(getBadConfigs("from1", "from2"));
+  }
+
+  protected MFromConfig getMultipleOverridesFromConfig() {
+    return new MFromConfig(getMultipleOverrideConfigs("from1", "from2"));
+  }
+
+  protected MFromConfig getNonExistentOverridesFromConfig() {
+    return new MFromConfig(getBadConfigsWithNonExistingInputOverrides("from1", "from2"));
+  }
+
   protected MToConfig getToConfig() {
     return new MToConfig(getConfigs("to1", "to2"));
   }
@@ -885,20 +973,111 @@ abstract public class DerbyTestCase {
     return new MDriverConfig(getConfigs("d1", "d2"));
   }
 
+  protected MDriverConfig getBadDriverConfig() {
+    return new MDriverConfig(getBadConfigsWithSelfOverrides("d1", "d2"));
+  }
+
   protected List<MConfig> getConfigs(String configName1, String configName2) {
     List<MConfig> configs = new LinkedList<MConfig>();
 
     List<MInput<?>> inputs = new LinkedList<MInput<?>>();
-    MInput input = new MStringInput("I1", false, (short) 30);
+    MInput input = new MStringInput("I1", false, InputEditable.ANY, "I2", (short) 30);
+    inputs.add(input);
+    input = new MMapInput("I2", false, InputEditable.ANY, StringUtils.EMPTY);
+    inputs.add(input);
+    configs.add(new MConfig(configName1, inputs));
+
+    inputs = new LinkedList<MInput<?>>();
+    input = new MStringInput("I3", false, InputEditable.ANY, "I4", (short) 30);
+    inputs.add(input);
+    input = new MMapInput("I4", false, InputEditable.USER_ONLY, "I3");
+    inputs.add(input);
+    configs.add(new MConfig(configName2, inputs));
+
+    return configs;
+  }
+
+  protected List<MConfig> getBadConfigs(String configName1, String configName2) {
+    List<MConfig> configs = new LinkedList<MConfig>();
+
+    List<MInput<?>> inputs = new LinkedList<MInput<?>>();
+    // I1 overrides another user_only attribute, hence a bad config
+    MInput input = new MStringInput("I1", false, InputEditable.USER_ONLY, "I2", (short) 30);
+    inputs.add(input);
+    input = new MMapInput("I2", false, InputEditable.USER_ONLY, "I1");
+    inputs.add(input);
+    configs.add(new MConfig(configName1, inputs));
+
+    inputs = new LinkedList<MInput<?>>();
+    input = new MStringInput("I3", false, InputEditable.ANY, StringUtils.EMPTY, (short) 30);
+    inputs.add(input);
+    input = new MMapInput("I4", false, InputEditable.ANY, StringUtils.EMPTY);
+    inputs.add(input);
+    configs.add(new MConfig(configName2, inputs));
+
+    return configs;
+  }
+
+  protected List<MConfig> getBadConfigsWithSelfOverrides(String configName1, String configName2) {
+    List<MConfig> configs = new LinkedList<MConfig>();
+
+    List<MInput<?>> inputs = new LinkedList<MInput<?>>();
+    // I1 overrides another user_only attribute, hence a bad config
+    MInput input = new MStringInput("I1", false, InputEditable.USER_ONLY, "I2", (short) 30);
     inputs.add(input);
-    input = new MMapInput("I2", false);
+    input = new MMapInput("I2", false, InputEditable.USER_ONLY, "I2");
     inputs.add(input);
     configs.add(new MConfig(configName1, inputs));
 
     inputs = new LinkedList<MInput<?>>();
-    input = new MStringInput("I3", false, (short) 30);
+    input = new MStringInput("I3", false, InputEditable.ANY, StringUtils.EMPTY, (short) 30);
     inputs.add(input);
-    input = new MMapInput("I4", false);
+    input = new MMapInput("I4", false, InputEditable.ANY, StringUtils.EMPTY);
+    inputs.add(input);
+    configs.add(new MConfig(configName2, inputs));
+
+    return configs;
+  }
+
+  protected List<MConfig> getMultipleOverrideConfigs(String configName1, String configName2) {
+    List<MConfig> configs = new LinkedList<MConfig>();
+
+    List<MInput<?>> inputs = new LinkedList<MInput<?>>();
+    // I1 overrides another user_only attribute, hence a bad config
+    MInput input = new MStringInput("I1", false, InputEditable.USER_ONLY, "I2", (short) 30);
+    inputs.add(input);
+    input = new MMapInput("I2", false, InputEditable.ANY, "I1,I3");
+    inputs.add(input);
+    input = new MMapInput("I3", false, InputEditable.CONNECTOR_ONLY, "I1");
+    inputs.add(input);
+    configs.add(new MConfig(configName1, inputs));
+
+    inputs = new LinkedList<MInput<?>>();
+    input = new MStringInput("I3", false, InputEditable.ANY, StringUtils.EMPTY, (short) 30);
+    inputs.add(input);
+    input = new MMapInput("I4", false, InputEditable.ANY, StringUtils.EMPTY);
+    inputs.add(input);
+    configs.add(new MConfig(configName2, inputs));
+
+    return configs;
+  }
+
+  protected List<MConfig> getBadConfigsWithNonExistingInputOverrides(String configName1,
+      String configName2) {
+    List<MConfig> configs = new LinkedList<MConfig>();
+
+    List<MInput<?>> inputs = new LinkedList<MInput<?>>();
+    // I1 overrides another user_only attribute, hence a bad config
+    MInput input = new MStringInput("I1", false, InputEditable.USER_ONLY, "I2", (short) 30);
+    inputs.add(input);
+    input = new MMapInput("I2", false, InputEditable.USER_ONLY, "Foo");
+    inputs.add(input);
+    configs.add(new MConfig(configName1, inputs));
+
+    inputs = new LinkedList<MInput<?>>();
+    input = new MStringInput("I3", false, InputEditable.ANY, StringUtils.EMPTY, (short) 30);
+    inputs.add(input);
+    input = new MMapInput("I4", false, InputEditable.ANY, StringUtils.EMPTY);
     inputs.add(input);
     configs.add(new MConfig(configName2, inputs));
 
@@ -906,11 +1085,12 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * Find out number of entries in given table.
+   *Find out number of entries in given table.
    *
-   * @param table Table name
-   * @return Number of rows in the table
-   * @throws Exception
+   *@param table
+   *         Table name
+   *@return Number of rows in the table
+   *@throws Exception
    */
   protected long countForTable(String table) throws Exception {
     Statement stmt = null;
@@ -934,10 +1114,13 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * Assert row count for given table.
-   * @param table Table name
-   * @param expected Expected number of rows
-   * @throws Exception
+   *Assert row count for given table.
+   *
+   *@param table
+   *         Table name
+   *@param expected
+   *         Expected number of rows
+   *@throws Exception
    */
   protected void assertCountForTable(String table, long expected) throws Exception {
     long count = countForTable(table);
@@ -945,12 +1128,12 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * Printout repository content for advance debugging.
+   *Printout repository content for advance debugging.
    *
-   * This method is currently unused, but might be helpful in the future, so I'm
-   * letting it here.
+   *This method is currently unused, but might be helpful in the future, so I'm
+   *letting it here.
    *
-   * @throws Exception
+   *@throws Exception
    */
   protected void generateDatabaseState() throws Exception {
     for (String tbl : new String[] { "SQ_CONNECTOR", "SQ_CONFIG", "SQ_INPUT", "SQ_LINK",
@@ -960,9 +1143,11 @@ abstract public class DerbyTestCase {
   }
 
   /**
-   * Printout one single table.
-   * @param table Table name
-   * @throws Exception
+   *Printout one single table.
+   *
+   *@param table
+   *         Table name
+   *@throws Exception
    */
   protected void generateTableState(String table) throws Exception {
     PreparedStatement ps = null;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestConnectorHandling.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestConnectorHandling.java b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestConnectorHandling.java
index ca24398..1056c69 100644
--- a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestConnectorHandling.java
+++ b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestConnectorHandling.java
@@ -17,6 +17,7 @@
  */
 package org.apache.sqoop.repository.derby;
 
+import org.apache.sqoop.common.SqoopException;
 import org.apache.sqoop.model.MConnector;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
@@ -91,10 +92,27 @@ public class TestConnectorHandling extends DerbyTestCase {
     assertEquals(connector, retrieved);
   }
 
+  @Test(expectedExceptions = SqoopException.class)
+  public void testRegisterConnectorWithIncorrectInputOverridesAttribute() throws Exception {
+    MConnector connector = getConnectorWithIncorrectOverridesAttribute();
+    handler.registerConnector(connector, getDerbyDatabaseConnection());
+  }
+
+  @Test(expectedExceptions = SqoopException.class)
+  public void testRegisterConnectorWithIncorrectInputOverridesAttribute2() throws Exception {
+    MConnector connector = getConnectorWithIncorrectOverridesAttribute2();
+    handler.registerConnector(connector, getDerbyDatabaseConnection());
+  }
+
+  @Test
+  public void testRegisterConnectorWithMultipleInputOverridesAttribute() throws Exception {
+    MConnector connector = getConnectorWithMultipleOverridesAttribute();
+    handler.registerConnector(connector, getDerbyDatabaseConnection());
+  }
+
   @Test
   public void testFromDirection() throws Exception {
     MConnector connector = getConnector(true, false);
-
     handler.registerConnector(connector, getDerbyDatabaseConnection());
 
     // Connector should get persistence ID

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestDriverHandling.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestDriverHandling.java b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestDriverHandling.java
index e12bf46..a2c50f8 100644
--- a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestDriverHandling.java
+++ b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestDriverHandling.java
@@ -21,6 +21,7 @@ import static org.testng.AssertJUnit.assertEquals;
 import static org.testng.Assert.assertNotNull;
 import static org.testng.Assert.assertNull;
 
+import org.apache.sqoop.common.SqoopException;
 import org.apache.sqoop.model.MDriver;
 import org.apache.sqoop.model.MDriverConfig;
 import org.testng.annotations.BeforeMethod;
@@ -82,6 +83,11 @@ public class TestDriverHandling extends DerbyTestCase {
     assertEquals(driver.getVersion(), retrieved.getVersion());
   }
 
+  @Test(expectedExceptions = SqoopException.class)
+  public void testRegisterBadDriver() throws Exception {
+    MDriver driver = getBadDriver();
+    handler.registerDriver(driver, getDerbyDatabaseConnection());
+  }
 
   @Test
   public void testDriverVersionUpgrade() throws Exception {

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestInputTypes.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestInputTypes.java b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestInputTypes.java
index fb07152..fe872ca 100644
--- a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestInputTypes.java
+++ b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestInputTypes.java
@@ -26,6 +26,8 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.commons.lang.StringUtils;
+import org.apache.sqoop.model.InputEditable;
 import org.apache.sqoop.model.MBooleanInput;
 import org.apache.sqoop.model.MConfig;
 import org.apache.sqoop.model.MConnector;
@@ -99,11 +101,22 @@ public class TestInputTypes extends DerbyTestCase {
     // Connection object with all various values
     MLink link = new MLink(connector.getPersistenceId(), connector.getLinkConfig());
     MLinkConfig linkConfig = link.getConnectorLinkConfig();
-    linkConfig.getStringInput("l1.String").setValue("A");
-    linkConfig.getMapInput("l1.Map").setValue(map);
-    linkConfig.getIntegerInput("l1.Integer").setValue(1);
-    linkConfig.getBooleanInput("l1.Boolean").setValue(true);
-    linkConfig.getEnumInput("l1.Enum").setValue("YES");
+    assertEquals(linkConfig.getStringInput("l1.I1").getEditable(), InputEditable.ANY);
+    assertEquals(linkConfig.getStringInput("l1.I1").getOverrides(), "l1.I2");
+    assertEquals(linkConfig.getMapInput("l1.I2").getEditable(), InputEditable.CONNECTOR_ONLY);
+    assertEquals(linkConfig.getMapInput("l1.I2").getOverrides(), "l1.I5");
+    assertEquals(linkConfig.getIntegerInput("l1.I3").getEditable(), InputEditable.ANY);
+    assertEquals(linkConfig.getIntegerInput("l1.I3").getOverrides(), "l1.I1");
+    assertEquals(linkConfig.getBooleanInput("l1.I4").getEditable(), InputEditable.USER_ONLY);
+    assertEquals(linkConfig.getBooleanInput("l1.I4").getOverrides(), "");
+    assertEquals(linkConfig.getEnumInput("l1.I5").getEditable(), InputEditable.ANY);
+    assertEquals(linkConfig.getEnumInput("l1.I5").getOverrides(), "l1.I4,l1.I3");
+
+    linkConfig.getStringInput("l1.I1").setValue("A");
+    linkConfig.getMapInput("l1.I2").setValue(map);
+    linkConfig.getIntegerInput("l1.I3").setValue(1);
+    linkConfig.getBooleanInput("l1.I4").setValue(true);
+    linkConfig.getEnumInput("l1.I5").setValue("YES");
 
     // Create the link in repository
     handler.createLink(link, getDerbyDatabaseConnection());
@@ -112,11 +125,14 @@ public class TestInputTypes extends DerbyTestCase {
     // Retrieve created link
     MLink retrieved = handler.findLink(link.getPersistenceId(), getDerbyDatabaseConnection());
     linkConfig = retrieved.getConnectorLinkConfig();
-    assertEquals("A", linkConfig.getStringInput("l1.String").getValue());
-    assertEquals(map, linkConfig.getMapInput("l1.Map").getValue());
-    assertEquals(1, (int) linkConfig.getIntegerInput("l1.Integer").getValue());
-    assertEquals(true, (boolean) linkConfig.getBooleanInput("l1.Boolean").getValue());
-    assertEquals("YES", linkConfig.getEnumInput("l1.Enum").getValue());
+    assertEquals("A", linkConfig.getStringInput("l1.I1").getValue());
+    assertEquals(map, linkConfig.getMapInput("l1.I2").getValue());
+    assertEquals(1, (int) linkConfig.getIntegerInput("l1.I3").getValue());
+    assertEquals(true, (boolean) linkConfig.getBooleanInput("l1.I4").getValue());
+    assertEquals("YES", linkConfig.getEnumInput("l1.I5").getValue());
+    assertEquals(linkConfig.getEnumInput("l1.I5").getEditable(), InputEditable.ANY);
+    assertEquals(linkConfig.getEnumInput("l1.I5").getOverrides(), "l1.I4,l1.I3");
+
   }
 
   /**
@@ -133,19 +149,23 @@ public class TestInputTypes extends DerbyTestCase {
 
     inputs = new LinkedList<MInput<?>>();
 
-    input = new MStringInput(configName1 + ".String", false, (short) 30);
+    input = new MStringInput(configName1 + ".I1", false, InputEditable.ANY, configName1 + ".I2",
+        (short) 30);
     inputs.add(input);
 
-    input = new MMapInput(configName1 + ".Map", false);
+    input = new MMapInput(configName1 + ".I2", false, InputEditable.CONNECTOR_ONLY, configName1
+        + ".I5");
     inputs.add(input);
 
-    input = new MIntegerInput(configName1 + ".Integer", false);
+    input = new MIntegerInput(configName1 + ".I3", false, InputEditable.ANY, configName1 + ".I1");
     inputs.add(input);
 
-    input = new MBooleanInput(configName1 + ".Boolean", false);
+    input = new MBooleanInput(configName1 + ".I4", false, InputEditable.USER_ONLY,
+        StringUtils.EMPTY);
     inputs.add(input);
 
-    input = new MEnumInput(configName1 + ".Enum", false, new String[] { "YES", "NO" });
+    input = new MEnumInput(configName1 + ".I5", false, InputEditable.ANY, configName1 + ".I4,"
+        + configName1 + ".I3", new String[] { "YES", "NO" });
     inputs.add(input);
 
     configs.add(new MConfig(configName1, inputs));

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/repository/repository-postgresql/src/test/java/org/apache/sqoop/integration/repository/postgresql/PostgresqlTestCase.java
----------------------------------------------------------------------
diff --git a/repository/repository-postgresql/src/test/java/org/apache/sqoop/integration/repository/postgresql/PostgresqlTestCase.java b/repository/repository-postgresql/src/test/java/org/apache/sqoop/integration/repository/postgresql/PostgresqlTestCase.java
index 08a3342..79381d8 100644
--- a/repository/repository-postgresql/src/test/java/org/apache/sqoop/integration/repository/postgresql/PostgresqlTestCase.java
+++ b/repository/repository-postgresql/src/test/java/org/apache/sqoop/integration/repository/postgresql/PostgresqlTestCase.java
@@ -17,6 +17,7 @@
  */
 package org.apache.sqoop.integration.repository.postgresql;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.sqoop.common.Direction;
 import org.apache.sqoop.common.test.db.DatabaseProvider;
 import org.apache.sqoop.common.test.db.PostgreSQLProvider;
@@ -148,16 +149,16 @@ abstract public class PostgresqlTestCase {
     List<MConfig> configs = new LinkedList<MConfig>();
 
     List<MInput<?>> inputs = new LinkedList<MInput<?>>();
-    MInput input = new MStringInput("I1", false, (short) 30);
+    MInput input = new MStringInput("I1", false, InputEditable.ANY, StringUtils.EMPTY, (short) 30);
     inputs.add(input);
-    input = new MMapInput("I2", false);
+    input = new MMapInput("I2", false, InputEditable.ANY, StringUtils.EMPTY);
     inputs.add(input);
     configs.add(new MConfig(configName1, inputs));
 
     inputs = new LinkedList<MInput<?>>();
-    input = new MStringInput("I3", false, (short) 30);
+    input = new MStringInput("I3", false, InputEditable.ANY, StringUtils.EMPTY, (short) 30);
     inputs.add(input);
-    input = new MMapInput("I4", false);
+    input = new MMapInput("I4", false, InputEditable.ANY, StringUtils.EMPTY);
     inputs.add(input);
     configs.add(new MConfig(configName2, inputs));
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/shell/src/main/java/org/apache/sqoop/shell/core/Constants.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/core/Constants.java b/shell/src/main/java/org/apache/sqoop/shell/core/Constants.java
index ca387d8..46f2b7f 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/core/Constants.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/core/Constants.java
@@ -372,6 +372,10 @@ public class Constants {
       "config.displayer.type";
   public static final String RES_CONFIG_DISPLAYER_SENSITIVE =
       "config.displayer.sensitive";
+  public static final String RES_CONFIG_DISPLAYER_EDITABLE =
+      "config.displayer.editable";
+  public static final String RES_CONFIG_DISPLAYER_OVERRIDES =
+      "config.displayer.overrides";
   public static final String RES_CONFIG_DISPLAYER_SIZE =
       "config.displayer.size";
   public static final String RES_CONFIG_DISPLAYER_POSSIBLE_VALUES =

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/shell/src/main/java/org/apache/sqoop/shell/utils/ConfigDisplayer.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/utils/ConfigDisplayer.java b/shell/src/main/java/org/apache/sqoop/shell/utils/ConfigDisplayer.java
index e240163..aeb6d41 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/utils/ConfigDisplayer.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/utils/ConfigDisplayer.java
@@ -115,6 +115,11 @@ public final class ConfigDisplayer {
         println(input.getType());
         print("        %s: ", resourceString(Constants.RES_CONFIG_DISPLAYER_SENSITIVE));
         println(input.isSensitive());
+        print("        %s: ", resourceString(Constants.RES_CONFIG_DISPLAYER_EDITABLE));
+        println(input.getEditable());
+        print("        %s: ", resourceString(Constants.RES_CONFIG_DISPLAYER_OVERRIDES));
+        println(input.getOverrides());
+
         if (input.getType() == MInputType.STRING) {
           print("        %s: ", resourceString(Constants.RES_CONFIG_DISPLAYER_SIZE));
           println(((MStringInput)input).getMaxLength());

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/shell/src/main/java/org/apache/sqoop/shell/utils/ConfigFiller.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/utils/ConfigFiller.java b/shell/src/main/java/org/apache/sqoop/shell/utils/ConfigFiller.java
index de40818..02278f5 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/utils/ConfigFiller.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/utils/ConfigFiller.java
@@ -22,6 +22,7 @@ import jline.ConsoleReader;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.lang.StringUtils;
 import org.apache.sqoop.common.Direction;
+import org.apache.sqoop.model.InputEditable;
 import org.apache.sqoop.model.MBooleanInput;
 import org.apache.sqoop.model.MLink;
 import org.apache.sqoop.model.MEnumInput;
@@ -53,7 +54,7 @@ public final class ConfigFiller {
    * Internal input that will be reused for loading names for link and
    * job objects.
    */
-  private static MStringInput nameInput = new MStringInput("object-name", false, (short)25);
+  private static MStringInput nameInput = new MStringInput("object-name", false, InputEditable.ANY, StringUtils.EMPTY, (short)25);
 
   /**
    * Fill job object based on CLI options.

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/shell/src/main/resources/shell-resource.properties
----------------------------------------------------------------------
diff --git a/shell/src/main/resources/shell-resource.properties b/shell/src/main/resources/shell-resource.properties
index 2b5a9b7..9c88235 100644
--- a/shell/src/main/resources/shell-resource.properties
+++ b/shell/src/main/resources/shell-resource.properties
@@ -211,6 +211,8 @@ config.displayer.help = Help
 config.displayer.input = Input
 config.displayer.type = Type
 config.displayer.sensitive = Sensitive
+config.displayer.editable = Editable By
+config.displayer.overrides = Overrides
 config.displayer.size = Size
 config.displayer.possible_values = Possible values
 config.displayer.unsupported_datatype = Unsupported data type


[2/2] sqoop git commit: SQOOP-1804: Add editable and override attribute to inputs

Posted by ja...@apache.org.
SQOOP-1804: Add editable and override attribute to inputs

(Veena Basavaraj 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/6fc50b08
Tree: http://git-wip-us.apache.org/repos/asf/sqoop/tree/6fc50b08
Diff: http://git-wip-us.apache.org/repos/asf/sqoop/diff/6fc50b08

Branch: refs/heads/sqoop2
Commit: 6fc50b08b0b4bc18305c829c4abc9bbcdbbd34b6
Parents: 433a42d
Author: Jarek Jarcec Cecho <ja...@apache.org>
Authored: Thu Feb 5 08:35:44 2015 -0800
Committer: Jarek Jarcec Cecho <ja...@apache.org>
Committed: Thu Feb 5 08:35:44 2015 -0800

----------------------------------------------------------------------
 .../sqoop/error/code/CommonRepositoryError.java |  15 +-
 .../sqoop/json/util/ConfigInputConstants.java   |   4 +-
 .../json/util/ConfigInputSerialization.java     |  20 +-
 .../org/apache/sqoop/model/ConfigUtils.java     |  59 +++-
 .../main/java/org/apache/sqoop/model/Input.java |  17 +
 .../org/apache/sqoop/model/InputEditable.java   |  41 +++
 .../org/apache/sqoop/model/MBooleanInput.java   |   8 +-
 .../java/org/apache/sqoop/model/MConfig.java    |  21 +-
 .../java/org/apache/sqoop/model/MEnumInput.java |  10 +-
 .../java/org/apache/sqoop/model/MInput.java     |  30 +-
 .../org/apache/sqoop/model/MIntegerInput.java   |   8 +-
 .../java/org/apache/sqoop/model/MMapInput.java  |   8 +-
 .../org/apache/sqoop/model/MStringInput.java    |  10 +-
 .../java/org/apache/sqoop/model/ModelError.java |  10 +
 .../apache/sqoop/json/util/ConfigTestUtil.java  |  40 ++-
 .../json/util/TestConfigSerialization.java      |  14 +-
 .../org/apache/sqoop/model/TestConfigUtils.java | 259 +++++++++++---
 .../sqoop/model/TestMAccountableEntity.java     |   3 +-
 .../apache/sqoop/model/TestMBooleanInput.java   |  25 +-
 .../org/apache/sqoop/model/TestMConfig.java     |  23 +-
 .../org/apache/sqoop/model/TestMConfigList.java |  11 +-
 .../org/apache/sqoop/model/TestMConnector.java  |   5 +-
 .../org/apache/sqoop/model/TestMEnumInput.java  |  14 +-
 .../apache/sqoop/model/TestMIntegerInput.java   |  21 +-
 .../java/org/apache/sqoop/model/TestMJob.java   |   9 +-
 .../java/org/apache/sqoop/model/TestMLink.java  |   5 +-
 .../org/apache/sqoop/model/TestMMapInput.java   |  21 +-
 .../apache/sqoop/model/TestMNamedElement.java   |   3 +-
 .../apache/sqoop/model/TestMStringInput.java    |  25 +-
 .../sqoop/model/TestMValidatedElement.java      |   5 +-
 .../jdbc/TestGenericJdbcConnectorUpgrader.java  |  39 ++-
 .../sqoop/driver/TestDriverConfigUpgrader.java  |  18 +-
 .../common/CommonRepositoryHandler.java         | 298 +++++++++++-----
 ...RepositoryInsertUpdateDeleteSelectQuery.java |  62 +++-
 .../common/CommonRepositorySchemaConstants.java |  13 +
 .../derby/DerbyRepositoryHandler.java           |   7 +-
 .../repository/derby/DerbySchemaConstants.java  |   9 +
 .../derby/DerbySchemaCreateQuery.java           |  27 ++
 .../derby/DerbySchemaUpgradeQuery.java          |   7 +-
 .../sqoop/repository/derby/DerbyTestCase.java   | 349 ++++++++++++++-----
 .../repository/derby/TestConnectorHandling.java |  20 +-
 .../repository/derby/TestDriverHandling.java    |   6 +
 .../sqoop/repository/derby/TestInputTypes.java  |  50 ++-
 .../postgresql/PostgresqlTestCase.java          |   9 +-
 .../org/apache/sqoop/shell/core/Constants.java  |   4 +
 .../sqoop/shell/utils/ConfigDisplayer.java      |   5 +
 .../apache/sqoop/shell/utils/ConfigFiller.java  |   3 +-
 .../main/resources/shell-resource.properties    |   2 +
 48 files changed, 1264 insertions(+), 408 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/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 952be3f..74e5de3 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
@@ -181,7 +181,20 @@ public enum CommonRepositoryError implements ErrorCode {
   COMMON_0043("Could not set connector direction"),
 
   /** The system was unable to register driver due to a server error **/
-  COMMON_0044("Registration of driver failed")
+  COMMON_0044("Registration of driver failed"),
+
+  /**
+   * Config Input that is set to USER_ONLY editable cannot override other
+   * USER_ONLY input
+   **/
+  COMMON_0045("Config Input cannot override USER_ONLY attribute"),
+
+  /** Config Input cannot override itself */
+  COMMON_0046("Config Input cannot override itself"),
+
+  COMMON_0047("Config Input relation insertion failed"),
+
+  COMMON_0048("Config Input overrides could not be fetched"),
 
   ;
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/main/java/org/apache/sqoop/json/util/ConfigInputConstants.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/json/util/ConfigInputConstants.java b/common/src/main/java/org/apache/sqoop/json/util/ConfigInputConstants.java
index 5d261de..21739da 100644
--- a/common/src/main/java/org/apache/sqoop/json/util/ConfigInputConstants.java
+++ b/common/src/main/java/org/apache/sqoop/json/util/ConfigInputConstants.java
@@ -36,8 +36,10 @@ public class ConfigInputConstants {
   public static final String CONFIG_INPUT_TYPE = "type";
   public static final String CONFIG_INPUT_SENSITIVE = "sensitive";
   public static final String CONFIG_INPUT_SIZE = "size";
+  public static final String CONFIG_INPUT_EDITABLE = "editable";
+  public static final String CONFIG_INPUT_OVERRIDES = "overrides";
   public static final String CONFIG_INPUT_VALUE = "value";
-  public static final String CONFIG_INPUT_VALUES = "values";
+  public static final String CONFIG_INPUT_ENUM_VALUES = "values";
 
   private ConfigInputConstants() {
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/main/java/org/apache/sqoop/json/util/ConfigInputSerialization.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/json/util/ConfigInputSerialization.java b/common/src/main/java/org/apache/sqoop/json/util/ConfigInputSerialization.java
index 8b11ed5..f671447 100644
--- a/common/src/main/java/org/apache/sqoop/json/util/ConfigInputSerialization.java
+++ b/common/src/main/java/org/apache/sqoop/json/util/ConfigInputSerialization.java
@@ -21,6 +21,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.sqoop.classification.InterfaceAudience;
 import org.apache.sqoop.classification.InterfaceStability;
 import org.apache.sqoop.common.SqoopException;
+import org.apache.sqoop.model.InputEditable;
 import org.apache.sqoop.model.MBooleanInput;
 import org.apache.sqoop.model.MEnumInput;
 import org.apache.sqoop.model.MConfig;
@@ -83,6 +84,8 @@ public final class ConfigInputSerialization {
       input.put(ConfigInputConstants.CONFIG_INPUT_NAME, mInput.getName());
       input.put(ConfigInputConstants.CONFIG_INPUT_TYPE, mInput.getType().toString());
       input.put(ConfigInputConstants.CONFIG_INPUT_SENSITIVE, mInput.isSensitive());
+      input.put(ConfigInputConstants.CONFIG_INPUT_EDITABLE, mInput.getEditable().name());
+      input.put(ConfigInputConstants.CONFIG_INPUT_OVERRIDES, mInput.getOverrides());
 
       // String specific serialization
       if (mInput.getType() == MInputType.STRING) {
@@ -92,7 +95,7 @@ public final class ConfigInputSerialization {
 
       // Enum specific serialization
       if(mInput.getType() == MInputType.ENUM) {
-        input.put(ConfigInputConstants.CONFIG_INPUT_VALUES,
+        input.put(ConfigInputConstants.CONFIG_INPUT_ENUM_VALUES,
           StringUtils.join(((MEnumInput)mInput).getValues(), ","));
       }
 
@@ -145,28 +148,31 @@ public final class ConfigInputSerialization {
           MInputType.valueOf((String) input.get(ConfigInputConstants.CONFIG_INPUT_TYPE));
       String name = (String) input.get(ConfigInputConstants.CONFIG_INPUT_NAME);
       Boolean sensitive = (Boolean) input.get(ConfigInputConstants.CONFIG_INPUT_SENSITIVE);
+      InputEditable editable =  InputEditable.valueOf((String)input.get(ConfigInputConstants.CONFIG_INPUT_EDITABLE));
+      String overrides = (String) input.get(ConfigInputConstants.CONFIG_INPUT_OVERRIDES);
+
       MInput mInput = null;
       switch (type) {
       case STRING: {
         long size = (Long) input.get(ConfigInputConstants.CONFIG_INPUT_SIZE);
-        mInput = new MStringInput(name, sensitive.booleanValue(), (short) size);
+        mInput = new MStringInput(name, sensitive.booleanValue(), editable, overrides, (short) size);
         break;
       }
       case MAP: {
-        mInput = new MMapInput(name, sensitive.booleanValue());
+        mInput = new MMapInput(name, sensitive.booleanValue(), editable, overrides);
         break;
       }
       case INTEGER: {
-        mInput = new MIntegerInput(name, sensitive.booleanValue());
+        mInput = new MIntegerInput(name, sensitive.booleanValue(), editable, overrides);
         break;
       }
       case BOOLEAN: {
-        mInput = new MBooleanInput(name, sensitive.booleanValue());
+        mInput = new MBooleanInput(name, sensitive.booleanValue(), editable, overrides);
         break;
       }
       case ENUM: {
-        String values = (String) input.get(ConfigInputConstants.CONFIG_INPUT_VALUES);
-        mInput = new MEnumInput(name, sensitive.booleanValue(), values.split(","));
+        String values = (String) input.get(ConfigInputConstants.CONFIG_INPUT_ENUM_VALUES);
+        mInput = new MEnumInput(name, sensitive.booleanValue(), editable, overrides, values.split(","));
         break;
       }
       default:

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/main/java/org/apache/sqoop/model/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/ConfigUtils.java b/common/src/main/java/org/apache/sqoop/model/ConfigUtils.java
index 279f3a6..1ec763b 100644
--- a/common/src/main/java/org/apache/sqoop/model/ConfigUtils.java
+++ b/common/src/main/java/org/apache/sqoop/model/ConfigUtils.java
@@ -21,6 +21,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.sqoop.classification.InterfaceAudience;
 import org.apache.sqoop.classification.InterfaceStability;
 import org.apache.sqoop.common.SqoopException;
+import org.apache.sqoop.error.code.CommonRepositoryError;
 import org.apache.sqoop.json.JSONUtils;
 import org.apache.sqoop.utils.ClassUtils;
 import org.apache.sqoop.validation.ConfigValidationRunner;
@@ -29,6 +30,7 @@ import org.apache.sqoop.validation.ConfigValidationResult;
 import org.json.simple.JSONObject;
 
 import java.lang.reflect.Field;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
@@ -137,6 +139,8 @@ public class  ConfigUtils {
       if(inputAnnotation != null) {
         boolean sensitive = inputAnnotation.sensitive();
         short maxLen = inputAnnotation.size();
+        InputEditable editable = inputAnnotation.editable();
+        String overrides = inputAnnotation.overrides();
         Class<?> type = field.getType();
 
         MInput input;
@@ -149,15 +153,15 @@ public class  ConfigUtils {
 
         // Instantiate corresponding MInput<?> structure
         if (type == String.class) {
-          input = new MStringInput(inputName, sensitive, maxLen);
+          input = new MStringInput(inputName, sensitive, editable, overrides, maxLen);
         } else if (type.isAssignableFrom(Map.class)) {
-          input = new MMapInput(inputName, sensitive);
+          input = new MMapInput(inputName, sensitive, editable, overrides);
         } else if (type == Integer.class) {
-          input = new MIntegerInput(inputName, sensitive);
+          input = new MIntegerInput(inputName, sensitive, editable, overrides);
         } else if (type == Boolean.class) {
-          input = new MBooleanInput(inputName, sensitive);
+          input = new MBooleanInput(inputName, sensitive, editable, overrides);
         } else if (type.isEnum()) {
-          input = new MEnumInput(inputName, sensitive,
+          input = new MEnumInput(inputName, sensitive, editable, overrides,
               ClassUtils.getEnumStrings(type));
         } else {
           throw new SqoopException(ModelError.MODEL_004, "Unsupported type "
@@ -183,8 +187,13 @@ public class  ConfigUtils {
         inputs.add(input);
       }
     }
+    MConfig config = new MConfig(configName, inputs);
+    // validation has to happen only when all inputs have been parsed
+    for (MInput<?> input : config.getInputs()) {
+      validateInputOverridesAttribute(input, config);
+    }
 
-    return new MConfig(configName, inputs);
+    return config;
   }
 
   private static Field getFieldFromName(Class<?> klass, String name) {
@@ -631,4 +640,42 @@ public class  ConfigUtils {
       throw new SqoopException(ModelError.MODEL_015, e);
     }
   }
+
+  /**
+   * Validate that the input override attribute adheres to the rules imposed
+   * NOTE: all input names in a config class will and must be unique, check the name exists and it is not self override
+   * Rule #1.
+   * If editable == USER_ONLY ( cannot override itself ) can override other  CONNECTOR_ONLY and ANY inputs,
+   * but cannot overriding other USER_ONLY attributes
+   * Rule #2.
+   * If editable == CONNECTOR_ONLY or ANY ( cannot override itself ) can override any other attribute in the config object
+   * @param currentInput
+   *
+   */
+  public static void validateInputOverridesAttribute(MInput<?> currentInput, MConfig config) {
+
+    // split the overrides string into comma separated list
+    String overrides = currentInput.getOverrides();
+    if (StringUtils.isEmpty(overrides)) {
+      return;
+    }
+    String[] overrideInputs = overrides.split("\\,");
+    for (String override : overrideInputs) {
+      if (!config.getInputNames().contains(override)) {
+        throw new SqoopException(ModelError.MODEL_017, "for input :"
+            + currentInput.toString());
+      }
+      if (override.equals(currentInput.getName())) {
+        throw new SqoopException(ModelError.MODEL_018, "for input :"
+            + currentInput.toString());
+      }
+      if (currentInput.getEditable().equals(InputEditable.USER_ONLY)) {
+        if (config.getUserOnlyEditableInputNames().contains(override)) {
+          throw new SqoopException(ModelError.MODEL_019, "for input :"
+              + currentInput.toString());
+        }
+      }
+    }
+    return;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/main/java/org/apache/sqoop/model/Input.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/Input.java b/common/src/main/java/org/apache/sqoop/model/Input.java
index b6305f2..883abe6 100644
--- a/common/src/main/java/org/apache/sqoop/model/Input.java
+++ b/common/src/main/java/org/apache/sqoop/model/Input.java
@@ -43,6 +43,15 @@ public @interface Input {
   boolean sensitive() default false;
 
   /**
+   * Indicates the entity that can edit the input's values, all inputs are
+   * created/deleted only by the connector code, other entities do not have
+   * access to either create/delete an input
+   *
+   * @return editable
+   */
+  InputEditable editable() default InputEditable.ANY;
+
+  /**
    * Maximal length of field if applicable.
    *
    * @return Maximal length
@@ -50,9 +59,17 @@ public @interface Input {
   short size() default -1;
 
   /**
+   * In-order to express dependency on other inputs, the value supports a comma
+   * separated list of other inputs in the config class. It validates the
+   * attribute value obeys the expected conditions
+   */
+  String overrides() default "";
+
+  /**
    * List of validators associated with this input.
    *
    * @return
    */
   Validator[] validators() default {};
+
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/main/java/org/apache/sqoop/model/InputEditable.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/InputEditable.java b/common/src/main/java/org/apache/sqoop/model/InputEditable.java
new file mode 100644
index 0000000..3589dfc
--- /dev/null
+++ b/common/src/main/java/org/apache/sqoop/model/InputEditable.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.sqoop.model;
+
+/**
+ * Various supported roles for editing input values that belong to a config
+ * NOTE: In future this can be extended based on the roles supported in sqoop
+ * for instance, we could have sqoop ADMIN_ONLY editable inputs
+ */
+public enum InputEditable {
+  /**
+   * Sqoop user alone can edit the input values via rest API or shell command line
+   */
+  USER_ONLY,
+
+  /**
+   * Connector code alone can edit the input values
+   */
+  CONNECTOR_ONLY,
+
+  /**
+   * Either Connector code or the sqoop user alone can edit the input values
+   */
+
+  ANY
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/main/java/org/apache/sqoop/model/MBooleanInput.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/MBooleanInput.java b/common/src/main/java/org/apache/sqoop/model/MBooleanInput.java
index c7fcf90..d55a0e7 100644
--- a/common/src/main/java/org/apache/sqoop/model/MBooleanInput.java
+++ b/common/src/main/java/org/apache/sqoop/model/MBooleanInput.java
@@ -27,8 +27,8 @@ import org.apache.sqoop.classification.InterfaceStability;
 @InterfaceStability.Unstable
 public class MBooleanInput extends MInput<Boolean> {
 
-  public MBooleanInput(String name, boolean sensitive) {
-    super(name, sensitive);
+  public MBooleanInput(String name, boolean sensitive, InputEditable editable, String overrides) {
+    super(name, sensitive, editable, overrides);
   }
 
   @Override
@@ -78,8 +78,8 @@ public class MBooleanInput extends MInput<Boolean> {
 
   @Override
   public Object clone(boolean cloneWithValue) {
-    MBooleanInput copy = new MBooleanInput(getName(), isSensitive());
-    copy.setPersistenceId(this.getPersistenceId());
+    MBooleanInput copy = new MBooleanInput(getName(), isSensitive(), getEditable(), getOverrides());
+    copy.setPersistenceId(getPersistenceId());
     if(cloneWithValue) {
       copy.setValue(getValue());
     }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/main/java/org/apache/sqoop/model/MConfig.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/MConfig.java b/common/src/main/java/org/apache/sqoop/model/MConfig.java
index d128441..1bae6d4 100644
--- a/common/src/main/java/org/apache/sqoop/model/MConfig.java
+++ b/common/src/main/java/org/apache/sqoop/model/MConfig.java
@@ -22,7 +22,9 @@ import org.apache.sqoop.classification.InterfaceStability;
 import org.apache.sqoop.common.SqoopException;
 
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 /**
  * Represents a group of inputs that are processed together. This allows the
@@ -34,17 +36,34 @@ import java.util.List;
 public final class MConfig extends MValidatedElement implements MClonable {
 
   private final List<MInput<?>> inputs;
+  private Set<String> inputNames = new HashSet<String>();
+  private Set<String> userOnlyEditableInputNames = new HashSet<String>();
 
   public MConfig(String name, List<MInput<?>> inputs) {
     super(name);
-
     this.inputs = inputs;
+    if (inputs != null && inputs.size() > 0) {
+      for (MInput<?> input : inputs) {
+        inputNames.add(input.getName());
+        if (input.getEditable().equals(InputEditable.USER_ONLY)) {
+          userOnlyEditableInputNames.add(input.getName());
+        }
+      }
+    }
   }
 
   public List<MInput<?>> getInputs() {
     return inputs;
   }
 
+  public Set<String> getInputNames() {
+    return inputNames;
+  }
+
+  public Set<String> getUserOnlyEditableInputNames() {
+    return userOnlyEditableInputNames;
+  }
+
   public MInput<?> getInput(String inputName) {
     for(MInput<?> input: inputs) {
       if(inputName.equals(input.getName())) {

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/main/java/org/apache/sqoop/model/MEnumInput.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/MEnumInput.java b/common/src/main/java/org/apache/sqoop/model/MEnumInput.java
index 74838fc..e630536 100644
--- a/common/src/main/java/org/apache/sqoop/model/MEnumInput.java
+++ b/common/src/main/java/org/apache/sqoop/model/MEnumInput.java
@@ -36,8 +36,8 @@ public class MEnumInput extends MInput<String> {
    */
   String []values;
 
-  public MEnumInput(String name, boolean sensitive, String[] values) {
-    super(name, sensitive);
+  public MEnumInput(String name, boolean sensitive, InputEditable editable, String overrides, String[] values) {
+    super(name, sensitive, editable, overrides);
     this.values = values;
   }
 
@@ -131,9 +131,9 @@ public class MEnumInput extends MInput<String> {
 
   @Override
   public MEnumInput clone(boolean cloneWithValue) {
-    MEnumInput copy = new MEnumInput(this.getName(),
-        this.isSensitive(), this.getValues());
-    copy.setPersistenceId(this.getPersistenceId());
+    MEnumInput copy = new MEnumInput(getName(), isSensitive(), getEditable(), getOverrides(),
+        getValues());
+    copy.setPersistenceId(getPersistenceId());
     if(cloneWithValue) {
       copy.setValue(this.getValue());
     }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/main/java/org/apache/sqoop/model/MInput.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/MInput.java b/common/src/main/java/org/apache/sqoop/model/MInput.java
index d1c2ab6..d5ce074 100644
--- a/common/src/main/java/org/apache/sqoop/model/MInput.java
+++ b/common/src/main/java/org/apache/sqoop/model/MInput.java
@@ -31,15 +31,22 @@ import org.apache.sqoop.classification.InterfaceStability;
 public abstract class MInput<T> extends MValidatedElement implements MClonable {
   private final boolean sensitive;
 
+  private final String overrides;
+
+  private final InputEditable editable;
+
   private T value;
 
-  protected MInput(String name, boolean sensitive) {
+  protected MInput(String name, boolean sensitive, InputEditable editable, String overrides) {
     super(name);
     this.sensitive = sensitive;
+    this.editable = editable;
+    this.overrides = overrides;
   }
 
   /**
-   * @param value the value to be set for this parameter
+   * @param value
+   *          the value to be set for this parameter
    */
   public void setValue(T value) {
     this.value = value;
@@ -60,6 +67,22 @@ public abstract class MInput<T> extends MValidatedElement implements MClonable {
   }
 
   /**
+   * @return the editable {@link#InputEditable}attribute for the input
+   */
+  public InputEditable getEditable() {
+    return editable;
+  }
+
+  /**
+   * @return the overrides attribute for the input
+   * An input can override the value of one or more other inputs when edited
+   */
+  public String getOverrides() {
+    return overrides;
+  }
+
+  /**
+  /**
    * @return a URL-safe string representation of the value
    */
   public abstract String getUrlSafeValueString();
@@ -127,7 +150,8 @@ public abstract class MInput<T> extends MValidatedElement implements MClonable {
   public final String toString() {
     StringBuilder sb = new StringBuilder("input-").append(getName());
     sb.append(":").append(getPersistenceId()).append(":");
-    sb.append(getType());
+    sb.append(getType()).append(":").append(isSensitive()).append(":").append(getEditable().name())
+        .append(":").append(getOverrides());
     if (hasExtraInfo()) {
       sb.append(":").append(getExtraInfoToString());
     }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/main/java/org/apache/sqoop/model/MIntegerInput.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/MIntegerInput.java b/common/src/main/java/org/apache/sqoop/model/MIntegerInput.java
index de50cd4..90b2d95 100644
--- a/common/src/main/java/org/apache/sqoop/model/MIntegerInput.java
+++ b/common/src/main/java/org/apache/sqoop/model/MIntegerInput.java
@@ -29,8 +29,8 @@ import org.apache.sqoop.classification.InterfaceStability;
 @InterfaceStability.Unstable
 public class MIntegerInput extends MInput<Integer> {
 
-  public MIntegerInput(String name, boolean sensitive) {
-    super(name, sensitive);
+  public MIntegerInput(String name, boolean sensitive, InputEditable editable, String overrides) {
+    super(name, sensitive, editable, overrides);
   }
 
   @Override
@@ -87,8 +87,8 @@ public class MIntegerInput extends MInput<Integer> {
 
   @Override
   public MIntegerInput clone(boolean cloneWithValue) {
-    MIntegerInput copy = new MIntegerInput(this.getName(), this.isSensitive());
-    copy.setPersistenceId(this.getPersistenceId());
+    MIntegerInput copy = new MIntegerInput(getName(), isSensitive(), getEditable(), getOverrides());
+    copy.setPersistenceId(getPersistenceId());
     if(cloneWithValue) {
       copy.setValue(this.getValue());
     }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/main/java/org/apache/sqoop/model/MMapInput.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/MMapInput.java b/common/src/main/java/org/apache/sqoop/model/MMapInput.java
index cf781b2..ce0f0f7 100644
--- a/common/src/main/java/org/apache/sqoop/model/MMapInput.java
+++ b/common/src/main/java/org/apache/sqoop/model/MMapInput.java
@@ -29,8 +29,8 @@ import org.apache.sqoop.utils.UrlSafeUtils;
 @InterfaceStability.Unstable
 public final class MMapInput extends MInput<Map<String, String>> {
 
-  public MMapInput(String name, boolean sensitive) {
-    super(name, sensitive);
+  public MMapInput(String name, boolean sensitive, InputEditable editable, String overrides) {
+    super(name, sensitive, editable, overrides);
   }
 
   @Override
@@ -114,8 +114,8 @@ public final class MMapInput extends MInput<Map<String, String>> {
 
   @Override
   public MMapInput clone(boolean cloneWithValue) {
-    MMapInput copy = new MMapInput(this.getName(), this.isSensitive());
-    copy.setPersistenceId(this.getPersistenceId());
+    MMapInput copy = new MMapInput(getName(), isSensitive(), getEditable(), getOverrides());
+    copy.setPersistenceId(getPersistenceId());
     if(cloneWithValue && this.getValue() != null) {
       Map<String, String> copyMap = new HashMap<String, String>();
       Set<Map.Entry<String, String>> entry = this.getValue().entrySet();

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/main/java/org/apache/sqoop/model/MStringInput.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/MStringInput.java b/common/src/main/java/org/apache/sqoop/model/MStringInput.java
index 4854838..7365098 100644
--- a/common/src/main/java/org/apache/sqoop/model/MStringInput.java
+++ b/common/src/main/java/org/apache/sqoop/model/MStringInput.java
@@ -38,8 +38,8 @@ public final class MStringInput extends MInput<String> {
    * @param sensitive a flag indicating if the string should be masked
    * @param maxLength the maximum length of the string
    */
-  public MStringInput(String name, boolean sensitive, short maxLength) {
-    super(name, sensitive);
+  public MStringInput(String name, boolean sensitive, InputEditable editable, String overrides, short maxLength) {
+    super(name, sensitive, editable, overrides);
     this.maxLength = maxLength;
   }
 
@@ -111,9 +111,9 @@ public final class MStringInput extends MInput<String> {
 
   @Override
   public MStringInput clone(boolean cloneWithValue) {
-    MStringInput copy = new MStringInput(this.getName(),
-        this.isSensitive(), this.getMaxLength());
-    copy.setPersistenceId(this.getPersistenceId());
+    MStringInput copy = new MStringInput(getName(), isSensitive(), getEditable(), getOverrides(),
+        getMaxLength());
+    copy.setPersistenceId(getPersistenceId());
     if(cloneWithValue) {
       copy.setValue(this.getValue());
     }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/main/java/org/apache/sqoop/model/ModelError.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/ModelError.java b/common/src/main/java/org/apache/sqoop/model/ModelError.java
index dcb137a..17e70df 100644
--- a/common/src/main/java/org/apache/sqoop/model/ModelError.java
+++ b/common/src/main/java/org/apache/sqoop/model/ModelError.java
@@ -60,6 +60,16 @@ public enum ModelError implements ErrorCode {
 
   MODEL_016("Can't instantiate class"),
 
+  MODEL_017("Config Input override name does nto exist"),
+  /**
+   * Config Input that is set to USER_ONLY editable cannot override other
+   * USER_ONLY input
+   **/
+  MODEL_018("Config Input cannot override USER_ONLY attribute"),
+
+  /** Config Input cannot override itself */
+  MODEL_019("Config Input cannot override itself"),
+
   ;
 
   private final String message;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/test/java/org/apache/sqoop/json/util/ConfigTestUtil.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/json/util/ConfigTestUtil.java b/common/src/test/java/org/apache/sqoop/json/util/ConfigTestUtil.java
index 7f0e2f1..fbc7faa 100644
--- a/common/src/test/java/org/apache/sqoop/json/util/ConfigTestUtil.java
+++ b/common/src/test/java/org/apache/sqoop/json/util/ConfigTestUtil.java
@@ -23,6 +23,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.ResourceBundle;
 
+import org.apache.commons.lang.StringUtils;
+import org.apache.sqoop.model.InputEditable;
 import org.apache.sqoop.model.MConfig;
 import org.apache.sqoop.model.MDriverConfig;
 import org.apache.sqoop.model.MFromConfig;
@@ -42,11 +44,11 @@ public class ConfigTestUtil {
     List<MConfig> driverConfigs = new ArrayList<MConfig>();
     inputs = new ArrayList<MInput<?>>();
 
-    input = new MIntegerInput("numExtractors", false);
+    input = new MIntegerInput("numExtractors", false, InputEditable.ANY, StringUtils.EMPTY);
     input.setPersistenceId(1);
     inputs.add(input);
 
-    input = new MIntegerInput("numLoaders", false);
+    input = new MIntegerInput("numLoaders", false, InputEditable.USER_ONLY, StringUtils.EMPTY);
     input.setPersistenceId(2);
     inputs.add(input);
 
@@ -63,16 +65,16 @@ public class ConfigTestUtil {
     List<MConfig> linkConfig = new ArrayList<MConfig>();
     inputs = new ArrayList<MInput<?>>();
 
-    input = new MStringInput("url", false, (short) 10);
+    input = new MStringInput("url", false, InputEditable.USER_ONLY, StringUtils.EMPTY, (short) 10);
     input.setPersistenceId(1);
     inputs.add(input);
 
-    input = new MStringInput("username", false, (short) 10);
+    input = new MStringInput("username", false, InputEditable.USER_ONLY, "password", (short) 10);
     input.setPersistenceId(2);
     input.setValue("test");
     inputs.add(input);
 
-    input = new MStringInput("password", true, (short) 10);
+    input = new MStringInput("password", true, InputEditable.USER_ONLY, StringUtils.EMPTY, (short) 10);
     input.setPersistenceId(3);
     input.setValue("test");
     inputs.add(input);
@@ -92,33 +94,33 @@ public class ConfigTestUtil {
 
     inputs = new ArrayList<MInput<?>>();
 
-    input = new MStringInput("A", false, (short) 10);
+    input = new MStringInput("A", false, InputEditable.USER_ONLY, StringUtils.EMPTY, (short) 10);
     input.setPersistenceId(4);
     inputs.add(input);
 
-    input = new MStringInput("B", false, (short) 10);
+    input = new MStringInput("B", false, InputEditable.USER_ONLY, StringUtils.EMPTY, (short) 10);
     input.setPersistenceId(5);
     inputs.add(input);
 
-    input = new MStringInput("C", false, (short) 10);
+    input = new MStringInput("C", false, InputEditable.USER_ONLY, StringUtils.EMPTY, (short) 10);
     input.setPersistenceId(6);
     inputs.add(input);
 
     config = new MConfig("Z", inputs);
     config.setPersistenceId(11);
-    jobConfigs.add(config);
+ jobConfigs.add(config);
 
     inputs = new ArrayList<MInput<?>>();
 
-    input = new MStringInput("D", false, (short) 10);
+    input = new MStringInput("D", false, InputEditable.USER_ONLY, StringUtils.EMPTY, (short) 10);
     input.setPersistenceId(7);
     inputs.add(input);
 
-    input = new MStringInput("E", false, (short) 10);
+    input = new MStringInput("E", false, InputEditable.USER_ONLY, "D, F", (short) 10);
     input.setPersistenceId(8);
     inputs.add(input);
 
-    input = new MStringInput("F", false, (short) 10);
+    input = new MStringInput("F", false, InputEditable.USER_ONLY, StringUtils.EMPTY, (short) 10);
     input.setPersistenceId(9);
     inputs.add(input);
 
@@ -137,15 +139,15 @@ public class ConfigTestUtil {
 
     inputs = new ArrayList<MInput<?>>();
 
-    input = new MStringInput("A", false, (short) 10);
+    input = new MStringInput("A", false, InputEditable.ANY, StringUtils.EMPTY, (short) 10);
     input.setPersistenceId(4);
     inputs.add(input);
 
-    input = new MStringInput("B", false, (short) 10);
+    input = new MStringInput("B", false, InputEditable.ANY, StringUtils.EMPTY, (short) 10);
     input.setPersistenceId(5);
     inputs.add(input);
 
-    input = new MStringInput("C", false, (short) 10);
+    input = new MStringInput("C", false, InputEditable.ANY, StringUtils.EMPTY, (short) 10);
     input.setPersistenceId(6);
     inputs.add(input);
 
@@ -155,17 +157,17 @@ public class ConfigTestUtil {
 
     inputs = new ArrayList<MInput<?>>();
 
-    input = new MStringInput("D", false, (short) 10);
+    input = new MStringInput("D", false, InputEditable.ANY, StringUtils.EMPTY, (short) 10);
     input.setPersistenceId(7);
     inputs.add(input);
 
-    input = new MStringInput("E", false, (short) 10);
+    input = new MStringInput("E", false, InputEditable.ANY, StringUtils.EMPTY, (short) 10);
     input.setPersistenceId(8);
     inputs.add(input);
 
-    input = new MStringInput("F", false, (short) 10);
+    input = new MStringInput("F", false, InputEditable.ANY, StringUtils.EMPTY, (short) 10);
     input.setPersistenceId(9);
-    inputs.add(input);
+ inputs.add(input);
 
     config = new MConfig("to-table", inputs);
     config.setPersistenceId(12);

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/test/java/org/apache/sqoop/json/util/TestConfigSerialization.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/json/util/TestConfigSerialization.java b/common/src/test/java/org/apache/sqoop/json/util/TestConfigSerialization.java
index 10ac3cf..18a1d4f 100644
--- a/common/src/test/java/org/apache/sqoop/json/util/TestConfigSerialization.java
+++ b/common/src/test/java/org/apache/sqoop/json/util/TestConfigSerialization.java
@@ -25,8 +25,10 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.sqoop.common.SqoopException;
 import org.apache.sqoop.json.JSONUtils;
+import org.apache.sqoop.model.InputEditable;
 import org.apache.sqoop.model.MBooleanInput;
 import org.apache.sqoop.model.MConfig;
 import org.apache.sqoop.model.MConfigType;
@@ -121,7 +123,7 @@ public class TestConfigSerialization {
 
     inputs = new LinkedList<MInput<?>>();
 
-    input = new MMapInput("Map", false);
+    input = new MMapInput("Map", false, InputEditable.ANY, StringUtils.EMPTY);
     inputs.add(input);
 
     return new MConfig("c", inputs);
@@ -138,19 +140,19 @@ public class TestConfigSerialization {
 
     inputs = new LinkedList<MInput<?>>();
 
-    input = new MStringInput("String", false, (short)30);
+    input = new MStringInput("String", false, InputEditable.ANY, StringUtils.EMPTY, (short)30);
     inputs.add(input);
 
-    input = new MMapInput("Map", false);
+    input = new MMapInput("Map", false, InputEditable.ANY, StringUtils.EMPTY);
     inputs.add(input);
 
-    input = new MIntegerInput("Integer", false);
+    input = new MIntegerInput("Integer", false, InputEditable.ANY, StringUtils.EMPTY);
     inputs.add(input);
 
-    input = new MBooleanInput("Boolean", false);
+    input = new MBooleanInput("Boolean", false, InputEditable.ANY, StringUtils.EMPTY);
     inputs.add(input);
 
-    input = new MEnumInput("Enum", false, new String[] {"YES", "NO"});
+    input = new MEnumInput("Enum", false, InputEditable.ANY, StringUtils.EMPTY, new String[] {"YES", "NO"});
     inputs.add(input);
 
     return new MConfig("c", inputs);

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/test/java/org/apache/sqoop/model/TestConfigUtils.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/model/TestConfigUtils.java b/common/src/test/java/org/apache/sqoop/model/TestConfigUtils.java
index 64ffdd1..7eafdf4 100644
--- a/common/src/test/java/org/apache/sqoop/model/TestConfigUtils.java
+++ b/common/src/test/java/org/apache/sqoop/model/TestConfigUtils.java
@@ -17,125 +17,160 @@
  */
 package org.apache.sqoop.model;
 
+import org.testng.annotations.Test;
+import org.testng.Assert;
+import org.testng.AssertJUnit;
+import static org.testng.AssertJUnit.assertNull;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
-import junit.framework.TestCase;
-
+import org.apache.commons.lang.StringUtils;
 import org.apache.sqoop.common.SqoopException;
 
 /**
  * Test config utils
  */
-public class TestConfigUtils extends TestCase {
+public class TestConfigUtils {
 
+  @Test
   public void testConfigs() {
     TestConfiguration config = new TestConfiguration();
     config.aConfig.a1 = "value";
 
     List<MConfig> configsByInstance = ConfigUtils.toConfigs(config);
-    assertEquals(getConfigs(), configsByInstance);
-    assertEquals("value", configsByInstance.get(0).getInputs().get(0).getValue());
+    AssertJUnit.assertEquals(getConfigs(), configsByInstance);
+    AssertJUnit.assertEquals("value", configsByInstance.get(0).getInputs().get(0).getValue());
 
     List<MConfig> configsByClass = ConfigUtils.toConfigs(TestConfiguration.class);
-    assertEquals(getConfigs(), configsByClass);
+    AssertJUnit.assertEquals(getConfigs(), configsByClass);
 
     List<MConfig> configsByBoth = ConfigUtils.toConfigs(TestConfiguration.class, config);
-    assertEquals(getConfigs(), configsByBoth);
-    assertEquals("value", configsByBoth.get(0).getInputs().get(0).getValue());
+    AssertJUnit.assertEquals(getConfigs(), configsByBoth);
+    AssertJUnit.assertEquals("value", configsByBoth.get(0).getInputs().get(0).getValue());
+  }
+
+  @Test(expectedExceptions = SqoopException.class)
+  public void testBadConfigInputsWithNonExisitingOverride() {
+    TestBadConfiguration config = new TestBadConfiguration();
+    config.aBadConfig.a1 = "value";
+    ConfigUtils.toConfigs(config);
+  }
+
+  @Test(expectedExceptions = SqoopException.class)
+  public void testBadConfigInputsWithBadOverride() {
+    TestBadConfiguration1 config = new TestBadConfiguration1();
+    config.aBadConfig1.a1 = "value";
+    ConfigUtils.toConfigs(config);
+  }
+
+  @Test(expectedExceptions = SqoopException.class)
+  public void testBadConfigInputsWithSelfOverride() {
+    TestBadConfiguration2 config = new TestBadConfiguration2();
+    config.aBadConfig2.a1 = "value";
+    ConfigUtils.toConfigs(config);
   }
 
+  @Test
   public void testConfigsMissingAnnotation() {
     try {
       ConfigUtils.toConfigs(ConfigWithoutAnnotation.class);
-    } catch(SqoopException ex) {
-      assertEquals(ModelError.MODEL_003, ex.getErrorCode());
+    } catch (SqoopException ex) {
+      AssertJUnit.assertEquals(ModelError.MODEL_003, ex.getErrorCode());
       return;
     }
 
-    fail("Correct exception wasn't thrown");
+    Assert.fail("Correct exception wasn't thrown");
   }
 
+  @Test
   public void testNonUniqueConfigNameAttributes() {
     try {
       ConfigUtils.toConfigs(ConfigurationWithNonUniqueConfigNameAttribute.class);
     } catch (SqoopException ex) {
-      assertEquals(ModelError.MODEL_012, ex.getErrorCode());
+      AssertJUnit.assertEquals(ModelError.MODEL_012, ex.getErrorCode());
       return;
     }
 
-    fail("Correct exception wasn't thrown");
+    Assert.fail("Correct exception wasn't thrown");
   }
 
+  @Test
   public void testInvalidConfigNameAttribute() {
     try {
       ConfigUtils.toConfigs(ConfigurationWithInvalidConfigNameAttribute.class);
     } catch (SqoopException ex) {
-      assertEquals(ModelError.MODEL_013, ex.getErrorCode());
+      AssertJUnit.assertEquals(ModelError.MODEL_013, ex.getErrorCode());
       return;
     }
-    fail("Correct exception wasn't thrown");
+    Assert.fail("Correct exception wasn't thrown");
   }
 
+  @Test
   public void testInvalidConfigNameAttributeLength() {
     try {
       ConfigUtils.toConfigs(ConfigurationWithInvalidConfigNameAttributeLength.class);
     } catch (SqoopException ex) {
-      assertEquals(ModelError.MODEL_014, ex.getErrorCode());
+      AssertJUnit.assertEquals(ModelError.MODEL_014, ex.getErrorCode());
       return;
     }
-    fail("Correct exception wasn't thrown");
+    Assert.fail("Correct exception wasn't thrown");
   }
 
+  @Test
   public void testFailureOnPrimitiveType() {
     PrimitiveConfig config = new PrimitiveConfig();
 
     try {
       ConfigUtils.toConfigs(config);
-      fail("We were expecting exception for unsupported type.");
-    } catch(SqoopException ex) {
-      assertEquals(ModelError.MODEL_007, ex.getErrorCode());
+      Assert.fail("We were expecting exception for unsupported type.");
+    } catch (SqoopException ex) {
+      AssertJUnit.assertEquals(ModelError.MODEL_007, ex.getErrorCode());
     }
   }
 
+  @Test
   public void testFillValues() {
     List<MConfig> configs = getConfigs();
 
-    ((MStringInput)configs.get(0).getInputs().get(0)).setValue("value");
+    ((MStringInput) configs.get(0).getInputs().get(0)).setValue("value");
 
     TestConfiguration config = new TestConfiguration();
 
     ConfigUtils.fromConfigs(configs, config);
-    assertEquals("value", config.aConfig.a1);
+    AssertJUnit.assertEquals("value", config.aConfig.a1);
   }
 
+  @Test
   public void testFromConfigWithClass() {
     List<MConfig> configs = getConfigs();
 
-    ((MStringInput)configs.get(0).getInputs().get(0)).setValue("value");
+    ((MStringInput) configs.get(0).getInputs().get(0)).setValue("value");
 
-    TestConfiguration config = (TestConfiguration) ConfigUtils.fromConfigs(configs, TestConfiguration.class);
-    assertEquals("value", config.aConfig.a1);
+    TestConfiguration config = (TestConfiguration) ConfigUtils.fromConfigs(configs,
+        TestConfiguration.class);
+    AssertJUnit.assertEquals("value", config.aConfig.a1);
   }
 
+  @Test
   public void testFillValuesObjectReuse() {
     List<MConfig> configs = getConfigs();
 
-    ((MStringInput)configs.get(0).getInputs().get(0)).setValue("value");
+    ((MStringInput) configs.get(0).getInputs().get(0)).setValue("value");
 
     TestConfiguration config = new TestConfiguration();
     config.aConfig.a2 = "x";
     config.bConfig.b1 = "y";
 
     ConfigUtils.fromConfigs(configs, config);
-    assertEquals("value", config.aConfig.a1);
+    AssertJUnit.assertEquals("value", config.aConfig.a1);
     assertNull(config.aConfig.a2);
     assertNull(config.bConfig.b2);
     assertNull(config.bConfig.b2);
   }
 
+  @Test
   public void testJson() {
     TestConfiguration config = new TestConfiguration();
     config.aConfig.a1 = "A";
@@ -156,17 +191,17 @@ public class TestConfigUtils extends TestCase {
 
     ConfigUtils.fillValues(json, targetConfig);
 
-    assertEquals("A", targetConfig.aConfig.a1);
+    AssertJUnit.assertEquals("A", targetConfig.aConfig.a1);
     assertNull(targetConfig.aConfig.a2);
 
     assertNull(targetConfig.bConfig.b1);
-    assertEquals("B", targetConfig.bConfig.b2);
+    AssertJUnit.assertEquals("B", targetConfig.bConfig.b2);
 
-    assertEquals((Integer)4, targetConfig.cConfig.intValue);
-    assertEquals(1, targetConfig.cConfig.map.size());
-    assertTrue(targetConfig.cConfig.map.containsKey("C"));
-    assertEquals("D", targetConfig.cConfig.map.get("C"));
-    assertEquals(Enumeration.X, targetConfig.cConfig.enumeration);
+    AssertJUnit.assertEquals((Integer) 4, targetConfig.cConfig.intValue);
+    AssertJUnit.assertEquals(1, targetConfig.cConfig.map.size());
+    AssertJUnit.assertTrue(targetConfig.cConfig.map.containsKey("C"));
+    AssertJUnit.assertEquals("D", targetConfig.cConfig.map.get("C"));
+    AssertJUnit.assertEquals(Enumeration.X, targetConfig.cConfig.enumeration);
   }
 
   /**
@@ -180,26 +215,70 @@ public class TestConfigUtils extends TestCase {
 
     // Config A
     inputs = new LinkedList<MInput<?>>();
-    inputs.add(new MStringInput("aConfig.a1", false, (short)30));
-    inputs.add(new MStringInput("aConfig.a2", true, (short)-1));
+    inputs.add(new MStringInput("aConfig.a1", false, InputEditable.ANY, StringUtils.EMPTY,
+        (short) 30));
+    inputs.add(new MStringInput("aConfig.a2", true, InputEditable.ANY, StringUtils.EMPTY,
+        (short) -1));
     ret.add(new MConfig("aConfig", inputs));
 
     // Config B
     inputs = new LinkedList<MInput<?>>();
-    inputs.add(new MStringInput("bConfig.b1", false, (short)2));
-    inputs.add(new MStringInput("bConfig.b2", false, (short)3));
+    inputs.add(new MStringInput("bConfig.b1", false, InputEditable.ANY, StringUtils.EMPTY,
+        (short) 2));
+    inputs.add(new MStringInput("bConfig.b2", false, InputEditable.ANY, StringUtils.EMPTY,
+        (short) 3));
     ret.add(new MConfig("bConfig", inputs));
 
     // Config C
     inputs = new LinkedList<MInput<?>>();
-    inputs.add(new MIntegerInput("cConfig.intValue", false));
-    inputs.add(new MMapInput("cConfig.map", false));
-    inputs.add(new MEnumInput("cConfig.enumeration", false, new String[]{"X", "Y"}));
+    inputs.add(new MIntegerInput("cConfig.intValue", false, InputEditable.ANY, StringUtils.EMPTY));
+    inputs.add(new MMapInput("cConfig.map", false, InputEditable.ANY, StringUtils.EMPTY));
+    inputs.add(new MEnumInput("cConfig.enumeration", false, InputEditable.ANY, StringUtils.EMPTY,
+        new String[] { "X", "Y" }));
     ret.add(new MConfig("cConfig", inputs));
 
     return ret;
   }
 
+  protected List<MConfig> getBadConfigWithSelfOverrideInputs() {
+    List<MConfig> ret = new LinkedList<MConfig>();
+
+    List<MInput<?>> inputs;
+    // Config A
+    inputs = new LinkedList<MInput<?>>();
+    inputs.add(new MStringInput("aConfig.a1", false, InputEditable.ANY, "aConfig.a1", (short) 30));
+    inputs.add(new MStringInput("aConfig.a2", true, InputEditable.ANY, StringUtils.EMPTY,
+        (short) -1));
+    ret.add(new MConfig("aConfig", inputs));
+    return ret;
+  }
+
+  protected List<MConfig> getBadConfigWithNonExistingOverrideInputs() {
+    List<MConfig> ret = new LinkedList<MConfig>();
+
+    List<MInput<?>> inputs;
+    // Config A
+    inputs = new LinkedList<MInput<?>>();
+    inputs.add(new MStringInput("aConfig.a1", false, InputEditable.ANY, "aConfig.a3", (short) 30));
+    inputs.add(new MStringInput("aConfig.a2", true, InputEditable.ANY, StringUtils.EMPTY,
+        (short) -1));
+    ret.add(new MConfig("aConfig", inputs));
+    return ret;
+  }
+
+  protected List<MConfig> getBadConfigWithUserEditableOverrideInputs() {
+    List<MConfig> ret = new LinkedList<MConfig>();
+
+    List<MInput<?>> inputs;
+    // Config A
+    inputs = new LinkedList<MInput<?>>();
+    inputs.add(new MStringInput("aConfig.a1", false, InputEditable.ANY, "aConfig.a2", (short) 30));
+    inputs.add(new MStringInput("aConfig.a2", true, InputEditable.USER_ONLY, StringUtils.EMPTY,
+        (short) -1));
+    ret.add(new MConfig("aConfig", inputs));
+    return ret;
+  }
+
   @ConfigurationClass
   public static class ConfigurationWithNonUniqueConfigNameAttribute {
     public ConfigurationWithNonUniqueConfigNameAttribute() {
@@ -234,6 +313,39 @@ public class TestConfigUtils extends TestCase {
   }
 
   @ConfigurationClass
+  public static class TestBadConfiguration {
+
+    public TestBadConfiguration() {
+      aBadConfig = new ABadConfig();
+    }
+
+    @Config
+    ABadConfig aBadConfig;
+  }
+
+  @ConfigurationClass
+  public static class TestBadConfiguration1 {
+
+    public TestBadConfiguration1() {
+      aBadConfig1 = new ABadConfig1();
+    }
+
+    @Config
+    ABadConfig1 aBadConfig1;
+  }
+
+  @ConfigurationClass
+  public static class TestBadConfiguration2 {
+
+    public TestBadConfiguration2() {
+      aBadConfig2 = new ABadConfig2();
+    }
+
+    @Config
+    ABadConfig2 aBadConfig2;
+  }
+
+  @ConfigurationClass
   public static class TestConfiguration {
 
     public TestConfiguration() {
@@ -242,33 +354,68 @@ public class TestConfigUtils extends TestCase {
       cConfig = new CConfig();
     }
 
-    @Config AConfig aConfig;
-    @Config BConfig bConfig;
-    @Config CConfig cConfig;
+    @Config
+    AConfig aConfig;
+    @Config
+    BConfig bConfig;
+    @Config
+    CConfig cConfig;
   }
 
   @ConfigurationClass
   public static class PrimitiveConfig {
-    @Config DConfig dConfig;
+    @Config
+    DConfig dConfig;
   }
 
   @ConfigClass
   public static class AConfig {
-    @Input(size = 30)  String a1;
-    @Input(sensitive = true)  String a2;
+    @Input(size = 30)
+    String a1;
+    @Input(sensitive = true)
+    String a2;
+  }
+
+  @ConfigClass
+  public static class ABadConfig {
+    @Input(size = 30, editable = InputEditable.USER_ONLY, overrides = "a5")
+    String a1;
+    @Input(sensitive = true)
+    String a2;
+  }
+
+  @ConfigClass
+  public static class ABadConfig1 {
+    @Input(size = 30, editable = InputEditable.USER_ONLY, overrides = "a2")
+    String a1;
+    @Input(sensitive = true, editable = InputEditable.USER_ONLY, overrides = "a1")
+    String a2;
+  }
+
+  @ConfigClass
+  public static class ABadConfig2 {
+    @Input(size = 30, editable = InputEditable.USER_ONLY, overrides = "a1")
+    String a1;
+    @Input(sensitive = true, editable = InputEditable.USER_ONLY, overrides = "a2")
+    String a2;
   }
 
   @ConfigClass
   public static class BConfig {
-    @Input(size = 2) String b1;
-    @Input(size = 3) String b2;
+    @Input(size = 2)
+    String b1;
+    @Input(size = 3)
+    String b2;
   }
 
   @ConfigClass
   public static class CConfig {
-    @Input Integer intValue;
-    @Input Map<String, String> map;
-    @Input Enumeration enumeration;
+    @Input
+    Integer intValue;
+    @Input
+    Map<String, String> map;
+    @Input
+    Enumeration enumeration;
 
     public CConfig() {
       map = new HashMap<String, String>();
@@ -282,14 +429,14 @@ public class TestConfigUtils extends TestCase {
 
   @ConfigClass
   public static class DConfig {
-    @Input int value;
+    @Input
+    int value;
   }
 
   public static class ConfigWithoutAnnotation {
   }
 
   enum Enumeration {
-    X,
-    Y,
+    X, Y,
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/test/java/org/apache/sqoop/model/TestMAccountableEntity.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/model/TestMAccountableEntity.java b/common/src/test/java/org/apache/sqoop/model/TestMAccountableEntity.java
index c0644e7..0e66400 100644
--- a/common/src/test/java/org/apache/sqoop/model/TestMAccountableEntity.java
+++ b/common/src/test/java/org/apache/sqoop/model/TestMAccountableEntity.java
@@ -21,6 +21,7 @@ import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
 
+import org.apache.commons.lang.StringUtils;
 import org.testng.annotations.Test;
 
 import static org.testng.AssertJUnit.assertEquals;
@@ -36,7 +37,7 @@ public class TestMAccountableEntity {
   @Test
   public void testInitialization() {
     List<MConfig> configs = new ArrayList<MConfig>();
-    MIntegerInput input = new MIntegerInput("INTEGER-INPUT", false);
+    MIntegerInput input = new MIntegerInput("INTEGER-INPUT", false, InputEditable.ANY, StringUtils.EMPTY);
     List<MInput<?>> list = new ArrayList<MInput<?>>();
     list.add(input);
     MConfig config = new MConfig("CONFIGNAME", list);

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/test/java/org/apache/sqoop/model/TestMBooleanInput.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/model/TestMBooleanInput.java b/common/src/test/java/org/apache/sqoop/model/TestMBooleanInput.java
index 57d2da2..3e863fa 100644
--- a/common/src/test/java/org/apache/sqoop/model/TestMBooleanInput.java
+++ b/common/src/test/java/org/apache/sqoop/model/TestMBooleanInput.java
@@ -17,6 +17,7 @@
  */
 package org.apache.sqoop.model;
 
+import org.apache.commons.lang.StringUtils;
 import org.testng.annotations.Test;
 
 import static org.testng.AssertJUnit.assertEquals;
@@ -34,7 +35,8 @@ public class TestMBooleanInput {
    */
   @Test
   public void testInitialization() {
-    MBooleanInput input = new MBooleanInput("sqoopsqoop", true);
+    MBooleanInput input = new MBooleanInput("sqoopsqoop", true, InputEditable.ANY,
+        StringUtils.EMPTY);
     assertEquals("sqoopsqoop", input.getName());
     assertEquals(true, input.isSensitive());
     assertEquals(MInputType.BOOLEAN, input.getType());
@@ -46,17 +48,19 @@ public class TestMBooleanInput {
   @Test
   public void testEquals() {
     // Positive test
-    MBooleanInput input1 = new MBooleanInput("sqoopsqoop", true);
-    MBooleanInput input2 = new MBooleanInput("sqoopsqoop", true);
+    MBooleanInput input1 = new MBooleanInput("sqoopsqoop", true, InputEditable.ANY,
+        StringUtils.EMPTY);
+    MBooleanInput input2 = new MBooleanInput("sqoopsqoop", true, InputEditable.ANY,
+        StringUtils.EMPTY);
     assertTrue(input1.equals(input2));
 
     // Negative test
-    MBooleanInput input3 = new MBooleanInput("sqoopsqoop", false);
-    MBooleanInput input4 = new MBooleanInput("sqoopsqoop", true);
+    MBooleanInput input3 = new MBooleanInput("sqoopsqoop", false, InputEditable.ANY, StringUtils.EMPTY );
+    MBooleanInput input4 = new MBooleanInput("sqoopsqoop", true, InputEditable.ANY, StringUtils.EMPTY );
     assertFalse(input3.equals(input4));
 
-    MBooleanInput input5 = new MBooleanInput("sqoopsqoop", false);
-    MBooleanInput input6 = new MBooleanInput("sqoop", false);
+    MBooleanInput input5 = new MBooleanInput("sqoopsqoop", false, InputEditable.ANY, StringUtils.EMPTY );
+    MBooleanInput input6 = new MBooleanInput("sqoop", false, InputEditable.ANY, StringUtils.EMPTY );
     assertFalse(input5.equals(input6));
   }
 
@@ -65,7 +69,7 @@ public class TestMBooleanInput {
    */
   @Test
   public void testValue() {
-    MBooleanInput input1 = new MBooleanInput("sqoopsqoop", true);
+    MBooleanInput input1 = new MBooleanInput("sqoopsqoop", true, InputEditable.ANY, StringUtils.EMPTY );
     input1.setValue(true);
     assertEquals(true, input1.getValue().booleanValue());
     input1.setEmpty();
@@ -77,7 +81,7 @@ public class TestMBooleanInput {
    */
   @Test
   public void testUrlSafe() {
-    MBooleanInput input1 = new MBooleanInput("sqoopsqoop", true);
+    MBooleanInput input1 = new MBooleanInput("sqoopsqoop", true, InputEditable.ANY, StringUtils.EMPTY );
     input1.setValue(true);
     // Getting URL safe string
     String tmp = input1.getUrlSafeValueString();
@@ -91,7 +95,8 @@ public class TestMBooleanInput {
    */
   @Test
   public void testNamedElement() {
-    MBooleanInput input1 = new MBooleanInput("sqoopsqoop", true);
+    MBooleanInput input1 = new MBooleanInput("sqoopsqoop", true, InputEditable.ANY,
+        StringUtils.EMPTY);
     assertEquals("sqoopsqoop.label", input1.getLabelKey());
     assertEquals("sqoopsqoop.help", input1.getHelpKey());
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/test/java/org/apache/sqoop/model/TestMConfig.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/model/TestMConfig.java b/common/src/test/java/org/apache/sqoop/model/TestMConfig.java
index 908348d..665b64a 100644
--- a/common/src/test/java/org/apache/sqoop/model/TestMConfig.java
+++ b/common/src/test/java/org/apache/sqoop/model/TestMConfig.java
@@ -20,6 +20,7 @@ package org.apache.sqoop.model;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.commons.lang.StringUtils;
 import org.testng.annotations.Test;
 
 import static org.testng.Assert.*;
@@ -31,8 +32,8 @@ public class TestMConfig {
    */
   @Test
   public void testInitialization() {
-    MInput<String> input1 = new MStringInput("sqoopsqoop1", true, (short) 5);
-    MInput<String> input2 = new MStringInput("sqoopsqoop2", true, (short) 5);
+    MInput<String> input1 = new MStringInput("sqoopsqoop1", true, InputEditable.ANY, StringUtils.EMPTY , (short) 5);
+    MInput<String> input2 = new MStringInput("sqoopsqoop2", true, InputEditable.ANY, StringUtils.EMPTY , (short) 5);
 
     List<MInput<?>> list = new ArrayList<MInput<?>>();
     list.add(input1);
@@ -48,15 +49,15 @@ public class TestMConfig {
    */
   @Test
   public void testEquals() {
-    MInput<Integer> input1 = new MIntegerInput("sqoopsqoop1", false);
-    MInput<Integer> input2 = new MIntegerInput("sqoopsqoop2", false);
+    MInput<Integer> input1 = new MIntegerInput("sqoopsqoop1", false, InputEditable.ANY, StringUtils.EMPTY );
+    MInput<Integer> input2 = new MIntegerInput("sqoopsqoop2", false, InputEditable.ANY, StringUtils.EMPTY );
     List<MInput<?>> list1 = new ArrayList<MInput<?>>();
     list1.add(input1);
     list1.add(input2);
     MConfig mform1 = new MConfig("config", list1);
 
-    MInput<Integer> input3 = new MIntegerInput("sqoopsqoop1", false);
-    MInput<Integer> input4 = new MIntegerInput("sqoopsqoop2", false);
+    MInput<Integer> input3 = new MIntegerInput("sqoopsqoop1", false, InputEditable.ANY, StringUtils.EMPTY );
+    MInput<Integer> input4 = new MIntegerInput("sqoopsqoop2", false, InputEditable.ANY, StringUtils.EMPTY );
     List<MInput<?>> list2 = new ArrayList<MInput<?>>();
     list2.add(input3);
     list2.add(input4);
@@ -66,10 +67,12 @@ public class TestMConfig {
 
   @Test
   public void testGetInputs() {
-    MIntegerInput intInput = new MIntegerInput("Config.A", false);
-    MMapInput mapInput = new MMapInput("Config.B", false);
-    MStringInput stringInput = new MStringInput("Config.C", false, (short)3);
-    MEnumInput enumInput = new MEnumInput("Config.D", false, new String[] {"I", "V"});
+    MIntegerInput intInput = new MIntegerInput("Config.A", false, InputEditable.ANY, StringUtils.EMPTY );
+    MMapInput mapInput = new MMapInput("Config.B", false, InputEditable.ANY, StringUtils.EMPTY );
+    MStringInput stringInput = new MStringInput("Config.C", false, InputEditable.ANY,
+        StringUtils.EMPTY, (short) 3);
+    MEnumInput enumInput = new MEnumInput("Config.D", false, InputEditable.ANY, StringUtils.EMPTY,
+        new String[] { "I", "V" });
 
     List<MInput<?>> inputs = new ArrayList<MInput<?>>();
     inputs.add(intInput);

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/test/java/org/apache/sqoop/model/TestMConfigList.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/model/TestMConfigList.java b/common/src/test/java/org/apache/sqoop/model/TestMConfigList.java
index 4f8261c..6550dde 100644
--- a/common/src/test/java/org/apache/sqoop/model/TestMConfigList.java
+++ b/common/src/test/java/org/apache/sqoop/model/TestMConfigList.java
@@ -17,6 +17,7 @@
  */
 package org.apache.sqoop.model;
 
+import org.apache.commons.lang.StringUtils;
 import org.testng.annotations.Test;
 
 import java.util.ArrayList;
@@ -30,16 +31,18 @@ public class TestMConfigList {
   public void testGetInputs() {
     List<MConfig> configs = new LinkedList<MConfig>();
 
-    MIntegerInput intInput = new MIntegerInput("Config1.A", false);
-    MMapInput mapInput = new MMapInput("Config1.B", false);
+    MIntegerInput intInput = new MIntegerInput("Config1.A", false, InputEditable.ANY, StringUtils.EMPTY);
+    MMapInput mapInput = new MMapInput("Config1.B", false, InputEditable.ANY, StringUtils.EMPTY);
 
     List<MInput<?>> inputs = new ArrayList<MInput<?>>();
     inputs.add(intInput);
     inputs.add(mapInput);
     configs.add(new MConfig("Config1", inputs));
 
-    MStringInput stringInput = new MStringInput("Config2.C", false, (short)3);
-    MEnumInput enumInput = new MEnumInput("Config2.D", false, new String[] {"I", "V"});
+    MStringInput stringInput = new MStringInput("Config2.C", false, InputEditable.ANY,
+        StringUtils.EMPTY, (short) 3);
+    MEnumInput enumInput = new MEnumInput("Config2.D", false, InputEditable.ANY, StringUtils.EMPTY,
+        new String[] { "I", "V" });
 
     inputs = new ArrayList<MInput<?>>();
     inputs.add(stringInput);

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/test/java/org/apache/sqoop/model/TestMConnector.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/model/TestMConnector.java b/common/src/test/java/org/apache/sqoop/model/TestMConnector.java
index e1c02aa..cefa9f2 100644
--- a/common/src/test/java/org/apache/sqoop/model/TestMConnector.java
+++ b/common/src/test/java/org/apache/sqoop/model/TestMConnector.java
@@ -28,6 +28,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.sqoop.common.Direction;
 import org.testng.annotations.Test;
 
@@ -35,9 +36,9 @@ public class TestMConnector {
 
   private MConnector createConnector(List<Direction> supportedDirections) {
     List<MConfig> configs = new ArrayList<MConfig>();
-    MIntegerInput inputs = new MIntegerInput("INTEGER-INPUT", false);
+    MIntegerInput inputs = new MIntegerInput("INTEGER-INPUT", false, InputEditable.ANY, StringUtils.EMPTY);
     inputs.setValue(100);
-    MStringInput strInput = new MStringInput("STRING-INPUT",false,(short)20);
+    MStringInput strInput = new MStringInput("STRING-INPUT",false, InputEditable.ANY, StringUtils.EMPTY, (short)20);
     strInput.setValue("TEST-VALUE");
     List<MInput<?>> list = new ArrayList<MInput<?>>();
     list.add(inputs);

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/test/java/org/apache/sqoop/model/TestMEnumInput.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/model/TestMEnumInput.java b/common/src/test/java/org/apache/sqoop/model/TestMEnumInput.java
index 39f09ce..744ac50 100644
--- a/common/src/test/java/org/apache/sqoop/model/TestMEnumInput.java
+++ b/common/src/test/java/org/apache/sqoop/model/TestMEnumInput.java
@@ -17,6 +17,7 @@
  */
 package org.apache.sqoop.model;
 
+import org.apache.commons.lang.StringUtils;
 import org.testng.annotations.Test;
 
 import static org.testng.AssertJUnit.*;
@@ -33,18 +34,19 @@ public class TestMEnumInput {
   @Test
   public void testInitialization() {
     String[] values = { "value1", "value2" };
-    MEnumInput input = new MEnumInput("NAME", false, values);
+    MEnumInput input = new MEnumInput("NAME", false, InputEditable.ANY, StringUtils.EMPTY, values);
     assertEquals("NAME", input.getName());
     assertArrayEquals(values, input.getValues());
     assertEquals(MInputType.ENUM, input.getType());
 
-    MEnumInput input1 = new MEnumInput("NAME", false, values);
+    MEnumInput input1 = new MEnumInput("NAME", false, InputEditable.ANY, StringUtils.EMPTY, values);
     assertEquals(input1, input);
     String[] testVal = { "val", "test" };
-    MEnumInput input2 = new MEnumInput("NAME1", false, testVal);
+    MEnumInput input2 = new MEnumInput("NAME1", false, InputEditable.ANY, StringUtils.EMPTY,
+        testVal);
     assertFalse(input1.equals(input2));
 
-    MEnumInput input3 = new MEnumInput("NAME", false, values);
+    MEnumInput input3 = new MEnumInput("NAME", false, InputEditable.ANY, StringUtils.EMPTY, values);
     input3.setValue(Enumeration.value1);
     assertEquals("value1", input3.getValue());
   }
@@ -55,8 +57,8 @@ public class TestMEnumInput {
   @Test
   public void testSensitivity() {
     String[] values = { "value1", "value2" };
-    MEnumInput input1 = new MEnumInput("NAME", false, values);
-    MEnumInput input2 = new MEnumInput("NAME", true, values);
+    MEnumInput input1 = new MEnumInput("NAME", false, InputEditable.ANY, StringUtils.EMPTY, values);
+    MEnumInput input2 = new MEnumInput("NAME", true, InputEditable.ANY, StringUtils.EMPTY, values);
     assertFalse(input1.isSensitive());
     assertTrue(input2.isSensitive());
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/test/java/org/apache/sqoop/model/TestMIntegerInput.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/model/TestMIntegerInput.java b/common/src/test/java/org/apache/sqoop/model/TestMIntegerInput.java
index 69e511f..2c3decf 100644
--- a/common/src/test/java/org/apache/sqoop/model/TestMIntegerInput.java
+++ b/common/src/test/java/org/apache/sqoop/model/TestMIntegerInput.java
@@ -17,6 +17,7 @@
  */
 package org.apache.sqoop.model;
 
+import org.apache.commons.lang.StringUtils;
 import org.testng.annotations.Test;
 
 import static org.testng.AssertJUnit.assertEquals;
@@ -33,7 +34,7 @@ public class TestMIntegerInput {
    */
   @Test
   public void testInitialization() {
-    MIntegerInput input = new MIntegerInput("sqoopsqoop", false);
+    MIntegerInput input = new MIntegerInput("sqoopsqoop", false, InputEditable.ANY, StringUtils.EMPTY);
     assertEquals("sqoopsqoop", input.getName());
     assertEquals(MInputType.INTEGER, input.getType());
   }
@@ -44,13 +45,13 @@ public class TestMIntegerInput {
   @Test
   public void testEquals() {
     // Positive test
-    MIntegerInput input1 = new MIntegerInput("sqoopsqoop", false);
-    MIntegerInput input2 = new MIntegerInput("sqoopsqoop", false);
+    MIntegerInput input1 = new MIntegerInput("sqoopsqoop", false, InputEditable.ANY, StringUtils.EMPTY);
+    MIntegerInput input2 = new MIntegerInput("sqoopsqoop", false, InputEditable.ANY, StringUtils.EMPTY);
     assertTrue(input1.equals(input2));
 
     // Negative test
-    MIntegerInput input3 = new MIntegerInput("sqoopsqoop", false);
-    MIntegerInput input4 = new MIntegerInput("sqoopsqoop1", false);
+    MIntegerInput input3 = new MIntegerInput("sqoopsqoop", false, InputEditable.ANY, StringUtils.EMPTY);
+    MIntegerInput input4 = new MIntegerInput("sqoopsqoop1", false, InputEditable.ANY, StringUtils.EMPTY);
     assertFalse(input3.equals(input4));
   }
 
@@ -59,7 +60,7 @@ public class TestMIntegerInput {
    */
   @Test
   public void testValue() {
-    MIntegerInput input1 = new MIntegerInput("sqoopsqoop", false);
+    MIntegerInput input1 = new MIntegerInput("sqoopsqoop", false, InputEditable.ANY, StringUtils.EMPTY);
     input1.setValue(99);
     assertEquals(new Integer(99), input1.getValue());
     input1.setEmpty();
@@ -71,7 +72,7 @@ public class TestMIntegerInput {
    */
   @Test
   public void testUrlSafe() {
-    MIntegerInput input1 = new MIntegerInput("sqoopsqoop", false);
+    MIntegerInput input1 = new MIntegerInput("sqoopsqoop", false, InputEditable.ANY, StringUtils.EMPTY);
     input1.setValue(1001);
     // Getting URL safe string
     String tmp = input1.getUrlSafeValueString();
@@ -85,7 +86,7 @@ public class TestMIntegerInput {
    */
   @Test
   public void testNamedElement() {
-    MStringInput input1 = new MStringInput("sqoopsqoop", true, (short) 5);
+    MStringInput input1 = new MStringInput("sqoopsqoop", true, InputEditable.CONNECTOR_ONLY, StringUtils.EMPTY, (short) 5);
     assertEquals("sqoopsqoop.label", input1.getLabelKey());
     assertEquals("sqoopsqoop.help", input1.getHelpKey());
   }
@@ -95,8 +96,8 @@ public class TestMIntegerInput {
    */
   @Test
   public void testSensitivity() {
-    MIntegerInput input1 = new MIntegerInput("NAME", false);
-    MIntegerInput input2 = new MIntegerInput("NAME", true);
+    MIntegerInput input1 = new MIntegerInput("NAME", false, InputEditable.USER_ONLY, StringUtils.EMPTY);
+    MIntegerInput input2 = new MIntegerInput("NAME", true, InputEditable.CONNECTOR_ONLY, StringUtils.EMPTY);
     assertFalse(input1.isSensitive());
     assertTrue(input2.isSensitive());
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/test/java/org/apache/sqoop/model/TestMJob.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/model/TestMJob.java b/common/src/test/java/org/apache/sqoop/model/TestMJob.java
index 605f429..ee8f45f 100644
--- a/common/src/test/java/org/apache/sqoop/model/TestMJob.java
+++ b/common/src/test/java/org/apache/sqoop/model/TestMJob.java
@@ -23,6 +23,7 @@ import static org.testng.Assert.assertNull;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.sqoop.common.Direction;
 import org.testng.annotations.Test;
 
@@ -104,9 +105,9 @@ public class TestMJob {
 
   private MFromConfig fromConfig() {
     List<MConfig> configs = new ArrayList<MConfig>();
-    MIntegerInput input = new MIntegerInput("INTEGER-INPUT", false);
+    MIntegerInput input = new MIntegerInput("INTEGER-INPUT", false, InputEditable.ANY, StringUtils.EMPTY);
     input.setValue(100);
-    MStringInput strInput = new MStringInput("STRING-INPUT",false,(short)20);
+    MStringInput strInput = new MStringInput("STRING-INPUT",false, InputEditable.ANY, StringUtils.EMPTY, (short)20);
     strInput.setValue("TEST-VALUE");
     List<MInput<?>> list = new ArrayList<MInput<?>>();
     list.add(input);
@@ -118,7 +119,7 @@ public class TestMJob {
 
   private MToConfig toConfig() {
     List<MConfig> configs = new ArrayList<MConfig>();
-    MMapInput input = new MMapInput("MAP-INPUT", false);
+    MMapInput input = new MMapInput("MAP-INPUT", false, InputEditable.ANY, StringUtils.EMPTY);
     List<MInput<?>> list = new ArrayList<MInput<?>>();
     list.add(input);
     MConfig config = new MConfig("CONFIGTONAME", list);
@@ -128,7 +129,7 @@ public class TestMJob {
 
   private MDriverConfig driverConfig() {
     List<MConfig> configs = new ArrayList<MConfig>();
-    MMapInput input = new MMapInput("MAP-INPUT", false);
+    MMapInput input = new MMapInput("MAP-INPUT", false, InputEditable.ANY, StringUtils.EMPTY);
     List<MInput<?>> list = new ArrayList<MInput<?>>();
     list.add(input);
     MConfig config = new MConfig("CONFIGDRIVERNAME", list);

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/test/java/org/apache/sqoop/model/TestMLink.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/model/TestMLink.java b/common/src/test/java/org/apache/sqoop/model/TestMLink.java
index e985c17..f22719f 100644
--- a/common/src/test/java/org/apache/sqoop/model/TestMLink.java
+++ b/common/src/test/java/org/apache/sqoop/model/TestMLink.java
@@ -20,6 +20,7 @@ package org.apache.sqoop.model;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.commons.lang.StringUtils;
 import org.testng.annotations.Test;
 
 import static org.testng.Assert.*;
@@ -81,9 +82,9 @@ public class TestMLink {
 
   private MLinkConfig linkConfig() {
     List<MConfig> configs = new ArrayList<MConfig>();
-    MIntegerInput input = new MIntegerInput("INTEGER-INPUT", false);
+    MIntegerInput input = new MIntegerInput("INTEGER-INPUT", false, InputEditable.ANY, StringUtils.EMPTY);
     input.setValue(100);
-    MStringInput strInput = new MStringInput("STRING-INPUT",false,(short)20);
+    MStringInput strInput = new MStringInput("STRING-INPUT",false, InputEditable.ANY, StringUtils.EMPTY, (short)20);
     strInput.setValue("TEST-VALUE");
     List<MInput<?>> list = new ArrayList<MInput<?>>();
     list.add(input);

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/test/java/org/apache/sqoop/model/TestMMapInput.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/model/TestMMapInput.java b/common/src/test/java/org/apache/sqoop/model/TestMMapInput.java
index c9b786b..53a3549 100644
--- a/common/src/test/java/org/apache/sqoop/model/TestMMapInput.java
+++ b/common/src/test/java/org/apache/sqoop/model/TestMMapInput.java
@@ -20,6 +20,7 @@ package org.apache.sqoop.model;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.commons.lang.StringUtils;
 import org.testng.annotations.Test;
 
 import static org.testng.AssertJUnit.assertEquals;
@@ -37,7 +38,7 @@ public class TestMMapInput {
    */
   @Test
   public void testInitialization() {
-    MMapInput input = new MMapInput("sqoopsqoop", false);
+    MMapInput input = new MMapInput("sqoopsqoop", false, InputEditable.ANY, StringUtils.EMPTY);
     assertEquals("sqoopsqoop", input.getName());
     assertEquals(MInputType.MAP, input.getType());
   }
@@ -48,13 +49,13 @@ public class TestMMapInput {
   @Test
   public void testEquals() {
     // Positive test
-    MMapInput input1 = new MMapInput("sqoopsqoop", false);
-    MMapInput input2 = new MMapInput("sqoopsqoop", false);
+    MMapInput input1 = new MMapInput("sqoopsqoop", false, InputEditable.ANY, StringUtils.EMPTY);
+    MMapInput input2 = new MMapInput("sqoopsqoop", false, InputEditable.ANY, StringUtils.EMPTY);
     assertTrue(input1.equals(input2));
 
     // Negative test
-    MMapInput input3 = new MMapInput("sqoopsqoop", false);
-    MMapInput input4 = new MMapInput("sqoopsqoop1", false);
+    MMapInput input3 = new MMapInput("sqoopsqoop", false, InputEditable.ANY, StringUtils.EMPTY);
+    MMapInput input4 = new MMapInput("sqoopsqoop1", false, InputEditable.ANY, StringUtils.EMPTY);
     assertFalse(input3.equals(input4));
   }
 
@@ -63,7 +64,7 @@ public class TestMMapInput {
    */
   @Test
   public void testValue() {
-    MMapInput input1 = new MMapInput("sqoopsqoop", false);
+    MMapInput input1 = new MMapInput("sqoopsqoop", false, InputEditable.ANY, StringUtils.EMPTY);
     Map<String, String> map1 = new HashMap<String, String>();
     input1.setValue(map1);
     assertEquals(map1, input1.getValue());
@@ -76,7 +77,7 @@ public class TestMMapInput {
    */
   @Test
   public void testUrlSafe() {
-    MMapInput input1 = new MMapInput("sqoopsqoop", false);
+    MMapInput input1 = new MMapInput("sqoopsqoop", false, InputEditable.ANY, StringUtils.EMPTY);
     Map<String, String> map1 = new HashMap<String, String>();
     input1.setValue(map1);
     // Getting URL safe string
@@ -97,7 +98,7 @@ public class TestMMapInput {
    */
   @Test
   public void testNamedElement() {
-    MStringInput input1 = new MStringInput("sqoopsqoop", true, (short) 5);
+    MStringInput input1 = new MStringInput("sqoopsqoop", true, InputEditable.ANY, StringUtils.EMPTY, (short) 5);
     assertEquals("sqoopsqoop.label", input1.getLabelKey());
     assertEquals("sqoopsqoop.help", input1.getHelpKey());
   }
@@ -107,8 +108,8 @@ public class TestMMapInput {
    */
   @Test
   public void testSensitivity() {
-    MMapInput input1 = new MMapInput("NAME", false);
-    MMapInput input2 = new MMapInput("NAME", true);
+    MMapInput input1 = new MMapInput("NAME", false, InputEditable.ANY, StringUtils.EMPTY );
+    MMapInput input2 = new MMapInput("NAME", true, InputEditable.ANY, StringUtils.EMPTY );
     assertFalse(input1.isSensitive());
     assertTrue(input2.isSensitive());
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/test/java/org/apache/sqoop/model/TestMNamedElement.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/model/TestMNamedElement.java b/common/src/test/java/org/apache/sqoop/model/TestMNamedElement.java
index f851cbd..c523d65 100644
--- a/common/src/test/java/org/apache/sqoop/model/TestMNamedElement.java
+++ b/common/src/test/java/org/apache/sqoop/model/TestMNamedElement.java
@@ -17,6 +17,7 @@
  */
 package org.apache.sqoop.model;
 
+import org.apache.commons.lang.StringUtils;
 import org.testng.annotations.Test;
 
 import static org.testng.Assert.*;
@@ -31,7 +32,7 @@ public class TestMNamedElement {
    */
   @Test
   public void testInitialization() {
-    MNamedElement named = new MIntegerInput("SQOOP", false);
+    MNamedElement named = new MIntegerInput("SQOOP", false, InputEditable.ANY, StringUtils.EMPTY);
     assertEquals("SQOOP", named.getName());
     assertEquals("SQOOP.label", named.getLabelKey());
     assertEquals("SQOOP.help", named.getHelpKey());

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/test/java/org/apache/sqoop/model/TestMStringInput.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/model/TestMStringInput.java b/common/src/test/java/org/apache/sqoop/model/TestMStringInput.java
index a4faf95..37a04c2 100644
--- a/common/src/test/java/org/apache/sqoop/model/TestMStringInput.java
+++ b/common/src/test/java/org/apache/sqoop/model/TestMStringInput.java
@@ -17,6 +17,7 @@
  */
 package org.apache.sqoop.model;
 
+import org.apache.commons.lang.StringUtils;
 import org.testng.annotations.Test;
 
 import static org.testng.Assert.*;
@@ -32,7 +33,8 @@ public class TestMStringInput {
   @Test
   public void testInitialization() {
     short len = 6;
-    MStringInput input = new MStringInput("sqoopsqoop", true, len);
+    MStringInput input = new MStringInput("sqoopsqoop", true, InputEditable.ANY, StringUtils.EMPTY,
+        len);
     assertEquals("sqoopsqoop", input.getName());
     assertEquals(true, input.isSensitive());
     assertEquals(len, input.getMaxLength());
@@ -46,13 +48,17 @@ public class TestMStringInput {
   public void testEquals() {
     short len = 6;
     // Positive test
-    MStringInput input1 = new MStringInput("sqoopsqoop", true, len);
-    MStringInput input2 = new MStringInput("sqoopsqoop", true, len);
+    MStringInput input1 = new MStringInput("sqoopsqoop", true, InputEditable.ANY,
+        StringUtils.EMPTY, len);
+    MStringInput input2 = new MStringInput("sqoopsqoop", true, InputEditable.ANY,
+        StringUtils.EMPTY, len);
     assertTrue(input1.equals(input2));
 
     // Negative test
-    MStringInput input3 = new MStringInput("sqoopsqoop", false, len);
-    MStringInput input4 = new MStringInput("sqoopsqoop", true, len);
+    MStringInput input3 = new MStringInput("sqoopsqoop", false, InputEditable.ANY,
+        StringUtils.EMPTY, len);
+    MStringInput input4 = new MStringInput("sqoopsqoop", true, InputEditable.ANY,
+        StringUtils.EMPTY, len);
     assertFalse(input3.equals(input4));
   }
 
@@ -61,7 +67,8 @@ public class TestMStringInput {
    */
   @Test
   public void testValue() {
-    MStringInput input1 = new MStringInput("sqoopsqoop", true, (short) 5);
+    MStringInput input1 = new MStringInput("sqoopsqoop", true, InputEditable.ANY,
+        StringUtils.EMPTY, (short) 5);
     input1.setValue("sqoop");
     assertEquals("sqoop", input1.getValue());
     input1.setEmpty();
@@ -73,7 +80,8 @@ public class TestMStringInput {
    */
   @Test
   public void testUrlSafe() {
-    MStringInput input1 = new MStringInput("sqoopsqoop", true, (short) 5);
+    MStringInput input1 = new MStringInput("sqoopsqoop", true, InputEditable.ANY,
+        StringUtils.EMPTY, (short) 5);
     String s = "Sqoop%$!@#&*()Sqoop";
     input1.setValue(s);
     // Getting URL safe string
@@ -88,7 +96,8 @@ public class TestMStringInput {
    */
   @Test
   public void testNamedElement() {
-    MStringInput input1 = new MStringInput("sqoopsqoop", true, (short) 5);
+    MStringInput input1 = new MStringInput("sqoopsqoop", true, InputEditable.ANY,
+        StringUtils.EMPTY, (short) 5);
     assertEquals("sqoopsqoop.label", input1.getLabelKey());
     assertEquals("sqoopsqoop.help", input1.getHelpKey());
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6fc50b08/common/src/test/java/org/apache/sqoop/model/TestMValidatedElement.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/sqoop/model/TestMValidatedElement.java b/common/src/test/java/org/apache/sqoop/model/TestMValidatedElement.java
index f0bdda4..6fee4b5 100644
--- a/common/src/test/java/org/apache/sqoop/model/TestMValidatedElement.java
+++ b/common/src/test/java/org/apache/sqoop/model/TestMValidatedElement.java
@@ -17,6 +17,7 @@
  */
 package org.apache.sqoop.model;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.sqoop.validation.Message;
 import org.apache.sqoop.validation.Status;
 import org.testng.annotations.Test;
@@ -33,7 +34,7 @@ public class TestMValidatedElement {
    */
   @Test
   public void testInitialization() {
-    MValidatedElement input = new MIntegerInput("input", false);
+    MValidatedElement input = new MIntegerInput("input", false,InputEditable.ANY, StringUtils.EMPTY );
     assertEquals("input", input.getName());
     assertEquals(Status.OK, input.getValidationStatus());
   }
@@ -43,7 +44,7 @@ public class TestMValidatedElement {
    */
   @Test
   public void testVarious() {
-    MValidatedElement input = new MIntegerInput("input", false);
+    MValidatedElement input = new MIntegerInput("input", false, InputEditable.ANY, StringUtils.EMPTY );
 
     // Default status
     assertEquals(Status.OK, input.getValidationStatus());