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:39 UTC

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

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestLinkHandling.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestLinkHandling.java b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestLinkHandling.java
new file mode 100644
index 0000000..7378eda
--- /dev/null
+++ b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestLinkHandling.java
@@ -0,0 +1,244 @@
+/**
+ * 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.MLink;
+import org.apache.sqoop.model.MForm;
+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 link methods on Derby repository.
+ */
+public class TestLinkHandling 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
+    loadConnectorAndDriverConfig();
+  }
+
+  public void testFindLink() throws Exception {
+    // Let's try to find non existing link
+    try {
+      handler.findLink(1, getDerbyDatabaseConnection());
+      fail();
+    } catch(SqoopException ex) {
+      assertEquals(DerbyRepoError.DERBYREPO_0024, ex.getErrorCode());
+    }
+
+    // Load prepared connections into database
+    loadLinks();
+
+    MLink connA = handler.findLink(1, getDerbyDatabaseConnection());
+    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 testFindLinks() throws Exception {
+    List<MLink> list;
+
+    // Load empty list on empty repository
+    list = handler.findLinks(getDerbyDatabaseConnection());
+    assertEquals(0, list.size());
+
+    loadLinks();
+
+    // Load all two connections on loaded repository
+    list = handler.findLinks(getDerbyDatabaseConnection());
+    assertEquals(2, list.size());
+
+    assertEquals("CA", list.get(0).getName());
+    assertEquals("CB", list.get(1).getName());
+  }
+
+  public void testExistsLink() throws Exception {
+    // There shouldn't be anything on empty repository
+    assertFalse(handler.existsLink(1, getDerbyDatabaseConnection()));
+    assertFalse(handler.existsLink(2, getDerbyDatabaseConnection()));
+    assertFalse(handler.existsLink(3, getDerbyDatabaseConnection()));
+
+    loadLinks();
+
+    assertTrue(handler.existsLink(1, getDerbyDatabaseConnection()));
+    assertTrue(handler.existsLink(2, getDerbyDatabaseConnection()));
+    assertFalse(handler.existsLink(3, getDerbyDatabaseConnection()));
+  }
+
+  public void testCreateLink() throws Exception {
+    MLink link = getLink();
+
+    // Load some data
+    fillLink(link);
+
+    handler.createLink(link, getDerbyDatabaseConnection());
+
+    assertEquals(1, link.getPersistenceId());
+    assertCountForTable("SQOOP.SQ_CONNECTION", 1);
+    assertCountForTable("SQOOP.SQ_CONNECTION_INPUT", 4);
+
+    MLink retrieved = handler.findLink(1, getDerbyDatabaseConnection());
+    assertEquals(1, retrieved.getPersistenceId());
+
+    List<MForm> forms;
+    forms = link.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 = link.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 link
+    link = getLink();
+    fillLink(link);
+
+    handler.createLink(link, getDerbyDatabaseConnection());
+
+    assertEquals(2, link.getPersistenceId());
+    assertCountForTable("SQOOP.SQ_CONNECTION", 2);
+    assertCountForTable("SQOOP.SQ_CONNECTION_INPUT", 8);
+  }
+
+  public void testInUseLink() throws Exception {
+    loadLinks();
+
+    assertFalse(handler.inUseLink(1, getDerbyDatabaseConnection()));
+
+    loadJobs();
+
+    assertTrue(handler.inUseLink(1, getDerbyDatabaseConnection()));
+  }
+
+  public void testUpdateLink() throws Exception {
+    loadLinks();
+
+    MLink link = handler.findLink(1, getDerbyDatabaseConnection());
+
+    List<MForm> forms;
+
+    forms = link.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 = link.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
+
+    link.setName("name");
+
+    handler.updateLink(link, getDerbyDatabaseConnection());
+
+    assertEquals(1, link.getPersistenceId());
+    assertCountForTable("SQOOP.SQ_CONNECTION", 2);
+    assertCountForTable("SQOOP.SQ_CONNECTION_INPUT", 10);
+
+    MLink retrieved = handler.findLink(1, getDerbyDatabaseConnection());
+    assertEquals("name", link.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 testEnableAndDisableLink() throws Exception {
+    loadLinks();
+
+    // disable link 1
+    handler.enableLink(1, false, getDerbyDatabaseConnection());
+
+    MLink retrieved = handler.findLink(1, getDerbyDatabaseConnection());
+    assertNotNull(retrieved);
+    assertEquals(false, retrieved.getEnabled());
+
+    // enable link 1
+    handler.enableLink(1, true, getDerbyDatabaseConnection());
+
+    retrieved = handler.findLink(1, getDerbyDatabaseConnection());
+    assertNotNull(retrieved);
+    assertEquals(true, retrieved.getEnabled());
+  }
+
+  public void testDeleteLink() throws Exception {
+    loadLinks();
+
+    handler.deleteLink(1, getDerbyDatabaseConnection());
+    assertCountForTable("SQOOP.SQ_CONNECTION", 1);
+    assertCountForTable("SQOOP.SQ_CONNECTION_INPUT", 4);
+
+    handler.deleteLink(2, getDerbyDatabaseConnection());
+    assertCountForTable("SQOOP.SQ_CONNECTION", 0);
+    assertCountForTable("SQOOP.SQ_CONNECTION_INPUT", 0);
+  }
+
+  public MLink getLink() {
+    return new MLink(1,
+      handler.findConnector("A", getDerbyDatabaseConnection()).getConnectionForms(),
+      handler.findDriverConfig(getDerbyDatabaseConnection()).getConnectionForms()
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestSubmissionHandling.java
----------------------------------------------------------------------
diff --git a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestSubmissionHandling.java b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestSubmissionHandling.java
index 8fce0dd..38a995f 100644
--- a/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestSubmissionHandling.java
+++ b/repository/repository-derby/src/test/java/org/apache/sqoop/repository/derby/TestSubmissionHandling.java
@@ -44,10 +44,10 @@ public class TestSubmissionHandling extends DerbyTestCase {
     createSchema();
 
     // We always needs connector and framework structures in place
-    loadConnectorAndFramework();
+    loadConnectorAndDriverConfig();
 
     // We also always needs connection metadata in place
-    loadConnections();
+    loadLinks();
 
     // And finally we always needs job metadata in place
     loadJobs();
@@ -56,34 +56,34 @@ public class TestSubmissionHandling extends DerbyTestCase {
   public void testFindSubmissionsUnfinished() throws Exception {
     List<MSubmission> submissions;
 
-    submissions = handler.findSubmissionsUnfinished(getDerbyConnection());
+    submissions = handler.findSubmissionsUnfinished(getDerbyDatabaseConnection());
     assertNotNull(submissions);
     assertEquals(0, submissions.size());
 
     loadSubmissions();
 
-    submissions = handler.findSubmissionsUnfinished(getDerbyConnection());
+    submissions = handler.findSubmissionsUnfinished(getDerbyDatabaseConnection());
     assertNotNull(submissions);
     assertEquals(2, submissions.size());
   }
 
   public void testExistsSubmission() throws Exception {
     // There shouldn't be anything on empty repository
-    assertFalse(handler.existsSubmission(1, getDerbyConnection()));
-    assertFalse(handler.existsSubmission(2, getDerbyConnection()));
-    assertFalse(handler.existsSubmission(3, getDerbyConnection()));
-    assertFalse(handler.existsSubmission(4, getDerbyConnection()));
-    assertFalse(handler.existsSubmission(5, getDerbyConnection()));
-    assertFalse(handler.existsSubmission(6, getDerbyConnection()));
+    assertFalse(handler.existsSubmission(1, getDerbyDatabaseConnection()));
+    assertFalse(handler.existsSubmission(2, getDerbyDatabaseConnection()));
+    assertFalse(handler.existsSubmission(3, getDerbyDatabaseConnection()));
+    assertFalse(handler.existsSubmission(4, getDerbyDatabaseConnection()));
+    assertFalse(handler.existsSubmission(5, getDerbyDatabaseConnection()));
+    assertFalse(handler.existsSubmission(6, getDerbyDatabaseConnection()));
 
     loadSubmissions();
 
-    assertTrue(handler.existsSubmission(1, getDerbyConnection()));
-    assertTrue(handler.existsSubmission(2, getDerbyConnection()));
-    assertTrue(handler.existsSubmission(3, getDerbyConnection()));
-    assertTrue(handler.existsSubmission(4, getDerbyConnection()));
-    assertTrue(handler.existsSubmission(5, getDerbyConnection()));
-    assertFalse(handler.existsSubmission(6, getDerbyConnection()));
+    assertTrue(handler.existsSubmission(1, getDerbyDatabaseConnection()));
+    assertTrue(handler.existsSubmission(2, getDerbyDatabaseConnection()));
+    assertTrue(handler.existsSubmission(3, getDerbyDatabaseConnection()));
+    assertTrue(handler.existsSubmission(4, getDerbyDatabaseConnection()));
+    assertTrue(handler.existsSubmission(5, getDerbyDatabaseConnection()));
+    assertFalse(handler.existsSubmission(6, getDerbyDatabaseConnection()));
   }
 
   public void testCreateSubmission() throws Exception {
@@ -111,13 +111,13 @@ public class TestSubmissionHandling extends DerbyTestCase {
     submission.setExceptionStackTrace("Yeah it happens");
     submission.setCounters(counters);
 
-    handler.createSubmission(submission, getDerbyConnection());
+    handler.createSubmission(submission, getDerbyDatabaseConnection());
 
     assertEquals(1, submission.getPersistenceId());
     assertCountForTable("SQOOP.SQ_SUBMISSION", 1);
 
     List<MSubmission> submissions =
-      handler.findSubmissionsUnfinished(getDerbyConnection());
+      handler.findSubmissionsUnfinished(getDerbyDatabaseConnection());
     assertNotNull(submissions);
     assertEquals(1, submissions.size());
 
@@ -162,7 +162,7 @@ public class TestSubmissionHandling extends DerbyTestCase {
     // Let's create second (simpler) connection
     submission =
       new MSubmission(1, new Date(), SubmissionStatus.SUCCEEDED, "job-x");
-    handler.createSubmission(submission, getDerbyConnection());
+    handler.createSubmission(submission, getDerbyDatabaseConnection());
 
     assertEquals(2, submission.getPersistenceId());
     assertCountForTable("SQOOP.SQ_SUBMISSION", 2);
@@ -172,16 +172,16 @@ public class TestSubmissionHandling extends DerbyTestCase {
     loadSubmissions();
 
     List<MSubmission> submissions =
-      handler.findSubmissionsUnfinished(getDerbyConnection());
+      handler.findSubmissionsUnfinished(getDerbyDatabaseConnection());
     assertNotNull(submissions);
     assertEquals(2, submissions.size());
 
     MSubmission submission = submissions.get(0);
     submission.setStatus(SubmissionStatus.SUCCEEDED);
 
-    handler.updateSubmission(submission, getDerbyConnection());
+    handler.updateSubmission(submission, getDerbyDatabaseConnection());
 
-    submissions = handler.findSubmissionsUnfinished(getDerbyConnection());
+    submissions = handler.findSubmissionsUnfinished(getDerbyDatabaseConnection());
     assertNotNull(submissions);
     assertEquals(1, submissions.size());
   }
@@ -190,7 +190,7 @@ public class TestSubmissionHandling extends DerbyTestCase {
     loadSubmissions();
     List<MSubmission> submissions;
 
-    submissions = handler.findSubmissionsUnfinished(getDerbyConnection());
+    submissions = handler.findSubmissionsUnfinished(getDerbyDatabaseConnection());
     assertNotNull(submissions);
     assertEquals(2, submissions.size());
     assertCountForTable("SQOOP.SQ_SUBMISSION", 5);
@@ -198,23 +198,23 @@ public class TestSubmissionHandling extends DerbyTestCase {
     Calendar calendar = Calendar.getInstance();
     // 2012-01-03 05:05:05
     calendar.set(2012, Calendar.JANUARY, 3, 5, 5, 5);
-    handler.purgeSubmissions(calendar.getTime(), getDerbyConnection());
+    handler.purgeSubmissions(calendar.getTime(), getDerbyDatabaseConnection());
 
-    submissions = handler.findSubmissionsUnfinished(getDerbyConnection());
+    submissions = handler.findSubmissionsUnfinished(getDerbyDatabaseConnection());
     assertNotNull(submissions);
     assertEquals(1, submissions.size());
     assertCountForTable("SQOOP.SQ_SUBMISSION", 2);
 
-    handler.purgeSubmissions(new Date(), getDerbyConnection());
+    handler.purgeSubmissions(new Date(), getDerbyDatabaseConnection());
 
-    submissions = handler.findSubmissionsUnfinished(getDerbyConnection());
+    submissions = handler.findSubmissionsUnfinished(getDerbyDatabaseConnection());
     assertNotNull(submissions);
     assertEquals(0, submissions.size());
     assertCountForTable("SQOOP.SQ_SUBMISSION", 0);
 
-    handler.purgeSubmissions(new Date(), getDerbyConnection());
+    handler.purgeSubmissions(new Date(), getDerbyDatabaseConnection());
 
-    submissions = handler.findSubmissionsUnfinished(getDerbyConnection());
+    submissions = handler.findSubmissionsUnfinished(getDerbyDatabaseConnection());
     assertNotNull(submissions);
     assertEquals(0, submissions.size());
     assertCountForTable("SQOOP.SQ_SUBMISSION", 0);
@@ -230,16 +230,16 @@ public class TestSubmissionHandling extends DerbyTestCase {
     loadSubmissions();
     assertCountForTable("SQOOP.SQ_SUBMISSION", 5);
 
-    handler.deleteJob(1, getDerbyConnection());
+    handler.deleteJob(1, getDerbyDatabaseConnection());
     assertCountForTable("SQOOP.SQ_SUBMISSION", 3);
 
-    handler.deleteJob(2, getDerbyConnection());
+    handler.deleteJob(2, getDerbyDatabaseConnection());
     assertCountForTable("SQOOP.SQ_SUBMISSION", 2);
 
-    handler.deleteJob(3, getDerbyConnection());
+    handler.deleteJob(3, getDerbyDatabaseConnection());
     assertCountForTable("SQOOP.SQ_SUBMISSION", 1);
 
-    handler.deleteJob(4, getDerbyConnection());
+    handler.deleteJob(4, getDerbyDatabaseConnection());
     assertCountForTable("SQOOP.SQ_SUBMISSION", 0);
   }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/server/src/main/java/org/apache/sqoop/handler/ConnectionRequestHandler.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/handler/ConnectionRequestHandler.java b/server/src/main/java/org/apache/sqoop/handler/ConnectionRequestHandler.java
deleted file mode 100644
index 89ad3ec..0000000
--- a/server/src/main/java/org/apache/sqoop/handler/ConnectionRequestHandler.java
+++ /dev/null
@@ -1,274 +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.handler;
-
-import org.apache.log4j.Logger;
-import org.apache.sqoop.audit.AuditLoggerManager;
-import org.apache.sqoop.common.SqoopException;
-import org.apache.sqoop.connector.ConnectorManager;
-import org.apache.sqoop.connector.spi.SqoopConnector;
-import org.apache.sqoop.framework.FrameworkManager;
-import org.apache.sqoop.json.ConnectionBean;
-import org.apache.sqoop.json.JsonBean;
-import org.apache.sqoop.json.ValidationResultBean;
-import org.apache.sqoop.model.FormUtils;
-import org.apache.sqoop.model.MConnection;
-import org.apache.sqoop.model.MConnectionForms;
-import org.apache.sqoop.repository.Repository;
-import org.apache.sqoop.repository.RepositoryManager;
-import org.apache.sqoop.server.RequestContext;
-import org.apache.sqoop.server.RequestHandler;
-import org.apache.sqoop.server.common.ServerError;
-import org.apache.sqoop.utils.ClassUtils;
-import org.apache.sqoop.validation.Status;
-import org.apache.sqoop.validation.ValidationResult;
-import org.apache.sqoop.validation.ValidationRunner;
-import org.json.simple.JSONObject;
-import org.json.simple.JSONValue;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Locale;
-
-/**
- * Connection request handler is supporting following resources:
- *
- * GET /v1/connection/:xid
- * Return details about one particular connection with id :xid or about all of
- * them if :xid equals to "all".
- *
- * POST /v1/connection
- * Create new connection
- *
- * PUT /v1/connection/:xid
- * Update connection with id :xid.
- *
- * PUT /v1/connection/:xid/enable
- * Enable connection with id :xid
- *
- * PUT /v1/connection/:xid/disable
- * Disable connection with id :xid
- *
- * DELETE /v1/connection/:xid
- * Remove connection with id :xid
- *
- * Planned resources:
- *
- * GET /v1/connection
- * Get brief list of all connections present in the system.
- *
- */
-public class ConnectionRequestHandler implements RequestHandler {
-
-  private static final Logger LOG =
-      Logger.getLogger(ConnectionRequestHandler.class);
-
-  private static final String ENABLE = "enable";
-  private static final String DISABLE = "disable";
-
-  public ConnectionRequestHandler() {
-    LOG.info("ConnectionRequestHandler initialized");
-  }
-
-  @Override
-  public JsonBean handleEvent(RequestContext ctx) {
-    switch (ctx.getMethod()) {
-      case GET:
-        return getConnections(ctx);
-      case POST:
-          return createUpdateConnection(ctx, false);
-      case PUT:
-        if (ctx.getLastURLElement().equals(ENABLE)) {
-          return enableConnection(ctx, true);
-        } else if (ctx.getLastURLElement().equals(DISABLE)) {
-          return enableConnection(ctx, false);
-        } else {
-          return createUpdateConnection(ctx, true);
-        }
-      case DELETE:
-        return deleteConnection(ctx);
-    }
-
-    return null;
-  }
-
-  /**
-   * Delete connection from metadata repository.
-   *
-   * @param ctx Context object
-   * @return Empty bean
-   */
-  private JsonBean deleteConnection(RequestContext ctx) {
-    String sxid = ctx.getLastURLElement();
-    long xid = Long.valueOf(sxid);
-
-    AuditLoggerManager.getInstance()
-        .logAuditEvent(ctx.getUserName(), ctx.getRequest().getRemoteAddr(),
-        "delete", "connection", sxid);
-
-    Repository repository = RepositoryManager.getInstance().getRepository();
-    repository.deleteConnection(xid);
-
-    return JsonBean.EMPTY_BEAN;
-  }
-
-  /**
-   * Update or create connection metadata in repository.
-   *
-   * @param ctx Context object
-   * @return Validation bean object
-   */
-  private JsonBean createUpdateConnection(RequestContext ctx, boolean update) {
-//    Check that given ID equals with sent ID, otherwise report an error UPDATE
-//    String sxid = ctx.getLastURLElement();
-//    long xid = Long.valueOf(sxid);
-
-    String username = ctx.getUserName();
-
-    ConnectionBean bean = new ConnectionBean();
-
-    try {
-      JSONObject json =
-        (JSONObject) JSONValue.parse(ctx.getRequest().getReader());
-
-      bean.restore(json);
-    } catch (IOException e) {
-      throw new SqoopException(ServerError.SERVER_0003,
-        "Can't read request content", e);
-    }
-
-    // Get connection object
-    List<MConnection> connections = bean.getConnections();
-
-    if(connections.size() != 1) {
-      throw new SqoopException(ServerError.SERVER_0003,
-        "Expected one connection metadata but got " + connections.size());
-    }
-
-    MConnection connection = connections.get(0);
-
-    // Verify that user is not trying to spoof us
-    MConnectionForms connectorForms =
-      ConnectorManager.getInstance().getConnectorMetadata(connection.getConnectorId())
-      .getConnectionForms();
-    MConnectionForms frameworkForms = FrameworkManager.getInstance().getFramework()
-      .getConnectionForms();
-
-    if(!connectorForms.equals(connection.getConnectorPart())
-      || !frameworkForms.equals(connection.getFrameworkPart())) {
-      throw new SqoopException(ServerError.SERVER_0003,
-        "Detected incorrect form structure");
-    }
-
-    // Responsible connector for this session
-    SqoopConnector connector = ConnectorManager.getInstance().getConnector(connection.getConnectorId());
-
-    // We need translate forms to configuration objects
-    Object connectorConfig = ClassUtils.instantiate(connector.getConnectionConfigurationClass());
-    Object frameworkConfig = ClassUtils.instantiate(FrameworkManager.getInstance().getConnectionConfigurationClass());
-
-    FormUtils.fromForms(connection.getConnectorPart().getForms(), connectorConfig);
-    FormUtils.fromForms(connection.getFrameworkPart().getForms(), frameworkConfig);
-
-    // Validate both parts
-    ValidationRunner validationRunner = new ValidationRunner();
-    ValidationResult connectorValidation = validationRunner.validate(connectorConfig);
-    ValidationResult frameworkValidation = validationRunner.validate(frameworkConfig);
-
-    Status finalStatus = Status.getWorstStatus(connectorValidation.getStatus(), frameworkValidation.getStatus());
-
-    // Return back validations in all cases
-    ValidationResultBean outputBean = new ValidationResultBean(connectorValidation, frameworkValidation);
-
-    // If we're good enough let's perform the action
-    if(finalStatus.canProceed()) {
-      if(update) {
-        AuditLoggerManager.getInstance()
-            .logAuditEvent(ctx.getUserName(), ctx.getRequest().getRemoteAddr(),
-            "update", "connection", String.valueOf(connection.getPersistenceId()));
-
-        connection.setLastUpdateUser(username);
-        RepositoryManager.getInstance().getRepository().updateConnection(connection);
-      } else {
-        connection.setCreationUser(username);
-        connection.setLastUpdateUser(username);
-        RepositoryManager.getInstance().getRepository().createConnection(connection);
-        outputBean.setId(connection.getPersistenceId());
-
-        AuditLoggerManager.getInstance()
-            .logAuditEvent(ctx.getUserName(), ctx.getRequest().getRemoteAddr(),
-            "create", "connection", String.valueOf(connection.getPersistenceId()));
-      }
-    }
-
-    return outputBean;
-  }
-
-  private JsonBean getConnections(RequestContext ctx) {
-    String sxid = ctx.getLastURLElement();
-    ConnectionBean bean;
-
-    AuditLoggerManager.getInstance()
-        .logAuditEvent(ctx.getUserName(), ctx.getRequest().getRemoteAddr(),
-        "get", "connection", sxid);
-
-    Locale locale = ctx.getAcceptLanguageHeader();
-    Repository repository = RepositoryManager.getInstance().getRepository();
-
-    if (sxid.equals("all")) {
-
-      List<MConnection> connections = repository.findConnections();
-      bean = new ConnectionBean(connections);
-
-      // Add associated resources into the bean
-      for( MConnection connection : connections) {
-        long connectorId = connection.getConnectorId();
-        if(!bean.hasConnectorBundle(connectorId)) {
-          bean.addConnectorBundle(connectorId,
-            ConnectorManager.getInstance().getResourceBundle(connectorId, locale));
-        }
-      }
-    } else {
-      long xid = Long.valueOf(sxid);
-
-      MConnection connection = repository.findConnection(xid);
-      long connectorId = connection.getConnectorId();
-
-      bean = new ConnectionBean(connection);
-
-      bean.addConnectorBundle(connectorId,
-        ConnectorManager.getInstance().getResourceBundle(connectorId, locale));
-    }
-
-    // Sent framework resource bundle in all cases
-    bean.setFrameworkBundle(FrameworkManager.getInstance().getBundle(locale));
-
-    return bean;
-  }
-
-  private JsonBean enableConnection(RequestContext ctx, boolean enabled) {
-    String[] elements = ctx.getUrlElements();
-    String sxid = elements[elements.length - 2];
-    long xid = Long.valueOf(sxid);
-
-    Repository repository = RepositoryManager.getInstance().getRepository();
-    repository.enableConnection(xid, enabled);
-
-    return JsonBean.EMPTY_BEAN;
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/server/src/main/java/org/apache/sqoop/handler/DriverConfigRequestHandler.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/handler/DriverConfigRequestHandler.java b/server/src/main/java/org/apache/sqoop/handler/DriverConfigRequestHandler.java
new file mode 100644
index 0000000..c0d363e
--- /dev/null
+++ b/server/src/main/java/org/apache/sqoop/handler/DriverConfigRequestHandler.java
@@ -0,0 +1,50 @@
+/**
+ * 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.handler;
+
+import org.apache.log4j.Logger;
+import org.apache.sqoop.audit.AuditLoggerManager;
+import org.apache.sqoop.driver.Driver;
+import org.apache.sqoop.json.DriverConfigBean;
+import org.apache.sqoop.json.JsonBean;
+import org.apache.sqoop.server.RequestContext;
+import org.apache.sqoop.server.RequestHandler;
+
+/**
+ * Driver Config request handler is supporting following resources:
+ *
+ */
+public class DriverConfigRequestHandler  implements RequestHandler {
+
+  private static final Logger LOG =
+      Logger.getLogger(DriverConfigRequestHandler.class);
+
+  public DriverConfigRequestHandler() {
+    LOG.info("DriverConfigRequestHandler initialized");
+  }
+
+  @Override
+  public JsonBean handleEvent(RequestContext ctx) {
+    AuditLoggerManager.getInstance()
+        .logAuditEvent(ctx.getUserName(), ctx.getRequest().getRemoteAddr(),
+        "get", "framework", "");
+
+    return new DriverConfigBean(Driver.getInstance().getDriverConfig(),
+      Driver.getInstance().getBundle(ctx.getAcceptLanguageHeader()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/server/src/main/java/org/apache/sqoop/handler/FrameworkRequestHandler.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/handler/FrameworkRequestHandler.java b/server/src/main/java/org/apache/sqoop/handler/FrameworkRequestHandler.java
deleted file mode 100644
index 616e974..0000000
--- a/server/src/main/java/org/apache/sqoop/handler/FrameworkRequestHandler.java
+++ /dev/null
@@ -1,53 +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.handler;
-
-import org.apache.log4j.Logger;
-
-import org.apache.sqoop.audit.AuditLoggerManager;
-import org.apache.sqoop.framework.FrameworkManager;
-import org.apache.sqoop.json.FrameworkBean;
-import org.apache.sqoop.json.JsonBean;
-import org.apache.sqoop.server.RequestContext;
-import org.apache.sqoop.server.RequestHandler;
-
-/**
- * Framework request handler is supporting following resources:
- *
- * GET /v1/framework
- * Get framework metadata.
- */
-public class FrameworkRequestHandler  implements RequestHandler {
-
-  private static final Logger LOG =
-      Logger.getLogger(FrameworkRequestHandler.class);
-
-  public FrameworkRequestHandler() {
-    LOG.info("FrameworkRequestHandler initialized");
-  }
-
-  @Override
-  public JsonBean handleEvent(RequestContext ctx) {
-    AuditLoggerManager.getInstance()
-        .logAuditEvent(ctx.getUserName(), ctx.getRequest().getRemoteAddr(),
-        "get", "framework", "");
-
-    return new FrameworkBean(FrameworkManager.getInstance().getFramework(),
-      FrameworkManager.getInstance().getBundle(ctx.getAcceptLanguageHeader()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/server/src/main/java/org/apache/sqoop/handler/JobRequestHandler.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/handler/JobRequestHandler.java b/server/src/main/java/org/apache/sqoop/handler/JobRequestHandler.java
index e2fc86c..b61d3f5 100644
--- a/server/src/main/java/org/apache/sqoop/handler/JobRequestHandler.java
+++ b/server/src/main/java/org/apache/sqoop/handler/JobRequestHandler.java
@@ -23,7 +23,7 @@ import org.apache.sqoop.common.Direction;
 import org.apache.sqoop.common.SqoopException;
 import org.apache.sqoop.connector.ConnectorManager;
 import org.apache.sqoop.connector.spi.SqoopConnector;
-import org.apache.sqoop.framework.FrameworkManager;
+import org.apache.sqoop.driver.Driver;
 import org.apache.sqoop.json.JobBean;
 import org.apache.sqoop.json.JsonBean;
 import org.apache.sqoop.json.ValidationResultBean;
@@ -170,7 +170,7 @@ public class JobRequestHandler implements RequestHandler {
     MJobForms toConnectorForms = ConnectorManager.getInstance()
         .getConnectorMetadata(job.getConnectorId(Direction.TO))
         .getJobForms(Direction.TO);
-    MJobForms frameworkForms = FrameworkManager.getInstance().getFramework()
+    MJobForms frameworkForms = Driver.getInstance().getDriverConfig()
       .getJobForms();
 
     if(!fromConnectorForms.equals(job.getConnectorPart(Direction.FROM))
@@ -196,7 +196,7 @@ public class JobRequestHandler implements RequestHandler {
 
     // We need translate forms to configuration objects
     Object fromConnectorConfig = ClassUtils.instantiate(fromConnector.getJobConfigurationClass(Direction.FROM));
-    Object frameworkConfig = ClassUtils.instantiate(FrameworkManager.getInstance().getJobConfigurationClass());
+    Object frameworkConfig = ClassUtils.instantiate(Driver.getInstance().getJobConfigurationClass());
     Object toConnectorConfig = ClassUtils.instantiate(toConnector.getJobConfigurationClass(Direction.TO));
 
     FormUtils.fromForms(job.getConnectorPart(Direction.FROM).getForms(), fromConnectorConfig);
@@ -259,8 +259,8 @@ public class JobRequestHandler implements RequestHandler {
       // @TODO(Abe): From/To.
       for( MJob job : jobs) {
         long connectorId = job.getConnectorId(Direction.FROM);
-        if(!bean.hasConnectorBundle(connectorId)) {
-          bean.addConnectorBundle(connectorId,
+        if(!bean.hasConnectorConfigBundle(connectorId)) {
+          bean.addConnectorConfigBundle(connectorId,
             ConnectorManager.getInstance().getResourceBundle(connectorId, locale));
         }
       }
@@ -274,12 +274,12 @@ public class JobRequestHandler implements RequestHandler {
 
       bean = new JobBean(job);
 
-      bean.addConnectorBundle(connectorId,
+      bean.addConnectorConfigBundle(connectorId,
         ConnectorManager.getInstance().getResourceBundle(connectorId, locale));
     }
 
     // Sent framework resource bundle in all cases
-    bean.setFrameworkBundle(FrameworkManager.getInstance().getBundle(locale));
+    bean.setDriverConfigBundle(Driver.getInstance().getBundle(locale));
 
     return bean;
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/server/src/main/java/org/apache/sqoop/handler/LinkRequestHandler.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/handler/LinkRequestHandler.java b/server/src/main/java/org/apache/sqoop/handler/LinkRequestHandler.java
new file mode 100644
index 0000000..23fc9f1
--- /dev/null
+++ b/server/src/main/java/org/apache/sqoop/handler/LinkRequestHandler.java
@@ -0,0 +1,271 @@
+/**
+ * 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.handler;
+
+import org.apache.log4j.Logger;
+import org.apache.sqoop.audit.AuditLoggerManager;
+import org.apache.sqoop.common.SqoopException;
+import org.apache.sqoop.connector.ConnectorManager;
+import org.apache.sqoop.connector.spi.SqoopConnector;
+import org.apache.sqoop.driver.Driver;
+import org.apache.sqoop.json.LinkBean;
+import org.apache.sqoop.json.JsonBean;
+import org.apache.sqoop.json.ValidationResultBean;
+import org.apache.sqoop.model.FormUtils;
+import org.apache.sqoop.model.MLink;
+import org.apache.sqoop.model.MConnectionForms;
+import org.apache.sqoop.repository.Repository;
+import org.apache.sqoop.repository.RepositoryManager;
+import org.apache.sqoop.server.RequestContext;
+import org.apache.sqoop.server.RequestHandler;
+import org.apache.sqoop.server.common.ServerError;
+import org.apache.sqoop.utils.ClassUtils;
+import org.apache.sqoop.validation.Status;
+import org.apache.sqoop.validation.ValidationResult;
+import org.apache.sqoop.validation.ValidationRunner;
+import org.json.simple.JSONObject;
+import org.json.simple.JSONValue;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Locale;
+
+/**
+ * Connection request handler is supporting following resources:
+ *
+ * GET /v1/connection/:xid
+ * Return details about one particular connection with id :xid or about all of
+ * them if :xid equals to "all".
+ *
+ * POST /v1/connection
+ * Create new connection
+ *
+ * PUT /v1/connection/:xid
+ * Update connection with id :xid.
+ *
+ * PUT /v1/connection/:xid/enable
+ * Enable connection with id :xid
+ *
+ * PUT /v1/connection/:xid/disable
+ * Disable connection with id :xid
+ *
+ * DELETE /v1/connection/:xid
+ * Remove connection with id :xid
+ *
+ * Planned resources:
+ *
+ * GET /v1/connection
+ * Get brief list of all connections present in the system.
+ *
+ */
+public class LinkRequestHandler implements RequestHandler {
+
+  private static final Logger LOG =
+      Logger.getLogger(LinkRequestHandler.class);
+
+  private static final String ENABLE = "enable";
+  private static final String DISABLE = "disable";
+
+  public LinkRequestHandler() {
+    LOG.info("LinkRequestHandler initialized");
+  }
+
+  @Override
+  public JsonBean handleEvent(RequestContext ctx) {
+    switch (ctx.getMethod()) {
+      case GET:
+        return getConnections(ctx);
+      case POST:
+          return createUpdateConnection(ctx, false);
+      case PUT:
+        if (ctx.getLastURLElement().equals(ENABLE)) {
+          return enableConnection(ctx, true);
+        } else if (ctx.getLastURLElement().equals(DISABLE)) {
+          return enableConnection(ctx, false);
+        } else {
+          return createUpdateConnection(ctx, true);
+        }
+      case DELETE:
+        return deleteConnection(ctx);
+    }
+
+    return null;
+  }
+
+  /**
+   * Delete connection from metadata repository.
+   *
+   * @param ctx Context object
+   * @return Empty bean
+   */
+  private JsonBean deleteConnection(RequestContext ctx) {
+    String sxid = ctx.getLastURLElement();
+    long xid = Long.valueOf(sxid);
+
+    AuditLoggerManager.getInstance()
+        .logAuditEvent(ctx.getUserName(), ctx.getRequest().getRemoteAddr(),
+        "delete", "connection", sxid);
+
+    Repository repository = RepositoryManager.getInstance().getRepository();
+    repository.deleteLink(xid);
+
+    return JsonBean.EMPTY_BEAN;
+  }
+
+  /**
+   * Update or create connection metadata in repository.
+   *
+   * @param ctx Context object
+   * @return Validation bean object
+   */
+  private JsonBean createUpdateConnection(RequestContext ctx, boolean update) {
+//    Check that given ID equals with sent ID, otherwise report an error UPDATE
+//    String sxid = ctx.getLastURLElement();
+//    long xid = Long.valueOf(sxid);
+
+    String username = ctx.getUserName();
+
+    LinkBean bean = new LinkBean();
+
+    try {
+      JSONObject json =
+        (JSONObject) JSONValue.parse(ctx.getRequest().getReader());
+
+      bean.restore(json);
+    } catch (IOException e) {
+      throw new SqoopException(ServerError.SERVER_0003,
+        "Can't read request content", e);
+    }
+
+    // Get connection object
+    List<MLink> connections = bean.getLinks();
+
+    if(connections.size() != 1) {
+      throw new SqoopException(ServerError.SERVER_0003,
+        "Expected one connection metadata but got " + connections.size());
+    }
+
+    MLink connection = connections.get(0);
+
+    // Verify that user is not trying to spoof us
+    MConnectionForms connectorForms =
+      ConnectorManager.getInstance().getConnectorMetadata(connection.getConnectorId())
+      .getConnectionForms();
+    MConnectionForms frameworkForms = Driver.getInstance().getDriverConfig()
+      .getConnectionForms();
+
+    if(!connectorForms.equals(connection.getConnectorPart())
+      || !frameworkForms.equals(connection.getFrameworkPart())) {
+      throw new SqoopException(ServerError.SERVER_0003,
+        "Detected incorrect form structure");
+    }
+
+    // Responsible connector for this session
+    SqoopConnector connector = ConnectorManager.getInstance().getConnector(connection.getConnectorId());
+
+    // We need translate forms to configuration objects
+    Object connectorConfig = ClassUtils.instantiate(connector.getLinkConfigurationClass());
+    Object frameworkConfig = ClassUtils.instantiate(Driver.getInstance().getLinkConfigurationClass());
+
+    FormUtils.fromForms(connection.getConnectorPart().getForms(), connectorConfig);
+    FormUtils.fromForms(connection.getFrameworkPart().getForms(), frameworkConfig);
+
+    // Validate both parts
+    ValidationRunner validationRunner = new ValidationRunner();
+    ValidationResult connectorValidation = validationRunner.validate(connectorConfig);
+    ValidationResult frameworkValidation = validationRunner.validate(frameworkConfig);
+
+    Status finalStatus = Status.getWorstStatus(connectorValidation.getStatus(), frameworkValidation.getStatus());
+
+    // Return back validations in all cases
+    ValidationResultBean outputBean = new ValidationResultBean(connectorValidation, frameworkValidation);
+
+    // If we're good enough let's perform the action
+    if(finalStatus.canProceed()) {
+      if(update) {
+        AuditLoggerManager.getInstance()
+            .logAuditEvent(ctx.getUserName(), ctx.getRequest().getRemoteAddr(),
+            "update", "connection", String.valueOf(connection.getPersistenceId()));
+
+        connection.setLastUpdateUser(username);
+        RepositoryManager.getInstance().getRepository().updateLink(connection);
+      } else {
+        connection.setCreationUser(username);
+        connection.setLastUpdateUser(username);
+        RepositoryManager.getInstance().getRepository().createLink(connection);
+        outputBean.setId(connection.getPersistenceId());
+
+        AuditLoggerManager.getInstance()
+            .logAuditEvent(ctx.getUserName(), ctx.getRequest().getRemoteAddr(),
+            "create", "connection", String.valueOf(connection.getPersistenceId()));
+      }
+    }
+
+    return outputBean;
+  }
+
+  private JsonBean getConnections(RequestContext ctx) {
+    String sxid = ctx.getLastURLElement();
+    LinkBean bean;
+
+    AuditLoggerManager.getInstance()
+        .logAuditEvent(ctx.getUserName(), ctx.getRequest().getRemoteAddr(),
+        "get", "connection", sxid);
+
+    Locale locale = ctx.getAcceptLanguageHeader();
+    Repository repository = RepositoryManager.getInstance().getRepository();
+
+    if (sxid.equals("all")) {
+
+      List<MLink> connections = repository.findLinks();
+      bean = new LinkBean(connections);
+
+      // Add associated resources into the bean
+      for( MLink connection : connections) {
+        long connectorId = connection.getConnectorId();
+        if(!bean.hasConnectorBundle(connectorId)) {
+          bean.addConnectorConfigBundle(connectorId,
+            ConnectorManager.getInstance().getResourceBundle(connectorId, locale));
+        }
+      }
+    } else {
+      long xid = Long.valueOf(sxid);
+
+      MLink connection = repository.findLink(xid);
+      long connectorId = connection.getConnectorId();
+
+      bean = new LinkBean(connection);
+
+      bean.addConnectorConfigBundle(connectorId,
+        ConnectorManager.getInstance().getResourceBundle(connectorId, locale));
+    }
+
+    // Sent framework resource bundle in all cases
+    bean.setDriverConfigBundle(Driver.getInstance().getBundle(locale));
+    return bean;
+  }
+
+  private JsonBean enableConnection(RequestContext ctx, boolean enabled) {
+    String[] elements = ctx.getUrlElements();
+    String sLinkId = elements[elements.length - 2];
+    long linkId = Long.valueOf(sLinkId);
+    Repository repository = RepositoryManager.getInstance().getRepository();
+    repository.enableLink(linkId, enabled);
+    return JsonBean.EMPTY_BEAN;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/server/src/main/java/org/apache/sqoop/handler/SubmissionRequestHandler.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/handler/SubmissionRequestHandler.java b/server/src/main/java/org/apache/sqoop/handler/SubmissionRequestHandler.java
index 6e21dd9..8555b0c 100644
--- a/server/src/main/java/org/apache/sqoop/handler/SubmissionRequestHandler.java
+++ b/server/src/main/java/org/apache/sqoop/handler/SubmissionRequestHandler.java
@@ -22,8 +22,7 @@ import java.util.List;
 import org.apache.log4j.Logger;
 import org.apache.sqoop.audit.AuditLoggerManager;
 import org.apache.sqoop.common.SqoopException;
-import org.apache.sqoop.framework.FrameworkManager;
-import org.apache.sqoop.framework.JobManager;
+import org.apache.sqoop.driver.JobManager;
 import org.apache.sqoop.json.JsonBean;
 import org.apache.sqoop.json.SubmissionBean;
 import org.apache.sqoop.model.MSubmission;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/server/src/main/java/org/apache/sqoop/server/v1/ConnectionServlet.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/server/v1/ConnectionServlet.java b/server/src/main/java/org/apache/sqoop/server/v1/ConnectionServlet.java
deleted file mode 100644
index e8af4d7..0000000
--- a/server/src/main/java/org/apache/sqoop/server/v1/ConnectionServlet.java
+++ /dev/null
@@ -1,56 +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.server.v1;
-
-import org.apache.sqoop.handler.ConnectionRequestHandler;
-import org.apache.sqoop.json.JsonBean;
-import org.apache.sqoop.server.RequestContext;
-import org.apache.sqoop.server.RequestHandler;
-import org.apache.sqoop.server.SqoopProtocolServlet;
-
-/**
- *
- */
-public class ConnectionServlet extends SqoopProtocolServlet {
-
-  private RequestHandler requestHandler;
-
-  public ConnectionServlet() {
-    requestHandler = new ConnectionRequestHandler();
-  }
-
-  @Override
-  protected JsonBean handleGetRequest(RequestContext ctx) throws Exception {
-    return requestHandler.handleEvent(ctx);
-  }
-
-  @Override
-  protected JsonBean handlePostRequest(RequestContext ctx) throws Exception {
-    return requestHandler.handleEvent(ctx);
-  }
-
-  @Override
-  protected JsonBean handlePutRequest(RequestContext ctx) throws Exception {
-    return requestHandler.handleEvent(ctx);
-  }
-
-  @Override
-  protected JsonBean handleDeleteRequest(RequestContext ctx) throws Exception {
-    return requestHandler.handleEvent(ctx);
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/server/src/main/java/org/apache/sqoop/server/v1/DriverConfigServlet.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/server/v1/DriverConfigServlet.java b/server/src/main/java/org/apache/sqoop/server/v1/DriverConfigServlet.java
new file mode 100644
index 0000000..c2b1f9f
--- /dev/null
+++ b/server/src/main/java/org/apache/sqoop/server/v1/DriverConfigServlet.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.sqoop.server.v1;
+
+import org.apache.sqoop.handler.DriverConfigRequestHandler;
+import org.apache.sqoop.json.JsonBean;
+import org.apache.sqoop.server.RequestContext;
+import org.apache.sqoop.server.RequestHandler;
+import org.apache.sqoop.server.SqoopProtocolServlet;
+
+/**
+ * Get driver config
+ */
+@SuppressWarnings("serial")
+public class DriverConfigServlet extends SqoopProtocolServlet {
+  private RequestHandler driverConfigRequestHandler;
+
+  public DriverConfigServlet() {
+    driverConfigRequestHandler = new DriverConfigRequestHandler();
+  }
+
+  @Override
+  protected JsonBean handleGetRequest(RequestContext ctx) throws Exception {
+    return driverConfigRequestHandler.handleEvent(ctx);
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/server/src/main/java/org/apache/sqoop/server/v1/FrameworkServlet.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/server/v1/FrameworkServlet.java b/server/src/main/java/org/apache/sqoop/server/v1/FrameworkServlet.java
deleted file mode 100644
index 9c99731..0000000
--- a/server/src/main/java/org/apache/sqoop/server/v1/FrameworkServlet.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.sqoop.server.v1;
-
-import org.apache.sqoop.handler.FrameworkRequestHandler;
-import org.apache.sqoop.json.JsonBean;
-import org.apache.sqoop.server.RequestContext;
-import org.apache.sqoop.server.RequestHandler;
-import org.apache.sqoop.server.SqoopProtocolServlet;
-
-/**
- * Get framework metadata
- */
-public class FrameworkServlet extends SqoopProtocolServlet {
-  private RequestHandler frameworkRequestHandler;
-
-  public FrameworkServlet() {
-    frameworkRequestHandler = new FrameworkRequestHandler();
-  }
-
-  @Override
-  protected JsonBean handleGetRequest(RequestContext ctx) throws Exception {
-    return frameworkRequestHandler.handleEvent(ctx);
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/server/src/main/java/org/apache/sqoop/server/v1/JobServlet.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/server/v1/JobServlet.java b/server/src/main/java/org/apache/sqoop/server/v1/JobServlet.java
index 34a0ffb..d295237 100644
--- a/server/src/main/java/org/apache/sqoop/server/v1/JobServlet.java
+++ b/server/src/main/java/org/apache/sqoop/server/v1/JobServlet.java
@@ -26,31 +26,32 @@ import org.apache.sqoop.server.SqoopProtocolServlet;
 /**
  *
  */
+@SuppressWarnings("serial")
 public class JobServlet extends SqoopProtocolServlet {
 
-  private RequestHandler requestHandler;
+  private RequestHandler jobRequestHandler;
 
   public JobServlet() {
-    requestHandler = new JobRequestHandler();
+    jobRequestHandler = new JobRequestHandler();
   }
 
   @Override
   protected JsonBean handleGetRequest(RequestContext ctx) throws Exception {
-    return requestHandler.handleEvent(ctx);
+    return jobRequestHandler.handleEvent(ctx);
   }
 
   @Override
   protected JsonBean handlePostRequest(RequestContext ctx) throws Exception {
-    return requestHandler.handleEvent(ctx);
+    return jobRequestHandler.handleEvent(ctx);
   }
 
   @Override
   protected JsonBean handlePutRequest(RequestContext ctx) throws Exception {
-    return requestHandler.handleEvent(ctx);
+    return jobRequestHandler.handleEvent(ctx);
   }
 
   @Override
   protected JsonBean handleDeleteRequest(RequestContext ctx) throws Exception {
-    return requestHandler.handleEvent(ctx);
+    return jobRequestHandler.handleEvent(ctx);
   }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/server/src/main/java/org/apache/sqoop/server/v1/LinkServlet.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/server/v1/LinkServlet.java b/server/src/main/java/org/apache/sqoop/server/v1/LinkServlet.java
new file mode 100644
index 0000000..20e5009
--- /dev/null
+++ b/server/src/main/java/org/apache/sqoop/server/v1/LinkServlet.java
@@ -0,0 +1,57 @@
+/**
+ * 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.server.v1;
+
+import org.apache.sqoop.handler.LinkRequestHandler;
+import org.apache.sqoop.json.JsonBean;
+import org.apache.sqoop.server.RequestContext;
+import org.apache.sqoop.server.RequestHandler;
+import org.apache.sqoop.server.SqoopProtocolServlet;
+
+/**
+ *
+ */
+@SuppressWarnings("serial")
+public class LinkServlet extends SqoopProtocolServlet {
+
+  private RequestHandler linkRequestHandler;
+
+  public LinkServlet() {
+    linkRequestHandler = new LinkRequestHandler();
+  }
+
+  @Override
+  protected JsonBean handleGetRequest(RequestContext ctx) throws Exception {
+    return linkRequestHandler.handleEvent(ctx);
+  }
+
+  @Override
+  protected JsonBean handlePostRequest(RequestContext ctx) throws Exception {
+    return linkRequestHandler.handleEvent(ctx);
+  }
+
+  @Override
+  protected JsonBean handlePutRequest(RequestContext ctx) throws Exception {
+    return linkRequestHandler.handleEvent(ctx);
+  }
+
+  @Override
+  protected JsonBean handleDeleteRequest(RequestContext ctx) throws Exception {
+    return linkRequestHandler.handleEvent(ctx);
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/server/src/main/java/org/apache/sqoop/server/v1/SubmissionServlet.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/sqoop/server/v1/SubmissionServlet.java b/server/src/main/java/org/apache/sqoop/server/v1/SubmissionServlet.java
index 7252e11..5c1d883 100644
--- a/server/src/main/java/org/apache/sqoop/server/v1/SubmissionServlet.java
+++ b/server/src/main/java/org/apache/sqoop/server/v1/SubmissionServlet.java
@@ -26,26 +26,27 @@ import org.apache.sqoop.server.SqoopProtocolServlet;
 /**
  *
  */
+@SuppressWarnings("serial")
 public class SubmissionServlet extends SqoopProtocolServlet {
 
-  private RequestHandler requestHandler;
+  private RequestHandler submissionRequestHandler;
 
   public SubmissionServlet() {
-    requestHandler = new SubmissionRequestHandler();
+    submissionRequestHandler = new SubmissionRequestHandler();
   }
 
   @Override
   protected JsonBean handleGetRequest(RequestContext ctx) throws Exception {
-    return requestHandler.handleEvent(ctx);
+    return submissionRequestHandler.handleEvent(ctx);
   }
 
   @Override
   protected JsonBean handlePostRequest(RequestContext ctx) throws Exception {
-    return requestHandler.handleEvent(ctx);
+    return submissionRequestHandler.handleEvent(ctx);
   }
 
   @Override
   protected JsonBean handleDeleteRequest(RequestContext ctx) throws Exception {
-    return requestHandler.handleEvent(ctx);
+    return submissionRequestHandler.handleEvent(ctx);
   }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/server/src/main/webapp/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/server/src/main/webapp/WEB-INF/web.xml b/server/src/main/webapp/WEB-INF/web.xml
index f053062..d31120a 100644
--- a/server/src/main/webapp/WEB-INF/web.xml
+++ b/server/src/main/webapp/WEB-INF/web.xml
@@ -51,28 +51,29 @@ limitations under the License.
     <url-pattern>/v1/connector/*</url-pattern>
   </servlet-mapping>
 
-  <!-- Framework servlet -->
+  <!-- Driver Config servlet -->
   <servlet>
-    <servlet-name>v1.FrameworkServlet</servlet-name>
-    <servlet-class>org.apache.sqoop.server.v1.FrameworkServlet</servlet-class>
+    <servlet-name>v1.DriverConfigServlet</servlet-name>
+    <servlet-class>org.apache.sqoop.server.v1.DriverConfigServlet</servlet-class>
     <load-on-startup>1</load-on-startup>
   </servlet>
 
   <servlet-mapping>
-    <servlet-name>v1.FrameworkServlet</servlet-name>
-    <url-pattern>/v1/framework/*</url-pattern>
+    <servlet-name>v1.DriverConfigServlet</servlet-name>
+    <url-pattern>/v1/config/driver/*</url-pattern>
   </servlet-mapping>
 
-  <!-- Connection servlet -->
+
+  <!-- Link servlet -->
   <servlet>
-    <servlet-name>v1.ConnectionServlet</servlet-name>
-    <servlet-class>org.apache.sqoop.server.v1.ConnectionServlet</servlet-class>
+    <servlet-name>v1.LinkServlet</servlet-name>
+    <servlet-class>org.apache.sqoop.server.v1.LinkServlet</servlet-class>
     <load-on-startup>1</load-on-startup>
   </servlet>
 
   <servlet-mapping>
-    <servlet-name>v1.ConnectionServlet</servlet-name>
-    <url-pattern>/v1/connection/*</url-pattern>
+    <servlet-name>v1.LinkServlet</servlet-name>
+    <url-pattern>/v1/link/*</url-pattern>
   </servlet-mapping>
 
   <!-- Job servlet -->

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/shell/src/main/java/org/apache/sqoop/shell/CloneCommand.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/CloneCommand.java b/shell/src/main/java/org/apache/sqoop/shell/CloneCommand.java
index 4cdf0e4..f71f82c 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/CloneCommand.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/CloneCommand.java
@@ -31,7 +31,7 @@ public class CloneCommand extends SqoopCommand {
       Constants.CMD_CLONE,
       Constants.CMD_CLONE_SC,
       ImmutableMap.of(
-        Constants.FN_CONNECTION, CloneConnectionFunction.class,
+        Constants.FN_LINK, CloneLinkFunction.class,
         Constants.FN_JOB, CloneJobFunction.class
       )
     );

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/shell/src/main/java/org/apache/sqoop/shell/CloneConnectionFunction.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/CloneConnectionFunction.java b/shell/src/main/java/org/apache/sqoop/shell/CloneConnectionFunction.java
deleted file mode 100644
index d912c1c..0000000
--- a/shell/src/main/java/org/apache/sqoop/shell/CloneConnectionFunction.java
+++ /dev/null
@@ -1,109 +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.shell;
-
-import jline.ConsoleReader;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.sqoop.model.MConnection;
-import org.apache.sqoop.model.MPersistableEntity;
-import org.apache.sqoop.shell.core.Constants;
-import org.apache.sqoop.shell.utils.ConnectionDynamicFormOptions;
-import org.apache.sqoop.shell.utils.FormOptions;
-import org.apache.sqoop.validation.Status;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.ResourceBundle;
-
-import static org.apache.sqoop.shell.ShellEnvironment.*;
-import static org.apache.sqoop.shell.utils.FormFiller.*;
-
-/**
- *
- */
-@SuppressWarnings("serial")
-public class CloneConnectionFunction extends SqoopFunction {
-  @SuppressWarnings("static-access")
-  public CloneConnectionFunction() {
-    this.addOption(OptionBuilder
-      .withDescription(resourceString(Constants.RES_PROMPT_CONN_ID))
-      .withLongOpt(Constants.OPT_XID)
-      .hasArg()
-      .isRequired()
-      .create(Constants.OPT_XID_CHAR)
-    );
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public Object executeFunction(CommandLine line, boolean isInteractive) throws IOException {
-    return cloneConnection(getLong(line, Constants.OPT_XID), line.getArgList(), isInteractive);
-  }
-
-  private Status cloneConnection(Long connectionId, List<String> args, boolean isInteractive) throws IOException {
-    printlnResource(Constants.RES_CLONE_CLONING_CONN, connectionId);
-
-    ConsoleReader reader = new ConsoleReader();
-
-    MConnection connection = client.getConnection(connectionId);
-    // Remove persistent id as we're making a clone
-    connection.setPersistenceId(MPersistableEntity.PERSISTANCE_ID_DEFAULT);
-
-    Status status = Status.FINE;
-
-    ResourceBundle connectorBundle = client.getResourceBundle(connection.getConnectorId());
-    ResourceBundle frameworkBundle = client.getFrameworkResourceBundle();
-
-    if (isInteractive) {
-      printlnResource(Constants.RES_PROMPT_UPDATE_CONN_METADATA);
-
-      do {
-        // Print error introduction if needed
-        if( !status.canProceed() ) {
-          errorIntroduction();
-        }
-
-        // Fill in data from user
-        if(!fillConnection(reader, connection, connectorBundle, frameworkBundle)) {
-          return null;
-        }
-
-        status = client.createConnection(connection);
-      } while(!status.canProceed());
-    } else {
-      ConnectionDynamicFormOptions options = new ConnectionDynamicFormOptions();
-      options.prepareOptions(connection);
-      CommandLine line = FormOptions.parseOptions(options, 0, args, false);
-      if (fillConnection(line, connection)) {
-        status = client.createConnection(connection);
-        if (!status.canProceed()) {
-          printConnectionValidationMessages(connection);
-          return null;
-        }
-      } else {
-        printConnectionValidationMessages(connection);
-        return null;
-      }
-    }
-
-    printlnResource(Constants.RES_CLONE_CONN_SUCCESSFUL, status.name(), connection.getPersistenceId());
-
-    return status;
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/shell/src/main/java/org/apache/sqoop/shell/CloneJobFunction.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/CloneJobFunction.java b/shell/src/main/java/org/apache/sqoop/shell/CloneJobFunction.java
index ed461ab..97d8e6f 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/CloneJobFunction.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/CloneJobFunction.java
@@ -63,10 +63,10 @@ public class CloneJobFunction extends SqoopFunction {
     MJob job = client.getJob(jobId);
     job.setPersistenceId(MPersistableEntity.PERSISTANCE_ID_DEFAULT);
 
-    ResourceBundle fromConnectorBundle = client.getResourceBundle(
+    ResourceBundle fromConnectorBundle = client.getConnectorConfigResourceBundle(
         job.getConnectorId(Direction.FROM));
-    ResourceBundle frameworkBundle = client.getFrameworkResourceBundle();
-    ResourceBundle toConnectorBundle = client.getResourceBundle(
+    ResourceBundle driverConfigBundle = client.getDriverConfigBundle();
+    ResourceBundle toConnectorBundle = client.getConnectorConfigResourceBundle(
         job.getConnectorId(Direction.TO));
 
     Status status = Status.FINE;
@@ -75,7 +75,7 @@ public class CloneJobFunction extends SqoopFunction {
     job.setPersistenceId(MPersistableEntity.PERSISTANCE_ID_DEFAULT);
 
     if (isInteractive) {
-      printlnResource(Constants.RES_PROMPT_UPDATE_JOB_METADATA);
+      printlnResource(Constants.RES_PROMPT_UPDATE_JOB_CONFIG);
 
       do {
         // Print error introduction if needed
@@ -84,19 +84,19 @@ public class CloneJobFunction extends SqoopFunction {
         }
 
         // Fill in data from user
-        if(!fillJob(reader, job, fromConnectorBundle, frameworkBundle, toConnectorBundle)) {
+        if(!fillJob(reader, job, fromConnectorBundle, driverConfigBundle, toConnectorBundle)) {
           return null;
         }
 
         // Try to create
-        status = client.createJob(job);
+        status = client.saveJob(job);
       } while(!status.canProceed());
     } else {
       JobDynamicFormOptions options = new JobDynamicFormOptions();
       options.prepareOptions(job);
       CommandLine line = FormOptions.parseOptions(options, 0, args, false);
       if (fillJob(line, job)) {
-        status = client.createJob(job);
+        status = client.saveJob(job);
         if (!status.canProceed()) {
           printJobValidationMessages(job);
           return null;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/shell/src/main/java/org/apache/sqoop/shell/CloneLinkFunction.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/CloneLinkFunction.java b/shell/src/main/java/org/apache/sqoop/shell/CloneLinkFunction.java
new file mode 100644
index 0000000..d9babe0
--- /dev/null
+++ b/shell/src/main/java/org/apache/sqoop/shell/CloneLinkFunction.java
@@ -0,0 +1,109 @@
+/**
+ * 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.shell;
+
+import jline.ConsoleReader;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.sqoop.model.MLink;
+import org.apache.sqoop.model.MPersistableEntity;
+import org.apache.sqoop.shell.core.Constants;
+import org.apache.sqoop.shell.utils.LinkDynamicFormOptions;
+import org.apache.sqoop.shell.utils.FormOptions;
+import org.apache.sqoop.validation.Status;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.ResourceBundle;
+
+import static org.apache.sqoop.shell.ShellEnvironment.*;
+import static org.apache.sqoop.shell.utils.FormFiller.*;
+
+/**
+ *
+ */
+@SuppressWarnings("serial")
+public class CloneLinkFunction extends SqoopFunction {
+  @SuppressWarnings("static-access")
+  public CloneLinkFunction() {
+    this.addOption(OptionBuilder
+      .withDescription(resourceString(Constants.RES_PROMPT_LINK_ID))
+      .withLongOpt(Constants.OPT_LID)
+      .hasArg()
+      .isRequired()
+      .create(Constants.OPT_LID_CHAR)
+    );
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public Object executeFunction(CommandLine line, boolean isInteractive) throws IOException {
+    return cloneLink(getLong(line, Constants.OPT_LID), line.getArgList(), isInteractive);
+  }
+
+  private Status cloneLink(Long connectionId, List<String> args, boolean isInteractive) throws IOException {
+    printlnResource(Constants.RES_CLONE_CLONING_LINK, connectionId);
+
+    ConsoleReader reader = new ConsoleReader();
+
+    MLink connection = client.getLink(connectionId);
+    // Remove persistent id as we're making a clone
+    connection.setPersistenceId(MPersistableEntity.PERSISTANCE_ID_DEFAULT);
+
+    Status status = Status.FINE;
+
+    ResourceBundle connectorConfigBundle = client.getConnectorConfigResourceBundle(connection.getConnectorId());
+    ResourceBundle driverConfigBundle = client.getDriverConfigBundle();
+
+    if (isInteractive) {
+      printlnResource(Constants.RES_PROMPT_UPDATE_LINK_CONFIG);
+
+      do {
+        // Print error introduction if needed
+        if( !status.canProceed() ) {
+          errorIntroduction();
+        }
+
+        // Fill in data from user
+        if(!fillLink(reader, connection, connectorConfigBundle, driverConfigBundle)) {
+          return null;
+        }
+
+        status = client.saveLink(connection);
+      } while(!status.canProceed());
+    } else {
+      LinkDynamicFormOptions options = new LinkDynamicFormOptions();
+      options.prepareOptions(connection);
+      CommandLine line = FormOptions.parseOptions(options, 0, args, false);
+      if (fillConnection(line, connection)) {
+        status = client.saveLink(connection);
+        if (!status.canProceed()) {
+          printLinkValidationMessages(connection);
+          return null;
+        }
+      } else {
+        printLinkValidationMessages(connection);
+        return null;
+      }
+    }
+
+    printlnResource(Constants.RES_CLONE_LINK_SUCCESSFUL, status.name(), connection.getPersistenceId());
+
+    return status;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/shell/src/main/java/org/apache/sqoop/shell/CreateCommand.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/CreateCommand.java b/shell/src/main/java/org/apache/sqoop/shell/CreateCommand.java
index fce7c86..1ef2418 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/CreateCommand.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/CreateCommand.java
@@ -31,7 +31,7 @@ public class CreateCommand extends SqoopCommand {
       Constants.CMD_CREATE,
       Constants.CMD_CREATE_SC,
       ImmutableMap.of(
-        Constants.FN_CONNECTION, CreateConnectionFunction.class,
+        Constants.FN_LINK, CreateLinkFunction.class,
         Constants.FN_JOB, CreateJobFunction.class
       )
     );

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/shell/src/main/java/org/apache/sqoop/shell/CreateConnectionFunction.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/CreateConnectionFunction.java b/shell/src/main/java/org/apache/sqoop/shell/CreateConnectionFunction.java
deleted file mode 100644
index 92a8aa5..0000000
--- a/shell/src/main/java/org/apache/sqoop/shell/CreateConnectionFunction.java
+++ /dev/null
@@ -1,108 +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.shell;
-
-import jline.ConsoleReader;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.OptionBuilder;
-import org.apache.sqoop.model.MConnection;
-import org.apache.sqoop.shell.core.Constants;
-import org.apache.sqoop.shell.utils.ConnectionDynamicFormOptions;
-import org.apache.sqoop.shell.utils.FormDisplayer;
-import org.apache.sqoop.shell.utils.FormOptions;
-import org.apache.sqoop.validation.Status;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.ResourceBundle;
-
-import static org.apache.sqoop.shell.ShellEnvironment.*;
-import static org.apache.sqoop.shell.utils.FormFiller.*;
-
-/**
- *
- */
-@SuppressWarnings("serial")
-public class CreateConnectionFunction extends SqoopFunction {
-  @SuppressWarnings("static-access")
-  public CreateConnectionFunction() {
-    this.addOption(OptionBuilder
-      .withDescription(resourceString(Constants.RES_CONNECTOR_ID))
-      .withLongOpt(Constants.OPT_CID)
-      .isRequired()
-      .hasArg()
-      .create(Constants.OPT_CID_CHAR));
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public Object executeFunction(CommandLine line, boolean isInteractive) throws IOException {
-    return createConnection(getLong(line, Constants.OPT_CID), line.getArgList(), isInteractive);
-  }
-
-  private Status createConnection(long connectorId, List<String> args, boolean isInteractive) throws IOException {
-    printlnResource(Constants.RES_CREATE_CREATING_CONN, connectorId);
-
-    ConsoleReader reader = new ConsoleReader();
-
-    MConnection connection = client.newConnection(connectorId);
-
-    ResourceBundle connectorBundle = client.getResourceBundle(connectorId);
-    ResourceBundle frameworkBundle = client.getFrameworkResourceBundle();
-
-    Status status = Status.FINE;
-
-    if (isInteractive) {
-      printlnResource(Constants.RES_PROMPT_FILL_CONN_METADATA);
-
-      do {
-        // Print error introduction if needed
-        if( !status.canProceed() ) {
-          errorIntroduction();
-        }
-
-        // Fill in data from user
-        if(!fillConnection(reader, connection, connectorBundle, frameworkBundle)) {
-          return null;
-        }
-
-        // Try to create
-        status = client.createConnection(connection);
-      } while(!status.canProceed());
-    } else {
-      ConnectionDynamicFormOptions options = new ConnectionDynamicFormOptions();
-      options.prepareOptions(connection);
-      CommandLine line = FormOptions.parseOptions(options, 0, args, false);
-      if (fillConnection(line, connection)) {
-        status = client.createConnection(connection);
-        if (!status.canProceed()) {
-          printConnectionValidationMessages(connection);
-          return null;
-        }
-      } else {
-        printConnectionValidationMessages(connection);
-        return null;
-      }
-    }
-
-    FormDisplayer.displayFormWarning(connection);
-    printlnResource(Constants.RES_CREATE_CONN_SUCCESSFUL, status.name(), connection.getPersistenceId());
-
-    return status;
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/shell/src/main/java/org/apache/sqoop/shell/CreateJobFunction.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/CreateJobFunction.java b/shell/src/main/java/org/apache/sqoop/shell/CreateJobFunction.java
index 88afde3..ccfed31 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/CreateJobFunction.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/CreateJobFunction.java
@@ -43,18 +43,18 @@ public class CreateJobFunction extends  SqoopFunction {
   @SuppressWarnings("static-access")
   public CreateJobFunction() {
     this.addOption(OptionBuilder
-      .withDescription(resourceString(Constants.RES_PROMPT_CONN_ID))
+      .withDescription(resourceString(Constants.RES_PROMPT_LINK_ID))
       .withLongOpt(Constants.OPT_FROM)
       .isRequired()
       .hasArg()
-      .create(Constants.OPT_FXID_CHAR)
+      .create(Constants.OPT_FROM_CHAR)
     );
     this.addOption(OptionBuilder
-      .withDescription(resourceString(Constants.RES_PROMPT_CONN_ID))
+      .withDescription(resourceString(Constants.RES_PROMPT_LINK_ID))
       .withLongOpt(Constants.OPT_TO)
       .isRequired()
       .hasArg()
-      .create(Constants.OPT_TXID_CHAR)
+      .create(Constants.OPT_TO_CHAR)
     );
   }
 
@@ -71,19 +71,19 @@ public class CreateJobFunction extends  SqoopFunction {
     printlnResource(Constants.RES_CREATE_CREATING_JOB, fromConnectionId, toConnectionId);
 
     ConsoleReader reader = new ConsoleReader();
-    MJob job = client.newJob(fromConnectionId, toConnectionId);
+    MJob job = client.createJob(fromConnectionId, toConnectionId);
 
     // @TODO(Abe): From/To.
-    ResourceBundle fromConnectorBundle = client.getResourceBundle(
+    ResourceBundle fromConnectorBundle = client.getConnectorConfigResourceBundle(
         job.getConnectorId(Direction.FROM));
-    ResourceBundle toConnectorBundle = client.getResourceBundle(
+    ResourceBundle toConnectorBundle = client.getConnectorConfigResourceBundle(
         job.getConnectorId(Direction.TO));
-    ResourceBundle frameworkBundle = client.getFrameworkResourceBundle();
+    ResourceBundle driverConfigBundle = client.getDriverConfigBundle();
 
     Status status = Status.FINE;
 
     if (isInteractive) {
-      printlnResource(Constants.RES_PROMPT_FILL_JOB_METADATA);
+      printlnResource(Constants.RES_PROMPT_FILL_JOB_CONFIG);
 
       do {
         // Print error introduction if needed
@@ -92,19 +92,19 @@ public class CreateJobFunction extends  SqoopFunction {
         }
 
         // Fill in data from user
-        if(!fillJob(reader, job, fromConnectorBundle, frameworkBundle, toConnectorBundle)) {
+        if(!fillJob(reader, job, fromConnectorBundle, driverConfigBundle, toConnectorBundle)) {
           return null;
         }
 
         // Try to create
-        status = client.createJob(job);
+        status = client.saveJob(job);
       } while(!status.canProceed());
     } else {
       JobDynamicFormOptions options = new JobDynamicFormOptions();
       options.prepareOptions(job);
       CommandLine line = FormOptions.parseOptions(options, 0, args, false);
       if (fillJob(line, job)) {
-        status = client.createJob(job);
+        status = client.saveJob(job);
         if (!status.canProceed()) {
           printJobValidationMessages(job);
           return null;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/shell/src/main/java/org/apache/sqoop/shell/CreateLinkFunction.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/CreateLinkFunction.java b/shell/src/main/java/org/apache/sqoop/shell/CreateLinkFunction.java
new file mode 100644
index 0000000..33d60c8
--- /dev/null
+++ b/shell/src/main/java/org/apache/sqoop/shell/CreateLinkFunction.java
@@ -0,0 +1,108 @@
+/**
+ * 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.shell;
+
+import jline.ConsoleReader;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.sqoop.model.MLink;
+import org.apache.sqoop.shell.core.Constants;
+import org.apache.sqoop.shell.utils.LinkDynamicFormOptions;
+import org.apache.sqoop.shell.utils.FormDisplayer;
+import org.apache.sqoop.shell.utils.FormOptions;
+import org.apache.sqoop.validation.Status;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.ResourceBundle;
+
+import static org.apache.sqoop.shell.ShellEnvironment.*;
+import static org.apache.sqoop.shell.utils.FormFiller.*;
+
+/**
+ *
+ */
+@SuppressWarnings("serial")
+public class CreateLinkFunction extends SqoopFunction {
+  @SuppressWarnings("static-access")
+  public CreateLinkFunction() {
+    this.addOption(OptionBuilder
+      .withDescription(resourceString(Constants.RES_CONNECTOR_ID))
+      .withLongOpt(Constants.OPT_CID)
+      .isRequired()
+      .hasArg()
+      .create(Constants.OPT_CID_CHAR));
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public Object executeFunction(CommandLine line, boolean isInteractive) throws IOException {
+    return createLink(getLong(line, Constants.OPT_CID), line.getArgList(), isInteractive);
+  }
+
+  private Status createLink(long connectorId, List<String> args, boolean isInteractive) throws IOException {
+    printlnResource(Constants.RES_CREATE_CREATING_LINK, connectorId);
+
+    ConsoleReader reader = new ConsoleReader();
+
+    MLink link = client.createLink(connectorId);
+
+    ResourceBundle connectorConfigBundle = client.getConnectorConfigResourceBundle(connectorId);
+    ResourceBundle driverConfigBundle = client.getDriverConfigBundle();
+
+    Status status = Status.FINE;
+
+    if (isInteractive) {
+      printlnResource(Constants.RES_PROMPT_FILL_LINK_CONFIG);
+
+      do {
+        // Print error introduction if needed
+        if( !status.canProceed() ) {
+          errorIntroduction();
+        }
+
+        // Fill in data from user
+        if(!fillLink(reader, link, connectorConfigBundle, driverConfigBundle)) {
+          return null;
+        }
+
+        // Try to create
+        status = client.saveLink(link);
+      } while(!status.canProceed());
+    } else {
+      LinkDynamicFormOptions options = new LinkDynamicFormOptions();
+      options.prepareOptions(link);
+      CommandLine line = FormOptions.parseOptions(options, 0, args, false);
+      if (fillConnection(line, link)) {
+        status = client.saveLink(link);
+        if (!status.canProceed()) {
+          printLinkValidationMessages(link);
+          return null;
+        }
+      } else {
+        printLinkValidationMessages(link);
+        return null;
+      }
+    }
+
+    FormDisplayer.displayFormWarning(link);
+    printlnResource(Constants.RES_CREATE_LINK_SUCCESSFUL, status.name(), link.getPersistenceId());
+
+    return status;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/shell/src/main/java/org/apache/sqoop/shell/DeleteCommand.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/sqoop/shell/DeleteCommand.java b/shell/src/main/java/org/apache/sqoop/shell/DeleteCommand.java
index 107e5e0..26e2bf6 100644
--- a/shell/src/main/java/org/apache/sqoop/shell/DeleteCommand.java
+++ b/shell/src/main/java/org/apache/sqoop/shell/DeleteCommand.java
@@ -31,7 +31,7 @@ public class DeleteCommand extends SqoopCommand {
       Constants.CMD_DELETE,
       Constants.CMD_DELETE_SC,
       ImmutableMap.of(
-        Constants.FN_CONNECTION, DeleteConnectionFunction.class,
+        Constants.FN_LINK, DeleteLinkFunction.class,
         Constants.FN_JOB, DeleteJobFunction.class
       )
     );