You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sqoop.apache.org by bo...@apache.org on 2017/09/25 10:36:57 UTC

[2/3] sqoop git commit: SQOOP-3216: Expanded Metastore support for MySql, Oracle, Postgresql, MSSql, and DB2

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d0770ac6/src/java/org/apache/sqoop/metastore/hsqldb/HsqldbJobStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/metastore/hsqldb/HsqldbJobStorage.java b/src/java/org/apache/sqoop/metastore/hsqldb/HsqldbJobStorage.java
deleted file mode 100644
index a0f29fd..0000000
--- a/src/java/org/apache/sqoop/metastore/hsqldb/HsqldbJobStorage.java
+++ /dev/null
@@ -1,805 +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.metastore.hsqldb;
-
-import java.io.IOException;
-
-import java.sql.Connection;
-import java.sql.DatabaseMetaData;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import org.apache.hadoop.conf.Configuration;
-
-import com.cloudera.sqoop.SqoopOptions;
-import com.cloudera.sqoop.metastore.JobData;
-import com.cloudera.sqoop.metastore.JobStorage;
-import com.cloudera.sqoop.tool.SqoopTool;
-
-/**
- * JobStorage implementation that uses an HSQLDB-backed database to
- * hold job information.
- */
-public class HsqldbJobStorage extends JobStorage {
-
-  public static final Log LOG = LogFactory.getLog(
-      HsqldbJobStorage.class.getName());
-
-  /** descriptor key identifying the connect string for the metastore. */
-  public static final String META_CONNECT_KEY = "metastore.connect.string";
-
-  /** descriptor key identifying the username to use when connecting
-   * to the metastore.
-   */
-  public static final String META_USERNAME_KEY = "metastore.username";
-
-  /** descriptor key identifying the password to use when connecting
-   * to the metastore.
-   */
-  public static final String META_PASSWORD_KEY = "metastore.password";
-
-
-  /** Default name for the root metadata table in HSQLDB. */
-  private static final String DEFAULT_ROOT_TABLE_NAME = "SQOOP_ROOT";
-
-  /** Configuration key used to override root table name. */
-  public static final String ROOT_TABLE_NAME_KEY =
-       "sqoop.hsqldb.root.table.name";
-
-  /** root metadata table key used to define the current schema version. */
-  private static final String STORAGE_VERSION_KEY =
-      "sqoop.hsqldb.job.storage.version";
-
-  /** The current version number for the schema edition. */
-  private static final int CUR_STORAGE_VERSION = 0;
-
-  /** root metadata table key used to define the job table name. */
-  private static final String SESSION_TABLE_KEY =
-      "sqoop.hsqldb.job.info.table";
-
-  /** Default value for SESSION_TABLE_KEY. */
-  private static final String DEFAULT_SESSION_TABLE_NAME =
-      "SQOOP_SESSIONS";
-
-  /** Per-job key with propClass 'schema' that defines the set of
-   * properties valid to be defined for propClass 'SqoopOptions'. */
-  private static final String PROPERTY_SET_KEY =
-      "sqoop.property.set.id";
-
-  /** Current value for PROPERTY_SET_KEY. */
-  private static final String CUR_PROPERTY_SET_ID = "0";
-
-  // The following are values for propClass in the v0 schema which
-  // describe different aspects of the stored metadata.
-
-  /** Property class for properties about the stored data itself. */
-  private static final String PROPERTY_CLASS_SCHEMA = "schema";
-
-  /** Property class for properties that are loaded into SqoopOptions. */
-  private static final String PROPERTY_CLASS_SQOOP_OPTIONS = "SqoopOptions";
-
-  /** Property class for properties that are loaded into a Configuration. */
-  private static final String PROPERTY_CLASS_CONFIG = "config";
-
-  /**
-   * Per-job key with propClass 'schema' that specifies the SqoopTool
-   * to load.
-   */
-  private static final String SQOOP_TOOL_KEY = "sqoop.tool";
-
-
-  private Map<String, String> connectedDescriptor;
-  private String metastoreConnectStr;
-  private String metastoreUser;
-  private String metastorePassword;
-  private Connection connection;
-
-  protected Connection getConnection() {
-    return this.connection;
-  }
-
-  // After connection to the database and initialization of the
-  // schema, this holds the name of the job table.
-  private String jobTableName;
-
-  protected void setMetastoreConnectStr(String connectStr) {
-    this.metastoreConnectStr = connectStr;
-  }
-
-  protected void setMetastoreUser(String user) {
-    this.metastoreUser = user;
-  }
-
-  protected void setMetastorePassword(String pass) {
-    this.metastorePassword = pass;
-  }
-
-  private static final String DB_DRIVER_CLASS = "org.hsqldb.jdbcDriver";
-
-  /**
-   * Set the descriptor used to open() this storage.
-   */
-  protected void setConnectedDescriptor(Map<String, String> descriptor) {
-    this.connectedDescriptor = descriptor;
-  }
-
-  @Override
-  /**
-   * Initialize the connection to the database.
-   */
-  public void open(Map<String, String> descriptor) throws IOException {
-    setMetastoreConnectStr(descriptor.get(META_CONNECT_KEY));
-    setMetastoreUser(descriptor.get(META_USERNAME_KEY));
-    setMetastorePassword(descriptor.get(META_PASSWORD_KEY));
-    setConnectedDescriptor(descriptor);
-
-    init();
-  }
-
-  protected void init() throws IOException {
-    try {
-      // Load/initialize the JDBC driver.
-      Class.forName(DB_DRIVER_CLASS);
-    } catch (ClassNotFoundException cnfe) {
-      throw new IOException("Could not load HSQLDB JDBC driver", cnfe);
-    }
-
-    try {
-      if (null == metastoreUser) {
-        this.connection = DriverManager.getConnection(metastoreConnectStr);
-      } else {
-        this.connection = DriverManager.getConnection(metastoreConnectStr,
-            metastoreUser, metastorePassword);
-      }
-
-      connection.setTransactionIsolation(Connection.TRANSACTION_SERIALIZABLE);
-      connection.setAutoCommit(false);
-
-      // Initialize the root schema.
-      if (!rootTableExists()) {
-        createRootTable();
-      }
-
-      // Check the schema version.
-      String curStorageVerStr = getRootProperty(STORAGE_VERSION_KEY, null);
-      int actualStorageVer = -1;
-      try {
-        actualStorageVer = Integer.valueOf(curStorageVerStr);
-      } catch (NumberFormatException nfe) {
-        LOG.warn("Could not interpret as a number: " + curStorageVerStr);
-      }
-      if (actualStorageVer != CUR_STORAGE_VERSION) {
-        LOG.error("Can not interpret metadata schema");
-        LOG.error("The metadata schema version is " + curStorageVerStr);
-        LOG.error("The highest version supported is " + CUR_STORAGE_VERSION);
-        LOG.error("To use this version of Sqoop, "
-            + "you must downgrade your metadata schema.");
-        throw new IOException("Invalid metadata version.");
-      }
-
-      // Initialize the versioned schema.
-      initV0Schema();
-    } catch (SQLException sqle) {
-      if (null != connection) {
-        try {
-          connection.rollback();
-        } catch (SQLException e2) {
-          LOG.warn("Error rolling back transaction in error handler: " + e2);
-        }
-      }
-
-      throw new IOException("Exception creating SQL connection", sqle);
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (null != this.connection) {
-      try {
-        LOG.debug("Flushing current transaction");
-        this.connection.commit();
-      } catch (SQLException sqlE) {
-        throw new IOException("Exception committing connection", sqlE);
-      }
-
-      try {
-        LOG.debug("Closing connection");
-        this.connection.close();
-      } catch (SQLException sqlE) {
-        throw new IOException("Exception closing connection", sqlE);
-      } finally {
-        this.connection = null;
-      }
-    }
-  }
-
-  @Override
-  /** {@inheritDoc} */
-  public boolean canAccept(Map<String, String> descriptor) {
-    // We return true if the desciptor contains a connect string to find
-    // the database.
-    return descriptor.get(META_CONNECT_KEY) != null;
-  }
-
-  @Override
-  /** {@inheritDoc} */
-  public JobData read(String jobName) throws IOException {
-    try {
-      if (!jobExists(jobName)) {
-        LOG.error("Cannot restore job: " + jobName);
-        LOG.error("(No such job)");
-        throw new IOException("Cannot restore missing job " + jobName);
-      }
-
-      LOG.debug("Restoring job: " + jobName);
-      Properties schemaProps = getV0Properties(jobName,
-          PROPERTY_CLASS_SCHEMA);
-      Properties sqoopOptProps = getV0Properties(jobName,
-          PROPERTY_CLASS_SQOOP_OPTIONS);
-      Properties configProps = getV0Properties(jobName,
-          PROPERTY_CLASS_CONFIG);
-
-      // Check that we're not using a saved job from a previous
-      // version whose functionality has been deprecated.
-      String thisPropSetId = schemaProps.getProperty(PROPERTY_SET_KEY);
-      LOG.debug("System property set: " + CUR_PROPERTY_SET_ID);
-      LOG.debug("Stored property set: " + thisPropSetId);
-      if (!CUR_PROPERTY_SET_ID.equals(thisPropSetId)) {
-        LOG.warn("The property set present in this database was written by");
-        LOG.warn("an incompatible version of Sqoop. This may result in an");
-        LOG.warn("incomplete operation.");
-        // TODO(aaron): Should this fail out-right?
-      }
-
-      String toolName = schemaProps.getProperty(SQOOP_TOOL_KEY);
-      if (null == toolName) {
-        // Don't know what tool to create.
-        throw new IOException("Incomplete metadata; missing "
-            + SQOOP_TOOL_KEY);
-      }
-
-      SqoopTool tool = SqoopTool.getTool(toolName);
-      if (null == tool) {
-        throw new IOException("Error in job metadata: invalid tool "
-            + toolName);
-      }
-
-      Configuration conf = new Configuration();
-      for (Map.Entry<Object, Object> entry : configProps.entrySet()) {
-        conf.set(entry.getKey().toString(), entry.getValue().toString());
-      }
-
-      SqoopOptions opts = new SqoopOptions();
-      opts.setConf(conf);
-      opts.loadProperties(sqoopOptProps);
-
-      // Set the job connection information for this job.
-      opts.setJobName(jobName);
-      opts.setStorageDescriptor(connectedDescriptor);
-
-      return new JobData(opts, tool);
-    } catch (SQLException sqlE) {
-      throw new IOException("Error communicating with database", sqlE);
-    }
-  }
-
-  private boolean jobExists(String jobName) throws SQLException {
-    PreparedStatement s = connection.prepareStatement(
-        "SELECT COUNT(job_name) FROM " + this.jobTableName
-        + " WHERE job_name = ? GROUP BY job_name");
-    ResultSet rs = null;
-    try {
-      s.setString(1, jobName);
-      rs = s.executeQuery();
-      if (rs.next()) {
-        return true; // We got a result, meaning the job exists.
-      }
-    } finally {
-      if (null != rs) {
-        try {
-          rs.close();
-        } catch (SQLException sqlE) {
-          LOG.warn("Error closing result set: " + sqlE);
-        }
-      }
-
-      s.close();
-    }
-
-    return false; // No result.
-  }
-
-  @Override
-  /** {@inheritDoc} */
-  public void delete(String jobName) throws IOException {
-    try {
-      if (!jobExists(jobName)) {
-        LOG.error("No such job: " + jobName);
-      } else {
-        LOG.debug("Deleting job: " + jobName);
-        PreparedStatement s = connection.prepareStatement("DELETE FROM "
-            + this.jobTableName + " WHERE job_name = ?");
-        try {
-          s.setString(1, jobName);
-          s.executeUpdate();
-        } finally {
-          s.close();
-        }
-        connection.commit();
-      }
-    } catch (SQLException sqlEx) {
-      try {
-        connection.rollback();
-      } catch (SQLException e2) {
-        LOG.warn("Error rolling back transaction in error handler: " + e2);
-      }
-      throw new IOException("Error communicating with database", sqlEx);
-    }
-  }
-
-  @Override
-  /** {@inheritDoc} */
-  public void create(String jobName, JobData data)
-      throws IOException {
-    try {
-      if (jobExists(jobName)) {
-        LOG.error("Cannot create job " + jobName
-            + ": it already exists");
-        throw new IOException("Job " + jobName + " already exists");
-      }
-    } catch (SQLException sqlE) {
-      throw new IOException("Error communicating with database", sqlE);
-    }
-
-    createInternal(jobName, data);
-  }
-
-  /**
-   * Actually insert/update the resources for this job.
-   */
-  private void createInternal(String jobName, JobData data)
-      throws IOException {
-    try {
-      LOG.debug("Creating job: " + jobName);
-
-      // Save the name of the Sqoop tool.
-      setV0Property(jobName, PROPERTY_CLASS_SCHEMA, SQOOP_TOOL_KEY,
-          data.getSqoopTool().getToolName());
-
-      // Save the property set id.
-      setV0Property(jobName, PROPERTY_CLASS_SCHEMA, PROPERTY_SET_KEY,
-          CUR_PROPERTY_SET_ID);
-
-      // Save all properties of the SqoopOptions.
-      Properties props = data.getSqoopOptions().writeProperties();
-      setV0Properties(jobName, PROPERTY_CLASS_SQOOP_OPTIONS, props);
-
-      // And save all unique properties of the configuration.
-      Configuration saveConf = data.getSqoopOptions().getConf();
-      Configuration baseConf = new Configuration();
-
-      for (Map.Entry<String, String> entry : saveConf) {
-        String key = entry.getKey();
-        String rawVal = saveConf.getRaw(key);
-        String baseVal = baseConf.getRaw(key);
-        if (baseVal != null && rawVal.equals(baseVal)) {
-          continue; // Don't save this; it's set in the base configuration.
-        }
-
-        LOG.debug("Saving " + key + " => " + rawVal + " / " + baseVal);
-        setV0Property(jobName, PROPERTY_CLASS_CONFIG, key, rawVal);
-      }
-
-      connection.commit();
-    } catch (SQLException sqlE) {
-      try {
-        connection.rollback();
-      } catch (SQLException sqlE2) {
-        LOG.warn("Exception rolling back transaction during error handling: "
-            + sqlE2);
-      }
-      throw new IOException("Error communicating with database", sqlE);
-    }
-  }
-
-  @Override
-  /** {@inheritDoc} */
-  public void update(String jobName, JobData data)
-      throws IOException {
-    try {
-      if (!jobExists(jobName)) {
-        LOG.error("Cannot update job " + jobName + ": not found");
-        throw new IOException("Job " + jobName + " does not exist");
-      }
-    } catch (SQLException sqlE) {
-      throw new IOException("Error communicating with database", sqlE);
-    }
-
-    // Since we set properties with update-or-insert, this is the same
-    // as create on this system.
-    createInternal(jobName, data);
-  }
-
-  @Override
-  /** {@inheritDoc} */
-  public List<String> list() throws IOException {
-    ResultSet rs = null;
-    try {
-      PreparedStatement s = connection.prepareStatement(
-          "SELECT DISTINCT job_name FROM " + this.jobTableName);
-      try {
-        rs = s.executeQuery();
-        ArrayList<String> jobs = new ArrayList<String>();
-        while (rs.next()) {
-          jobs.add(rs.getString(1));
-        }
-
-        return jobs;
-      } finally {
-        if (null != rs) {
-          try {
-            rs.close();
-          } catch (SQLException sqlE) {
-            LOG.warn("Error closing resultset: " + sqlE);
-          }
-        }
-
-        if (null != s) {
-          s.close();
-        }
-      }
-    } catch (SQLException sqlE) {
-      throw new IOException("Error communicating with database", sqlE);
-    }
-  }
-
-  // Determine the name to use for the root metadata table.
-  private String getRootTableName() {
-    Configuration conf = getConf();
-    return conf.get(ROOT_TABLE_NAME_KEY, DEFAULT_ROOT_TABLE_NAME);
-  }
-
-  private boolean tableExists(String table) throws SQLException {
-    LOG.debug("Checking for table: " + table);
-    DatabaseMetaData dbmd = connection.getMetaData();
-    String [] tableTypes = { "TABLE" };
-    ResultSet rs = dbmd.getTables(null, null, null, tableTypes);
-    if (null != rs) {
-      try {
-        while (rs.next()) {
-          if (table.equalsIgnoreCase(rs.getString("TABLE_NAME"))) {
-            LOG.debug("Found table: " + table);
-            return true;
-          }
-        }
-      } finally {
-        rs.close();
-      }
-    }
-
-    LOG.debug("Could not find table.");
-    return false;
-  }
-
-  private boolean rootTableExists() throws SQLException {
-    String rootTableName = getRootTableName();
-    return tableExists(rootTableName);
-  }
-
-  private void createRootTable() throws SQLException {
-    String rootTableName = getRootTableName();
-    LOG.debug("Creating root table: " + rootTableName);
-
-    // TODO: Sanity-check the value of rootTableName to ensure it is
-    // not a SQL-injection attack vector.
-    Statement s = connection.createStatement();
-    try {
-      s.executeUpdate("CREATE TABLE " + rootTableName + " ("
-          + "version INT, "
-          + "propname VARCHAR(128) NOT NULL, "
-          + "propval VARCHAR(256), "
-          + "CONSTRAINT " + rootTableName + "_unq UNIQUE (version, propname))");
-    } finally {
-      s.close();
-    }
-
-    setRootProperty(STORAGE_VERSION_KEY, null,
-        Integer.toString(CUR_STORAGE_VERSION));
-
-    LOG.debug("Saving root table.");
-    connection.commit();
-  }
-
-  /**
-   * Look up a value for the specified version (may be null) in the
-   * root metadata table.
-   */
-  private String getRootProperty(String propertyName, Integer version)
-      throws SQLException {
-    LOG.debug("Looking up property " + propertyName + " for version "
-        + version);
-    PreparedStatement s = null;
-    ResultSet rs = null;
-
-    try {
-      if (null == version) {
-        s = connection.prepareStatement(
-          "SELECT propval FROM " + getRootTableName()
-          + " WHERE version IS NULL AND propname = ?");
-        s.setString(1, propertyName);
-      } else {
-        s = connection.prepareStatement(
-          "SELECT propval FROM " + getRootTableName() + " WHERE version = ? "
-          + " AND propname = ?");
-        s.setInt(1, version);
-        s.setString(2, propertyName);
-      }
-
-      rs = s.executeQuery();
-      if (!rs.next()) {
-        LOG.debug(" => (no result)");
-        return null; // No such result.
-      } else {
-        String result = rs.getString(1); // Return the only result col.
-        LOG.debug(" => " + result);
-        return result;
-      }
-    } finally {
-      if (null != rs) {
-        try {
-          rs.close();
-        } catch (SQLException sqlE) {
-          LOG.warn("Error closing resultset: " + sqlE);
-        }
-      }
-
-      if (null != s) {
-        s.close();
-      }
-    }
-  }
-
-  /**
-   * Set a value for the specified version (may be null) in the root
-   * metadata table.
-   */
-  private void setRootProperty(String propertyName, Integer version,
-      String val) throws SQLException {
-    LOG.debug("Setting property " + propertyName + " for version "
-        + version + " => " + val);
-
-    PreparedStatement s;
-    String curVal = getRootProperty(propertyName, version);
-    if (null == curVal) {
-      // INSERT the row.
-      s = connection.prepareStatement("INSERT INTO " + getRootTableName()
-          + " (propval, propname, version) VALUES ( ? , ? , ? )");
-    } else if (version == null) {
-      // UPDATE an existing row with a null version
-      s = connection.prepareStatement("UPDATE " + getRootTableName()
-          + " SET propval = ? WHERE  propname = ? AND version IS NULL");
-    } else {
-      // UPDATE an existing row with non-null version.
-      s = connection.prepareStatement("UPDATE " + getRootTableName()
-          + " SET propval = ? WHERE  propname = ? AND version = ?");
-    }
-
-    try {
-      s.setString(1, val);
-      s.setString(2, propertyName);
-      if (null != version) {
-        s.setInt(3, version);
-      }
-      s.executeUpdate();
-    } finally {
-      s.close();
-    }
-  }
-
-  /**
-   * Create the jobs table in the V0 schema.
-   */
-  private void createJobTable() throws SQLException {
-    String curTableName = DEFAULT_SESSION_TABLE_NAME;
-    int tableNum = -1;
-    while (true) {
-      if (tableExists(curTableName)) {
-        tableNum++;
-        curTableName = DEFAULT_SESSION_TABLE_NAME + "_" + tableNum;
-      } else {
-        break;
-      }
-    }
-
-    // curTableName contains a table name that does not exist.
-    // Create this table.
-    LOG.debug("Creating job storage table: " + curTableName);
-    Statement s = connection.createStatement();
-    try {
-      s.executeUpdate("CREATE TABLE " + curTableName + " ("
-          + "job_name VARCHAR(64) NOT NULL, "
-          + "propname VARCHAR(128) NOT NULL, "
-          + "propval VARCHAR(1024), "
-          + "propclass VARCHAR(32) NOT NULL, "
-          + "CONSTRAINT " + curTableName + "_unq UNIQUE "
-          + "(job_name, propname, propclass))");
-
-      // Then set a property in the root table pointing to it.
-      setRootProperty(SESSION_TABLE_KEY, 0, curTableName);
-      connection.commit();
-    } finally {
-      s.close();
-    }
-
-    this.jobTableName = curTableName;
-  }
-
-  /**
-   * Given a root schema that exists,
-   * initialize a version-0 key/value storage schema on top of it,
-   * if it does not already exist.
-   */
-  private void initV0Schema() throws SQLException {
-    this.jobTableName = getRootProperty(SESSION_TABLE_KEY, 0);
-    if (null == this.jobTableName) {
-      createJobTable();
-    }
-    if (!tableExists(this.jobTableName)) {
-      LOG.debug("Could not find job table: " + jobTableName);
-      createJobTable();
-    }
-  }
-
-  /**
-   * INSERT or UPDATE a single (job, propname, class) to point
-   * to the specified property value.
-   */
-  private void setV0Property(String jobName, String propClass,
-      String propName, String propVal) throws SQLException {
-    LOG.debug("Job: " + jobName + "; Setting property "
-        + propName + " with class " + propClass + " => " + propVal);
-
-    PreparedStatement s = null;
-    try {
-      String curValue = getV0Property(jobName, propClass, propName);
-      if (null == curValue) {
-        // Property is not yet set.
-        s = connection.prepareStatement("INSERT INTO " + this.jobTableName
-            + " (propval, job_name, propclass, propname) "
-            + "VALUES (?, ?, ?, ?)");
-      } else {
-        // Overwrite existing property.
-        s = connection.prepareStatement("UPDATE " + this.jobTableName
-            + " SET propval = ? WHERE job_name = ? AND propclass = ? "
-            + "AND propname = ?");
-      }
-
-      s.setString(1, propVal);
-      s.setString(2, jobName);
-      s.setString(3, propClass);
-      s.setString(4, propName);
-
-      s.executeUpdate();
-    } finally {
-      if (null != s) {
-        s.close();
-      }
-    }
-  }
-
-  /**
-   * Return a string containing the value of a specified property,
-   * or null if it is not set.
-   */
-  private String getV0Property(String jobName, String propClass,
-      String propertyName) throws SQLException {
-    LOG.debug("Job: " + jobName + "; Getting property "
-        + propertyName + " with class " + propClass);
-
-    ResultSet rs = null;
-    PreparedStatement s = connection.prepareStatement(
-        "SELECT propval FROM " + this.jobTableName
-        + " WHERE job_name = ? AND propclass = ? AND propname = ?");
-
-    try {
-      s.setString(1, jobName);
-      s.setString(2, propClass);
-      s.setString(3, propertyName);
-      rs = s.executeQuery();
-
-      if (!rs.next()) {
-        LOG.debug(" => (no result)");
-        return null;
-      }
-
-      String result = rs.getString(1);
-      LOG.debug(" => " + result);
-      return result;
-    } finally {
-      if (null != rs) {
-        try {
-          rs.close();
-        } catch (SQLException sqlE) {
-          LOG.warn("Error closing resultset: " + sqlE);
-        }
-      }
-
-      s.close();
-    }
-  }
-
-  /**
-   * Get a java.util.Properties containing all propName -&gt; propVal
-   * bindings for a given (jobName, propClass).
-   */
-  private Properties getV0Properties(String jobName, String propClass)
-      throws SQLException {
-    LOG.debug("Job: " + jobName
-        + "; Getting properties with class " + propClass);
-
-    ResultSet rs = null;
-    PreparedStatement s = connection.prepareStatement(
-        "SELECT propname, propval FROM " + this.jobTableName
-        + " WHERE job_name = ? AND propclass = ?");
-    try {
-      s.setString(1, jobName);
-      s.setString(2, propClass);
-      rs = s.executeQuery();
-
-      Properties p = new Properties();
-      while (rs.next()) {
-        p.setProperty(rs.getString(1), rs.getString(2));
-      }
-
-      return p;
-    } finally {
-      if (null != rs) {
-        try {
-          rs.close();
-        } catch (SQLException sqlE) {
-          LOG.warn("Error closing result set: " + sqlE);
-        }
-      }
-
-      s.close();
-    }
-  }
-
-  private void setV0Properties(String jobName, String propClass,
-      Properties properties) throws SQLException {
-    LOG.debug("Job: " + jobName
-        + "; Setting bulk properties for class " + propClass);
-
-    for (Map.Entry<Object, Object> entry : properties.entrySet()) {
-      String key = entry.getKey().toString();
-      String val = entry.getValue().toString();
-      setV0Property(jobName, propClass, key, val);
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d0770ac6/src/java/org/apache/sqoop/tool/BaseSqoopTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/tool/BaseSqoopTool.java b/src/java/org/apache/sqoop/tool/BaseSqoopTool.java
index 1564bdc..6a4dcb0 100644
--- a/src/java/org/apache/sqoop/tool/BaseSqoopTool.java
+++ b/src/java/org/apache/sqoop/tool/BaseSqoopTool.java
@@ -216,6 +216,8 @@ public abstract class BaseSqoopTool extends com.cloudera.sqoop.tool.SqoopTool {
 
   // Arguments for the saved job management system.
   public static final String STORAGE_METASTORE_ARG = "meta-connect";
+  public static final String METASTORE_USER_ARG = "meta-username";
+  public static final String METASTORE_PASS_ARG = "meta-password";
   public static final String JOB_CMD_CREATE_ARG = "create";
   public static final String JOB_CMD_DELETE_ARG = "delete";
   public static final String JOB_CMD_EXEC_ARG = "exec";
@@ -379,6 +381,16 @@ public abstract class BaseSqoopTool extends com.cloudera.sqoop.tool.SqoopTool {
         .withDescription("Specify JDBC connect string for the metastore")
         .withLongOpt(STORAGE_METASTORE_ARG)
         .create());
+    relatedOpts.addOption(OptionBuilder.withArgName("metastore-db-username")
+        .hasArg()
+        .withDescription("Specify the username string for the metastore")
+        .withLongOpt(METASTORE_USER_ARG)
+        .create());
+    relatedOpts.addOption(OptionBuilder.withArgName("metastore-db-password")
+        .hasArg()
+        .withDescription("Specify the password string for the metastore")
+        .withLongOpt(METASTORE_PASS_ARG)
+        .create());
 
     // Create an option-group surrounding the operations a user
     // can perform on jobs.

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d0770ac6/src/java/org/apache/sqoop/tool/JobTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/tool/JobTool.java b/src/java/org/apache/sqoop/tool/JobTool.java
index 054e274..dbe8934 100644
--- a/src/java/org/apache/sqoop/tool/JobTool.java
+++ b/src/java/org/apache/sqoop/tool/JobTool.java
@@ -18,12 +18,21 @@
 
 package org.apache.sqoop.tool;
 
+import static org.apache.sqoop.manager.JdbcDrivers.DB2;
+import static org.apache.sqoop.manager.JdbcDrivers.HSQLDB;
+import static org.apache.sqoop.manager.JdbcDrivers.MYSQL;
+import static org.apache.sqoop.manager.JdbcDrivers.ORACLE;
+import static org.apache.sqoop.manager.JdbcDrivers.POSTGRES;
+import static org.apache.sqoop.manager.JdbcDrivers.SQLSERVER;
+
 import java.io.IOException;
 
 import java.util.Arrays;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.Set;
 import java.util.TreeMap;
 
 import org.apache.commons.cli.CommandLine;
@@ -38,10 +47,11 @@ import org.apache.hadoop.util.ToolRunner;
 import com.cloudera.sqoop.SqoopOptions;
 import com.cloudera.sqoop.SqoopOptions.InvalidOptionsException;
 import com.cloudera.sqoop.cli.ToolOptions;
-import com.cloudera.sqoop.metastore.hsqldb.HsqldbJobStorage;
+import com.cloudera.sqoop.metastore.GenericJobStorage;
 import com.cloudera.sqoop.metastore.JobData;
 import com.cloudera.sqoop.metastore.JobStorage;
 import com.cloudera.sqoop.metastore.JobStorageFactory;
+import org.apache.sqoop.manager.JdbcDrivers;
 import org.apache.sqoop.util.LoggingUtils;
 
 /**
@@ -53,6 +63,8 @@ public class JobTool extends com.cloudera.sqoop.tool.BaseSqoopTool {
       JobTool.class.getName());
   private static final String DASH_STR  = "--";
 
+  private static Set<JdbcDrivers> SUPPORTED_DRIVERS = EnumSet.of(HSQLDB, MYSQL, ORACLE, POSTGRES, DB2, SQLSERVER);
+
   private enum JobOp {
     JobCreate,
     JobDelete,
@@ -345,11 +357,7 @@ public class JobTool extends com.cloudera.sqoop.tool.BaseSqoopTool {
 
     this.storageDescriptor = new TreeMap<String, String>();
 
-    if (in.hasOption(STORAGE_METASTORE_ARG)) {
-      this.storageDescriptor.put(HsqldbJobStorage.META_CONNECT_KEY,
-          in.getOptionValue(STORAGE_METASTORE_ARG));
-    }
-
+    applyMetastoreOptions(in, out);
     // These are generated via an option group; exactly one
     // of this exhaustive list will always be selected.
     if (in.hasOption(JOB_CMD_CREATE_ARG)) {
@@ -369,6 +377,44 @@ public class JobTool extends com.cloudera.sqoop.tool.BaseSqoopTool {
     }
   }
 
+  private void applyMetastoreOptions(CommandLine in, SqoopOptions out) throws InvalidOptionsException {
+    String metaConnectString;
+    String metaUsernameString;
+    String metaPasswordString;
+    if (in.hasOption(STORAGE_METASTORE_ARG)) {
+      metaConnectString = in.getOptionValue(STORAGE_METASTORE_ARG);
+      this.storageDescriptor.put(GenericJobStorage.META_DRIVER_KEY, chooseDriverType(metaConnectString));
+      this.storageDescriptor.put(GenericJobStorage.META_CONNECT_KEY, metaConnectString);
+    } else {
+      metaConnectString = out.getMetaConnectStr();
+      this.storageDescriptor.put(GenericJobStorage.META_DRIVER_KEY, chooseDriverType(metaConnectString));
+      this.storageDescriptor.put(GenericJobStorage.META_CONNECT_KEY, metaConnectString);
+    }
+    if (in.hasOption(METASTORE_USER_ARG)) {
+      metaUsernameString = in.getOptionValue(METASTORE_USER_ARG);
+      this.storageDescriptor.put(GenericJobStorage.META_USERNAME_KEY, metaUsernameString);
+    } else {
+      metaUsernameString = out.getMetaUsername();
+      this.storageDescriptor.put(GenericJobStorage.META_USERNAME_KEY, metaUsernameString);
+    }
+    if (in.hasOption(METASTORE_PASS_ARG)) {
+      metaPasswordString = in.getOptionValue(METASTORE_PASS_ARG);
+      this.storageDescriptor.put(GenericJobStorage.META_PASSWORD_KEY, metaPasswordString);
+    } else {
+      metaPasswordString = out.getMetaPassword();
+      this.storageDescriptor.put(GenericJobStorage.META_PASSWORD_KEY, metaPasswordString);
+    }
+  }
+
+  private String chooseDriverType(String metaConnectString) throws InvalidOptionsException {
+    for (JdbcDrivers driver : SUPPORTED_DRIVERS) {
+      if (metaConnectString.startsWith(driver.getSchemePrefix())) {
+        return driver.getDriverClass();
+      }
+    }
+    throw new InvalidOptionsException("current meta-connect scheme not compatible with metastore");
+  }
+
   @Override
   /** {@inheritDoc} */
   public void validateOptions(SqoopOptions options)

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d0770ac6/src/test/com/cloudera/sqoop/TestIncrementalImport.java
----------------------------------------------------------------------
diff --git a/src/test/com/cloudera/sqoop/TestIncrementalImport.java b/src/test/com/cloudera/sqoop/TestIncrementalImport.java
index 52a55b7..166792b 100644
--- a/src/test/com/cloudera/sqoop/TestIncrementalImport.java
+++ b/src/test/com/cloudera/sqoop/TestIncrementalImport.java
@@ -31,6 +31,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import com.cloudera.sqoop.metastore.SavedJobsTestBase;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -43,11 +44,11 @@ import com.cloudera.sqoop.manager.ConnManager;
 import com.cloudera.sqoop.manager.HsqldbManager;
 import com.cloudera.sqoop.manager.ManagerFactory;
 import com.cloudera.sqoop.metastore.JobData;
-import com.cloudera.sqoop.metastore.TestSavedJobs;
 import com.cloudera.sqoop.testutil.BaseSqoopTestCase;
 import com.cloudera.sqoop.testutil.CommonArgs;
 import com.cloudera.sqoop.tool.ImportTool;
 import com.cloudera.sqoop.tool.JobTool;
+import org.apache.sqoop.metastore.GenericJobStorage;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -71,25 +72,37 @@ public class TestIncrementalImport  {
 
   // What database do we read from.
   public static final String SOURCE_DB_URL = "jdbc:hsqldb:mem:incremental";
+  public static final String AUTO_STORAGE_PASSWORD = "";
+  public static final String AUTO_STORAGE_USERNAME = "SA";
 
   @Rule
   public ExpectedException thrown = ExpectedException.none();
 
   @Before
   public void setUp() throws Exception {
-    // Delete db state between tests.
-    TestSavedJobs.resetJobSchema();
+    // Delete db state between tests
+    System.setProperty(GenericJobStorage.AUTO_STORAGE_USER_KEY, AUTO_STORAGE_USERNAME);
+    System.setProperty(GenericJobStorage.AUTO_STORAGE_PASS_KEY, AUTO_STORAGE_PASSWORD);
+    System.setProperty(GenericJobStorage.AUTO_STORAGE_CONNECT_STRING_KEY,
+            SOURCE_DB_URL);
     resetSourceDataSchema();
   }
 
   public static void resetSourceDataSchema() throws SQLException {
     SqoopOptions options = new SqoopOptions();
     options.setConnectString(SOURCE_DB_URL);
-    TestSavedJobs.resetSchema(options);
+    options.setUsername(AUTO_STORAGE_USERNAME);
+    options.setPassword(AUTO_STORAGE_PASSWORD);
+    SavedJobsTestBase.resetSchema(options);
   }
 
   public static Configuration newConf() {
-    return TestSavedJobs.newConf();
+    Configuration conf = new Configuration();
+    conf.set(GenericJobStorage.AUTO_STORAGE_USER_KEY, AUTO_STORAGE_USERNAME);
+    conf.set(GenericJobStorage.AUTO_STORAGE_PASS_KEY, AUTO_STORAGE_PASSWORD);
+    conf.set(GenericJobStorage.AUTO_STORAGE_CONNECT_STRING_KEY,
+            SOURCE_DB_URL);
+    return conf;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d0770ac6/src/test/com/cloudera/sqoop/metastore/JobToolTestBase.java
----------------------------------------------------------------------
diff --git a/src/test/com/cloudera/sqoop/metastore/JobToolTestBase.java b/src/test/com/cloudera/sqoop/metastore/JobToolTestBase.java
new file mode 100644
index 0000000..2f46ec9
--- /dev/null
+++ b/src/test/com/cloudera/sqoop/metastore/JobToolTestBase.java
@@ -0,0 +1,215 @@
+/**
+ * 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 com.cloudera.sqoop.metastore;
+
+import static org.junit.Assert.assertEquals;
+
+import com.cloudera.sqoop.SqoopOptions;
+import com.cloudera.sqoop.testutil.BaseSqoopTestCase;
+import com.cloudera.sqoop.testutil.CommonArgs;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.sqoop.manager.ConnManager;
+import org.apache.sqoop.manager.DefaultManagerFactory;
+import org.apache.sqoop.Sqoop;
+import org.apache.sqoop.tool.JobTool;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Base test class for JobTool, implemented for specific database services in sub-classes
+ */
+
+public abstract class JobToolTestBase extends BaseSqoopTestCase {
+
+    public static final Log LOG = LogFactory
+            .getLog(MetaConnectIncrementalImportTestBase.class.getName());
+
+    private String metaConnectString;
+    private String metaUser;
+    private String metaPass;
+    private ConnManager cm;
+
+
+    public JobToolTestBase(String metaConnectString, String metaUser, String metaPass) {
+        this.metaConnectString = metaConnectString;
+        this.metaUser = metaUser;
+        this.metaPass = metaPass;
+    }
+
+    @Before
+    public void setUp() {
+        super.setUp();
+
+        SqoopOptions options = getSqoopOptions();
+
+        Connection conn = getConnection(options);
+
+        try {
+            Statement statement = conn.createStatement();
+            statement.execute("DROP TABLE " + cm.escapeTableName("SQOOP_ROOT"));
+            statement.execute("DROP TABLE " + cm.escapeTableName("SQOOP_SESSIONS"));
+            conn.commit();
+        } catch (Exception e) {
+            LOG.error("Failed to clear metastore database");
+        }
+        //Methods from BaseSqoopTestClass reference the test Hsqldb database, not the metastore
+        try{
+            dropTableIfExists("CarLocations");
+        } catch (SQLException e) {
+            LOG.error("Failed to drop table CarLocations");
+        }
+        setCurTableName("CarLocations");
+        createTableWithColTypesAndNames(
+                new String [] {"carId", "Locations"},
+                new String [] {"INTEGER", "VARCHAR"},
+                new String [] {"1", "'Lexus'"});
+    }
+
+    private Connection getConnection(SqoopOptions options) {
+        try {
+            com.cloudera.sqoop.metastore.JobData jd = new com.cloudera.sqoop.metastore.JobData(options, null);
+            DefaultManagerFactory dmf = new DefaultManagerFactory();
+            cm = dmf.accept(jd);
+            return cm.getConnection();
+        } catch (SQLException e) {
+            LOG.error("Failed to create a connection to the Metastore");
+            return  null;
+        }
+    }
+
+    private SqoopOptions getSqoopOptions() {
+        SqoopOptions options = new SqoopOptions();
+        options.setConnectString(metaConnectString);
+        options.setUsername(metaUser);
+        options.setPassword(metaPass);
+        return options;
+    }
+
+    @After
+    public void tearDown() {
+        super.tearDown();
+
+        try {
+            cm.close();
+        } catch (SQLException e) {
+            LOG.error("Failed to close ConnManager");
+        }
+
+    }
+
+    protected String[] getCreateJob(String metaConnectString, String metaUser, String metaPass) {
+        List<String> args = new ArrayList<>();
+        CommonArgs.addHadoopFlags(args);
+        args.add("--create");
+        args.add("testJob");
+        args.add("--meta-connect");
+        args.add(metaConnectString);
+        args.add("--meta-username");
+        args.add(metaUser);
+        args.add("--meta-password");
+        args.add(metaPass);
+        args.add("--");
+        args.add("list-tables");
+        args.add("--connect");
+        args.add(getConnectString());
+
+        return args.toArray(new String[0]);
+    }
+
+    protected String[] getExecJob(String metaConnectString, String metaUser, String metaPass) {
+        List<String> args = new ArrayList<>();
+        CommonArgs.addHadoopFlags(args);
+        args.add("--exec");
+        args.add("testJob");
+        args.add("--meta-connect");
+        args.add(metaConnectString);
+        args.add("--meta-username");
+        args.add(metaUser);
+        args.add("--meta-password");
+        args.add(metaPass);
+
+        return args.toArray(new String[0]);
+    }
+
+
+    protected String[] getDeleteJob(String metaConnectString, String metaUser, String metaPass) {
+        List<String> args = new ArrayList<>();
+        CommonArgs.addHadoopFlags(args);
+        args.add("--delete");
+        args.add("testJob");
+        args.add("--meta-connect");
+        args.add(metaConnectString);
+        args.add("--meta-username");
+        args.add(metaUser);
+        args.add("--meta-password");
+        args.add(metaPass);
+
+        return args.toArray(new String[0]);
+    }
+
+    @Test
+    public void testCreateJob() throws IOException {
+        org.apache.sqoop.tool.JobTool jobTool = new org.apache.sqoop.tool.JobTool();
+        org.apache.sqoop.Sqoop sqoop = new Sqoop(jobTool);
+        String[] args = getCreateJob(metaConnectString, metaUser, metaPass);
+        assertEquals("Error creating Sqoop Job", 0, Sqoop.runSqoop(sqoop, args));
+    }
+
+    @Test
+    public void testExecJob() throws IOException {
+        Configuration conf = new Configuration();
+        //creates the job
+        JobTool jobToolCreate = new JobTool();
+        Sqoop sqoopCreate = new Sqoop(jobToolCreate, conf);
+        String[] argsCreate = getCreateJob(metaConnectString, metaUser, metaPass);
+        Sqoop.runSqoop(sqoopCreate, argsCreate);
+        //executes the job
+        JobTool jobToolExec = new JobTool();
+        Sqoop sqoopExec = new Sqoop(jobToolExec);
+        String[] argsExec = getExecJob(metaConnectString, metaUser, metaPass);
+        assertEquals("Error executing Sqoop Job", 0, Sqoop.runSqoop(sqoopExec, argsExec));
+    }
+
+    @Test
+    public void testDeleteJob() throws IOException {
+        Configuration conf = new Configuration();
+        //Creates the job
+        JobTool jobToolCreate = new JobTool();
+        Sqoop sqoopCreate = new Sqoop(jobToolCreate, conf);
+        String[] argsCreate = getCreateJob(metaConnectString, metaUser, metaPass);
+        Sqoop.runSqoop(sqoopCreate, argsCreate);
+        //Deletes the job
+        JobTool jobToolDelete = new JobTool();
+        Sqoop sqoopExec = new Sqoop(jobToolDelete);
+        String[] argsDelete = getDeleteJob(metaConnectString, metaUser, metaPass);
+        assertEquals("Error deleting Sqoop Job", 0, Sqoop.runSqoop(sqoopExec, argsDelete));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d0770ac6/src/test/com/cloudera/sqoop/metastore/MetaConnectIncrementalImportTestBase.java
----------------------------------------------------------------------
diff --git a/src/test/com/cloudera/sqoop/metastore/MetaConnectIncrementalImportTestBase.java b/src/test/com/cloudera/sqoop/metastore/MetaConnectIncrementalImportTestBase.java
new file mode 100644
index 0000000..587aaff
--- /dev/null
+++ b/src/test/com/cloudera/sqoop/metastore/MetaConnectIncrementalImportTestBase.java
@@ -0,0 +1,215 @@
+/**
+ * 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 com.cloudera.sqoop.metastore;
+
+import static org.junit.Assert.assertEquals;
+
+import com.cloudera.sqoop.SqoopOptions;
+import com.cloudera.sqoop.testutil.BaseSqoopTestCase;
+import com.cloudera.sqoop.testutil.CommonArgs;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.sqoop.manager.ConnManager;
+import org.apache.sqoop.manager.DefaultManagerFactory;
+import org.apache.sqoop.tool.JobTool;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+
+
+/**
+ * Base test class for Incremental Import Metastore data, implemented for specific database services in sub-classes
+ */
+
+public abstract class MetaConnectIncrementalImportTestBase extends BaseSqoopTestCase {
+
+    public static final Log LOG = LogFactory
+            .getLog(MetaConnectIncrementalImportTestBase.class.getName());
+
+    private String metaConnectString;
+    private String metaUser;
+    private String metaPass;
+
+    private Connection connMeta;
+    private ConnManager cm;
+
+    public MetaConnectIncrementalImportTestBase(String metaConnectString, String metaUser, String metaPass) {
+        this.metaConnectString = metaConnectString;
+        this.metaUser = metaUser;
+        this.metaPass = metaPass;
+    }
+
+    @Before
+    public void setUp() {
+        super.setUp();
+    }
+
+    @After
+    public void tearDown() {
+        super.tearDown();
+    }
+
+    protected String[] getIncrementalJob(String metaConnectString, String metaUser, String metaPass) {
+        List<String> args = new ArrayList<>();
+        CommonArgs.addHadoopFlags(args);
+        args.add("--create");
+        args.add("testJob");
+        args.add("--meta-connect");
+        args.add(metaConnectString);
+        args.add("--meta-username");
+        args.add(metaUser);
+        args.add("--meta-password");
+        args.add(metaPass);
+        args.add("--");
+        args.add("import");
+        args.add("-m");
+        args.add("1");
+        args.add("--connect");
+        args.add(getConnectString());
+        args.add("--table");
+        args.add("CARLOCATIONS");
+        args.add("--incremental");
+        args.add("append");
+        args.add("--check-column");
+        args.add("CARID");
+        args.add("--last-value");
+        args.add("0");
+        args.add("--as-textfile");
+
+        return args.toArray(new String[0]);
+    }
+
+
+    protected String[] getExecJob(String metaConnectString, String metaUser, String metaPass) {
+        List<String> args = new ArrayList<>();
+        CommonArgs.addHadoopFlags(args);
+        args.add("--exec");
+        args.add("testJob");
+        args.add("--meta-connect");
+        args.add(metaConnectString);
+        args.add("--meta-username");
+        args.add(metaUser);
+        args.add("--meta-password");
+        args.add(metaPass);
+
+        return args.toArray(new String[0]);
+    }
+
+    @Test
+    public void testIncrementalJob() throws SQLException {
+        resetTable();
+
+        initMetastoreConnection();
+
+        resetMetastoreSchema();
+
+        //creates Job
+        createJob();
+
+        //Executes the import
+        execJob();
+
+        //Ensures the saveIncrementalState saved the right row
+        checkIncrementalState(1);
+
+        //Adds rows to the import table
+        Statement insertStmt = getConnection().createStatement();
+        insertStmt.executeUpdate("INSERT INTO CARLOCATIONS VALUES (2, 'lexus')");
+        getConnection().commit();
+
+        //Execute the import again
+        execJob();
+
+        //Ensures the last incremental value is updated correctly.
+        checkIncrementalState(2);
+
+        cm.close();
+    }
+
+    private void checkIncrementalState(int expected) throws SQLException {
+        Statement getSaveIncrementalState = connMeta.createStatement();
+        ResultSet lastCol = getSaveIncrementalState.executeQuery(
+                "SELECT propVal FROM " + cm.escapeTableName("SQOOP_SESSIONS") + " WHERE propname = 'incremental.last.value'");
+        lastCol.next();
+        assertEquals("Last row value differs from expected",
+                expected, lastCol.getInt("propVal"));
+    }
+
+    private void execJob() {
+        JobTool jobToolExec = new JobTool();
+        org.apache.sqoop.Sqoop sqoopExec = new org.apache.sqoop.Sqoop(jobToolExec);
+        String[] argsExec = getExecJob(metaConnectString, metaUser, metaPass);
+        assertEquals("Sqoop Job did not execute properly",
+                0, org.apache.sqoop.Sqoop.runSqoop(sqoopExec, argsExec));
+    }
+
+    private void createJob() {
+        Configuration conf = new Configuration();
+        conf.set(org.apache.sqoop.SqoopOptions.METASTORE_PASSWORD_KEY, "true");
+        JobTool jobToolCreate = new JobTool();
+        org.apache.sqoop.Sqoop sqoopCreate = new org.apache.sqoop.Sqoop(jobToolCreate, conf);
+        String[] argsCreate = getIncrementalJob(metaConnectString, metaUser, metaPass);
+        org.apache.sqoop.Sqoop.runSqoop(sqoopCreate, argsCreate);
+    }
+
+    private void resetTable() throws SQLException {
+        //Resets the target table
+        dropTableIfExists("CARLOCATIONS");
+        setCurTableName("CARLOCATIONS");
+        createTableWithColTypesAndNames(
+                new String [] {"CARID", "LOCATIONS"},
+                new String [] {"INTEGER", "VARCHAR"},
+                new String [] {"1", "'Lexus'"});
+    }
+
+    private void resetMetastoreSchema() {
+        try {
+            //Resets the metastore schema
+            Statement metastoreStatement = connMeta.createStatement();
+            metastoreStatement.execute("DROP TABLE " + cm.escapeTableName("SQOOP_ROOT"));
+            metastoreStatement.execute("DROP TABLE " + cm.escapeTableName("SQOOP_SESSIONS"));
+            connMeta.commit();
+        }
+        catch (Exception e) {
+            LOG.error( e.getLocalizedMessage() );
+        }
+    }
+
+    private void initMetastoreConnection() throws SQLException{
+        SqoopOptions options = new SqoopOptions();
+        options.setConnectString(metaConnectString);
+        options.setUsername(metaUser);
+        options.setPassword(metaPass);
+        com.cloudera.sqoop.metastore.JobData jd =
+                new com.cloudera.sqoop.metastore.JobData(options, new JobTool());
+        DefaultManagerFactory dmf = new DefaultManagerFactory();
+        cm = dmf.accept(jd);
+        connMeta= cm.getConnection();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d0770ac6/src/test/com/cloudera/sqoop/metastore/SavedJobsTestBase.java
----------------------------------------------------------------------
diff --git a/src/test/com/cloudera/sqoop/metastore/SavedJobsTestBase.java b/src/test/com/cloudera/sqoop/metastore/SavedJobsTestBase.java
new file mode 100644
index 0000000..81789e7
--- /dev/null
+++ b/src/test/com/cloudera/sqoop/metastore/SavedJobsTestBase.java
@@ -0,0 +1,314 @@
+/**
+ * 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 com.cloudera.sqoop.metastore;
+
+import static org.apache.sqoop.metastore.GenericJobStorage.META_CONNECT_KEY;
+import static org.apache.sqoop.metastore.GenericJobStorage.META_DRIVER_KEY;
+import static org.apache.sqoop.metastore.GenericJobStorage.META_PASSWORD_KEY;
+import static org.apache.sqoop.metastore.GenericJobStorage.META_USERNAME_KEY;
+
+import static org.hamcrest.core.IsCollectionContaining.hasItems;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+import com.cloudera.sqoop.manager.ConnManager;
+import com.cloudera.sqoop.SqoopOptions;
+import com.cloudera.sqoop.tool.VersionTool;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.sqoop.manager.DefaultManagerFactory;
+import org.apache.sqoop.tool.ImportTool;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * Test the metastore and job-handling features,
+ * implemented for specific database services in sub-classes.
+ */
+public abstract class SavedJobsTestBase {
+
+  public static final String TEST_JOB = "testJob";
+  public static final String TEST_TABLE_NAME = "abcd";
+  public static final String TEST_TABLE_NAME_2 = "efgh";
+  public static final String TEST_JOB_2 = "testJob2";
+  public static final String TEST_JOB_3 = "testJob3";
+  public static final String TEST_TABLE_NAME_3 = "ijkl";
+  private String metaConnect;
+  private String metaUser;
+  private String metaPassword;
+  private String driverClass;
+  private JobStorage storage;
+
+  private Configuration conf;
+  private Map<String, String> descriptor;
+
+  public SavedJobsTestBase(String metaConnect, String metaUser, String metaPassword, String driverClass){
+    this.metaConnect = metaConnect;
+    this.metaUser = metaUser;
+    this.metaPassword = metaPassword;
+    this.driverClass = driverClass;
+    this.descriptor = new TreeMap<>();
+  }
+
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
+  @Before
+  public void setUp() throws Exception {
+    // Delete db state between tests.
+    resetJobSchema();
+    conf = newConf();
+
+    descriptor.put(META_CONNECT_KEY, metaConnect);
+    descriptor.put(META_USERNAME_KEY, metaUser);
+    descriptor.put(META_PASSWORD_KEY, metaPassword);
+    descriptor.put(META_DRIVER_KEY, driverClass);
+
+    JobStorageFactory ssf = new JobStorageFactory(conf);
+    storage = ssf.getJobStorage(descriptor);
+    storage.open(descriptor);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    descriptor.clear();
+    storage.close();
+  }
+
+  public void resetJobSchema()
+          throws SQLException {
+    SqoopOptions options = new SqoopOptions();
+    options.setConnectString(metaConnect);
+    options.setUsername(metaUser);
+    options.setPassword(metaPassword);
+    options.setDriverClassName(driverClass);
+
+    resetSchema(options);
+  }
+
+  /**
+   * Drop all tables in the configured HSQLDB-based schema/user/pass.
+   */
+  public static void resetSchema(SqoopOptions options) throws SQLException {
+    JobData jd = new JobData();
+    jd.setSqoopOptions(options);
+    DefaultManagerFactory dmf = new DefaultManagerFactory();
+    ConnManager manager = dmf.accept(jd);
+    Connection c = manager.getConnection();
+    Statement s = c.createStatement();
+    try {
+      String [] tables = manager.listTables();
+      for (String table : tables) {
+        if(table.equals("SQOOP_ROOT") || table.equals("SQOOP_SESSIONS")){
+          s.execute("DROP TABLE " + manager.escapeTableName(table));
+        }
+      }
+
+      c.commit();
+    } finally {
+      s.close();
+    }
+  }
+
+  public Configuration newConf() {
+    Configuration conf = new Configuration();
+    conf.set(META_CONNECT_KEY, metaConnect);
+    conf.set(META_USERNAME_KEY, metaUser);
+    conf.set(META_PASSWORD_KEY, metaPassword);
+    conf.set(META_DRIVER_KEY, driverClass);
+
+    return conf;
+  }
+
+  @Test
+  public void testReadJobDoesExistPasses() throws Exception{
+    storage.create(TEST_JOB, createTestJobData(TEST_TABLE_NAME));
+
+    assertEquals("Read did not return job data correctly",
+            storage.read(TEST_JOB).getSqoopOptions().getTableName(),
+            TEST_TABLE_NAME);
+  }
+
+  @Test
+  public void testUpdateJob() throws  Exception {
+    storage.create(TEST_JOB, createTestJobData(TEST_TABLE_NAME));
+
+    storage.update(TEST_JOB, createTestJobData(TEST_TABLE_NAME_2) );
+
+    assertEquals("Update did not change data correctly",
+            storage.read(TEST_JOB).getSqoopOptions().getTableName(),
+            TEST_TABLE_NAME_2);
+  }
+
+  @Test
+  public void testList() throws IOException {
+    storage.create(TEST_JOB, createTestJobData(TEST_TABLE_NAME));
+    storage.create(TEST_JOB_2, createTestJobData(TEST_TABLE_NAME_2));
+    storage.create(TEST_JOB_3, createTestJobData(TEST_TABLE_NAME_3));
+
+    assertThat("List did not return correct job data",
+            storage.list(), hasItems(TEST_JOB, TEST_JOB_2, TEST_JOB_3));
+  }
+
+  @Test
+  public void testCreateSameJob() throws IOException {
+
+    // Job list should start out empty.
+    List<String> jobs = storage.list();
+    assertEquals("Job list should start out empty", 0, jobs.size());
+
+    // Create a job that displays the version.
+    JobData data = new JobData(new SqoopOptions(), new VersionTool());
+    storage.create(TEST_JOB, data);
+
+    jobs = storage.list();
+    assertEquals("Test Job not created correctly",1, jobs.size());
+    assertEquals("Test Job data not returned correctly", TEST_JOB, jobs.get(0));
+
+    try {
+      // Try to create that same job name again. This should fail.
+      thrown.expect(IOException.class);
+      thrown.reportMissingExceptionWithMessage("Expected IOException since job already exists");
+      storage.create(TEST_JOB, data);
+    } finally {
+      jobs = storage.list();
+      assertEquals("Incorrect number of jobs present",1, jobs.size());
+
+      // Restore our job, check that it exists.
+      JobData outData = storage.read(TEST_JOB);
+      assertEquals("Test job does not exist", new VersionTool().getToolName(),
+          outData.getSqoopTool().getToolName());
+    }
+  }
+
+  @Test
+  public void testDeleteJob() throws IOException {
+    // Job list should start out empty.
+    List<String> jobs = storage.list();
+    assertEquals("Job List should start out empty", 0, jobs.size());
+
+    // Create a job that displays the version.
+    JobData data = new JobData(new SqoopOptions(), new VersionTool());
+    storage.create(TEST_JOB, data);
+
+    jobs = storage.list();
+    assertEquals("Incorrect number of jobs present",1, jobs.size());
+    assertEquals("Test Job created incorrectly", TEST_JOB, jobs.get(0));
+
+    // Now delete the job.
+    storage.delete(TEST_JOB);
+
+    // After delete, we should have no jobs.
+    jobs = storage.list();
+    assertEquals("Job was not deleted correctly", 0, jobs.size());
+  }
+
+  @Test
+  public void testRestoreNonExistingJob() throws IOException {
+      // Try to restore a job that doesn't exist. Watch it fail.
+      thrown.expect(IOException.class);
+      thrown.reportMissingExceptionWithMessage("Expected IOException since job doesn't exist");
+      storage.read("DoesNotExist");
+  }
+
+  @Test
+  public void testCreateJobWithExtraArgs() throws IOException {
+
+        // Job list should start out empty.
+        List<String> jobs = storage.list();
+        assertEquals("Job list should start out empty", 0, jobs.size());
+
+        // Create a job with extra args
+        com.cloudera.sqoop.SqoopOptions opts = new SqoopOptions();
+        String[] args = {"-schema", "test"};
+        opts.setExtraArgs(args);
+        JobData data = new JobData(opts, new VersionTool());
+        storage.create(TEST_JOB, data);
+
+        jobs = storage.list();
+        assertEquals("Incorrect number of jobs", 1, jobs.size());
+        assertEquals("Job not created properly", TEST_JOB, jobs.get(0));
+
+        // Restore our job, check that it exists.
+        JobData outData = storage.read(TEST_JOB);
+        assertEquals("Incorrect Tool in Test Job",
+                new VersionTool().getToolName(),
+                outData.getSqoopTool().getToolName());
+
+        String[] storedArgs = outData.getSqoopOptions().getExtraArgs();
+        for(int index = 0; index < args.length; ++index) {
+            assertEquals(args[index], storedArgs[index]);
+        }
+
+        // Now delete the job.
+        storage.delete(TEST_JOB);
+    }
+
+  @Test
+  public void testMultiConnections() throws IOException {
+
+    // Job list should start out empty.
+    List<String> jobs = storage.list();
+    assertEquals("Job list should start out empty", 0, jobs.size());
+
+    // Create a job that displays the version.
+    JobData data = new JobData(new SqoopOptions(), new VersionTool());
+    storage.create(TEST_JOB, data);
+
+    jobs = storage.list();
+    assertEquals("Incorrect number of jobs", 1, jobs.size());
+    assertEquals("Job not created correctly", TEST_JOB, jobs.get(0));
+
+    storage.close(); // Close the existing connection
+
+    // Now re-open the storage.
+    storage.open(descriptor);
+
+    jobs = storage.list();
+    assertEquals("Test Job did not persist through re-open", 1, jobs.size());
+    assertEquals("Job data not correct after re-open", TEST_JOB, jobs.get(0));
+
+    // Restore our job, check that it exists.
+    JobData outData = storage.read(TEST_JOB);
+    assertEquals("Incorrect Tool in Test Job",
+            new VersionTool().getToolName(),
+            outData.getSqoopTool().getToolName());
+  }
+
+  private com.cloudera.sqoop.metastore.JobData createTestJobData(String setTableName) throws IOException {
+    SqoopOptions testOpts = new SqoopOptions();
+    testOpts.setTableName(setTableName);
+    ImportTool testTool = new ImportTool();
+    return new com.cloudera.sqoop.metastore.JobData(testOpts,testTool);
+
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d0770ac6/src/test/com/cloudera/sqoop/metastore/TestSavedJobs.java
----------------------------------------------------------------------
diff --git a/src/test/com/cloudera/sqoop/metastore/TestSavedJobs.java b/src/test/com/cloudera/sqoop/metastore/TestSavedJobs.java
deleted file mode 100644
index 61d8c97..0000000
--- a/src/test/com/cloudera/sqoop/metastore/TestSavedJobs.java
+++ /dev/null
@@ -1,302 +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 com.cloudera.sqoop.metastore;
-
-import java.sql.SQLException;
-import java.sql.Statement;
-
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-
-import org.apache.hadoop.conf.Configuration;
-
-import com.cloudera.sqoop.SqoopOptions;
-import com.cloudera.sqoop.manager.HsqldbManager;
-import com.cloudera.sqoop.metastore.hsqldb.AutoHsqldbStorage;
-import com.cloudera.sqoop.tool.VersionTool;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
-import java.io.IOException;
-import java.sql.Connection;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Test the metastore and job-handling features.
- *
- * These all make use of the auto-connect hsqldb-based metastore.
- * The metastore URL is configured to be in-memory, and drop all
- * state between individual tests.
- */
-public class TestSavedJobs {
-
-  public static final String TEST_AUTOCONNECT_URL =
-      "jdbc:hsqldb:mem:sqoopmetastore";
-  public static final String TEST_AUTOCONNECT_USER = "SA";
-  public static final String TEST_AUTOCONNECT_PASS = "";
-
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
-
-  @Before
-  public void setUp() throws Exception {
-    // Delete db state between tests.
-    resetJobSchema();
-  }
-
-  public static void resetJobSchema() throws SQLException {
-    SqoopOptions options = new SqoopOptions();
-    options.setConnectString(TEST_AUTOCONNECT_URL);
-    options.setUsername(TEST_AUTOCONNECT_USER);
-    options.setPassword(TEST_AUTOCONNECT_PASS);
-
-    resetSchema(options);
-  }
-
-  /**
-   * Drop all tables in the configured HSQLDB-based schema/user/pass.
-   */
-  public static void resetSchema(SqoopOptions options) throws SQLException {
-    HsqldbManager manager = new HsqldbManager(options);
-    Connection c = manager.getConnection();
-    Statement s = c.createStatement();
-    try {
-      String [] tables = manager.listTables();
-      for (String table : tables) {
-        s.executeUpdate("DROP TABLE " + manager.escapeTableName(table));
-      }
-
-      c.commit();
-    } finally {
-      s.close();
-    }
-  }
-
-  public static Configuration newConf() {
-    Configuration conf = new Configuration();
-    conf.set(AutoHsqldbStorage.AUTO_STORAGE_USER_KEY, TEST_AUTOCONNECT_USER);
-    conf.set(AutoHsqldbStorage.AUTO_STORAGE_PASS_KEY, TEST_AUTOCONNECT_PASS);
-    conf.set(AutoHsqldbStorage.AUTO_STORAGE_CONNECT_STRING_KEY,
-        TEST_AUTOCONNECT_URL);
-
-    return conf;
-  }
-
-  @Test
-  public void testAutoConnect() throws IOException {
-    // By default, we should be able to auto-connect with an
-    // empty connection descriptor. We should see an empty
-    // job set.
-
-    Configuration conf = newConf();
-    JobStorageFactory ssf = new JobStorageFactory(conf);
-
-    Map<String, String> descriptor = new TreeMap<String, String>();
-    JobStorage storage = ssf.getJobStorage(descriptor);
-
-    storage.open(descriptor);
-    List<String> jobs = storage.list();
-    assertEquals(0, jobs.size());
-    storage.close();
-  }
-
-  @Test
-  public void testCreateSameJob() throws IOException {
-    Configuration conf = newConf();
-    JobStorageFactory ssf = new JobStorageFactory(conf);
-
-    Map<String, String> descriptor = new TreeMap<String, String>();
-    JobStorage storage = ssf.getJobStorage(descriptor);
-
-    storage.open(descriptor);
-
-    // Job list should start out empty.
-    List<String> jobs = storage.list();
-    assertEquals(0, jobs.size());
-
-    // Create a job that displays the version.
-    JobData data = new JobData(new SqoopOptions(), new VersionTool());
-    storage.create("versionJob", data);
-
-    jobs = storage.list();
-    assertEquals(1, jobs.size());
-    assertEquals("versionJob", jobs.get(0));
-
-    try {
-      // Try to create that same job name again. This should fail.
-      thrown.expect(IOException.class);
-      thrown.reportMissingExceptionWithMessage("Expected IOException since job already exists");
-      storage.create("versionJob", data);
-    } finally {
-      jobs = storage.list();
-      assertEquals(1, jobs.size());
-
-      // Restore our job, check that it exists.
-      JobData outData = storage.read("versionJob");
-      assertEquals(new VersionTool().getToolName(),
-          outData.getSqoopTool().getToolName());
-
-      storage.close();
-    }
-  }
-
-  @Test
-  public void testDeleteJob() throws IOException {
-    Configuration conf = newConf();
-    JobStorageFactory ssf = new JobStorageFactory(conf);
-
-    Map<String, String> descriptor = new TreeMap<String, String>();
-    JobStorage storage = ssf.getJobStorage(descriptor);
-
-    storage.open(descriptor);
-
-    // Job list should start out empty.
-    List<String> jobs = storage.list();
-    assertEquals(0, jobs.size());
-
-    // Create a job that displays the version.
-    JobData data = new JobData(new SqoopOptions(), new VersionTool());
-    storage.create("versionJob", data);
-
-    jobs = storage.list();
-    assertEquals(1, jobs.size());
-    assertEquals("versionJob", jobs.get(0));
-
-    // Now delete the job.
-    storage.delete("versionJob");
-
-    // After delete, we should have no jobs.
-    jobs = storage.list();
-    assertEquals(0, jobs.size());
-
-    storage.close();
-  }
-
-  @Test
-  public void testRestoreNonExistingJob() throws IOException {
-    Configuration conf = newConf();
-    JobStorageFactory ssf = new JobStorageFactory(conf);
-
-    Map<String, String> descriptor = new TreeMap<String, String>();
-    JobStorage storage = ssf.getJobStorage(descriptor);
-
-    storage.open(descriptor);
-
-    try {
-      // Try to restore a job that doesn't exist. Watch it fail.
-      thrown.expect(IOException.class);
-      thrown.reportMissingExceptionWithMessage("Expected IOException since job doesn't exist");
-      storage.read("DoesNotExist");
-    } finally {
-      storage.close();
-    }
-  }
-
-  @Test
-    public void testCreateJobWithExtraArgs() throws IOException {
-        Configuration conf = newConf();
-        JobStorageFactory ssf = new JobStorageFactory(conf);
-
-        Map<String, String> descriptor = new TreeMap<String, String>();
-        JobStorage storage = ssf.getJobStorage(descriptor);
-
-        storage.open(descriptor);
-
-        // Job list should start out empty.
-        List<String> jobs = storage.list();
-        assertEquals(0, jobs.size());
-
-        // Create a job with extra args
-        com.cloudera.sqoop.SqoopOptions opts = new SqoopOptions();
-        String[] args = {"-schema", "test"};
-        opts.setExtraArgs(args);
-        JobData data = new JobData(opts, new VersionTool());
-        storage.create("versionJob", data);
-
-        jobs = storage.list();
-        assertEquals(1, jobs.size());
-        assertEquals("versionJob", jobs.get(0));
-
-        // Restore our job, check that it exists.
-        JobData outData = storage.read("versionJob");
-        assertEquals(new VersionTool().getToolName(),
-                outData.getSqoopTool().getToolName());
-
-        String[] storedArgs = outData.getSqoopOptions().getExtraArgs();
-        for(int index = 0; index < args.length; ++index) {
-            assertEquals(args[index], storedArgs[index]);
-        }
-
-        // Now delete the job.
-        storage.delete("versionJob");
-
-        storage.close();
-    }
-
-  @Test
-  public void testMultiConnections() throws IOException {
-    // Ensure that a job can be retrieved when the storage is
-    // closed and reopened.
-
-    Configuration conf = newConf();
-    JobStorageFactory ssf = new JobStorageFactory(conf);
-
-    Map<String, String> descriptor = new TreeMap<String, String>();
-    JobStorage storage = ssf.getJobStorage(descriptor);
-
-    storage.open(descriptor);
-
-    // Job list should start out empty.
-    List<String> jobs = storage.list();
-    assertEquals(0, jobs.size());
-
-    // Create a job that displays the version.
-    JobData data = new JobData(new SqoopOptions(), new VersionTool());
-    storage.create("versionJob", data);
-
-    jobs = storage.list();
-    assertEquals(1, jobs.size());
-    assertEquals("versionJob", jobs.get(0));
-
-    storage.close(); // Close the existing connection
-
-    // Now re-open the storage.
-    ssf = new JobStorageFactory(newConf());
-    storage = ssf.getJobStorage(descriptor);
-    storage.open(descriptor);
-
-    jobs = storage.list();
-    assertEquals(1, jobs.size());
-    assertEquals("versionJob", jobs.get(0));
-
-    // Restore our job, check that it exists.
-    JobData outData = storage.read("versionJob");
-    assertEquals(new VersionTool().getToolName(),
-        outData.getSqoopTool().getToolName());
-
-    storage.close();
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d0770ac6/src/test/com/cloudera/sqoop/metastore/db2/DB2JobToolTest.java
----------------------------------------------------------------------
diff --git a/src/test/com/cloudera/sqoop/metastore/db2/DB2JobToolTest.java b/src/test/com/cloudera/sqoop/metastore/db2/DB2JobToolTest.java
new file mode 100644
index 0000000..b92d36a
--- /dev/null
+++ b/src/test/com/cloudera/sqoop/metastore/db2/DB2JobToolTest.java
@@ -0,0 +1,65 @@
+/**
+ * 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 com.cloudera.sqoop.metastore.db2;
+
+import com.cloudera.sqoop.metastore.JobToolTestBase;
+
+/**
+ * Test that the Job Tool works in DB2
+ *
+ * This uses JDBC to store and retrieve metastore data from a DB2 server
+ *
+ * Since this requires a DB2 installation,
+ * this class is named in such a way that Sqoop's default QA process does
+ * not run it. You need to run this manually with
+ * -Dtestcase=DB2JobToolTest or -Dthirdparty=true.
+ *
+ * You need to put DB2 JDBC driver library (db2jcc4.jar) in a location
+ * where Sqoop will be able to access it (since this library cannot be checked
+ * into Apache's tree for licensing reasons) and set it's path through -Dsqoop.thirdparty.lib.dir.
+ *
+ *   Once you have a running DB2 database,
+ *   Set server URL, database name, username, and password with system variables
+ *   -Dsqoop.test.db2.connectstring.host_url, -Dsqoop.test.db2.connectstring.database,
+ *   -Dsqoop.test.db2.connectstring.username and -Dsqoop.test.db2.connectstring.password respectively
+ */
+
+public class DB2JobToolTest extends JobToolTestBase {
+
+    private static final String HOST_URL = System.getProperty(
+        "sqoop.test.db2.connectstring.host_url",
+        "jdbc:db2://db2host:50000");
+
+    private static final String DATABASE_NAME = System.getProperty(
+        "sqoop.test.db2.connectstring.database",
+        "SQOOP");
+    private static final String DATABASE_USER = System.getProperty(
+        "sqoop.test.db2.connectstring.username",
+        "SQOOP");
+    private static final String DATABASE_PASSWORD = System.getProperty(
+        "sqoop.test.db2.connectstring.password",
+        "SQOOP");
+    private static final String CONNECT_STRING = HOST_URL
+        + "/" + DATABASE_NAME
+        + ":currentSchema=" + DATABASE_USER +";";
+
+    public DB2JobToolTest() {
+        super(CONNECT_STRING, DATABASE_USER, DATABASE_PASSWORD);
+    }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d0770ac6/src/test/com/cloudera/sqoop/metastore/db2/DB2MetaConnectIncrementalImportTest.java
----------------------------------------------------------------------
diff --git a/src/test/com/cloudera/sqoop/metastore/db2/DB2MetaConnectIncrementalImportTest.java b/src/test/com/cloudera/sqoop/metastore/db2/DB2MetaConnectIncrementalImportTest.java
new file mode 100644
index 0000000..c1ae70c
--- /dev/null
+++ b/src/test/com/cloudera/sqoop/metastore/db2/DB2MetaConnectIncrementalImportTest.java
@@ -0,0 +1,65 @@
+/**
+ * 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 com.cloudera.sqoop.metastore.db2;
+
+import com.cloudera.sqoop.metastore.MetaConnectIncrementalImportTestBase;
+
+/**
+ * Test that Incremental-Import values are stored correctly in DB2
+ *
+ * This uses JDBC to store and retrieve metastore data from a DB2 server
+ *
+ * Since this requires a DB2 installation,
+ * this class is named in such a way that Sqoop's default QA process does
+ * not run it. You need to run this manually with
+ * -Dtestcase=DB2MetaConnectIncrementalImportTest or -Dthirdparty=true.
+ *
+ * You need to put DB2 JDBC driver library (db2jcc4.jar) in a location
+ * where Sqoop will be able to access it (since this library cannot be checked
+ * into Apache's tree for licensing reasons) and set it's path through -Dsqoop.thirdparty.lib.dir.
+ *
+ *   Once you have a running DB2 database,
+ *   Set server URL, database name, username, and password with system variables
+ *   -Dsqoop.test.db2.connectstring.host_url, -Dsqoop.test.db2.connectstring.database,
+ *   -Dsqoop.test.db2.connectstring.username and -Dsqoop.test.db2.connectstring.password respectively
+ */
+
+public class DB2MetaConnectIncrementalImportTest extends MetaConnectIncrementalImportTestBase {
+
+    private static final String HOST_URL = System.getProperty(
+        "sqoop.test.db2.connectstring.host_url",
+        "jdbc:db2://db2host:50000");
+
+    private static final String DATABASE_NAME = System.getProperty(
+        "sqoop.test.db2.connectstring.database",
+        "SQOOP");
+    private static final String DATABASE_USER = System.getProperty(
+        "sqoop.test.db2.connectstring.username",
+        "SQOOP");
+    private static final String DATABASE_PASSWORD = System.getProperty(
+        "sqoop.test.db2.connectstring.password",
+        "SQOOP");
+    private static final String CONNECT_STRING = HOST_URL
+        + "/" + DATABASE_NAME
+        + ":currentSchema=" + DATABASE_USER +";";
+
+    public DB2MetaConnectIncrementalImportTest() {
+        super(CONNECT_STRING, DATABASE_USER, DATABASE_PASSWORD);
+    }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d0770ac6/src/test/com/cloudera/sqoop/metastore/db2/DB2SavedJobsTest.java
----------------------------------------------------------------------
diff --git a/src/test/com/cloudera/sqoop/metastore/db2/DB2SavedJobsTest.java b/src/test/com/cloudera/sqoop/metastore/db2/DB2SavedJobsTest.java
new file mode 100644
index 0000000..efeef62
--- /dev/null
+++ b/src/test/com/cloudera/sqoop/metastore/db2/DB2SavedJobsTest.java
@@ -0,0 +1,66 @@
+/**
+ * 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 com.cloudera.sqoop.metastore.db2;
+
+import com.cloudera.sqoop.metastore.SavedJobsTestBase;
+import org.apache.sqoop.manager.JdbcDrivers;
+
+/**
+ * Test of GenericJobStorage compatibility with DB2
+ *
+ * This uses JDBC to store and retrieve metastore data from a DB2 server
+ *
+ * Since this requires a DB2 installation,
+ * this class is named in such a way that Sqoop's default QA process does
+ * not run it. You need to run this manually with
+ * -Dtestcase=DB2SavedJobsTest or -Dthirdparty=true.
+ *
+ * You need to put DB2 JDBC driver library (db2jcc4.jar) in a location
+ * where Sqoop will be able to access it (since this library cannot be checked
+ * into Apache's tree for licensing reasons) and set it's path through -Dsqoop.thirdparty.lib.dir.
+ *
+ *   Once you have a running DB2 database,
+ *   Set server URL, database name, username, and password with system variables
+ *   -Dsqoop.test.db2.connectstring.host_url, -Dsqoop.test.db2.connectstring.database,
+ *   -Dsqoop.test.db2.connectstring.username and -Dsqoop.test.db2.connectstring.password respectively
+ */
+
+public class DB2SavedJobsTest extends SavedJobsTestBase {
+
+    private static final String HOST_URL = System.getProperty(
+        "sqoop.test.db2.connectstring.host_url",
+        "jdbc:db2://db2host:50000");
+
+    private static final String DATABASE_NAME = System.getProperty(
+        "sqoop.test.db2.connectstring.database",
+        "SQOOP");
+    private static final String DATABASE_USER = System.getProperty(
+        "sqoop.test.db2.connectstring.username",
+        "SQOOP");
+    private static final String DATABASE_PASSWORD = System.getProperty(
+        "sqoop.test.db2.connectstring.password",
+        "SQOOP");
+    private static final String CONNECT_STRING = HOST_URL
+        + "/" + DATABASE_NAME
+        + ":currentSchema=" + DATABASE_USER +";";
+
+    public DB2SavedJobsTest() {
+        super(CONNECT_STRING, DATABASE_USER, DATABASE_PASSWORD, JdbcDrivers.DB2.getDriverClass());
+    }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d0770ac6/src/test/com/cloudera/sqoop/metastore/hsqldb/HsqldbJobToolTest.java
----------------------------------------------------------------------
diff --git a/src/test/com/cloudera/sqoop/metastore/hsqldb/HsqldbJobToolTest.java b/src/test/com/cloudera/sqoop/metastore/hsqldb/HsqldbJobToolTest.java
new file mode 100644
index 0000000..07eefee
--- /dev/null
+++ b/src/test/com/cloudera/sqoop/metastore/hsqldb/HsqldbJobToolTest.java
@@ -0,0 +1,38 @@
+/**
+ * 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 com.cloudera.sqoop.metastore.hsqldb;
+
+import com.cloudera.sqoop.metastore.JobToolTestBase;
+
+/**
+ * Test that the Job Tool works in Hsqldb
+ *
+ * This class is named in such a way that Sqoop's default QA process does
+ * not run it. You need to run this manually with
+ * -Dtestcase=HsqldbJobToolTest or -Dthirdparty=true.
+ *
+ * This uses JDBC to store and retrieve metastore data from a local Hsqldb server
+ */
+
+public class HsqldbJobToolTest extends JobToolTestBase {
+
+    public HsqldbJobToolTest() {
+        super( "jdbc:hsqldb:mem:sqoopmetastore", "SA" , "");
+    }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d0770ac6/src/test/com/cloudera/sqoop/metastore/hsqldb/HsqldbMetaConnectIncrementalImportTest.java
----------------------------------------------------------------------
diff --git a/src/test/com/cloudera/sqoop/metastore/hsqldb/HsqldbMetaConnectIncrementalImportTest.java b/src/test/com/cloudera/sqoop/metastore/hsqldb/HsqldbMetaConnectIncrementalImportTest.java
new file mode 100644
index 0000000..d302bfb
--- /dev/null
+++ b/src/test/com/cloudera/sqoop/metastore/hsqldb/HsqldbMetaConnectIncrementalImportTest.java
@@ -0,0 +1,38 @@
+/**
+ * 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 com.cloudera.sqoop.metastore.hsqldb;
+
+import com.cloudera.sqoop.metastore.MetaConnectIncrementalImportTestBase;
+
+/**
+ * Test that Incremental-Import values are stored correctly in Hsqldb
+ *
+ * This class is named in such a way that Sqoop's default QA process does
+ * not run it. You need to run this manually with
+ * -Dtestcase=HsqldbMetaConnectIncrementalImportTest or -Dthirdparty=true.
+ *
+ * This uses JDBC to store and retrieve metastore data from a local Hsqldb server
+ */
+
+public class HsqldbMetaConnectIncrementalImportTest extends MetaConnectIncrementalImportTestBase {
+
+    public HsqldbMetaConnectIncrementalImportTest() {
+        super( "jdbc:hsqldb:mem:sqoopmetastore", "SA" , "");
+    }
+}