You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sqoop.apache.org by ab...@apache.org on 2014/10/10 04:52:09 UTC

[40/52] [abbrv] SQOOP-1498: Sqoop2: Repository Object refactoring (objects prefixed with M)

http://git-wip-us.apache.org/repos/asf/sqoop/blob/8362c73c/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 5dd7970..39702ca 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
@@ -28,31 +28,38 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.sql.Timestamp;
 import java.sql.Types;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
 
-import org.apache.log4j.Logger;
 import org.apache.commons.lang.StringUtils;
+import org.apache.log4j.Logger;
 import org.apache.sqoop.common.Direction;
 import org.apache.sqoop.common.DirectionError;
 import org.apache.sqoop.common.SqoopException;
 import org.apache.sqoop.connector.ConnectorHandler;
 import org.apache.sqoop.connector.ConnectorManagerUtils;
 import org.apache.sqoop.model.MBooleanInput;
-import org.apache.sqoop.model.MLink;
-import org.apache.sqoop.model.MConnectionForms;
-import org.apache.sqoop.model.MEnumInput;
-import org.apache.sqoop.model.MIntegerInput;
-import org.apache.sqoop.model.MJob;
-import org.apache.sqoop.model.MJobForms;
+import org.apache.sqoop.model.MConfig;
+import org.apache.sqoop.model.MConfigType;
 import org.apache.sqoop.model.MConnector;
-import org.apache.sqoop.model.MForm;
-import org.apache.sqoop.model.MFormType;
+import org.apache.sqoop.model.MDriver;
 import org.apache.sqoop.model.MDriverConfig;
+import org.apache.sqoop.model.MEnumInput;
+import org.apache.sqoop.model.MFromConfig;
 import org.apache.sqoop.model.MInput;
 import org.apache.sqoop.model.MInputType;
+import org.apache.sqoop.model.MIntegerInput;
+import org.apache.sqoop.model.MJob;
+import org.apache.sqoop.model.MLink;
+import org.apache.sqoop.model.MLinkConfig;
 import org.apache.sqoop.model.MMapInput;
 import org.apache.sqoop.model.MStringInput;
 import org.apache.sqoop.model.MSubmission;
+import org.apache.sqoop.model.MToConfig;
 import org.apache.sqoop.repository.JdbcRepositoryContext;
 import org.apache.sqoop.repository.JdbcRepositoryHandler;
 import org.apache.sqoop.submission.SubmissionStatus;
@@ -93,74 +100,69 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
         mc.getUniqueName());
     }
     mc.setPersistenceId(getConnectorId(mc, conn));
-    insertFormsForConnector(mc, conn);
+    insertConfigsForConnector(mc, conn);
   }
 
   /**
-   * Helper method to insert the forms from the  into the
-   * repository. The job and connector forms within <code>mc</code> will get
-   * updated with the id of the forms when this function returns.
-   * @param mDriverConfig The MFramework instance to use to upgrade.
-   * @param conn JDBC link to use for updating the forms
+   * Helper method to insert the configs from the  into the
+   * repository.
+   * @param mDriver The driver instance to use to upgrade.
+   * @param conn JDBC link to use for updating the configs
    */
-  private void insertFormsForFramework(MDriverConfig mDriverConfig, Connection conn) {
-    PreparedStatement baseFormStmt = null;
+  private void insertConfigsForDriver(MDriver mDriver, Connection conn) {
+    PreparedStatement baseConfigStmt = null;
     PreparedStatement baseInputStmt = null;
     try{
-      baseFormStmt = conn.prepareStatement(STMT_INSERT_FORM_BASE,
+      baseConfigStmt = conn.prepareStatement(STMT_INSERT_CONFIG_BASE,
         Statement.RETURN_GENERATED_KEYS);
 
       baseInputStmt = conn.prepareStatement(STMT_INSERT_INPUT_BASE,
         Statement.RETURN_GENERATED_KEYS);
 
-      // Register connector forms
-      registerForms(null, null, mDriverConfig.getConnectionForms().getForms(),
-        MFormType.CONNECTION.name(), baseFormStmt, baseInputStmt);
-
-      // Register job forms
-      registerForms(null, null, mDriverConfig.getJobForms().getForms(),
-        MFormType.JOB.name(), baseFormStmt, baseInputStmt);
+      // Register the job config type, since driver config is per job
+      registerConfigs(null, null, mDriver.getDriverConfig().getConfigs(),
+        MConfigType.JOB.name(), baseConfigStmt, baseInputStmt);
 
     } catch (SQLException ex) {
-      throw new SqoopException(DerbyRepoError.DERBYREPO_0014, mDriverConfig.toString(), ex);
+      throw new SqoopException(DerbyRepoError.DERBYREPO_0014, mDriver.toString(), ex);
     } finally {
-      closeStatements(baseFormStmt, baseInputStmt);
+      closeStatements(baseConfigStmt, baseInputStmt);
     }
   }
 
   /**
-   * Helper method to insert the forms from the MConnector into the
-   * repository. The job and connector forms within <code>mc</code> will get
-   * updated with the id of the forms when this function returns.
-   * @param mc The connector to use for updating forms
-   * @param conn JDBC link to use for updating the forms
+   * Helper method to insert the configs from the MConnector into the
+   * repository. The job and connector configs within <code>mc</code> will get
+   * updated with the id of the configs when this function returns.
+   * @param mc The connector to use for updating configs
+   * @param conn JDBC link to use for updating the configs
    */
-  private void insertFormsForConnector (MConnector mc, Connection conn) {
+  private void insertConfigsForConnector (MConnector mc, Connection conn) {
     long connectorId = mc.getPersistenceId();
-    PreparedStatement baseFormStmt = null;
+    PreparedStatement baseConfigStmt = null;
     PreparedStatement baseInputStmt = null;
     try{
-      baseFormStmt = conn.prepareStatement(STMT_INSERT_FORM_BASE,
+      baseConfigStmt = conn.prepareStatement(STMT_INSERT_CONFIG_BASE,
         Statement.RETURN_GENERATED_KEYS);
 
       baseInputStmt = conn.prepareStatement(STMT_INSERT_INPUT_BASE,
         Statement.RETURN_GENERATED_KEYS);
 
-      // Register connector forms
-      registerForms(connectorId, null, mc.getConnectionForms().getForms(),
-        MFormType.CONNECTION.name(), baseFormStmt, baseInputStmt);
+      // Register link type config
+      registerConfigs(connectorId, null, mc.getLinkConfig().getConfigs(),
+        MConfigType.LINK.name(), baseConfigStmt, baseInputStmt);
 
-      // Register all jobs
-      registerForms(connectorId, Direction.FROM, mc.getJobForms(Direction.FROM).getForms(),
-        MFormType.JOB.name(), baseFormStmt, baseInputStmt);
-      registerForms(connectorId, Direction.TO, mc.getJobForms(Direction.TO).getForms(),
-        MFormType.JOB.name(), baseFormStmt, baseInputStmt);
+      // Register both from/to job type config
+      registerConfigs(connectorId, Direction.FROM, mc.getConfig(Direction.FROM).getConfigs(),
+        MConfigType.JOB.name(), baseConfigStmt, baseInputStmt);
+      registerConfigs(connectorId, Direction.TO, mc.getConfig(Direction.TO).getConfigs(),
+        MConfigType.JOB.name(), baseConfigStmt, baseInputStmt);
 
     } catch (SQLException ex) {
       throw new SqoopException(DerbyRepoError.DERBYREPO_0014,
         mc.toString(), ex);
     } finally {
-      closeStatements(baseFormStmt, baseInputStmt);
+      closeStatements(baseConfigStmt, baseInputStmt);
     }
 
   }
@@ -239,7 +241,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
         }
       } else {
         LOG.warn("Even though embedded Derby driver was loaded, the connect "
-            + "URL is of an unexpected form: " + connectUrl + ". Therefore no "
+            + "URL is of an unexpected config: " + connectUrl + ". Therefore no "
             + "attempt will be made to shutdown embedded Derby instance.");
       }
     }
@@ -335,9 +337,9 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
   /**
    * Create or update driver version
    * @param conn Connection to the the repository
-   * @param mDriverConfig
+   * @param mDriver
    */
-  private void createOrUpdateDriverVersion(Connection conn, MDriverConfig mDriverConfig) {
+  private void createOrUpdateDriverSystemVersion(Connection conn, String version) {
     ResultSet rs = null;
     PreparedStatement stmt = null;
     try {
@@ -348,7 +350,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
 
       stmt = conn.prepareStatement(STMT_INSERT_SYSTEM);
       stmt.setString(1, DerbyRepoConstants.SYSKEY_DRIVER_VERSION);
-      stmt.setString(2, mDriverConfig.getVersion());
+      stmt.setString(2, version);
       stmt.executeUpdate();
     } catch (SQLException e) {
       logException(e);
@@ -369,11 +371,11 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
     if(version <= 0) {
       runQuery(QUERY_CREATE_SCHEMA_SQOOP, conn);
       runQuery(QUERY_CREATE_TABLE_SQ_CONNECTOR, conn);
-      runQuery(QUERY_CREATE_TABLE_SQ_FORM, conn);
+      runQuery(QUERY_CREATE_TABLE_SQ_CONFIG, conn);
       runQuery(QUERY_CREATE_TABLE_SQ_INPUT, conn);
-      runQuery(QUERY_CREATE_TABLE_SQ_CONNECTION, conn);
+      runQuery(QUERY_CREATE_TABLE_SQ_LINK, conn);
       runQuery(QUERY_CREATE_TABLE_SQ_JOB, conn);
-      runQuery(QUERY_CREATE_TABLE_SQ_CONNECTION_INPUT, conn);
+      runQuery(QUERY_CREATE_TABLE_SQ_LINK_INPUT, conn);
       runQuery(QUERY_CREATE_TABLE_SQ_JOB_INPUT, conn);
       runQuery(QUERY_CREATE_TABLE_SQ_SUBMISSION, conn);
       runQuery(QUERY_CREATE_TABLE_SQ_COUNTER_GROUP, conn);
@@ -382,10 +384,10 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
     }
     if(version <= 1) {
       runQuery(QUERY_CREATE_TABLE_SQ_SYSTEM, conn);
-      runQuery(QUERY_UPGRADE_TABLE_SQ_CONNECTION_ADD_COLUMN_ENABLED, conn);
+      runQuery(QUERY_UPGRADE_TABLE_SQ_LINK_ADD_COLUMN_ENABLED, conn);
       runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_ADD_COLUMN_ENABLED, conn);
-      runQuery(QUERY_UPGRADE_TABLE_SQ_CONNECTION_ADD_COLUMN_CREATION_USER, conn);
-      runQuery(QUERY_UPGRADE_TABLE_SQ_CONNECTION_ADD_COLUMN_UPDATE_USER, conn);
+      runQuery(QUERY_UPGRADE_TABLE_SQ_LINK_ADD_COLUMN_CREATION_USER, conn);
+      runQuery(QUERY_UPGRADE_TABLE_SQ_LINK_ADD_COLUMN_UPDATE_USER, conn);
       runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_ADD_COLUMN_CREATION_USER, conn);
       runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_ADD_COLUMN_UPDATE_USER, conn);
       runQuery(QUERY_UPGRADE_TABLE_SQ_SUBMISSION_ADD_COLUMN_CREATION_USER, conn);
@@ -397,17 +399,17 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
     }
     if(version <= 3) {
       // Schema modifications
-      runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_RENAME_COLUMN_SQF_OPERATION_TO_SQF_DIRECTION, conn);
-      runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_RENAME_COLUMN_SQB_CONNECTION_TO_SQB_FROM_CONNECTION, conn);
-      runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_ADD_COLUMN_SQB_TO_CONNECTION, conn);
-      runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_REMOVE_CONSTRAINT_SQB_SQN, conn);
-      runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_ADD_CONSTRAINT_SQB_SQN_FROM, conn);
-      runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_ADD_CONSTRAINT_SQB_SQN_TO, conn);
+      runQuery(QUERY_UPGRADE_TABLE_SQ_CONFIG_RENAME_COLUMN_SQ_CFG_OPERATION_TO_SQ_CFG_DIRECTION, conn);
+      runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_RENAME_COLUMN_SQB_LINK_TO_SQB_FROM_LINK, conn);
+      runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_ADD_COLUMN_SQB_TO_LINK, conn);
+      runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_REMOVE_CONSTRAINT_SQB_SQ_LNK, conn);
+      runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_ADD_CONSTRAINT_SQB_SQ_LNK_FROM, conn);
+      runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_ADD_CONSTRAINT_SQB_SQ_LNK_TO, conn);
 
       // Data modifications only for non-fresh install.
       if (version > 0) {
         // Register HDFS connector
-        updateJobData(conn, registerHdfsConnector(conn));
+        updteJobInternals(conn, registerHdfsConnector(conn));
       }
 
       // Wait to remove SQB_TYPE (IMPORT/EXPORT) until we update data.
