You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sqoop.apache.org by an...@apache.org on 2018/01/18 14:03:53 UTC

[07/32] sqoop git commit: SQOOP-3273: Removing com.cloudera.sqoop packages

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6984a36c/src/test/org/apache/sqoop/manager/TestSqlManager.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/TestSqlManager.java b/src/test/org/apache/sqoop/manager/TestSqlManager.java
index 571ed50..185f5a7 100644
--- a/src/test/org/apache/sqoop/manager/TestSqlManager.java
+++ b/src/test/org/apache/sqoop/manager/TestSqlManager.java
@@ -18,19 +18,240 @@
 
 package org.apache.sqoop.manager;
 
-import static org.junit.Assert.assertArrayEquals;
-
 import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
+import java.sql.Types;
+import java.util.Map;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.sqoop.SqoopOptions;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
-import com.cloudera.sqoop.SqoopOptions;
+import org.apache.sqoop.testutil.HsqldbTestServer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertArrayEquals;
+
 /**
  * Test methods of the generic SqlManager implementation.
  */
 public class TestSqlManager {
 
+  public static final Log LOG = LogFactory.getLog(TestSqlManager.class.getName());
+
+  /** the name of a table that doesn't exist. */
+  static final String MISSING_TABLE = "MISSING_TABLE";
+
+  // instance variables populated during setUp, used during tests
+  private HsqldbTestServer testServer;
+  private ConnManager manager;
+
+  @Before
+  public void setUp() {
+    testServer = new HsqldbTestServer();
+    try {
+      testServer.resetServer();
+    } catch (SQLException sqlE) {
+      LOG.error("Got SQLException: " + sqlE.toString());
+      fail("Got SQLException: " + sqlE.toString());
+    } catch (ClassNotFoundException cnfe) {
+      LOG.error("Could not find class for db driver: " + cnfe.toString());
+      fail("Could not find class for db driver: " + cnfe.toString());
+    }
+
+    manager = testServer.getManager();
+  }
+
+  @After
+  public void tearDown() {
+    try {
+      manager.close();
+    } catch (SQLException sqlE) {
+      LOG.error("Got SQLException: " + sqlE.toString());
+      fail("Got SQLException: " + sqlE.toString());
+    }
+  }
+
+  @Test
+  public void testListColNames() {
+    String [] colNames = manager.getColumnNames(
+        HsqldbTestServer.getTableName());
+    assertNotNull("manager returned no colname list", colNames);
+    assertEquals("Table list should be length 2", 2, colNames.length);
+    String [] knownFields = HsqldbTestServer.getFieldNames();
+    for (int i = 0; i < colNames.length; i++) {
+      assertEquals(knownFields[i], colNames[i]);
+    }
+  }
+
+  @Test
+  public void testListColTypes() {
+    Map<String, Integer> types = manager.getColumnTypes(
+        HsqldbTestServer.getTableName());
+
+    assertNotNull("manager returned no types map", types);
+    assertEquals("Map should be size=2", 2, types.size());
+    assertEquals(types.get("INTFIELD1").intValue(), Types.INTEGER);
+    assertEquals(types.get("INTFIELD2").intValue(), Types.INTEGER);
+  }
+
+  @Test
+  public void testMissingTableColNames() {
+    String [] colNames = manager.getColumnNames(MISSING_TABLE);
+    assertNull("No column names should be returned for missing table",
+        colNames);
+  }
+
+  @Test
+  public void testMissingTableColTypes() {
+    Map<String, Integer> colTypes = manager.getColumnTypes(MISSING_TABLE);
+    assertNull("No column types should be returned for missing table",
+        colTypes);
+  }
+
+  @Test
+  public void testListTables() {
+    String [] tables = manager.listTables();
+    for (String table : tables) {
+      System.err.println("Got table: " + table);
+    }
+    assertNotNull("manager returned no table list", tables);
+    assertEquals("Table list should be length 1", 1, tables.length);
+    assertEquals(HsqldbTestServer.getTableName(), tables[0]);
+  }
+
+  // constants related to testReadTable()
+  static final int EXPECTED_NUM_ROWS = 4;
+  static final int EXPECTED_COL1_SUM = 16;
+  static final int EXPECTED_COL2_SUM = 20;
+
+  @Test
+  public void testReadTable() {
+    ResultSet results = null;
+    try {
+      results = manager.readTable(HsqldbTestServer.getTableName(),
+          HsqldbTestServer.getFieldNames());
+
+      assertNotNull("ResultSet from readTable() is null!", results);
+
+      ResultSetMetaData metaData = results.getMetaData();
+      assertNotNull("ResultSetMetadata is null in readTable()", metaData);
+
+      // ensure that we get the correct number of columns back
+      assertEquals("Number of returned columns was unexpected!",
+          metaData.getColumnCount(),
+          HsqldbTestServer.getFieldNames().length);
+
+      // should get back 4 rows. They are:
+      // 1 2
+      // 3 4
+      // 5 6
+      // 7 8
+      // .. so while order isn't guaranteed, we should get back 16 on the left
+      // and 20 on the right.
+      int sumCol1 = 0, sumCol2 = 0, rowCount = 0;
+      while (results.next()) {
+        rowCount++;
+        sumCol1 += results.getInt(1);
+        sumCol2 += results.getInt(2);
+      }
+
+      assertEquals("Expected 4 rows back", EXPECTED_NUM_ROWS, rowCount);
+      assertEquals("Expected left sum of 16", EXPECTED_COL1_SUM, sumCol1);
+      assertEquals("Expected right sum of 20", EXPECTED_COL2_SUM, sumCol2);
+    } catch (SQLException sqlException) {
+      fail("SQL Exception: " + sqlException.toString());
+    } finally {
+      if (null != results) {
+        try {
+          results.close();
+        } catch (SQLException sqlE) {
+          fail("SQL Exception in ResultSet.close(): " + sqlE.toString());
+        }
+      }
+
+      manager.release();
+    }
+  }
+
+  @Test
+  public void testReadMissingTable() {
+    ResultSet results = null;
+    try {
+      String [] colNames = { "*" };
+      results = manager.readTable(MISSING_TABLE, colNames);
+      assertNull("Expected null resultset from readTable(MISSING_TABLE)",
+          results);
+    } catch (SQLException sqlException) {
+      // we actually expect this. pass.
+    } finally {
+      if (null != results) {
+        try {
+          results.close();
+        } catch (SQLException sqlE) {
+          fail("SQL Exception in ResultSet.close(): " + sqlE.toString());
+        }
+      }
+
+      manager.release();
+    }
+  }
+
+  @Test
+  public void getPrimaryKeyFromMissingTable() {
+    String primaryKey = manager.getPrimaryKey(MISSING_TABLE);
+    assertNull("Expected null pkey for missing table", primaryKey);
+  }
+
+  @Test
+  public void getPrimaryKeyFromTableWithoutKey() {
+    String primaryKey = manager.getPrimaryKey(HsqldbTestServer.getTableName());
+    assertNull("Expected null pkey for table without key", primaryKey);
+  }
+
+  // constants for getPrimaryKeyFromTable()
+  static final String TABLE_WITH_KEY = "TABLE_WITH_KEY";
+  static final String KEY_FIELD_NAME = "KEYFIELD";
+
+  @Test
+  public void getPrimaryKeyFromTable() {
+    // first, create a table with a primary key
+    Connection conn = null;
+    try {
+      conn = testServer.getConnection();
+      PreparedStatement statement = conn.prepareStatement(
+          "CREATE TABLE " + TABLE_WITH_KEY + "(" + KEY_FIELD_NAME
+              + " INT NOT NULL PRIMARY KEY, foo INT)",
+          ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+      statement.executeUpdate();
+      statement.close();
+    } catch (SQLException sqlException) {
+      fail("Could not create table with primary key: "
+          + sqlException.toString());
+    } finally {
+      if (null != conn) {
+        try {
+          conn.close();
+        } catch (SQLException sqlE) {
+          LOG.warn("Got SQLException during close: " + sqlE.toString());
+        }
+      }
+    }
+
+    String primaryKey = manager.getPrimaryKey(TABLE_WITH_KEY);
+    assertEquals("Expected null pkey for table without key", primaryKey,
+        KEY_FIELD_NAME);
+  }
+
   @Test
   public void testFilteringSpecifiedColumnNamesWhenNoneSpecified() {
     SqoopOptions opts = new SqoopOptions();

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6984a36c/src/test/org/apache/sqoop/manager/cubrid/CubridAuthTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/cubrid/CubridAuthTest.java b/src/test/org/apache/sqoop/manager/cubrid/CubridAuthTest.java
index 78103ec..82fac12 100644
--- a/src/test/org/apache/sqoop/manager/cubrid/CubridAuthTest.java
+++ b/src/test/org/apache/sqoop/manager/cubrid/CubridAuthTest.java
@@ -32,9 +32,9 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import com.cloudera.sqoop.SqoopOptions;
-import com.cloudera.sqoop.manager.ConnManager;
-import com.cloudera.sqoop.testutil.ImportJobTestCase;
+import org.apache.sqoop.SqoopOptions;
+import org.apache.sqoop.manager.ConnManager;
+import org.apache.sqoop.testutil.ImportJobTestCase;
 
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6984a36c/src/test/org/apache/sqoop/manager/cubrid/CubridCompatTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/cubrid/CubridCompatTest.java b/src/test/org/apache/sqoop/manager/cubrid/CubridCompatTest.java
index 05c7bf1..8a075e8 100644
--- a/src/test/org/apache/sqoop/manager/cubrid/CubridCompatTest.java
+++ b/src/test/org/apache/sqoop/manager/cubrid/CubridCompatTest.java
@@ -27,8 +27,8 @@ 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.testutil.ManagerCompatTestCase;
+import org.apache.sqoop.SqoopOptions;
+import org.apache.sqoop.testutil.ManagerCompatTestCase;
 
 /**
  * Test the basic Cubrid connection manager with the various column types.

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6984a36c/src/test/org/apache/sqoop/manager/cubrid/CubridManagerExportTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/cubrid/CubridManagerExportTest.java b/src/test/org/apache/sqoop/manager/cubrid/CubridManagerExportTest.java
new file mode 100644
index 0000000..4de8e40
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/cubrid/CubridManagerExportTest.java
@@ -0,0 +1,305 @@
+/**
+ * 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.manager.cubrid;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.Writer;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.sqoop.manager.CubridManager;
+import org.junit.After;
+import org.junit.Before;
+
+import org.apache.sqoop.SqoopOptions;
+import org.apache.sqoop.TestExport;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * Test the CubridManager implementation.
+ *
+ * This uses JDBC to export data from HDFS to an Cubrid database.
+ *
+ * Since this requires an Cubrid 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=CubridManagerExportTest.
+ *
+ * You need to put Cubrid JDBC driver library (JDBC-9.2.2.0003-cubrid.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).
+ *
+ * To set up your test environment:
+ *   Install Cubrid 9.2.2
+ *   ref:http://www.cubrid.org/wiki_tutorials/entry/installing-cubrid-on-linux-using-shell-and-rpm
+ *   Create a database SQOOPCUBRIDTEST
+ *   $cubrid createdb SQOOPCUBRIDTEST en_us.utf8
+ *   Start cubrid and database
+ *   $cubrid service start
+ *   $cubrid server start SQOOPCUBRIDTEST
+ *   Create a login SQOOPUSER with password PASSWORD and grant all
+ *   $csql -u dba SQOOPCUBRIDTEST
+ *   csql>CREATE USER SQOOPUSER password 'PASSWORD';
+ */
+public class CubridManagerExportTest extends TestExport {
+
+  public static final Log LOG = LogFactory.getLog(
+      CubridManagerExportTest.class.getName());
+
+  static final String TABLE_PREFIX = "EXPORT_CUBRID_";
+
+  // instance variables populated during setUp, used during tests.
+  private CubridManager manager;
+  private Connection conn;
+
+  @Override
+  protected Connection getConnection() {
+    return conn;
+  }
+
+  @Override
+  protected boolean useHsqldbTestServer() {
+    return false;
+  }
+
+  @Override
+  protected String getConnectString() {
+    return CubridTestUtils.getConnectString();
+  }
+
+  @Override
+  protected String getTablePrefix() {
+    return TABLE_PREFIX;
+  }
+
+  @Override
+  protected String getDropTableStatement(String tableName) {
+    return "DROP TABLE IF EXISTS " + tableName;
+  }
+
+  /**
+   * Cubrid could not support --staging-table, Diable this test case.
+   */
+  @Override
+  @Test
+  public void testMultiTransactionWithStaging() throws IOException,
+      SQLException {
+    return;
+  }
+
+  /**
+   * Cubrid could not support --staging-table, Diable this test case.
+   */
+  @Override
+  @Test
+  public void testMultiMapTextExportWithStaging() throws IOException,
+      SQLException {
+    return;
+  }
+
+  public void createTableAndPopulateData(String table) {
+    String fulltableName = manager.escapeTableName(table);
+
+    Statement stmt = null;
+
+    // Drop the existing table, if there is one.
+    try {
+      conn = manager.getConnection();
+      stmt = conn.createStatement();
+      stmt.execute("DROP TABLE IF EXISTS " + fulltableName);
+      conn.commit();
+    } catch (SQLException sqlE) {
+      LOG.info("Table was not dropped: " + sqlE.getMessage());
+    } finally {
+      try {
+        if (null != stmt) {
+          stmt.close();
+        }
+      } catch (Exception ex) {
+        LOG.warn("Exception while closing stmt", ex);
+      }
+    }
+
+    // Create and populate table
+    try {
+      conn = manager.getConnection();
+      conn.setAutoCommit(false);
+      stmt = conn.createStatement();
+
+      // create the database table and populate it with data.
+      stmt.executeUpdate("CREATE TABLE "
+          + fulltableName + " ("
+          + "id INT NOT NULL, "
+          + "name VARCHAR(24) NOT NULL, "
+          + "salary FLOAT, " + "dept VARCHAR(32), "
+          + "PRIMARY KEY (id))");
+      conn.commit();
+    } catch (SQLException sqlE) {
+      LOG.error("Encountered SQL Exception: ", sqlE);
+      sqlE.printStackTrace();
+      fail("SQLException when running test setUp(): " + sqlE);
+    } finally {
+      try {
+        if (null != stmt) {
+          stmt.close();
+        }
+      } catch (Exception ex) {
+        LOG.warn(
+            "Exception while closing connection/stmt", ex);
+      }
+    }
+  }
+
+  @Before
+  public void setUp() {
+    super.setUp();
+
+    SqoopOptions options = new SqoopOptions(
+        CubridTestUtils.getConnectString(),
+        getTableName());
+    options.setUsername(CubridTestUtils.getCurrentUser());
+    options.setPassword(CubridTestUtils.getPassword());
+    this.manager = new CubridManager(options);
+    try {
+      this.conn = manager.getConnection();
+      this.conn.setAutoCommit(false);
+
+    } catch (SQLException sqlE) {
+      LOG.error(StringUtils.stringifyException(sqlE));
+      fail("Failed with sql exception in setup: " + sqlE);
+    }
+  }
+
+  @After
+  public void tearDown() {
+    super.tearDown();
+    try {
+      conn.close();
+      manager.close();
+    } catch (SQLException sqlE) {
+      LOG.error("Got SQLException: " + sqlE.toString());
+      fail("Got SQLException: " + sqlE.toString());
+    }
+  }
+
+  @Override
+  protected String[] getCodeGenArgv(String... extraArgs) {
+    String[] moreArgs = new String[extraArgs.length + 4];
+    int i = 0;
+    for (i = 0; i < extraArgs.length; i++) {
+      moreArgs[i] = extraArgs[i];
+    }
+
+    // Add username and password args.
+    moreArgs[i++] = "--username";
+    moreArgs[i++] = CubridTestUtils.getCurrentUser();
+    moreArgs[i++] = "--password";
+    moreArgs[i++] = CubridTestUtils.getPassword();
+
+    return super.getCodeGenArgv(moreArgs);
+  }
+
+  @Override
+  protected String[] getArgv(boolean includeHadoopFlags,
+      int rowsPerStatement,
+      int statementsPerTx, String... additionalArgv) {
+
+    String[] subArgv = newStrArray(additionalArgv, "--username",
+        CubridTestUtils.getCurrentUser(), "--password",
+        CubridTestUtils.getPassword());
+    return super.getArgv(includeHadoopFlags, rowsPerStatement,
+        statementsPerTx, subArgv);
+  }
+
+  protected void createTestFile(String filename,
+      String[] lines)
+      throws IOException {
+    File testdir = new File(getWarehouseDir());
+    if (!testdir.exists()) {
+      testdir.mkdirs();
+    }
+    File file = new File(getWarehouseDir() + "/" + filename);
+    Writer output = new BufferedWriter(new FileWriter(file));
+    for (String line : lines) {
+      output.write(line);
+      output.write("\n");
+    }
+    output.close();
+  }
+
+  public static void assertRowCount(long expected,
+      String tableName,
+      Connection connection) {
+    Statement stmt = null;
+    ResultSet rs = null;
+    try {
+      stmt = connection.createStatement();
+      rs = stmt.executeQuery("SELECT count(*) FROM "
+          + tableName);
+      rs.next();
+      assertEquals(expected, rs.getLong(1));
+    } catch (SQLException e) {
+      LOG.error("Can't verify number of rows", e);
+      fail();
+    } finally {
+      try {
+        connection.commit();
+        if (stmt != null) {
+          stmt.close();
+        }
+        if (rs != null) {
+          rs.close();
+        }
+      } catch (SQLException ex) {
+        LOG.info("Ignored exception in finally block.");
+      }
+    }
+  }
+
+  public String escapeTableOrSchemaName(String tableName) {
+    return "\"" + tableName + "\"";
+  }
+
+  /** Make sure mixed update/insert export work correctly. */
+  @Test
+  public void testUpsertTextExport() throws IOException, SQLException {
+    final int TOTAL_RECORDS = 10;
+    createTextFile(0, TOTAL_RECORDS, false);
+    createTable();
+    // first time will be insert.
+    runExport(getArgv(true, 10, 10,
+        newStrArray(null, "--update-key", "id",
+        "--update-mode", "allowinsert")));
+    // second time will be update.
+    runExport(getArgv(true, 10, 10,
+        newStrArray(null, "--update-key", "id",
+        "--update-mode", "allowinsert")));
+    verifyExport(TOTAL_RECORDS);
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6984a36c/src/test/org/apache/sqoop/manager/cubrid/CubridManagerImportTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/cubrid/CubridManagerImportTest.java b/src/test/org/apache/sqoop/manager/cubrid/CubridManagerImportTest.java
new file mode 100644
index 0000000..addf1ae
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/cubrid/CubridManagerImportTest.java
@@ -0,0 +1,297 @@
+/**
+ * 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.manager.cubrid;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.sqoop.manager.ConnManager;
+import org.apache.sqoop.manager.CubridManager;
+import org.apache.sqoop.util.FileListing;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.sqoop.SqoopOptions;
+import org.apache.sqoop.testutil.CommonArgs;
+import org.apache.sqoop.testutil.ImportJobTestCase;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Test the CubridManager implementation.
+ *
+ * This uses JDBC to import data from an Cubrid database into HDFS.
+ *
+ * Since this requires an Cubrid 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=CubridManagerImportTest.
+ *
+ * You need to put Cubrid JDBC driver library (JDBC-9.2.2.0003-cubrid.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).
+ *
+ * To set up your test environment:
+ *   Install Cubrid 9.2.2
+ *   ref:http://www.cubrid.org/wiki_tutorials/entry/installing-cubrid-on-linux-using-shell-and-rpm
+ *   Create a database SQOOPCUBRIDTEST
+ *   $cubrid createdb SQOOPCUBRIDTEST en_us.utf8
+ *   Start cubrid and database
+ *   $cubrid service start
+ *   $cubrid server start SQOOPCUBRIDTEST
+ *   Create a login SQOOPUSER with password PASSWORD and grant all
+ *   $csql -u dba SQOOPCUBRIDTEST
+ *   csql>CREATE USER SQOOPUSER password 'PASSWORD';
+ */
+public class CubridManagerImportTest extends ImportJobTestCase {
+
+  public static final Log LOG = LogFactory.getLog(
+      CubridManagerImportTest.class.getName());
+
+  static final String TABLE_NAME = "employees_cubrid";
+  static final String NULL_TABLE_NAME = "null_employees_cubrid";
+
+  // instance variables populated during setUp, used during tests
+  private CubridManager manager;
+
+  private Configuration conf = new Configuration();
+
+  @Override
+  protected Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  protected boolean useHsqldbTestServer() {
+    return false;
+  }
+
+  @Before
+  public void setUp() {
+    super.setUp();
+    LOG.debug("Setting up another CubridImport test: "
+      + CubridTestUtils.getConnectString());
+    setUpData(TABLE_NAME, false);
+    setUpData(NULL_TABLE_NAME, true);
+    LOG.debug("setUp complete.");
+  }
+
+  public void setUpData(String tableName, boolean nullEntry) {
+    SqoopOptions options = new SqoopOptions(
+        CubridTestUtils.getConnectString(), tableName);
+    options.setUsername(CubridTestUtils.getCurrentUser());
+    options.setPassword(CubridTestUtils.getPassword());
+
+    LOG.debug("Setting up another CubridImport test: "
+      + CubridTestUtils.getConnectString());
+
+    manager = new CubridManager(options);
+
+    Connection connection = null;
+    Statement st = null;
+
+    try {
+      connection = manager.getConnection();
+      connection.setAutoCommit(false);
+      st = connection.createStatement();
+
+      // create the database table and populate it with data.
+      st.executeUpdate("DROP TABLE IF EXISTS " + tableName);
+      st.executeUpdate("CREATE TABLE " + tableName + " ("
+          + manager.escapeColName("id")
+          + " INT NOT NULL PRIMARY KEY, "
+          + manager.escapeColName("name")
+          + " VARCHAR(24) NOT NULL, "
+          + manager.escapeColName("start_date") + " DATE, "
+          + manager.escapeColName("Salary") + " FLOAT, "
+          + manager.escapeColName("dept")
+          + " VARCHAR(32));");
+
+      st.executeUpdate("INSERT INTO " + tableName
+          + " VALUES(1,'Aaron','2009-05-14',"
+          + "1000000.00,'engineering');");
+      st.executeUpdate("INSERT INTO " + tableName
+          + " VALUES(2,'Bob','2009-04-20',400.00,'sales');");
+      st.executeUpdate("INSERT INTO " + tableName
+          + " VALUES(3,'Fred','2009-01-23',"
+          + "15.00,'marketing');");
+      if (nullEntry) {
+        st.executeUpdate("INSERT INTO " + tableName
+            + " VALUES(4,'Mike',NULL,NULL,NULL);");
+      }
+
+      connection.commit();
+    } catch (SQLException sqlE) {
+      LOG.error("Encountered SQL Exception: " + sqlE);
+      sqlE.printStackTrace();
+      fail("SQLException when running test setUp(): " + sqlE);
+    } finally {
+      try {
+        if (null != st) {
+          st.close();
+        }
+
+        if (null != connection) {
+          connection.close();
+        }
+      } catch (SQLException sqlE) {
+        LOG.warn("Got SQLException when closing connection: "
+          + sqlE);
+      }
+    }
+  }
+
+  @After
+  public void tearDown() {
+    super.tearDown();
+    try {
+      manager.close();
+    } catch (SQLException sqlE) {
+      LOG.error("Got SQLException: " + sqlE.toString());
+      fail("Got SQLException: " + sqlE.toString());
+    }
+  }
+
+  @Test
+  public void testImportSimple() throws IOException {
+    String[] expectedResults = {
+        "1,Aaron,2009-05-14,1000000.0,engineering",
+        "2,Bob,2009-04-20,400.0,sales",
+        "3,Fred,2009-01-23,15.0,marketing", };
+
+    doImportAndVerify(TABLE_NAME, expectedResults);
+  }
+
+  @Test
+  public void testListTables() throws IOException {
+    SqoopOptions options = new SqoopOptions(new Configuration());
+    options.setConnectString(CubridTestUtils.getConnectString());
+    options.setUsername(CubridTestUtils.getCurrentUser());
+    options.setPassword(CubridTestUtils.getPassword());
+
+    ConnManager mgr = new CubridManager(options);
+    String[] tables = mgr.listTables();
+    Arrays.sort(tables);
+    assertTrue(TABLE_NAME + " is not found!",
+        Arrays.binarySearch(tables, TABLE_NAME) >= 0);
+  }
+
+  @Test
+  public void testNullEscapeCharacters() throws Exception {
+    String[] expectedResults = {
+        "1,Aaron,2009-05-14,1000000.0,engineering",
+        "2,Bob,2009-04-20,400.0,sales",
+        "3,Fred,2009-01-23,15.0,marketing",
+        "4,Mike,cubrid,cubrid,cubrid", };
+
+    String[] extraArgs = {
+        "--null-string",
+        "cubrid",
+        "--null-non-string",
+        "cubrid", };
+
+    doImportAndVerify(NULL_TABLE_NAME, expectedResults, extraArgs);
+  }
+
+  private void doImportAndVerify(String tableName,
+      String[] expectedResults,
+      String... extraArgs) throws IOException {
+    Path warehousePath = new Path(this.getWarehouseDir());
+    Path tablePath = new Path(warehousePath, tableName);
+    Path filePath = new Path(tablePath, "part-m-00000");
+
+    File tableFile = new File(tablePath.toString());
+    if (tableFile.exists() && tableFile.isDirectory()) {
+      // remove the directory before running the import.
+      FileListing.recursiveDeleteDir(tableFile);
+    }
+
+    String[] argv = getArgv(tableName, extraArgs);
+    try {
+      runImport(argv);
+    } catch (IOException ioe) {
+      LOG.error("Got IOException during import: "
+        + ioe.toString());
+      ioe.printStackTrace();
+      fail(ioe.toString());
+    }
+
+    File f = new File(filePath.toString());
+    assertTrue("Could not find imported data file", f.exists());
+    BufferedReader r = null;
+    try {
+      // Read through the file and make sure it's all there.
+      r = new BufferedReader(new InputStreamReader(
+          new FileInputStream(f)));
+      for (String expectedLine : expectedResults) {
+        assertEquals(expectedLine, r.readLine());
+      }
+    } catch (IOException ioe) {
+      LOG.error("Got IOException verifying results: "
+        + ioe.toString());
+      ioe.printStackTrace();
+      fail(ioe.toString());
+    } finally {
+      IOUtils.closeStream(r);
+    }
+  }
+
+  private String[] getArgv(String tableName, String... extraArgs) {
+    ArrayList<String> args = new ArrayList<String>();
+
+    CommonArgs.addHadoopFlags(args);
+
+    args.add("--table");
+    args.add(tableName);
+    args.add("--warehouse-dir");
+    args.add(getWarehouseDir());
+    args.add("--connect");
+    args.add(CubridTestUtils.getConnectString());
+    args.add("--username");
+    args.add(CubridTestUtils.getCurrentUser());
+    args.add("--password");
+    args.add(CubridTestUtils.getPassword());
+    args.add("--num-mappers");
+    args.add("1");
+
+    if (extraArgs.length > 0) {
+      for (String arg : extraArgs) {
+        args.add(arg);
+      }
+    }
+
+    return args.toArray(new String[0]);
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6984a36c/src/test/org/apache/sqoop/manager/db2/DB2ImportAllTableWithSchemaManualTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/db2/DB2ImportAllTableWithSchemaManualTest.java b/src/test/org/apache/sqoop/manager/db2/DB2ImportAllTableWithSchemaManualTest.java
index db6e6bf..d1a6d69 100644
--- a/src/test/org/apache/sqoop/manager/db2/DB2ImportAllTableWithSchemaManualTest.java
+++ b/src/test/org/apache/sqoop/manager/db2/DB2ImportAllTableWithSchemaManualTest.java
@@ -26,7 +26,6 @@ import java.sql.Connection;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
-import java.util.Arrays;
 
 
 import org.apache.commons.logging.Log;
@@ -42,11 +41,11 @@ import org.junit.Before;
 import org.junit.Test;
 
 
-import com.cloudera.sqoop.SqoopOptions;
-import com.cloudera.sqoop.testutil.CommonArgs;
-import com.cloudera.sqoop.testutil.ImportJobTestCase;
-import com.cloudera.sqoop.tool.SqoopTool;
-import com.cloudera.sqoop.util.FileListing;
+import org.apache.sqoop.SqoopOptions;
+import org.apache.sqoop.testutil.CommonArgs;
+import org.apache.sqoop.testutil.ImportJobTestCase;
+import org.apache.sqoop.tool.SqoopTool;
+import org.apache.sqoop.util.FileListing;
 import org.apache.sqoop.util.LoggingUtils;
 
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6984a36c/src/test/org/apache/sqoop/manager/db2/DB2ManagerImportManualTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/db2/DB2ManagerImportManualTest.java b/src/test/org/apache/sqoop/manager/db2/DB2ManagerImportManualTest.java
new file mode 100644
index 0000000..b5d47f2
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/db2/DB2ManagerImportManualTest.java
@@ -0,0 +1,291 @@
+/**
+ * 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.manager.db2;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.sqoop.manager.Db2Manager;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.sqoop.SqoopOptions;
+import org.apache.sqoop.testutil.CommonArgs;
+import org.apache.sqoop.testutil.ImportJobTestCase;
+import org.apache.sqoop.util.FileListing;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Test the DB2Manager implementation.
+ *
+ * This uses JDBC to import data from an DB2 database into HDFS.
+ *
+ * Since this requires an DB2 Server 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=DB2ManagerImportManualTest
+ *
+ * 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).
+ *
+ * To set up your test environment:
+ *   Install DB2 Express 9.7 C server.
+ *   Create a database SQOOP
+ *   Create a login SQOOP with password PASSWORD and grant all
+ *   access for database SQOOP to user SQOOP.
+ */
+public class DB2ManagerImportManualTest extends ImportJobTestCase {
+
+  public static final Log LOG = LogFactory.getLog(
+        DB2ManagerImportManualTest.class.getName());
+
+  static final String HOST_URL = System.getProperty(
+          "sqoop.test.db2.connectstring.host_url",
+          "jdbc:db2://db2host:50000");
+
+  static final String DATABASE_NAME = System.getProperty(
+          "sqoop.test.db2.connectstring.database",
+          "SQOOP");
+  static final String DATABASE_USER = System.getProperty(
+          "sqoop.test.db2.connectstring.username",
+          "SQOOP");
+  static final String DATABASE_PASSWORD = System.getProperty(
+          "sqoop.test.db2.connectstring.password",
+          "SQOOP");
+  static final String TABLE_NAME = "EMPLOYEES_DB2";
+  static final String QUALIFIED_TABLE_NAME = DATABASE_USER + ".EMPLOYEES_DB2";
+  static final String CONNECT_STRING = HOST_URL
+              + "/" + DATABASE_NAME
+              + ":currentSchema=" + DATABASE_USER +";";
+
+  static {
+    LOG.info("Using DB2 CONNECT_STRING: " + CONNECT_STRING);
+  }
+
+  // instance variables populated during setUp, used during tests
+  private Db2Manager manager;
+  private boolean useQualifiedTableName;
+  private boolean useDefaultConnectManager;
+
+  @Override
+  protected boolean useHsqldbTestServer() {
+    return false;
+  }
+
+  @Override
+  protected String getTableName() {
+    return useQualifiedTableName ? QUALIFIED_TABLE_NAME : TABLE_NAME;
+  }
+
+  @Before
+  public void setUp() {
+    super.setUp();
+
+    SqoopOptions options = new SqoopOptions(CONNECT_STRING, getTableName());
+    options.setUsername(DATABASE_USER);
+    options.setPassword(DATABASE_PASSWORD);
+
+    manager = new Db2Manager(options);
+
+    // Drop the existing table, if there is one.
+    Connection conn = null;
+    Statement stmt = null;
+    try {
+      conn = manager.getConnection();
+      stmt = conn.createStatement();
+      stmt.execute("DROP TABLE " + getTableName());
+    } catch (SQLException sqlE) {
+      LOG.info("Table was not dropped: " + sqlE.getMessage());
+    } finally {
+      try {
+        if (null != stmt) {
+          stmt.close();
+        }
+      } catch (Exception ex) {
+        LOG.warn("Exception while closing stmt", ex);
+      }
+    }
+
+    // Create and populate table
+    try {
+      conn = manager.getConnection();
+      conn.setAutoCommit(false);
+      stmt = conn.createStatement();
+
+      // create the database table and populate it with data.
+      stmt.executeUpdate("CREATE TABLE " + getTableName() + " ("
+          + "id INT NOT NULL, "
+          + "name VARCHAR(24) NOT NULL, "
+          + "salary FLOAT, "
+          + "dept VARCHAR(32), "
+          + "PRIMARY KEY (id))");
+
+      stmt.executeUpdate("INSERT INTO " + getTableName() + " VALUES("
+          + "1,'Aaron', "
+          + "1000000.00,'engineering')");
+      stmt.executeUpdate("INSERT INTO " + getTableName() + " VALUES("
+          + "2,'Bob', "
+          + "400.00,'sales')");
+      stmt.executeUpdate("INSERT INTO " + getTableName() + " VALUES("
+          + "3,'Fred', 15.00,"
+          + "'marketing')");
+      conn.commit();
+    } catch (SQLException sqlE) {
+      LOG.error("Encountered SQL Exception: ", sqlE);
+      sqlE.printStackTrace();
+      fail("SQLException when running test setUp(): " + sqlE);
+    } finally {
+      try {
+        if (null != stmt) {
+          stmt.close();
+        }
+      } catch (Exception ex) {
+        LOG.warn("Exception while closing connection/stmt", ex);
+      }
+    }
+  }
+
+  @After
+  public void tearDown() {
+    super.tearDown();
+    try {
+      manager.close();
+    } catch (SQLException sqlE) {
+      LOG.error("Got SQLException: " + sqlE.toString());
+      fail("Got SQLException: " + sqlE.toString());
+    }
+  }
+
+  @Test
+  public void testDb2Import() throws IOException {
+    useQualifiedTableName = false;
+
+    // Verify that GenericJdbcManager works.
+    useDefaultConnectManager = true;
+    runDb2Test(getExpectedResults());
+
+    // Verify that Db2Manager works.
+    useDefaultConnectManager = false;
+    runDb2Test(getExpectedResults());
+  }
+
+  @Test
+  public void testDb2ImportQualifiedTableName() throws IOException {
+    useQualifiedTableName = true;
+
+    // Verify that GenericJdbcManager works.
+    useDefaultConnectManager = true;
+    runDb2Test(getExpectedResults());
+
+    // Verify that Db2Manager works.
+    useDefaultConnectManager = false;
+    runDb2Test(getExpectedResults());
+  }
+
+  private String [] getExpectedResults() {
+    return new String [] {
+      "1,Aaron,1000000.0,engineering",
+      "2,Bob,400.0,sales",
+      "3,Fred,15.0,marketing",
+    };
+  }
+
+  private String [] getArgv() {
+    ArrayList<String> args = new ArrayList<String>();
+
+    CommonArgs.addHadoopFlags(args);
+
+    args.add("--table");
+    args.add(getTableName());
+    args.add("--warehouse-dir");
+    args.add(getWarehouseDir());
+    args.add("--connect");
+    args.add(CONNECT_STRING);
+    args.add("--username");
+    args.add(DATABASE_USER);
+    args.add("--password");
+    args.add(DATABASE_PASSWORD);
+    args.add("--num-mappers");
+    args.add("1");
+
+    if (useDefaultConnectManager) {
+      // Specifying the driver class forces DefaultManagerFactory
+      // to instantiate GenericJdbcManager.
+      args.add("--driver");
+      args.add("com.ibm.db2.jcc.DB2Driver");
+    }
+
+    return args.toArray(new String[0]);
+  }
+
+  private void runDb2Test(String [] expectedResults) throws IOException {
+
+    Path warehousePath = new Path(this.getWarehouseDir());
+    Path tablePath = new Path(warehousePath, getTableName());
+    Path filePath = new Path(tablePath, "part-m-00000");
+
+    File tableFile = new File(tablePath.toString());
+    if (tableFile.exists() && tableFile.isDirectory()) {
+      // remove the directory before running the import.
+      FileListing.recursiveDeleteDir(tableFile);
+    }
+
+    String [] argv = getArgv();
+    try {
+      runImport(argv);
+    } catch (IOException ioe) {
+      LOG.error("Got IOException during import: " + ioe.toString());
+      ioe.printStackTrace();
+      fail(ioe.toString());
+    }
+
+    File f = new File(filePath.toString());
+    assertTrue("Could not find imported data file", f.exists());
+    BufferedReader r = null;
+    try {
+      // Read through the file and make sure it's all there.
+      r = new BufferedReader(new InputStreamReader(new FileInputStream(f)));
+      for (String expectedLine : expectedResults) {
+        assertEquals(expectedLine, r.readLine());
+      }
+    } catch (IOException ioe) {
+      LOG.error("Got IOException verifying results: " + ioe.toString());
+      ioe.printStackTrace();
+      fail(ioe.toString());
+    } finally {
+      IOUtils.closeStream(r);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6984a36c/src/test/org/apache/sqoop/manager/db2/DB2XmlTypeImportManualTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/db2/DB2XmlTypeImportManualTest.java b/src/test/org/apache/sqoop/manager/db2/DB2XmlTypeImportManualTest.java
index 2ae3af8..393a110 100644
--- a/src/test/org/apache/sqoop/manager/db2/DB2XmlTypeImportManualTest.java
+++ b/src/test/org/apache/sqoop/manager/db2/DB2XmlTypeImportManualTest.java
@@ -36,10 +36,10 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-import com.cloudera.sqoop.SqoopOptions;
-import com.cloudera.sqoop.testutil.CommonArgs;
-import com.cloudera.sqoop.testutil.ImportJobTestCase;
-import com.cloudera.sqoop.util.FileListing;
+import org.apache.sqoop.SqoopOptions;
+import org.apache.sqoop.testutil.CommonArgs;
+import org.apache.sqoop.testutil.ImportJobTestCase;
+import org.apache.sqoop.util.FileListing;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6984a36c/src/test/org/apache/sqoop/manager/hsqldb/TestHsqldbManager.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/hsqldb/TestHsqldbManager.java b/src/test/org/apache/sqoop/manager/hsqldb/TestHsqldbManager.java
new file mode 100644
index 0000000..745a812
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/hsqldb/TestHsqldbManager.java
@@ -0,0 +1,87 @@
+/**
+ * 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.manager.hsqldb;
+
+import java.sql.SQLException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.sqoop.manager.ConnManager;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.sqoop.testutil.HsqldbTestServer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+/**
+ * Test HsqldbManager-specific functionality that overrides SqlManager
+ * behavior.
+ */
+public class TestHsqldbManager {
+
+  public static final Log LOG = LogFactory.getLog(
+      TestHsqldbManager.class.getName());
+
+  // instance variables populated during setUp, used during tests
+  private HsqldbTestServer testServer;
+  private ConnManager manager;
+
+  @Before
+  public void setUp() {
+    testServer = new HsqldbTestServer();
+    try {
+      testServer.resetServer();
+    } catch (SQLException sqlE) {
+      LOG.error("Got SQLException: " + sqlE.toString());
+      fail("Got SQLException: " + sqlE.toString());
+    } catch (ClassNotFoundException cnfe) {
+      LOG.error("Could not find class for db driver: " + cnfe.toString());
+      fail("Could not find class for db driver: " + cnfe.toString());
+    }
+
+    manager = testServer.getManager();
+  }
+
+  @After
+  public void tearDown() {
+    try {
+      manager.close();
+    } catch (SQLException sqlE) {
+      LOG.error("Got SQLException: " + sqlE.toString());
+      fail("Got SQLException: " + sqlE.toString());
+    }
+  }
+
+  // note: hsql returns only the "PUBLIC" schema name; not individual user db
+  // names.
+  @Test
+  public void testListDatabases() {
+    String [] databases = manager.listDatabases();
+
+    assertNotNull("manager returned no database list", databases);
+    assertEquals("Database list should be length 1", 1, databases.length);
+    assertEquals(HsqldbTestServer.getSchemaName(), databases[0]);
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6984a36c/src/test/org/apache/sqoop/manager/mysql/DirectMySQLExportTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/mysql/DirectMySQLExportTest.java b/src/test/org/apache/sqoop/manager/mysql/DirectMySQLExportTest.java
new file mode 100644
index 0000000..b3570ff
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/mysql/DirectMySQLExportTest.java
@@ -0,0 +1,364 @@
+/**
+ * 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.manager.mysql;
+
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.sql.SQLException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.sqoop.manager.DirectMySQLManager;
+import org.junit.After;
+import org.junit.Before;
+
+import org.apache.sqoop.SqoopOptions;
+import org.apache.sqoop.TestExport;
+import org.apache.sqoop.mapreduce.MySQLExportMapper;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Test the DirectMySQLManager implementation's exportJob() functionality.
+ */
+public class DirectMySQLExportTest extends TestExport {
+
+  public static final Log LOG = LogFactory.getLog(
+      DirectMySQLExportTest.class.getName());
+
+  static final String TABLE_PREFIX = "EXPORT_MYSQL_";
+
+  // instance variables populated during setUp, used during tests.
+  private DirectMySQLManager manager;
+  private Connection conn;
+  private MySQLTestUtils mySQLTestUtils = new MySQLTestUtils();
+
+  @Override
+  protected Connection getConnection() {
+    return conn;
+  }
+
+  @Override
+  protected boolean useHsqldbTestServer() {
+    return false;
+  }
+
+  @Override
+  protected String getConnectString() {
+    return mySQLTestUtils.getMySqlConnectString();
+  }
+
+  @Override
+  protected String getTablePrefix() {
+    return TABLE_PREFIX;
+  }
+
+  @Override
+  protected String getDropTableStatement(String tableName) {
+    return "DROP TABLE IF EXISTS " + tableName;
+  }
+
+  @Before
+  public void setUp() {
+    super.setUp();
+
+    SqoopOptions options = new SqoopOptions(mySQLTestUtils.getMySqlConnectString(),
+        getTableName());
+    options.setUsername(mySQLTestUtils.getUserName());
+    mySQLTestUtils.addPasswordIfIsSet(options);
+    this.manager = new DirectMySQLManager(options);
+
+    try {
+      this.conn = manager.getConnection();
+      this.conn.setAutoCommit(false);
+    } catch (SQLException sqlE) {
+      LOG.error("Encountered SQL Exception: " + sqlE);
+      sqlE.printStackTrace();
+      fail("SQLException when running test setUp(): " + sqlE);
+    }
+  }
+
+  @After
+  public void tearDown() {
+    try {
+      Statement stmt = conn.createStatement();
+      stmt.execute(getDropTableStatement(getTableName()));
+    } catch(SQLException e) {
+      LOG.error("Can't clean up the database:", e);
+    }
+
+    super.tearDown();
+
+    if (null != this.conn) {
+      try {
+        this.conn.close();
+      } catch (SQLException sqlE) {
+        LOG.error("Got SQLException closing conn: " + sqlE.toString());
+      }
+    }
+  }
+
+  @Override
+  protected String [] getCodeGenArgv(String... extraArgs) {
+    return super.getCodeGenArgv(mySQLTestUtils.addUserNameAndPasswordToArgs(extraArgs));
+  }
+
+  @Override
+  protected String [] getArgv(boolean includeHadoopFlags,
+      int rowsPerStatement, int statementsPerTx, String... additionalArgv) {
+
+    String [] subArgv = newStrArray(mySQLTestUtils.addUserNameAndPasswordToArgs(additionalArgv),"--direct");
+    return super.getArgv(includeHadoopFlags, rowsPerStatement,
+        statementsPerTx, subArgv);
+  }
+
+  /**
+   * Test a single mapper that runs several transactions serially.
+   */
+  @Test
+  public void testMultiTxExport() throws IOException, SQLException {
+    multiFileTest(1, 20, 1,
+        "-D", MySQLExportMapper.MYSQL_CHECKPOINT_BYTES_KEY + "=10");
+  }
+
+  /**
+   * Test an authenticated export using mysqlimport.
+   */
+  @Test
+  public void testAuthExport() throws IOException, SQLException {
+    SqoopOptions options = new SqoopOptions(mySQLTestUtils.getMySqlConnectString(),
+        getTableName());
+    options.setUsername(mySQLTestUtils.getUserName());
+    options.setPassword(mySQLTestUtils.getUserPass());
+
+    manager = new DirectMySQLManager(options);
+
+    Connection connection = null;
+    Statement st = null;
+
+    String tableName = getTableName();
+
+    try {
+      connection = manager.getConnection();
+      connection.setAutoCommit(false);
+      st = connection.createStatement();
+
+      // create a target database table.
+      st.executeUpdate("DROP TABLE IF EXISTS " + tableName);
+      st.executeUpdate("CREATE TABLE " + tableName + " ("
+          + "id INT NOT NULL PRIMARY KEY, "
+          + "msg VARCHAR(24) NOT NULL)");
+      connection.commit();
+
+      // Write a file containing a record to export.
+      Path tablePath = getTablePath();
+      Path filePath = new Path(tablePath, "datafile");
+      Configuration conf = new Configuration();
+      conf.set("fs.default.name", "file:///");
+
+      FileSystem fs = FileSystem.get(conf);
+      fs.mkdirs(tablePath);
+      OutputStream os = fs.create(filePath);
+      BufferedWriter w = new BufferedWriter(new OutputStreamWriter(os));
+      w.write(getRecordLine(0));
+      w.write(getRecordLine(1));
+      w.write(getRecordLine(2));
+      w.close();
+      os.close();
+
+      // run the export and verify that the results are good.
+      runExport(getArgv(true, 10, 10,
+          "--username", mySQLTestUtils.getUserName(),
+          "--password", mySQLTestUtils.getUserPass(),
+          "--connect", mySQLTestUtils.getMySqlConnectString()));
+      verifyExport(3, connection);
+    } catch (SQLException sqlE) {
+      LOG.error("Encountered SQL Exception: " + sqlE);
+      sqlE.printStackTrace();
+      fail("SQLException when accessing target table. " + sqlE);
+    } finally {
+      try {
+        if (null != st) {
+          st.close();
+        }
+
+        if (null != connection) {
+          connection.close();
+        }
+      } catch (SQLException sqlE) {
+        LOG.warn("Got SQLException when closing connection: " + sqlE);
+      }
+    }
+  }
+
+  /**
+   * Test an authenticated export using mysqlimport.
+   */
+  @Test
+  public void testEscapedByExport() throws IOException, SQLException {
+    SqoopOptions options = new SqoopOptions(mySQLTestUtils.getMySqlConnectString(),
+        getTableName());
+    options.setUsername(mySQLTestUtils.getUserName());
+    options.setPassword(mySQLTestUtils.getUserPass());
+
+    manager = new DirectMySQLManager(options);
+
+    Connection connection = null;
+    Statement st = null;
+
+    String tableName = getTableName();
+
+    try {
+      connection = manager.getConnection();
+      connection.setAutoCommit(false);
+      st = connection.createStatement();
+
+      // create a target database table.
+      st.executeUpdate("DROP TABLE IF EXISTS " + tableName);
+      st.executeUpdate("CREATE TABLE " + tableName + " ("
+          + "id INT NOT NULL PRIMARY KEY, "
+          + "msg VARCHAR(24) NOT NULL, "
+          + "value VARCHAR(100) NOT NULL)");
+      connection.commit();
+
+      // Write a file containing a record to export.
+      Path tablePath = getTablePath();
+      Path filePath = new Path(tablePath, "datafile");
+      Configuration conf = new Configuration();
+      conf.set("fs.default.name", "file:///");
+
+      ColumnGenerator gen = new ColumnGenerator() {
+        public String getExportText(int rowNum) {
+          return "||" + rowNum;
+        }
+        public String getVerifyText(int rowNum) {
+          return "|" + rowNum;
+        }
+        public String getType() {
+          return "STRING";
+        }
+      };
+
+      FileSystem fs = FileSystem.get(conf);
+      fs.mkdirs(tablePath);
+      OutputStream os = fs.create(filePath);
+      BufferedWriter w = new BufferedWriter(new OutputStreamWriter(os));
+      w.write(getRecordLine(0, gen));
+      w.write(getRecordLine(1, gen));
+      w.write(getRecordLine(2, gen));
+      w.close();
+      os.close();
+
+      // run the export and verify that the results are good.
+      runExport(getArgv(true, 10, 10,
+          "--username", mySQLTestUtils.getUserName(),
+          "--password", mySQLTestUtils.getUserPass(),
+          "--connect", mySQLTestUtils.getMySqlConnectString(),
+          "--escaped-by", "|"));
+      verifyExport(3, connection);
+      verifyTableColumnContents(connection, tableName, "value", gen);
+    } catch (SQLException sqlE) {
+      LOG.error("Encountered SQL Exception: " + sqlE);
+      sqlE.printStackTrace();
+      fail("SQLException when accessing target table. " + sqlE);
+    } finally {
+      try {
+        if (null != st) {
+          st.close();
+        }
+
+        if (null != connection) {
+          connection.close();
+        }
+      } catch (SQLException sqlE) {
+        LOG.warn("Got SQLException when closing connection: " + sqlE);
+      }
+    }
+  }
+
+  @Test(expected = IOException.class)
+  public void testExportInputNullStringFailsValidate() throws IOException {
+    runExport(getArgv(true, 10, 10,
+            "--username", mySQLTestUtils.getUserName(),
+            "--password", mySQLTestUtils.getUserPass(),
+            "--connect", mySQLTestUtils.getMySqlConnectString(),
+            "--input-null-string", "null"));
+  }
+
+  @Test(expected = IOException.class)
+  public void testExportInputNullNonStringFailsValidate() throws IOException {
+    runExport(getArgv(true, 10, 10,
+            "--username", mySQLTestUtils.getUserName(),
+            "--password", mySQLTestUtils.getUserPass(),
+            "--connect", mySQLTestUtils.getMySqlConnectString(),
+            "--input-null-non-string", "null"));
+  }
+
+  @Ignore("Ignoring this test as staging is not supported in direct mode.")
+  @Override
+  @Test
+  public void testMultiMapTextExportWithStaging()
+    throws IOException, SQLException {
+  }
+
+  @Ignore("Ignoring this test as staging is not supported in direct mode.")
+  @Override
+  @Test
+  public void testMultiTransactionWithStaging()
+    throws IOException, SQLException {
+  }
+
+  @Ignore("Ignoring this test as --input-null-non-string is not supported in direct mode.")
+  @Override
+  @Test
+  public void testLessColumnsInFileThanInTableInputNullIntPassed() throws IOException, SQLException {
+  }
+
+  @Ignore("Ignoring this test as --input-null-string is not supported in direct mode.")
+  @Override
+  @Test
+  public void testLessColumnsInFileThanInTableInputNullStringPassed() throws IOException, SQLException {
+  }
+
+  private void verifyTableColumnContents(Connection connection,
+    String table, String column, ColumnGenerator gen)
+      throws IOException, SQLException {
+    Statement st = connection.createStatement();
+
+    // create a target database table.
+    assertTrue(st.execute("SELECT " + column + " FROM " + table));
+    ResultSet rs = st.getResultSet();
+
+    for (int row = 0; rs.next(); ++row) {
+      assertEquals(gen.getVerifyText(row), rs.getString(1));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6984a36c/src/test/org/apache/sqoop/manager/mysql/DirectMySQLTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/mysql/DirectMySQLTest.java b/src/test/org/apache/sqoop/manager/mysql/DirectMySQLTest.java
new file mode 100644
index 0000000..89a7fec
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/mysql/DirectMySQLTest.java
@@ -0,0 +1,425 @@
+/**
+ * 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.manager.mysql;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.FileInputStream;
+import java.io.File;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.sqoop.manager.ConnManager;
+import org.apache.sqoop.manager.DirectMySQLManager;
+import org.apache.sqoop.manager.MySQLManager;
+import org.apache.sqoop.SqoopOptions;
+import org.apache.sqoop.testutil.CommonArgs;
+import org.apache.sqoop.testutil.ImportJobTestCase;
+import org.apache.sqoop.util.FileListing;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Test the DirectMySQLManager implementation.
+ * This differs from MySQLManager only in its importTable() method, which
+ * uses mysqldump instead of mapreduce+DBInputFormat.
+ *
+ * Since this requires a MySQL installation on your local machine to use, this
+ * class is named in such a way that Hadoop's default QA process does not run
+ * it. You need to run this manually with -Dtestcase=DirectMySQLTest.
+ *
+ * You need to put MySQL's Connector/J JDBC driver library into a location
+ * where Hadoop will be able to access it (since this library cannot be checked
+ * into Apache's tree for licensing reasons).
+ *
+ * You should also create a database named 'sqooptestdb' and authorize yourself:
+ *
+ * CREATE DATABASE sqooptestdb;
+ * use mysql;
+ * GRANT ALL PRIVILEGES ON sqooptestdb.* TO 'yourusername'@'localhost';
+ * flush privileges;
+ *
+ */
+public class DirectMySQLTest extends ImportJobTestCase {
+
+  public static final Log LOG = LogFactory.getLog(
+      DirectMySQLTest.class.getName());
+
+  static final String TABLE_PREFIX = "EMPLOYEES_MYSQL_";
+
+  // instance variables populated during setUp, used during tests
+  private DirectMySQLManager manager;
+  private MySQLTestUtils mySQLTestUtils = new MySQLTestUtils();
+
+  @Override
+  protected String getTablePrefix() {
+    return TABLE_PREFIX;
+  }
+
+  @Before
+  public void setUp() {
+    super.setUp();
+
+    SqoopOptions options = new SqoopOptions(mySQLTestUtils.getMySqlConnectString(),
+        getTableName());
+    options.setUsername(mySQLTestUtils.getUserName());
+    mySQLTestUtils.addPasswordIfIsSet(options);
+
+    LOG.debug("Setting up another DirectMySQLTest: "
+        + mySQLTestUtils.getMySqlConnectString());
+
+    manager = new DirectMySQLManager(options);
+
+    Connection connection = null;
+    Statement st = null;
+
+    try {
+      connection = manager.getConnection();
+      connection.setAutoCommit(false);
+      st = connection.createStatement();
+
+      // create the database table and populate it with data.
+      st.executeUpdate("DROP TABLE IF EXISTS " + getTableName());
+      st.executeUpdate("CREATE TABLE " + getTableName() + " ("
+          + "id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, "
+          + "name VARCHAR(24) NOT NULL, "
+          + "overly_large_number INT UNSIGNED,"
+          + "start_date DATE, "
+          + "salary FLOAT, "
+          + "dept VARCHAR(32))");
+
+      st.executeUpdate("INSERT INTO " + getTableName() + " VALUES("
+          + "NULL,'Aaron',0,'2009-05-14',1000000.00,'engineering')");
+      st.executeUpdate("INSERT INTO " + getTableName() + " VALUES("
+          + "NULL,'Bob',100,'2009-04-20',400.00,'sales')");
+      st.executeUpdate("INSERT INTO " + getTableName() + " VALUES("
+          + "NULL,'Fred',4000000000,'2009-01-23',15.00,'marketing')");
+      connection.commit();
+    } catch (SQLException sqlE) {
+      LOG.error("Encountered SQL Exception: " + sqlE);
+      sqlE.printStackTrace();
+      fail("SQLException when running test setUp(): " + sqlE);
+    }
+  }
+
+  @After
+  public void tearDown() {
+    try {
+      Statement stmt = manager.getConnection().createStatement();
+      stmt.execute("DROP TABLE " + getTableName());
+    } catch(SQLException e) {
+      LOG.error("Can't clean up the database:", e);
+    }
+
+    super.tearDown();
+  }
+
+  private String [] getArgv(boolean mysqlOutputDelims, boolean isDirect,
+      String tableName, String... extraArgs) {
+    ArrayList<String> args = new ArrayList<String>();
+
+    CommonArgs.addHadoopFlags(args);
+
+    args.add("--table");
+    args.add(tableName);
+    args.add("--warehouse-dir");
+    args.add(getWarehouseDir());
+    args.add("--connect");
+    args.add(mySQLTestUtils.getMySqlConnectString());
+    if (isDirect) {
+      args.add("--direct");
+    }
+    args.add("--username");
+    args.add(mySQLTestUtils.getUserName());
+    mySQLTestUtils.addPasswordIfIsSet(args);
+    args.add("--where");
+    args.add("id > 1");
+    args.add("--num-mappers");
+    args.add("1");
+
+    if (mysqlOutputDelims) {
+      args.add("--mysql-delimiters");
+    }
+
+    if (null != extraArgs) {
+      for (String arg : extraArgs) {
+        args.add(arg);
+      }
+    }
+
+    return args.toArray(new String[0]);
+  }
+
+  private void doImport(boolean mysqlOutputDelims, boolean isDirect,
+      String tableName, String [] expectedResults, String [] extraArgs)
+      throws IOException {
+
+    Path warehousePath = new Path(this.getWarehouseDir());
+    Path tablePath = new Path(warehousePath, tableName);
+
+    Path filePath = new Path(tablePath, "part-m-00000");
+
+    File tableFile = new File(tablePath.toString());
+    if (tableFile.exists() && tableFile.isDirectory()) {
+      // remove the directory before running the import.
+      FileListing.recursiveDeleteDir(tableFile);
+    }
+
+    String [] argv = getArgv(mysqlOutputDelims, isDirect, tableName, extraArgs);
+    runImport(argv);
+
+    File f = new File(filePath.toString());
+    assertTrue("Could not find imported data file: " + f, f.exists());
+    BufferedReader r = null;
+    try {
+      // Read through the file and make sure it's all there.
+      r = new BufferedReader(new InputStreamReader(new FileInputStream(f)));
+      for (String expectedLine : expectedResults) {
+        assertEquals(expectedLine, r.readLine());
+      }
+    } catch (IOException ioe) {
+      LOG.error("Got IOException verifying results: " + ioe.toString());
+      ioe.printStackTrace();
+      fail(ioe.toString());
+    } finally {
+      IOUtils.closeStream(r);
+    }
+  }
+
+  @Test
+  public void testDirectBulkImportWithDefaultDelims() throws IOException {
+    // no quoting of strings allowed.
+    String [] expectedResults = {
+      "2,Bob,100,2009-04-20,400,sales",
+      "3,Fred,4000000000,2009-01-23,15,marketing",
+    };
+
+    doImport(false, true, getTableName(), expectedResults, null);
+  }
+
+  @Test
+  public void testWithExtraParams() throws IOException {
+    // no quoting of strings allowed.
+    String [] expectedResults = {
+      "2,Bob,100,2009-04-20,400,sales",
+      "3,Fred,4000000000,2009-01-23,15,marketing",
+    };
+
+    String [] extraArgs = { "--", "--lock-tables" };
+
+    doImport(false, true, getTableName(), expectedResults, extraArgs);
+  }
+
+  @Test
+  public void testMultiMappers() throws IOException {
+    // no quoting of strings allowed.
+    String [] expectedResults = {
+      "2,Bob,100,2009-04-20,400,sales",
+      "3,Fred,4000000000,2009-01-23,15,marketing",
+    };
+
+    String [] extraArgs = { "-m", "2" };
+
+    doImport(false, true, getTableName(), expectedResults, extraArgs);
+  }
+
+  @Test
+  public void testJdbcColumnSubset() throws IOException {
+    // Test that column subsets work in JDBC mode.
+    LOG.info("Starting JDBC Column Subset test.");
+
+    String [] expectedResults = {
+      "2,Bob,400.0",
+      "3,Fred,15.0",
+    };
+
+    String [] extraArgs = { "--columns", "id,name,salary" };
+    doImport(false, false, getTableName(), expectedResults, extraArgs);
+  }
+
+  @Test
+  public void testDirectColumnSubset() throws IOException {
+    // Using a column subset should actually force direct mode off, but this
+    // should just warn the user and do a normal import.
+    LOG.info("Starting Direct Column Subset test.");
+
+    String [] expectedResults = {
+      "2,Bob,400.0",
+      "3,Fred,15.0",
+    };
+
+    String [] extraArgs = { "--columns", "id,name,salary" };
+    doImport(false, true, getTableName(), expectedResults, extraArgs);
+  }
+
+  @Test
+  public void testDirectBulkImportWithMySQLQuotes() throws IOException {
+    // mysql quotes all string-based output.
+    String [] expectedResults = {
+      "2,'Bob',100,'2009-04-20',400,'sales'",
+      "3,'Fred',4000000000,'2009-01-23',15,'marketing'",
+    };
+
+    doImport(true, true, getTableName(), expectedResults, null);
+  }
+
+  @Test
+  public void testMySQLJdbcImport() throws IOException {
+    String [] expectedResults = {
+      "2,Bob,100,2009-04-20,400.0,sales",
+      "3,Fred,4000000000,2009-01-23,15.0,marketing",
+    };
+
+    doImport(false, false, getTableName(), expectedResults, null);
+  }
+
+  @Test
+  public void testJdbcEscapedTableName() throws Exception {
+    // Test a JDBC-based import of a table whose name is
+    // a reserved sql keyword (and is thus `quoted`)
+    final String RESERVED_TABLE_NAME = "TABLE";
+    SqoopOptions options = new SqoopOptions(mySQLTestUtils.getMySqlConnectString(),
+        RESERVED_TABLE_NAME);
+    options.setUsername(mySQLTestUtils.getUserName());
+    mySQLTestUtils.addPasswordIfIsSet(options);
+    ConnManager mgr = new MySQLManager(options);
+
+    Connection connection = null;
+    Statement st = null;
+
+    try {
+      connection = mgr.getConnection();
+      connection.setAutoCommit(false);
+      st = connection.createStatement();
+
+      // create the database table and populate it with data.
+      st.executeUpdate("DROP TABLE IF EXISTS `" + RESERVED_TABLE_NAME + "`");
+      st.executeUpdate("CREATE TABLE `" + RESERVED_TABLE_NAME + "` ("
+          + "id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, "
+          + "name VARCHAR(24) NOT NULL, "
+          + "start_date DATE, "
+          + "salary FLOAT, "
+          + "dept VARCHAR(32))");
+
+      st.executeUpdate("INSERT INTO `" + RESERVED_TABLE_NAME + "` VALUES("
+          + "2,'Aaron','2009-05-14',1000000.00,'engineering')");
+      st.close();
+      connection.commit();
+
+      String [] expectedResults = {
+          "2,Aaron,2009-05-14,1000000.0,engineering",
+      };
+
+      doImport(false, false, RESERVED_TABLE_NAME, expectedResults, null);
+
+      st = connection.createStatement();
+      st.execute("DROP TABLE `" + RESERVED_TABLE_NAME + "`");
+    } finally {
+      if (null != st) {
+        st.close();
+      }
+
+      if (null != connection) {
+        connection.close();
+      }
+    }
+
+  }
+
+  @Test(expected = IOException.class)
+  public void testSqoopNullStringValueFailsValidate() throws Exception {
+    String [] expectedResults =  {};
+    String [] extraArgs =  {"--null-string", "abc"};
+
+    doImport(false, true, getTableName(), expectedResults, extraArgs);
+  }
+
+  @Test(expected = IOException.class)
+  public void testSqoopNullNonStringValueFailsValidate() throws Exception {
+    String [] expectedResults =  {};
+    String [] extraArgs =  {"--null-non-string", "abc"};
+
+    doImport(false, true, getTableName(), expectedResults, extraArgs);
+  }
+
+  @Test
+  public void testJdbcEscapedColumnName() throws Exception {
+    // Test a JDBC-based import of a table with a column whose name is
+    // a reserved sql keyword (and is thus `quoted`).
+    final String TABLE_NAME = "mysql_escaped_col_table";
+    SqoopOptions options = new SqoopOptions(mySQLTestUtils.getMySqlConnectString(),
+        TABLE_NAME);
+    options.setUsername(mySQLTestUtils.getUserName());
+    mySQLTestUtils.addPasswordIfIsSet(options);
+    ConnManager mgr = new MySQLManager(options);
+
+    Connection connection = null;
+    Statement st = null;
+
+    try {
+      connection = mgr.getConnection();
+      connection.setAutoCommit(false);
+      st = connection.createStatement();
+
+      // create the database table and populate it with data.
+      st.executeUpdate("DROP TABLE IF EXISTS " + TABLE_NAME);
+      st.executeUpdate("CREATE TABLE " + TABLE_NAME + " ("
+          + "id INT NOT NULL PRIMARY KEY AUTO_INCREMENT, "
+          + "`table` VARCHAR(24) NOT NULL, "
+          + "`CREATE` DATE, "
+          + "salary FLOAT, "
+          + "dept VARCHAR(32))");
+
+      st.executeUpdate("INSERT INTO " + TABLE_NAME + " VALUES("
+          + "2,'Aaron','2009-05-14',1000000.00,'engineering')");
+      st.close();
+      connection.commit();
+
+      String [] expectedResults = {
+          "2,Aaron,2009-05-14,1000000.0,engineering",
+      };
+
+      doImport(false, false, TABLE_NAME, expectedResults, null);
+
+      st = connection.createStatement();
+      st.execute("DROP TABLE " + TABLE_NAME);
+    } finally {
+      if (null != st) {
+        st.close();
+      }
+
+      if (null != connection) {
+        connection.close();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6984a36c/src/test/org/apache/sqoop/manager/mysql/JdbcMySQLExportTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/mysql/JdbcMySQLExportTest.java b/src/test/org/apache/sqoop/manager/mysql/JdbcMySQLExportTest.java
new file mode 100644
index 0000000..f655bcc
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/mysql/JdbcMySQLExportTest.java
@@ -0,0 +1,187 @@
+/**
+ * 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.manager.mysql;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.sqoop.manager.MySQLManager;
+import org.junit.After;
+import org.junit.Before;
+
+import org.apache.sqoop.SqoopOptions;
+import org.apache.sqoop.TestExport;
+import org.junit.Test;
+
+import static org.junit.Assert.fail;
+
+/**
+ * Test the MySQLManager implementation's exportJob() functionality.
+ * This does a better test of ExportOutputFormat than TestExport does,
+ * because it supports multi-row INSERT statements.
+ */
+public class JdbcMySQLExportTest extends TestExport {
+
+  public static final Log LOG = LogFactory.getLog(
+      JdbcMySQLExportTest.class.getName());
+
+  static final String TABLE_PREFIX = "EXPORT_MYSQL_J_";
+
+  // instance variables populated during setUp, used during tests.
+  private MySQLManager manager;
+  private Connection conn;
+  private MySQLTestUtils mySqlTestUtils = new MySQLTestUtils();
+
+  @Override
+  protected Connection getConnection() {
+    return conn;
+  }
+
+  // MySQL allows multi-row INSERT statements.
+  @Override
+  protected int getMaxRowsPerStatement() {
+    return 1000;
+  }
+
+  @Override
+  protected boolean useHsqldbTestServer() {
+    return false;
+  }
+
+  @Override
+  protected String getConnectString() {
+    return mySqlTestUtils.getMySqlConnectString();
+  }
+
+  @Override
+  protected String getTablePrefix() {
+    return TABLE_PREFIX;
+  }
+
+  @Override
+  protected String getDropTableStatement(String tableName) {
+    return "DROP TABLE IF EXISTS " + tableName;
+  }
+
+  @Before
+  public void setUp() {
+    super.setUp();
+
+    SqoopOptions options = new SqoopOptions(mySqlTestUtils.getMySqlConnectString(),
+        getTableName());
+    options.setUsername(mySqlTestUtils.getUserName());
+    mySqlTestUtils.addPasswordIfIsSet(options);
+    this.manager = new MySQLManager(options);
+    try {
+      this.conn = manager.getConnection();
+      this.conn.setAutoCommit(false);
+    } catch (SQLException sqlE) {
+      LOG.error(StringUtils.stringifyException(sqlE));
+      fail("Failed with sql exception in setup: " + sqlE);
+    }
+  }
+
+  @After
+  public void tearDown() {
+    try {
+      Statement stmt = conn.createStatement();
+      stmt.execute(getDropTableStatement(getTableName()));
+      stmt.execute(getDropTableStatement(getStagingTableName()));
+    } catch(SQLException e) {
+      LOG.error("Can't clean up the database:", e);
+    }
+
+    super.tearDown();
+
+    if (null != this.conn) {
+      try {
+        this.conn.close();
+      } catch (SQLException sqlE) {
+        LOG.error("Got SQLException closing conn: " + sqlE.toString());
+      }
+    }
+  }
+
+  @Override
+  protected String [] getCodeGenArgv(String... extraArgs) {
+    return super.getCodeGenArgv(mySqlTestUtils.addUserNameAndPasswordToArgs(extraArgs));
+  }
+
+  @Override
+  protected String [] getArgv(boolean includeHadoopFlags,
+      int rowsPerStatement, int statementsPerTx, String... additionalArgv) {
+
+    String [] subArgv = newStrArray(mySqlTestUtils.addUserNameAndPasswordToArgs(additionalArgv));
+    return super.getArgv(includeHadoopFlags, rowsPerStatement,
+        statementsPerTx, subArgv);
+  }
+
+  @Test
+  public void testIntColInBatchMode() throws IOException, SQLException {
+    final int TOTAL_RECORDS = 10;
+
+    // generate a column equivalent to rownum.
+    ColumnGenerator gen = new ColumnGenerator() {
+      public String getExportText(int rowNum) {
+        return "" + rowNum;
+      }
+      public String getVerifyText(int rowNum) {
+        return "" + rowNum;
+      }
+      public String getType() {
+        return "INTEGER";
+      }
+    };
+
+    createTextFile(0, TOTAL_RECORDS, false, gen);
+    createTable(gen);
+    runExport(getArgv(true, 10, 10, "--batch"));
+    verifyExport(TOTAL_RECORDS);
+    assertColMinAndMax(forIdx(0), gen);
+  }
+
+  @Test
+  public void testUpsert() throws IOException, SQLException {
+    final int TOTAL_RECORDS = 10;
+
+    createTextFile(0, TOTAL_RECORDS, false);
+    createTable();
+
+    // Insert only
+    runExport(getArgv(true, 10, 10, "--update-key", "id",
+      "--update-mode", "allowinsert"));
+    verifyExport(TOTAL_RECORDS);
+
+    // Update only
+    runExport(getArgv(true, 10, 10, "--update-key", "id",
+      "--update-mode", "allowinsert"));
+    verifyExport(TOTAL_RECORDS);
+
+    // Insert & update
+    createTextFile(0, TOTAL_RECORDS * 2, false);
+    runExport(getArgv(true, 10, 10, "--update-key", "id",
+      "--update-mode", "allowinsert"));
+    verifyExport(TOTAL_RECORDS * 2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6984a36c/src/test/org/apache/sqoop/manager/mysql/MySQLAllTablesTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/mysql/MySQLAllTablesTest.java b/src/test/org/apache/sqoop/manager/mysql/MySQLAllTablesTest.java
new file mode 100644
index 0000000..baf0e2a
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/mysql/MySQLAllTablesTest.java
@@ -0,0 +1,69 @@
+/**
+ * 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.manager.mysql;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+import org.apache.hadoop.conf.Configuration;
+
+import org.apache.sqoop.SqoopOptions;
+import org.apache.sqoop.TestAllTables;
+
+/**
+ * Test the --all-tables functionality with MySQL.
+ */
+public class MySQLAllTablesTest extends TestAllTables {
+
+  private MySQLTestUtils mySQLTestUtils = new MySQLTestUtils();
+
+  @Override
+  protected boolean useHsqldbTestServer() {
+    return false;
+  }
+
+  @Override
+  protected String getConnectString() {
+    return mySQLTestUtils.getMySqlConnectString();
+  }
+
+  @Override
+  protected SqoopOptions getSqoopOptions(Configuration conf) {
+    SqoopOptions opts = new SqoopOptions(conf);
+    opts.setUsername(mySQLTestUtils.getUserName());
+    mySQLTestUtils.addPasswordIfIsSet(opts);
+    return opts;
+  }
+
+  @Override
+  protected void dropTableIfExists(String table) throws SQLException {
+    Connection conn = getManager().getConnection();
+    PreparedStatement statement = conn.prepareStatement(
+        "DROP TABLE IF EXISTS " + table,
+        ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+    try {
+      statement.executeUpdate();
+      conn.commit();
+    } finally {
+      statement.close();
+    }
+  }
+}