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 05:07:09 UTC

[17/50] [abbrv] SQOOP-1497: Sqoop2: Entity Nomenclature Revisited

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/client/src/test/java/org/apache/sqoop/client/TestSqoopClient.java
----------------------------------------------------------------------
diff --git a/client/src/test/java/org/apache/sqoop/client/TestSqoopClient.java b/client/src/test/java/org/apache/sqoop/client/TestSqoopClient.java
index 731f6cc..54ea3d2 100644
--- a/client/src/test/java/org/apache/sqoop/client/TestSqoopClient.java
+++ b/client/src/test/java/org/apache/sqoop/client/TestSqoopClient.java
@@ -17,13 +17,13 @@
  */
 package org.apache.sqoop.client;
 
-import org.apache.sqoop.client.request.SqoopRequests;
+import org.apache.sqoop.client.request.SqoopResourceRequests;
 import org.apache.sqoop.common.SqoopException;
 import org.apache.sqoop.json.ConnectorBean;
-import org.apache.sqoop.json.FrameworkBean;
+import org.apache.sqoop.json.DriverConfigBean;
 import org.apache.sqoop.model.MConnectionForms;
 import org.apache.sqoop.model.MConnector;
-import org.apache.sqoop.model.MFramework;
+import org.apache.sqoop.model.MDriverConfig;
 import org.apache.sqoop.model.MJobForms;
 import org.apache.sqoop.utils.MapResourceBundle;
 import org.junit.Before;
