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

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

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/FromJobConfig.java
----------------------------------------------------------------------
diff --git a/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/FromJobConfig.java b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/FromJobConfig.java
new file mode 100644
index 0000000..8b23144
--- /dev/null
+++ b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/FromJobConfig.java
@@ -0,0 +1,68 @@
+/**
+ * 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.connector.jdbc.configuration;
+
+import org.apache.sqoop.connector.jdbc.GenericJdbcConnectorConstants;
+import org.apache.sqoop.model.FormClass;
+import org.apache.sqoop.model.Input;
+import org.apache.sqoop.model.Validator;
+import org.apache.sqoop.validation.Status;
+import org.apache.sqoop.validation.validators.AbstractValidator;
+import org.apache.sqoop.validation.validators.NullOrContains;
+
+/**
+ *
+ */
+@FormClass( validators = {@Validator(FromJobConfig.FormValidator.class)})
+public class FromJobConfig {
+  @Input(size = 50)
+  public String schemaName;
+
+  @Input(size = 50)
+  public String tableName;
+
+  @Input(size = 2000, validators = {@Validator(value = NullOrContains.class, strArg = GenericJdbcConnectorConstants.SQL_CONDITIONS_TOKEN)})
+  public String sql;
+
+  @Input(size = 50)
+  public String columns;
+
+  @Input(size = 50)
+  public String partitionColumn;
+
+  @Input
+  public Boolean partitionColumnNull;
+
+  @Input(size = 50)
+  public String boundaryQuery;
+
+  public static class FormValidator extends AbstractValidator<FromJobConfig> {
+    @Override
+    public void validate(FromJobConfig form) {
+      if(form.tableName == null && form.sql == null) {
+        addMessage(Status.UNACCEPTABLE, "Either table name or SQL must be specified");
+      }
+      if(form.tableName != null && form.sql != null) {
+        addMessage(Status.UNACCEPTABLE, "Both table name and SQL cannot be specified");
+      }
+      if(form.schemaName != null && form.sql != null) {
+        addMessage(Status.UNACCEPTABLE, "Both schema name and SQL cannot be specified");
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/FromJobConfiguration.java
----------------------------------------------------------------------
diff --git a/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/FromJobConfiguration.java b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/FromJobConfiguration.java
index 64e9a9a..b036421 100644
--- a/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/FromJobConfiguration.java
+++ b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/FromJobConfiguration.java
@@ -25,9 +25,9 @@ import org.apache.sqoop.model.Form;
  */
 @ConfigurationClass
 public class FromJobConfiguration {
-  @Form public FromTableForm fromTable;
+  @Form public FromJobConfig fromJobConfig;
 
   public FromJobConfiguration() {
-    fromTable = new FromTableForm();
+    fromJobConfig = new FromJobConfig();
   }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/FromTableForm.java
----------------------------------------------------------------------
diff --git a/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/FromTableForm.java b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/FromTableForm.java
deleted file mode 100644
index 72902a2..0000000
--- a/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/FromTableForm.java
+++ /dev/null
@@ -1,68 +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.connector.jdbc.configuration;
-
-import org.apache.sqoop.connector.jdbc.GenericJdbcConnectorConstants;
-import org.apache.sqoop.model.FormClass;
-import org.apache.sqoop.model.Input;
-import org.apache.sqoop.model.Validator;
-import org.apache.sqoop.validation.Status;
-import org.apache.sqoop.validation.validators.AbstractValidator;
-import org.apache.sqoop.validation.validators.NullOrContains;
-
-/**
- *
- */
-@FormClass( validators = {@Validator(FromTableForm.FormValidator.class)})
-public class FromTableForm {
-  @Input(size = 50)
-  public String schemaName;
-
-  @Input(size = 50)
-  public String tableName;
-
-  @Input(size = 2000, validators = {@Validator(value = NullOrContains.class, strArg = GenericJdbcConnectorConstants.SQL_CONDITIONS_TOKEN)})
-  public String sql;
-
-  @Input(size = 50)
-  public String columns;
-
-  @Input(size = 50)
-  public String partitionColumn;
-
-  @Input
-  public Boolean partitionColumnNull;
-
-  @Input(size = 50)
-  public String boundaryQuery;
-
-  public static class FormValidator extends AbstractValidator<FromTableForm> {
-    @Override
-    public void validate(FromTableForm form) {
-      if(form.tableName == null && form.sql == null) {
-        addMessage(Status.UNACCEPTABLE, "Either fromTable name or SQL must be specified");
-      }
-      if(form.tableName != null && form.sql != null) {
-        addMessage(Status.UNACCEPTABLE, "Both fromTable name and SQL cannot be specified");
-      }
-      if(form.schemaName != null && form.sql != null) {
-        addMessage(Status.UNACCEPTABLE, "Both schema name and SQL cannot be specified");
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/LinkConfig.java
----------------------------------------------------------------------
diff --git a/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/LinkConfig.java b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/LinkConfig.java
new file mode 100644
index 0000000..7b76eeb
--- /dev/null
+++ b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/LinkConfig.java
@@ -0,0 +1,64 @@
+/**
+ * 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.connector.jdbc.configuration;
+
+import org.apache.sqoop.model.FormClass;
+import org.apache.sqoop.model.Input;
+import org.apache.sqoop.model.Validator;
+import org.apache.sqoop.validation.Status;
+import org.apache.sqoop.validation.validators.AbstractValidator;
+import org.apache.sqoop.validation.validators.NotEmpty;
+import org.apache.sqoop.validation.validators.ClassAvailable;
+import org.apache.sqoop.validation.validators.StartsWith;
+
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Map;
+
+/**
+ *
+ */
+@FormClass(validators = {@Validator(LinkConfig.FormValidator.class)})
+public class LinkConfig {
+  @Input(size = 128, validators = {@Validator(NotEmpty.class), @Validator(ClassAvailable.class)} )
+  public String jdbcDriver;
+
+  @Input(size = 128, validators = {@Validator(value = StartsWith.class, strArg = "jdbc:")} )
+  public String connectionString;
+
+  @Input(size = 40)
+  public String username;
+
+  @Input(size = 40, sensitive = true)
+  public String password;
+
+  @Input
+  public Map<String, String> jdbcProperties;
+
+  public static class FormValidator extends AbstractValidator<LinkConfig> {
+    @Override
+    public void validate(LinkConfig linkConfig) {
+      // See if we can connect to the database
+      try {
+        DriverManager.getConnection(linkConfig.connectionString, linkConfig.username, linkConfig.password);
+      } catch (SQLException e) {
+        addMessage(Status.ACCEPTABLE, "Can't connect to the database with given credentials: " + e.getMessage());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/LinkConfiguration.java
----------------------------------------------------------------------
diff --git a/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/LinkConfiguration.java b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/LinkConfiguration.java
new file mode 100644
index 0000000..7d614f7
--- /dev/null
+++ b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/LinkConfiguration.java
@@ -0,0 +1,34 @@
+/**
+ * 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.connector.jdbc.configuration;
+
+import org.apache.sqoop.model.ConfigurationClass;
+import org.apache.sqoop.model.Form;
+
+/**
+ *
+ */
+@ConfigurationClass
+public class LinkConfiguration {
+
+  @Form public LinkConfig link;
+
+  public LinkConfiguration() {
+    link = new LinkConfig();
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/ToJobConfig.java
----------------------------------------------------------------------
diff --git a/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/ToJobConfig.java b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/ToJobConfig.java
new file mode 100644
index 0000000..a42a6ec
--- /dev/null
+++ b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/ToJobConfig.java
@@ -0,0 +1,55 @@
+/**
+ * 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.connector.jdbc.configuration;
+
+import org.apache.sqoop.model.FormClass;
+import org.apache.sqoop.model.Input;
+import org.apache.sqoop.model.Validator;
+import org.apache.sqoop.validation.Status;
+import org.apache.sqoop.validation.validators.AbstractValidator;
+
+/**
+ *
+ */
+@FormClass(validators = {@Validator(ToJobConfig.FormValidator.class)})
+public class ToJobConfig {
+  @Input(size = 50)   public String schemaName;
+  @Input(size = 2000) public String tableName;
+  @Input(size = 50)   public String sql;
+  @Input(size = 50)   public String columns;
+  @Input(size = 2000) public String stageTableName;
+  @Input              public Boolean clearStageTable;
+
+  public static class FormValidator extends AbstractValidator<ToJobConfig> {
+    @Override
+    public void validate(ToJobConfig form) {
+      if(form.tableName == null && form.sql == null) {
+        addMessage(Status.UNACCEPTABLE, "Either table name or SQL must be specified");
+      }
+      if(form.tableName != null && form.sql != null) {
+        addMessage(Status.UNACCEPTABLE, "Both table name and SQL cannot be specified");
+      }
+      if(form.tableName == null && form.stageTableName != null) {
+        addMessage(Status.UNACCEPTABLE, "Stage table name cannot be specified without specifying table name");
+      }
+      if(form.stageTableName == null && form.clearStageTable != null) {
+        addMessage(Status.UNACCEPTABLE, "Clear stage table cannot be specified without specifying name of the stage table.");
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/ToJobConfiguration.java
----------------------------------------------------------------------
diff --git a/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/ToJobConfiguration.java b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/ToJobConfiguration.java
index 9215e6c..ad68681 100644
--- a/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/ToJobConfiguration.java
+++ b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/ToJobConfiguration.java
@@ -25,9 +25,9 @@ import org.apache.sqoop.model.Form;
  */
 @ConfigurationClass
 public class ToJobConfiguration {
-  @Form public ToTableForm toTable;
+  @Form public ToJobConfig toJobConfig;
 
   public ToJobConfiguration() {
-    toTable = new ToTableForm();
+    toJobConfig = new ToJobConfig();
   }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/ToTableForm.java
----------------------------------------------------------------------
diff --git a/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/ToTableForm.java b/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/ToTableForm.java
deleted file mode 100644
index 642da11..0000000
--- a/connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/configuration/ToTableForm.java
+++ /dev/null
@@ -1,55 +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.connector.jdbc.configuration;
-
-import org.apache.sqoop.model.FormClass;
-import org.apache.sqoop.model.Input;
-import org.apache.sqoop.model.Validator;
-import org.apache.sqoop.validation.Status;
-import org.apache.sqoop.validation.validators.AbstractValidator;
-
-/**
- *
- */
-@FormClass(validators = {@Validator(ToTableForm.FormValidator.class)})
-public class ToTableForm {
-  @Input(size = 50)   public String schemaName;
-  @Input(size = 2000) public String tableName;
-  @Input(size = 50)   public String sql;
-  @Input(size = 50)   public String columns;
-  @Input(size = 2000) public String stageTableName;
-  @Input              public Boolean clearStageTable;
-
-  public static class FormValidator extends AbstractValidator<ToTableForm> {
-    @Override
-    public void validate(ToTableForm form) {
-      if(form.tableName == null && form.sql == null) {
-        addMessage(Status.UNACCEPTABLE, "Either fromTable name or SQL must be specified");
-      }
-      if(form.tableName != null && form.sql != null) {
-        addMessage(Status.UNACCEPTABLE, "Both fromTable name and SQL cannot be specified");
-      }
-      if(form.tableName == null && form.stageTableName != null) {
-        addMessage(Status.UNACCEPTABLE, "Stage fromTable name cannot be specified without specifying fromTable name");
-      }
-      if(form.stageTableName == null && form.clearStageTable != null) {
-        addMessage(Status.UNACCEPTABLE, "Clear stage fromTable cannot be specified without specifying name of the stage fromTable.");
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-generic-jdbc/src/main/resources/generic-jdbc-connector-config.properties
----------------------------------------------------------------------
diff --git a/connector/connector-generic-jdbc/src/main/resources/generic-jdbc-connector-config.properties b/connector/connector-generic-jdbc/src/main/resources/generic-jdbc-connector-config.properties
new file mode 100644
index 0000000..921ac28
--- /dev/null
+++ b/connector/connector-generic-jdbc/src/main/resources/generic-jdbc-connector-config.properties
@@ -0,0 +1,127 @@
+# 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.
+
+# Generic JDBC Connector Resources
+
+############################
+# Link Config
+#
+link.label = Link configuration
+link.help = You must supply the information requested in order to \
+                   create a connection object.
+
+# jdbc driver
+link.jdbcDriver.label = JDBC Driver Class
+link.jdbcDriver.help = Enter the fully qualified class name of the JDBC \
+                   driver that will be used for establishing this connection.
+
+# connect string
+link.connectionString.label = JDBC Connection String
+link.connectionString.help = Enter the value of JDBC connection string to be \
+                   used by this connector for creating connections.
+
+# username string
+link.username.label = Username
+link.username.help = Enter the username to be used for connecting to the \
+                   database.
+
+# password string
+link.password.label = Password
+link.password.help = Enter the password to be used for connecting to the \
+                   database.
+
+# jdbc properties
+link.jdbcProperties.label = JDBC Connection Properties
+link.jdbcProperties.help = Enter any JDBC properties that should be \
+                   supplied during the creation of connection.
+
+# From Job Config
+#
+fromJobConfig.label = From database configuration
+fromJobConfig.help = You must supply the information requested in order to create \
+                 a job object.
+
+# From schema name
+fromJobConfig.schemaName.label = Schema name
+fromJobConfig.schemaName.help = Schema name to process data in the remote database
+
+# From table name
+fromJobConfig.tableName.label = Table name
+fromJobConfig.tableName.help = Table name to process data in the remote database
+
+# From table SQL
+fromJobConfig.sql.label = Table SQL statement
+fromJobConfig.sql.help = SQL statement to process data in the remote database
+
+# From table columns
+fromJobConfig.columns.label = Table column names
+fromJobConfig.columns.help = Specific columns of a table name or a table SQL
+
+# From table warehouse
+fromJobConfig.warehouse.label = Data warehouse
+fromJobConfig.warehouse.help = The root directory for data
+
+# From table datadir
+fromJobConfig.dataDirectory.label = Data directory
+fromJobConfig.dataDirectory.help = The sub-directory under warehouse for data
+
+# From table pcol
+fromJobConfig.partitionColumn.label = Partition column name
+fromJobConfig.partitionColumn.help = A specific column for data partition
+
+# From table pcol is null
+fromJobConfig.partitionColumnNull.label = Nulls in partition column
+fromJobConfig.partitionColumnNull.help = Whether there are null values in partition column
+
+# From table boundary
+fromJobConfig.boundaryQuery.label = Boundary query
+fromJobConfig.boundaryQuery.help = The boundary query for data partition
+
+# ToJob Config
+#
+toJobConfig.label = To database configuration
+toJobConfig.help = You must supply the information requested in order to create \
+                 a job object.
+
+# To schema name
+toJobConfig.schemaName.label = Schema name
+toJobConfig.schemaName.help = Schema name to process data in the remote database
+
+# To table name
+toJobConfig.tableName.label = Table name
+toJobConfig.tableName.help = Table name to process data in the remote database
+
+# To table SQL
+toJobConfig.sql.label = Table SQL statement
+toJobConfig.sql.help = SQL statement to process data in the remote database
+
+# To table columns
+toJobConfig.columns.label = Table column names
+toJobConfig.columns.help = Specific columns of a table name or a table SQL
+
+# To stage table name
+toJobConfig.stageTableName.label = Stage table name
+toJobConfig.stageTableName.help = Name of the stage table to use
+
+# To clear stage table
+toJobConfig.clearStageTable.label = Clear stage table
+toJobConfig.clearStageTable.help = Indicate if the stage table should be cleared
+
+# Placeholders to have some entities created
+ignored.label = Ignored
+ignored.help = This is completely ignored
+
+ignored.ignored.label = Ignored
+ignored.ignored.help = This is completely ignored

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-generic-jdbc/src/main/resources/generic-jdbc-connector-resources.properties
----------------------------------------------------------------------
diff --git a/connector/connector-generic-jdbc/src/main/resources/generic-jdbc-connector-resources.properties b/connector/connector-generic-jdbc/src/main/resources/generic-jdbc-connector-resources.properties
deleted file mode 100644
index a1302c0..0000000
--- a/connector/connector-generic-jdbc/src/main/resources/generic-jdbc-connector-resources.properties
+++ /dev/null
@@ -1,127 +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.
-
-# Generic JDBC Connector Resources
-
-############################
-# Connection Form
-#
-connection.label = Connection configuration
-connection.help = You must supply the information requested in order to \
-                   create a connection object.
-
-# jdbc driver
-connection.jdbcDriver.label = JDBC Driver Class
-connection.jdbcDriver.help = Enter the fully qualified class name of the JDBC \
-                   driver that will be used for establishing this connection.
-
-# connect string
-connection.connectionString.label = JDBC Connection String
-connection.connectionString.help = Enter the value of JDBC connection string to be \
-                   used by this connector for creating connections.
-
-# username string
-connection.username.label = Username
-connection.username.help = Enter the username to be used for connecting to the \
-                   database.
-
-# password string
-connection.password.label = Password
-connection.password.help = Enter the password to be used for connecting to the \
-                   database.
-
-# jdbc properties
-connection.jdbcProperties.label = JDBC Connection Properties
-connection.jdbcProperties.help = Enter any JDBC properties that should be \
-                   supplied during the creation of connection.
-
-# From Table Form
-#
-fromTable.label = From database configuration
-fromTable.help = You must supply the information requested in order to create \
-                 a job object.
-
-# From schema name
-fromTable.schemaName.label = Schema name
-fromTable.schemaName.help = Schema name to process data in the remote database
-
-# From table name
-fromTable.tableName.label = Table name
-fromTable.tableName.help = Table name to process data in the remote database
-
-# From table SQL
-fromTable.sql.label = Table SQL statement
-fromTable.sql.help = SQL statement to process data in the remote database
-
-# From table columns
-fromTable.columns.label = Table column names
-fromTable.columns.help = Specific columns of a table name or a table SQL
-
-# From table warehouse
-fromTable.warehouse.label = Data warehouse
-fromTable.warehouse.help = The root directory for data
-
-# From table datadir
-fromTable.dataDirectory.label = Data directory
-fromTable.dataDirectory.help = The sub-directory under warehouse for data
-
-# From table pcol
-fromTable.partitionColumn.label = Partition column name
-fromTable.partitionColumn.help = A specific column for data partition
-
-# From table pcol is null
-fromTable.partitionColumnNull.label = Nulls in partition column
-fromTable.partitionColumnNull.help = Whether there are null values in partition column
-
-# From table boundary
-fromTable.boundaryQuery.label = Boundary query
-fromTable.boundaryQuery.help = The boundary query for data partition
-
-# To table form
-#
-toTable.label = To database configuration
-toTable.help = You must supply the information requested in order to create \
-                 a job object.
-
-# From schema name
-toTable.schemaName.label = Schema name
-toTable.schemaName.help = Schema name to process data in the remote database
-
-# From table name
-toTable.tableName.label = Table name
-toTable.tableName.help = Table name to process data in the remote database
-
-# From table SQL
-toTable.sql.label = Table SQL statement
-toTable.sql.help = SQL statement to process data in the remote database
-
-# From table columns
-toTable.columns.label = Table column names
-toTable.columns.help = Specific columns of a table name or a table SQL
-
-# To stage table name
-toTable.stageTableName.label = Stage table name
-toTable.stageTableName.help = Name of the stage table to use
-
-# To clear stage table
-toTable.clearStageTable.label = Clear stage table
-toTable.clearStageTable.help = Indicate if the stage table should be cleared
-
-# Placeholders to have some entities created
-ignored.label = Ignored
-ignored.help = This is completely ignored
-
-ignored.ignored.label = Ignored
-ignored.ignored.help = This is completely ignored

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestExtractor.java
----------------------------------------------------------------------
diff --git a/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestExtractor.java b/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestExtractor.java
index 2b1dec2..776359a 100644
--- a/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestExtractor.java
+++ b/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestExtractor.java
@@ -21,7 +21,7 @@ import junit.framework.TestCase;
 
 import org.apache.sqoop.common.MutableContext;
 import org.apache.sqoop.common.MutableMapContext;
-import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
+import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
 import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
 import org.apache.sqoop.job.etl.Extractor;
 import org.apache.sqoop.job.etl.ExtractorContext;
@@ -67,10 +67,10 @@ public class TestExtractor extends TestCase {
   public void testQuery() throws Exception {
     MutableContext context = new MutableMapContext();
 
-    ConnectionConfiguration connectionConfig = new ConnectionConfiguration();
+    LinkConfiguration connectionConfig = new LinkConfiguration();
 
-    connectionConfig.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connectionConfig.connection.connectionString = GenericJdbcTestConstants.URL;
+    connectionConfig.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connectionConfig.link.connectionString = GenericJdbcTestConstants.URL;
 
     FromJobConfiguration jobConfig = new FromJobConfiguration();
 
@@ -99,10 +99,10 @@ public class TestExtractor extends TestCase {
   public void testSubquery() throws Exception {
     MutableContext context = new MutableMapContext();
 
-    ConnectionConfiguration connectionConfig = new ConnectionConfiguration();
+    LinkConfiguration connectionConfig = new LinkConfiguration();
 
-    connectionConfig.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connectionConfig.connection.connectionString = GenericJdbcTestConstants.URL;
+    connectionConfig.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connectionConfig.link.connectionString = GenericJdbcTestConstants.URL;
 
     FromJobConfiguration jobConfig = new FromJobConfiguration();
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestFromInitializer.java
----------------------------------------------------------------------
diff --git a/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestFromInitializer.java b/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestFromInitializer.java
index 2d07130..bf66bb1 100644
--- a/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestFromInitializer.java
+++ b/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestFromInitializer.java
@@ -23,7 +23,7 @@ import junit.framework.TestCase;
 
 import org.apache.sqoop.common.MutableContext;
 import org.apache.sqoop.common.MutableMapContext;
-import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
+import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
 import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
 import org.apache.sqoop.job.Constants;
 import org.apache.sqoop.job.etl.Initializer;
@@ -112,12 +112,12 @@ public class TestFromInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testTableName() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.fromTable.tableName = schemalessTableName;
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.fromJobConfig.tableName = schemalessTableName;
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);
@@ -138,13 +138,13 @@ public class TestFromInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testTableNameWithTableColumns() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.fromTable.tableName = schemalessTableName;
-    jobConf.fromTable.columns = tableColumns;
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.fromJobConfig.tableName = schemalessTableName;
+    jobConf.fromJobConfig.columns = tableColumns;
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);
@@ -165,13 +165,13 @@ public class TestFromInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testTableSql() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.fromTable.sql = schemalessTableSql;
-    jobConf.fromTable.partitionColumn = "DCOL";
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.fromJobConfig.sql = schemalessTableSql;
+    jobConf.fromJobConfig.partitionColumn = "DCOL";
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);
@@ -192,14 +192,14 @@ public class TestFromInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testTableSqlWithTableColumns() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.fromTable.sql = schemalessTableSql;
-    jobConf.fromTable.columns = tableColumns;
-    jobConf.fromTable.partitionColumn = "DCOL";
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.fromJobConfig.sql = schemalessTableSql;
+    jobConf.fromJobConfig.columns = tableColumns;
+    jobConf.fromJobConfig.partitionColumn = "DCOL";
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);
@@ -221,15 +221,15 @@ public class TestFromInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testTableNameWithSchema() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     String fullTableName = executor.delimitIdentifier(schemaName) + "." + executor.delimitIdentifier(tableName);
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.fromTable.schemaName = schemaName;
-    jobConf.fromTable.tableName = tableName;
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.fromJobConfig.schemaName = schemaName;
+    jobConf.fromJobConfig.tableName = tableName;
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);
@@ -250,16 +250,16 @@ public class TestFromInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testTableNameWithTableColumnsWithSchema() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     String fullTableName = executor.delimitIdentifier(schemaName) + "." + executor.delimitIdentifier(tableName);
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.fromTable.schemaName = schemaName;
-    jobConf.fromTable.tableName = tableName;
-    jobConf.fromTable.columns = tableColumns;
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.fromJobConfig.schemaName = schemaName;
+    jobConf.fromJobConfig.tableName = tableName;
+    jobConf.fromJobConfig.columns = tableColumns;
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);
@@ -280,16 +280,16 @@ public class TestFromInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testTableSqlWithSchema() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     String fullTableName = executor.delimitIdentifier(schemaName) + "." + executor.delimitIdentifier(tableName);
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.fromTable.schemaName = schemaName;
-    jobConf.fromTable.sql = tableSql;
-    jobConf.fromTable.partitionColumn = "DCOL";
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.fromJobConfig.schemaName = schemaName;
+    jobConf.fromJobConfig.sql = tableSql;
+    jobConf.fromJobConfig.partitionColumn = "DCOL";
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);
@@ -311,14 +311,14 @@ public class TestFromInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testGetSchemaForTable() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.fromTable.schemaName = schemaName;
-    jobConf.fromTable.tableName = tableName;
-    jobConf.fromTable.partitionColumn = "DCOL";
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.fromJobConfig.schemaName = schemaName;
+    jobConf.fromJobConfig.tableName = tableName;
+    jobConf.fromJobConfig.partitionColumn = "DCOL";
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);
@@ -327,19 +327,19 @@ public class TestFromInitializer extends TestCase {
     Initializer initializer = new GenericJdbcFromInitializer();
     initializer.initialize(initializerContext, connConf, jobConf);
     Schema schema = initializer.getSchema(initializerContext, connConf, jobConf);
-    assertEquals(getSchema(jobConf.fromTable.schemaName + "." + tableName), schema);
+    assertEquals(getSchema(jobConf.fromJobConfig.schemaName + "." + tableName), schema);
   }
 
   @SuppressWarnings("unchecked")
   public void testGetSchemaForSql() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.fromTable.schemaName = schemaName;
-    jobConf.fromTable.sql = tableSql;
-    jobConf.fromTable.partitionColumn = "DCOL";
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.fromJobConfig.schemaName = schemaName;
+    jobConf.fromJobConfig.sql = tableSql;
+    jobConf.fromJobConfig.partitionColumn = "DCOL";
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);
@@ -353,17 +353,17 @@ public class TestFromInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testTableSqlWithTableColumnsWithSchema() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     String fullTableName = executor.delimitIdentifier(schemaName) + "." + executor.delimitIdentifier(tableName);
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.fromTable.schemaName = schemaName;
-    jobConf.fromTable.sql = tableSql;
-    jobConf.fromTable.columns = tableColumns;
-    jobConf.fromTable.partitionColumn = "DCOL";
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.fromJobConfig.schemaName = schemaName;
+    jobConf.fromJobConfig.sql = tableSql;
+    jobConf.fromJobConfig.columns = tableColumns;
+    jobConf.fromJobConfig.partitionColumn = "DCOL";
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestLoader.java
----------------------------------------------------------------------
diff --git a/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestLoader.java b/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestLoader.java
index d7e8c6c..c349676 100644
--- a/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestLoader.java
+++ b/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestLoader.java
@@ -26,7 +26,7 @@ import java.util.Collection;
 
 import org.apache.sqoop.common.MutableContext;
 import org.apache.sqoop.common.MutableMapContext;
-import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
+import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
 import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
 import org.apache.sqoop.etl.io.DataReader;
 import org.apache.sqoop.job.etl.Loader;
@@ -82,10 +82,10 @@ public class TestLoader {
   public void testInsert() throws Exception {
     MutableContext context = new MutableMapContext();
 
-    ConnectionConfiguration connectionConfig = new ConnectionConfiguration();
+    LinkConfiguration connectionConfig = new LinkConfiguration();
 
-    connectionConfig.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connectionConfig.connection.connectionString = GenericJdbcTestConstants.URL;
+    connectionConfig.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connectionConfig.link.connectionString = GenericJdbcTestConstants.URL;
 
     ToJobConfiguration jobConfig = new ToJobConfiguration();
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestPartitioner.java
----------------------------------------------------------------------
diff --git a/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestPartitioner.java b/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestPartitioner.java
index f1023c8..949877b 100644
--- a/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestPartitioner.java
+++ b/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestPartitioner.java
@@ -29,7 +29,7 @@ import junit.framework.TestCase;
 
 import org.apache.sqoop.common.MutableContext;
 import org.apache.sqoop.common.MutableMapContext;
-import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
+import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
 import org.apache.sqoop.connector.jdbc.configuration.FromJobConfiguration;
 import org.apache.sqoop.job.etl.Partition;
 import org.apache.sqoop.job.etl.Partitioner;
@@ -55,7 +55,7 @@ public class TestPartitioner extends TestCase {
         GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE,
         String.valueOf(START + NUMBER_OF_ROWS - 1));
 
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     Partitioner partitioner = new GenericJdbcPartitioner();
@@ -86,7 +86,7 @@ public class TestPartitioner extends TestCase {
         GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE,
         String.valueOf(START + NUMBER_OF_ROWS - 1));
 
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     Partitioner partitioner = new GenericJdbcPartitioner();
@@ -115,7 +115,7 @@ public class TestPartitioner extends TestCase {
         GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE,
         String.valueOf(START + NUMBER_OF_ROWS - 1));
 
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     Partitioner partitioner = new GenericJdbcPartitioner();
@@ -151,7 +151,7 @@ public class TestPartitioner extends TestCase {
         GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE,
         String.valueOf((double)(START + NUMBER_OF_ROWS - 1)));
 
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     Partitioner partitioner = new GenericJdbcPartitioner();
@@ -182,7 +182,7 @@ public class TestPartitioner extends TestCase {
         GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE,
         String.valueOf((double)(START + NUMBER_OF_ROWS - 1)));
 
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     Partitioner partitioner = new GenericJdbcPartitioner();
@@ -203,7 +203,7 @@ public class TestPartitioner extends TestCase {
     context.setString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MINVALUE, String.valueOf(START));
     context.setString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE, String.valueOf(START + NUMBER_OF_ROWS - 1));
 
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     Partitioner partitioner = new GenericJdbcPartitioner();
@@ -226,7 +226,7 @@ public class TestPartitioner extends TestCase {
     context.setString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MINVALUE, String.valueOf(new BigDecimal(START)));
     context.setString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE, String.valueOf(new BigDecimal(START + NUMBER_OF_ROWS - 1)));
 
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     Partitioner partitioner = new GenericJdbcPartitioner();
@@ -247,7 +247,7 @@ public class TestPartitioner extends TestCase {
     context.setString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MINVALUE, String.valueOf(new BigDecimal(START)));
     context.setString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE, String.valueOf(new BigDecimal(START)));
 
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     Partitioner partitioner = new GenericJdbcPartitioner();
@@ -272,7 +272,7 @@ public class TestPartitioner extends TestCase {
         .toString());
 
 
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     Partitioner partitioner = new GenericJdbcPartitioner();
@@ -300,7 +300,7 @@ public class TestPartitioner extends TestCase {
         Time.valueOf("10:40:50").toString());
 
 
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     Partitioner partitioner = new GenericJdbcPartitioner();
@@ -325,7 +325,7 @@ public class TestPartitioner extends TestCase {
     context.setString(GenericJdbcConnectorConstants.CONNECTOR_JDBC_PARTITION_MAXVALUE,
         Timestamp.valueOf("2013-12-31 10:40:50.654").toString());
 
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     Partitioner partitioner = new GenericJdbcPartitioner();
@@ -349,7 +349,7 @@ public class TestPartitioner extends TestCase {
     context.setString(GenericJdbcConnectorConstants
         .CONNECTOR_JDBC_PARTITION_MAXVALUE, "1");
 
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     Partitioner partitioner = new GenericJdbcPartitioner();
@@ -372,7 +372,7 @@ public class TestPartitioner extends TestCase {
     context.setString(GenericJdbcConnectorConstants
         .CONNECTOR_JDBC_PARTITION_MAXVALUE, "Z");
 
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     Partitioner partitioner = new GenericJdbcPartitioner();
@@ -419,7 +419,7 @@ public class TestPartitioner extends TestCase {
     context.setString(GenericJdbcConnectorConstants
       .CONNECTOR_JDBC_PARTITION_MAXVALUE, "Warty Warthog");
 
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
     Partitioner partitioner = new GenericJdbcPartitioner();
     PartitionerContext partitionerContext = new PartitionerContext(context, 5, null);
@@ -442,7 +442,7 @@ public class TestPartitioner extends TestCase {
     context.setString(GenericJdbcConnectorConstants
         .CONNECTOR_JDBC_PARTITION_MAXVALUE, "AAF");
 
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
 
     Partitioner partitioner = new GenericJdbcPartitioner();
@@ -471,9 +471,9 @@ public class TestPartitioner extends TestCase {
     context.setString(GenericJdbcConnectorConstants
         .CONNECTOR_JDBC_PARTITION_MAXVALUE, "AAE");
 
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     FromJobConfiguration jobConf = new FromJobConfiguration();
-    jobConf.fromTable.partitionColumnNull = true;
+    jobConf.fromJobConfig.partitionColumnNull = true;
 
     Partitioner partitioner = new GenericJdbcPartitioner();
     PartitionerContext partitionerContext = new PartitionerContext(context, 5, null);

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestToInitializer.java
----------------------------------------------------------------------
diff --git a/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestToInitializer.java b/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestToInitializer.java
index 4767215..00c2aab 100644
--- a/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestToInitializer.java
+++ b/connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestToInitializer.java
@@ -21,7 +21,7 @@ import junit.framework.TestCase;
 import org.apache.sqoop.common.MutableContext;
 import org.apache.sqoop.common.MutableMapContext;
 import org.apache.sqoop.common.SqoopException;
-import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
+import org.apache.sqoop.connector.jdbc.configuration.LinkConfiguration;
 import org.apache.sqoop.connector.jdbc.configuration.ToJobConfiguration;
 import org.apache.sqoop.job.etl.Initializer;
 import org.apache.sqoop.job.etl.InitializerContext;
@@ -75,14 +75,14 @@ public class TestToInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testTableName() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     ToJobConfiguration jobConf = new ToJobConfiguration();
 
     String fullTableName = executor.delimitIdentifier(schemalessTableName);
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.toTable.tableName = schemalessTableName;
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.toJobConfig.tableName = schemalessTableName;
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);
@@ -96,15 +96,15 @@ public class TestToInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testTableNameWithTableColumns() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     ToJobConfiguration jobConf = new ToJobConfiguration();
 
     String fullTableName = executor.delimitIdentifier(schemalessTableName);
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.toTable.tableName = schemalessTableName;
-    jobConf.toTable.columns = tableColumns;
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.toJobConfig.tableName = schemalessTableName;
+    jobConf.toJobConfig.columns = tableColumns;
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);
@@ -118,12 +118,12 @@ public class TestToInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testTableSql() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     ToJobConfiguration jobConf = new ToJobConfiguration();
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.toTable.sql = schemalessTableSql;
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.toJobConfig.sql = schemalessTableSql;
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);
@@ -137,15 +137,15 @@ public class TestToInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testTableNameWithSchema() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     ToJobConfiguration jobConf = new ToJobConfiguration();
 
     String fullTableName = executor.delimitIdentifier(schemaName) + "." + executor.delimitIdentifier(tableName);
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.toTable.schemaName = schemaName;
-    jobConf.toTable.tableName = tableName;
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.toJobConfig.schemaName = schemaName;
+    jobConf.toJobConfig.tableName = tableName;
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);
@@ -159,16 +159,16 @@ public class TestToInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testTableNameWithTableColumnsWithSchema() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     ToJobConfiguration jobConf = new ToJobConfiguration();
 
     String fullTableName = executor.delimitIdentifier(schemaName) + "." + executor.delimitIdentifier(tableName);
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.toTable.schemaName = schemaName;
-    jobConf.toTable.tableName = tableName;
-    jobConf.toTable.columns = tableColumns;
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.toJobConfig.schemaName = schemaName;
+    jobConf.toJobConfig.tableName = tableName;
+    jobConf.toJobConfig.columns = tableColumns;
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);
@@ -182,13 +182,13 @@ public class TestToInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testTableSqlWithSchema() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     ToJobConfiguration jobConf = new ToJobConfiguration();
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.toTable.schemaName = schemaName;
-    jobConf.toTable.sql = tableSql;
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.toJobConfig.schemaName = schemaName;
+    jobConf.toJobConfig.sql = tableSql;
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);
@@ -216,13 +216,13 @@ public class TestToInitializer extends TestCase {
   }
 
   public void testNonExistingStageTable() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     ToJobConfiguration jobConf = new ToJobConfiguration();
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.toTable.tableName = schemalessTableName;
-    jobConf.toTable.stageTableName = stageTableName;
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.toJobConfig.tableName = schemalessTableName;
+    jobConf.toJobConfig.stageTableName = stageTableName;
 
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);
@@ -239,15 +239,15 @@ public class TestToInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testNonEmptyStageTable() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     ToJobConfiguration jobConf = new ToJobConfiguration();
 
     String fullStageTableName = executor.delimitIdentifier(stageTableName);
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.toTable.tableName = schemalessTableName;
-    jobConf.toTable.stageTableName = stageTableName;
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.toJobConfig.tableName = schemalessTableName;
+    jobConf.toJobConfig.stageTableName = stageTableName;
     createTable(fullStageTableName);
     executor.executeUpdate("INSERT INTO " + fullStageTableName +
       " VALUES(1, 1.1, 'one')");
@@ -264,17 +264,16 @@ public class TestToInitializer extends TestCase {
     }
   }
 
-  @SuppressWarnings("unchecked")
   public void testClearStageTableValidation() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     ToJobConfiguration jobConf = new ToJobConfiguration();
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
     //specifying clear stage table flag without specifying name of
     // the stage table
-    jobConf.toTable.tableName = schemalessTableName;
-    jobConf.toTable.clearStageTable = false;
+    jobConf.toJobConfig.tableName = schemalessTableName;
+    jobConf.toJobConfig.clearStageTable = false;
     ValidationRunner validationRunner = new ValidationRunner();
     ValidationResult result = validationRunner.validate(jobConf);
     assertEquals("User should not specify clear stage table flag without " +
@@ -282,49 +281,48 @@ public class TestToInitializer extends TestCase {
       Status.UNACCEPTABLE,
         result.getStatus());
     assertTrue(result.getMessages().containsKey(
-      "toTable"));
+      "toJobConfig"));
 
-    jobConf.toTable.clearStageTable = true;
+    jobConf.toJobConfig.clearStageTable = true;
     result = validationRunner.validate(jobConf);
     assertEquals("User should not specify clear stage table flag without " +
       "specifying name of the stage table",
       Status.UNACCEPTABLE,
         result.getStatus());
     assertTrue(result.getMessages().containsKey(
-      "toTable"));
+      "toJobConfig"));
   }
 
-  @SuppressWarnings("unchecked")
   public void testStageTableWithoutTable() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     ToJobConfiguration jobConf = new ToJobConfiguration();
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
     //specifying stage table without specifying table name
-    jobConf.toTable.stageTableName = stageTableName;
-    jobConf.toTable.sql = "";
+    jobConf.toJobConfig.stageTableName = stageTableName;
+    jobConf.toJobConfig.sql = "";
 
     ValidationRunner validationRunner = new ValidationRunner();
     ValidationResult result = validationRunner.validate(jobConf);
     assertEquals("Stage table name cannot be specified without specifying " +
       "table name", Status.UNACCEPTABLE, result.getStatus());
     assertTrue(result.getMessages().containsKey(
-      "toTable"));
+      "toJobConfig"));
   }
 
   @SuppressWarnings("unchecked")
   public void testClearStageTable() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     ToJobConfiguration jobConf = new ToJobConfiguration();
 
     String fullStageTableName = executor.delimitIdentifier(stageTableName);
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.toTable.tableName = schemalessTableName;
-    jobConf.toTable.stageTableName = stageTableName;
-    jobConf.toTable.clearStageTable = true;
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.toJobConfig.tableName = schemalessTableName;
+    jobConf.toJobConfig.stageTableName = stageTableName;
+    jobConf.toJobConfig.clearStageTable = true;
     createTable(fullStageTableName);
     executor.executeUpdate("INSERT INTO " + fullStageTableName +
       " VALUES(1, 1.1, 'one')");
@@ -340,15 +338,15 @@ public class TestToInitializer extends TestCase {
 
   @SuppressWarnings("unchecked")
   public void testStageTable() throws Exception {
-    ConnectionConfiguration connConf = new ConnectionConfiguration();
+    LinkConfiguration connConf = new LinkConfiguration();
     ToJobConfiguration jobConf = new ToJobConfiguration();
 
     String fullStageTableName = executor.delimitIdentifier(stageTableName);
 
-    connConf.connection.jdbcDriver = GenericJdbcTestConstants.DRIVER;
-    connConf.connection.connectionString = GenericJdbcTestConstants.URL;
-    jobConf.toTable.tableName = schemalessTableName;
-    jobConf.toTable.stageTableName = stageTableName;
+    connConf.link.jdbcDriver = GenericJdbcTestConstants.DRIVER;
+    connConf.link.connectionString = GenericJdbcTestConstants.URL;
+    jobConf.toJobConfig.tableName = schemalessTableName;
+    jobConf.toJobConfig.stageTableName = stageTableName;
     createTable(fullStageTableName);
     MutableContext context = new MutableMapContext();
     InitializerContext initializerContext = new InitializerContext(context);

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsConfigUpgrader.java
----------------------------------------------------------------------
diff --git a/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsConfigUpgrader.java b/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsConfigUpgrader.java
new file mode 100644
index 0000000..47b186c
--- /dev/null
+++ b/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsConfigUpgrader.java
@@ -0,0 +1,83 @@
+/*
+ * 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.connector.hdfs;
+
+import org.apache.log4j.Logger;
+import org.apache.sqoop.common.SqoopException;
+import org.apache.sqoop.connector.spi.RepositoryUpgrader;
+import org.apache.sqoop.model.MConnectionForms;
+import org.apache.sqoop.model.MForm;
+import org.apache.sqoop.model.MInput;
+import org.apache.sqoop.model.MJobForms;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class HdfsConfigUpgrader extends RepositoryUpgrader {
+  private static final Logger LOG =
+      Logger.getLogger(HdfsConfigUpgrader.class);
+
+  /*
+   * For now, there is no real upgrade. So copy all data over,
+   * set the validation messages and error messages to be the same as for the
+   * inputs in the original one.
+   */
+
+  @Override
+  public void upgrade(MConnectionForms original,
+                      MConnectionForms upgradeTarget) {
+    doUpgrade(original.getForms(), upgradeTarget.getForms());
+  }
+
+  @Override
+  public void upgrade(MJobForms original, MJobForms upgradeTarget) {
+    doUpgrade(original.getForms(), upgradeTarget.getForms());
+  }
+
+  @SuppressWarnings("unchecked")
+  private void doUpgrade(List<MForm> original, List<MForm> target) {
+    // Easier to find the form in the original forms list if we use a map.
+    // Since the constructor of MJobForms takes a list,
+    // index is not guaranteed to be the same, so we need to look for
+    // equivalence
+    Map<String, MForm> formMap = new HashMap<String, MForm>();
+    for (MForm form : original) {
+      formMap.put(form.getName(), form);
+    }
+    for (MForm form : target) {
+      List<MInput<?>> inputs = form.getInputs();
+      MForm originalForm = formMap.get(form.getName());
+      if (originalForm == null) {
+        LOG.warn("Form: '" + form.getName() + "' not present in old " +
+            "connector. So it and its inputs will not be transferred by the upgrader.");
+        continue;
+      }
+      for (MInput input : inputs) {
+        try {
+          MInput originalInput = originalForm.getInput(input.getName());
+          input.setValue(originalInput.getValue());
+        } catch (SqoopException ex) {
+          LOG.warn("Input: '" + input.getName() + "' not present in old " +
+              "connector. So it will not be transferred by the upgrader.");
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsConnector.java
----------------------------------------------------------------------
diff --git a/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsConnector.java b/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsConnector.java
index 883636c..70833a0 100644
--- a/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsConnector.java
+++ b/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsConnector.java
@@ -20,10 +20,10 @@ package org.apache.sqoop.connector.hdfs;
 
 import org.apache.sqoop.common.Direction;
 import org.apache.sqoop.common.VersionInfo;
-import org.apache.sqoop.connector.hdfs.configuration.ConnectionConfiguration;
+import org.apache.sqoop.connector.hdfs.configuration.LinkConfiguration;
 import org.apache.sqoop.connector.hdfs.configuration.FromJobConfiguration;
 import org.apache.sqoop.connector.hdfs.configuration.ToJobConfiguration;
-import org.apache.sqoop.connector.spi.MetadataUpgrader;
+import org.apache.sqoop.connector.spi.RepositoryUpgrader;
 import org.apache.sqoop.connector.spi.SqoopConnector;
 import org.apache.sqoop.job.etl.From;
 import org.apache.sqoop.job.etl.To;
@@ -71,8 +71,8 @@ public class HdfsConnector extends SqoopConnector {
    * @return Get connection configuration class
    */
   @Override
-  public Class getConnectionConfigurationClass() {
-    return ConnectionConfiguration.class;
+  public Class getLinkConfigurationClass() {
+    return LinkConfiguration.class;
   }
 
   /**
@@ -108,7 +108,7 @@ public class HdfsConnector extends SqoopConnector {
   }
 
   /**
-   * Returns validation object that Sqoop framework can use to validate user
+   * Returns validation object that Sqoop can use to validate user
    * supplied forms before accepting them. This object will be used both for
    * connection and job forms.
    *
@@ -120,13 +120,13 @@ public class HdfsConnector extends SqoopConnector {
   }
 
   /**
-   * Returns an {@linkplain org.apache.sqoop.connector.spi.MetadataUpgrader} object that can upgrade the
+   * Returns an {@linkplain org.apache.sqoop.connector.spi.RepositoryUpgrader} object that can upgrade the
    * connection and job metadata.
    *
    * @return MetadataUpgrader object
    */
   @Override
-  public MetadataUpgrader getMetadataUpgrader() {
-    return new HdfsMetadataUpgrader();
+  public RepositoryUpgrader getRepositoryUpgrader() {
+    return new HdfsConfigUpgrader();
   }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsConstants.java b/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsConstants.java
index a27aff1..6e369c6 100644
--- a/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsConstants.java
+++ b/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsConstants.java
@@ -24,7 +24,7 @@ public final class HdfsConstants extends Constants {
 
   // Resource bundle name
   public static final String RESOURCE_BUNDLE_NAME =
-          "hdfs-connector-resources";
+          "hdfs-connector-config";
 
   public static final char DEFAULT_RECORD_DELIMITER = '\n';
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsExtractor.java
----------------------------------------------------------------------
diff --git a/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsExtractor.java b/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsExtractor.java
index 7447071..436d243 100644
--- a/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsExtractor.java
+++ b/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsExtractor.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
 import org.apache.hadoop.util.LineReader;
 import org.apache.sqoop.common.SqoopException;
-import org.apache.sqoop.connector.hdfs.configuration.ConnectionConfiguration;
+import org.apache.sqoop.connector.hdfs.configuration.LinkConfiguration;
 import org.apache.sqoop.connector.hdfs.configuration.FromJobConfiguration;
 import org.apache.sqoop.etl.io.DataWriter;
 import org.apache.sqoop.job.etl.Extractor;
@@ -42,7 +42,7 @@ import java.io.IOException;
  * Extract from HDFS.
  * Default field delimiter of a record is comma.
  */
-public class HdfsExtractor extends Extractor<ConnectionConfiguration, FromJobConfiguration, HdfsPartition> {
+public class HdfsExtractor extends Extractor<LinkConfiguration, FromJobConfiguration, HdfsPartition> {
 
   public static final Logger LOG = Logger.getLogger(HdfsExtractor.class);
 
@@ -52,7 +52,7 @@ public class HdfsExtractor extends Extractor<ConnectionConfiguration, FromJobCon
 
   @Override
   public void extract(ExtractorContext context,
-      ConnectionConfiguration connectionConfiguration,
+      LinkConfiguration connectionConfiguration,
       FromJobConfiguration jobConfiguration, HdfsPartition partition) {
 
     conf = ((PrefixContext) context.getContext()).getConfiguration();

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsInitializer.java
----------------------------------------------------------------------
diff --git a/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsInitializer.java b/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsInitializer.java
index d2d12a8..923f904 100644
--- a/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsInitializer.java
+++ b/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsInitializer.java
@@ -29,17 +29,16 @@ public class HdfsInitializer extends Initializer {
    * promoted to all other part of the workflow automatically.
    *
    * @param context Initializer context object
-   * @param connection       Connector's connection configuration object
-   * @param job      Connector's job configuration object
+   * @param linkConf       Connector's link configuration object
+   * @param jobConf      Connector's job configuration object
    */
   @Override
-  public void initialize(InitializerContext context, Object connection, Object job) {
+  public void initialize(InitializerContext context, Object linkConf, Object jobConf) {
 
   }
 
-
   @Override
-  public Schema getSchema(InitializerContext context, Object connection, Object job) {
+  public Schema getSchema(InitializerContext context, Object linkConf, Object jobConf) {
     return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/049994a0/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsLoader.java
----------------------------------------------------------------------
diff --git a/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsLoader.java b/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsLoader.java
index 61676fe..4c546ba 100644
--- a/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsLoader.java
+++ b/connector/connector-hdfs/src/main/java/org/apache/sqoop/connector/hdfs/HdfsLoader.java
@@ -23,8 +23,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.sqoop.common.PrefixContext;
 import org.apache.sqoop.common.SqoopException;
-import org.apache.sqoop.connector.hdfs.configuration.ConnectionConfiguration;
-import org.apache.sqoop.connector.hdfs.configuration.OutputFormat;
+import org.apache.sqoop.connector.hdfs.configuration.LinkConfiguration;
+import org.apache.sqoop.connector.hdfs.configuration.ToFormat;
 import org.apache.sqoop.connector.hdfs.configuration.ToJobConfiguration;
 import org.apache.sqoop.connector.hdfs.hdfsWriter.GenericHdfsWriter;
 import org.apache.sqoop.connector.hdfs.hdfsWriter.HdfsSequenceWriter;
@@ -37,24 +37,24 @@ import org.apache.sqoop.utils.ClassUtils;
 import java.io.IOException;
 import java.util.UUID;
 
-public class HdfsLoader extends Loader<ConnectionConfiguration, ToJobConfiguration> {
+public class HdfsLoader extends Loader<LinkConfiguration, ToJobConfiguration> {
   /**
    * Load data to target.
    *
    * @param context Loader context object
-   * @param connection       Connection configuration
-   * @param job      Job configuration
+   * @param linkConf       Link configuration
+   * @param toJobConf      Job configuration
    * @throws Exception
    */
   @Override
-  public void load(LoaderContext context, ConnectionConfiguration connection, ToJobConfiguration job) throws Exception {
+  public void load(LoaderContext context, LinkConfiguration linkConf, ToJobConfiguration toJobConf) throws Exception {
 
     DataReader reader = context.getDataReader();
 
     Configuration conf = ((PrefixContext)context.getContext()).getConfiguration();
 
-    String directoryName = job.output.outputDirectory;
-    String codecname = getCompressionCodecName(job);
+    String directoryName = toJobConf.toJobConfig.outputDirectory;
+    String codecname = getCompressionCodecName(toJobConf);
 
     CompressionCodec codec = null;
     if (codecname != null) {
@@ -73,12 +73,12 @@ public class HdfsLoader extends Loader<ConnectionConfiguration, ToJobConfigurati
       }
     }
 
-    String filename = directoryName + "/" + UUID.randomUUID() + getExtension(job,codec);
+    String filename = directoryName + "/" + UUID.randomUUID() + getExtension(toJobConf,codec);
 
     try {
       Path filepath = new Path(filename);
 
-      GenericHdfsWriter filewriter = getWriter(job);
+      GenericHdfsWriter filewriter = getWriter(toJobConf);
 
       filewriter.initialize(filepath,conf,codec);
 
@@ -95,18 +95,15 @@ public class HdfsLoader extends Loader<ConnectionConfiguration, ToJobConfigurati
 
   }
 
-  private GenericHdfsWriter getWriter(ToJobConfiguration job) {
-    if (job.output.outputFormat == OutputFormat.SEQUENCE_FILE)
-      return new HdfsSequenceWriter();
-    else
-      return new HdfsTextWriter();
+  private GenericHdfsWriter getWriter(ToJobConfiguration toJobConf) {
+    return (toJobConf.toJobConfig.outputFormat == ToFormat.SEQUENCE_FILE) ? new HdfsSequenceWriter()
+        : new HdfsTextWriter();
   }
 
-
-  private String getCompressionCodecName(ToJobConfiguration jobConf) {
-    if(jobConf.output.compression == null)
+  private String getCompressionCodecName(ToJobConfiguration toJobConf) {
+    if(toJobConf.toJobConfig.compression == null)
       return null;
-    switch(jobConf.output.compression) {
+    switch(toJobConf.toJobConfig.compression) {
       case NONE:
         return null;
       case DEFAULT:
@@ -124,14 +121,14 @@ public class HdfsLoader extends Loader<ConnectionConfiguration, ToJobConfigurati
       case SNAPPY:
         return "org.apache.hadoop.io.compress.SnappyCodec";
       case CUSTOM:
-        return jobConf.output.customCompression.trim();
+        return toJobConf.toJobConfig.customCompression.trim();
     }
     return null;
   }
 
   //TODO: We should probably support configurable extensions at some point
-  private static String getExtension(ToJobConfiguration job, CompressionCodec codec) {
-    if (job.output.outputFormat == OutputFormat.SEQUENCE_FILE)
+  private static String getExtension(ToJobConfiguration toJobConf, CompressionCodec codec) {
+    if (toJobConf.toJobConfig.outputFormat == ToFormat.SEQUENCE_FILE)
       return ".seq";
     if (codec == null)
       return ".txt";