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:51:40 UTC

[11/52] [abbrv] SQOOP-1497: Sqoop2: Entity Nomenclature Revisited

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/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 68aea9c..5dd7970 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
@@ -30,8 +30,6 @@ import java.sql.Timestamp;
 import java.sql.Types;
 import java.util.*;
 
-import javax.sql.DataSource;
-
 import org.apache.log4j.Logger;
 import org.apache.commons.lang.StringUtils;
 import org.apache.sqoop.common.Direction;
@@ -40,7 +38,7 @@ 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.MConnection;
+import org.apache.sqoop.model.MLink;
 import org.apache.sqoop.model.MConnectionForms;
 import org.apache.sqoop.model.MEnumInput;
 import org.apache.sqoop.model.MIntegerInput;
@@ -49,7 +47,7 @@ import org.apache.sqoop.model.MJobForms;
 import org.apache.sqoop.model.MConnector;
 import org.apache.sqoop.model.MForm;
 import org.apache.sqoop.model.MFormType;
-import org.apache.sqoop.model.MFramework;
+import org.apache.sqoop.model.MDriverConfig;
 import org.apache.sqoop.model.MInput;
 import org.apache.sqoop.model.MInputType;
 import org.apache.sqoop.model.MMapInput;
@@ -57,7 +55,6 @@ import org.apache.sqoop.model.MStringInput;
 import org.apache.sqoop.model.MSubmission;
 import org.apache.sqoop.repository.JdbcRepositoryContext;
 import org.apache.sqoop.repository.JdbcRepositoryHandler;
-import org.apache.sqoop.repository.JdbcRepositoryTransactionFactory;
 import org.apache.sqoop.submission.SubmissionStatus;
 import org.apache.sqoop.submission.counter.Counter;
 import org.apache.sqoop.submission.counter.CounterGroup;
@@ -78,15 +75,13 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
 
   /**
    * Unique name of HDFS Connector.
-   * HDFS Connector was originally part of the Sqoop framework, but now is its
+   * HDFS Connector was originally part of the Sqoop driver, but now is its
    * own connector. This constant is used to pre-register the HDFS Connector
    * so that jobs that are being upgraded can reference the HDFS Connector.
    */
   private static final String CONNECTOR_HDFS = "hdfs-connector";
 
   private JdbcRepositoryContext repoContext;
-  private DataSource dataSource;
-  private JdbcRepositoryTransactionFactory txFactory;
 
   /**
    * {@inheritDoc}
@@ -105,10 +100,10 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * 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 mf The MFramework instance to use to upgrade.
-   * @param conn JDBC connection to use for updating the forms
+   * @param mDriverConfig The MFramework instance to use to upgrade.
+   * @param conn JDBC link to use for updating the forms
    */
-  private void insertFormsForFramework(MFramework mf, Connection conn) {
+  private void insertFormsForFramework(MDriverConfig mDriverConfig, Connection conn) {
     PreparedStatement baseFormStmt = null;
     PreparedStatement baseInputStmt = null;
     try{
@@ -119,15 +114,15 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
         Statement.RETURN_GENERATED_KEYS);
 
       // Register connector forms
-      registerForms(null, null, mf.getConnectionForms().getForms(),
+      registerForms(null, null, mDriverConfig.getConnectionForms().getForms(),
         MFormType.CONNECTION.name(), baseFormStmt, baseInputStmt);
 
       // Register job forms
-      registerForms(null, null, mf.getJobForms().getForms(),
+      registerForms(null, null, mDriverConfig.getJobForms().getForms(),
         MFormType.JOB.name(), baseFormStmt, baseInputStmt);
 
     } catch (SQLException ex) {
-      throw new SqoopException(DerbyRepoError.DERBYREPO_0014, mf.toString(), ex);
+      throw new SqoopException(DerbyRepoError.DERBYREPO_0014, mDriverConfig.toString(), ex);
     } finally {
       closeStatements(baseFormStmt, baseInputStmt);
     }
@@ -138,7 +133,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * 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 connection to use for updating the forms
+   * @param conn JDBC link to use for updating the forms
    */
   private void insertFormsForConnector (MConnector mc, Connection conn) {
     long connectorId = mc.getPersistenceId();
@@ -205,8 +200,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
   @Override
   public synchronized void initialize(JdbcRepositoryContext ctx) {
     repoContext = ctx;
-    dataSource = repoContext.getDataSource();
-    txFactory = repoContext.getTransactionFactory();
+    repoContext.getDataSource();
     LOG.info("DerbyRepositoryHandler initialized.");
   }
 
@@ -313,24 +307,24 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
   }
 
   /**
-   * Detect version of the framework
+   * Detect version of the driver
    *
-   * @param conn Connection to metadata repository
-   * @return Version of the MFramework
+   * @param conn Connection to the repository
+   * @return Version of the Driver
    */
-  private String detectFrameworkVersion (Connection conn) {
+  private String detectDriverVersion (Connection conn) {
     ResultSet rs = null;
     PreparedStatement stmt = null;
     try {
       stmt = conn.prepareStatement(DerbySchemaQuery.STMT_SELECT_SYSTEM);
-      stmt.setString(1, DerbyRepoConstants.SYSKEY_FRAMEWORK_VERSION);
+      stmt.setString(1, DerbyRepoConstants.SYSKEY_DRIVER_VERSION);
       rs = stmt.executeQuery();
       if(!rs.next()) {
         return null;
       }
       return rs.getString(1);
     } catch (SQLException e) {
-      LOG.info("Can't fetch framework version.", e);
+      LOG.info("Can't fetch driver version.", e);
       return null;
     } finally {
       closeResultSets(rs);
@@ -339,23 +333,22 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
   }
 
   /**
-   * Create or update framework version
-   * @param conn Connection to the metadata repository
-   * @param mFramework
+   * Create or update driver version
+   * @param conn Connection to the the repository
+   * @param mDriverConfig
    */
-  private void createOrUpdateFrameworkVersion(Connection conn,
-      MFramework mFramework) {
+  private void createOrUpdateDriverVersion(Connection conn, MDriverConfig mDriverConfig) {
     ResultSet rs = null;
     PreparedStatement stmt = null;
     try {
       stmt = conn.prepareStatement(STMT_DELETE_SYSTEM);
-      stmt.setString(1, DerbyRepoConstants.SYSKEY_FRAMEWORK_VERSION);
+      stmt.setString(1, DerbyRepoConstants.SYSKEY_DRIVER_VERSION);
       stmt.executeUpdate();
       closeStatements(stmt);
 
       stmt = conn.prepareStatement(STMT_INSERT_SYSTEM);
-      stmt.setString(1, DerbyRepoConstants.SYSKEY_FRAMEWORK_VERSION);
-      stmt.setString(2, mFramework.getVersion());
+      stmt.setString(1, DerbyRepoConstants.SYSKEY_DRIVER_VERSION);
+      stmt.setString(2, mDriverConfig.getVersion());
       stmt.executeUpdate();
     } catch (SQLException e) {
       logException(e);
@@ -446,7 +439,7 @@ 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/connection must be added.
+   * 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
    * are removed.
@@ -471,11 +464,11 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    *    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.
-   * 9. Create an HDFS connection to reference and update
-   *    jobs to reference that connection. IMPORT jobs
+   * 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 'fromTable' and 'toTable'.
+   * 10. Update 'table' form names to 'fromJobConfig' and 'toTable'.
    *     Also update the relevant inputs as well.
    * @param conn
    */
@@ -510,7 +503,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
     runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_FRAMEWORK_INDEX, conn,
         new Long(0), "throttling");
 
-    MConnection hdfsConnection = createHdfsConnection(conn);
+    MLink hdfsConnection = createHdfsConnection(conn);
     runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_UPDATE_SQB_TO_CONNECTION_COPY_SQB_FROM_CONNECTION, conn,
         "EXPORT");
     runQuery(QUERY_UPGRADE_TABLE_SQ_JOB_UPDATE_SQB_FROM_CONNECTION, conn,
@@ -519,13 +512,13 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
         new Long(hdfsConnection.getPersistenceId()), "IMPORT");
 
     runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_SQF_NAME, conn,
-        "fromTable", "table", Direction.FROM.toString());
+        "fromJobConfig", "table", Direction.FROM.toString());
     runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_TABLE_INPUT_NAMES, conn,
-        Direction.FROM.toString().toLowerCase(), "fromTable", Direction.FROM.toString());
+        Direction.FROM.toString().toLowerCase(), "fromJobConfig", Direction.FROM.toString());
     runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_SQF_NAME, conn,
-        "toTable", "table", Direction.TO.toString());
+        "toJobConfig", "table", Direction.TO.toString());
     runQuery(QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_TABLE_INPUT_NAMES, conn,
-        Direction.TO.toString().toLowerCase(), "toTable", Direction.TO.toString());
+        Direction.TO.toString().toLowerCase(), "toJobConfig", Direction.TO.toString());
 
     if (LOG.isTraceEnabled()) {
       LOG.trace("Updated existing data for generic connectors.");
@@ -583,27 +576,27 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
   }
 
   /**
-   * Create an HDFS connection.
-   * Intended to be used when moving HDFS connector out of framework
+   * Create an HDFS link.
+   * Intended to be used when moving HDFS connector out of driverConfig
    * to its own connector.
    *
    * NOTE: Upgrade path only!
    */