@@ -32,7 +32,6 @@ import org.junit.Test;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.ResourceBundle;
@@ -43,14 +42,14 @@ import static org.mockito.Mockito.*;
 
 public class TestSqoopClient {
 
-  SqoopRequests requests;
+  SqoopResourceRequests resourceRequests;
   SqoopClient client;
 
   @Before
   public void setUp() {
-    requests = mock(SqoopRequests.class);
+    resourceRequests = mock(SqoopResourceRequests.class);
     client = new SqoopClient("my-cool-server");
-    client.setSqoopRequests(requests);
+    client.setSqoopRequests(resourceRequests);
   }
 
   /**
@@ -59,26 +58,26 @@ public class TestSqoopClient {
    */
   @Test
   public void testGetConnector() {
-    when(requests.readConnector(1L)).thenReturn(connectorBean(connector(1)));
+    when(resourceRequests.readConnector(1L)).thenReturn(connectorBean(connector(1)));
     MConnector connector = client.getConnector(1);
     assertEquals(1, connector.getPersistenceId());
 
-    client.getResourceBundle(1L);
+    client.getConnectorConfigResourceBundle(1L);
 
-    verify(requests, times(1)).readConnector(1L);
+    verify(resourceRequests, times(1)).readConnector(1L);
   }
 
   @Test
   public void testGetConnectorByString() {
-    when(requests.readConnector(null)).thenReturn(connectorBean(connector(1)));
+    when(resourceRequests.readConnector(null)).thenReturn(connectorBean(connector(1)));
     MConnector connector = client.getConnector("A1");
     assertEquals(1, connector.getPersistenceId());
     assertEquals("A1", connector.getUniqueName());
 
-    client.getResourceBundle(1L);
+    client.getConnectorConfigResourceBundle(1L);
 
-    verify(requests, times(0)).readConnector(1L);
-    verify(requests, times(1)).readConnector(null);
+    verify(resourceRequests, times(0)).readConnector(1L);
+    verify(resourceRequests, times(1)).readConnector(null);
   }
 
   /**
@@ -87,41 +86,41 @@ public class TestSqoopClient {
    */
   @Test
   public void testGetConnectorBundle() {
-    when(requests.readConnector(1L)).thenReturn(connectorBean(connector(1)));
-    client.getResourceBundle(1L);
+    when(resourceRequests.readConnector(1L)).thenReturn(connectorBean(connector(1)));
+    client.getConnectorConfigResourceBundle(1L);
 
     MConnector connector = client.getConnector(1);
     assertEquals(1, connector.getPersistenceId());
 
-    verify(requests, times(1)).readConnector(1L);
+    verify(resourceRequests, times(1)).readConnector(1L);
   }
 
   /**
-   * Retrieve framework information, request to framework bundle should not
+   * Retrieve driverConfig information, request to driverConfig bundle should not
    * require additional HTTP request.
    */
   @Test
-  public void testGetFramework() {
-    when(requests.readFramework()).thenReturn(frameworkBean(framework()));
+  public void testGetDriverConfig() {
+    when(resourceRequests.readDriverConfig()).thenReturn(driverConfigBean(driverConfig()));
 
-    client.getFramework();
-    client.getFrameworkResourceBundle();
+    client.getDriverConfig();
+    client.getDriverConfigBundle();
 
-    verify(requests, times(1)).readFramework();
+    verify(resourceRequests, times(1)).readDriverConfig();
   }
 
   /**
-   * Retrieve framework bundle, request to framework metadata should not
+   * Retrieve driverConfig bundle, request to driverConfig metadata should not
    * require additional HTTP request.
    */
   @Test
-  public void testGetFrameworkBundle() {
-    when(requests.readFramework()).thenReturn(frameworkBean(framework()));
+  public void testGetDriverConfigBundle() {
+    when(resourceRequests.readDriverConfig()).thenReturn(driverConfigBean(driverConfig()));
 
-    client.getFrameworkResourceBundle();
-    client.getFramework();
+    client.getDriverConfigBundle();
+    client.getDriverConfig();
 
-    verify(requests, times(1)).readFramework();
+    verify(resourceRequests, times(1)).readDriverConfig();
   }
 
   /**
@@ -132,16 +131,16 @@ public class TestSqoopClient {
   public void testGetConnectors() {
     MConnector connector;
 
-    when(requests.readConnector(null)).thenReturn(connectorBean(connector(1), connector(2)));
+    when(resourceRequests.readConnector(null)).thenReturn(connectorBean(connector(1), connector(2)));
     Collection<MConnector> connectors = client.getConnectors();
     assertEquals(2, connectors.size());
 
-    client.getResourceBundle(1);
+    client.getConnectorConfigResourceBundle(1);
     connector = client.getConnector(1);
     assertEquals(1, connector.getPersistenceId());
 
     connector = client.getConnector(2);
-    client.getResourceBundle(2);
+    client.getConnectorConfigResourceBundle(2);
     assertEquals(2, connector.getPersistenceId());
 
     connectors = client.getConnectors();
@@ -158,8 +157,8 @@ public class TestSqoopClient {
     connector = client.getConnector("A3");
     assertNull(connector);
 
-    verify(requests, times(1)).readConnector(null);
-    verifyNoMoreInteractions(requests);
+    verify(resourceRequests, times(1)).readConnector(null);
+    verifyNoMoreInteractions(resourceRequests);
   }
 
 
@@ -170,32 +169,32 @@ public class TestSqoopClient {
   @Test
   public void testGetConnectorOneByOne() {
     ConnectorBean bean = connectorBean(connector(1), connector(2));
-    when(requests.readConnector(null)).thenReturn(bean);
-    when(requests.readConnector(1L)).thenReturn(bean);
-    when(requests.readConnector(2L)).thenReturn(bean);
+    when(resourceRequests.readConnector(null)).thenReturn(bean);
+    when(resourceRequests.readConnector(1L)).thenReturn(bean);
+    when(resourceRequests.readConnector(2L)).thenReturn(bean);
 
-    client.getResourceBundle(1);
+    client.getConnectorConfigResourceBundle(1);
     client.getConnector(1);
 
     client.getConnector(2);
-    client.getResourceBundle(2);
+    client.getConnectorConfigResourceBundle(2);
 
     Collection<MConnector> connectors = client.getConnectors();
     assertEquals(2, connectors.size());
 
-    verify(requests, times(1)).readConnector(null);
-    verify(requests, times(1)).readConnector(1L);
-    verify(requests, times(1)).readConnector(2L);
-    verifyNoMoreInteractions(requests);
+    verify(resourceRequests, times(1)).readConnector(null);
+    verify(resourceRequests, times(1)).readConnector(1L);
+    verify(resourceRequests, times(1)).readConnector(2L);
+    verifyNoMoreInteractions(resourceRequests);
   }
 
   /**
-   * Connection for non-existing connector can't be created.
+   * Link for non-existing connector can't be created.
    */
   @Test(expected = SqoopException.class)
-  public void testNewConnection() {
-    when(requests.readConnector(null)).thenReturn(connectorBean(connector(1)));
-    client.newConnection("non existing connector");
+  public void testCreateLink() {
+    when(resourceRequests.readConnector(null)).thenReturn(connectorBean(connector(1)));
+    client.createLink("non existing connector");
   }
 
   private ConnectorBean connectorBean(MConnector...connectors) {
@@ -208,8 +207,8 @@ public class TestSqoopClient {
     }
     return new ConnectorBean(connectorList, bundles);
   }
-  private FrameworkBean frameworkBean(MFramework framework) {
-    return new FrameworkBean(framework, new MapResourceBundle(null));
+  private DriverConfigBean driverConfigBean(MDriverConfig driverConfig) {
+    return new DriverConfigBean(driverConfig, new MapResourceBundle(null));
   }
 
   private MConnector connector(long id) {
@@ -219,10 +218,10 @@ public class TestSqoopClient {
     return connector;
   }
 
-  private MFramework framework() {
-    MFramework framework = new MFramework(new MConnectionForms(null),
+  private MDriverConfig driverConfig() {
+    MDriverConfig driverConfig = new MDriverConfig(new MConnectionForms(null),
         new MJobForms(null), "1");
-    framework.setPersistenceId(1);
-    return framework;
+    driverConfig.setPersistenceId(1);
+    return driverConfig;
   }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/common/SqoopException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/common/SqoopException.java b/common/src/main/java/org/apache/sqoop/common/SqoopException.java
index 98f9dc7..6c5dc2a 100644
--- a/common/src/main/java/org/apache/sqoop/common/SqoopException.java
+++ b/common/src/main/java/org/apache/sqoop/common/SqoopException.java
@@ -19,7 +19,7 @@ package org.apache.sqoop.common;
 
 
 /**
- * Base exception for Sqoop framework. This exception requires the specification
+ * Base exception for Sqoop driver. This exception requires the specification
  * of an error code for reference purposes. Where necessary the appropriate
  * constructor can be used to pass in additional message beyond what is
  * specified by the error code and/or the causal exception.

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/etl/io/DataReader.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/etl/io/DataReader.java b/common/src/main/java/org/apache/sqoop/etl/io/DataReader.java
index a34dfb4..a555431 100644
--- a/common/src/main/java/org/apache/sqoop/etl/io/DataReader.java
+++ b/common/src/main/java/org/apache/sqoop/etl/io/DataReader.java
@@ -18,20 +18,20 @@
 package org.apache.sqoop.etl.io;
 
 /**
- * An intermediate layer for passing data from the execution framework
- * to the ETL framework.
+ * An intermediate layer for passing data from the execution engine
+ * to the ETL engine.
  */
 public abstract class DataReader {
 
   /**
-   * Read data from the execution framework as an object array.
+   * Read data from the execution engine as an object array.
    * @return - array of objects with each column represented as an object
    * @throws Exception
    */
   public abstract Object[] readArrayRecord() throws Exception;
 
   /**
-   * Read data from execution framework as text - as a CSV record.
+   * Read data from execution engine as text - as a CSV record.
    * public abstract Object readContent(int type) throws Exception;
    * @return - CSV formatted data.
    * @throws Exception
@@ -39,7 +39,7 @@ public abstract class DataReader {
   public abstract String readTextRecord() throws Exception;
 
   /**
-   * Read data from execution framework as a native format.
+   * Read data from execution engine as a native format.
    * @return - the content in the native format of the intermediate data
    * format being used.
    * @throws Exception

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/job/etl/ActorContext.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/job/etl/ActorContext.java b/common/src/main/java/org/apache/sqoop/job/etl/ActorContext.java
deleted file mode 100644
index 98b2f5e..0000000
--- a/common/src/main/java/org/apache/sqoop/job/etl/ActorContext.java
+++ /dev/null
@@ -1,77 +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.job.etl;
-
-import org.apache.sqoop.common.ImmutableContext;
-
-/**
- * Basic context class for each actor containing only the connector/framework
- * context object.
- */
-public abstract class ActorContext {
-
-  ImmutableContext context;
-
-  public ActorContext(ImmutableContext context) {
-    this.context = context;
-  }
-
-  /**
-   * Context object associated with the particular actor
-   *
-   * @return
-   */
-  public ImmutableContext getContext() {
-    return context;
-  }
-
-  /**
-   * Convenience method that will return value from wrapped context class.
-   */
-  public String getString(String key) {
-    return context.getString(key);
-  }
-
-  /**
-   * Convenience method that will return value from wrapped context class.
-   */
-  public String getString(String key, String defaultValue) {
-    return context.getString(key, defaultValue);
-  }
-
-  /**
-   * Convenience method that will return value from wrapped context class.
-   */
-  public long getLong(String key, long defaultValue) {
-    return context.getLong(key, defaultValue);
-  }
-
-  /**
-   * Convenience method that will return value from wrapped context class.
-   */
-  public int getInt(String key, int defaultValue) {
-    return context.getInt(key, defaultValue);
-  }
-
-  /**
-   * Convenience method that will return value from wrapped context class.
-   */
-  public boolean getBoolean(String key, boolean defaultValue) {
-    return context.getBoolean(key, defaultValue);
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/job/etl/DestroyerContext.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/job/etl/DestroyerContext.java b/common/src/main/java/org/apache/sqoop/job/etl/DestroyerContext.java
index 2f29de4..7afb120 100644
--- a/common/src/main/java/org/apache/sqoop/job/etl/DestroyerContext.java
+++ b/common/src/main/java/org/apache/sqoop/job/etl/DestroyerContext.java
@@ -25,7 +25,7 @@ import org.apache.sqoop.schema.Schema;
  *
  * This class is wrapping information if the run was successful or not.
  */
-public class DestroyerContext extends ActorContext {
+public class DestroyerContext extends TransferableContext {
 
   private boolean success;
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/job/etl/ExtractorContext.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/job/etl/ExtractorContext.java b/common/src/main/java/org/apache/sqoop/job/etl/ExtractorContext.java
index af03f0a..fd73890 100644
--- a/common/src/main/java/org/apache/sqoop/job/etl/ExtractorContext.java
+++ b/common/src/main/java/org/apache/sqoop/job/etl/ExtractorContext.java
@@ -26,7 +26,7 @@ import org.apache.sqoop.schema.Schema;
  *
  * This class is wrapping writer object.
  */
-public class ExtractorContext extends ActorContext {
+public class ExtractorContext extends TransferableContext {
 
   private DataWriter writer;
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/job/etl/InitializerContext.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/job/etl/InitializerContext.java b/common/src/main/java/org/apache/sqoop/job/etl/InitializerContext.java
index d2e2dfc..4de00a6 100644
--- a/common/src/main/java/org/apache/sqoop/job/etl/InitializerContext.java
+++ b/common/src/main/java/org/apache/sqoop/job/etl/InitializerContext.java
@@ -25,7 +25,7 @@ import org.apache.sqoop.common.MutableContext;
  *
  * This class is returning mutable context instead of immutable.
  */
-public class InitializerContext extends ActorContext {
+public class InitializerContext extends TransferableContext {
 
   public InitializerContext(MutableContext context) {
     super(context);

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/job/etl/LoaderContext.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/job/etl/LoaderContext.java b/common/src/main/java/org/apache/sqoop/job/etl/LoaderContext.java
index f2e6b97..563b9ad 100644
--- a/common/src/main/java/org/apache/sqoop/job/etl/LoaderContext.java
+++ b/common/src/main/java/org/apache/sqoop/job/etl/LoaderContext.java
@@ -26,7 +26,7 @@ import org.apache.sqoop.schema.Schema;
  *
  * This class is also wrapping reader object.
  */
-public class LoaderContext extends ActorContext {
+public class LoaderContext extends TransferableContext {
 
   private DataReader reader;
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/job/etl/PartitionerContext.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/job/etl/PartitionerContext.java b/common/src/main/java/org/apache/sqoop/job/etl/PartitionerContext.java
index e7daeee..79901fd 100644
--- a/common/src/main/java/org/apache/sqoop/job/etl/PartitionerContext.java
+++ b/common/src/main/java/org/apache/sqoop/job/etl/PartitionerContext.java
@@ -25,7 +25,7 @@ import org.apache.sqoop.schema.Schema;
  *
  * This class is also wrapping number of maximal allowed partitions.
  */
-public class PartitionerContext extends ActorContext {
+public class PartitionerContext extends TransferableContext {
 
   private long maxPartitions;
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/job/etl/TransferableContext.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/job/etl/TransferableContext.java b/common/src/main/java/org/apache/sqoop/job/etl/TransferableContext.java
new file mode 100644
index 0000000..e57bf45
--- /dev/null
+++ b/common/src/main/java/org/apache/sqoop/job/etl/TransferableContext.java
@@ -0,0 +1,76 @@
+/**
+ * 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.job.etl;
+
+import org.apache.sqoop.common.ImmutableContext;
+
+/**
+ * Base context class for the {@link Transferable} components
+ */
+public abstract class TransferableContext {
+
+  ImmutableContext context;
+
+  public TransferableContext(ImmutableContext context) {
+    this.context = context;
+  }
+
+  /**
+   * Context object associated with the particular actor
+   *
+   * @return
+   */
+  public ImmutableContext getContext() {
+    return context;
+  }
+
+  /**
+   * Convenience method that will return value from wrapped context class.
+   */
+  public String getString(String key) {
+    return context.getString(key);
+  }
+
+  /**
+   * Convenience method that will return value from wrapped context class.
+   */
+  public String getString(String key, String defaultValue) {
+    return context.getString(key, defaultValue);
+  }
+
+  /**
+   * Convenience method that will return value from wrapped context class.
+   */
+  public long getLong(String key, long defaultValue) {
+    return context.getLong(key, defaultValue);
+  }
+
+  /**
+   * Convenience method that will return value from wrapped context class.
+   */
+  public int getInt(String key, int defaultValue) {
+    return context.getInt(key, defaultValue);
+  }
+
+  /**
+   * Convenience method that will return value from wrapped context class.
+   */
+  public boolean getBoolean(String key, boolean defaultValue) {
+    return context.getBoolean(key, defaultValue);
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/json/ConnectionBean.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/json/ConnectionBean.java b/common/src/main/java/org/apache/sqoop/json/ConnectionBean.java
deleted file mode 100644
index 688bb64..0000000
--- a/common/src/main/java/org/apache/sqoop/json/ConnectionBean.java
+++ /dev/null
@@ -1,185 +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.json;
-
-import org.apache.sqoop.model.MConnection;
-import org.apache.sqoop.model.MConnectionForms;
-import org.apache.sqoop.model.MForm;
-import org.json.simple.JSONArray;
-import org.json.simple.JSONObject;
-
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.ResourceBundle;
-import java.util.Set;
-
-import static org.apache.sqoop.json.util.FormSerialization.*;
-import static org.apache.sqoop.json.util.ResourceBundleSerialization.*;
-
-/**
- * Connection representation that is being send across the network between
- * Sqoop server and client. Server might optionally send resource bundles
- * associated with the connections to spare client of sending another HTTP
- * requests to obtain them.
- */
-public class ConnectionBean implements JsonBean {
-
-  private static final String CONNECTOR_ID = "connector-id";
-  private static final String CONNECTOR_PART = "connector";
-  private static final String FRAMEWORK_PART = "framework";
-
-  // Compulsory
-  private List<MConnection> connections;
-
-  // Optional
-  private Map<Long, ResourceBundle> connectorBundles;
-  private ResourceBundle frameworkBundle;
-
-  // For "extract"
-  public ConnectionBean(MConnection connection) {
-    this();
-    this.connections = new ArrayList<MConnection>();
-    this.connections.add(connection);
-  }
-
-  public ConnectionBean(List<MConnection> connections) {
-    this();
-    this.connections = connections;
-  }
-
-  // For "restore"
-  public ConnectionBean() {
-    connectorBundles = new HashMap<Long, ResourceBundle>();
-  }
-
-  public void setFrameworkBundle(ResourceBundle frameworkBundle) {
-    this.frameworkBundle = frameworkBundle;
-  }
-
-  public void addConnectorBundle(Long id, ResourceBundle connectorBundle) {
-    connectorBundles.put(id, connectorBundle);
-  }
-
-  public boolean hasConnectorBundle(Long id) {
-    return connectorBundles.containsKey(id);
-  }
-
-  public List<MConnection> getConnections() {
-    return connections;
-  }
-
-  public ResourceBundle getConnectorBundle(Long id) {
-    return connectorBundles.get(id);
-  }
-
-  public ResourceBundle getFrameworkBundle() {
-    return frameworkBundle;
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public JSONObject extract(boolean skipSensitive) {
-    JSONArray array = new JSONArray();
-
-    for(MConnection connection : connections) {
-      JSONObject object = new JSONObject();
-
-      object.put(ID, connection.getPersistenceId());
-      object.put(NAME, connection.getName());
-      object.put(ENABLED, connection.getEnabled());
-      object.put(CREATION_USER, connection.getCreationUser());
-      object.put(CREATION_DATE, connection.getCreationDate().getTime());
-      object.put(UPDATE_USER, connection.getLastUpdateUser());
-      object.put(UPDATE_DATE, connection.getLastUpdateDate().getTime());
-      object.put(CONNECTOR_ID, connection.getConnectorId());
-      object.put(CONNECTOR_PART,
-        extractForms(connection.getConnectorPart().getForms(), skipSensitive));
-      object.put(FRAMEWORK_PART,
-        extractForms(connection.getFrameworkPart().getForms(), skipSensitive));
-
-      array.add(object);
-    }
-
-    JSONObject all = new JSONObject();
-    all.put(ALL, array);
-
-    if(!connectorBundles.isEmpty()) {
-      JSONObject bundles = new JSONObject();
-
-      for(Map.Entry<Long, ResourceBundle> entry : connectorBundles.entrySet()) {
-        bundles.put(entry.getKey().toString(),
-                    extractResourceBundle(entry.getValue()));
-      }
-
-      all.put(CONNECTOR_RESOURCES, bundles);
-    }
-    if(frameworkBundle != null) {
-      all.put(FRAMEWORK_RESOURCES,extractResourceBundle(frameworkBundle));
-    }
-    return all;
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public void restore(JSONObject jsonObject) {
-    connections = new ArrayList<MConnection>();
-
-    JSONArray array = (JSONArray) jsonObject.get(ALL);
-
-    for (Object obj : array) {
-      JSONObject object = (JSONObject) obj;
-
-      long connectorId = (Long) object.get(CONNECTOR_ID);
-      JSONArray connectorPart = (JSONArray) object.get(CONNECTOR_PART);
-      JSONArray frameworkPart = (JSONArray) object.get(FRAMEWORK_PART);
-
-      List<MForm> connectorForms = restoreForms(connectorPart);
-      List<MForm> frameworkForms = restoreForms(frameworkPart);
-
-      MConnection connection = new MConnection(connectorId,
-        new MConnectionForms(connectorForms),
-        new MConnectionForms(frameworkForms));
-
-      connection.setPersistenceId((Long) object.get(ID));
-      connection.setName((String) object.get(NAME));
-      connection.setEnabled((Boolean) object.get(ENABLED));
-      connection.setCreationUser((String) object.get(CREATION_USER));
-      connection.setCreationDate(new Date((Long) object.get(CREATION_DATE)));
-      connection.setLastUpdateUser((String) object.get(UPDATE_USER));
-      connection.setLastUpdateDate(new Date((Long) object.get(UPDATE_DATE)));
-
-      connections.add(connection);
-    }
-
-    if(jsonObject.containsKey(CONNECTOR_RESOURCES)) {
-      JSONObject bundles = (JSONObject) jsonObject.get(CONNECTOR_RESOURCES);
-      Set<Map.Entry<String, JSONObject>> entrySet = bundles.entrySet();
-      for (Map.Entry<String, JSONObject> entry : entrySet) {
-        connectorBundles.put(Long.parseLong(entry.getKey()),
-                             restoreResourceBundle(entry.getValue()));
-      }
-    }
-    if(jsonObject.containsKey(FRAMEWORK_RESOURCES)) {
-      frameworkBundle = restoreResourceBundle(
-        (JSONObject) jsonObject.get(FRAMEWORK_RESOURCES));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/json/ConnectionValidationBean.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/json/ConnectionValidationBean.java b/common/src/main/java/org/apache/sqoop/json/ConnectionValidationBean.java
deleted file mode 100644
index ffdd13e..0000000
--- a/common/src/main/java/org/apache/sqoop/json/ConnectionValidationBean.java
+++ /dev/null
@@ -1,143 +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.json;
-
-import org.apache.sqoop.validation.Status;
-import org.apache.sqoop.validation.Validation;
-import org.json.simple.JSONObject;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Bean for sending validations across network. This bean will move two
- * validation objects at one time - one for connector and second for framework
- * part of validated entity. Optionally validation bean can also transfer
- * created persistent id in case that new entity was created.
- */
-public class ConnectionValidationBean implements JsonBean {
-
-  private static final String ID = "id";
-  private static final String FRAMEWORK = "framework";
-  private static final String CONNECTOR = "connector";
-  private static final String STATUS = "status";
-  private static final String MESSAGE = "message";
-  private static final String MESSAGES = "messages";
-
-  private Long id;
-  private Validation connectorValidation;
-  private Validation frameworkValidation;
-
-  // For "extract"
-  public ConnectionValidationBean(Validation connector, Validation framework) {
-    this();
-
-    this.connectorValidation = connector;
-    this.frameworkValidation = framework;
-  }
-
-  // For "restore"
-  public ConnectionValidationBean() {
-    id = null;
-  }
-
-  public Validation getConnectorValidation() {
-    return connectorValidation;
-  }
-
-  public Validation getFrameworkValidation() {
-    return frameworkValidation;
-  }
-
-  public void setId(Long id) {
-    this.id = id;
-  }
-
-  public Long getId() {
-    return id;
-  }
-
-  @SuppressWarnings("unchecked")
-  public JSONObject extract(boolean skipSensitive) {
-    JSONObject object = new JSONObject();
-
-    // Optionally transfer id
-    if(id != null) {
-      object.put(ID, id);
-    }
-
-    object.put(CONNECTOR, extractValidation(connectorValidation));
-    object.put(FRAMEWORK, extractValidation(frameworkValidation));
-
-    return object;
-  }
-
-  @SuppressWarnings("unchecked")
-  private JSONObject extractValidation(Validation validation) {
-    JSONObject object = new JSONObject();
-
-    object.put(STATUS, validation.getStatus().name());
-
-    JSONObject jsonMessages = new JSONObject();
-    Map<Validation.FormInput, Validation.Message> messages = validation.getMessages();
-
-    for(Map.Entry<Validation.FormInput, Validation.Message> entry : messages.entrySet()) {
-      JSONObject jsonEntry = new JSONObject();
-      jsonEntry.put(STATUS, entry.getValue().getStatus().name());
-      jsonEntry.put(MESSAGE, entry.getValue().getMessage());
-      jsonMessages.put(entry.getKey(), jsonEntry);
-    }
-
-    object.put(MESSAGES, jsonMessages);
-
-    return object;
-  }
-
-  @Override
-  public void restore(JSONObject jsonObject) {
-    // Optional and accepting NULLs
-    id = (Long) jsonObject.get(ID);
-
-    connectorValidation = restoreValidation(
-      (JSONObject)jsonObject.get(CONNECTOR));
-    frameworkValidation = restoreValidation(
-      (JSONObject)jsonObject.get(FRAMEWORK));
-  }
-
-  public Validation restoreValidation(JSONObject jsonObject) {
-    JSONObject jsonMessages = (JSONObject) jsonObject.get(MESSAGES);
-    Map<Validation.FormInput, Validation.Message> messages
-      = new HashMap<Validation.FormInput, Validation.Message>();
-
-    for(Object key : jsonMessages.keySet()) {
-      JSONObject jsonMessage = (JSONObject) jsonMessages.get(key);
-
-      Status status = Status.valueOf((String) jsonMessage.get(STATUS));
-      String stringMessage = (String) jsonMessage.get(MESSAGE);
-
-      Validation.Message message
-        = new Validation.Message(status, stringMessage);
-
-      messages.put(new Validation.FormInput((String)key), message);
-    }
-
-    Status status = Status.valueOf((String) jsonObject.get(STATUS));
-
-    return new Validation(status, messages);
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/json/ConnectorBean.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/json/ConnectorBean.java b/common/src/main/java/org/apache/sqoop/json/ConnectorBean.java
index 5069f0b..465dd69 100644
--- a/common/src/main/java/org/apache/sqoop/json/ConnectorBean.java
+++ b/common/src/main/java/org/apache/sqoop/json/ConnectorBean.java
@@ -17,6 +17,19 @@
  */
 package org.apache.sqoop.json;
 
+import static org.apache.sqoop.json.util.FormSerialization.ALL;
+import static org.apache.sqoop.json.util.FormSerialization.CLASS;
+import static org.apache.sqoop.json.util.FormSerialization.CON_FORMS;
+import static org.apache.sqoop.json.util.FormSerialization.ID;
+import static org.apache.sqoop.json.util.FormSerialization.JOB_FORMS;
+import static org.apache.sqoop.json.util.FormSerialization.NAME;
+import static org.apache.sqoop.json.util.FormSerialization.VERSION;
+import static org.apache.sqoop.json.util.FormSerialization.extractForms;
+import static org.apache.sqoop.json.util.FormSerialization.restoreForms;
+import static org.apache.sqoop.json.util.ResourceBundleSerialization.CONNECTOR_CONFIGS;
+import static org.apache.sqoop.json.util.ResourceBundleSerialization.extractResourceBundle;
+import static org.apache.sqoop.json.util.ResourceBundleSerialization.restoreResourceBundle;
+
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -26,15 +39,12 @@ import java.util.Set;
 
 import org.apache.sqoop.common.Direction;
 import org.apache.sqoop.model.MConnectionForms;
-import org.apache.sqoop.model.MJobForms;
 import org.apache.sqoop.model.MConnector;
 import org.apache.sqoop.model.MForm;
+import org.apache.sqoop.model.MJobForms;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
 
-import static org.apache.sqoop.json.util.FormSerialization.*;
-import static org.apache.sqoop.json.util.ResourceBundleSerialization.*;
-
 public class ConnectorBean implements JsonBean {
 
   private List<MConnector> connectors;
@@ -93,8 +103,7 @@ public class ConnectorBean implements JsonBean {
         jsonBundles.put(entry.getKey().toString(),
                          extractResourceBundle(entry.getValue()));
       }
-
-      all.put(CONNECTOR_RESOURCES, jsonBundles);
+      all.put(CONNECTOR_CONFIGS, jsonBundles);
     }
 
     return all;
@@ -116,28 +125,24 @@ public class ConnectorBean implements JsonBean {
       String version = (String) object.get(VERSION);
 
       List<MForm> connForms = restoreForms((JSONArray) object.get(CON_FORMS));
-
       JSONObject jobJson = (JSONObject) object.get(JOB_FORMS);
       JSONArray fromJobJson = (JSONArray)jobJson.get(Direction.FROM.name());
       JSONArray toJobJson = (JSONArray)jobJson.get(Direction.TO.name());
-      List<MForm> fromJobForms =
-          restoreForms(fromJobJson);
-      List<MForm> toJobForms =
-          restoreForms(toJobJson);
+      List<MForm> fromJobForms = restoreForms(fromJobJson);
+      List<MForm> toJobForms = restoreForms(toJobJson);
       MJobForms fromJob = new MJobForms(fromJobForms);
       MJobForms toJob = new MJobForms(toJobForms);
       MConnectionForms connection = new MConnectionForms(connForms);
-
-      MConnector connector = new MConnector(uniqueName, className, version, connection, fromJob, toJob);
+      MConnector connector = new MConnector(uniqueName, className, version, connection, fromJob,
+          toJob);
       connector.setPersistenceId(connectorId);
-
       connectors.add(connector);
     }
 
-    if(jsonObject.containsKey(CONNECTOR_RESOURCES)) {
+    if(jsonObject.containsKey(CONNECTOR_CONFIGS)) {
       bundles = new HashMap<Long, ResourceBundle>();
 
-      JSONObject jsonBundles = (JSONObject) jsonObject.get(CONNECTOR_RESOURCES);
+      JSONObject jsonBundles = (JSONObject) jsonObject.get(CONNECTOR_CONFIGS);
       Set<Map.Entry<String, JSONObject>> entrySet = jsonBundles.entrySet();
       for (Map.Entry<String, JSONObject> entry : entrySet) {
         bundles.put(Long.parseLong(entry.getKey()),

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/json/DriverConfigBean.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/json/DriverConfigBean.java b/common/src/main/java/org/apache/sqoop/json/DriverConfigBean.java
new file mode 100644
index 0000000..a2cc8b8
--- /dev/null
+++ b/common/src/main/java/org/apache/sqoop/json/DriverConfigBean.java
@@ -0,0 +1,92 @@
+/**
+ * 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.json;
+
+import org.apache.sqoop.model.MConnectionForms;
+import org.apache.sqoop.model.MForm;
+import org.apache.sqoop.model.MDriverConfig;
+import org.apache.sqoop.model.MJobForms;
+import org.json.simple.JSONArray;
+import org.json.simple.JSONObject;
+
+import java.util.List;
+import java.util.ResourceBundle;
+
+import static org.apache.sqoop.json.util.FormSerialization.*;
+import static org.apache.sqoop.json.util.ResourceBundleSerialization.*;
+
+public class DriverConfigBean implements JsonBean {
+
+  private MDriverConfig driverConfig;
+
+  private ResourceBundle bundle;
+
+  // for "extract"
+  public DriverConfigBean(MDriverConfig driverConfig, ResourceBundle bundle) {
+    this.driverConfig = driverConfig;
+    this.bundle = bundle;
+  }
+
+  // for "restore"
+  public DriverConfigBean() {
+  }
+
+  public MDriverConfig getDriverConfig() {
+    return driverConfig;
+  }
+
+  public ResourceBundle getResourceBundle() {
+    return bundle;
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public JSONObject extract(boolean skipSensitive) {
+    // TODO(Abe): Add From/To connection forms.
+    JSONArray conForms =
+      extractForms(driverConfig.getConnectionForms().getForms(), skipSensitive);
+    JSONArray jobForms = extractForms(driverConfig.getJobForms().getForms(), skipSensitive);
+
+    JSONObject result = new JSONObject();
+    result.put(ID, driverConfig.getPersistenceId());
+    result.put(DRIVER_VERSION, driverConfig.getVersion());
+    result.put(CON_FORMS, conForms);
+    result.put(JOB_FORMS, jobForms);
+    result.put(CONFIGS, extractResourceBundle(bundle));
+    return result;
+  }
+
+  @Override
+  public void restore(JSONObject jsonObject) {
+    long id = (Long) jsonObject.get(ID);
+    String driverVersion = (String) jsonObject.get(DRIVER_VERSION);
+
+    List<MForm> connForms = restoreForms((JSONArray) jsonObject.get(CON_FORMS));
+    List<MForm> jobForms = restoreForms((JSONArray) jsonObject.get(JOB_FORMS));
+
+    // TODO(Abe): Get From/To connection forms.
+    driverConfig = new MDriverConfig(
+        new MConnectionForms(connForms),
+        new MJobForms(jobForms),
+        driverVersion);
+    driverConfig.setPersistenceId(id);
+
+    bundle = restoreResourceBundle((JSONObject) jsonObject.get(CONFIGS));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/json/FrameworkBean.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/json/FrameworkBean.java b/common/src/main/java/org/apache/sqoop/json/FrameworkBean.java
deleted file mode 100644
index abbdcc6..0000000
--- a/common/src/main/java/org/apache/sqoop/json/FrameworkBean.java
+++ /dev/null
@@ -1,102 +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.json;
-
-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.MJob;
-import org.apache.sqoop.model.MJobForms;
-import org.json.simple.JSONArray;
-import org.json.simple.JSONObject;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.ResourceBundle;
-import java.util.Set;
-
-import static org.apache.sqoop.json.util.FormSerialization.*;
-import static org.apache.sqoop.json.util.ResourceBundleSerialization.*;
-
-/**
- *
- */
-public class FrameworkBean implements JsonBean {
-
-
-  private MFramework framework;
-
-  private ResourceBundle bundle;
-
-  // for "extract"
-  public FrameworkBean(MFramework framework, ResourceBundle bundle) {
-    this.framework = framework;
-    this.bundle = bundle;
-  }
-
-  // for "restore"
-  public FrameworkBean() {
-  }
-
-  public MFramework getFramework() {
-    return framework;
-  }
-
-  public ResourceBundle getResourceBundle() {
-    return bundle;
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public JSONObject extract(boolean skipSensitive) {
-    // @TODO(Abe): Add From/To connection forms.
-    JSONArray conForms =
-      extractForms(framework.getConnectionForms().getForms(), skipSensitive);
-    JSONArray jobForms = extractForms(framework.getJobForms().getForms(), skipSensitive);
-
-    JSONObject result = new JSONObject();
-    result.put(ID, framework.getPersistenceId());
-    result.put(FRAMEWORK_VERSION, framework.getVersion());
-    result.put(CON_FORMS, conForms);
-    result.put(JOB_FORMS, jobForms);
-    result.put(RESOURCES, extractResourceBundle(bundle));
-    return result;
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public void restore(JSONObject jsonObject) {
-    long id = (Long) jsonObject.get(ID);
-    String frameworkVersion = (String) jsonObject.get(FRAMEWORK_VERSION);
-
-    List<MForm> connForms = restoreForms((JSONArray) jsonObject.get(CON_FORMS));
-    List<MForm> jobForms = restoreForms((JSONArray) jsonObject.get(JOB_FORMS));
-
-    // @TODO(Abe): Get From/To connection forms.
-    framework = new MFramework(
-        new MConnectionForms(connForms),
-        new MJobForms(jobForms),
-        frameworkVersion);
-    framework.setPersistenceId(id);
-
-    bundle = restoreResourceBundle((JSONObject) jsonObject.get(RESOURCES));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/json/JobBean.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/json/JobBean.java b/common/src/main/java/org/apache/sqoop/json/JobBean.java
index 91edcfc..a3e6071 100644
--- a/common/src/main/java/org/apache/sqoop/json/JobBean.java
+++ b/common/src/main/java/org/apache/sqoop/json/JobBean.java
@@ -43,20 +43,20 @@ public class JobBean implements JsonBean {
   private static final String ALL = "all";
   private static final String ID = "id";
   private static final String NAME = "name";
-  private static final String FROM_CONNECTION_ID = "from-connection-id";
-  private static final String TO_CONNECTION_ID = "to-connection-id";
+  private static final String FROM_LINK_ID = "from-link-id";
+  private static final String TO_LINK_ID = "to-link-id";
   private static final String FROM_CONNECTOR_ID = "from-connector-id";
   private static final String TO_CONNECTOR_ID = "to-connector-id";
   private static final String FROM_CONNECTOR_PART = "from-connector";
   private static final String TO_CONNECTOR_PART = "to-connector";
   private static final String FRAMEWORK_PART = "framework";
 
-  // Compulsory
+  // Required
   private List<MJob> jobs;
 
   // Optional
-  private Map<Long, ResourceBundle> connectorBundles;
-  private ResourceBundle frameworkBundle;
+  private Map<Long, ResourceBundle> connectorConfigBundles;
+  private ResourceBundle driverConfigBundle;
 
   // For "extract"
   public JobBean(MJob job) {
@@ -72,31 +72,31 @@ public class JobBean implements JsonBean {
 
   // For "restore"
   public JobBean() {
-    connectorBundles = new HashMap<Long, ResourceBundle>();
+    connectorConfigBundles = new HashMap<Long, ResourceBundle>();
   }
 
-  public void setFrameworkBundle(ResourceBundle frameworkBundle) {
-    this.frameworkBundle = frameworkBundle;
+  public void setDriverConfigBundle(ResourceBundle driverConfigBundle) {
+    this.driverConfigBundle = driverConfigBundle;
   }
 
-  public void addConnectorBundle(Long id, ResourceBundle connectorBundle) {
-    connectorBundles.put(id, connectorBundle);
+  public void addConnectorConfigBundle(Long id, ResourceBundle connectorConfigBundle) {
+    connectorConfigBundles.put(id, connectorConfigBundle);
   }
 
-  public boolean hasConnectorBundle(Long id) {
-    return connectorBundles.containsKey(id);
+  public boolean hasConnectorConfigBundle(Long id) {
+    return connectorConfigBundles.containsKey(id);
   }
 
   public List<MJob> getJobs() {
     return jobs;
   }
 
-  public ResourceBundle getConnectorBundle(Long id) {
-    return connectorBundles.get(id);
+  public ResourceBundle getConnectorConfigBundle(Long id) {
+    return connectorConfigBundles.get(id);
   }
 
-  public ResourceBundle getFrameworkBundle() {
-    return frameworkBundle;
+  public ResourceBundle getDriverConfigBundle() {
+    return driverConfigBundle;
   }
 
   @Override
@@ -114,8 +114,8 @@ public class JobBean implements JsonBean {
       object.put(CREATION_DATE, job.getCreationDate().getTime());
       object.put(UPDATE_USER, job.getLastUpdateUser());
       object.put(UPDATE_DATE, job.getLastUpdateDate().getTime());
-      object.put(FROM_CONNECTION_ID, job.getConnectionId(Direction.FROM));
-      object.put(TO_CONNECTION_ID, job.getConnectionId(Direction.TO));
+      object.put(FROM_LINK_ID, job.getLinkId(Direction.FROM));
+      object.put(TO_LINK_ID, job.getLinkId(Direction.TO));
       object.put(FROM_CONNECTOR_ID, job.getConnectorId(Direction.FROM));
       object.put(TO_CONNECTOR_ID, job.getConnectorId(Direction.TO));
       object.put(FROM_CONNECTOR_PART,
@@ -131,18 +131,17 @@ public class JobBean implements JsonBean {
     JSONObject all = new JSONObject();
     all.put(ALL, array);
 
-    if(!connectorBundles.isEmpty()) {
+    if(!connectorConfigBundles.isEmpty()) {
       JSONObject bundles = new JSONObject();
 
-      for(Map.Entry<Long, ResourceBundle> entry : connectorBundles.entrySet()) {
+      for(Map.Entry<Long, ResourceBundle> entry : connectorConfigBundles.entrySet()) {
         bundles.put(entry.getKey().toString(),
                     extractResourceBundle(entry.getValue()));
       }
-
-      all.put(CONNECTOR_RESOURCES, bundles);
+      all.put(CONNECTOR_CONFIGS, bundles);
     }
-    if(frameworkBundle != null) {
-      all.put(FRAMEWORK_RESOURCES,extractResourceBundle(frameworkBundle));
+    if(driverConfigBundle != null) {
+      all.put(DRIVER_CONFIGS,extractResourceBundle(driverConfigBundle));
     }
     return all;
   }
@@ -159,8 +158,8 @@ public class JobBean implements JsonBean {
 
       long fromConnectorId = (Long) object.get(FROM_CONNECTOR_ID);
       long toConnectorId = (Long) object.get(TO_CONNECTOR_ID);
-      long fromConnectionId = (Long) object.get(FROM_CONNECTION_ID);
-      long toConnectionId = (Long) object.get(TO_CONNECTION_ID);
+      long fromConnectionId = (Long) object.get(FROM_LINK_ID);
+      long toConnectionId = (Long) object.get(TO_LINK_ID);
       JSONArray fromConnectorPart = (JSONArray) object.get(FROM_CONNECTOR_PART);
       JSONArray toConnectorPart = (JSONArray) object.get(TO_CONNECTOR_PART);
       JSONArray frameworkPart = (JSONArray) object.get(FRAMEWORK_PART);
@@ -190,17 +189,17 @@ public class JobBean implements JsonBean {
       jobs.add(job);
     }
 
-    if(jsonObject.containsKey(CONNECTOR_RESOURCES)) {
-      JSONObject bundles = (JSONObject) jsonObject.get(CONNECTOR_RESOURCES);
+    if(jsonObject.containsKey(CONNECTOR_CONFIGS)) {
+      JSONObject bundles = (JSONObject) jsonObject.get(CONNECTOR_CONFIGS);
       Set<Map.Entry<String, JSONObject>> entrySet = bundles.entrySet();
       for (Map.Entry<String, JSONObject> entry : entrySet) {
-        connectorBundles.put(Long.parseLong(entry.getKey()),
+        connectorConfigBundles.put(Long.parseLong(entry.getKey()),
                              restoreResourceBundle(entry.getValue()));
       }
     }
-    if(jsonObject.containsKey(FRAMEWORK_RESOURCES)) {
-      frameworkBundle = restoreResourceBundle(
-        (JSONObject) jsonObject.get(FRAMEWORK_RESOURCES));
+    if(jsonObject.containsKey(DRIVER_CONFIGS)) {
+      driverConfigBundle = restoreResourceBundle(
+        (JSONObject) jsonObject.get(DRIVER_CONFIGS));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/json/LinkBean.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/json/LinkBean.java b/common/src/main/java/org/apache/sqoop/json/LinkBean.java
new file mode 100644
index 0000000..8981ea7
--- /dev/null
+++ b/common/src/main/java/org/apache/sqoop/json/LinkBean.java
@@ -0,0 +1,184 @@
+/**
+ * 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.json;
+
+import org.apache.sqoop.model.MLink;
+import org.apache.sqoop.model.MConnectionForms;
+import org.apache.sqoop.model.MForm;
+import org.json.simple.JSONArray;
+import org.json.simple.JSONObject;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.ResourceBundle;
+import java.util.Set;
+
+import static org.apache.sqoop.json.util.FormSerialization.*;
+import static org.apache.sqoop.json.util.ResourceBundleSerialization.*;
+
+/**
+ * Link representation that is being send across the network between
+ * Sqoop server and client. Server might optionally send configs
+ * associated with the links to spare client of sending another HTTP
+ * requests to obtain them.
+ */
+public class LinkBean implements JsonBean {
+
+  private static final String CONNECTOR_ID = "connector-id";
+  private static final String CONNECTOR_PART = "connector";
+  private static final String FRAMEWORK_PART = "framework";
+
+  // Required
+  private List<MLink> links;
+
+  // Optional
+  private Map<Long, ResourceBundle> connectorConfigBundles;
+  private ResourceBundle driverConfigBundle;
+
+  // For "extract"
+  public LinkBean(MLink link) {
+    this();
+    this.links = new ArrayList<MLink>();
+    this.links.add(link);
+  }
+
+  public LinkBean(List<MLink> links) {
+    this();
+    this.links = links;
+  }
+
+  // For "restore"
+  public LinkBean() {
+    connectorConfigBundles = new HashMap<Long, ResourceBundle>();
+  }
+
+  public void setDriverConfigBundle(ResourceBundle driverConfigBundle) {
+    this.driverConfigBundle = driverConfigBundle;
+  }
+
+  public void addConnectorConfigBundle(Long id, ResourceBundle connectorConfigBundle) {
+    connectorConfigBundles.put(id, connectorConfigBundle);
+  }
+
+  public boolean hasConnectorBundle(Long id) {
+    return connectorConfigBundles.containsKey(id);
+  }
+
+  public List<MLink> getLinks() {
+    return links;
+  }
+
+  public ResourceBundle getConnectorBundle(Long id) {
+    return connectorConfigBundles.get(id);
+  }
+
+  public ResourceBundle getFrameworkBundle() {
+    return driverConfigBundle;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public JSONObject extract(boolean skipSensitive) {
+    JSONArray array = new JSONArray();
+
+    for(MLink link : links) {
+      JSONObject object = new JSONObject();
+
+      object.put(ID, link.getPersistenceId());
+      object.put(NAME, link.getName());
+      object.put(ENABLED, link.getEnabled());
+      object.put(CREATION_USER, link.getCreationUser());
+      object.put(CREATION_DATE, link.getCreationDate().getTime());
+      object.put(UPDATE_USER, link.getLastUpdateUser());
+      object.put(UPDATE_DATE, link.getLastUpdateDate().getTime());
+      object.put(CONNECTOR_ID, link.getConnectorId());
+      object.put(CONNECTOR_PART,
+        extractForms(link.getConnectorPart().getForms(), skipSensitive));
+      object.put(FRAMEWORK_PART,
+        extractForms(link.getFrameworkPart().getForms(), skipSensitive));
+
+      array.add(object);
+    }
+
+    JSONObject all = new JSONObject();
+    all.put(ALL, array);
+
+    if(!connectorConfigBundles.isEmpty()) {
+      JSONObject bundles = new JSONObject();
+
+      for(Map.Entry<Long, ResourceBundle> entry : connectorConfigBundles.entrySet()) {
+        bundles.put(entry.getKey().toString(),
+                    extractResourceBundle(entry.getValue()));
+      }
+      all.put(CONNECTOR_CONFIGS, bundles);
+    }
+    if(driverConfigBundle != null) {
+      all.put(DRIVER_CONFIGS,extractResourceBundle(driverConfigBundle));
+    }
+    return all;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void restore(JSONObject jsonObject) {
+    links = new ArrayList<MLink>();
+
+    JSONArray array = (JSONArray) jsonObject.get(ALL);
+
+    for (Object obj : array) {
+      JSONObject object = (JSONObject) obj;
+
+      long connectorId = (Long) object.get(CONNECTOR_ID);
+      JSONArray connectorPart = (JSONArray) object.get(CONNECTOR_PART);
+      JSONArray frameworkPart = (JSONArray) object.get(FRAMEWORK_PART);
+
+      List<MForm> connectorForms = restoreForms(connectorPart);
+      List<MForm> frameworkForms = restoreForms(frameworkPart);
+
+      MLink link = new MLink(connectorId,
+        new MConnectionForms(connectorForms),
+        new MConnectionForms(frameworkForms));
+
+      link.setPersistenceId((Long) object.get(ID));
+      link.setName((String) object.get(NAME));
+      link.setEnabled((Boolean) object.get(ENABLED));
+      link.setCreationUser((String) object.get(CREATION_USER));
+      link.setCreationDate(new Date((Long) object.get(CREATION_DATE)));
+      link.setLastUpdateUser((String) object.get(UPDATE_USER));
+      link.setLastUpdateDate(new Date((Long) object.get(UPDATE_DATE)));
+
+      links.add(link);
+    }
+
+    if(jsonObject.containsKey(CONNECTOR_CONFIGS)) {
+      JSONObject bundles = (JSONObject) jsonObject.get(CONNECTOR_CONFIGS);
+      Set<Map.Entry<String, JSONObject>> entrySet = bundles.entrySet();
+      for (Map.Entry<String, JSONObject> entry : entrySet) {
+        connectorConfigBundles.put(Long.parseLong(entry.getKey()),
+                             restoreResourceBundle(entry.getValue()));
+      }
+    }
+    if(jsonObject.containsKey(DRIVER_CONFIGS)) {
+      driverConfigBundle = restoreResourceBundle(
+        (JSONObject) jsonObject.get(DRIVER_CONFIGS));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/json/LinkValidationBean.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/json/LinkValidationBean.java b/common/src/main/java/org/apache/sqoop/json/LinkValidationBean.java
new file mode 100644
index 0000000..a6a6b65
--- /dev/null
+++ b/common/src/main/java/org/apache/sqoop/json/LinkValidationBean.java
@@ -0,0 +1,143 @@
+/**
+ * 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.json;
+
+import org.apache.sqoop.validation.Status;
+import org.apache.sqoop.validation.Validation;
+import org.json.simple.JSONObject;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Bean for sending validations across network. This bean will move two
+ * validation objects at one time - one for connector and second for framework
+ * part of validated entity. Optionally validation bean can also transfer
+ * created persistent id in case that new entity was created.
+ */
+public class LinkValidationBean implements JsonBean {
+
+  private static final String ID = "id";
+  private static final String FRAMEWORK = "framework";
+  private static final String CONNECTOR = "connector";
+  private static final String STATUS = "status";
+  private static final String MESSAGE = "message";
+  private static final String MESSAGES = "messages";
+
+  private Long id;
+  private Validation connectorValidation;
+  private Validation frameworkValidation;
+
+  // For "extract"
+  public LinkValidationBean(Validation connector, Validation framework) {
+    this();
+
+    this.connectorValidation = connector;
+    this.frameworkValidation = framework;
+  }
+
+  // For "restore"
+  public LinkValidationBean() {
+    id = null;
+  }
+
+  public Validation getConnectorValidation() {
+    return connectorValidation;
+  }
+
+  public Validation getFrameworkValidation() {
+    return frameworkValidation;
+  }
+
+  public void setId(Long id) {
+    this.id = id;
+  }
+
+  public Long getId() {
+    return id;
+  }
+
+  @SuppressWarnings("unchecked")
+  public JSONObject extract(boolean skipSensitive) {
+    JSONObject object = new JSONObject();
+
+    // Optionally transfer id
+    if(id != null) {
+      object.put(ID, id);
+    }
+
+    object.put(CONNECTOR, extractValidation(connectorValidation));
+    object.put(FRAMEWORK, extractValidation(frameworkValidation));
+
+    return object;
+  }
+
+  @SuppressWarnings("unchecked")
+  private JSONObject extractValidation(Validation validation) {
+    JSONObject object = new JSONObject();
+
+    object.put(STATUS, validation.getStatus().name());
+
+    JSONObject jsonMessages = new JSONObject();
+    Map<Validation.FormInput, Validation.Message> messages = validation.getMessages();
+
+    for(Map.Entry<Validation.FormInput, Validation.Message> entry : messages.entrySet()) {
+      JSONObject jsonEntry = new JSONObject();
+      jsonEntry.put(STATUS, entry.getValue().getStatus().name());
+      jsonEntry.put(MESSAGE, entry.getValue().getMessage());
+      jsonMessages.put(entry.getKey(), jsonEntry);
+    }
+
+    object.put(MESSAGES, jsonMessages);
+
+    return object;
+  }
+
+  @Override
+  public void restore(JSONObject jsonObject) {
+    // Optional and accepting NULLs
+    id = (Long) jsonObject.get(ID);
+
+    connectorValidation = restoreValidation(
+      (JSONObject)jsonObject.get(CONNECTOR));
+    frameworkValidation = restoreValidation(
+      (JSONObject)jsonObject.get(FRAMEWORK));
+  }
+
+  public Validation restoreValidation(JSONObject jsonObject) {
+    JSONObject jsonMessages = (JSONObject) jsonObject.get(MESSAGES);
+    Map<Validation.FormInput, Validation.Message> messages
+      = new HashMap<Validation.FormInput, Validation.Message>();
+
+    for(Object key : jsonMessages.keySet()) {
+      JSONObject jsonMessage = (JSONObject) jsonMessages.get(key);
+
+      Status status = Status.valueOf((String) jsonMessage.get(STATUS));
+      String stringMessage = (String) jsonMessage.get(MESSAGE);
+
+      Validation.Message message
+        = new Validation.Message(status, stringMessage);
+
+      messages.put(new Validation.FormInput((String)key), message);
+    }
+
+    Status status = Status.valueOf((String) jsonObject.get(STATUS));
+
+    return new Validation(status, messages);
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/json/util/FormSerialization.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/json/util/FormSerialization.java b/common/src/main/java/org/apache/sqoop/json/util/FormSerialization.java
index 9558941..77f6191 100644
--- a/common/src/main/java/org/apache/sqoop/json/util/FormSerialization.java
+++ b/common/src/main/java/org/apache/sqoop/json/util/FormSerialization.java
@@ -44,7 +44,7 @@ public final class FormSerialization {
   public static final String ID = "id";
   public static final String NAME = "name";
   public static final String VERSION = "version";
-  public static final String FRAMEWORK_VERSION = "framework-version";
+  public static final String DRIVER_VERSION = "driver-version";
   public static final String CLASS = "class";
   public static final String ENABLED = "enabled";
   public static final String CREATION_USER = "creation-user";

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/json/util/ResourceBundleSerialization.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/json/util/ResourceBundleSerialization.java b/common/src/main/java/org/apache/sqoop/json/util/ResourceBundleSerialization.java
index 9f3ad62..20ee5f3 100644
--- a/common/src/main/java/org/apache/sqoop/json/util/ResourceBundleSerialization.java
+++ b/common/src/main/java/org/apache/sqoop/json/util/ResourceBundleSerialization.java
@@ -33,9 +33,9 @@ import java.util.ResourceBundle;
  */
 public final class ResourceBundleSerialization {
 
-  public static final String RESOURCES = "resources";
-  public static final String CONNECTOR_RESOURCES = "resources-connector";
-  public static final String FRAMEWORK_RESOURCES = "resources-framework";
+  public static final String CONFIGS = "configs";
+  public static final String CONNECTOR_CONFIGS = "connector-configs";
+  public static final String DRIVER_CONFIGS = "driver-configs";
 
   @SuppressWarnings("unchecked")
   public static JSONArray extractResourceBundles(List<ResourceBundle> bundles) {
@@ -50,27 +50,20 @@ public final class ResourceBundleSerialization {
   @SuppressWarnings("unchecked")
   public static JSONObject extractResourceBundle(ResourceBundle bundle) {
     JSONObject json = new JSONObject();
-
     Enumeration<String> keys = bundle.getKeys();
-
     while(keys.hasMoreElements()) {
       String key = keys.nextElement();
-
       json.put(key, bundle.getString(key));
-
     }
-
     return json;
   }
 
   @SuppressWarnings("unchecked")
   public static List<ResourceBundle> restoreResourceBundles(JSONArray array) {
     List<ResourceBundle> bundles = new LinkedList<ResourceBundle>();
-
     for (Object item : array) {
       bundles.add(restoreResourceBundle((JSONObject) item));
     }
-
     return bundles;
   }
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/model/Input.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/Input.java b/common/src/main/java/org/apache/sqoop/model/Input.java
index 2b32e43..db3e0e6 100644
--- a/common/src/main/java/org/apache/sqoop/model/Input.java
+++ b/common/src/main/java/org/apache/sqoop/model/Input.java
@@ -30,7 +30,7 @@ import java.lang.annotation.Target;
 @Target(ElementType.FIELD)
 public @interface Input {
   /**
-   * Sqoop framework will ensure that sensitive information will not be easily
+   * Sqoop will ensure that sensitive information will not be easily
    * accessible.
    *
    * @return True if field is sensitive

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/model/MConnection.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/MConnection.java b/common/src/main/java/org/apache/sqoop/model/MConnection.java
deleted file mode 100644
index f84abbf..0000000
--- a/common/src/main/java/org/apache/sqoop/model/MConnection.java
+++ /dev/null
@@ -1,130 +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.model;
-
-/**
- * Model describing entire connection object including both connector and
- * framework part.
- */
-public class MConnection extends MAccountableEntity implements MClonable {
-  private long connectorId;
-
-  private final MConnectionForms connectorPart;
-  private final MConnectionForms frameworkPart;
-
-  /**
-   * Default constructor to build new MConnection model.
-   *
-   * @param connectorId Connector id
-   * @param connectorPart Connector forms
-   * @param frameworkPart Framework forms
-   */
-  public MConnection(long connectorId,
-                     MConnectionForms connectorPart,
-                     MConnectionForms frameworkPart) {
-    this.connectorId = connectorId;
-    this.connectorPart = connectorPart;
-    this.frameworkPart = frameworkPart;
-  }
-
-  /**
-   * Constructor to create deep copy of another MConnection model.
-   *
-   * @param other MConnection model to copy
-   */
-  public MConnection(MConnection other) {
-    this(other, other.connectorPart.clone(true), other.frameworkPart.clone(true));
-  }
-
-  /**
-   * Construct new MConnection model as a copy of another with replaced forms.
-   *
-   * This method is suitable only for metadata upgrade path and should not be
-   * used otherwise.
-   *
-   * @param other MConnection model to copy
-   * @param connectorPart Connector forms
-   * @param frameworkPart Framework forms
-   */
-  public MConnection(MConnection other, MConnectionForms connectorPart, MConnectionForms frameworkPart) {
-    super(other);
-    this.connectorId = other.connectorId;
-    this.connectorPart = connectorPart;
-    this.frameworkPart = frameworkPart;
-    this.setPersistenceId(other.getPersistenceId());
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder("connection: ").append(getName());
-    sb.append(" connector-part: ").append(connectorPart);
-    sb.append(", framework-part: ").append(frameworkPart);
-
-    return sb.toString();
-  }
-
-  public long getConnectorId() {
-    return connectorId;
-  }
-
-  public void setConnectorId(long connectorId) {
-    this.connectorId = connectorId;
-  }
-
-  public MConnectionForms getConnectorPart() {
-    return connectorPart;
-  }
-
-  public MConnectionForms getFrameworkPart() {
-    return frameworkPart;
-  }
-
-  public MForm getConnectorForm(String formName) {
-    return connectorPart.getForm(formName);
-  }
-
-  public MForm getFrameworkForm(String formName) {
-    return frameworkPart.getForm(formName);
-  }
-
-  @Override
-  public MConnection clone(boolean cloneWithValue) {
-    if(cloneWithValue) {
-      return new MConnection(this);
-    } else {
-      return new MConnection(connectorId, connectorPart.clone(false), frameworkPart.clone(false));
-    }
-  }
-
-  @Override
-  public boolean equals(Object object) {
-    if(object == this) {
-      return true;
-    }
-
-    if(!(object instanceof MConnection)) {
-      return false;
-    }
-
-    MConnection mc = (MConnection)object;
-    return (mc.connectorId == this.connectorId)
-        && (mc.getPersistenceId() == this.getPersistenceId())
-        && (mc.connectorPart.equals(this.connectorPart))
-        && (mc.frameworkPart.equals(this.frameworkPart));
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/model/MDriverConfig.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/MDriverConfig.java b/common/src/main/java/org/apache/sqoop/model/MDriverConfig.java
new file mode 100644
index 0000000..62eb197
--- /dev/null
+++ b/common/src/main/java/org/apache/sqoop/model/MDriverConfig.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.sqoop.model;
+
+/**
+ * Describes the configs associated with the {@link Driver} for executing sqoop jobs.
+ */
+public class MDriverConfig extends MPersistableEntity implements MClonable {
+
+  private final MConnectionForms connectionForms;
+  private final MJobForms jobForms;
+  String version;
+
+  public MDriverConfig(MConnectionForms connectionForms, MJobForms jobForms, String version) {
+    this.connectionForms = connectionForms;
+    this.jobForms = jobForms;
+    this.version = version;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("driver-");
+    sb.append(getPersistenceId()).append(":");
+    sb.append("version = " + version);
+    sb.append(", ").append(connectionForms.toString());
+    sb.append(jobForms.toString());
+
+    return sb.toString();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == this) {
+      return true;
+    }
+
+    if (!(other instanceof MDriverConfig)) {
+      return false;
+    }
+
+    MDriverConfig mo = (MDriverConfig) other;
+    return version.equals(mo.getVersion()) &&
+      connectionForms.equals(mo.connectionForms) &&
+      jobForms.equals(mo.jobForms);
+  }
+
+  @Override
+  public int hashCode() {
+    int result = connectionForms.hashCode();
+    result = 31 * result + jobForms.hashCode();
+    result = 31 * result + version.hashCode();
+    return result;
+  }
+
+  public MConnectionForms getConnectionForms() {
+    return connectionForms;
+  }
+
+  public MJobForms getJobForms() {
+    return jobForms;
+  }
+
+  @Override
+  public MDriverConfig clone(boolean cloneWithValue) {
+    //Framework never have any values filled
+    cloneWithValue = false;
+    MDriverConfig copy = new MDriverConfig(this.getConnectionForms().clone(cloneWithValue),
+        this.getJobForms().clone(cloneWithValue), this.version);
+    copy.setPersistenceId(this.getPersistenceId());
+    return copy;
+  }
+
+  public String getVersion() {
+    return version;
+  }
+
+  public void setVersion(String version) {
+    this.version = version;
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/model/MFramework.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/MFramework.java b/common/src/main/java/org/apache/sqoop/model/MFramework.java
deleted file mode 100644
index 580db9c..0000000
--- a/common/src/main/java/org/apache/sqoop/model/MFramework.java
+++ /dev/null
@@ -1,99 +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.model;
-
-/**
- * Metadata describing framework options for connection and jobForms for each
- * supported jobForms type.
- */
-public class MFramework extends MPersistableEntity implements MClonable {
-
-  private final MConnectionForms connectionForms;
-  private final MJobForms jobForms;
-  String version;
-
-  public MFramework(MConnectionForms connectionForms, MJobForms jobForms,
-    String version) {
-    this.version = version;
-    this.connectionForms = connectionForms;
-    this.jobForms = jobForms;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder("framework-");
-    sb.append(getPersistenceId()).append(":");
-    sb.append("version = " + version);
-    sb.append(", ").append(connectionForms.toString());
-    sb.append(jobForms.toString());
-
-    return sb.toString();
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other == this) {
-      return true;
-    }
-
-    if (!(other instanceof MFramework)) {
-      return false;
-    }
-
-    MFramework mo = (MFramework) other;
-    return version.equals(mo.getVersion()) &&
-      connectionForms.equals(mo.connectionForms) &&
-      jobForms.equals(mo.jobForms);
-  }
-
-  @Override
-  public int hashCode() {
-    int result = connectionForms.hashCode();
-    result = 31 * result + jobForms.hashCode();
-    result = 31 * result + version.hashCode();
-    return result;
-  }
-
-  public MConnectionForms getConnectionForms() {
-    return connectionForms;
-  }
-
-  public MJobForms getJobForms() {
-    return jobForms;
-  }
-
-  @Override
-  public MFramework clone(boolean cloneWithValue) {
-    //Framework never have any values filled
-    cloneWithValue = false;
-    MFramework copy = new MFramework(this.getConnectionForms().clone(cloneWithValue),
-        this.getJobForms().clone(cloneWithValue), this.version);
-    copy.setPersistenceId(this.getPersistenceId());
-    return copy;
-  }
-
-  public String getVersion() {
-    return version;
-  }
-
-  public void setVersion(String version) {
-    this.version = version;
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/model/MInput.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/MInput.java b/common/src/main/java/org/apache/sqoop/model/MInput.java
index 1f4c425..0074267 100644
--- a/common/src/main/java/org/apache/sqoop/model/MInput.java
+++ b/common/src/main/java/org/apache/sqoop/model/MInput.java
@@ -18,7 +18,7 @@
 package org.apache.sqoop.model;
 
 /**
- * Represents a parameter input used by the connector for creating a connection
+ * Represents a parameter input used by the connector for creating a link
  * or a job object.
  * @param <T> the value type associated with this parameter
  * @param boolean whether or not the field contains sensitive information

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/model/MJob.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/MJob.java b/common/src/main/java/org/apache/sqoop/model/MJob.java
index 182bbfb..c9b45a5 100644
--- a/common/src/main/java/org/apache/sqoop/model/MJob.java
+++ b/common/src/main/java/org/apache/sqoop/model/MJob.java
@@ -30,17 +30,17 @@ public class MJob extends MAccountableEntity implements MClonable {
    * Connector reference.
    *
    * Job object do not immediately depend on connector as there is indirect
-   * dependency through connection object, but having this dependency explicitly
+   * dependency through link object, but having this dependency explicitly
    * carried along helps a lot.
    */
   private final long fromConnectorId;
   private final long toConnectorId;
 
   /**
-   * Corresponding connection objects for connector.
+   * Corresponding link objects for connector.
    */
-  private final long fromConnectionId;
-  private final long toConnectionId;
+  private final long fromLinkId;
+  private final long toLinkId;
 
   private final MJobForms fromConnectorPart;
   private final MJobForms toConnectorPart;
@@ -51,8 +51,8 @@ public class MJob extends MAccountableEntity implements MClonable {
    *
    * @param fromConnectorId FROM Connector id
    * @param toConnectorId TO Connector id
-   * @param fromConnectionId FROM Connection id
-   * @param toConnectionId TO Connection id
+   * @param fromLinkId FROM Link id
+   * @param toLinkId TO Link id
    * @param fromPart FROM Connector forms
    * @param toPart TO Connector forms
    * @param frameworkPart Framework forms
@@ -66,8 +66,8 @@ public class MJob extends MAccountableEntity implements MClonable {
               MJobForms frameworkPart) {
     this.fromConnectorId = fromConnectorId;
     this.toConnectorId = toConnectorId;
-    this.fromConnectionId = fromConnectionId;
-    this.toConnectionId = toConnectionId;
+    this.fromLinkId = fromConnectionId;
+    this.toLinkId = toConnectionId;
     this.fromConnectorPart = fromPart;
     this.toConnectorPart = toPart;
     this.frameworkPart = frameworkPart;
@@ -101,8 +101,8 @@ public class MJob extends MAccountableEntity implements MClonable {
 
     this.fromConnectorId = other.getConnectorId(Direction.FROM);
     this.toConnectorId = other.getConnectorId(Direction.TO);
-    this.fromConnectionId = other.getConnectionId(Direction.FROM);
-    this.toConnectionId = other.getConnectionId(Direction.TO);
+    this.fromLinkId = other.getLinkId(Direction.FROM);
+    this.toLinkId = other.getLinkId(Direction.TO);
     this.fromConnectorPart = fromPart;
     this.toConnectorPart = toPart;
     this.frameworkPart = frameworkPart;
@@ -119,13 +119,13 @@ public class MJob extends MAccountableEntity implements MClonable {
     return sb.toString();
   }
 
-  public long getConnectionId(Direction type) {
+  public long getLinkId(Direction type) {
     switch(type) {
       case FROM:
-        return fromConnectionId;
+        return fromLinkId;
 
       case TO:
-        return toConnectionId;
+        return toLinkId;
 
       default:
         throw new SqoopException(DirectionError.DIRECTION_0000, "Direction: " + type);
@@ -170,8 +170,8 @@ public class MJob extends MAccountableEntity implements MClonable {
       return new MJob(
           getConnectorId(Direction.FROM),
           getConnectorId(Direction.TO),
-          getConnectionId(Direction.FROM),
-          getConnectionId(Direction.TO),
+          getLinkId(Direction.FROM),
+          getLinkId(Direction.TO),
           getConnectorPart(Direction.FROM).clone(false),
           getConnectorPart(Direction.TO).clone(false),
           frameworkPart.clone(false));
@@ -191,8 +191,8 @@ public class MJob extends MAccountableEntity implements MClonable {
     MJob job = (MJob)object;
     return (job.getConnectorId(Direction.FROM) == this.getConnectorId(Direction.FROM))
         && (job.getConnectorId(Direction.TO) == this.getConnectorId(Direction.TO))
-        && (job.getConnectionId(Direction.FROM) == this.getConnectionId(Direction.FROM))
-        && (job.getConnectionId(Direction.TO) == this.getConnectionId(Direction.TO))
+        && (job.getLinkId(Direction.FROM) == this.getLinkId(Direction.FROM))
+        && (job.getLinkId(Direction.TO) == this.getLinkId(Direction.TO))
         && (job.getPersistenceId() == this.getPersistenceId())
         && (job.getConnectorPart(Direction.FROM).equals(this.getConnectorPart(Direction.FROM)))
         && (job.getConnectorPart(Direction.TO).equals(this.getConnectorPart(Direction.TO)))

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/common/src/main/java/org/apache/sqoop/model/MLink.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/sqoop/model/MLink.java b/common/src/main/java/org/apache/sqoop/model/MLink.java
new file mode 100644
index 0000000..6a8c424
--- /dev/null
+++ b/common/src/main/java/org/apache/sqoop/model/MLink.java
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.sqoop.model;
+
+/**
+ * Model describing the link object and its corresponding configs
+ */
+public class MLink extends MAccountableEntity implements MClonable {
+  private long connectorId;
+
+  private final MConnectionForms connectorPart;
+  private final MConnectionForms frameworkPart;
+
+  /**
+   * Default constructor to build new MConnection model.
+   *
+   * @param connectorId Connector id
+   * @param connectorPart Connector forms
+   * @param frameworkPart Framework forms
+   */
+  public MLink(long connectorId,
+                     MConnectionForms connectorPart,
+                     MConnectionForms frameworkPart) {
+    this.connectorId = connectorId;
+    this.connectorPart = connectorPart;
+    this.frameworkPart = frameworkPart;
+  }
+
+  /**
+   * Constructor to create deep copy of another MConnection model.
+   *
+   * @param other MConnection model to copy
+   */
+  public MLink(MLink other) {
+    this(other, other.connectorPart.clone(true), other.frameworkPart.clone(true));
+  }
+
+  /**
+   * Construct new MConnection model as a copy of another with replaced forms.
+   *
+   * This method is suitable only for metadata upgrade path and should not be
+   * used otherwise.
+   *
+   * @param other MConnection model to copy
+   * @param connectorPart Connector forms
+   * @param frameworkPart Framework forms
+   */
+  public MLink(MLink other, MConnectionForms connectorPart, MConnectionForms frameworkPart) {
+    super(other);
+    this.connectorId = other.connectorId;
+    this.connectorPart = connectorPart;
+    this.frameworkPart = frameworkPart;
+    this.setPersistenceId(other.getPersistenceId());
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("connection: ").append(getName());
+    sb.append(" connector-part: ").append(connectorPart);
+    sb.append(", framework-part: ").append(frameworkPart);
+
+    return sb.toString();
+  }
+
+  public long getConnectorId() {
+    return connectorId;
+  }
+
+  public void setConnectorId(long connectorId) {
+    this.connectorId = connectorId;
+  }
+
+  public MConnectionForms getConnectorPart() {
+    return connectorPart;
+  }
+
+  public MConnectionForms getFrameworkPart() {
+    return frameworkPart;
+  }
+
+  public MForm getConnectorForm(String formName) {
+    return connectorPart.getForm(formName);
+  }
+
+  public MForm getFrameworkForm(String formName) {
+    return frameworkPart.getForm(formName);
+  }
+
+  @Override
+  public MLink clone(boolean cloneWithValue) {
+    if(cloneWithValue) {
+      return new MLink(this);
+    } else {
+      return new MLink(connectorId, connectorPart.clone(false), frameworkPart.clone(false));
+    }
+  }
+
+  @Override
+  public boolean equals(Object object) {
+    if(object == this) {
+      return true;
+    }
+
+    if(!(object instanceof MLink)) {
+      return false;
+    }
+
+    MLink mc = (MLink)object;
+    return (mc.connectorId == this.connectorId)
+        && (mc.getPersistenceId() == this.getPersistenceId())
+        && (mc.connectorPart.equals(this.connectorPart))
+        && (mc.frameworkPart.equals(this.frameworkPart));
+  }
+}