@@ -440,84 +442,84 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * Upgrade job data from IMPORT/EXPORT to FROM/TO.
    * Since the framework is no longer responsible for HDFS,
    * the HDFS connector/link must be added.
-   * Also, the framework forms are moved around such that
-   * they belong to the added HDFS connector. Any extra forms
+   * Also, the framework configs are moved around such that
+   * they belong to the added HDFS connector. Any extra configs
    * are removed.
-   * NOTE: Connector forms should have a direction (FROM/TO),
-   * but framework forms should not.
+   * NOTE: Connector configs should have a direction (FROM/TO),
+   * but framework configs should not.
    *
    * Here's a brief list describing the data migration process.
-   * 1. Change SQ_FORM.SQF_DIRECTION from IMPORT to FROM.
-   * 2. Change SQ_FORM.SQF_DIRECTION from EXPORT to TO.
-   * 3. Change EXPORT to TO in newly existing SQF_DIRECTION.
-   *    This should affect connectors only since Connector forms
-   *    should have had a value for SQF_OPERATION.
-   * 4. Change IMPORT to FROM in newly existing SQF_DIRECTION.
-   *    This should affect connectors only since Connector forms
-   *    should have had a value for SQF_OPERATION.
+   * 1. Change SQ_CONFIG.SQ_CFG_DIRECTION from IMPORT to FROM.
+   * 2. Change SQ_CONFIG.SQ_CFG_DIRECTION from EXPORT to TO.
+   * 3. Change EXPORT to TO in newly existing SQ_CFG_DIRECTION.
+   *    This should affect connectors only since Connector configs
+   *    should have had a value for SQ_CFG_OPERATION.
+   * 4. Change IMPORT to FROM in newly existing SQ_CFG_DIRECTION.
+   *    This should affect connectors only since Connector configs
+   *    should have had a value for SQ_CFG_OPERATION.
    * 5. Add HDFS connector for jobs to reference.
-   * 6. Set 'input' and 'output' forms connector.
+   * 6. Set 'input' and 'output' configs connector.
    *    to HDFS connector.
-   * 7. Throttling form was originally the second form in
-   *    the framework. It should now be the first form.
-   * 8. Remove the EXPORT throttling form and ensure all of
-   *    its dependencies point to the IMPORT throttling form.
-   *    Then make sure the throttling form does not have a direction.
-   *    Framework forms should not have a direction.
+   * 7. Throttling config was originally the second config in
+   *    the framework. It should now be the first config.
+   * 8. Remove the EXPORT throttling config and ensure all of
+   *    its dependencies point to the IMPORT throttling config.
+   *    Then make sure the throttling config does not have a direction.
+   *    Framework configs should not have a direction.
    * 9. Create an HDFS link to reference and update
    *    jobs to reference that link. IMPORT jobs
    *    should have TO HDFS connector, EXPORT jobs should have
    *    FROM HDFS connector.
-   * 10. Update 'table' form names to 'fromJobConfig' and 'toTable'.
+   * 10. Update 'table' config names to 'fromJobConfig' and 'toTable'.
    *     Also update the relevant inputs as well.
    * @param conn
    */
-  private void updateJobData(Connection conn, long connectorId) {
+  private void updteJobInternals(Connection conn, long connectorId) {
     if (LOG.isTraceEnabled()) {
       LOG.trace("Updating existing data for generic connectors.");
     }
 
-    runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_SQF_OPERATION_TO_SQF_DIRECTION, conn,
+    runQuery(QUERY_UPGRADE_TABLE_SQ_CONFIG_UPDATE_SQ_CFG_OPERATION_TO_SQ_CFG_DIRECTION, conn,
         Direction.FROM.toString(), "IMPORT");
-    runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_SQF_OPERATION_TO_SQF_DIRECTION, conn,
+    runQuery(QUERY_UPGRADE_TABLE_SQ_CONFIG_UPDATE_SQ_CFG_OPERATION_TO_SQ_CFG_DIRECTION, conn,
         Direction.TO.toString(), "EXPORT");
 
-    runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_CONNECTOR_HDFS_FORM_DIRECTION, conn,
+    runQuery(QUERY_UPGRADE_TABLE_SQ_CONFIG_UPDATE_CONNECTOR_HDFS_CONFIG_DIRECTION, conn,
         Direction.FROM.toString(),
         "input");
-    runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_CONNECTOR_HDFS_FORM_DIRECTION, conn,
+    runQuery(QUERY_UPGRADE_TABLE_SQ_CONFIG_UPDATE_CONNECTOR_HDFS_CONFIG_DIRECTION, conn,
         Direction.TO.toString(),
         "output");
 
-    runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_CONNECTOR, conn,
+    runQuery(QUERY_UPGRADE_TABLE_SQ_CONFIG_UPDATE_CONNECTOR, conn,
         new Long(connectorId), "input", "output");
 
-    runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_INPUT_UPDATE_THROTTLING_FORM_INPUTS, conn,
+    runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_INPUT_UPDATE_THROTTLING_CONFIG_INPUTS, conn,
         "IMPORT", "EXPORT");
-    runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_REMOVE_EXTRA_FORM_INPUTS, conn,
+    runQuery(QUERY_UPGRADE_TABLE_SQ_CONFIG_REMOVE_EXTRA_CONFIG_INPUTS, conn,
         "throttling", "EXPORT");
-    runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_REMOVE_EXTRA_FRAMEWORK_FORM, conn,
+    runQuery(QUERY_UPGRADE_TABLE_SQ_CONFIG_REMOVE_EXTRA_DRIVER_CONFIG, conn,
         "throttling", "EXPORT");
-    runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_DIRECTION_TO_NULL, conn,
+    runQuery(QUERY_UPGRADE_TABLE_SQ_CONFIG_UPDATE_DIRECTION_TO_NULL, conn,
         "throttling");
-    runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_FRAMEWORK_INDEX, conn,
+    runQuery(QUERY_UPGRADE_TABLE_SQ_CONFIG_UPDATE_DRIVER_INDEX, conn,
         new Long(0), "throttling");
 
-    MLink hdfsConnection = createHdfsConnection(conn);
-    runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_UPDATE_SQB_TO_CONNECTION_COPY_SQB_FROM_CONNECTION, conn,
+    MLink hdfsLink = createHdfsLink(conn);
+    runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_UPDATE_SQB_TO_LINK_COPY_SQB_FROM_LINK, conn,
         "EXPORT");
-    runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_UPDATE_SQB_FROM_CONNECTION, conn,
-        new Long(hdfsConnection.getPersistenceId()), "EXPORT");
-    runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_UPDATE_SQB_TO_CONNECTION, conn,
-        new Long(hdfsConnection.getPersistenceId()), "IMPORT");
+    runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_UPDATE_SQB_FROM_LINK, conn,
+        new Long(hdfsLink.getPersistenceId()), "EXPORT");
+    runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_UPDATE_SQB_TO_LINK, conn,
+        new Long(hdfsLink.getPersistenceId()), "IMPORT");
 
-    runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_SQF_NAME, conn,
+    runQuery(QUERY_UPGRADE_TABLE_SQ_CONFIG_UPDATE_SQ_CFG_NAME, conn,
         "fromJobConfig", "table", Direction.FROM.toString());
-    runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_TABLE_INPUT_NAMES, conn,
+    runQuery(QUERY_UPGRADE_TABLE_SQ_CONFIG_UPDATE_TABLE_INPUT_NAMES, conn,
         Direction.FROM.toString().toLowerCase(), "fromJobConfig", Direction.FROM.toString());