-  private MConnection createHdfsConnection(Connection conn) {
+  private MLink createHdfsConnection(Connection conn) {
     if (LOG.isTraceEnabled()) {
-      LOG.trace("Creating HDFS connection.");
+      LOG.trace("Creating HDFS link.");
     }
 
     MConnector hdfsConnector = this.findConnector(CONNECTOR_HDFS, conn);
-    MFramework framework = findFramework(conn);
-    MConnection hdfsConnection = new MConnection(
+    MDriverConfig driverConfig = findDriverConfig(conn);
+    MLink hdfsConnection = new MLink(
         hdfsConnector.getPersistenceId(),
         hdfsConnector.getConnectionForms(),
-        framework.getConnectionForms());
-    this.createConnection(hdfsConnection, conn);
+        driverConfig.getConnectionForms());
+    this.createLink(hdfsConnection, conn);
 
     if (LOG.isTraceEnabled()) {
-      LOG.trace("Created HDFS connection.");
+      LOG.trace("Created HDFS link.");
     }
 
     return hdfsConnection;
@@ -682,8 +675,8 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * {@inheritDoc}
    */
   @Override
-  public void registerFramework(MFramework mf, Connection conn) {
-    if (mf.hasPersistenceId()) {
+  public void registerDriverConfig(MDriverConfig mDriverConfig, Connection conn) {
+    if (mDriverConfig.hasPersistenceId()) {
       throw new SqoopException(DerbyRepoError.DERBYREPO_0011,
         "Framework metadata");
     }
@@ -697,32 +690,32 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
           Statement.RETURN_GENERATED_KEYS);
 
       // Register connector forms
-      registerForms(null, null, mf.getConnectionForms().getForms(),
+      registerForms(null, null, mDriverConfig.getConnectionForms().getForms(),
         MFormType.CONNECTION.name(), baseFormStmt, baseInputStmt);
 
       // Register all jobs
-      registerForms(null, null, mf.getJobForms().getForms(),
+      registerForms(null, null, mDriverConfig.getJobForms().getForms(),
         MFormType.JOB.name(), baseFormStmt, baseInputStmt);
 
       // We're using hardcoded value for framework metadata as they are
       // represented as NULL in the database.
-      mf.setPersistenceId(1);
+      mDriverConfig.setPersistenceId(1);
     } catch (SQLException ex) {
-      logException(ex, mf);
+      logException(ex, mDriverConfig);
       throw new SqoopException(DerbyRepoError.DERBYREPO_0014, ex);
     } finally {
       closeStatements(baseFormStmt, baseInputStmt);
     }
-    createOrUpdateFrameworkVersion(conn, mf);
+    createOrUpdateDriverVersion(conn, mDriverConfig);
   }
 
   /**
    * {@inheritDoc}
    */
   @Override
-  public MFramework findFramework(Connection conn) {
-    LOG.debug("Looking up framework metadata");
-    MFramework mf = null;
+  public MDriverConfig findDriverConfig(Connection conn) {
+    LOG.debug("Looking up driver config");
+    MDriverConfig mDriverConfig = null;
     PreparedStatement formFetchStmt = null;
     PreparedStatement inputFetchStmt = null;
     try {
@@ -739,16 +732,16 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
         return null;
       }
 
-      mf = new MFramework(new MConnectionForms(connectionForms),
-        new MJobForms(jobForms), detectFrameworkVersion(conn));
+      mDriverConfig = new MDriverConfig(new MConnectionForms(connectionForms),
+        new MJobForms(jobForms), detectDriverVersion(conn));
 
-      // 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.
-      mf.setPersistenceId(1);
+      mDriverConfig.setPersistenceId(1);
 
     } catch (SQLException ex) {
       throw new SqoopException(DerbyRepoError.DERBYREPO_0004,
-        "Framework metadata", ex);
+        "Driver config", ex);
     } finally {
       if (formFetchStmt != null) {
         try {
@@ -766,10 +759,8 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
       }
     }
 
-    LOG.debug("Looking up framework metadta found: " + mf);
-
-    // Returned loaded framework metadata
-    return mf;
+    LOG.debug("Looking up driver config found:" + mDriverConfig);
+    return mDriverConfig;
   }
 
   /**
@@ -784,19 +775,19 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * {@inheritDoc}
    */
   @Override
-  public void createConnection(MConnection connection, Connection conn) {
+  public void createLink(MLink link, Connection conn) {
     PreparedStatement stmt = null;
     int result;
     try {
       stmt = conn.prepareStatement(STMT_INSERT_CONNECTION,
         Statement.RETURN_GENERATED_KEYS);
-      stmt.setString(1, connection.getName());
-      stmt.setLong(2, connection.getConnectorId());
-      stmt.setBoolean(3, connection.getEnabled());
-      stmt.setString(4, connection.getCreationUser());
-      stmt.setTimestamp(5, new Timestamp(connection.getCreationDate().getTime()));
-      stmt.setString(6, connection.getLastUpdateUser());
-      stmt.setTimestamp(7, new Timestamp(connection.getLastUpdateDate().getTime()));
+      stmt.setString(1, link.getName());
+      stmt.setLong(2, link.getConnectorId());
+      stmt.setBoolean(3, link.getEnabled());
+      stmt.setString(4, link.getCreationUser());
+      stmt.setTimestamp(5, new Timestamp(link.getCreationDate().getTime()));
+      stmt.setString(6, link.getLastUpdateUser());
+      stmt.setTimestamp(7, new Timestamp(link.getLastUpdateDate().getTime()));
 
       result = stmt.executeUpdate();
       if (result != 1) {
@@ -814,17 +805,17 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
 
       createInputValues(STMT_INSERT_CONNECTION_INPUT,
         connectionId,
-        connection.getConnectorPart().getForms(),
+        link.getConnectorPart().getForms(),
         conn);
       createInputValues(STMT_INSERT_CONNECTION_INPUT,
         connectionId,
-        connection.getFrameworkPart().getForms(),
+        link.getFrameworkPart().getForms(),
         conn);
 
-      connection.setPersistenceId(connectionId);
+      link.setPersistenceId(connectionId);
 
     } catch (SQLException ex) {
-      logException(ex, connection);
+      logException(ex, link);
       throw new SqoopException(DerbyRepoError.DERBYREPO_0019, ex);
     } finally {
       closeStatements(stmt);
@@ -835,36 +826,36 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * {@inheritDoc}
    */
   @Override
-  public void updateConnection(MConnection connection, Connection conn) {
+  public void updateLink(MLink link, Connection conn) {
     PreparedStatement deleteStmt = null;
     PreparedStatement updateStmt = null;
     try {
       // Firstly remove old values
       deleteStmt = conn.prepareStatement(STMT_DELETE_CONNECTION_INPUT);
-      deleteStmt.setLong(1, connection.getPersistenceId());
+      deleteStmt.setLong(1, link.getPersistenceId());
       deleteStmt.executeUpdate();
 
       // Update CONNECTION table
       updateStmt = conn.prepareStatement(STMT_UPDATE_CONNECTION);
-      updateStmt.setString(1, connection.getName());
-      updateStmt.setString(2, connection.getLastUpdateUser());
+      updateStmt.setString(1, link.getName());
+      updateStmt.setString(2, link.getLastUpdateUser());
       updateStmt.setTimestamp(3, new Timestamp(new Date().getTime()));
 
-      updateStmt.setLong(4, connection.getPersistenceId());
+      updateStmt.setLong(4, link.getPersistenceId());
       updateStmt.executeUpdate();
 
       // And reinsert new values
       createInputValues(STMT_INSERT_CONNECTION_INPUT,
-        connection.getPersistenceId(),
-        connection.getConnectorPart().getForms(),
+        link.getPersistenceId(),
+        link.getConnectorPart().getForms(),
         conn);
       createInputValues(STMT_INSERT_CONNECTION_INPUT,
-        connection.getPersistenceId(),
-        connection.getFrameworkPart().getForms(),
+        link.getPersistenceId(),
+        link.getFrameworkPart().getForms(),
         conn);
 
     } catch (SQLException ex) {
-      logException(ex, connection);
+      logException(ex, link);
       throw new SqoopException(DerbyRepoError.DERBYREPO_0021, ex);
     } finally {
       closeStatements(deleteStmt, updateStmt);
@@ -875,7 +866,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * {@inheritDoc}
    */
   @Override
-  public boolean existsConnection(long id, Connection conn) {
+  public boolean existsLink(long id, Connection conn) {
     PreparedStatement stmt = null;
     ResultSet rs = null;
     try {
@@ -897,7 +888,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
   }
 
   @Override
-  public boolean inUseConnection(long connectionId, Connection conn) {
+  public boolean inUseLink(long connectionId, Connection conn) {
     PreparedStatement stmt = null;
     ResultSet rs = null;
 
@@ -921,7 +912,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
   }
 
   @Override
-  public void enableConnection(long connectionId, boolean enabled, Connection conn) {
+  public void enableLink(long connectionId, boolean enabled, Connection conn) {
     PreparedStatement enableConn = null;
 
     try {
@@ -941,11 +932,11 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * {@inheritDoc}
    */
   @Override
-  public void deleteConnection(long id, Connection conn) {
+  public void deleteLink(long id, Connection conn) {
     PreparedStatement dltConn = null;
 
     try {
-      deleteConnectionInputs(id, conn);
+      deleteLinkInputs(id, conn);
       dltConn = conn.prepareStatement(STMT_DELETE_CONNECTION);
       dltConn.setLong(1, id);
       dltConn.executeUpdate();
@@ -961,7 +952,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * {@inheritDoc}
    */
   @Override
-  public void deleteConnectionInputs(long id, Connection conn) {
+  public void deleteLinkInputs(long id, Connection conn) {
     PreparedStatement dltConnInput = null;
     try {
       dltConnInput = conn.prepareStatement(STMT_DELETE_CONNECTION_INPUT);
@@ -979,20 +970,20 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * {@inheritDoc}
    */
   @Override
-  public MConnection findConnection(long id, Connection conn) {
+  public MLink findLink(long id, Connection conn) {
     PreparedStatement stmt = null;
     try {
       stmt = conn.prepareStatement(STMT_SELECT_CONNECTION_SINGLE);
       stmt.setLong(1, id);
 
-      List<MConnection> connections = loadConnections(stmt, conn);
+      List<MLink> connections = loadLinks(stmt, conn);
 
       if(connections.size() != 1) {
         throw new SqoopException(DerbyRepoError.DERBYREPO_0024, "Couldn't find"
-          + " connection with id " + id);
+          + " link with id " + id);
       }
 
-      // Return the first and only one connection object
+      // Return the first and only one link object
       return connections.get(0);
 
     } catch (SQLException ex) {
@@ -1007,12 +998,12 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * {@inheritDoc}
    */
   @Override
-  public List<MConnection> findConnections(Connection conn) {
+  public List<MLink> findLinks(Connection conn) {
     PreparedStatement stmt = null;
     try {
       stmt = conn.prepareStatement(STMT_SELECT_CONNECTION_ALL);
 
-      return loadConnections(stmt, conn);
+      return loadLinks(stmt, conn);
 
     } catch (SQLException ex) {
       logException(ex);
@@ -1029,13 +1020,13 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    *
    */
   @Override
-  public List<MConnection> findConnectionsForConnector(long connectorID, Connection conn) {
+  public List<MLink> findLinksForConnector(long connectorID, Connection conn) {
     PreparedStatement stmt = null;
     try {
       stmt = conn.prepareStatement(STMT_SELECT_CONNECTION_FOR_CONNECTOR);
       stmt.setLong(1, connectorID);
 
-      return loadConnections(stmt, conn);
+      return loadLinks(stmt, conn);
 
     } catch (SQLException ex) {
       logException(ex, connectorID);
@@ -1084,7 +1075,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * {@inheritDoc}
    */
   @Override
-  public void updateFramework(MFramework mFramework, Connection conn) {
+  public void updateDriverConfig(MDriverConfig mDriverConfig, Connection conn) {
     PreparedStatement deleteForm = null;
     PreparedStatement deleteInput = null;
     try {
@@ -1095,13 +1086,13 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
       deleteForm.executeUpdate();
 
     } catch (SQLException e) {
-      logException(e, mFramework);
+      logException(e, mDriverConfig);
       throw new SqoopException(DerbyRepoError.DERBYREPO_0044, e);
     } finally {
       closeStatements(deleteForm, deleteInput);
     }
-    createOrUpdateFrameworkVersion(conn, mFramework);
-    insertFormsForFramework(mFramework, conn);
+    createOrUpdateDriverVersion(conn, mDriverConfig);
+    insertFormsForFramework(mDriverConfig, conn);
 
   }
 
@@ -1116,8 +1107,8 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
       stmt = conn.prepareStatement(STMT_INSERT_JOB,
         Statement.RETURN_GENERATED_KEYS);
       stmt.setString(1, job.getName());
-      stmt.setLong(2, job.getConnectionId(Direction.FROM));
-      stmt.setLong(3, job.getConnectionId(Direction.TO));
+      stmt.setLong(2, job.getLinkId(Direction.FROM));
+      stmt.setLong(3, job.getLinkId(Direction.TO));
       stmt.setBoolean(4, job.getEnabled());
       stmt.setString(5, job.getCreationUser());
       stmt.setTimestamp(6, new Timestamp(job.getCreationDate().getTime()));
@@ -1318,7 +1309,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
           + " job with id " + id);
       }
 
-      // Return the first and only one connection object
+      // Return the first and only one link object
       return jobs.get(0);
 
     } catch (SQLException ex) {
@@ -1652,7 +1643,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * Resolves counter group database id.
    *
    * @param group Given group
-   * @param conn Connection to metastore
+   * @param conn Connection to database
    * @return Id
    * @throws SQLException
    */
@@ -1693,7 +1684,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * Resolves counter id.
    *
    * @param counter Given counter
-   * @param conn connection to metastore
+   * @param conn Connection to database
    * @return Id
    * @throws SQLException
    */
@@ -1734,7 +1725,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * Create MSubmission structure from result set.
    *
    * @param rs Result set, only active row will be fetched
-   * @param conn Connection to metastore
+   * @param conn Connection to database
    * @return Created MSubmission structure
    * @throws SQLException
    */
@@ -1836,10 +1827,10 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
     return connectors;
   }
 
-  private List<MConnection> loadConnections(PreparedStatement stmt,
+  private List<MLink> loadLinks(PreparedStatement stmt,
                                             Connection conn)
                                             throws SQLException {
-    List<MConnection> connections = new ArrayList<MConnection>();
+    List<MLink> links = new ArrayList<MLink>();
     ResultSet rsConnection = null;
     PreparedStatement formConnectorFetchStmt = null;
     PreparedStatement formFrameworkFetchStmt = null;
@@ -1879,19 +1870,19 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
         loadFrameworkForms(frameworkConnForms, frameworkJobForms,
             formFrameworkFetchStmt, inputFetchStmt, 2);
 
-        MConnection connection = new MConnection(connectorId,
+        MLink link = new MLink(connectorId,
           new MConnectionForms(connectorConnForms),
           new MConnectionForms(frameworkConnForms));
 
-        connection.setPersistenceId(id);
-        connection.setName(name);
-        connection.setCreationUser(creationUser);
-        connection.setCreationDate(creationDate);
-        connection.setLastUpdateUser(updateUser);
-        connection.setLastUpdateDate(lastUpdateDate);
-        connection.setEnabled(enabled);
+        link.setPersistenceId(id);
+        link.setName(name);
+        link.setCreationUser(creationUser);
+        link.setCreationDate(creationDate);
+        link.setLastUpdateUser(updateUser);
+        link.setLastUpdateDate(lastUpdateDate);
+        link.setEnabled(enabled);
 
-        connections.add(connection);
+        links.add(link);
       }
     } finally {
       closeResultSets(rsConnection);
@@ -1899,7 +1890,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
         formFrameworkFetchStmt, inputFetchStmt);
     }
 
-    return connections;
+    return links;
   }
 
   private List<MJob> loadJobs(PreparedStatement stmt,
@@ -1925,8 +1916,8 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
         long toConnectorId = rsJob.getLong(2);
         long id = rsJob.getLong(3);
         String name = rsJob.getString(4);
-        long fromConnectionId = rsJob.getLong(5);
-        long toConnectionId = rsJob.getLong(6);
+        long fromLinkId = rsJob.getLong(5);
+        long toLinkId = rsJob.getLong(6);
         boolean enabled = rsJob.getBoolean(7);
         String createBy = rsJob.getString(8);
         Date creationDate = rsJob.getTimestamp(9);
@@ -1971,7 +1962,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
 
         MJob job = new MJob(
           fromConnectorId, toConnectorId,
-          fromConnectionId, toConnectionId,
+          fromLinkId, toLinkId,
           new MJobForms(fromConnectorFromJobForms),
           new MJobForms(toConnectorToJobForms),
           new MJobForms(frameworkJobForms));
@@ -2142,7 +2133,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * Use given prepared statements to load all forms and corresponding inputs
    * from Derby.
    *
-   * @param connectionForms List of connection forms that will be filled up
+   * @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 inputFetchStmt Prepare statement for fetching inputs
@@ -2159,14 +2150,13 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
     while (rsetForm.next()) {
       long formId = rsetForm.getLong(1);
       Long formConnectorId = rsetForm.getLong(2);
-      String direction = rsetForm.getString(3);
       String formName = rsetForm.getString(4);
       String formType = rsetForm.getString(5);
       int formIndex = rsetForm.getInt(6);
       List<MInput<?>> formInputs = new ArrayList<MInput<?>>();
 
-      MForm mf = new MForm(formName, formInputs);
-      mf.setPersistenceId(formId);
+      MForm mDriverConfig = new MForm(formName, formInputs);
+      mDriverConfig.setPersistenceId(formId);
 
       inputFetchStmt.setLong(formPosition, formId);
 
@@ -2217,52 +2207,52 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
           input.restoreFromUrlSafeValueString(value);
         }
 
-        if (mf.getInputs().size() != inputIndex) {
+        if (mDriverConfig.getInputs().size() != inputIndex) {
           throw new SqoopException(DerbyRepoError.DERBYREPO_0009,
-            "form: " + mf
+            "form: " + mDriverConfig
             + "; input: " + input
             + "; index: " + inputIndex
-            + "; expected: " + mf.getInputs().size()
+            + "; expected: " + mDriverConfig.getInputs().size()
           );
         }
 
-        mf.getInputs().add(input);
+        mDriverConfig.getInputs().add(input);
       }
 
-      if (mf.getInputs().size() == 0) {
+      if (mDriverConfig.getInputs().size() == 0) {
         throw new SqoopException(DerbyRepoError.DERBYREPO_0008,
           "connector-" + formConnectorId
-          + "; form: " + mf
+          + "; form: " + mDriverConfig
         );
       }
 
-      MFormType mft = MFormType.valueOf(formType);
-      switch (mft) {
+      MFormType mDriverConfigt = MFormType.valueOf(formType);
+      switch (mDriverConfigt) {
       case CONNECTION:
         if (connectionForms.size() != formIndex) {
           throw new SqoopException(DerbyRepoError.DERBYREPO_0010,
             "connector-" + formConnectorId
-            + "; form: " + mf
+            + "; form: " + mDriverConfig
             + "; index: " + formIndex
             + "; expected: " + connectionForms.size()
           );
         }
-        connectionForms.add(mf);
+        connectionForms.add(mDriverConfig);
         break;
       case JOB:
         if (jobForms.size() != formIndex) {
           throw new SqoopException(DerbyRepoError.DERBYREPO_0010,
             "connector-" + formConnectorId
-            + "; form: " + mf
+            + "; form: " + mDriverConfig
             + "; index: " + formIndex
             + "; expected: " + jobForms.size()
           );
         }
-        jobForms.add(mf);
+        jobForms.add(mDriverConfig);
         break;
       default:
         throw new SqoopException(DerbyRepoError.DERBYREPO_0007,
-            "connector-" + formConnectorId + ":" + mf);
+            "connector-" + formConnectorId + ":" + mDriverConfig);
       }
     }
   }
@@ -2273,7 +2263,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
    * Use given prepared statements to load all forms and corresponding inputs
    * from Derby.
    *
-   * @param connectionForms List of connection forms that will be filled up
+   * @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
@@ -2298,8 +2288,8 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
       int formIndex = rsetForm.getInt(6);
       List<MInput<?>> formInputs = new ArrayList<MInput<?>>();
 
-      MForm mf = new MForm(formName, formInputs);
-      mf.setPersistenceId(formId);
+      MForm mDriverConfig = new MForm(formName, formInputs);
+      mDriverConfig.setPersistenceId(formId);
 
       inputFetchStmt.setLong(formPosition, formId);
 
@@ -2317,7 +2307,7 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
 
         MInputType mit = MInputType.valueOf(inputType);
 
-        MInput input = null;
+        MInput<?> input = null;
         switch (mit) {
           case STRING:
             input = new MStringInput(inputName, inputSensitivity, inputStrLength);
@@ -2350,37 +2340,37 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
           input.restoreFromUrlSafeValueString(value);
         }
 
-        if (mf.getInputs().size() != inputIndex) {
+        if (mDriverConfig.getInputs().size() != inputIndex) {
           throw new SqoopException(DerbyRepoError.DERBYREPO_0009,
-              "form: " + mf
+              "form: " + mDriverConfig
                   + "; input: " + input
                   + "; index: " + inputIndex
-                  + "; expected: " + mf.getInputs().size()
+                  + "; expected: " + mDriverConfig.getInputs().size()
           );
         }
 
-        mf.getInputs().add(input);
+        mDriverConfig.getInputs().add(input);
       }
 
-      if (mf.getInputs().size() == 0) {
+      if (mDriverConfig.getInputs().size() == 0) {
         throw new SqoopException(DerbyRepoError.DERBYREPO_0008,
             "connector-" + formConnectorId
-                + "; form: " + mf
+                + "; form: " + mDriverConfig
         );
       }
 
-      MFormType mft = MFormType.valueOf(formType);
-      switch (mft) {
+      MFormType mDriverConfigt = MFormType.valueOf(formType);
+      switch (mDriverConfigt) {
         case CONNECTION:
           if (connectionForms.size() != formIndex) {
             throw new SqoopException(DerbyRepoError.DERBYREPO_0010,
                 "connector-" + formConnectorId
-                    + "; form: " + mf
+                    + "; form: " + mDriverConfig
                     + "; index: " + formIndex
                     + "; expected: " + connectionForms.size()
             );
           }
-          connectionForms.add(mf);
+          connectionForms.add(mDriverConfig);
           break;
         case JOB:
           Direction type = Direction.valueOf(operation);
@@ -2401,17 +2391,17 @@ public class DerbyRepositoryHandler extends JdbcRepositoryHandler {
           if (jobForms.size() != formIndex) {
             throw new SqoopException(DerbyRepoError.DERBYREPO_0010,
                 "connector-" + formConnectorId
-                    + "; form: " + mf
+                    + "; form: " + mDriverConfig
                     + "; index: " + formIndex
                     + "; expected: " + jobForms.size()
             );
           }
 
-          jobForms.add(mf);
+          jobForms.add(mDriverConfig);
           break;
         default:
           throw new SqoopException(DerbyRepoError.DERBYREPO_0007,
-              "connector-" + formConnectorId + ":" + mf);
+              "connector-" + formConnectorId + ":" + mDriverConfig);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaQuery.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaQuery.java b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaQuery.java
index 061551e..ad42901 100644
--- a/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaQuery.java
+++ b/repository/repository-derby/src/main/java/org/apache/sqoop/repository/derby/DerbySchemaQuery.java
@@ -1012,7 +1012,7 @@ public final class DerbySchemaQuery {
   /**
    * Intended to rename forms based on direction.
    * e.g. If SQ_FORM.SQF_NAME = 'table' and parameter 1 = 'from'
-   * then SQ_FORM.SQF_NAME = 'fromTable'.
+   * then SQ_FORM.SQF_NAME = 'fromJobConfig'.
    */
   public static final String QUERY_UPGRADE_TABLE_SQ_FORM_UPDATE_TABLE_INPUT_NAMES =
       "UPDATE " + TABLE_SQ_INPUT + " SET "

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/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 29da340..998f5b7 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
@@ -18,13 +18,14 @@
 package org.apache.sqoop.repository.derby;
 
 import junit.framework.TestCase;
+
 import org.apache.sqoop.common.Direction;
-import org.apache.sqoop.framework.FrameworkManager;
-import org.apache.sqoop.model.MConnection;
+import org.apache.sqoop.driver.Driver;
+import org.apache.sqoop.model.MLink;
 import org.apache.sqoop.model.MConnectionForms;
 import org.apache.sqoop.model.MConnector;
 import org.apache.sqoop.model.MForm;
-import org.apache.sqoop.model.MFramework;
+import org.apache.sqoop.model.MDriverConfig;
 import org.apache.sqoop.model.MInput;
 import org.apache.sqoop.model.MJob;
 import org.apache.sqoop.model.MJobForms;
@@ -48,7 +49,7 @@ import static org.apache.sqoop.repository.derby.DerbySchemaQuery.*;
  */
 abstract public class DerbyTestCase extends TestCase {
 
-  private static int SYSTEM_VERSION = 4;
+  private static int LATEST_SYSTEM_VERSION = 4;
 
   public static final String DERBY_DRIVER =
     "org.apache.derby.jdbc.EmbeddedDriver";
@@ -62,14 +63,14 @@ abstract public class DerbyTestCase extends TestCase {
   public void setUp() throws Exception {
     super.setUp();
 
-    // Create connection to the database
+    // Create link to the database
     Class.forName(DERBY_DRIVER).newInstance();
     connection = DriverManager.getConnection(getStartJdbcUrl());
   }
 
   @Override
   public void tearDown() throws Exception {
-    // Close active connection
+    // Close active link
     if(connection != null) {
       connection.close();
     }
@@ -130,11 +131,11 @@ abstract public class DerbyTestCase extends TestCase {
 
     runQuery("INSERT INTO SQOOP.SQ_SYSTEM(SQM_KEY, SQM_VALUE) VALUES('version', '"  + version + "')");
     runQuery("INSERT INTO SQOOP.SQ_SYSTEM(SQM_KEY, SQM_VALUE) " +
-      "VALUES('framework.version', '1')");
+      "VALUES('" + DerbyRepoConstants.SYSKEY_DRIVER_VERSION + "', '1')");
   }
 
   protected void createSchema() throws Exception {
-    createSchema(SYSTEM_VERSION);
+    createSchema(LATEST_SYSTEM_VERSION);
   }
 
   /**
@@ -146,7 +147,7 @@ abstract public class DerbyTestCase extends TestCase {
   protected void runQuery(String query, String... args) throws Exception {
     PreparedStatement stmt = null;
     try {
-      stmt = getDerbyConnection().prepareStatement(query);
+      stmt = getDerbyDatabaseConnection().prepareStatement(query);
 
       for (int i = 0; i < args.length; ++i) {
         stmt.setString(i + 1, args[i]);
@@ -160,7 +161,7 @@ abstract public class DerbyTestCase extends TestCase {
     }
   }
 
-  protected Connection getDerbyConnection() {
+  protected Connection getDerbyDatabaseConnection() {
     return connection;
   }
 
@@ -176,7 +177,7 @@ abstract public class DerbyTestCase extends TestCase {
     return JDBC_URL + ";drop=true";
   }
 
-  protected void loadConnectorAndFrameworkVersion2() throws Exception {
+  protected void loadConnectorAndDriverConfigVersion2() throws Exception {
     // Connector entry
     runQuery("INSERT INTO SQOOP.SQ_CONNECTOR(SQC_NAME, SQC_CLASS, SQC_VERSION)"
         + "VALUES('A', 'org.apache.sqoop.test.A', '1.0-test')");
@@ -255,7 +256,7 @@ abstract public class DerbyTestCase extends TestCase {
         + "('throttling.loaders',10,1,'INTEGER','false',NULL,NULL)");
   }
 
-  protected void loadConnectorAndFrameworkVersion4() throws Exception {
+  protected void loadConnectorAndDriverConfigVersion4() throws Exception {
     // Connector entry
     runQuery("INSERT INTO SQOOP.SQ_CONNECTOR(SQC_NAME, SQC_CLASS, SQC_VERSION)"
         + "VALUES('A', 'org.apache.sqoop.test.A', '1.0-test')");
@@ -308,10 +309,10 @@ abstract public class DerbyTestCase extends TestCase {
     }
 
     // Input entries
-    // Connector connection parts: 0-3
+    // Connector link parts: 0-3
     // Connector job (FROM) parts: 4-7
     // Connector job (TO) parts: 8-11
-    // Framework connection parts: 12-15
+    // Framework link parts: 12-15
     // Framework job parts: 16-19
     for (int i = 0; i < 5; i++) {
       // First form
@@ -333,19 +334,19 @@ abstract public class DerbyTestCase extends TestCase {
   }
 
   /**
-   * Load testing connector and framework metadata into repository.
+   * Load testing connector and driver config into repository.
    *
    * @param version system version (2 or 4)
    * @throws Exception
    */
-  protected void loadConnectorAndFramework(int version) throws Exception {
+  protected void loadConnectorAndDriverConfig(int version) throws Exception {
     switch(version) {
       case 2:
-        loadConnectorAndFrameworkVersion2();
+        loadConnectorAndDriverConfigVersion2();
         break;
 
       case 4:
-        loadConnectorAndFrameworkVersion4();
+        loadConnectorAndDriverConfigVersion4();
         break;
 
       default:
@@ -353,20 +354,20 @@ abstract public class DerbyTestCase extends TestCase {
     }
   }
 
-  protected void loadConnectorAndFramework() throws Exception {
-    loadConnectorAndFramework(SYSTEM_VERSION);
+  protected void loadConnectorAndDriverConfig() throws Exception {
+    loadConnectorAndDriverConfig(LATEST_SYSTEM_VERSION);
   }
 
   /**
-   * Load testing connection objects into metadata repository.
+   * Load testing link objects into  repository.
    *
    * @param version system version (2 or 4)
    * @throws Exception
    */
-  public void loadConnections(int version) throws Exception {
+  public void loadLinks(int version) throws Exception {
     switch (version) {
       case 2:
-        // Insert two connections - CA and CB
+        // Insert two links - CA and CB
         runQuery("INSERT INTO SQOOP.SQ_CONNECTION(SQN_NAME, SQN_CONNECTOR) "
             + "VALUES('CA', 1)");
         runQuery("INSERT INTO SQOOP.SQ_CONNECTION(SQN_NAME, SQN_CONNECTOR) "
@@ -382,7 +383,7 @@ abstract public class DerbyTestCase extends TestCase {
         break;
 
       case 4:
-        // Insert two connections - CA and CB
+        // Insert two links - CA and CB
         runQuery("INSERT INTO SQOOP.SQ_CONNECTION(SQN_NAME, SQN_CONNECTOR) "
             + "VALUES('CA', 1)");
         runQuery("INSERT INTO SQOOP.SQ_CONNECTION(SQN_NAME, SQN_CONNECTOR) "
@@ -402,12 +403,12 @@ abstract public class DerbyTestCase extends TestCase {
     }
   }
 
-  public void loadConnections() throws Exception {
-    loadConnections(SYSTEM_VERSION);
+  public void loadLinks() throws Exception {
+    loadLinks(LATEST_SYSTEM_VERSION);
   }
 
   /**
-   * Load testing job objects into metadata repository.
+   * Load testing job objects into  repository.
    *
    * @param version system version (2 or 4)
    * @throws Exception
@@ -470,7 +471,7 @@ abstract public class DerbyTestCase extends TestCase {
   }
 
   public void loadJobs() throws Exception {
-    loadJobs(SYSTEM_VERSION);
+    loadJobs(LATEST_SYSTEM_VERSION);
   }
 
   /**
@@ -483,7 +484,7 @@ abstract public class DerbyTestCase extends TestCase {
   }
 
   /**
-   * Load testing submissions into the metadata repository.
+   * Load testing submissions into the repository.
    *
    * @throws Exception
    */
@@ -527,22 +528,22 @@ abstract public class DerbyTestCase extends TestCase {
 
   protected MConnector getConnector() {
     return new MConnector("A", "org.apache.sqoop.test.A", "1.0-test",
-      getConnectionForms(), new MJobForms(getForms()), new MJobForms(getForms()));
+      getConnectionForms(), getJobForms(), getJobForms());
   }
 
-  protected MFramework getFramework() {
-    return new MFramework(getConnectionForms(), new MJobForms(getForms()),
-      FrameworkManager.CURRENT_FRAMEWORK_VERSION);
+  protected MDriverConfig getDriverConfig() {
+    return new MDriverConfig(getConnectionForms(), getJobForms(),
+        Driver.CURRENT_DRIVER_VERSION);
   }
 
-  protected void fillConnection(MConnection connection) {
+  protected void fillLink(MLink link) {
     List<MForm> forms;
 
-    forms = connection.getConnectorPart().getForms();
+    forms = link.getConnectorPart().getForms();
     ((MStringInput)forms.get(0).getInputs().get(0)).setValue("Value1");
     ((MStringInput)forms.get(1).getInputs().get(0)).setValue("Value2");
 
-    forms = connection.getFrameworkPart().getForms();
+    forms = link.getFrameworkPart().getForms();
     ((MStringInput)forms.get(0).getInputs().get(0)).setValue("Value13");
     ((MStringInput)forms.get(1).getInputs().get(0)).setValue("Value15");
   }
@@ -566,6 +567,10 @@ abstract public class DerbyTestCase extends TestCase {
   protected MConnectionForms getConnectionForms() {
     return new MConnectionForms(getForms());
   }
+  
+  protected MJobForms getJobForms() {
+    return  new MJobForms(getForms());
+  }
 
   protected List<MForm> getForms() {
     List<MForm> forms = new LinkedList<MForm>();
@@ -602,7 +607,7 @@ abstract public class DerbyTestCase extends TestCase {
     ResultSet rs = null;
 
     try {
-      stmt = getDerbyConnection().createStatement();
+      stmt = getDerbyDatabaseConnection().createStatement();
 
       rs = stmt.executeQuery("SELECT COUNT(*) FROM "+ table);
       rs.next();
@@ -658,7 +663,7 @@ abstract public class DerbyTestCase extends TestCase {
     ResultSetMetaData rsmt = null;
 
     try {
-      ps = getDerbyConnection().prepareStatement("SELECT * FROM " + table);
+      ps = getDerbyDatabaseConnection().prepareStatement("SELECT * FROM " + table);
       rs = ps.executeQuery();
 
       rsmt = rs.getMetaData();

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestConnectionHandling.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestConnectionHandling.java b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestConnectionHandling.java
deleted file mode 100644
index f9e9217..0000000
--- a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestConnectionHandling.java
+++ /dev/null
@@ -1,245 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.sqoop.repository.derby;
-
-import org.apache.sqoop.common.SqoopException;
-import org.apache.sqoop.model.MConnection;
-import org.apache.sqoop.model.MForm;
-import org.apache.sqoop.model.MJob;
-import org.apache.sqoop.model.MMapInput;
-import org.apache.sqoop.model.MStringInput;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Test connection methods on Derby repository.
- */
-public class TestConnectionHandling extends DerbyTestCase {
-
-  DerbyRepositoryHandler handler;
-
-  @Override
-  public void setUp() throws Exception {
-    super.setUp();
-
-    handler = new DerbyRepositoryHandler();
-
-    // We always needs schema for this test case
-    createSchema();
-
-    // We always needs connector and framework structures in place
-    loadConnectorAndFramework();
-  }
-
-  public void testFindConnection() throws Exception {
-    // Let's try to find non existing connection
-    try {
-      handler.findConnection(1, getDerbyConnection());
-      fail();
-    } catch(SqoopException ex) {
-      assertEquals(DerbyRepoError.DERBYREPO_0024, ex.getErrorCode());
-    }
-
-    // Load prepared connections into database
-    loadConnections();
-
-    MConnection connA = handler.findConnection(1, getDerbyConnection());
-    assertNotNull(connA);
-    assertEquals(1, connA.getPersistenceId());
-    assertEquals("CA", connA.getName());
-
-    List<MForm> forms;
-
-    // Check connector part
-    forms = connA.getConnectorPart().getForms();
-    assertEquals("Value1", forms.get(0).getInputs().get(0).getValue());
-    assertNull(forms.get(0).getInputs().get(1).getValue());
-    assertEquals("Value3", forms.get(1).getInputs().get(0).getValue());
-    assertNull(forms.get(1).getInputs().get(1).getValue());
-
-    // Check framework part
-    forms = connA.getFrameworkPart().getForms();
-    assertEquals("Value13", forms.get(0).getInputs().get(0).getValue());
-    assertNull(forms.get(0).getInputs().get(1).getValue());
-    assertEquals("Value15", forms.get(1).getInputs().get(0).getValue());
-    assertNull(forms.get(1).getInputs().get(1).getValue());
-  }
-
-  public void testFindConnections() throws Exception {
-    List<MConnection> list;
-
-    // Load empty list on empty repository
-    list = handler.findConnections(getDerbyConnection());
-    assertEquals(0, list.size());
-
-    loadConnections();
-
-    // Load all two connections on loaded repository
-    list = handler.findConnections(getDerbyConnection());
-    assertEquals(2, list.size());
-
-    assertEquals("CA", list.get(0).getName());
-    assertEquals("CB", list.get(1).getName());
-  }
-
-  public void testExistsConnection() throws Exception {
-    // There shouldn't be anything on empty repository
-    assertFalse(handler.existsConnection(1, getDerbyConnection()));
-    assertFalse(handler.existsConnection(2, getDerbyConnection()));
-    assertFalse(handler.existsConnection(3, getDerbyConnection()));
-
-    loadConnections();
-
-    assertTrue(handler.existsConnection(1, getDerbyConnection()));
-    assertTrue(handler.existsConnection(2, getDerbyConnection()));
-    assertFalse(handler.existsConnection(3, getDerbyConnection()));
-  }
-
-  public void testCreateConnection() throws Exception {
-    MConnection connection = getConnection();
-
-    // Load some data
-    fillConnection(connection);
-
-    handler.createConnection(connection, getDerbyConnection());
-
-    assertEquals(1, connection.getPersistenceId());
-    assertCountForTable("SQOOP.SQ_CONNECTION", 1);
-    assertCountForTable("SQOOP.SQ_CONNECTION_INPUT", 4);
-
-    MConnection retrieved = handler.findConnection(1, getDerbyConnection());
-    assertEquals(1, retrieved.getPersistenceId());
-
-    List<MForm> forms;
-    forms = connection.getConnectorPart().getForms();
-    assertEquals("Value1", forms.get(0).getInputs().get(0).getValue());
-    assertNull(forms.get(0).getInputs().get(1).getValue());
-    assertEquals("Value2", forms.get(1).getInputs().get(0).getValue());
-    assertNull(forms.get(1).getInputs().get(1).getValue());
-
-    forms = connection.getFrameworkPart().getForms();
-    assertEquals("Value13", forms.get(0).getInputs().get(0).getValue());
-    assertNull(forms.get(0).getInputs().get(1).getValue());
-    assertEquals("Value15", forms.get(1).getInputs().get(0).getValue());
-    assertNull(forms.get(1).getInputs().get(1).getValue());
-
-    // Let's create second connection
-    connection = getConnection();
-    fillConnection(connection);
-
-    handler.createConnection(connection, getDerbyConnection());
-
-    assertEquals(2, connection.getPersistenceId());
-    assertCountForTable("SQOOP.SQ_CONNECTION", 2);
-    assertCountForTable("SQOOP.SQ_CONNECTION_INPUT", 8);
-  }
-
-  public void testInUseConnection() throws Exception {
-    loadConnections();
-
-    assertFalse(handler.inUseConnection(1, getDerbyConnection()));
-
-    loadJobs();
-
-    assertTrue(handler.inUseConnection(1, getDerbyConnection()));
-  }
-
-  public void testUpdateConnection() throws Exception {
-    loadConnections();
-
-    MConnection connection = handler.findConnection(1, getDerbyConnection());
-
-    List<MForm> forms;
-
-    forms = connection.getConnectorPart().getForms();
-    ((MStringInput)forms.get(0).getInputs().get(0)).setValue("Updated");
-    ((MMapInput)forms.get(0).getInputs().get(1)).setValue(null);
-    ((MStringInput)forms.get(1).getInputs().get(0)).setValue("Updated");
-    ((MMapInput)forms.get(1).getInputs().get(1)).setValue(null);
-
-    forms = connection.getFrameworkPart().getForms();
-    ((MStringInput)forms.get(0).getInputs().get(0)).setValue("Updated");
-    ((MMapInput)forms.get(0).getInputs().get(1)).setValue(new HashMap<String, String>()); // inject new map value
-    ((MStringInput)forms.get(1).getInputs().get(0)).setValue("Updated");
-    ((MMapInput)forms.get(1).getInputs().get(1)).setValue(new HashMap<String, String>()); // inject new map value
-
-    connection.setName("name");
-
-    handler.updateConnection(connection, getDerbyConnection());
-
-    assertEquals(1, connection.getPersistenceId());
-    assertCountForTable("SQOOP.SQ_CONNECTION", 2);
-    assertCountForTable("SQOOP.SQ_CONNECTION_INPUT", 10);
-
-    MConnection retrieved = handler.findConnection(1, getDerbyConnection());
-    assertEquals("name", connection.getName());
-
-    forms = retrieved.getConnectorPart().getForms();
-    assertEquals("Updated", forms.get(0).getInputs().get(0).getValue());
-    assertNull(forms.get(0).getInputs().get(1).getValue());
-    assertEquals("Updated", forms.get(1).getInputs().get(0).getValue());
-    assertNull(forms.get(1).getInputs().get(1).getValue());
-
-    forms = retrieved.getFrameworkPart().getForms();
-    assertEquals("Updated", forms.get(0).getInputs().get(0).getValue());
-    assertNotNull(forms.get(0).getInputs().get(1).getValue());
-    assertEquals(((Map)forms.get(0).getInputs().get(1).getValue()).size(), 0);
-    assertEquals("Updated", forms.get(1).getInputs().get(0).getValue());
-    assertNotNull(forms.get(1).getInputs().get(1).getValue());
-    assertEquals(((Map)forms.get(1).getInputs().get(1).getValue()).size(), 0);
-  }
-
-  public void testEnableAndDisableConnection() throws Exception {
-    loadConnections();
-
-    // disable connection 1
-    handler.enableConnection(1, false, getDerbyConnection());
-
-    MConnection retrieved = handler.findConnection(1, getDerbyConnection());
-    assertNotNull(retrieved);
-    assertEquals(false, retrieved.getEnabled());
-
-    // enable connection 1
-    handler.enableConnection(1, true, getDerbyConnection());
-
-    retrieved = handler.findConnection(1, getDerbyConnection());
-    assertNotNull(retrieved);
-    assertEquals(true, retrieved.getEnabled());
-  }
-
-  public void testDeleteConnection() throws Exception {
-    loadConnections();
-
-    handler.deleteConnection(1, getDerbyConnection());
-    assertCountForTable("SQOOP.SQ_CONNECTION", 1);
-    assertCountForTable("SQOOP.SQ_CONNECTION_INPUT", 4);
-
-    handler.deleteConnection(2, getDerbyConnection());
-    assertCountForTable("SQOOP.SQ_CONNECTION", 0);
-    assertCountForTable("SQOOP.SQ_CONNECTION_INPUT", 0);
-  }
-
-  public MConnection getConnection() {
-    return new MConnection(1,
-      handler.findConnector("A", getDerbyConnection()).getConnectionForms(),
-      handler.findFramework(getDerbyConnection()).getConnectionForms()
-    );
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/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 745e128..8a5823d 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
@@ -40,14 +40,14 @@ public class TestConnectorHandling extends DerbyTestCase {
 
   public void testFindConnector() throws Exception {
     // On empty repository, no connectors should be there
-    assertNull(handler.findConnector("A", getDerbyConnection()));
-    assertNull(handler.findConnector("B", getDerbyConnection()));
+    assertNull(handler.findConnector("A", getDerbyDatabaseConnection()));
+    assertNull(handler.findConnector("B", getDerbyDatabaseConnection()));
 
     // Load connector into repository
-    loadConnectorAndFramework();
+    loadConnectorAndDriverConfig();
 
     // Retrieve it
-    MConnector connector = handler.findConnector("A", getDerbyConnection());
+    MConnector connector = handler.findConnector("A", getDerbyDatabaseConnection());
     assertNotNull(connector);
 
     // Get original structure
@@ -59,13 +59,13 @@ public class TestConnectorHandling extends DerbyTestCase {
 
   public void testFindAllConnectors() throws Exception {
     // No connectors in an empty repository, we expect an empty list
-    assertEquals(handler.findConnectors(getDerbyConnection()).size(),0);
+    assertEquals(handler.findConnectors(getDerbyDatabaseConnection()).size(),0);
 
-    loadConnectorAndFramework();
+    loadConnectorAndDriverConfig();
     addConnector();
 
     // Retrieve connectors
-    List<MConnector> connectors = handler.findConnectors(getDerbyConnection());
+    List<MConnector> connectors = handler.findConnectors(getDerbyDatabaseConnection());
     assertNotNull(connectors);
     assertEquals(connectors.size(),2);
     assertEquals(connectors.get(0).getUniqueName(),"A");
@@ -77,7 +77,7 @@ public class TestConnectorHandling extends DerbyTestCase {
   public void testRegisterConnector() throws Exception {
     MConnector connector = getConnector();
 
-    handler.registerConnector(connector, getDerbyConnection());
+    handler.registerConnector(connector, getDerbyDatabaseConnection());
 
     // Connector should get persistence ID
     assertEquals(1, connector.getPersistenceId());
@@ -88,7 +88,7 @@ public class TestConnectorHandling extends DerbyTestCase {
     assertCountForTable("SQOOP.SQ_INPUT", 12);
 
     // Registered connector should be easily recovered back
-    MConnector retrieved = handler.findConnector("A", getDerbyConnection());
+    MConnector retrieved = handler.findConnector("A", getDerbyDatabaseConnection());
     assertNotNull(retrieved);
     assertEquals(connector, retrieved);
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestDriverConfigHandling.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestDriverConfigHandling.java b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestDriverConfigHandling.java
new file mode 100644
index 0000000..567dda6
--- /dev/null
+++ b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestDriverConfigHandling.java
@@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.sqoop.repository.derby;
+
+import org.apache.sqoop.driver.Driver;
+import org.apache.sqoop.model.MDriverConfig;
+
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+/**
+ * Test driver config methods on Derby repository.
+ */
+public class TestDriverConfigHandling extends DerbyTestCase {
+
+  DerbyRepositoryHandler handler;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+
+    handler = new DerbyRepositoryHandler();
+
+    // We always needs schema for this test case
+    createSchema();
+  }
+
+  public void testFindDriverConfig() throws Exception {
+    // On empty repository, no driverConfig should be there
+    assertNull(handler.findDriverConfig(getDerbyDatabaseConnection()));
+    // Load Connector and DriverConfig into repository
+    loadConnectorAndDriverConfig();
+    // Retrieve it
+    MDriverConfig driverConfig = handler.findDriverConfig(getDerbyDatabaseConnection());
+    assertNotNull(driverConfig);
+
+    // Get original structure
+    MDriverConfig originalDriverConfig = getDriverConfig();
+
+    // And compare them
+    assertEquals(originalDriverConfig, driverConfig);
+  }
+
+  public void testRegisterConnector() throws Exception {
+    MDriverConfig driverConfig = getDriverConfig();
+    handler.registerDriverConfig(driverConfig, getDerbyDatabaseConnection());
+
+    // Connector should get persistence ID
+    assertEquals(1, driverConfig.getPersistenceId());
+
+    // Now check content in corresponding tables
+    assertCountForTable("SQOOP.SQ_CONNECTOR", 0);
+    assertCountForTable("SQOOP.SQ_FORM", 4);
+    assertCountForTable("SQOOP.SQ_INPUT", 8);
+
+    // Registered framework should be easily recovered back
+    MDriverConfig retrieved = handler.findDriverConfig(getDerbyDatabaseConnection());
+    assertNotNull(retrieved);
+    assertEquals(driverConfig, retrieved);
+    assertEquals(driverConfig.getVersion(), retrieved.getVersion());
+  }
+
+  private String getDriverVersion() throws Exception {
+    final String frameworkVersionQuery =
+      "SELECT SQM_VALUE FROM SQOOP.SQ_SYSTEM WHERE SQM_KEY=?";
+    String retVal = null;
+    PreparedStatement preparedStmt = null;
+    ResultSet resultSet = null;
+    try {
+      preparedStmt =
+        getDerbyDatabaseConnection().prepareStatement(frameworkVersionQuery);
+      preparedStmt.setString(1, DerbyRepoConstants.SYSKEY_DRIVER_VERSION);
+      resultSet = preparedStmt.executeQuery();
+      if(resultSet.next())
+        retVal = resultSet.getString(1);
+      return retVal;
+    } finally {
+      if(preparedStmt !=null) {
+        try {
+          preparedStmt.close();
+        } catch(SQLException e) {
+        }
+      }
+      if(resultSet != null) {
+        try {
+          resultSet.close();
+        } catch(SQLException e) {
+        }
+      }
+    }
+  }
+
+  public void testDriverVersion() throws Exception {
+    handler.registerDriverConfig(getDriverConfig(), getDerbyDatabaseConnection());
+
+    final String lowerVersion = Integer.toString(
+      Integer.parseInt(Driver.CURRENT_DRIVER_VERSION) - 1);
+    assertEquals(Driver.CURRENT_DRIVER_VERSION, getDriverVersion());
+    runQuery("UPDATE SQOOP.SQ_SYSTEM SET SQM_VALUE='" + lowerVersion +
+      "' WHERE SQM_KEY = '" + DerbyRepoConstants.SYSKEY_DRIVER_VERSION + "'");
+    assertEquals(lowerVersion, getDriverVersion());
+
+    MDriverConfig framework = getDriverConfig();
+    handler.updateDriverConfig(framework, getDerbyDatabaseConnection());
+
+    assertEquals(Driver.CURRENT_DRIVER_VERSION, framework.getVersion());
+
+    assertEquals(Driver.CURRENT_DRIVER_VERSION, getDriverVersion());
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestFrameworkHandling.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestFrameworkHandling.java b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestFrameworkHandling.java
deleted file mode 100644
index 006ec9c..0000000
--- a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestFrameworkHandling.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.sqoop.repository.derby;
-
-import org.apache.sqoop.framework.FrameworkManager;
-import org.apache.sqoop.model.MFramework;
-
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-
-/**
- * Test framework methods on Derby repository.
- */
-public class TestFrameworkHandling extends DerbyTestCase {
-
-  DerbyRepositoryHandler handler;
-
-  @Override
-  public void setUp() throws Exception {
-    super.setUp();
-
-    handler = new DerbyRepositoryHandler();
-
-    // We always needs schema for this test case
-    createSchema();
-  }
-
-  public void testFindFramework() throws Exception {
-    // On empty repository, no framework should be there
-    assertNull(handler.findFramework(getDerbyConnection()));
-
-    // Load framework into repository
-    loadConnectorAndFramework();
-
-    // Retrieve it
-    MFramework framework = handler.findFramework(getDerbyConnection());
-    assertNotNull(framework);
-
-    // Get original structure
-    MFramework original = getFramework();
-
-    // And compare them
-    assertEquals(original, framework);
-  }
-
-  public void testRegisterConnector() throws Exception {
-    MFramework framework = getFramework();
-
-    handler.registerFramework(framework, getDerbyConnection());
-
-    // Connector should get persistence ID
-    assertEquals(1, framework.getPersistenceId());
-
-    // Now check content in corresponding tables
-    assertCountForTable("SQOOP.SQ_CONNECTOR", 0);
-    assertCountForTable("SQOOP.SQ_FORM", 4);
-    assertCountForTable("SQOOP.SQ_INPUT", 8);
-
-    // Registered framework should be easily recovered back
-    MFramework retrieved = handler.findFramework(getDerbyConnection());
-    assertNotNull(retrieved);
-    assertEquals(framework, retrieved);
-    assertEquals(framework.getVersion(), retrieved.getVersion());
-  }
-
-  private String getFrameworkVersion() throws Exception {
-    final String frameworkVersionQuery =
-      "SELECT SQM_VALUE FROM SQOOP.SQ_SYSTEM WHERE SQM_KEY=?";
-    String retVal = null;
-    PreparedStatement preparedStmt = null;
-    ResultSet resultSet = null;
-    try {
-      preparedStmt =
-        getDerbyConnection().prepareStatement(frameworkVersionQuery);
-      preparedStmt.setString(1, DerbyRepoConstants.SYSKEY_FRAMEWORK_VERSION);
-      resultSet = preparedStmt.executeQuery();
-      if(resultSet.next())
-        retVal = resultSet.getString(1);
-      return retVal;
-    } finally {
-      if(preparedStmt !=null) {
-        try {
-          preparedStmt.close();
-        } catch(SQLException e) {
-        }
-      }
-      if(resultSet != null) {
-        try {
-          resultSet.close();
-        } catch(SQLException e) {
-        }
-      }
-    }
-  }
-
-  public void testFrameworkVersion() throws Exception {
-    handler.registerFramework(getFramework(), getDerbyConnection());
-
-    final String lowerVersion = Integer.toString(
-      Integer.parseInt(FrameworkManager.CURRENT_FRAMEWORK_VERSION) - 1);
-    assertEquals(FrameworkManager.CURRENT_FRAMEWORK_VERSION, getFrameworkVersion());
-    runQuery("UPDATE SQOOP.SQ_SYSTEM SET SQM_VALUE='" + lowerVersion +
-      "' WHERE SQM_KEY = '" + DerbyRepoConstants.SYSKEY_FRAMEWORK_VERSION + "'");
-    assertEquals(lowerVersion, getFrameworkVersion());
-
-    MFramework framework = getFramework();
-    handler.updateFramework(framework, getDerbyConnection());
-
-    assertEquals(FrameworkManager.CURRENT_FRAMEWORK_VERSION, framework.getVersion());
-
-    assertEquals(FrameworkManager.CURRENT_FRAMEWORK_VERSION, getFrameworkVersion());
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/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 15f9539..c4a5829 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
@@ -18,12 +18,12 @@
 package org.apache.sqoop.repository.derby;
 
 import org.apache.sqoop.model.MBooleanInput;
-import org.apache.sqoop.model.MConnection;
+import org.apache.sqoop.model.MLink;
 import org.apache.sqoop.model.MConnectionForms;
 import org.apache.sqoop.model.MConnector;
 import org.apache.sqoop.model.MEnumInput;
 import org.apache.sqoop.model.MForm;
-import org.apache.sqoop.model.MFramework;
+import org.apache.sqoop.model.MDriverConfig;
 import org.apache.sqoop.model.MInput;
 import org.apache.sqoop.model.MIntegerInput;
 import org.apache.sqoop.model.MMapInput;
@@ -56,17 +56,17 @@ public class TestInputTypes extends DerbyTestCase {
    * Ensure that metadata with all various data types can be successfully
    * serialized into repository and retrieved back.
    */
-  public void testMetadataSerialization() throws Exception {
+  public void testEntitySerialization() throws Exception {
     MConnector connector = getConnector();
 
     // Serialize the connector with all data types into repository
-    handler.registerConnector(connector, getDerbyConnection());
+    handler.registerConnector(connector, getDerbyDatabaseConnection());
 
     // Successful serialization should update the ID
     assertNotSame(connector.getPersistenceId(), MPersistableEntity.PERSISTANCE_ID_DEFAULT);
 
     // Retrieve registered connector
-    MConnector retrieved = handler.findConnector(connector.getUniqueName(), getDerbyConnection());
+    MConnector retrieved = handler.findConnector(connector.getUniqueName(), getDerbyDatabaseConnection());
     assertNotNull(retrieved);
 
     // Original and retrieved connectors should be the same
@@ -76,33 +76,33 @@ public class TestInputTypes extends DerbyTestCase {
   /**
    * Test that serializing actual data is not an issue.
    */
-  public void testDataSerialization() throws Exception {
+  public void testEntityDataSerialization() throws Exception {
     MConnector connector = getConnector();
-    MFramework framework = getFramework();
+    MDriverConfig driverConfig = getDriverConfig();
 
-    // Register metadata for everything and our new connector
-    handler.registerConnector(connector, getDerbyConnection());
-    handler.registerFramework(framework, getDerbyConnection());
+    // Register objects for everything and our new connector
+    handler.registerConnector(connector, getDerbyDatabaseConnection());
+    handler.registerDriverConfig(driverConfig, getDerbyDatabaseConnection());
 
     // Inserted values
     Map<String, String> map = new HashMap<String, String>();
     map.put("A", "B");
 
     // Connection object with all various values
-    MConnection connection = new MConnection(connector.getPersistenceId(), connector.getConnectionForms(), framework.getConnectionForms());
-    MConnectionForms forms = connection.getConnectorPart();
+    MLink link = new MLink(connector.getPersistenceId(), connector.getConnectionForms(), driverConfig.getConnectionForms());
+    MConnectionForms forms = link.getConnectorPart();
     forms.getStringInput("f.String").setValue("A");
     forms.getMapInput("f.Map").setValue(map);
     forms.getIntegerInput("f.Integer").setValue(1);
     forms.getBooleanInput("f.Boolean").setValue(true);
     forms.getEnumInput("f.Enum").setValue("YES");
 
-    // Create the connection in repository
-    handler.createConnection(connection, getDerbyConnection());
-    assertNotSame(connection.getPersistenceId(), MPersistableEntity.PERSISTANCE_ID_DEFAULT);
+    // Create the link in repository
+    handler.createLink(link, getDerbyDatabaseConnection());
+    assertNotSame(link.getPersistenceId(), MPersistableEntity.PERSISTANCE_ID_DEFAULT);
 
-    // Retrieve created connection
-    MConnection retrieved = handler.findConnection(connection.getPersistenceId(), getDerbyConnection());
+    // Retrieve created link
+    MLink retrieved = handler.findLink(link.getPersistenceId(), getDerbyDatabaseConnection());
     forms = retrieved.getConnectorPart();
     assertEquals("A", forms.getStringInput("f.String").getValue());
     assertEquals(map, forms.getMapInput("f.Map").getValue());

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestInternals.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestInternals.java b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestInternals.java
index 913439b..37aa8d4 100644
--- a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestInternals.java
+++ b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestInternals.java
@@ -34,25 +34,25 @@ public class TestInternals extends DerbyTestCase {
   }
 
   public void testSuitableInternals() throws Exception {
-    assertFalse(handler.haveSuitableInternals(getDerbyConnection()));
+    assertFalse(handler.haveSuitableInternals(getDerbyDatabaseConnection()));
     createSchema(); // Test code is building the structures
-    assertTrue(handler.haveSuitableInternals(getDerbyConnection()));
+    assertTrue(handler.haveSuitableInternals(getDerbyDatabaseConnection()));
   }
 
   public void testCreateorUpdateInternals() throws Exception {
-    assertFalse(handler.haveSuitableInternals(getDerbyConnection()));
-    handler.createOrUpdateInternals(getDerbyConnection());
-    assertTrue(handler.haveSuitableInternals(getDerbyConnection()));
+    assertFalse(handler.haveSuitableInternals(getDerbyDatabaseConnection()));
+    handler.createOrUpdateInternals(getDerbyDatabaseConnection());
+    assertTrue(handler.haveSuitableInternals(getDerbyDatabaseConnection()));
   }
 
   public void testUpgradeVersion2ToVersion4() throws Exception {
     createSchema(2);
-    assertFalse(handler.haveSuitableInternals(getDerbyConnection()));
-    loadConnectorAndFramework(2);
-    loadConnections(2);
+    assertFalse(handler.haveSuitableInternals(getDerbyDatabaseConnection()));
+    loadConnectorAndDriverConfig(2);
+    loadLinks(2);
     loadJobs(2);
-    handler.createOrUpdateInternals(getDerbyConnection());
-    assertTrue(handler.haveSuitableInternals(getDerbyConnection()));
+    handler.createOrUpdateInternals(getDerbyDatabaseConnection());
+    assertTrue(handler.haveSuitableInternals(getDerbyDatabaseConnection()));
   }
 
   private class TestDerbyRepositoryHandler extends DerbyRepositoryHandler {

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestJobHandling.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestJobHandling.java b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestJobHandling.java
index e658c11..f22c351 100644
--- a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestJobHandling.java
+++ b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestJobHandling.java
@@ -45,16 +45,16 @@ public class TestJobHandling extends DerbyTestCase {
     createSchema();
 
     // We always needs connector and framework structures in place
-    loadConnectorAndFramework();
+    loadConnectorAndDriverConfig();
 
     // We always needs connection metadata in place
-    loadConnections();
+    loadLinks();
   }
 
   public void testFindJob() throws Exception {
     // Let's try to find non existing job
     try {
-      handler.findJob(1, getDerbyConnection());
+      handler.findJob(1, getDerbyDatabaseConnection());
       fail();
     } catch(SqoopException ex) {
       assertEquals(DerbyRepoError.DERBYREPO_0030, ex.getErrorCode());
@@ -63,7 +63,7 @@ public class TestJobHandling extends DerbyTestCase {
     // Load prepared connections into database
     loadJobs();
 
-    MJob jobImport = handler.findJob(1, getDerbyConnection());
+    MJob jobImport = handler.findJob(1, getDerbyDatabaseConnection());
     assertNotNull(jobImport);
     assertEquals(1, jobImport.getPersistenceId());
     assertEquals("JA", jobImport.getName());
@@ -98,13 +98,13 @@ public class TestJobHandling extends DerbyTestCase {
     List<MJob> list;
 
     // Load empty list on empty repository
-    list = handler.findJobs(getDerbyConnection());
+    list = handler.findJobs(getDerbyDatabaseConnection());
     assertEquals(0, list.size());
 
     loadJobs();
 
     // Load all two connections on loaded repository
-    list = handler.findJobs(getDerbyConnection());
+    list = handler.findJobs(getDerbyDatabaseConnection());
     assertEquals(4, list.size());
 
     assertEquals("JA", list.get(0).getName());
@@ -118,29 +118,29 @@ public class TestJobHandling extends DerbyTestCase {
 
   public void testExistsJob() throws Exception {
     // There shouldn't be anything on empty repository
-    assertFalse(handler.existsJob(1, getDerbyConnection()));
-    assertFalse(handler.existsJob(2, getDerbyConnection()));
-    assertFalse(handler.existsJob(3, getDerbyConnection()));
-    assertFalse(handler.existsJob(4, getDerbyConnection()));
-    assertFalse(handler.existsJob(5, getDerbyConnection()));
+    assertFalse(handler.existsJob(1, getDerbyDatabaseConnection()));
+    assertFalse(handler.existsJob(2, getDerbyDatabaseConnection()));
+    assertFalse(handler.existsJob(3, getDerbyDatabaseConnection()));
+    assertFalse(handler.existsJob(4, getDerbyDatabaseConnection()));
+    assertFalse(handler.existsJob(5, getDerbyDatabaseConnection()));
 
     loadJobs();
 
-    assertTrue(handler.existsJob(1, getDerbyConnection()));
-    assertTrue(handler.existsJob(2, getDerbyConnection()));
-    assertTrue(handler.existsJob(3, getDerbyConnection()));
-    assertTrue(handler.existsJob(4, getDerbyConnection()));
-    assertFalse(handler.existsJob(5, getDerbyConnection()));
+    assertTrue(handler.existsJob(1, getDerbyDatabaseConnection()));
+    assertTrue(handler.existsJob(2, getDerbyDatabaseConnection()));
+    assertTrue(handler.existsJob(3, getDerbyDatabaseConnection()));
+    assertTrue(handler.existsJob(4, getDerbyDatabaseConnection()));
+    assertFalse(handler.existsJob(5, getDerbyDatabaseConnection()));
   }
 
   public void testInUseJob() throws Exception {
     loadJobs();
     loadSubmissions();
 
-    assertTrue(handler.inUseJob(1, getDerbyConnection()));
-    assertFalse(handler.inUseJob(2, getDerbyConnection()));
-    assertFalse(handler.inUseJob(3, getDerbyConnection()));
-    assertFalse(handler.inUseJob(4, getDerbyConnection()));
+    assertTrue(handler.inUseJob(1, getDerbyDatabaseConnection()));
+    assertFalse(handler.inUseJob(2, getDerbyDatabaseConnection()));
+    assertFalse(handler.inUseJob(3, getDerbyDatabaseConnection()));
+    assertFalse(handler.inUseJob(4, getDerbyDatabaseConnection()));
   }
 
   public void testCreateJob() throws Exception {
@@ -149,13 +149,13 @@ public class TestJobHandling extends DerbyTestCase {
     // Load some data
     fillJob(job);
 
-    handler.createJob(job, getDerbyConnection());
+    handler.createJob(job, getDerbyDatabaseConnection());
 
     assertEquals(1, job.getPersistenceId());
     assertCountForTable("SQOOP.SQ_JOB", 1);
     assertCountForTable("SQOOP.SQ_JOB_INPUT", 6);
 
-    MJob retrieved = handler.findJob(1, getDerbyConnection());
+    MJob retrieved = handler.findJob(1, getDerbyDatabaseConnection());
     assertEquals(1, retrieved.getPersistenceId());
 
     List<MForm> forms;
@@ -176,7 +176,7 @@ public class TestJobHandling extends DerbyTestCase {
     job = getJob();
     fillJob(job);
 
-    handler.createJob(job, getDerbyConnection());
+    handler.createJob(job, getDerbyDatabaseConnection());
 
     assertEquals(2, job.getPersistenceId());
     assertCountForTable("SQOOP.SQ_JOB", 2);
@@ -189,7 +189,7 @@ public class TestJobHandling extends DerbyTestCase {
     assertCountForTable("SQOOP.SQ_JOB", 4);
     assertCountForTable("SQOOP.SQ_JOB_INPUT", 24);
 
-    MJob job = handler.findJob(1, getDerbyConnection());
+    MJob job = handler.findJob(1, getDerbyDatabaseConnection());
 
     List<MForm> forms;
 
@@ -208,13 +208,13 @@ public class TestJobHandling extends DerbyTestCase {
 
     job.setName("name");
 
-    handler.updateJob(job, getDerbyConnection());
+    handler.updateJob(job, getDerbyDatabaseConnection());
 
     assertEquals(1, job.getPersistenceId());
     assertCountForTable("SQOOP.SQ_JOB", 4);
     assertCountForTable("SQOOP.SQ_JOB_INPUT", 26);
 
-    MJob retrieved = handler.findJob(1, getDerbyConnection());
+    MJob retrieved = handler.findJob(1, getDerbyDatabaseConnection());
     assertEquals("name", retrieved.getName());
 
     forms = job.getConnectorPart(Direction.FROM).getForms();
@@ -237,16 +237,16 @@ public class TestJobHandling extends DerbyTestCase {
     loadJobs();
 
     // disable job 1
-    handler.enableJob(1, false, getDerbyConnection());
+    handler.enableJob(1, false, getDerbyDatabaseConnection());
 
-    MJob retrieved = handler.findJob(1, getDerbyConnection());
+    MJob retrieved = handler.findJob(1, getDerbyDatabaseConnection());
     assertNotNull(retrieved);
     assertEquals(false, retrieved.getEnabled());
 
     // enable job 1
-    handler.enableJob(1, true, getDerbyConnection());
+    handler.enableJob(1, true, getDerbyDatabaseConnection());
 
-    retrieved = handler.findJob(1, getDerbyConnection());
+    retrieved = handler.findJob(1, getDerbyDatabaseConnection());
     assertNotNull(retrieved);
     assertEquals(true, retrieved.getEnabled());
   }
@@ -254,28 +254,28 @@ public class TestJobHandling extends DerbyTestCase {
   public void testDeleteJob() throws Exception {
     loadJobs();
 
-    handler.deleteJob(1, getDerbyConnection());
+    handler.deleteJob(1, getDerbyDatabaseConnection());
     assertCountForTable("SQOOP.SQ_JOB", 3);
     assertCountForTable("SQOOP.SQ_JOB_INPUT", 18);
 
-    handler.deleteJob(2, getDerbyConnection());
+    handler.deleteJob(2, getDerbyDatabaseConnection());
     assertCountForTable("SQOOP.SQ_JOB", 2);
     assertCountForTable("SQOOP.SQ_JOB_INPUT", 12);
 
-    handler.deleteJob(3, getDerbyConnection());
+    handler.deleteJob(3, getDerbyDatabaseConnection());
     assertCountForTable("SQOOP.SQ_JOB", 1);
     assertCountForTable("SQOOP.SQ_JOB_INPUT", 6);
 
-    handler.deleteJob(4, getDerbyConnection());
+    handler.deleteJob(4, getDerbyDatabaseConnection());
     assertCountForTable("SQOOP.SQ_JOB", 0);
     assertCountForTable("SQOOP.SQ_JOB_INPUT", 0);
   }
 
   public MJob getJob() {
     return new MJob(1, 1, 1, 1,
-      handler.findConnector("A", getDerbyConnection()).getJobForms(Direction.FROM),
-      handler.findConnector("A", getDerbyConnection()).getJobForms(Direction.TO),
-      handler.findFramework(getDerbyConnection()).getJobForms()
+      handler.findConnector("A", getDerbyDatabaseConnection()).getJobForms(Direction.FROM),
+      handler.findConnector("A", getDerbyDatabaseConnection()).getJobForms(Direction.TO),
+      handler.findDriverConfig(getDerbyDatabaseConnection()).getJobForms()
     );
   }
 }