You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sqoop.apache.org by ve...@apache.org on 2014/06/29 18:03:13 UTC

[2/7] SQOOP-1287: Add high performance Oracle connector into Sqoop (David Robson via Venkat Ranganathan)

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/SystemImportTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/SystemImportTest.java b/src/test/org/apache/sqoop/manager/oracle/SystemImportTest.java
new file mode 100644
index 0000000..0f85854
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/SystemImportTest.java
@@ -0,0 +1,315 @@
+/**
+ * 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.oracle;
+
+import static org.junit.Assert.*;
+
+import java.lang.reflect.Method;
+import java.math.BigDecimal;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.sql.Blob;
+import java.sql.Clob;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.sql.Struct;
+import java.util.Arrays;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.sqoop.manager.oracle.util.*;
+import org.junit.Test;
+
+import com.cloudera.sqoop.lib.BlobRef;
+import com.cloudera.sqoop.lib.ClobRef;
+import com.cloudera.sqoop.lib.SqoopRecord;
+import com.cloudera.sqoop.manager.OracleUtils;
+
+/**
+ * OraOop system tests of importing data from oracle to hadoop.
+ */
+public class SystemImportTest extends OraOopTestCase {
+
+  private static Class<?> preparedStatementClass;
+  private static Method methSetBinaryDouble;
+  private static Method methSetBinaryFloat;
+
+  static {
+    try {
+      preparedStatementClass =
+          Class.forName("oracle.jdbc.OraclePreparedStatement");
+      methSetBinaryDouble =
+          preparedStatementClass.getMethod("setBinaryDouble", int.class,
+              double.class);
+      methSetBinaryFloat =
+          preparedStatementClass.getMethod("setBinaryFloat", int.class,
+              float.class);
+    } catch (Exception e) {
+      throw new RuntimeException(
+          "Problem getting Oracle JDBC methods via reflection.", e);
+    }
+  }
+
+  /**
+   * Generates pseudo-random test data across all supported data types in an
+   * Oracle database. Imports the data into Hadoop and compares with the data in
+   * Oracle.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void importTest() throws Exception {
+    // Generate test data in oracle
+    setSqoopTargetDirectory(getSqoopTargetDirectory()
+        + OracleUtils.SYSTEMTEST_TABLE_NAME);
+    int numRows = OracleUtils.SYSTEMTEST_NUM_ROWS;
+    Connection conn = getTestEnvConnection();
+    OraOopOracleQueries.setConnectionTimeZone(conn, "GMT");
+    try {
+      Statement s = conn.createStatement();
+      try {
+        s.executeUpdate("CREATE TABLE "
+            + OracleUtils.SYSTEMTEST_TABLE_NAME
+            + " (id NUMBER(10) PRIMARY KEY, bd BINARY_DOUBLE, bf BINARY_FLOAT, "
+            + "b BLOB, c CHAR(12), cl CLOB, d DATE, "
+            + "f FLOAT(126), l LONG, nc NCHAR(30), ncl NCLOB, n NUMBER(9,2), "
+            + "nvc NVARCHAR2(30), r ROWID, u URITYPE, iym INTERVAL YEAR(2) TO "
+            + "MONTH, ids INTERVAL DAY(2) TO SECOND(6), "
+            + "t TIMESTAMP(6), tz TIMESTAMP(6) WITH TIME ZONE, "
+            + "tltz TIMESTAMP(6) WITH LOCAL TIME ZONE, rawcol RAW(21))");
+        BinaryDoubleGenerator bdg = new BinaryDoubleGenerator();
+        BinaryFloatGenerator bfg = new BinaryFloatGenerator();
+        BlobGenerator bg = new BlobGenerator(conn, 2 * 1024, 8 * 1024);
+        CharGenerator cg = new CharGenerator(12, 12);
+        CharGenerator clobg = new CharGenerator(2 * 1024, 8 * 1024);
+        TimestampGenerator dateg = new TimestampGenerator(0);
+        FloatGenerator fg = new FloatGenerator(126);
+        CharGenerator lg = new CharGenerator(2 * 1024, 8 * 1024);
+        NCharGenerator ncg = new NCharGenerator(30, 30);
+        NCharGenerator nclobg = new NCharGenerator(2 * 1024, 8 * 1024);
+        BigDecimalGenerator ng = new BigDecimalGenerator(9, 2);
+        NCharGenerator nvcg = new NCharGenerator(1, 30);
+        RowIdGenerator rg = new RowIdGenerator();
+        URIGenerator ug = new URIGenerator();
+        IntervalYearMonthGenerator iymg = new IntervalYearMonthGenerator(2);
+        IntervalDaySecondGenerator idsg = new IntervalDaySecondGenerator(2, 6);
+        TimestampGenerator tg = new TimestampGenerator(6);
+        TimestampGenerator tzg = new TimestampGenerator(6);
+        TimestampGenerator tltzg = new TimestampGenerator(6);
+        BytesGenerator rawg = new BytesGenerator(21, 21);
+        PreparedStatement ps =
+            conn.prepareStatement("INSERT INTO "
+              + OracleUtils.SYSTEMTEST_TABLE_NAME
+              + " ( id, bd, bf, b, c, cl, d, f, nc, ncl, n, nvc, r, u, iym, "
+              + "ids, t, tz, tltz, rawcol ) VALUES ( ?, ?, ?, ?, ?, ?, ?, ?, "
+              + "?, ?, ?, ?, ?, sys.UriFactory.getUri(?), ?, ?, ?, ?, ?, ? )");
+        try {
+          for (int i = 0; i < numRows; i++) {
+            ps.setInt(1, i);
+            methSetBinaryDouble.invoke(ps, 2, bdg.next());
+            methSetBinaryFloat.invoke(ps, 3, bfg.next());
+            ps.setBlob(4, bg.next());
+            ps.setString(5, cg.next());
+            ps.setString(6, clobg.next());
+            ps.setTimestamp(7, dateg.next());
+            ps.setBigDecimal(8, fg.next());
+            ps.setString(9, ncg.next());
+            ps.setString(10, nclobg.next());
+            ps.setBigDecimal(11, ng.next());
+            ps.setString(12, nvcg.next());
+            ps.setRowId(13, rg.next());
+            ps.setString(14, ug.next());
+            ps.setString(15, iymg.next());
+            ps.setString(16, idsg.next());
+            ps.setTimestamp(17, tg.next());
+            ps.setTimestamp(18, tzg.next());
+            ps.setTimestamp(19, tltzg.next());
+            ps.setBytes(20, rawg.next());
+            ps.executeUpdate();
+          }
+        } finally {
+          ps.close();
+        }
+
+        // Can't bind > 4000 bytes of data to LONG and LOB columns in the same
+        // statement, so do LONG by itself
+        ps =
+            conn.prepareStatement("UPDATE " + OracleUtils.SYSTEMTEST_TABLE_NAME
+                + " SET l = ? WHERE id = ?");
+        try {
+          for (int i = 0; i < numRows; i++) {
+            ps.setString(1, lg.next());
+            ps.setInt(2, i);
+            ps.executeUpdate();
+          }
+        } finally {
+          ps.close();
+        }
+
+        try {
+          // Import test data into hadoop
+
+          int retCode =
+            runImport(OracleUtils.SYSTEMTEST_TABLE_NAME, getSqoopConf(), true);
+          assertEquals("Return code should be 0", 0, retCode);
+
+          // Add sqoop generated code to the classpath
+          String sqoopGenJarPath =
+              "file://" + getSqoopGenLibDirectory() + "/"
+                  + getSqoopGenClassName() + ".jar";
+          URLClassLoader loader =
+              new URLClassLoader(new URL[] { new URL(sqoopGenJarPath) },
+                  getClass().getClassLoader());
+          Thread.currentThread().setContextClassLoader(loader);
+
+          // Read test data from hadoop
+          Configuration hadoopConf = getSqoopConf();
+          FileSystem hdfs = FileSystem.get(hadoopConf);
+          Path path = new Path(getSqoopTargetDirectory());
+          FileStatus[] statuses = hdfs.listStatus(path);
+          int hadoopRecordCount = 0;
+          for (FileStatus status : statuses) {
+            if (status.getPath().getName().startsWith("part-m-")) {
+
+              SequenceFile.Reader reader =
+                  new SequenceFile.Reader(hdfs, status.getPath(), hadoopConf);
+              LongWritable key = new LongWritable();
+              @SuppressWarnings("unchecked")
+              SqoopRecord value =
+                  ((Class<SqoopRecord>) reader.getValueClass())
+                      .getConstructor().newInstance();
+              ps =
+                  conn.prepareStatement("SELECT bd, bf, b, c, cl, d, f, l, nc, "
+                      + "ncl, nvc, r, u, iym, ids, t, tz, tltz, rawcol FROM "
+                      + OracleUtils.SYSTEMTEST_TABLE_NAME + " WHERE id = ?");
+              while (reader.next(key, value)) {
+                // Compare test data from hadoop with data in oracle
+                Map<String, Object> fields = value.getFieldMap();
+                BigDecimal id = (BigDecimal) fields.get("ID");
+                ps.setBigDecimal(1, id);
+                ResultSet rs = ps.executeQuery();
+                assertTrue("Did not find row with id " + id + " in oracle", rs
+                    .next());
+                assertEquals("BinaryDouble did not match for row " + id, fields
+                    .get("BD"), rs.getDouble(1));
+                assertEquals("BinaryFloat did not match for row " + id, fields
+                    .get("BF"), rs.getFloat(2));
+                // LONG column needs to be read before BLOB column
+                assertEquals("Long did not match for row " + id, fields
+                    .get("L"), rs.getString(8));
+                BlobRef hadoopBlob = (BlobRef) fields.get("B");
+                Blob oraBlob = rs.getBlob(3);
+                assertTrue("Blob did not match for row " + id, Arrays.equals(
+                    hadoopBlob.getData(), oraBlob.getBytes(1L, (int) oraBlob
+                        .length())));
+                assertEquals("Char did not match for row " + id, fields
+                    .get("C"), rs.getString(4));
+                ClobRef hadoopClob = (ClobRef) fields.get("CL");
+                Clob oraClob = rs.getClob(5);
+                assertEquals("Clob did not match for row " + id, hadoopClob
+                  .getData(), oraClob.getSubString(1, (int) oraClob.length()));
+                assertEquals("Date did not match for row " + id, fields
+                    .get("D"), rs.getString(6));
+                BigDecimal hadoopFloat = (BigDecimal) fields.get("F");
+                BigDecimal oraFloat = rs.getBigDecimal(7);
+                assertEquals("Float did not match for row " + id, hadoopFloat,
+                    oraFloat);
+                assertEquals("NChar did not match for row " + id, fields
+                    .get("NC"), rs.getString(9));
+                assertEquals("NClob did not match for row " + id, fields
+                    .get("NCL"), rs.getString(10));
+                assertEquals("NVarChar did not match for row " + id, fields
+                    .get("NVC"), rs.getString(11));
+                assertEquals("RowId did not match for row " + id, fields
+                    .get("R"), new String(rs.getRowId(12).getBytes()));
+                Struct url = (Struct) rs.getObject(13); // TODO: Find a fix for
+                                                        // this workaround
+                String urlString = (String) url.getAttributes()[0];
+                if (url.getSQLTypeName().equals("SYS.HTTPURITYPE")) {
+                  urlString = "http://" + urlString;
+                } else if (url.getSQLTypeName().equals("SYS.DBURITYPE")) {
+                  urlString = "/ORADB" + urlString;
+                }
+                assertEquals("UriType did not match for row " + id, fields
+                    .get("U"), urlString);
+                assertEquals("Interval Year to Month did not match for row "
+                    + id, fields.get("IYM"), rs.getString(14));
+                String ids = (String) fields.get("IDS"); // Strip trailing zeros
+                                                         // to match oracle
+                                                         // format
+                int lastNonZero = ids.length() - 1;
+                while (ids.charAt(lastNonZero) == '0') {
+                  lastNonZero--;
+                }
+                ids = ids.substring(0, lastNonZero + 1);
+                assertEquals("Interval Day to Second did not match for row "
+                    + id, ids, rs.getString(15));
+                assertEquals("Timestamp did not match for row " + id, fields
+                    .get("T"), rs.getString(16));
+                assertEquals("Timestamp with Time Zone did not match for row "
+                    + id, fields.get("TZ"), rs.getString(17));
+                assertEquals(
+                    "Timestamp with Local Time Zone did not match for row "
+                        + id, fields.get("TLTZ"), rs.getString(18));
+                BytesWritable rawCol = (BytesWritable) fields.get("RAWCOL");
+                byte[] rawColData =
+                    Arrays.copyOf(rawCol.getBytes(), rawCol.getLength());
+                assertTrue("RAW did not match for row " + id, Arrays.equals(
+                    rawColData, rs.getBytes(19)));
+
+                assertFalse("Found multiple rows with id " + id + " in oracle",
+                    rs.next());
+                hadoopRecordCount++;
+              }
+              reader.close();
+            }
+          }
+          ResultSet rs =
+              s.executeQuery("SELECT COUNT(*) FROM "
+                  + OracleUtils.SYSTEMTEST_TABLE_NAME);
+          rs.next();
+          int oracleRecordCount = rs.getInt(1);
+          assertEquals(
+              "Number of records in Hadoop does not match number of "
+              + "records in oracle",
+              hadoopRecordCount, oracleRecordCount);
+          rs.close();
+        } finally {
+          // Delete test data from hadoop
+          cleanupFolders();
+        }
+      } finally {
+        // Delete test data from oracle
+        s.executeUpdate("DROP TABLE " + OracleUtils.SYSTEMTEST_TABLE_NAME);
+        s.close();
+      }
+
+    } finally {
+      closeTestEnvConnection();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/TestOraOopDataDrivenDBInputFormat.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/TestOraOopDataDrivenDBInputFormat.java b/src/test/org/apache/sqoop/manager/oracle/TestOraOopDataDrivenDBInputFormat.java
new file mode 100644
index 0000000..7d3abfd
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/TestOraOopDataDrivenDBInputFormat.java
@@ -0,0 +1,131 @@
+/**
+ * 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.oracle;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.cloudera.sqoop.lib.SqoopRecord;
+
+import org.apache.sqoop.manager.oracle.OraOopConstants.
+           OraOopOracleBlockToSplitAllocationMethod;
+
+/**
+ * Unit tests for OraOopDataDrivenDBInputFormat.
+ */
+public class TestOraOopDataDrivenDBInputFormat extends OraOopTestCase {
+
+  /**
+   * We're just exposing a protected method so that it can be called by this
+   * unit test...
+   */
+  public class Exposer<T extends SqoopRecord> extends
+      OraOopDataDrivenDBInputFormat<T> {
+
+    @Override
+    public
+        List<InputSplit>
+        groupTableDataChunksIntoSplits(
+            List<? extends OraOopOracleDataChunk> dataChunks,
+            int desiredNumberOfSplits,
+            OraOopConstants.OraOopOracleBlockToSplitAllocationMethod
+                blockAllocationMethod) {
+
+      return super.groupTableDataChunksIntoSplits(dataChunks,
+          desiredNumberOfSplits, blockAllocationMethod);
+    }
+
+  }
+
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testgroupTableDataChunksIntoSplits() {
+
+    List<OraOopOracleDataChunk> dataChunks =
+        new ArrayList<OraOopOracleDataChunk>();
+
+    int startBlockNumber = 1;
+    for (int idx = 0; idx < 241; idx++) {
+      OraOopOracleDataChunk dataChunk =
+          new OraOopOracleDataChunkExtent("23480", 666, 1, startBlockNumber,
+              startBlockNumber + 8);
+      startBlockNumber += 8;
+      dataChunks.add(dataChunk);
+    }
+
+    @SuppressWarnings("rawtypes")
+    Exposer e = new Exposer();
+
+    // Prevent setJdbcFetchSize() from logging information about the fetch-size
+    // changing. Otherwise, the junit output will be polluted with messages
+    // about
+    // things that aren't actually a problem...
+    boolean logIsBeingCached = Exposer.LOG.getCacheLogEntries();
+    Exposer.LOG.setCacheLogEntries(true);
+
+    List<InputSplit> splits =
+        e.groupTableDataChunksIntoSplits(dataChunks, 32,
+            OraOopOracleBlockToSplitAllocationMethod.SEQUENTIAL);
+
+    Exposer.LOG.setCacheLogEntries(logIsBeingCached);
+
+    int highestNumberOfDataChunksAllocatedToASplit = 0;
+    int lowestNumberOfDataChunksAllocatedToASplit = Integer.MAX_VALUE;
+
+    // Check that all splits have data-chunks assigned to them...
+    for (InputSplit split : splits) {
+      int dataChunksAllocatedToThisSplit =
+          ((OraOopDBInputSplit) split).getNumberOfDataChunks();
+      highestNumberOfDataChunksAllocatedToASplit =
+          Math.max(highestNumberOfDataChunksAllocatedToASplit,
+              dataChunksAllocatedToThisSplit);
+      lowestNumberOfDataChunksAllocatedToASplit =
+          Math.min(lowestNumberOfDataChunksAllocatedToASplit,
+              dataChunksAllocatedToThisSplit);
+    }
+
+    if (lowestNumberOfDataChunksAllocatedToASplit == 0) {
+      Assert
+          .fail("There is a split that has not had any "
+              + "data-chunks allocated to it.");
+    }
+
+    // Check that the split with the least data-chunks has at least
+    // 75% of the number of data-chunks of the split with the most
+    // data-chunks...
+    double minExpectedWorkloadRatio = 0.75;
+    double actualWorkloadRatio =
+        (double) lowestNumberOfDataChunksAllocatedToASplit
+            / highestNumberOfDataChunksAllocatedToASplit;
+    if (actualWorkloadRatio < minExpectedWorkloadRatio) {
+      Assert.fail(String.format(
+          "There is too much difference in the amount of work assigned "
+              + "to the 'smallest' split and the 'largest' split. "
+              + "The split with the least work should have at least %s "
+              + "of the workload of the 'largest' split, but it actually "
+              + "only has %s of the workload of the 'largest' split.",
+          minExpectedWorkloadRatio, actualWorkloadRatio));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/TestOraOopJdbcUrl.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/TestOraOopJdbcUrl.java b/src/test/org/apache/sqoop/manager/oracle/TestOraOopJdbcUrl.java
new file mode 100644
index 0000000..2cd77e6
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/TestOraOopJdbcUrl.java
@@ -0,0 +1,276 @@
+/**
+ * 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.oracle;
+
+import static org.junit.Assert.*;
+import junit.framework.Assert;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.sqoop.manager.oracle.OraOopUtilities.
+           JdbcOracleThinConnectionParsingError;
+
+/**
+ * Unit tests for OraOopJdbcUrl.
+ */
+public class TestOraOopJdbcUrl extends OraOopTestCase {
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+
+  }
+
+  @Before
+  public void setUp() throws Exception {
+
+  }
+
+  @After
+  public void tearDown() throws Exception {
+
+  }
+
+  @Test
+  public void testParseJdbcOracleThinConnectionString() {
+
+    OraOopUtilities.JdbcOracleThinConnection actual;
+
+    // Null JDBC URL...
+    try {
+      actual = new OraOopJdbcUrl(null).parseJdbcOracleThinConnectionString();
+      Assert.fail("An IllegalArgumentException should be been thrown.");
+    } catch (IllegalArgumentException ex) {
+      /* This is what we want to happen. */
+    } catch (JdbcOracleThinConnectionParsingError ex) {
+      Assert.fail("An IllegalArgumentException should be been thrown.");
+    }
+
+    // Empty JDBC URL...
+    try {
+      actual = new OraOopJdbcUrl("").parseJdbcOracleThinConnectionString();
+      Assert.fail("An IllegalArgumentException should be been thrown.");
+    } catch (IllegalArgumentException ex) {
+      /* This is what we want to happen. */
+    } catch (JdbcOracleThinConnectionParsingError ex) {
+      Assert.fail("An IllegalArgumentException should be been thrown.");
+    }
+
+    // Incorrect number of fragments in the URL...
+    try {
+      actual =
+          new OraOopJdbcUrl("jdbc:oracle:oci8:@dbname.domain")
+              .parseJdbcOracleThinConnectionString();
+      Assert.fail(
+          "A JdbcOracleThinConnectionParsingError should be been thrown.");
+    } catch (JdbcOracleThinConnectionParsingError ex) {
+      // This is what we want to happen.
+      assertTrue(
+          "An exception should be thown that tells us there's an incorrect "
+          + "number of fragments in the JDBC URL.",
+          ex.getMessage()
+              .toLowerCase()
+              .contains(
+                  "there should be 5 or 6 colon-separated pieces of data in "
+                  + "the jdbc url"));
+    }
+
+    // Incorrect driver-type (i.e. not using the "thin" driver)...
+    try {
+      actual =
+          new OraOopJdbcUrl(
+              "jdbc:oracle:loremipsum:@hostname.domain.com.au:port1521:dbsid")
+              .parseJdbcOracleThinConnectionString();
+      Assert.fail(
+          "A JdbcOracleThinConnectionParsingError should be been thrown.");
+    } catch (JdbcOracleThinConnectionParsingError ex) {
+      // This is what we want to happen.
+      assertTrue(
+          "An exception should be thown that refers to the fact that the thin "
+          + "JDBC driver is not being used.",
+          ex.getMessage().toLowerCase().contains(
+              "oracle \"thin\" jdbc driver is not being used"));
+
+      assertTrue(
+          "An exception should be thown that tells us which JDBC driver "
+          + "was specified.",
+          ex.getMessage().toLowerCase().contains("loremipsum"));
+
+    }
+
+    // Invalid JDBC URL (unparsable port number)...
+    try {
+      actual =
+          new OraOopJdbcUrl(
+              "jdbc:oracle:thin:@hostname.domain.com.au:port1521:dbsid")
+              .parseJdbcOracleThinConnectionString();
+      Assert.fail(
+          "An JdbcOracleThinConnectionParsingError should be been thrown.");
+    } catch (JdbcOracleThinConnectionParsingError ex) {
+      assertTrue(
+         "The invalid port number should be included in the exception message.",
+         ex.getMessage().toLowerCase().contains("port1521"));
+    }
+
+    // Invalid JDBC URL (negative port number)...
+    try {
+      actual =
+          new OraOopJdbcUrl(
+              "jdbc:oracle:thin:@hostname.domain.com.au:-1521:dbsid")
+              .parseJdbcOracleThinConnectionString();
+      Assert.fail(
+          "An JdbcOracleThinConnectionParsingError should be been thrown.");
+    } catch (JdbcOracleThinConnectionParsingError ex) {
+      assertTrue(
+         "The invalid port number should be included in the exception message.",
+         ex.getMessage().toLowerCase().contains("-1521"));
+    }
+
+    // Valid JDBC URL...
+    try {
+      actual =
+          new OraOopJdbcUrl(
+              "JDBC:Oracle:tHiN:@hostname.domain.com.au:1521:dbsid")
+              .parseJdbcOracleThinConnectionString();
+      Assert.assertEquals("hostname.domain.com.au", actual.getHost());
+      Assert.assertEquals(1521, actual.getPort());
+      Assert.assertEquals("dbsid", actual.getSid());
+    } catch (JdbcOracleThinConnectionParsingError ex) {
+      Assert.fail(ex.getMessage());
+    }
+
+    // Valid JDBC URL...
+    try {
+      actual =
+          new OraOopJdbcUrl(
+              " JDBC : Oracle : tHiN : @hostname.domain.com.au : 1529 : dbsid")
+              .parseJdbcOracleThinConnectionString();
+      Assert.assertEquals("hostname.domain.com.au", actual.getHost());
+      Assert.assertEquals(1529, actual.getPort());
+      Assert.assertEquals("dbsid", actual.getSid());
+    } catch (JdbcOracleThinConnectionParsingError ex) {
+      Assert.fail(ex.getMessage());
+    }
+
+    // Valid (sid-based) JDBC URL with parameters...
+    try {
+      actual =
+          new OraOopJdbcUrl(
+              "jdbc:oracle:thin:@hostname:1521:dbsid?param1=loremipsum")
+              .parseJdbcOracleThinConnectionString();
+      Assert.assertEquals("hostname", actual.getHost());
+      Assert.assertEquals(1521, actual.getPort());
+      Assert.assertEquals("dbsid", actual.getSid());
+      Assert.assertEquals(null, actual.getService());
+    } catch (JdbcOracleThinConnectionParsingError ex) {
+      Assert.fail(ex.getMessage());
+    }
+
+    // Valid (service-based) JDBC URL...
+    try {
+      actual =
+          new OraOopJdbcUrl(
+              "jdbc:oracle:thin:@hostname:1521/dbservice.dbdomain")
+              .parseJdbcOracleThinConnectionString();
+      Assert.assertEquals("hostname", actual.getHost());
+      Assert.assertEquals(1521, actual.getPort());
+      Assert.assertEquals(null, actual.getSid());
+      Assert.assertEquals("dbservice.dbdomain", actual.getService());
+    } catch (JdbcOracleThinConnectionParsingError ex) {
+      Assert.fail(ex.getMessage());
+    }
+
+    // Valid (service-based) JDBC URL with slashes...
+    try {
+      actual =
+          new OraOopJdbcUrl(
+              "jdbc:oracle:thin:@//hostname:1521/dbservice.dbdomain")
+              .parseJdbcOracleThinConnectionString();
+      Assert.assertEquals("hostname", actual.getHost());
+      Assert.assertEquals(1521, actual.getPort());
+      Assert.assertEquals(null, actual.getSid());
+      Assert.assertEquals("dbservice.dbdomain", actual.getService());
+    } catch (JdbcOracleThinConnectionParsingError ex) {
+      Assert.fail(ex.getMessage());
+    }
+
+    // Valid (service-based) JDBC URL with parameters...
+    try {
+      actual = new OraOopJdbcUrl(
+         "jdbc:oracle:thin:@hostname:1521/dbservice.dbdomain?param1=loremipsum")
+              .parseJdbcOracleThinConnectionString();
+      Assert.assertEquals("hostname", actual.getHost());
+      Assert.assertEquals(1521, actual.getPort());
+      Assert.assertEquals(null, actual.getSid());
+      Assert.assertEquals("dbservice.dbdomain", actual.getService());
+    } catch (JdbcOracleThinConnectionParsingError ex) {
+      Assert.fail(ex.getMessage());
+    }
+
+    // Valid (service-based) JDBC URL with slashes and parameters...
+    try {
+      actual = new OraOopJdbcUrl(
+       "jdbc:oracle:thin:@//hostname:1521/dbservice.dbdomain?param1=loremipsum")
+              .parseJdbcOracleThinConnectionString();
+      Assert.assertEquals("hostname", actual.getHost());
+      Assert.assertEquals(1521, actual.getPort());
+      Assert.assertEquals(null, actual.getSid());
+      Assert.assertEquals("dbservice.dbdomain", actual.getService());
+    } catch (JdbcOracleThinConnectionParsingError ex) {
+      Assert.fail(ex.getMessage());
+    }
+  }
+
+  @Test
+  public void testGetConnectionUrl() {
+
+    String actual;
+
+    // Null JDBC URL...
+    try {
+      actual = new OraOopJdbcUrl(null).getConnectionUrl();
+      Assert.fail("An IllegalArgumentException should be been thrown.");
+    } catch (IllegalArgumentException ex) {
+      /* This is what we want to happen. */
+    }
+
+    // Empty JDBC URL...
+    try {
+      actual = new OraOopJdbcUrl("").getConnectionUrl();
+      Assert.fail("An IllegalArgumentException should be been thrown.");
+    } catch (IllegalArgumentException ex) {
+      /* This is what we want to happen. */
+    }
+
+    // JDBC URL...
+    actual =
+        new OraOopJdbcUrl("jdbc:oracle:thin:@hostname.domain:1521:dbsid")
+            .getConnectionUrl();
+    Assert.assertEquals("jdbc:oracle:thin:@hostname.domain:1521:dbsid", actual);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/TestOraOopUtilities.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/TestOraOopUtilities.java b/src/test/org/apache/sqoop/manager/oracle/TestOraOopUtilities.java
new file mode 100644
index 0000000..93592af
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/TestOraOopUtilities.java
@@ -0,0 +1,619 @@
+/**
+ * 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.oracle;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+/**
+ * Unit tests for OraOopUtilities.
+ */
+public class TestOraOopUtilities extends OraOopTestCase {
+
+  @Test
+  public void testdecodeOracleTableName() {
+
+    OracleTable context = null;
+
+    // These are the possibilities for double-quote location...
+    // table
+    // "table"
+    // schema.table
+    // schema."table"
+    // "schema".table
+    // "schema"."table"
+
+    // table
+    context = OraOopUtilities.decodeOracleTableName("oraoop", "junk", null);
+    Assert.assertEquals(context.getSchema(), "ORAOOP");
+    Assert.assertEquals(context.getName(), "JUNK");
+
+    // "table"
+    context = OraOopUtilities.decodeOracleTableName("oraoop", "\"Junk\"", null);
+    Assert.assertEquals(context.getSchema(), "ORAOOP");
+    Assert.assertEquals(context.getName(), "Junk");
+
+    // schema.table
+    context =
+        OraOopUtilities.decodeOracleTableName("oraoop", "targusr.junk", null);
+    Assert.assertEquals(context.getSchema(), "TARGUSR");
+    Assert.assertEquals(context.getName(), "JUNK");
+
+    // schema."table"
+    context =
+        OraOopUtilities.decodeOracleTableName("oraoop", "targusr.\"Junk\"",
+            null);
+    Assert.assertEquals(context.getSchema(), "TARGUSR");
+    Assert.assertEquals(context.getName(), "Junk");
+
+    // "schema".table
+    context =
+        OraOopUtilities.decodeOracleTableName("oraoop", "\"Targusr\".junk",
+            null);
+    Assert.assertEquals(context.getSchema(), "Targusr");
+    Assert.assertEquals(context.getName(), "JUNK");
+
+    // "schema"."table"
+    String inputStr = "\"Targusr\".\"Junk\"";
+    context = OraOopUtilities.decodeOracleTableName("oraoop", inputStr, null);
+    Assert.assertEquals(context.getSchema(), "Targusr");
+    Assert.assertEquals(context.getName(), "Junk");
+
+    // Test for "." within schema...
+    context =
+        OraOopUtilities.decodeOracleTableName("oraoop", "\"targ.usr\".junk",
+            null);
+    Assert.assertEquals(context.getSchema(), "targ.usr");
+    Assert.assertEquals(context.getName(), "JUNK");
+
+    // Test for "." within table...
+    context =
+        OraOopUtilities.decodeOracleTableName("oraoop",
+            "targusr.\"junk.tab.with.dots\"", null);
+    Assert.assertEquals(context.getSchema(), "TARGUSR");
+    Assert.assertEquals(context.getName(), "junk.tab.with.dots");
+
+    // Test for "." within schema and within table...
+    context =
+        OraOopUtilities.decodeOracleTableName("oraoop",
+            "\"targ.usr\".\"junk.tab.with.dots\"", null);
+    Assert.assertEquals(context.getSchema(), "targ.usr");
+    Assert.assertEquals(context.getName(), "junk.tab.with.dots");
+  }
+
+  @Test
+  public void testgetCurrentMethodName() {
+
+    String actual = OraOopUtilities.getCurrentMethodName();
+    String expected = "testgetCurrentMethodName()";
+
+    Assert.assertEquals(expected, actual);
+
+  }
+
+  @Test
+  public void testgenerateDataChunkId() {
+
+    String expected;
+    String actual;
+
+    expected = "1_1";
+    actual = OraOopUtilities.generateDataChunkId(1, 1);
+    Assert.assertEquals(expected, actual);
+
+    expected = "1234_99";
+    actual = OraOopUtilities.generateDataChunkId(1234, 99);
+    Assert.assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testgetDuplicatedStringArrayValues() {
+
+    try {
+      OraOopUtilities.getDuplicatedStringArrayValues(null, false);
+      Assert.fail("An IllegalArgumentException should be been thrown.");
+    } catch (IllegalArgumentException ex) {
+      // This is what we want to happen.
+    }
+
+    String[] duplicates = null;
+
+    duplicates =
+        OraOopUtilities.getDuplicatedStringArrayValues(new String[] {}, false);
+    Assert.assertEquals(0, duplicates.length);
+
+    duplicates =
+        OraOopUtilities.getDuplicatedStringArrayValues(new String[] { "a", "b",
+            "c", }, false);
+    Assert.assertEquals(0, duplicates.length);
+
+    duplicates =
+        OraOopUtilities.getDuplicatedStringArrayValues(new String[] { "a", "A",
+            "b", }, false);
+    Assert.assertEquals(0, duplicates.length);
+
+    duplicates =
+        OraOopUtilities.getDuplicatedStringArrayValues(new String[] { "a", "A",
+            "b", }, true);
+    Assert.assertEquals(1, duplicates.length);
+    Assert.assertEquals("A", duplicates[0]);
+
+    duplicates =
+        OraOopUtilities.getDuplicatedStringArrayValues(new String[] { "A", "a",
+            "b", }, true);
+    Assert.assertEquals(1, duplicates.length);
+    Assert.assertEquals("a", duplicates[0]);
+
+    duplicates =
+        OraOopUtilities.getDuplicatedStringArrayValues(new String[] { "A", "a",
+            "b", "A", }, false);
+    Assert.assertEquals(1, duplicates.length);
+    Assert.assertEquals("A", duplicates[0]);
+
+    duplicates =
+        OraOopUtilities.getDuplicatedStringArrayValues(new String[] { "A", "a",
+            "b", "A", }, true);
+    Assert.assertEquals(2, duplicates.length);
+    Assert.assertEquals("a", duplicates[0]);
+    Assert.assertEquals("A", duplicates[1]);
+
+    duplicates =
+        OraOopUtilities.getDuplicatedStringArrayValues(new String[] { "A", "a",
+            "b", "A", "A", }, true);
+    Assert.assertEquals(2, duplicates.length);
+    Assert.assertEquals("a", duplicates[0]);
+    Assert.assertEquals("A", duplicates[1]);
+  }
+
+  @Test
+  public void testgetFullExceptionMessage() {
+
+    try {
+
+      try {
+        try {
+          throw new IOException("lorem ipsum!");
+        } catch (IOException ex) {
+          throw new SQLException("dolor sit amet", ex);
+        }
+      } catch (SQLException ex) {
+        throw new RuntimeException("consectetur adipisicing elit", ex);
+      }
+
+    } catch (Exception ex) {
+      String msg = OraOopUtilities.getFullExceptionMessage(ex);
+      if (!msg.contains("IOException") || !msg.contains("lorem ipsum!")) {
+        Assert
+            .fail("Inner exception text has not been included in the message");
+      }
+      if (!msg.contains("SQLException") || !msg.contains("dolor sit amet")) {
+        Assert
+            .fail("Inner exception text has not been included in the message");
+      }
+      if (!msg.contains("RuntimeException")
+          || !msg.contains("consectetur adipisicing elit")) {
+        Assert
+            .fail("Outer exception text has not been included in the message");
+      }
+    }
+  }
+
+  @Test
+  public void testGetOraOopOracleDataChunkMethod() {
+    try {
+      OraOopUtilities.getOraOopOracleDataChunkMethod(null);
+      Assert.fail("An IllegalArgumentException should be been thrown.");
+    } catch (IllegalArgumentException ex) {
+      // This is what we want to happen.
+    }
+
+    OraOopConstants.OraOopOracleDataChunkMethod dataChunkMethod;
+    Configuration conf = new Configuration();
+
+    // Check the default is ROWID
+    dataChunkMethod = OraOopUtilities.getOraOopOracleDataChunkMethod(conf);
+    Assert.assertEquals(OraOopConstants.OraOopOracleDataChunkMethod.ROWID,
+        dataChunkMethod);
+
+    // Invalid value specified
+    OraOopUtilities.LOG.setCacheLogEntries(true);
+    OraOopUtilities.LOG.clearCache();
+    conf.set(OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD, "loremipsum");
+    dataChunkMethod = OraOopUtilities.getOraOopOracleDataChunkMethod(conf);
+    String logText = OraOopUtilities.LOG.getLogEntries();
+    OraOopUtilities.LOG.setCacheLogEntries(false);
+    if (!logText.toLowerCase().contains("loremipsum")) {
+      Assert
+          .fail("The LOG should inform the user they've selected an invalid "
+              + "data chunk method - and what that was.");
+    }
+    Assert.assertEquals("Should have used the default value",
+        OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD_DEFAULT,
+        dataChunkMethod);
+
+    // Valid value specified
+    conf.set(OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD, "partition");
+    dataChunkMethod = OraOopUtilities.getOraOopOracleDataChunkMethod(conf);
+    Assert.assertEquals(OraOopConstants.OraOopOracleDataChunkMethod.PARTITION,
+        dataChunkMethod);
+  }
+
+  @Test
+  public void testgetOraOopOracleBlockToSplitAllocationMethod() {
+
+    // Invalid arguments test...
+    try {
+      OraOopUtilities.getOraOopOracleBlockToSplitAllocationMethod(null,
+          OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.RANDOM);
+      Assert.fail("An IllegalArgumentException should be been thrown.");
+    } catch (IllegalArgumentException ex) {
+      // This is what we want to happen.
+    }
+
+    OraOopConstants.OraOopOracleBlockToSplitAllocationMethod allocationMethod;
+    org.apache.hadoop.conf.Configuration conf = new Configuration();
+
+    // No configuration property - and RANDOM used by default...
+    allocationMethod =
+        OraOopUtilities.getOraOopOracleBlockToSplitAllocationMethod(conf,
+            OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.RANDOM);
+    Assert.assertEquals(
+        OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.RANDOM,
+        allocationMethod);
+
+    // No configuration property - and SEQUENTIAL used by default...
+    allocationMethod =
+        OraOopUtilities.getOraOopOracleBlockToSplitAllocationMethod(
+           conf,
+           OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.SEQUENTIAL);
+    Assert.assertEquals(
+        OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.SEQUENTIAL,
+        allocationMethod);
+
+    // An invalid property value specified...
+    OraOopUtilities.LOG.setCacheLogEntries(true);
+    OraOopUtilities.LOG.clearCache();
+    conf.set(OraOopConstants.ORAOOP_ORACLE_BLOCK_TO_SPLIT_ALLOCATION_METHOD,
+        "loremipsum");
+    allocationMethod =
+        OraOopUtilities.getOraOopOracleBlockToSplitAllocationMethod(
+           conf,
+           OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.SEQUENTIAL);
+    String logText = OraOopUtilities.LOG.getLogEntries();
+    OraOopUtilities.LOG.setCacheLogEntries(false);
+    if (!logText.toLowerCase().contains("loremipsum")) {
+      Assert
+          .fail("The LOG should inform the user they've selected an invalid "
+              + "allocation method - and what that was.");
+    }
+
+    if (!logText.contains("ROUNDROBIN or SEQUENTIAL or RANDOM")) {
+      Assert.fail("The LOG should inform the user what the valid choices are.");
+    }
+
+    // An valid property value specified...
+    conf.set(OraOopConstants.ORAOOP_ORACLE_BLOCK_TO_SPLIT_ALLOCATION_METHOD,
+        "sequential");
+    allocationMethod =
+        OraOopUtilities.getOraOopOracleBlockToSplitAllocationMethod(
+           conf,
+           OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.SEQUENTIAL);
+    Assert.assertEquals(
+        OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.SEQUENTIAL,
+        allocationMethod);
+  }
+
+  @Test
+  public void testgetOraOopTableImportWhereClauseLocation() {
+
+    // Invalid arguments test...
+    try {
+      OraOopUtilities.getOraOopTableImportWhereClauseLocation(null,
+          OraOopConstants.OraOopTableImportWhereClauseLocation.SPLIT);
+      Assert.fail("An IllegalArgumentException should be been thrown.");
+    } catch (IllegalArgumentException ex) {
+      // This is what we want to happen.
+    }
+
+    OraOopConstants.OraOopTableImportWhereClauseLocation location;
+    org.apache.hadoop.conf.Configuration conf = new Configuration();
+
+    // No configuration property - and SPLIT used by default...
+    location =
+        OraOopUtilities.getOraOopTableImportWhereClauseLocation(conf,
+            OraOopConstants.OraOopTableImportWhereClauseLocation.SPLIT);
+    Assert.assertEquals(
+        OraOopConstants.OraOopTableImportWhereClauseLocation.SPLIT, location);
+
+    // An invalid property value specified...
+    OraOopUtilities.LOG.setCacheLogEntries(true);
+    OraOopUtilities.LOG.clearCache();
+    conf.set(OraOopConstants.ORAOOP_TABLE_IMPORT_WHERE_CLAUSE_LOCATION,
+        "loremipsum");
+    location =
+        OraOopUtilities.getOraOopTableImportWhereClauseLocation(conf,
+            OraOopConstants.OraOopTableImportWhereClauseLocation.SPLIT);
+    String logText = OraOopUtilities.LOG.getLogEntries();
+    OraOopUtilities.LOG.setCacheLogEntries(false);
+    if (!logText.toLowerCase().contains("loremipsum")) {
+      Assert
+          .fail("The LOG should inform the user they've selected an invalid "
+              + "where-clause-location - and what that was.");
+    }
+
+    if (!logText.contains("SUBSPLIT or SPLIT")) {
+      Assert.fail("The LOG should inform the user what the valid choices are.");
+    }
+
+    // An valid property value specified...
+    conf.set(OraOopConstants.ORAOOP_TABLE_IMPORT_WHERE_CLAUSE_LOCATION,
+        "split");
+    location =
+        OraOopUtilities.getOraOopTableImportWhereClauseLocation(conf,
+            OraOopConstants.OraOopTableImportWhereClauseLocation.SUBSPLIT);
+    Assert.assertEquals(
+        OraOopConstants.OraOopTableImportWhereClauseLocation.SPLIT, location);
+
+  }
+
+  @Test
+  public void testpadLeft() {
+
+    String expected = "   a";
+    String actual = OraOopUtilities.padLeft("a", 4);
+    Assert.assertEquals(expected, actual);
+
+    expected = "abcd";
+    actual = OraOopUtilities.padLeft("abcd", 3);
+    Assert.assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testpadRight() {
+
+    String expected = "a   ";
+    String actual = OraOopUtilities.padRight("a", 4);
+    Assert.assertEquals(expected, actual);
+
+    expected = "abcd";
+    actual = OraOopUtilities.padRight("abcd", 3);
+    Assert.assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testReplaceConfigurationExpression() {
+
+    org.apache.hadoop.conf.Configuration conf = new Configuration();
+
+    // Default value used...
+    String actual =
+        OraOopUtilities.replaceConfigurationExpression(
+            "alter session set timezone = '{oracle.sessionTimeZone|GMT}';",
+            conf);
+    String expected = "alter session set timezone = 'GMT';";
+    Assert.assertEquals("OraOop configuration expression failure.", expected,
+        actual);
+
+    // Configuration property value exists...
+    conf.set("oracle.sessionTimeZone", "Africa/Algiers");
+    actual =
+        OraOopUtilities.replaceConfigurationExpression(
+            "alter session set timezone = '{oracle.sessionTimeZone|GMT}';",
+            conf);
+    expected = "alter session set timezone = 'Africa/Algiers';";
+    Assert.assertEquals("OraOop configuration expression failure.", expected,
+        actual);
+
+    // Multiple properties in one expression...
+    conf.set("expr1", "1");
+    conf.set("expr2", "2");
+    conf.set("expr3", "3");
+    conf.set("expr4", "4");
+    actual =
+        OraOopUtilities.replaceConfigurationExpression("set {expr1}={expr2};",
+            conf);
+    expected = "set 1=2;";
+    Assert.assertEquals("OraOop configuration expression failure.", expected,
+        actual);
+
+    actual =
+        OraOopUtilities.replaceConfigurationExpression(
+            "set {expr4|0}={expr5|5};", conf);
+    expected = "set 4=5;";
+    Assert.assertEquals("OraOop configuration expression failure.", expected,
+        actual);
+  }
+
+  @Test
+  public void testStackContainsClass() {
+
+    if (OraOopUtilities.stackContainsClass("lorem.ipsum.dolor")) {
+      Assert.fail("There's no way the stack actually contains this!");
+    }
+
+    String expected = "org.apache.sqoop.manager.oracle.TestOraOopUtilities";
+    if (!OraOopUtilities.stackContainsClass(expected)) {
+      Assert.fail("The stack should contain the class:" + expected);
+    }
+  }
+
+  @Test
+  public void testGetImportHint() {
+    org.apache.hadoop.conf.Configuration conf = new Configuration();
+
+    String hint = OraOopUtilities.getImportHint(conf);
+    Assert.assertEquals("Default import hint", "/*+ NO_INDEX(t) */ ", hint);
+
+    conf.set("oraoop.import.hint", "NO_INDEX(t) SCN_ASCENDING");
+    hint = OraOopUtilities.getImportHint(conf);
+    Assert.assertEquals("Changed import hint",
+        "/*+ NO_INDEX(t) SCN_ASCENDING */ ", hint);
+
+    conf.set("oraoop.import.hint", "       ");
+    hint = OraOopUtilities.getImportHint(conf);
+    Assert.assertEquals("Whitespace import hint", "", hint);
+
+    conf.set("oraoop.import.hint", "");
+    hint = OraOopUtilities.getImportHint(conf);
+    Assert.assertEquals("Blank import hint", "", hint);
+
+  }
+
+  @Test
+  public void testSplitStringList() {
+    List<String> result = null;
+    List<String> expected = null;
+
+    expected = new ArrayList<String>();
+    expected.add("abcde");
+    expected.add("ghijklm");
+    result = OraOopUtilities.splitStringList("abcde,ghijklm");
+    Assert.assertEquals(expected, result);
+
+    expected = new ArrayList<String>();
+    expected.add("\"abcde\"");
+    expected.add("\"ghijklm\"");
+    result = OraOopUtilities.splitStringList("\"abcde\",\"ghijklm\"");
+    Assert.assertEquals(expected, result);
+
+    expected = new ArrayList<String>();
+    expected.add("abcde");
+    expected.add("\"ghijklm\"");
+    result = OraOopUtilities.splitStringList("abcde,\"ghijklm\"");
+    Assert.assertEquals(expected, result);
+
+    expected = new ArrayList<String>();
+    expected.add("\"abcde\"");
+    expected.add("ghijklm");
+    result = OraOopUtilities.splitStringList("\"abcde\",ghijklm");
+    Assert.assertEquals(expected, result);
+
+    expected = new ArrayList<String>();
+    expected.add("\"ab,cde\"");
+    expected.add("ghijklm");
+    result = OraOopUtilities.splitStringList("\"ab,cde\",ghijklm");
+    Assert.assertEquals(expected, result);
+
+    expected = new ArrayList<String>();
+    expected.add("abcde");
+    expected.add("\"ghi,jklm\"");
+    result = OraOopUtilities.splitStringList("abcde,\"ghi,jklm\"");
+    Assert.assertEquals(expected, result);
+
+    expected = new ArrayList<String>();
+    expected.add("\"ab,cde\"");
+    expected.add("\"ghi,jklm\"");
+    result = OraOopUtilities.splitStringList("\"ab,cde\",\"ghi,jklm\"");
+    Assert.assertEquals(expected, result);
+
+    expected = new ArrayList<String>();
+    expected.add("\"ab,cde\"");
+    expected.add("\"ghi,jklm\"");
+    expected.add("\",Lorem\"");
+    expected.add("\"ip!~sum\"");
+    expected.add("\"do,lo,,r\"");
+    expected.add("\"s#it\"");
+    expected.add("\"am$e$t\"");
+    result =
+        OraOopUtilities
+            .splitStringList("\"ab,cde\",\"ghi,jklm\",\",Lorem\",\"ip!~sum\","
+                + "\"do,lo,,r\",\"s#it\",\"am$e$t\"");
+    Assert.assertEquals(expected, result);
+
+    expected = new ArrayList<String>();
+    expected.add("LOREM");
+    expected.add("IPSUM");
+    expected.add("DOLOR");
+    expected.add("SIT");
+    expected.add("AMET");
+    result = OraOopUtilities.splitStringList("LOREM,IPSUM,DOLOR,SIT,AMET");
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testSplitOracleStringList() {
+    List<String> result = null;
+    List<String> expected = null;
+
+    expected = new ArrayList<String>();
+    expected.add("LOREM");
+    expected.add("IPSUM");
+    expected.add("DOLOR");
+    expected.add("SIT");
+    expected.add("AMET");
+    result =
+        OraOopUtilities.splitOracleStringList("lorem,ipsum,dolor,sit,amet");
+    Assert.assertEquals(expected, result);
+
+    expected = new ArrayList<String>();
+    expected.add("LOREM");
+    expected.add("ipsum");
+    expected.add("dolor");
+    expected.add("SIT");
+    expected.add("amet");
+    result =
+        OraOopUtilities
+            .splitOracleStringList("lorem,\"ipsum\",\"dolor\",sit,\"amet\"");
+    Assert.assertEquals(expected, result);
+
+    expected = new ArrayList<String>();
+    expected.add("LOREM");
+    expected.add("ip,sum");
+    expected.add("dol$or");
+    expected.add("SIT");
+    expected.add("am!~#et");
+    result =
+        OraOopUtilities
+          .splitOracleStringList("lorem,\"ip,sum\",\"dol$or\",sit,\"am!~#et\"");
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testAppendJavaSecurityEgd() {
+    String confProperty = "mapred.child.java.opts";
+    String confValue = "-Djava.security.egd=file:///dev/urandom";
+    Configuration conf = new Configuration();
+
+    String expected = confValue;
+    String actual = null;
+    conf.set(confProperty, "");
+    OraOopUtilities.appendJavaSecurityEgd(conf);
+    actual = conf.get(confProperty);
+    Assert.assertEquals("Append to empty string", expected, actual);
+
+    expected = "-Djava.security.egd=file:/dev/random";
+    conf.set(confProperty, expected);
+    OraOopUtilities.appendJavaSecurityEgd(conf);
+    actual = conf.get(confProperty);
+    Assert.assertEquals("Append to empty string", expected, actual);
+
+    expected = confValue + " -Xmx201m";
+    conf.set(confProperty, "-Xmx201m");
+    OraOopUtilities.appendJavaSecurityEgd(conf);
+    actual = conf.get(confProperty);
+    Assert.assertEquals("Append to empty string", expected, actual);
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/TestOracleTable.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/TestOracleTable.java b/src/test/org/apache/sqoop/manager/oracle/TestOracleTable.java
new file mode 100644
index 0000000..854d826
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/TestOracleTable.java
@@ -0,0 +1,42 @@
+/**
+ * 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.oracle;
+
+import junit.framework.Assert;
+
+import org.junit.Test;
+
+/**
+ * Unit tests for OracleTable.
+ */
+public class TestOracleTable extends OraOopTestCase {
+
+  @Test
+  public void testToString() {
+    OracleTable table = new OracleTable("ORAOOP", "TEST_TABLE");
+    Assert.assertEquals("\"ORAOOP\".\"TEST_TABLE\"", table.toString());
+
+    table = new OracleTable("", "TEST_TABLE2");
+    Assert.assertEquals("\"TEST_TABLE2\"", table.toString());
+
+    table = new OracleTable("TEST_TABLE3");
+    Assert.assertEquals("\"TEST_TABLE3\"", table.toString());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/TimestampDataTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/TimestampDataTest.java b/src/test/org/apache/sqoop/manager/oracle/TimestampDataTest.java
new file mode 100644
index 0000000..f338595
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/TimestampDataTest.java
@@ -0,0 +1,51 @@
+/**
+ * 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.oracle;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Test;
+
+/**
+ * These tests need to be separate as changing the mapping type for timestamp
+ * requires the tests to be run in a different process. Maven needs to be setup
+ * to fork per test class.
+ */
+public class TimestampDataTest extends OraOopTestCase {
+
+  @Test
+  public void testProductImportTimezone() throws Exception {
+    setSqoopTargetDirectory(getSqoopTargetDirectory() + "tst_product_timezone");
+    createTable("table_tst_product.xml");
+
+    Configuration sqoopConf = getSqoopConf();
+    sqoopConf.setBoolean(OraOopConstants.ORAOOP_MAP_TIMESTAMP_AS_STRING, false);
+
+    try {
+      int retCode = runImport("tst_product", sqoopConf, false);
+      Assert.assertEquals("Return code should be 0", 0, retCode);
+
+    } finally {
+      cleanupFolders();
+      closeTestEnvConnection();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/BigDecimalGenerator.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/BigDecimalGenerator.java b/src/test/org/apache/sqoop/manager/oracle/util/BigDecimalGenerator.java
new file mode 100644
index 0000000..5a7c698
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/BigDecimalGenerator.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.sqoop.manager.oracle.util;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+
+/**
+ * Generates BigDecimal test data.
+ */
+public class BigDecimalGenerator extends OraOopTestDataGenerator<BigDecimal> {
+  private final int precision;
+  private final int scale;
+
+  /**
+   * Create a BigDecimalGenerator suitable for populating an Oracle
+   * NUMBER(precision,scale) field.
+   *
+   * @param precision
+   *          Maximum number of decimal digits in generated BigDecimals
+   * @param scale
+   *          Number of decimal digits to the right of the decimal point in
+   *          generated BigDecimals
+   */
+  public BigDecimalGenerator(int precision, int scale) {
+    super();
+    this.precision = precision;
+    this.scale = scale;
+  }
+
+  @Override
+  public BigDecimal next() {
+    BigInteger unscaled =
+        BigInteger.valueOf(rng.nextInt((int) Math.pow(10, precision)));
+    BigDecimal value = new BigDecimal(unscaled, scale);
+    if (rng.nextBoolean()) {
+      value = value.negate();
+    }
+    return value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/BinaryDoubleGenerator.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/BinaryDoubleGenerator.java b/src/test/org/apache/sqoop/manager/oracle/util/BinaryDoubleGenerator.java
new file mode 100644
index 0000000..27fe34e
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/BinaryDoubleGenerator.java
@@ -0,0 +1,32 @@
+/**
+ * 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.oracle.util;
+
+/**
+ * Generates Double test data. Test data is distributed over the entire range of
+ * possible doubles, including NaN, positive and negative infinity and positive
+ * and negative zero.
+ */
+public class BinaryDoubleGenerator extends OraOopTestDataGenerator<Double> {
+  @Override
+  public Double next() {
+    return Double.longBitsToDouble(rng.nextLong());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/BinaryFloatGenerator.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/BinaryFloatGenerator.java b/src/test/org/apache/sqoop/manager/oracle/util/BinaryFloatGenerator.java
new file mode 100644
index 0000000..2e1be29
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/BinaryFloatGenerator.java
@@ -0,0 +1,32 @@
+/**
+ * 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.oracle.util;
+
+/**
+ * Generates Float test data. Test data is distributed over the entire range of
+ * possible floats, including NaN, positive and negative infinity and positive
+ * and negative zero.
+ */
+public class BinaryFloatGenerator extends OraOopTestDataGenerator<Float> {
+  @Override
+  public Float next() {
+    return Float.intBitsToFloat(rng.nextInt());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/BlobGenerator.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/BlobGenerator.java b/src/test/org/apache/sqoop/manager/oracle/util/BlobGenerator.java
new file mode 100644
index 0000000..6279f15
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/BlobGenerator.java
@@ -0,0 +1,103 @@
+/**
+ * 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.oracle.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.reflect.Method;
+import java.sql.Blob;
+import java.sql.Connection;
+
+/**
+ * Generates Blob test data.
+ */
+public class BlobGenerator extends OraOopTestDataGenerator<Blob> {
+  private static Class<?> blobClass;
+  private static Method methCreateTemporary;
+  private static Method methGetBufferSize;
+  private static int durationSession;
+
+  static {
+    try {
+      blobClass = Class.forName("oracle.sql.BLOB");
+      methCreateTemporary =
+          blobClass.getMethod("createTemporary", Connection.class,
+              boolean.class, int.class);
+      methGetBufferSize = blobClass.getMethod("getBufferSize");
+      durationSession = blobClass.getField("DURATION_SESSION").getInt(null);
+    } catch (Exception e) {
+      throw new RuntimeException(
+          "Problem getting Oracle JDBC methods via reflection.", e);
+    }
+  }
+
+  private Connection conn;
+  private int minBytes;
+  private int maxBytes;
+
+  /**
+   * Create a generator that will generate BLOBs with length varying between
+   * minBytes and maxBytes.
+   *
+   * @param conn
+   *          Oracle connection to use when creating BLOBs
+   * @param minBytes
+   *          Minimum number of bytes in generated BLOBs
+   * @param maxBytes
+   *          Maximum number of bytes in generated BLOBs
+   */
+  public BlobGenerator(Connection conn, int minBytes, int maxBytes) {
+    super();
+    this.conn = conn;
+    this.minBytes = minBytes;
+    this.maxBytes = maxBytes;
+  }
+
+  @Override
+  public Blob next() {
+    try {
+      Blob blob =
+          (Blob) methCreateTemporary.invoke(null, conn, false, durationSession);
+
+      int blobSize =
+          (int) (rng.nextDouble() * (maxBytes - minBytes) + minBytes);
+      byte[] blobData = new byte[blobSize];
+      rng.nextBytes(blobData);
+
+      // blob.setBytes(blobData);
+
+      OutputStream os = blob.setBinaryStream(1);
+      InputStream is = new ByteArrayInputStream(blobData);
+      int bufferSize = (Integer) methGetBufferSize.invoke(blob);
+      byte[] buffer = new byte[bufferSize];
+      int bytesRead = 0;
+      while ((bytesRead = is.read(buffer)) != -1) {
+        os.write(buffer, 0, bytesRead);
+      }
+      os.close();
+      is.close();
+
+      return blob;
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/BytesGenerator.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/BytesGenerator.java b/src/test/org/apache/sqoop/manager/oracle/util/BytesGenerator.java
new file mode 100644
index 0000000..5b105e6
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/BytesGenerator.java
@@ -0,0 +1,52 @@
+/**
+ * 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.oracle.util;
+
+/**
+ * Generates Blob test data.
+ */
+public class BytesGenerator extends OraOopTestDataGenerator<byte[]> {
+  private int minBytes;
+  private int maxBytes;
+
+  /**
+   * Create a generator that will generate arrays of bytes with length varying
+   * between minBytes and maxBytes.
+   *
+   * @param minBytes
+   *          Minimum number of bytes in generated RAWs
+   * @param maxBytes
+   *          Maximum number of bytes in generated RAWs
+   */
+  public BytesGenerator(int minBytes, int maxBytes) {
+    super();
+    this.minBytes = minBytes;
+    this.maxBytes = maxBytes;
+  }
+
+  @Override
+  public byte[] next() {
+    int rawSize = (int) (rng.nextDouble() * (maxBytes - minBytes) + minBytes);
+    byte[] rawData = new byte[rawSize];
+    rng.nextBytes(rawData);
+
+    return rawData;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/CharGenerator.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/CharGenerator.java b/src/test/org/apache/sqoop/manager/oracle/util/CharGenerator.java
new file mode 100644
index 0000000..a053909
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/CharGenerator.java
@@ -0,0 +1,54 @@
+/**
+ * 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.oracle.util;
+
+/**
+ * Generates String test data. All generated characters will be encodable in
+ * US-ASCII.
+ */
+public class CharGenerator extends OraOopTestDataGenerator<String> {
+  private int minLength;
+  private int maxLength;
+
+  /**
+   * Create a CharGenerator that will generate Strings between minLength and
+   * maxLength in length.
+   *
+   * @param minLength
+   *          Minimum length for generated strings
+   * @param maxLength
+   *          Maximum length for generated strings
+   */
+  public CharGenerator(int minLength, int maxLength) {
+    super();
+    this.minLength = minLength;
+    this.maxLength = maxLength;
+  }
+
+  @Override
+  public String next() {
+    int length = minLength + rng.nextInt(maxLength - minLength + 1);
+    StringBuilder sb = new StringBuilder();
+    while (sb.length() < length) {
+      sb.append(Character.toChars(rng.nextInt(128)));
+    }
+    return sb.toString().substring(0, length);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/FloatGenerator.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/FloatGenerator.java b/src/test/org/apache/sqoop/manager/oracle/util/FloatGenerator.java
new file mode 100644
index 0000000..f345d8d
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/FloatGenerator.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.sqoop.manager.oracle.util;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+
+/**
+ * Generates test data for Oracle FLOAT columns.
+ */
+public class FloatGenerator extends OraOopTestDataGenerator<BigDecimal> {
+  private static final int MIN_SCALE = -125;
+  private static final int MAX_SCALE = 125;
+  private final int precision;
+
+  /**
+   * Create a float generator with the specified binary precision.
+   *
+   * @param precision
+   *          The number of bits in the value of generated numbers
+   */
+  public FloatGenerator(int precision) {
+    super();
+    this.precision = precision;
+  }
+
+  @Override
+  public BigDecimal next() {
+    BigInteger unscaled = new BigInteger(precision, rng);
+    BigDecimal unscaledBD = new BigDecimal(unscaled);
+    int scale =
+        rng.nextInt(MAX_SCALE - MIN_SCALE + 1) + MIN_SCALE
+            - unscaledBD.precision();
+    BigDecimal result = new BigDecimal(unscaled, -scale);
+    if (rng.nextBoolean()) {
+      result = result.negate();
+    }
+    return result;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/HadoopFiles.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/HadoopFiles.java b/src/test/org/apache/sqoop/manager/oracle/util/HadoopFiles.java
new file mode 100644
index 0000000..6592036
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/HadoopFiles.java
@@ -0,0 +1,37 @@
+/**
+ * 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.oracle.util;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Helper class for HDFS related functions.
+ */
+public final class HadoopFiles {
+  private HadoopFiles() {
+  }
+
+  public static void delete(Path file, boolean recursive) throws Exception {
+    FileSystem fileSystem = FileSystem.get(file.toUri(), new Configuration());
+    fileSystem.delete(file, recursive);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/IntervalDaySecondGenerator.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/IntervalDaySecondGenerator.java b/src/test/org/apache/sqoop/manager/oracle/util/IntervalDaySecondGenerator.java
new file mode 100644
index 0000000..a762643
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/IntervalDaySecondGenerator.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.sqoop.manager.oracle.util;
+
+/**
+ * Generates test data for Oracle INTERVAL DAY TO SECOND columns.
+ */
+public class IntervalDaySecondGenerator extends
+                 OraOopTestDataGenerator<String> {
+  private final int daysPrecision;
+  private final int minDays;
+  private final int maxDays;
+  private final int secondsPrecision;
+  private final int maxFractionalSeconds;
+
+  /**
+   * Create a generator that will generate intervals with the specified
+   * precision for days and seconds.
+   *
+   * @param daysPrecision
+   *          Number of decimal digits in the days part of each interval
+   * @param secondsPrecision
+   *          Number of decimal digits after the decimal point in seconds part
+   *          of each interval.
+   */
+  public IntervalDaySecondGenerator(int daysPrecision, int secondsPrecision) {
+    super();
+    this.daysPrecision = daysPrecision;
+    this.minDays = -(int) Math.pow(10, daysPrecision) + 1;
+    this.maxDays = (int) Math.pow(10, daysPrecision) - 1;
+    this.secondsPrecision = secondsPrecision;
+    this.maxFractionalSeconds = (int) Math.pow(10, secondsPrecision);
+  }
+
+  @Override
+  public String next() {
+    int days = minDays + rng.nextInt(maxDays - minDays + 1);
+    int hours = rng.nextInt(24);
+    int minutes = rng.nextInt(60);
+    int seconds = rng.nextInt(60);
+    int fractionalSeconds = rng.nextInt(maxFractionalSeconds);
+    String val =
+        String.format("%+0" + daysPrecision + "d %02d:%02d:%02d.%0"
+            + secondsPrecision + "d", days, hours, minutes, seconds,
+            fractionalSeconds);
+    return val;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/IntervalYearMonthGenerator.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/IntervalYearMonthGenerator.java b/src/test/org/apache/sqoop/manager/oracle/util/IntervalYearMonthGenerator.java
new file mode 100644
index 0000000..af57253
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/IntervalYearMonthGenerator.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.sqoop.manager.oracle.util;
+
+/**
+ * Generates test data for Oracle INTERVAL YEAR TO MONTH columns.
+ */
+public class IntervalYearMonthGenerator extends
+                 OraOopTestDataGenerator<String> {
+  private final int precision;
+  private final int minYear;
+  private final int maxYear;
+
+  /**
+   * Create a generator that will generate intervals with the specified
+   * precision for years.
+   *
+   * @param precision
+   *          Number of decimal digits in the years part of each interval
+   */
+  public IntervalYearMonthGenerator(int precision) {
+    super();
+    this.precision = precision;
+    this.minYear = -(int) Math.pow(10, precision) + 1;
+    this.maxYear = (int) Math.pow(10, precision) - 1;
+  }
+
+  @Override
+  public String next() {
+    int years = minYear + rng.nextInt(maxYear - minYear + 1);
+    int months = rng.nextInt(12);
+    return String.format("%+0" + precision + "d-%02d", years, months);
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/NCharGenerator.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/NCharGenerator.java b/src/test/org/apache/sqoop/manager/oracle/util/NCharGenerator.java
new file mode 100644
index 0000000..b3abe33
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/NCharGenerator.java
@@ -0,0 +1,54 @@
+/**
+ * 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.oracle.util;
+
+/**
+ * Generates String test data. All generated characters will be encodable in
+ * UTF-8.
+ */
+public class NCharGenerator extends OraOopTestDataGenerator<String> {
+  private int minLength;
+  private int maxLength;
+
+  /**
+   * Create an NCharGenerator that will generate Strings between minLength and
+   * maxLength in length.
+   *
+   * @param minLength
+   *          Minimum length for generated strings
+   * @param maxLength
+   *          Maximum length for generated strings
+   */
+  public NCharGenerator(int minLength, int maxLength) {
+    super();
+    this.minLength = minLength;
+    this.maxLength = maxLength;
+  }
+
+  @Override
+  public String next() {
+    int length = minLength + rng.nextInt(maxLength - minLength + 1);
+    StringBuilder sb = new StringBuilder();
+    while (sb.length() < length) {
+      sb.append(Character.toChars(rng.nextInt(0x10FFFF)));
+    }
+    return sb.toString().substring(0, length);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/OraOopTestDataGenerator.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/OraOopTestDataGenerator.java b/src/test/org/apache/sqoop/manager/oracle/util/OraOopTestDataGenerator.java
new file mode 100644
index 0000000..d8c5eb8
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/OraOopTestDataGenerator.java
@@ -0,0 +1,67 @@
+/**
+ * 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.oracle.util;
+
+import java.util.Random;
+
+/**
+ * Abstract framework class for generating test data.
+ *
+ * @param <T>
+ *          The type that will be generated
+ */
+public abstract class OraOopTestDataGenerator<T> {
+  protected Random rng;
+  private long seed;
+
+  /**
+   * Initialise with a default seed for the random number generator.
+   */
+  public OraOopTestDataGenerator() {
+    this(0);
+  }
+
+  /**
+   * Initialise with a given seed for the random number generator.
+   *
+   * @param seed
+   *          The seed to initialise the rng with.
+   */
+  public OraOopTestDataGenerator(long seed) {
+    this.seed = seed;
+    if (seed == 0) {
+      rng = new Random();
+    } else {
+      rng = new Random(seed);
+    }
+  }
+
+  /**
+   * Reset the rng to its initial state.
+   */
+  public void reset() {
+    rng = new Random(seed);
+  }
+
+  /**
+   * @return The next item of test data. The same sequence will be re-generated
+   *         after a call to reset.
+   */
+  public abstract T next();
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/OraOopTestUtils.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/OraOopTestUtils.java b/src/test/org/apache/sqoop/manager/oracle/util/OraOopTestUtils.java
new file mode 100644
index 0000000..c94560b
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/OraOopTestUtils.java
@@ -0,0 +1,60 @@
+/**
+ * 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.oracle.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+/**
+ * Utility methods for OraOop system tests.
+ */
+public final class OraOopTestUtils {
+  private OraOopTestUtils() {
+  }
+  /**
+   * Pipe data from an input stream to an output stream in a separate thread.
+   *
+   * @param in
+   *          Stream to pipe data from
+   * @param out
+   *          Stream to pipe data to
+   * @return The thread in which data is being piped.
+   */
+  public static Thread backgroundPipe(final InputStream in,
+      final OutputStream out) {
+    Thread pipe = new Thread() {
+      @Override
+      public void run() {
+        try {
+          byte[] buffer = new byte[10 * 1024];
+          int len;
+          while ((len = in.read(buffer)) != -1) {
+            out.write(buffer, 0, len);
+          }
+          out.flush();
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    };
+    pipe.start();
+    return pipe;
+  }
+}