-    runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_SQF_NAME, conn,
+    runQuery(QUERY_UPGRADE_TABLE_SQ_CONFIG_UPDATE_SQ_CFG_NAME, conn,
         "toJobConfig", "table", Direction.TO.toString());
-    runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_TABLE_INPUT_NAMES, conn,
+    runQuery(QUERY_UPGRADE_TABLE_SQ_CONFIG_UPDATE_TABLE_INPUT_NAMES, conn,
         Direction.TO.toString().toLowerCase(), "toJobConfig", Direction.TO.toString());
 
     if (LOG.isTraceEnabled()) {
@@ -526,7 +528,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
   }
 
   /**
-   * Pre-register HDFS Connector so that metadata upgrade will work.
+   * Pre-register HDFS Connector so that config upgrade will work.
    */
   protected long registerHdfsConnector(Connection conn) {
     if (LOG.isTraceEnabled()) {
@@ -536,7 +538,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
     List<URL> connectorConfigs = ConnectorManagerUtils.getConnectorConfigs();
 
     if (LOG.isInfoEnabled()) {
-      LOG.info("Connector config urls: " + connectorConfigs);
+      LOG.info("Connector configs: " + connectorConfigs);
     }
 
     ConnectorHandler handler = null;
@@ -582,24 +584,22 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    *
    * NOTE: Upgrade path only!
    */
-  private MLink createHdfsConnection(Connection conn) {
+  private MLink createHdfsLink(Connection conn) {
     if (LOG.isTraceEnabled()) {
       LOG.trace("Creating HDFS link.");
     }
 
     MConnector hdfsConnector = this.findConnector(CONNECTOR_HDFS, conn);
-    MDriverConfig driverConfig = findDriverConfig(conn);
-    MLink hdfsConnection = new MLink(
+    MLink hdfsLink = new MLink(
         hdfsConnector.getPersistenceId(),
-        hdfsConnector.getConnectionForms(),
-        driverConfig.getConnectionForms());
-    this.createLink(hdfsConnection, conn);
+        hdfsConnector.getLinkConfig());
+    this.createLink(hdfsLink, conn);
 
     if (LOG.isTraceEnabled()) {
       LOG.trace("Created HDFS link.");
     }
 
-    return hdfsConnection;
+    return hdfsLink;
   }
 
   /**
@@ -675,92 +675,80 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * {@inheritDoc}
    */
   @Override
-  public void registerDriverConfig(MDriverConfig mDriverConfig, Connection conn) {
-    if (mDriverConfig.hasPersistenceId()) {
+  public void registerDriver(MDriver mDriver, Connection conn) {
+    if (mDriver.hasPersistenceId()) {
       throw new SqoopException(DerbyRepoError.DERBYREPO_0011,
-        "Framework metadata");
+        "Driver");
     }
 
-    PreparedStatement baseFormStmt = null;
+    PreparedStatement baseConfigStmt = null;
     PreparedStatement baseInputStmt = null;
     try {
-      baseFormStmt = conn.prepareStatement(STMT_INSERT_FORM_BASE,
+      baseConfigStmt = conn.prepareStatement(STMT_INSERT_CONFIG_BASE,
           Statement.RETURN_GENERATED_KEYS);
       baseInputStmt = conn.prepareStatement(STMT_INSERT_INPUT_BASE,
           Statement.RETURN_GENERATED_KEYS);
 
-      // Register connector forms
-      registerForms(null, null, mDriverConfig.getConnectionForms().getForms(),
-        MFormType.CONNECTION.name(), baseFormStmt, baseInputStmt);
-
-      // Register all jobs
-      registerForms(null, null, mDriverConfig.getJobForms().getForms(),
-        MFormType.JOB.name(), baseFormStmt, baseInputStmt);
+      // Register a driver config as a job type with no owner/connector and direction
+      registerConfigs(null/* owner*/, null /*direction*/, mDriver.getDriverConfig().getConfigs(),
+        MConfigType.JOB.name(), baseConfigStmt, baseInputStmt);
 
-      // We're using hardcoded value for framework metadata as they are
+      // We're using hardcoded value for driver config as they are
       // represented as NULL in the database.
-      mDriverConfig.setPersistenceId(1);
+      mDriver.setPersistenceId(1);
     } catch (SQLException ex) {
-      logException(ex, mDriverConfig);
+      logException(ex, mDriver);
       throw new SqoopException(DerbyRepoError.DERBYREPO_0014, ex);
     } finally {
-      closeStatements(baseFormStmt, baseInputStmt);
+      closeStatements(baseConfigStmt, baseInputStmt);
     }
-    createOrUpdateDriverVersion(conn, mDriverConfig);
+    createOrUpdateDriverSystemVersion(conn, mDriver.getVersion());
   }
 
   /**
    * {@inheritDoc}
    */
   @Override
-  public MDriverConfig findDriverConfig(Connection conn) {
-    LOG.debug("Looking up driver config");
-    MDriverConfig mDriverConfig = null;
-    PreparedStatement formFetchStmt = null;
-    PreparedStatement inputFetchStmt = null;
+  public MDriver findDriver(Connection conn) {
+    LOG.debug("Looking up Driver config to create a driver ");
+    MDriver mDriver = null;
+    PreparedStatement driverConfigFetchStmt = null;
+    PreparedStatement driverConfigInputFetchStmt = null;
     try {
-      formFetchStmt = conn.prepareStatement(STMT_FETCH_FORM_FRAMEWORK);
-      inputFetchStmt = conn.prepareStatement(STMT_FETCH_INPUT);
-
-      List<MForm> connectionForms = new ArrayList<MForm>();
-      List<MForm> jobForms = new ArrayList<MForm>();
+      driverConfigFetchStmt = conn.prepareStatement(STMT_FETCH_CONFIG_DRIVER);
+      driverConfigInputFetchStmt = conn.prepareStatement(STMT_FETCH_INPUT);
+      List<MConfig> driverConfigs = new ArrayList<MConfig>();
+      loadDriverConfigs(driverConfigs, driverConfigFetchStmt, driverConfigInputFetchStmt, 1);
 
-      loadFrameworkForms(connectionForms, jobForms, formFetchStmt, inputFetchStmt, 1);
-
-      // Return nothing If there aren't any framework metadata
-      if(connectionForms.isEmpty() && jobForms.isEmpty()) {
+      if(driverConfigs.isEmpty()) {
         return null;
       }
 
-      mDriverConfig = new MDriverConfig(new MConnectionForms(connectionForms),
-        new MJobForms(jobForms), detectDriverVersion(conn));
-
-      // We're using hardcoded value for driver config as they are
-      // represented as NULL in the database.
-      mDriverConfig.setPersistenceId(1);
+      mDriver = new MDriver(new MDriverConfig(driverConfigs), detectDriverVersion(conn));
+      mDriver.setPersistenceId(1);
 
     } catch (SQLException ex) {
       throw new SqoopException(DerbyRepoError.DERBYREPO_0004,
         "Driver config", ex);
     } finally {
-      if (formFetchStmt != null) {
+      if (driverConfigFetchStmt != null) {
         try {
-          formFetchStmt.close();
+          driverConfigFetchStmt.close();
         } catch (SQLException ex) {
-          LOG.error("Unable to close form fetch statement", ex);
+          LOG.error("Unable to close config fetch statement", ex);
         }
       }
-      if (inputFetchStmt != null) {
+      if (driverConfigInputFetchStmt != null) {
         try {
-          inputFetchStmt.close();
+          driverConfigInputFetchStmt.close();
         } catch (SQLException ex) {
           LOG.error("Unable to close input fetch statement", ex);
         }
       }
     }
 
-    LOG.debug("Looking up driver config found:" + mDriverConfig);
-    return mDriverConfig;
+    LOG.debug("Looking up Driver config and created driver:" + mDriver);
+    return mDriver;
   }
 
   /**
@@ -779,7 +767,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
     PreparedStatement stmt = null;
     int result;
     try {
-      stmt = conn.prepareStatement(STMT_INSERT_CONNECTION,
+      stmt = conn.prepareStatement(STMT_INSERT_LINK,
         Statement.RETURN_GENERATED_KEYS);
       stmt.setString(1, link.getName());
       stmt.setLong(2, link.getConnectorId());
@@ -803,15 +791,10 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
 
       long connectionId = rsetConnectionId.getLong(1);
 
-      createInputValues(STMT_INSERT_CONNECTION_INPUT,
+      createInputValues(STMT_INSERT_LINK_INPUT,
         connectionId,
-        link.getConnectorPart().getForms(),
+        link.getConnectorLinkConfig().getConfigs(),
         conn);
-      createInputValues(STMT_INSERT_CONNECTION_INPUT,
-        connectionId,
-        link.getFrameworkPart().getForms(),
-        conn);
-
       link.setPersistenceId(connectionId);
 
     } catch (SQLException ex) {
@@ -831,12 +814,12 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
     PreparedStatement updateStmt = null;
     try {
       // Firstly remove old values
-      deleteStmt = conn.prepareStatement(STMT_DELETE_CONNECTION_INPUT);
+      deleteStmt = conn.prepareStatement(STMT_DELETE_LINK_INPUT);
       deleteStmt.setLong(1, link.getPersistenceId());
       deleteStmt.executeUpdate();
 
-      // Update CONNECTION table
-      updateStmt = conn.prepareStatement(STMT_UPDATE_CONNECTION);
+      // Update LINK_CONFIG table
+      updateStmt = conn.prepareStatement(STMT_UPDATE_LINK);
       updateStmt.setString(1, link.getName());
       updateStmt.setString(2, link.getLastUpdateUser());
       updateStmt.setTimestamp(3, new Timestamp(new Date().getTime()));
@@ -845,13 +828,9 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
       updateStmt.executeUpdate();
 
       // And reinsert new values
-      createInputValues(STMT_INSERT_CONNECTION_INPUT,
-        link.getPersistenceId(),
-        link.getConnectorPart().getForms(),
-        conn);
-      createInputValues(STMT_INSERT_CONNECTION_INPUT,
+      createInputValues(STMT_INSERT_LINK_INPUT,
         link.getPersistenceId(),
-        link.getFrameworkPart().getForms(),
+        link.getConnectorLinkConfig().getConfigs(),
         conn);
 
     } catch (SQLException ex) {
@@ -870,7 +849,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
     PreparedStatement stmt = null;
     ResultSet rs = null;
     try {
-      stmt = conn.prepareStatement(STMT_SELECT_CONNECTION_CHECK);
+      stmt = conn.prepareStatement(STMT_SELECT_LINK_CHECK);
       stmt.setLong(1, id);
       rs = stmt.executeQuery();
 
@@ -893,7 +872,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
     ResultSet rs = null;
 
     try {
-      stmt = conn.prepareStatement(STMT_SELECT_JOBS_FOR_CONNECTION_CHECK);
+      stmt = conn.prepareStatement(STMT_SELECT_JOBS_FOR_LINK_CHECK);
       stmt.setLong(1, connectionId);
       rs = stmt.executeQuery();
 
@@ -916,7 +895,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
     PreparedStatement enableConn = null;
 
     try {
-      enableConn = conn.prepareStatement(STMT_ENABLE_CONNECTION);
+      enableConn = conn.prepareStatement(STMT_ENABLE_LINK);
       enableConn.setBoolean(1, enabled);
       enableConn.setLong(2, connectionId);
       enableConn.executeUpdate();
@@ -937,7 +916,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
 
     try {
       deleteLinkInputs(id, conn);
-      dltConn = conn.prepareStatement(STMT_DELETE_CONNECTION);
+      dltConn = conn.prepareStatement(STMT_DELETE_LINK);
       dltConn.setLong(1, id);
       dltConn.executeUpdate();
     } catch (SQLException ex) {
@@ -955,7 +934,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
   public void deleteLinkInputs(long id, Connection conn) {
     PreparedStatement dltConnInput = null;
     try {
-      dltConnInput = conn.prepareStatement(STMT_DELETE_CONNECTION_INPUT);
+      dltConnInput = conn.prepareStatement(STMT_DELETE_LINK_INPUT);
       dltConnInput.setLong(1, id);
       dltConnInput.executeUpdate();
     } catch (SQLException ex) {
@@ -973,7 +952,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
   public MLink findLink(long id, Connection conn) {
     PreparedStatement stmt = null;
     try {
-      stmt = conn.prepareStatement(STMT_SELECT_CONNECTION_SINGLE);
+      stmt = conn.prepareStatement(STMT_SELECT_LINK_SINGLE);
       stmt.setLong(1, id);
 
       List<MLink> connections = loadLinks(stmt, conn);
@@ -1001,7 +980,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
   public List<MLink> findLinks(Connection conn) {
     PreparedStatement stmt = null;
     try {
-      stmt = conn.prepareStatement(STMT_SELECT_CONNECTION_ALL);
+      stmt = conn.prepareStatement(STMT_SELECT_LINK_ALL);
 
       return loadLinks(stmt, conn);
 
@@ -1023,7 +1002,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
   public List<MLink> findLinksForConnector(long connectorID, Connection conn) {
     PreparedStatement stmt = null;
     try {
-      stmt = conn.prepareStatement(STMT_SELECT_CONNECTION_FOR_CONNECTOR);
+      stmt = conn.prepareStatement(STMT_SELECT_LINK_FOR_CONNECTOR);
       stmt.setLong(1, connectorID);
 
       return loadLinks(stmt, conn);
@@ -1042,12 +1021,12 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
   @Override
   public void updateConnector(MConnector mConnector, Connection conn) {
     PreparedStatement updateConnectorStatement = null;
-    PreparedStatement deleteForm = null;
+    PreparedStatement deleteConfig = null;
     PreparedStatement deleteInput = null;
     try {
       updateConnectorStatement = conn.prepareStatement(STMT_UPDATE_CONNECTOR);
       deleteInput = conn.prepareStatement(STMT_DELETE_INPUTS_FOR_CONNECTOR);
-      deleteForm = conn.prepareStatement(STMT_DELETE_FORMS_FOR_CONNECTOR);
+      deleteConfig = conn.prepareStatement(STMT_DELETE_CONFIGS_FOR_CONNECTOR);
       updateConnectorStatement.setString(1, mConnector.getUniqueName());
       updateConnectorStatement.setString(2, mConnector.getClassName());
       updateConnectorStatement.setString(3, mConnector.getVersion());
@@ -1057,17 +1036,17 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
         throw new SqoopException(DerbyRepoError.DERBYREPO_0038);
       }
       deleteInput.setLong(1, mConnector.getPersistenceId());
-      deleteForm.setLong(1, mConnector.getPersistenceId());
+      deleteConfig.setLong(1, mConnector.getPersistenceId());
       deleteInput.executeUpdate();
-      deleteForm.executeUpdate();
+      deleteConfig.executeUpdate();
 
     } catch (SQLException e) {
       logException(e, mConnector);
       throw new SqoopException(DerbyRepoError.DERBYREPO_0038, e);
     } finally {
-      closeStatements(updateConnectorStatement, deleteForm, deleteInput);
+      closeStatements(updateConnectorStatement, deleteConfig, deleteInput);
     }
-    insertFormsForConnector(mConnector, conn);
+    insertConfigsForConnector(mConnector, conn);
 
   }
 
@@ -1075,25 +1054,24 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * {@inheritDoc}
    */
   @Override
-  public void updateDriverConfig(MDriverConfig mDriverConfig, Connection conn) {
-    PreparedStatement deleteForm = null;
+  public void updateDriver(MDriver mDriver, Connection conn) {
+    PreparedStatement deleteConfig = null;
     PreparedStatement deleteInput = null;
     try {
-      deleteInput = conn.prepareStatement(STMT_DELETE_FRAMEWORK_INPUTS);
-      deleteForm = conn.prepareStatement(STMT_DELETE_FRAMEWORK_FORMS);
+      deleteInput = conn.prepareStatement(STMT_DELETE_DRIVER_INPUTS);
+      deleteConfig = conn.prepareStatement(STMT_DELETE_DRIVER_CONFIGS);
 
       deleteInput.executeUpdate();
-      deleteForm.executeUpdate();
+      deleteConfig.executeUpdate();
 
     } catch (SQLException e) {
-      logException(e, mDriverConfig);
+      logException(e, mDriver);
       throw new SqoopException(DerbyRepoError.DERBYREPO_0044, e);
     } finally {
-      closeStatements(deleteForm, deleteInput);
+      closeStatements(deleteConfig, deleteInput);
     }
-    createOrUpdateDriverVersion(conn, mDriverConfig);
-    insertFormsForFramework(mDriverConfig, conn);
-
+    createOrUpdateDriverSystemVersion(conn, mDriver.getVersion());
+    insertConfigsForDriver(mDriver, conn);
   }
 
   /**
@@ -1104,8 +1082,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
     PreparedStatement stmt = null;
     int result;
     try {
-      stmt = conn.prepareStatement(STMT_INSERT_JOB,
-        Statement.RETURN_GENERATED_KEYS);
+      stmt = conn.prepareStatement(STMT_INSERT_JOB, Statement.RETURN_GENERATED_KEYS);
       stmt.setString(1, job.getName());
       stmt.setLong(2, job.getLinkId(Direction.FROM));
       stmt.setLong(3, job.getLinkId(Direction.TO));
@@ -1129,17 +1106,20 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
 
       long jobId = rsetJobId.getLong(1);
 
+      // from config for the job
       createInputValues(STMT_INSERT_JOB_INPUT,
                         jobId,
-                        job.getConnectorPart(Direction.FROM).getForms(),
+                        job.getJobConfig(Direction.FROM).getConfigs(),
                         conn);
+      // to config for the job
       createInputValues(STMT_INSERT_JOB_INPUT,
                         jobId,
-                        job.getConnectorPart(Direction.TO).getForms(),
+                        job.getJobConfig(Direction.TO).getConfigs(),
                         conn);
+      // driver config per job
       createInputValues(STMT_INSERT_JOB_INPUT,
                         jobId,
-                        job.getFrameworkPart().getForms(),
+                        job.getDriverConfig().getConfigs(),
                         conn);
 
       job.setPersistenceId(jobId);
@@ -1177,15 +1157,15 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
       // And reinsert new values
       createInputValues(STMT_INSERT_JOB_INPUT,
                         job.getPersistenceId(),
-                        job.getConnectorPart(Direction.FROM).getForms(),
+                        job.getJobConfig(Direction.FROM).getConfigs(),
                         conn);
       createInputValues(STMT_INSERT_JOB_INPUT,
                         job.getPersistenceId(),
-                        job.getConnectorPart(Direction.TO).getForms(),
+                        job.getJobConfig(Direction.TO).getConfigs(),
                         conn);
       createInputValues(STMT_INSERT_JOB_INPUT,
                         job.getPersistenceId(),
-                        job.getFrameworkPart().getForms(),
+                        job.getDriverConfig().getConfigs(),
                         conn);
 
     } catch (SQLException ex) {
@@ -1788,13 +1768,13 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
   private List<MConnector> loadConnectors(PreparedStatement stmt,Connection conn) throws SQLException {
     List<MConnector> connectors = new ArrayList<MConnector>();
     ResultSet rsConnectors = null;
-    PreparedStatement formFetchStmt = null;
-    PreparedStatement inputFetchStmt = null;
+    PreparedStatement connectorConfigFetchStmt = null;
+    PreparedStatement connectorConfigInputFetchStmt = null;
 
     try {
       rsConnectors = stmt.executeQuery();
-      formFetchStmt = conn.prepareStatement(STMT_FETCH_FORM_CONNECTOR);
-      inputFetchStmt = conn.prepareStatement(STMT_FETCH_INPUT);
+      connectorConfigFetchStmt = conn.prepareStatement(STMT_FETCH_CONFIG_CONNECTOR);
+      connectorConfigInputFetchStmt = conn.prepareStatement(STMT_FETCH_INPUT);
 
       while(rsConnectors.next()) {
         long connectorId = rsConnectors.getLong(1);
@@ -1802,26 +1782,26 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
         String connectorClassName = rsConnectors.getString(3);
         String connectorVersion = rsConnectors.getString(4);
 
-        formFetchStmt.setLong(1, connectorId);
+        connectorConfigFetchStmt.setLong(1, connectorId);
 
-        List<MForm> connectionForms = new ArrayList<MForm>();
-        List<MForm> fromJobForms = new ArrayList<MForm>();
-        List<MForm> toJobForms = new ArrayList<MForm>();
+        List<MConfig> linkConfig = new ArrayList<MConfig>();
+        List<MConfig> fromConfig = new ArrayList<MConfig>();
+        List<MConfig> toConfig = new ArrayList<MConfig>();
 
-        loadConnectorForms(connectionForms, fromJobForms, toJobForms,
-            formFetchStmt, inputFetchStmt, 1);
+        loadConfigTypes(linkConfig, fromConfig, toConfig,
+            connectorConfigFetchStmt, connectorConfigInputFetchStmt, 1);
 
         MConnector mc = new MConnector(connectorName, connectorClassName, connectorVersion,
-                                       new MConnectionForms(connectionForms),
-                                       new MJobForms(fromJobForms),
-                                       new MJobForms(toJobForms));
+                                       new MLinkConfig(linkConfig),
+                                       new MFromConfig(fromConfig),
+                                       new MToConfig(toConfig));
         mc.setPersistenceId(connectorId);
 
         connectors.add(mc);
       }
     } finally {
       closeResultSets(rsConnectors);
-      closeStatements(formFetchStmt,inputFetchStmt);
+      closeStatements(connectorConfigFetchStmt,connectorConfigInputFetchStmt);
     }
 
     return connectors;
@@ -1832,16 +1812,15 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
                                             throws SQLException {
     List<MLink> links = new ArrayList<MLink>();
     ResultSet rsConnection = null;
-    PreparedStatement formConnectorFetchStmt = null;
-    PreparedStatement formFrameworkFetchStmt = null;
-    PreparedStatement inputFetchStmt = null;
+    PreparedStatement connectorConfigFetchStatement = null;
+    PreparedStatement connectorConfigInputStatement = null;
 
     try {
       rsConnection = stmt.executeQuery();
 
-      formConnectorFetchStmt = conn.prepareStatement(STMT_FETCH_FORM_CONNECTOR);
-      formFrameworkFetchStmt = conn.prepareStatement(STMT_FETCH_FORM_FRAMEWORK);
-      inputFetchStmt = conn.prepareStatement(STMT_FETCH_CONNECTION_INPUT);
+      //
+      connectorConfigFetchStatement = conn.prepareStatement(STMT_FETCH_CONFIG_CONNECTOR);
+      connectorConfigInputStatement = conn.prepareStatement(STMT_FETCH_LINK_INPUT);
 
       while(rsConnection.next()) {
         long id = rsConnection.getLong(1);
@@ -1853,26 +1832,17 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
         String updateUser = rsConnection.getString(7);
         Date lastUpdateDate = rsConnection.getTimestamp(8);
 
-        formConnectorFetchStmt.setLong(1, connectorId);
-
-        inputFetchStmt.setLong(1, id);
-        //inputFetchStmt.setLong(2, XXX); // Will be filled by loadFrameworkForms
-        inputFetchStmt.setLong(3, id);
-
-        List<MForm> connectorConnForms = new ArrayList<MForm>();
-        List<MForm> frameworkConnForms = new ArrayList<MForm>();
-        List<MForm> frameworkJobForms = new ArrayList<MForm>();
-        List<MForm> fromJobForms = new ArrayList<MForm>();
-        List<MForm> toJobForms = new ArrayList<MForm>();
+        connectorConfigFetchStatement.setLong(1, connectorId);
+        connectorConfigInputStatement.setLong(1, id);
+        connectorConfigInputStatement.setLong(3, id);
 
-        loadConnectorForms(connectorConnForms, fromJobForms, toJobForms,
-            formConnectorFetchStmt, inputFetchStmt, 2);
-        loadFrameworkForms(frameworkConnForms, frameworkJobForms,
-            formFrameworkFetchStmt, inputFetchStmt, 2);
+        List<MConfig> connectorLinkConfig = new ArrayList<MConfig>();
+        List<MConfig> fromConfig = new ArrayList<MConfig>();
+        List<MConfig> toConfig = new ArrayList<MConfig>();
 
-        MLink link = new MLink(connectorId,
-          new MConnectionForms(connectorConnForms),
-          new MConnectionForms(frameworkConnForms));
+        loadConfigTypes(connectorLinkConfig, fromConfig, toConfig, connectorConfigFetchStatement,
+            connectorConfigInputStatement, 2);
+        MLink link = new MLink(connectorId, new MLinkConfig(connectorLinkConfig));
 
         link.setPersistenceId(id);
         link.setName(name);
@@ -1886,8 +1856,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
       }
     } finally {
       closeResultSets(rsConnection);
-      closeStatements(formConnectorFetchStmt,
-        formFrameworkFetchStmt, inputFetchStmt);
+      closeStatements(connectorConfigFetchStatement, connectorConfigInputStatement);
     }
 
     return links;
@@ -1898,20 +1867,21 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
                               throws SQLException {
     List<MJob> jobs = new ArrayList<MJob>();
     ResultSet rsJob = null;
-    PreparedStatement toFormConnectorFetchStmt = null;
-    PreparedStatement fromFormConnectorFetchStmt = null;
-    PreparedStatement formFrameworkFetchStmt = null;
-    PreparedStatement inputFetchStmt = null;
+    PreparedStatement fromConfigFetchStmt = null;
+    PreparedStatement toConfigFetchStmt = null;
+    PreparedStatement driverConfigfetchStmt = null;
+    PreparedStatement jobInputFetchStmt = null;
 
     try {
       rsJob = stmt.executeQuery();
 
-      toFormConnectorFetchStmt = conn.prepareStatement(STMT_FETCH_FORM_CONNECTOR);
-      fromFormConnectorFetchStmt  = conn.prepareStatement(STMT_FETCH_FORM_CONNECTOR);
-      formFrameworkFetchStmt = conn.prepareStatement(STMT_FETCH_FORM_FRAMEWORK);
-      inputFetchStmt = conn.prepareStatement(STMT_FETCH_JOB_INPUT);
+      fromConfigFetchStmt  = conn.prepareStatement(STMT_FETCH_CONFIG_CONNECTOR);
+      toConfigFetchStmt = conn.prepareStatement(STMT_FETCH_CONFIG_CONNECTOR);
+      driverConfigfetchStmt = conn.prepareStatement(STMT_FETCH_CONFIG_DRIVER);
+      jobInputFetchStmt = conn.prepareStatement(STMT_FETCH_JOB_INPUT);
 
       while(rsJob.next()) {
+        // why use connector? why cant it be link id?
         long fromConnectorId = rsJob.getLong(1);
         long toConnectorId = rsJob.getLong(2);
         long id = rsJob.getLong(3);
@@ -1924,48 +1894,40 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
         String updateBy = rsJob.getString(10);
         Date lastUpdateDate = rsJob.getTimestamp(11);
 
-        fromFormConnectorFetchStmt.setLong(1, fromConnectorId);
-        toFormConnectorFetchStmt.setLong(1,toConnectorId);
+        fromConfigFetchStmt.setLong(1, fromConnectorId);
+        toConfigFetchStmt.setLong(1,toConnectorId);
 
-        inputFetchStmt.setLong(1, id);
-        //inputFetchStmt.setLong(1, XXX); // Will be filled by loadFrameworkForms
-        inputFetchStmt.setLong(3, id);
+        jobInputFetchStmt.setLong(1, id);
+        //inputFetchStmt.setLong(1, XXX); // Will be filled by loadFrameworkConfigs
+        jobInputFetchStmt.setLong(3, id);
 
-        List<MForm> toConnectorConnForms = new ArrayList<MForm>();
-        List<MForm> fromConnectorConnForms = new ArrayList<MForm>();
+        // FROM entity configs
+        List<MConfig> fromConnectorLinkConfig = new ArrayList<MConfig>();
+        List<MConfig> fromConnectorFromJobConfig = new ArrayList<MConfig>();
+        List<MConfig> fromConnectorToJobConfig = new ArrayList<MConfig>();
 
-        List<MForm> frameworkConnForms = new ArrayList<MForm>();
-        List<MForm> frameworkJobForms = new ArrayList<MForm>();
+        loadConfigTypes(fromConnectorLinkConfig, fromConnectorFromJobConfig, fromConnectorToJobConfig,
+            fromConfigFetchStmt, jobInputFetchStmt, 2);
 
-        // This looks confusing but our job has 2 connectors, each connector has two job forms
-        // To define the job, we need to TO job form of the TO connector
-        // and the FROM job form of the FROM connector
-        List<MForm> fromConnectorFromJobForms = new ArrayList<MForm>();
-        List<MForm> fromConnectorToJobForms = new ArrayList<MForm>();
-        List<MForm> toConnectorFromJobForms = new ArrayList<MForm>();
-        List<MForm> toConnectorToJobForms = new ArrayList<MForm>();
+        // TO entity configs
+        List<MConfig> toConnectorLinkConfig = new ArrayList<MConfig>();
+        List<MConfig> toConnectorFromJobConfig = new ArrayList<MConfig>();
+        List<MConfig> toConnectorToJobConfig = new ArrayList<MConfig>();
 
+        // ?? dont we need 2 different driver configs for the from/to?
+        List<MConfig> driverConfig = new ArrayList<MConfig>();
 
-        loadConnectorForms(fromConnectorConnForms,
-                fromConnectorFromJobForms,
-                fromConnectorToJobForms,
-                fromFormConnectorFetchStmt,
-                inputFetchStmt,
-                2);
-        loadConnectorForms(toConnectorConnForms,
-                toConnectorFromJobForms,
-                toConnectorToJobForms,
-                toFormConnectorFetchStmt, inputFetchStmt, 2);
+        loadConfigTypes(toConnectorLinkConfig, toConnectorFromJobConfig, toConnectorToJobConfig,
+            toConfigFetchStmt, jobInputFetchStmt, 2);
 
-        loadFrameworkForms(frameworkConnForms, frameworkJobForms,
-            formFrameworkFetchStmt, inputFetchStmt, 2);
+        loadDriverConfigs(driverConfig, driverConfigfetchStmt, jobInputFetchStmt, 2);
 
         MJob job = new MJob(
           fromConnectorId, toConnectorId,
           fromLinkId, toLinkId,
-          new MJobForms(fromConnectorFromJobForms),
-          new MJobForms(toConnectorToJobForms),
-          new MJobForms(frameworkJobForms));
+          new MFromConfig(fromConnectorFromJobConfig),
+          new MToConfig(toConnectorToJobConfig),
+          new MDriverConfig(driverConfig));
 
         job.setPersistenceId(id);
         job.setName(name);
@@ -1979,65 +1941,65 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
       }
     } finally {
       closeResultSets(rsJob);
-      closeStatements(fromFormConnectorFetchStmt, toFormConnectorFetchStmt, formFrameworkFetchStmt, inputFetchStmt);
+      closeStatements(fromConfigFetchStmt, toConfigFetchStmt, driverConfigfetchStmt, jobInputFetchStmt);
     }
 
     return jobs;
   }
 
   /**
-   * Register forms in derby database. This method will insert the ids
-   * generated by the repository into the forms passed in itself.
+   * Register configs in derby database. This method will insert the ids
+   * generated by the repository into the configs passed in itself.
    *
-   * Use given prepared statements to create entire form structure in database.
+   * Use given prepared statements to create entire config structure in database.
    *
    * @param connectorId
-   * @param forms
+   * @param configs
    * @param type
-   * @param baseFormStmt
+   * @param baseConfigStmt
    * @param baseInputStmt
-   * @return short number of forms registered.
+   * @return short number of configs registered.
    * @throws SQLException
    */
-  private short registerForms(Long connectorId, Direction direction,
-      List<MForm> forms, String type, PreparedStatement baseFormStmt,
+  private short registerConfigs(Long connectorId, Direction direction,
+      List<MConfig> configs, String type, PreparedStatement baseConfigStmt,
       PreparedStatement baseInputStmt)
           throws SQLException {
-    short formIndex = 0;
+    short configIndex = 0;
 
-    for (MForm form : forms) {
+    for (MConfig config : configs) {
       if(connectorId == null) {
-        baseFormStmt.setNull(1, Types.BIGINT);
+        baseConfigStmt.setNull(1, Types.BIGINT);
       } else {
-        baseFormStmt.setLong(1, connectorId);
+        baseConfigStmt.setLong(1, connectorId);
       }
       if(direction == null) {
-        baseFormStmt.setNull(2, Types.VARCHAR);
+        baseConfigStmt.setNull(2, Types.VARCHAR);
       } else {
-        baseFormStmt.setString(2, direction.name());
+        baseConfigStmt.setString(2, direction.name());
       }
-      baseFormStmt.setString(3, form.getName());
-      baseFormStmt.setString(4, type);
-      baseFormStmt.setShort(5, formIndex++);
+      baseConfigStmt.setString(3, config.getName());
+      baseConfigStmt.setString(4, type);
+      baseConfigStmt.setShort(5, configIndex++);
 
-      int baseFormCount = baseFormStmt.executeUpdate();
-      if (baseFormCount != 1) {
+      int baseConfigCount = baseConfigStmt.executeUpdate();
+      if (baseConfigCount != 1) {
         throw new SqoopException(DerbyRepoError.DERBYREPO_0015,
-          Integer.toString(baseFormCount));
+          Integer.toString(baseConfigCount));
       }
-      ResultSet rsetFormId = baseFormStmt.getGeneratedKeys();
-      if (!rsetFormId.next()) {
+      ResultSet rsetConfigId = baseConfigStmt.getGeneratedKeys();
+      if (!rsetConfigId.next()) {
         throw new SqoopException(DerbyRepoError.DERBYREPO_0016);
       }
 
-      long formId = rsetFormId.getLong(1);
-      form.setPersistenceId(formId);
+      long configId = rsetConfigId.getLong(1);
+      config.setPersistenceId(configId);
 
       // Insert all the inputs
-      List<MInput<?>> inputs = form.getInputs();
-      registerFormInputs(formId, inputs, baseInputStmt);
+      List<MInput<?>> inputs = config.getInputs();
+      registerConfigInputs(configId, inputs, baseInputStmt);
     }
-    return formIndex;
+    return configIndex;
   }
 
   /**
@@ -2045,17 +2007,17 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    *
    * Use given prepare statement to save all inputs into repository.
    *
-   * @param formId Identifier for corresponding form
+   * @param configId Identifier for corresponding config
    * @param inputs List of inputs that needs to be saved
    * @param baseInputStmt Statement that we can utilize
    * @throws SQLException In case of any failure on Derby side
    */
-  private void registerFormInputs(long formId, List<MInput<?>> inputs,
+  private void registerConfigInputs(long configId, List<MInput<?>> inputs,
       PreparedStatement baseInputStmt) throws SQLException {
     short inputIndex = 0;
     for (MInput<?> input : inputs) {
       baseInputStmt.setString(1, input.getName());
-      baseInputStmt.setLong(2, formId);
+      baseInputStmt.setLong(2, configId);
       baseInputStmt.setShort(3, inputIndex++);
       baseInputStmt.setString(4, input.getType().name());
       baseInputStmt.setBoolean(5, input.isSensitive());
@@ -2128,43 +2090,42 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
   }
 
   /**
-   * Load forms and corresponding inputs from Derby database.
+   * Load configs and corresponding inputs from Derby database.
    *
-   * Use given prepared statements to load all forms and corresponding inputs
+   * Use given prepared statements to load all configs and corresponding inputs
    * from Derby.
    *
-   * @param connectionForms List of link forms that will be filled up
-   * @param jobForms Map with job forms that will be filled up
-   * @param formFetchStmt Prepared statement for fetching forms
+   * @param driverConfig List of driver configs that will be filled up
+   * @param configFetchStatement Prepared statement for fetching configs
    * @param inputFetchStmt Prepare statement for fetching inputs
+   * @param configPosition position of the config
    * @throws SQLException In case of any failure on Derby side
    */
-  public void loadFrameworkForms(List<MForm> connectionForms,
-                                 List<MForm> jobForms,
-                                 PreparedStatement formFetchStmt,
+  public void loadDriverConfigs(List<MConfig> driverConfig,
+                                 PreparedStatement configFetchStatement,
                                  PreparedStatement inputFetchStmt,
-                                 int formPosition) throws SQLException {
+                                 int configPosition) throws SQLException {
 
     // Get list of structures from database
-    ResultSet rsetForm = formFetchStmt.executeQuery();
-    while (rsetForm.next()) {
-      long formId = rsetForm.getLong(1);
-      Long formConnectorId = rsetForm.getLong(2);
-      String formName = rsetForm.getString(4);
-      String formType = rsetForm.getString(5);
-      int formIndex = rsetForm.getInt(6);
-      List<MInput<?>> formInputs = new ArrayList<MInput<?>>();
+    ResultSet rsetConfig = configFetchStatement.executeQuery();
+    while (rsetConfig.next()) {
+      long configId = rsetConfig.getLong(1);
+      Long fromConnectorId = rsetConfig.getLong(2);
+      String configName = rsetConfig.getString(4);
+      String configTYpe = rsetConfig.getString(5);
+      int configIndex = rsetConfig.getInt(6);
+      List<MInput<?>> configInputs = new ArrayList<MInput<?>>();
 
-      MForm mDriverConfig = new MForm(formName, formInputs);
-      mDriverConfig.setPersistenceId(formId);
+      MConfig mDriverConfig = new MConfig(configName, configInputs);
+      mDriverConfig.setPersistenceId(configId);
 
-      inputFetchStmt.setLong(formPosition, formId);
+      inputFetchStmt.setLong(configPosition, configId);
 
       ResultSet rsetInput = inputFetchStmt.executeQuery();
       while (rsetInput.next()) {
         long inputId = rsetInput.getLong(1);
         String inputName = rsetInput.getString(2);
-        long inputForm = rsetInput.getLong(3);
+        long inputConfig = rsetInput.getLong(3);
         short inputIndex = rsetInput.getShort(4);
         String inputType = rsetInput.getString(5);
         boolean inputSensitivity = rsetInput.getBoolean(6);
@@ -2194,7 +2155,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
         default:
           throw new SqoopException(DerbyRepoError.DERBYREPO_0006,
               "input-" + inputName + ":" + inputId + ":"
-              + "form-" + inputForm + ":" + mit.name());
+              + "config-" + inputConfig + ":" + mit.name());
         }
 
         // Set persistent ID
@@ -2209,7 +2170,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
 
         if (mDriverConfig.getInputs().size() != inputIndex) {
           throw new SqoopException(DerbyRepoError.DERBYREPO_0009,
-            "form: " + mDriverConfig
+            "config: " + mDriverConfig
             + "; input: " + input
             + "; index: " + inputIndex
             + "; expected: " + mDriverConfig.getInputs().size()
@@ -2221,83 +2182,69 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
 
       if (mDriverConfig.getInputs().size() == 0) {
         throw new SqoopException(DerbyRepoError.DERBYREPO_0008,
-          "connector-" + formConnectorId
-          + "; form: " + mDriverConfig
+          "owner-" + fromConnectorId
+          + "; config: " + mDriverConfig
         );
       }
 
-      MFormType mDriverConfigt = MFormType.valueOf(formType);
-      switch (mDriverConfigt) {
-      case CONNECTION:
-        if (connectionForms.size() != formIndex) {
-          throw new SqoopException(DerbyRepoError.DERBYREPO_0010,
-            "connector-" + formConnectorId
-            + "; form: " + mDriverConfig
-            + "; index: " + formIndex
-            + "; expected: " + connectionForms.size()
-          );
-        }
-        connectionForms.add(mDriverConfig);
-        break;
+      MConfigType configType = MConfigType.valueOf(configTYpe);
+      switch (configType) {
       case JOB:
-        if (jobForms.size() != formIndex) {
+        if (driverConfig.size() != configIndex) {
           throw new SqoopException(DerbyRepoError.DERBYREPO_0010,
-            "connector-" + formConnectorId
-            + "; form: " + mDriverConfig
-            + "; index: " + formIndex
-            + "; expected: " + jobForms.size()
+            "owner-" + fromConnectorId
+            + "; config: " + configType
+            + "; index: " + configIndex
+            + "; expected: " + driverConfig.size()
           );
         }
-        jobForms.add(mDriverConfig);
+        driverConfig.add(mDriverConfig);
         break;
       default:
         throw new SqoopException(DerbyRepoError.DERBYREPO_0007,
-            "connector-" + formConnectorId + ":" + mDriverConfig);
+            "connector-" + fromConnectorId + ":" + configType);
       }
     }
   }
 
   /**
-   * Load forms and corresponding inputs from Derby database.
+   * Load configs and corresponding inputs from Derby database.
    *
-   * Use given prepared statements to load all forms and corresponding inputs
+   * Use given prepared statements to load all configs and corresponding inputs
    * from Derby.
    *
-   * @param connectionForms List of link forms that will be filled up
-   * @param fromJobForms FROM job forms that will be filled up
-   * @param toJobForms TO job forms that will be filled up
-   * @param formFetchStmt Prepared statement for fetching forms
+   * @param linkConfig List of link configs that will be filled up
+   * @param fromConfig FROM job configs that will be filled up
+   * @param toConfig TO job configs that will be filled up
+   * @param configFetchStmt Prepared statement for fetching configs
    * @param inputFetchStmt Prepare statement for fetching inputs
    * @throws SQLException In case of any failure on Derby side
    */
-  public void loadConnectorForms(List<MForm> connectionForms,
-                                 List<MForm> fromJobForms,
-                                 List<MForm> toJobForms,
-                                 PreparedStatement formFetchStmt,
-                                 PreparedStatement inputFetchStmt,
-                                 int formPosition) throws SQLException {
+  public void loadConfigTypes(List<MConfig> linkConfig, List<MConfig> fromConfig,
+      List<MConfig> toConfig, PreparedStatement configFetchStmt, PreparedStatement inputFetchStmt,
+      int configPosition) throws SQLException {
 
     // Get list of structures from database
-    ResultSet rsetForm = formFetchStmt.executeQuery();
-    while (rsetForm.next()) {
-      long formId = rsetForm.getLong(1);
-      Long formConnectorId = rsetForm.getLong(2);
-      String operation = rsetForm.getString(3);
-      String formName = rsetForm.getString(4);
-      String formType = rsetForm.getString(5);
-      int formIndex = rsetForm.getInt(6);
-      List<MInput<?>> formInputs = new ArrayList<MInput<?>>();
+    ResultSet rsetConfig = configFetchStmt.executeQuery();
+    while (rsetConfig.next()) {
+      long configId = rsetConfig.getLong(1);
+      Long configConnectorId = rsetConfig.getLong(2);
+      String operation = rsetConfig.getString(3);
+      String configName = rsetConfig.getString(4);
+      String configType = rsetConfig.getString(5);
+      int configIndex = rsetConfig.getInt(6);
+      List<MInput<?>> configInputs = new ArrayList<MInput<?>>();
 
-      MForm mDriverConfig = new MForm(formName, formInputs);
-      mDriverConfig.setPersistenceId(formId);
+      MConfig config = new MConfig(configName, configInputs);
+      config.setPersistenceId(configId);
 
-      inputFetchStmt.setLong(formPosition, formId);
+      inputFetchStmt.setLong(configPosition, configId);
 
       ResultSet rsetInput = inputFetchStmt.executeQuery();
       while (rsetInput.next()) {
         long inputId = rsetInput.getLong(1);
         String inputName = rsetInput.getString(2);
-        long inputForm = rsetInput.getLong(3);
+        long inputConfig = rsetInput.getLong(3);
         short inputIndex = rsetInput.getShort(4);
         String inputType = rsetInput.getString(5);
         boolean inputSensitivity = rsetInput.getBoolean(6);
@@ -2327,7 +2274,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
           default:
             throw new SqoopException(DerbyRepoError.DERBYREPO_0006,
                 "input-" + inputName + ":" + inputId + ":"
-                    + "form-" + inputForm + ":" + mit.name());
+                    + "config-" + inputConfig + ":" + mit.name());
         }
 
         // Set persistent ID
@@ -2340,75 +2287,75 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
           input.restoreFromUrlSafeValueString(value);
         }
 
-        if (mDriverConfig.getInputs().size() != inputIndex) {
+        if (config.getInputs().size() != inputIndex) {
           throw new SqoopException(DerbyRepoError.DERBYREPO_0009,
-              "form: " + mDriverConfig
+              "config: " + config
                   + "; input: " + input
                   + "; index: " + inputIndex
-                  + "; expected: " + mDriverConfig.getInputs().size()
+                  + "; expected: " + config.getInputs().size()
           );
         }
 
-        mDriverConfig.getInputs().add(input);
+        config.getInputs().add(input);
       }
 
-      if (mDriverConfig.getInputs().size() == 0) {
+      if (config.getInputs().size() == 0) {
         throw new SqoopException(DerbyRepoError.DERBYREPO_0008,
-            "connector-" + formConnectorId
-                + "; form: " + mDriverConfig
+            "connector-" + configConnectorId
+                + "; config: " + config
         );
       }
 
-      MFormType mDriverConfigt = MFormType.valueOf(formType);
-      switch (mDriverConfigt) {
-        case CONNECTION:
-          if (connectionForms.size() != formIndex) {
+      MConfigType mConfigType = MConfigType.valueOf(configType);
+      switch (mConfigType) {
+        case LINK:
+          if (linkConfig.size() != configIndex) {
             throw new SqoopException(DerbyRepoError.DERBYREPO_0010,
-                "connector-" + formConnectorId
-                    + "; form: " + mDriverConfig
-                    + "; index: " + formIndex
-                    + "; expected: " + connectionForms.size()
+                "connector-" + configConnectorId
+                    + "; config: " + config
+                    + "; index: " + configIndex
+                    + "; expected: " + linkConfig.size()
             );
           }
-          connectionForms.add(mDriverConfig);
+          linkConfig.add(config);
           break;
         case JOB:
           Direction type = Direction.valueOf(operation);
-          List<MForm> jobForms;
+          List<MConfig> jobConfigs;
           switch(type) {
             case FROM:
-              jobForms = fromJobForms;
+              jobConfigs = fromConfig;
               break;
 
             case TO:
-              jobForms = toJobForms;
+              jobConfigs = toConfig;
               break;
 
             default:
               throw new SqoopException(DirectionError.DIRECTION_0000, "Direction: " + type);
           }
 
-          if (jobForms.size() != formIndex) {
+          if (jobConfigs.size() != configIndex) {
             throw new SqoopException(DerbyRepoError.DERBYREPO_0010,
-                "connector-" + formConnectorId
-                    + "; form: " + mDriverConfig
-                    + "; index: " + formIndex
-                    + "; expected: " + jobForms.size()
+                "connector-" + configConnectorId
+                    + "; config: " + config
+                    + "; index: " + configIndex
+                    + "; expected: " + jobConfigs.size()
             );
           }
 
-          jobForms.add(mDriverConfig);
+          jobConfigs.add(config);
           break;
         default:
           throw new SqoopException(DerbyRepoError.DERBYREPO_0007,
-              "connector-" + formConnectorId + ":" + mDriverConfig);
+              "connector-" + configConnectorId + ":" + config);
       }
     }
   }
 
   private void createInputValues(String query,
                                  long id,
-                                 List<MForm> forms,
+                                 List<MConfig> configs,
                                  Connection conn) throws SQLException {
     PreparedStatement stmt = null;
     int result;
@@ -2416,8 +2363,8 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
     try {
       stmt = conn.prepareStatement(query);
 
-      for (MForm form : forms) {
-        for (MInput input : form.getInputs()) {
+      for (MConfig config : configs) {
+        for (MInput input : config.getInputs()) {
           // Skip empty values as we're not interested in storing those in db
           if (input.isEmpty()) {
             continue;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/8362c73c/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 58eed2d..fc3ec18 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
@@ -56,30 +56,30 @@ public final class DerbySchemaConstants {
 
   public static final String COLUMN_SQC_VERSION = "SQC_VERSION";
 
-  // SQ_FORM
+  // SQ_CONFIG
 
-  public static final String TABLE_SQ_FORM_NAME = "SQ_FORM";
+  public static final String TABLE_SQ_CONFIG_NAME = "SQ_CONFIG";
 
-  public static final String TABLE_SQ_FORM = SCHEMA_PREFIX
-    + TABLE_SQ_FORM_NAME;
+  public static final String TABLE_SQ_CONFIG = SCHEMA_PREFIX
+    + TABLE_SQ_CONFIG_NAME;
 
-  public static final String COLUMN_SQF_ID = "SQF_ID";
+  public static final String COLUMN_SQ_CFG_ID = "SQ_CFG_ID";
 
-  public static final String COLUMN_SQF_CONNECTOR = "SQF_CONNECTOR";
+  public static final String COLUMN_SQ_CFG_OWNER = "SQ_CFG_OWNER";
 
-  public static final String COLUMN_SQF_OPERATION = "SQF_OPERATION";
+  public static final String COLUMN_SQ_CFG_OPERATION = "SQ_CFG_OPERATION";
 
-  public static final String COLUMN_SQF_DIRECTION = "SQF_DIRECTION";
+  public static final String COLUMN_SQ_CFG_DIRECTION = "SQ_CFG_DIRECTION";
 
-  public static final String COLUMN_SQF_NAME = "SQF_NAME";
+  public static final String COLUMN_SQ_CFG_NAME = "SQ_CFG_NAME";
 
-  public static final String COLUMN_SQF_TYPE = "SQF_TYPE";
+  public static final String COLUMN_SQ_CFG_TYPE = "SQ_CFG_TYPE";
 
-  public static final String COLUMN_SQF_INDEX = "SQF_INDEX";
+  public static final String COLUMN_SQ_CFG_INDEX = "SQ_CFG_INDEX";
 
-  public static final String CONSTRAINT_SQF_SQC_NAME = CONSTRAINT_PREFIX + "SQF_SQC";
+  public static final String CONSTRAINT_SQ_CFG_SQC_NAME = CONSTRAINT_PREFIX + "SQ_CFG_SQC";
 
-  public static final String CONSTRAINT_SQF_SQC = SCHEMA_PREFIX + CONSTRAINT_SQF_SQC_NAME;
+  public static final String CONSTRAINT_SQ_CFG_SQC = SCHEMA_PREFIX + CONSTRAINT_SQ_CFG_SQC_NAME;
 
   // SQ_INPUT
 
@@ -92,7 +92,7 @@ public final class DerbySchemaConstants {
 
   public static final String COLUMN_SQI_NAME = "SQI_NAME";
 
-  public static final String COLUMN_SQI_FORM = "SQI_FORM";
+  public static final String COLUMN_SQI_CONFIG = "SQI_CONFIG";
 
   public static final String COLUMN_SQI_INDEX = "SQI_INDEX";
 
@@ -104,36 +104,34 @@ public final class DerbySchemaConstants {
 
   public static final String COLUMN_SQI_ENUMVALS = "SQI_ENUMVALS";
 
-  public static final String CONSTRAINT_SQI_SQF_NAME = CONSTRAINT_PREFIX + "SQI_SQF";
+  public static final String CONSTRAINT_SQI_SQ_CFG_NAME = CONSTRAINT_PREFIX + "SQI_SQ_CFG";
 
-  public static final String CONSTRAINT_SQI_SQF = SCHEMA_PREFIX + CONSTRAINT_SQI_SQF_NAME;
+  public static final String CONSTRAINT_SQI_SQ_CFG = SCHEMA_PREFIX + CONSTRAINT_SQI_SQ_CFG_NAME;
 
-  // SQ_CONNECTION
+  public static final String TABLE_SQ_LINK_NAME = "SQ_LINK";
 
-  public static final String TABLE_SQ_CONNECTION_NAME = "SQ_CONNECTION";
+  public static final String TABLE_SQ_LINK = SCHEMA_PREFIX
+      + TABLE_SQ_LINK_NAME;
 
-  public static final String TABLE_SQ_CONNECTION = SCHEMA_PREFIX
-    + TABLE_SQ_CONNECTION_NAME;
+  public static final String COLUMN_SQ_LNK_ID = "SQ_LNK_ID";
 
-  public static final String COLUMN_SQN_ID = "SQN_ID";
+  public static final String COLUMN_SQ_LNK_NAME = "SQ_LNK_NAME";
 
-  public static final String COLUMN_SQN_NAME = "SQN_NAME";
+  public static final String COLUMN_SQ_LNK_CONNECTOR = "SQ_LNK_CONNECTOR";
 
-  public static final String COLUMN_SQN_CONNECTOR = "SQN_CONNECTOR";
+  public static final String COLUMN_SQ_LNK_CREATION_USER = "SQ_LNK_CREATION_USER";
 
-  public static final String COLUMN_SQN_CREATION_USER = "SQN_CREATION_USER";
+  public static final String COLUMN_SQ_LNK_CREATION_DATE = "SQ_LNK_CREATION_DATE";
 
-  public static final String COLUMN_SQN_CREATION_DATE = "SQN_CREATION_DATE";
+  public static final String COLUMN_SQ_LNK_UPDATE_USER = "SQ_LNK_UPDATE_USER";
 
-  public static final String COLUMN_SQN_UPDATE_USER = "SQN_UPDATE_USER";
+  public static final String COLUMN_SQ_LNK_UPDATE_DATE = "SQ_LNK_UPDATE_DATE";
 
-  public static final String COLUMN_SQN_UPDATE_DATE = "SQN_UPDATE_DATE";
+  public static final String COLUMN_SQ_LNK_ENABLED = "SQ_LNK_ENABLED";
 
-  public static final String COLUMN_SQN_ENABLED = "SQN_ENABLED";
+  public static final String CONSTRAINT_SQ_LNK_SQC_NAME = CONSTRAINT_PREFIX + "SQ_LNK_SQC";
 
-  public static final String CONSTRAINT_SQN_SQC_NAME = CONSTRAINT_PREFIX + "SQN_SQC";
-
-  public static final String CONSTRAINT_SQN_SQC = SCHEMA_PREFIX + CONSTRAINT_SQN_SQC_NAME;
+  public static final String CONSTRAINT_SQ_LNK_SQC = SCHEMA_PREFIX + CONSTRAINT_SQ_LNK_SQC_NAME;
 
   // SQ_JOB
 
@@ -146,13 +144,13 @@ public final class DerbySchemaConstants {
 
   public static final String COLUMN_SQB_NAME = "SQB_NAME";
 
-  public static final String COLUMN_SQB_CONNECTION = "SQB_CONNECTION";
+  public static final String COLUMN_SQB_LINK = "SQB_LINK";
 
   public static final String COLUMN_SQB_TYPE = "SQB_TYPE";
 
-  public static final String COLUMN_SQB_FROM_CONNECTION = "SQB_FROM_CONNECTION";
+  public static final String COLUMN_SQB_FROM_LINK = "SQB_FROM_LINK";
 
-  public static final String COLUMN_SQB_TO_CONNECTION = "SQB_TO_CONNECTION";
+  public static final String COLUMN_SQB_TO_LINK = "SQB_TO_LINK";
 
   public static final String COLUMN_SQB_CREATION_USER = "SQB_CREATION_USER";
 
@@ -164,39 +162,37 @@ public final class DerbySchemaConstants {
 
   public static final String COLUMN_SQB_ENABLED = "SQB_ENABLED";
 
-  public static final String CONSTRAINT_SQB_SQN_NAME = CONSTRAINT_PREFIX + "SQB_SQN";
-
-  public static final String CONSTRAINT_SQB_SQN = SCHEMA_PREFIX + CONSTRAINT_SQB_SQN_NAME;
+  public static final String CONSTRAINT_SQB_SQ_LNK_NAME = CONSTRAINT_PREFIX + "SQB_SQ_LNK";
 
-  public static final String CONSTRAINT_SQB_SQN_FROM_NAME = CONSTRAINT_PREFIX + "SQB_SQN_FROM";
+  public static final String CONSTRAINT_SQB_SQ_LNK = SCHEMA_PREFIX + CONSTRAINT_SQB_SQ_LNK_NAME;
 
-  public static final String CONSTRAINT_SQB_SQN_FROM = SCHEMA_PREFIX + CONSTRAINT_SQB_SQN_FROM_NAME;
+  public static final String CONSTRAINT_SQB_SQ_LNK_FROM_NAME = CONSTRAINT_PREFIX + "SQB_SQ_LNK_FROM";
 
-  public static final String CONSTRAINT_SQB_SQN_TO_NAME = CONSTRAINT_PREFIX + "SQB_SQN_TO";
+  public static final String CONSTRAINT_SQB_SQ_LNK_FROM = SCHEMA_PREFIX + CONSTRAINT_SQB_SQ_LNK_FROM_NAME;
 
-  public static final String CONSTRAINT_SQB_SQN_TO = SCHEMA_PREFIX + CONSTRAINT_SQB_SQN_TO_NAME;
+  public static final String CONSTRAINT_SQB_SQ_LNK_TO_NAME = CONSTRAINT_PREFIX + "SQB_SQ_LNK_TO";
 
-  // SQ_CONNECTION_INPUT
+  public static final String CONSTRAINT_SQB_SQ_LNK_TO = SCHEMA_PREFIX + CONSTRAINT_SQB_SQ_LNK_TO_NAME;
 
-  public static final String TABLE_SQ_CONNECTION_INPUT_NAME =
-    "SQ_CONNECTION_INPUT";
+  public static final String TABLE_SQ_LINK_INPUT_NAME =
+    "SQ_LINK_INPUT";
 
-  public static final String TABLE_SQ_CONNECTION_INPUT = SCHEMA_PREFIX
-    + TABLE_SQ_CONNECTION_INPUT_NAME;
+  public static final String TABLE_SQ_LINK_INPUT = SCHEMA_PREFIX
+    + TABLE_SQ_LINK_INPUT_NAME;
 
-  public static final String COLUMN_SQNI_CONNECTION = "SQNI_CONNECTION";
+  public static final String COLUMN_SQ_LNKI_LINK = "SQ_LNKI_LINK";
 
-  public static final String COLUMN_SQNI_INPUT = "SQNI_INPUT";
+  public static final String COLUMN_SQ_LNKI_INPUT = "SQ_LNKI_INPUT";
 
-  public static final String COLUMN_SQNI_VALUE = "SQNI_VALUE";
+  public static final String COLUMN_SQ_LNKI_VALUE = "SQ_LNKI_VALUE";
 
-  public static final String CONSTRAINT_SQNI_SQN_NAME = CONSTRAINT_PREFIX + "SQNI_SQN";
+  public static final String CONSTRAINT_SQ_LNKI_SQ_LNK_NAME = CONSTRAINT_PREFIX + "SQ_LNKI_SQ_LNK";
 
-  public static final String CONSTRAINT_SQNI_SQN = SCHEMA_PREFIX + CONSTRAINT_SQNI_SQN_NAME;
+  public static final String CONSTRAINT_SQ_LNKI_SQ_LNK = SCHEMA_PREFIX + CONSTRAINT_SQ_LNKI_SQ_LNK_NAME;
 
-  public static final String CONSTRAINT_SQNI_SQI_NAME = CONSTRAINT_PREFIX + "SQNI_SQI";
+  public static final String CONSTRAINT_SQ_LNKI_SQI_NAME = CONSTRAINT_PREFIX + "SQ_LNKI_SQI";
 
-  public static final String CONSTRAINT_SQNI_SQI = SCHEMA_PREFIX + CONSTRAINT_SQNI_SQI_NAME;
+  public static final String CONSTRAINT_SQ_LNKI_SQI = SCHEMA_PREFIX + CONSTRAINT_SQ_LNKI_SQI_NAME;
 
   // SQ_JOB_INPUT
 
@@ -314,12 +310,12 @@ public final class DerbySchemaConstants {
   static {
     tablesV1 = new HashSet<String>();
     tablesV1.add(TABLE_SQ_CONNECTOR_NAME);
-    tablesV1.add(TABLE_SQ_CONNECTION_NAME);
-    tablesV1.add(TABLE_SQ_CONNECTION_INPUT_NAME);
+    tablesV1.add(TABLE_SQ_LINK_NAME);
+    tablesV1.add(TABLE_SQ_LINK_INPUT_NAME);
     tablesV1.add(TABLE_SQ_COUNTER_NAME);
     tablesV1.add(TABLE_SQ_COUNTER_GROUP_NAME);
     tablesV1.add(TABLE_SQ_COUNTER_SUBMISSION_NAME);
-    tablesV1.add(TABLE_SQ_FORM_NAME);
+    tablesV1.add(TABLE_SQ_CONFIG_NAME);
     tablesV1.add(TABLE_SQ_INPUT_NAME);
     tablesV1.add(TABLE_SQ_JOB_NAME);
     tablesV1.add(TABLE_SQ_JOB_INPUT_NAME);