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

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

Repository: sqoop
Updated Branches:
  refs/heads/trunk d03faf354 -> 6bfaa9d65


http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/OracleData.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/OracleData.java b/src/test/org/apache/sqoop/manager/oracle/util/OracleData.java
new file mode 100644
index 0000000..871d317
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/OracleData.java
@@ -0,0 +1,192 @@
+/**
+ * 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.net.URL;
+import java.sql.CallableStatement;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.List;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.sqoop.manager.oracle.OraOopTestCase;
+
+/**
+ * Class to load an Oracle table with data based on configuration file.
+ */
+public final class OracleData {
+  private OracleData() {
+  }
+
+  enum KeyType {
+    PRIMARY, UNIQUE
+  }
+
+  private static ClassLoader classLoader;
+  static {
+    classLoader = Thread.currentThread().getContextClassLoader();
+    if (classLoader == null) {
+      classLoader = OraOopTestCase.class.getClassLoader();
+    }
+  }
+
+  private static String getColumnList(List<OracleDataDefinition> columnList) {
+    StringBuilder result = new StringBuilder();
+    String delim = "";
+    for (OracleDataDefinition column : columnList) {
+      result.append(delim).append(column.getColumnName()).append(" ").append(
+          column.getDataType());
+      delim = ",\n";
+    }
+    return result.toString();
+  }
+
+  private static String
+      getDataExpression(List<OracleDataDefinition> columnList) {
+    StringBuilder result = new StringBuilder();
+    for (OracleDataDefinition column : columnList) {
+      result.append("l_ret_rec.").append(column.getColumnName()).append(" := ")
+          .append(column.getDataExpression()).append(";\n");
+    }
+    return result.toString();
+  }
+
+  private static void createPackageSpec(Connection conn,
+      OracleTableDefinition tableDefinition) throws Exception {
+    String pkgSql =
+        IOUtils.toString(classLoader.getResource(
+            "oraoop/pkg_tst_product_gen.psk").openStream());
+    pkgSql =
+        pkgSql.replaceAll("\\$COLUMN_LIST", getColumnList(tableDefinition
+            .getColumnList()));
+    pkgSql = pkgSql.replaceAll("\\$TABLE_NAME", tableDefinition.getTableName());
+    PreparedStatement stmt = conn.prepareStatement(pkgSql);
+    stmt.execute();
+  }
+
+  private static void createPackageBody(Connection conn,
+      OracleTableDefinition tableDefinition) throws Exception {
+    String pkgSql =
+        IOUtils.toString(classLoader.getResource(
+            "oraoop/pkg_tst_product_gen.pbk").openStream());
+    pkgSql =
+        pkgSql.replaceAll("\\$COLUMN_LIST", getColumnList(tableDefinition
+            .getColumnList()));
+    pkgSql = pkgSql.replaceAll("\\$TABLE_NAME", tableDefinition.getTableName());
+    pkgSql =
+        pkgSql.replaceAll("\\$DATA_EXPRESSION_LIST",
+            getDataExpression(tableDefinition.getColumnList()));
+    pkgSql =
+        pkgSql.replaceAll("\\$PARTITION_CLAUSE", tableDefinition
+            .getPartitionClause());
+    PreparedStatement stmt = conn.prepareStatement(pkgSql);
+    stmt.execute();
+  }
+
+  private static void createKey(Connection conn, KeyType keyType,
+      OracleTableDefinition tableDefinition) throws Exception {
+    List<String> columns = null;
+    switch (keyType) {
+      case PRIMARY:
+        columns = tableDefinition.getPrimaryKeyColumns();
+        break;
+      case UNIQUE:
+        columns = tableDefinition.getUniqueKeyColumns();
+        break;
+      default:
+        throw new RuntimeException("Missing key type.");
+    }
+    if (columns != null && columns.size() > 0) {
+      StringBuilder keyColumnList = new StringBuilder();
+      String delim = "";
+      for (String column : columns) {
+        keyColumnList.append(delim).append(column);
+        delim = ",";
+      }
+      String keySql =
+          "alter table \"$TABLE_NAME\" add constraint \"$TABLE_NAME_"
+              + ((keyType == KeyType.PRIMARY) ? "PK\" primary key"
+                  : "UK\" unique") + "($PK_COLUMN_LIST) "
+              + "using index (create unique index \"$TABLE_NAME_"
+              + ((keyType == KeyType.PRIMARY) ? "PK\"" : "UK\"")
+              + " on \"$TABLE_NAME\"($PK_COLUMN_LIST) " + "parallel nologging)";
+      keySql =
+          keySql.replaceAll("\\$TABLE_NAME", tableDefinition.getTableName());
+      keySql = keySql.replaceAll("\\$PK_COLUMN_LIST", keyColumnList.toString());
+      PreparedStatement stmt = conn.prepareStatement(keySql);
+      stmt.execute();
+    }
+  }
+
+  public static int getParallelProcesses(Connection conn) throws Exception {
+    PreparedStatement stmt =
+        conn.prepareStatement("SELECT cc.value value"
+            + "\n"
+            + "FROM"
+            + "\n"
+            + "  (SELECT to_number(value) value"
+            + "\n"
+            + "  FROM v$parameter"
+            + "\n"
+            + "  WHERE name='parallel_max_servers'"
+            + "\n"
+            + "  ) pms,"
+            + "\n"
+            + "  (SELECT to_number(value) value"
+            + "\n"
+            + "  FROM v$parameter"
+            + "\n"
+            + "  WHERE name='parallel_threads_per_cpu'"
+            + "\n"
+            + "  ) ptpc,"
+            + "\n"
+            + "  (SELECT to_number(value) value FROM v$parameter "
+            + "   WHERE name='cpu_count'"
+            + "\n" + "  ) cc");
+    ResultSet res = stmt.executeQuery();
+    res.next();
+    return res.getInt(1);
+  }
+
+  public static void createTable(Connection conn,
+      OracleTableDefinition tableDefinition, int parallelDegree,
+      int rowsPerSlave) throws Exception {
+    createPackageSpec(conn, tableDefinition);
+    createPackageBody(conn, tableDefinition);
+
+    CallableStatement procStmt =
+        conn.prepareCall("begin \"PKG_ODG_" + tableDefinition.getTableName()
+            + "\".prc_load_table(?,?); end;");
+    procStmt.setInt(1, parallelDegree);
+    procStmt.setInt(2, rowsPerSlave);
+    procStmt.execute();
+
+    createKey(conn, KeyType.PRIMARY, tableDefinition);
+    createKey(conn, KeyType.UNIQUE, tableDefinition);
+  }
+
+  public static void createTable(Connection conn, String fileName,
+      int parallelDegree, int rowsPerSlave) throws Exception {
+    URL file = classLoader.getResource("oraoop/" + fileName);
+    OracleTableDefinition tableDefinition = new OracleTableDefinition(file);
+    createTable(conn, tableDefinition, parallelDegree, rowsPerSlave);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/OracleDataDefinition.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/OracleDataDefinition.java b/src/test/org/apache/sqoop/manager/oracle/util/OracleDataDefinition.java
new file mode 100644
index 0000000..1256838
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/OracleDataDefinition.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.sqoop.manager.oracle.util;
+
+/**
+ * Holds column definition for generated Oracle table.
+ */
+public class OracleDataDefinition {
+
+  private String columnName;
+  private String dataType;
+  private String dataExpression;
+
+  public OracleDataDefinition(String columnName, String dataType,
+      String dataExpression) {
+    this.columnName = columnName;
+    this.dataType = dataType;
+    this.dataExpression = dataExpression;
+  }
+
+  public String getColumnName() {
+    return columnName;
+  }
+
+  public void setColumnName(String newColumnName) {
+    this.columnName = newColumnName;
+  }
+
+  public String getDataExpression() {
+    return dataExpression;
+  }
+
+  public void setDataExpression(String newDataExpression) {
+    this.dataExpression = newDataExpression;
+  }
+
+  public String getDataType() {
+    return dataType;
+  }
+
+  public void setDataType(String newDataType) {
+    this.dataType = newDataType;
+  }
+
+  @Override
+  public String toString() {
+    return this.getColumnName();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/OracleTableDefinition.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/OracleTableDefinition.java b/src/test/org/apache/sqoop/manager/oracle/util/OracleTableDefinition.java
new file mode 100644
index 0000000..5a8c42c
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/OracleTableDefinition.java
@@ -0,0 +1,150 @@
+/**
+ * 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.File;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+/**
+ * Holds table definition for generated Oracle table.
+ */
+public class OracleTableDefinition {
+
+  private String tableName;
+  private List<OracleDataDefinition> columnList =
+      new ArrayList<OracleDataDefinition>();
+  private List<String> primaryKeyColumns = new ArrayList<String>();
+  private List<String> uniqueKeyColumns = new ArrayList<String>();
+  private String partitionClause;
+
+  public List<String> getUniqueKeyColumns() {
+    return uniqueKeyColumns;
+  }
+
+  public void setUniqueKeyColumns(List<String> newUniqueKeyColumns) {
+    this.uniqueKeyColumns = newUniqueKeyColumns;
+  }
+
+  public List<String> getPrimaryKeyColumns() {
+    return primaryKeyColumns;
+  }
+
+  public void setPrimaryKeyColumns(List<String> newPrimaryKeyColumns) {
+    this.primaryKeyColumns = newPrimaryKeyColumns;
+  }
+
+  public List<OracleDataDefinition> getColumnList() {
+    return columnList;
+  }
+
+  public void setColumnList(List<OracleDataDefinition> newColumnList) {
+    this.columnList = newColumnList;
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public void setTableName(String newTableName) {
+    this.tableName = newTableName;
+  }
+
+  public String getPartitionClause() {
+    return partitionClause == null ? "" : partitionClause;
+  }
+
+  public void setPartitionClause(String newPartitionClause) {
+    this.partitionClause = newPartitionClause;
+  }
+
+  public OracleTableDefinition() {
+
+  }
+
+  public OracleTableDefinition(URL url) {
+    try {
+      DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
+      DocumentBuilder builder = factory.newDocumentBuilder();
+      Document doc = builder.parse(new File(url.toURI()));
+
+      Element table = doc.getDocumentElement();
+      this.tableName =
+          table.getElementsByTagName("name").item(0).getChildNodes().item(0)
+              .getNodeValue();
+      NodeList columns = table.getElementsByTagName("column");
+      for (int i = 0; i < columns.getLength(); i++) {
+        Node columnNode = columns.item(i);
+        if (columnNode.getNodeType() == Node.ELEMENT_NODE) {
+          Element columnElement = (Element) columnNode;
+          String name =
+              columnElement.getElementsByTagName("name").item(0)
+                  .getChildNodes().item(0).getNodeValue();
+          String dataType =
+              columnElement.getElementsByTagName("dataType").item(0)
+                  .getChildNodes().item(0).getNodeValue();
+          String dataExpression =
+              columnElement.getElementsByTagName("dataExpression").item(0)
+                  .getChildNodes().item(0).getNodeValue();
+          this.columnList.add(new OracleDataDefinition(name, dataType,
+              dataExpression));
+        }
+      }
+
+      NodeList primaryKeyColumnsNodeList =
+          table.getElementsByTagName("primaryKeyColumn");
+      for (int i = 0; i < primaryKeyColumnsNodeList.getLength(); i++) {
+        Node primaryKeyColumnNode = primaryKeyColumnsNodeList.item(i);
+        if (primaryKeyColumnNode.getNodeType() == Node.ELEMENT_NODE) {
+          Element primaryKeyColumnElement = (Element) primaryKeyColumnNode;
+          this.primaryKeyColumns.add(primaryKeyColumnElement.getChildNodes()
+              .item(0).getNodeValue());
+        }
+      }
+
+      NodeList uniqueKeyColumnsNodeList =
+          table.getElementsByTagName("uniqueKeyColumn");
+      for (int i = 0; i < uniqueKeyColumnsNodeList.getLength(); i++) {
+        Node uniqueKeyColumnNode = uniqueKeyColumnsNodeList.item(i);
+        if (uniqueKeyColumnNode.getNodeType() == Node.ELEMENT_NODE) {
+          Element uniqueKeyColumnElement = (Element) uniqueKeyColumnNode;
+          this.uniqueKeyColumns.add(uniqueKeyColumnElement.getChildNodes()
+              .item(0).getNodeValue());
+        }
+      }
+
+      Node partitionClauseNode =
+          table.getElementsByTagName("partitionClause").item(0);
+      if (partitionClauseNode != null) {
+        this.partitionClause =
+            partitionClauseNode.getChildNodes().item(0).getNodeValue();
+      }
+    } catch (Exception e) {
+      throw new RuntimeException("Could not load table configuration", e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/RowIdGenerator.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/RowIdGenerator.java b/src/test/org/apache/sqoop/manager/oracle/util/RowIdGenerator.java
new file mode 100644
index 0000000..bb3a71b
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/RowIdGenerator.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;
+
+import java.lang.reflect.Constructor;
+import java.nio.charset.Charset;
+import java.sql.RowId;
+
+/**
+ * Generates ROWID test data. ROWIDs are represented by 18 ASCII encoded
+ * characters from the set A-Za-z0-9/+
+ *
+ * Generated ROWIDs are unlikely to represent actual rows in any Oracle
+ * database, so should be used for import/export tests only, and not used to
+ * reference data.
+ */
+public class RowIdGenerator extends OraOopTestDataGenerator<RowId> {
+  private static final String VALID_CHARS =
+      "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789/+";
+  private static final int LENGTH = 18;
+  private static Class<?> rowIdClass;
+  private static Constructor<?> rowIdConstructor;
+
+  static {
+    try {
+      rowIdClass = Class.forName("oracle.sql.ROWID");
+      rowIdConstructor = rowIdClass.getConstructor(byte[].class);
+    } catch (Exception e) {
+      throw new RuntimeException(
+          "Problem getting Oracle JDBC methods via reflection.", e);
+    }
+  }
+
+  @Override
+  public RowId next() {
+    try {
+      StringBuffer sb = new StringBuffer();
+      while (sb.length() < LENGTH) {
+        sb.append(VALID_CHARS.charAt(rng.nextInt(VALID_CHARS.length())));
+      }
+      return (RowId) rowIdConstructor.newInstance(sb.toString().getBytes(
+          Charset.forName("US-ASCII")));
+    } 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/TimestampGenerator.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/TimestampGenerator.java b/src/test/org/apache/sqoop/manager/oracle/util/TimestampGenerator.java
new file mode 100644
index 0000000..b2ce366
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/TimestampGenerator.java
@@ -0,0 +1,71 @@
+/**
+ * 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.sql.Timestamp;
+import java.util.Calendar;
+import java.util.GregorianCalendar;
+
+/**
+ * Generates test data for Oracle DATE columns. Generated Timestamps are between
+ * 4711BC and 9999AD.
+ */
+public class TimestampGenerator extends OraOopTestDataGenerator<Timestamp> {
+  private static final int NANOS_DIGITS = 9;
+  private static final int MIN_YEAR = -4711;
+  private static final int MAX_YEAR = 9999;
+  private final int precision;
+  private final Calendar cal = Calendar.getInstance();
+
+  /**
+   * Create a TimestampGenerator that will generate Timestamps with a given
+   * precision.
+   *
+   * @param precision
+   *          Number of decimal digits after the decimal point in the seconds of
+   *          generated Timestamps.
+   */
+  public TimestampGenerator(int precision) {
+    this.precision = precision;
+  }
+
+  @Override
+  public Timestamp next() {
+    cal.clear();
+    cal.set(Calendar.YEAR, MIN_YEAR + rng.nextInt(MAX_YEAR - MIN_YEAR + 1));
+    cal.set(Calendar.DAY_OF_YEAR, 1 + rng.nextInt(cal
+        .getActualMaximum(Calendar.DAY_OF_YEAR)));
+    cal.set(Calendar.HOUR_OF_DAY, rng.nextInt(24));
+    cal.set(Calendar.MINUTE, rng.nextInt(60));
+    cal.set(Calendar.SECOND, rng.nextInt(
+        cal.getActualMaximum(Calendar.SECOND)));
+    // Workaround for oracle jdbc bugs related to BC leap years
+    if (cal.get(Calendar.ERA) == GregorianCalendar.BC
+      && cal.get(Calendar.MONTH) == 1 && cal.get(Calendar.DAY_OF_MONTH) >= 28) {
+      return next();
+    }
+    Timestamp timestamp = new Timestamp(cal.getTimeInMillis());
+    if (precision > 0) {
+      int nanos = rng.nextInt((int) Math.pow(10, precision));
+      timestamp.setNanos(nanos * (int) Math.pow(10, NANOS_DIGITS - precision));
+    }
+    return timestamp;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/util/URIGenerator.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/util/URIGenerator.java b/src/test/org/apache/sqoop/manager/oracle/util/URIGenerator.java
new file mode 100644
index 0000000..63fec01
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/util/URIGenerator.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;
+
+/**
+ * Generates test data for Oracle UriType columns. Generated Strings can be cast
+ * to URITypes with sys.UriFactory.getUri(value). Generated strings may be
+ * detect as any of HTTPURIType, DBURIType or XDBURIType.
+ *
+ * Generated URIs are unlikely to resolve successfully, so should be used for
+ * import/export tests only, and not used to reference data.
+ */
+public class URIGenerator extends OraOopTestDataGenerator<String> {
+  private static final int MIN_LENGTH = 15;
+  private static final int MAX_LENGTH = 30;
+
+  @Override
+  public String next() {
+    StringBuilder sb = new StringBuilder();
+    switch (rng.nextInt(3)) {
+      case 0: // Generate a String that will detect as an HTTPURIType
+        sb.append("http://");
+        break;
+      case 1: // Generate a String that will detect as an DBURIType
+        sb.append("/oradb/");
+        break;
+      case 2: // Generate a String that will detect as an XDBURIType
+        break;
+      default:
+        throw new RuntimeException("Invalid number generated.");
+    }
+
+    int length =
+        sb.length() + MIN_LENGTH + rng.nextInt(MAX_LENGTH - MIN_LENGTH + 1);
+    while (sb.length() < length) {
+      sb.append(Character.toChars(rng.nextInt(128)));
+    }
+    return sb.toString();
+  }
+
+}


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

Posted by ve...@apache.org.
SQOOP-1287: Add high performance Oracle connector into Sqoop
  (David Robson via Venkat Ranganathan)


Project: http://git-wip-us.apache.org/repos/asf/sqoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/sqoop/commit/6bfaa9d6
Tree: http://git-wip-us.apache.org/repos/asf/sqoop/tree/6bfaa9d6
Diff: http://git-wip-us.apache.org/repos/asf/sqoop/diff/6bfaa9d6

Branch: refs/heads/trunk
Commit: 6bfaa9d65395cf1cc365a4a6b9df492550c5c847
Parents: d03faf3
Author: Venkat Ranganathan <ve...@hortonworks.com>
Authored: Sun Jun 29 09:02:05 2014 -0700
Committer: Venkat Ranganathan <ve...@hortonworks.com>
Committed: Sun Jun 29 09:02:05 2014 -0700

----------------------------------------------------------------------
 COMPILING.txt                                   |    8 +-
 build.xml                                       |    5 +
 conf/oraoop-site-template.xml                   |  103 ++
 src/java/org/apache/sqoop/ConnFactory.java      |   10 +-
 .../sqoop/manager/oracle/OraOopConnManager.java |  630 +++++++
 .../sqoop/manager/oracle/OraOopConstants.java   |  512 ++++++
 .../manager/oracle/OraOopDBInputSplit.java      |  195 ++
 .../manager/oracle/OraOopDBRecordReader.java    |  468 +++++
 .../oracle/OraOopDataDrivenDBInputFormat.java   |  359 ++++
 .../sqoop/manager/oracle/OraOopGenerics.java    |   64 +
 .../sqoop/manager/oracle/OraOopJdbcUrl.java     |  232 +++
 .../apache/sqoop/manager/oracle/OraOopLog.java  |  235 +++
 .../sqoop/manager/oracle/OraOopLogFactory.java  |   54 +
 .../sqoop/manager/oracle/OraOopLogMessage.java  |   61 +
 .../manager/oracle/OraOopManagerFactory.java    | 1126 ++++++++++++
 .../manager/oracle/OraOopOracleDataChunk.java   |   68 +
 .../oracle/OraOopOracleDataChunkExtent.java     |   93 +
 .../oracle/OraOopOracleDataChunkPartition.java  |   78 +
 .../manager/oracle/OraOopOracleQueries.java     | 1687 ++++++++++++++++++
 .../manager/oracle/OraOopOutputFormatBase.java  |  713 ++++++++
 .../oracle/OraOopOutputFormatInsert.java        |  263 +++
 .../oracle/OraOopOutputFormatUpdate.java        |  418 +++++
 .../sqoop/manager/oracle/OraOopUtilities.java   | 1461 +++++++++++++++
 .../manager/oracle/OracleActiveInstance.java    |   44 +
 .../manager/oracle/OracleConnectionFactory.java |  217 +++
 .../sqoop/manager/oracle/OracleTable.java       |   68 +
 .../sqoop/manager/oracle/OracleTableColumn.java |   59 +
 .../manager/oracle/OracleTableColumns.java      |   43 +
 .../manager/oracle/OracleTablePartition.java    |   50 +
 .../manager/oracle/OracleTablePartitions.java   |   62 +
 .../sqoop/manager/oracle/OracleVersion.java     |   84 +
 .../com/cloudera/sqoop/manager/OracleUtils.java |   10 +
 .../sqoop/testutil/BaseSqoopTestCase.java       |    2 +
 .../sqoop/testutil/ExportJobTestCase.java       |    2 +
 .../sqoop/testutil/ImportJobTestCase.java       |    4 +
 src/test/oraoop/create_users.sql                |   49 +
 src/test/oraoop/pkg_tst_product_gen.pbk         |  126 ++
 src/test/oraoop/pkg_tst_product_gen.psk         |   45 +
 src/test/oraoop/table_tst_product.xml           |   90 +
 src/test/oraoop/table_tst_product_part.xml      |  103 ++
 .../oraoop/table_tst_product_special_chars.xml  |   90 +
 src/test/oraoop/table_tst_product_subpart.xml   |  105 ++
 .../apache/sqoop/manager/oracle/ExportTest.java |   68 +
 .../apache/sqoop/manager/oracle/ImportTest.java |  241 +++
 .../manager/oracle/OraOopOracleQueriesTest.java |   54 +
 .../sqoop/manager/oracle/OraOopTestCase.java    |  321 ++++
 .../manager/oracle/OraOopTestConstants.java     |   62 +
 .../oracle/OracleConnectionFactoryTest.java     |  520 ++++++
 .../sqoop/manager/oracle/SystemImportTest.java  |  315 ++++
 .../TestOraOopDataDrivenDBInputFormat.java      |  131 ++
 .../sqoop/manager/oracle/TestOraOopJdbcUrl.java |  276 +++
 .../manager/oracle/TestOraOopUtilities.java     |  619 +++++++
 .../sqoop/manager/oracle/TestOracleTable.java   |   42 +
 .../sqoop/manager/oracle/TimestampDataTest.java |   51 +
 .../oracle/util/BigDecimalGenerator.java        |   57 +
 .../oracle/util/BinaryDoubleGenerator.java      |   32 +
 .../oracle/util/BinaryFloatGenerator.java       |   32 +
 .../manager/oracle/util/BlobGenerator.java      |  103 ++
 .../manager/oracle/util/BytesGenerator.java     |   52 +
 .../manager/oracle/util/CharGenerator.java      |   54 +
 .../manager/oracle/util/FloatGenerator.java     |   57 +
 .../sqoop/manager/oracle/util/HadoopFiles.java  |   37 +
 .../oracle/util/IntervalDaySecondGenerator.java |   64 +
 .../oracle/util/IntervalYearMonthGenerator.java |   50 +
 .../manager/oracle/util/NCharGenerator.java     |   54 +
 .../oracle/util/OraOopTestDataGenerator.java    |   67 +
 .../manager/oracle/util/OraOopTestUtils.java    |   60 +
 .../sqoop/manager/oracle/util/OracleData.java   |  192 ++
 .../oracle/util/OracleDataDefinition.java       |   66 +
 .../oracle/util/OracleTableDefinition.java      |  150 ++
 .../manager/oracle/util/RowIdGenerator.java     |   64 +
 .../manager/oracle/util/TimestampGenerator.java |   71 +
 .../sqoop/manager/oracle/util/URIGenerator.java |   57 +
 73 files changed, 14112 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/COMPILING.txt
----------------------------------------------------------------------
diff --git a/COMPILING.txt b/COMPILING.txt
index ddbed30..ae28411 100644
--- a/COMPILING.txt
+++ b/COMPILING.txt
@@ -85,9 +85,12 @@ jdbc:mysql://localhost/
 
 === Oracle
 
-Install Oracle XE (Express edition) 10.2.0. Instructions for configuring the
+Install Oracle Enterprise Edition 10.2.0+. Instructions for configuring the
 database are in OracleManagerTest. Download the ojdbc6_g jar.
 
+If running the tests against Oracle XE (Express Edition) - a lot of them will
+fail as it does not include the partitioning feature.
+
 Use the system property sqoop.test.oracle.connectstring to specify the
 connection string for Oracle host used for testing. Specify this property on the
 command line or via the build.properties file. For example:
@@ -97,6 +100,9 @@ sqoop.test.oracle.connectstring=jdbc:oracle:thin:@//host.example.com/xe
 If not specified, the default value used for this property is:
 jdbc:oracle:thin:@//localhost/xe
 
+Users sqooptest and sqooptest2 should be created prior to running the tests.
+SQL script is available in src/test/oraoop/create_users.sql
+
 === PostgreSQL
 
 Install PostgreSQL 8.3.9. Download the postgresql 8.4 jdbc driver. Instructions

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 2dc99a8..ec5d2fa 100644
--- a/build.xml
+++ b/build.xml
@@ -827,6 +827,11 @@
     <mkdir dir="${cobertura.class.dir}" />
     <copy file="${test.dir}/fi-site.xml"
             todir="${test.build.extraconf}" />
+    <copy file="${basedir}/conf/oraoop-site-template.xml"
+            todir="${test.build.extraconf}" />
+    <copy todir="${test.build.extraconf}/oraoop">
+      <fileset dir="${test.dir}/oraoop"/>
+    </copy>
     <junit
       printsummary="yes" showoutput="${test.output}"
       haltonfailure="no" fork="yes" maxmemory="512m"

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/conf/oraoop-site-template.xml
----------------------------------------------------------------------
diff --git a/conf/oraoop-site-template.xml b/conf/oraoop-site-template.xml
new file mode 100644
index 0000000..eac9796
--- /dev/null
+++ b/conf/oraoop-site-template.xml
@@ -0,0 +1,103 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  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.
+-->
+
+<!-- Put OraOop-specific properties in this file. -->
+
+<configuration>
+
+  <property>
+    <name>oraoop.oracle.session.initialization.statements</name>
+    <value>alter session disable parallel query;
+           alter session set "_serial_direct_read"=true;
+           alter session set tracefile_identifier=oraoop;
+           --alter session set events '10046 trace name context forever, level 8';
+    </value>
+    <description>A semicolon-delimited list of Oracle statements that are executed, in order, to initialize each Oracle session.
+                 Use {[property_name]|[default_value]} characters to refer to a Sqoop/Hadoop configuration property.
+                 If the property does not exist, the specified default value will be used.
+                 E.g. {oracle.sessionTimeZone|GMT} will equate to the value of the property named "oracle.sessionTimeZone" or
+                 to "GMT" if this property has not been set.
+    </description>
+  </property>
+
+  <property>
+    <name>mapred.map.tasks.speculative.execution</name>
+    <value>false</value>
+    <description>Speculative execution is disabled to prevent redundant load on the Oracle database.
+    </description>
+  </property>
+
+  <property>
+    <name>oraoop.import.hint</name>
+    <value>NO_INDEX(t)</value>
+    <description>Hint to add to the SELECT statement for an IMPORT job.
+                 The table will have an alias of t which can be used in the hint.
+                 By default the NO_INDEX hint is applied to stop the use of an index.
+                 To override this in oraoop-site.xml set the value to a blank string.
+    </description>
+  </property>
+
+<!--
+  <property>
+    <name>oraoop.block.allocation</name>
+    <value>ROUNDROBIN</value>
+    <description>Supported values are: ROUNDROBIN or SEQUENTIAL or RANDOM.
+                 Refer to the OraOop documentation for more details.
+    </description>
+  </property>
+-->
+
+<!--
+  <property>
+    <name>oraoop.import.omit.lobs.and.long</name>
+    <value>false</value>
+    <description>If true, OraOop will omit BLOB, CLOB, NCLOB and LONG columns during an Import.
+    </description>
+  </property>
+-->
+
+<!--
+  <property>
+    <name>oraoop.table.import.where.clause.location</name>
+    <value>SUBSPLIT</value>
+    <description>Supported values are: SUBSPLIT or SPLIT.
+                 Refer to the OraOop documentation for more details.
+    </description>
+  </property>
+-->
+
+<!--
+  <property>
+    <name>oraoop.oracle.append.values.hint.usage</name>
+    <value>AUTO</value>
+    <description>Supported values are: AUTO or ON or OFF.
+                 ON:
+                     OraOop will use the APPEND_VALUES Oracle hint during a Sqoop export, when inserting
+                     data into an Oracle table.
+                 OFF:
+                     OraOop will not use the APPEND_VALUES Oracle hint during a Sqoop export.
+                 AUTO:
+                     For OraOop 1.1, the AUTO setting will not use the APPEND_VALUES hint.
+    </description>
+  </property>
+-->
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/ConnFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/ConnFactory.java b/src/java/org/apache/sqoop/ConnFactory.java
index 61d3307..2276525 100644
--- a/src/java/org/apache/sqoop/ConnFactory.java
+++ b/src/java/org/apache/sqoop/ConnFactory.java
@@ -43,6 +43,7 @@ import com.cloudera.sqoop.metastore.JobData;
 
 import com.cloudera.sqoop.util.ClassLoaderStack;
 import org.apache.sqoop.manager.GenericJdbcManager;
+import org.apache.sqoop.manager.oracle.OraOopManagerFactory;
 
 /**
  * Factory class to create the ConnManager type required
@@ -70,8 +71,12 @@ public class ConnFactory {
 
   // The default value for sqoop.connection.factories is the
   // name of the DefaultManagerFactory.
+  public static final String[] DEFAULT_FACTORY_CLASS_NAMES_ARR =
+      {OraOopManagerFactory.class.getName(),
+      DefaultManagerFactory.class.getName(), };
+
   public static final String DEFAULT_FACTORY_CLASS_NAMES =
-      DefaultManagerFactory.class.getName();
+      StringUtils.arrayToString(DEFAULT_FACTORY_CLASS_NAMES_ARR);
 
   /** The list of ManagerFactory instances consulted by getManager().
    */
@@ -84,7 +89,8 @@ public class ConnFactory {
   private void instantiateFactories(Configuration conf) {
     loadManagersFromConfDir(conf);
     String [] classNameArray =
-        conf.getStrings(FACTORY_CLASS_NAMES_KEY, DEFAULT_FACTORY_CLASS_NAMES);
+        conf.getStrings(FACTORY_CLASS_NAMES_KEY,
+            DEFAULT_FACTORY_CLASS_NAMES_ARR);
 
     for (String className : classNameArray) {
       try {

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopConnManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopConnManager.java b/src/java/org/apache/sqoop/manager/oracle/OraOopConnManager.java
new file mode 100644
index 0000000..302849c
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopConnManager.java
@@ -0,0 +1,630 @@
+/**
+ * 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.Connection;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.OutputFormat;
+
+import com.cloudera.sqoop.SqoopOptions;
+import com.cloudera.sqoop.manager.ExportJobContext;
+import com.cloudera.sqoop.manager.GenericJdbcManager;
+import com.cloudera.sqoop.manager.ImportJobContext;
+import com.cloudera.sqoop.mapreduce.JdbcExportJob;
+import com.cloudera.sqoop.mapreduce.JdbcUpdateExportJob;
+import com.cloudera.sqoop.util.ExportException;
+import com.cloudera.sqoop.util.ImportException;
+
+/**
+ * OraOop manager for high performance Oracle import / export.
+ * NOTES:
+ *   Escaping Column Names:
+ *   ----------------------
+ *   There are 3 main queries that occur during a Sqoop import.
+ *   (1) Selecting columns to obtain their data-type via getColTypesQuery();
+ *   (2) selecting column names via getColNamesQuery(); and
+ *   (3) getting the data during the import via
+ *       OraOopDBRecordReader.getSelectQuery();
+ *   In each of these queries, we'd ideally escape the column names so that
+ *   Oracle columns that require this work okay.
+ *   Unfortunately we can't do this, because if the user specifies column
+ *   names via the "--columns" clause, these names will be used (verbatim)
+ *   during OraOopDBRecordReader.getSelectQuery(). This means that we could
+ *   only escape the column names during OraOopDBRecordReader.getSelectQuery()
+ *   if the user entered them in the correct case.
+ *   Therefore, escapeColName() in this class does not actually do anything so
+ *   that OraOopDBRecordReader.getSelectQuery() generates a valid SQL statement
+ *   when the user utilises the "--columns" clause.
+ *   However, getColTypesQuery() and getColNamesQuery() do escape column names
+ *   via the method escapeOracleColumnName(). We also get getColumnTypes() to
+ *   unescape the column names so that Sqoop has the most accurate column
+ *   name strings.
+ */
+public class OraOopConnManager extends GenericJdbcManager {
+
+  public static final OraOopLog LOG = OraOopLogFactory
+      .getLog(OraOopConnManager.class.getName());
+  private List<String> columnNamesInOracleTable = null;
+  private Map<String, Integer> columnTypesInOracleTable = null;
+  private final String timestampJavaType;
+
+  public OraOopConnManager(final SqoopOptions sqoopOptions) {
+    super(OraOopConstants.ORACLE_JDBC_DRIVER_CLASS, sqoopOptions);
+    if (this.options.getConf().getBoolean(
+        OraOopConstants.ORAOOP_MAP_TIMESTAMP_AS_STRING,
+        OraOopConstants.ORAOOP_MAP_TIMESTAMP_AS_STRING_DEFAULT)) {
+      timestampJavaType = "String";
+    } else {
+      timestampJavaType = super.toJavaType(Types.TIMESTAMP);
+    }
+  }
+
+  @Override
+  protected Connection makeConnection() throws SQLException {
+
+    String connectStr = this.options.getConnectString();
+    String username = this.options.getUsername();
+    String password = this.options.getPassword();
+    Properties additionalProps = this.options.getConnectionParams();
+
+    Connection connection =
+        OracleConnectionFactory.createOracleJdbcConnection(this
+            .getDriverClass(), connectStr, username, password, additionalProps);
+
+    return connection;
+  }
+
+  @Override
+  public void close() throws SQLException {
+
+    super.close();
+  }
+
+  private List<String> getColumnNamesInOracleTable(String tableName) {
+
+    if (this.columnNamesInOracleTable == null) {
+
+      OracleTable tableContext = null;
+
+      try {
+        tableContext = getOracleTableContext();
+
+        Configuration conf = this.options.getConf();
+
+        this.columnNamesInOracleTable =
+            OraOopOracleQueries.getTableColumnNames(getConnection(),
+                tableContext, OraOopUtilities
+                    .omitLobAndLongColumnsDuringImport(conf), OraOopUtilities
+                    .recallSqoopJobType(conf), true, // <-
+                                                     // onlyOraOopSupportedTypes
+                true // <- omitOraOopPseudoColumns
+                );
+      } catch (SQLException ex) {
+        throw new RuntimeException(ex);
+      }
+    }
+
+    // Return a copy of our list, so the original will not be inadvertently
+    // altered...
+    return OraOopUtilities.copyStringList(this.columnNamesInOracleTable);
+  }
+
+  protected List<String> getSelectedColumnNamesInOracleTable(String tableName) {
+
+    List<String> colNamesInTable = getColumnNamesInOracleTable(tableName);
+
+    String[] selectedColumns = this.options.getColumns();
+    if (selectedColumns != null && selectedColumns.length > 0) {
+
+      for (int idx = 0; idx < selectedColumns.length; idx++) {
+
+        String selectedColumn = selectedColumns[idx];
+        // If the user did not escape this column name, then we should
+        // uppercase it...
+        if (!isEscaped(selectedColumn)) {
+          selectedColumns[idx] = selectedColumn.toUpperCase();
+        } else {
+          // If the user escaped this column name, then we should
+          // retain its case...
+          selectedColumns[idx] = unescapeOracleColumnName(selectedColumn);
+        }
+      }
+
+      // Ensure there are no duplicated column names...
+      String[] duplicates =
+          OraOopUtilities
+              .getDuplicatedStringArrayValues(selectedColumns, false);
+      if (duplicates.length > 0) {
+        StringBuilder msg = new StringBuilder();
+        msg.append("The following column names have been duplicated in the ");
+        msg.append("\"--columns\" clause:\n");
+
+        for (String duplicate : duplicates) {
+          msg.append("\t" + duplicate + "\n");
+        }
+
+        throw new RuntimeException(msg.toString());
+      }
+
+      // Ensure the user selected column names that actually exist...
+      for (String selectedColumn : selectedColumns) {
+        if (!colNamesInTable.contains(selectedColumn)) {
+          OracleTable tableContext = getOracleTableContext();
+          throw new RuntimeException(String.format(
+              "The column named \"%s\" does not exist within the table"
+                  + "%s (or is of an unsupported data-type).", selectedColumn,
+              tableContext.toString()));
+        }
+      }
+
+      // Remove any columns (that exist in the table) that were not
+      // selected by the user...
+      for (int idx = colNamesInTable.size() - 1; idx >= 0; idx--) {
+        String colName = colNamesInTable.get(idx);
+        if (!OraOopUtilities.stringArrayContains(selectedColumns, colName,
+            false)) {
+          colNamesInTable.remove(idx);
+        }
+      }
+    }
+
+    // To assist development/testing of Oracle data-types, you can use this
+    // to limit the number of columns from the table...
+    int columnNameLimit =
+        this.options.getConf().getInt("oraoop.column.limit", 0);
+    if (columnNameLimit > 0) {
+      columnNameLimit = Math.min(columnNameLimit, colNamesInTable.size());
+      colNamesInTable = colNamesInTable.subList(0, columnNameLimit);
+    }
+
+    return colNamesInTable;
+  }
+
+  @Override
+  protected String getColTypesQuery(String tableName) {
+
+    List<String> colNames = getSelectedColumnNamesInOracleTable(tableName);
+
+    StringBuilder sb = new StringBuilder();
+    sb.append("SELECT ");
+    for (int idx = 0; idx < colNames.size(); idx++) {
+      if (idx > 0) {
+        sb.append(",");
+      }
+      sb.append(escapeOracleColumnName(colNames.get(idx))); // <- See notes at
+                                                            // top about escaped
+                                                            // column names
+    }
+    sb.append(String.format(" FROM %s WHERE 0=1", tableName));
+
+    return sb.toString();
+  }
+
+  @Override
+  protected String getColNamesQuery(String tableName) {
+
+    // NOTE: This code is similar to getColTypesQuery() - except the
+    // escaping of column names and table name differs.
+
+    List<String> colNames = getSelectedColumnNamesInOracleTable(tableName);
+
+    StringBuilder sb = new StringBuilder();
+    sb.append("SELECT ");
+    for (int idx = 0; idx < colNames.size(); idx++) {
+      if (idx > 0) {
+        sb.append(",");
+      }
+      sb.append(escapeColName(colNames.get(idx))); // <- See notes at top about
+                                                   // escaped column names
+    }
+    sb.append(String.format(" FROM %s WHERE 1=0", escapeTableName(tableName)));
+
+    return sb.toString();
+  }
+
+  @Override
+  protected String getSplitColumn(SqoopOptions opts, String tableName) {
+
+    // If we're importing an Oracle table and will be generating
+    // "splits" based on its Oracle data-files, we don't actually require
+    // a primary key to exist, or for the user to identify the split-column.
+    // As a consequence, return "NotRequired" to prevent sqoop code
+    // such as SqlManager.importTable() from throwing an exception.
+    //
+    // NB: The tableName parameter will be null if no table is involved,
+    // such as when importing data via an (arbitrary) SQL query.
+    if (tableName != null) {
+      return OraOopConstants.TABLE_SPLIT_COLUMN_NOT_REQUIRED;
+    } else {
+      return super.getSplitColumn(opts, tableName);
+    }
+  }
+
+  @Override
+  public void importTable(ImportJobContext context) throws IOException,
+      ImportException {
+
+    logImportTableDetails(context);
+
+    context.setConnManager(this);
+
+    // Specify the Oracle-specific DBInputFormat for import.
+    context.setInputFormat(OraOopDataDrivenDBInputFormat.class);
+
+    super.importTable(context);
+  }
+
+  @Override
+  public void exportTable(ExportJobContext context) throws IOException,
+      ExportException {
+
+    logExportTableDetails(context);
+
+    if (this.columnTypesInOracleTable == null) {
+      throw new ExportException("The column-types for the table are not"
+          + "known.");
+    }
+    if (this.columnTypesInOracleTable.containsValue(OraOopOracleQueries
+        .getOracleType("BINARY_DOUBLE"))) {
+      context.getOptions().getConf().setBoolean(
+          OraOopConstants.TABLE_CONTAINS_BINARY_DOUBLE_COLUMN, true);
+    }
+    if (this.columnTypesInOracleTable.containsValue(OraOopOracleQueries
+        .getOracleType("BINARY_FLOAT"))) {
+      context.getOptions().getConf().setBoolean(
+          OraOopConstants.TABLE_CONTAINS_BINARY_FLOAT_COLUMN, true);
+    }
+
+    context.setConnManager(this);
+
+    @SuppressWarnings("rawtypes")
+    Class<? extends OutputFormat> oraOopOutputFormatClass;
+    try {
+      oraOopOutputFormatClass = OraOopOutputFormatInsert.class;
+    } catch (NoClassDefFoundError ex) {
+      explainWhyExportClassCannotBeLoaded(ex, "OraOopOutputFormatInsert");
+      throw ex;
+    }
+    JdbcExportJob exportJob =
+        new JdbcExportJob(context, null, null, oraOopOutputFormatClass);
+    exportJob.runExport();
+  }
+
+  @Override
+  public void updateTable(ExportJobContext context) throws IOException,
+      ExportException {
+
+    logExportTableDetails(context);
+
+    context.setConnManager(this);
+
+    @SuppressWarnings("rawtypes")
+    Class<? extends OutputFormat> oraOopOutputFormatClass;
+    try {
+      oraOopOutputFormatClass = OraOopOutputFormatUpdate.class;
+    } catch (NoClassDefFoundError ex) {
+      explainWhyExportClassCannotBeLoaded(ex, "OraOopOutputFormatUpdate");
+      throw ex;
+    }
+
+    JdbcUpdateExportJob exportJob =
+        new JdbcUpdateExportJob(context, null, null, oraOopOutputFormatClass);
+    exportJob.runExport();
+  }
+
+  @Override
+  protected void finalize() throws Throwable {
+
+    close();
+    super.finalize();
+  }
+
+  @Override
+  public String toHiveType(int sqlType) {
+
+    String hiveType = super.toHiveType(sqlType);
+
+    if (hiveType == null) {
+
+      // http://wiki.apache.org/hadoop/Hive/Tutorial#Primitive_Types
+
+      if (sqlType == OraOopOracleQueries.getOracleType("BFILE")
+          || sqlType == OraOopOracleQueries.getOracleType("INTERVALYM")
+          || sqlType == OraOopOracleQueries.getOracleType("INTERVALDS")
+          || sqlType == OraOopOracleQueries.getOracleType("NCLOB")
+          || sqlType == OraOopOracleQueries.getOracleType("NCHAR")
+          || sqlType == OraOopOracleQueries.getOracleType("NVARCHAR")
+          || sqlType == OraOopOracleQueries.getOracleType("OTHER")
+          || sqlType == OraOopOracleQueries.getOracleType("ROWID")
+          || sqlType == OraOopOracleQueries.getOracleType("TIMESTAMPTZ")
+          || sqlType == OraOopOracleQueries.getOracleType("TIMESTAMPLTZ")
+          || sqlType == OraOopOracleQueries.getOracleType("STRUCT")) {
+        hiveType = "STRING";
+      }
+
+      if (sqlType == OraOopOracleQueries.getOracleType("BINARY_FLOAT")) {
+        hiveType = "FLOAT";
+      }
+
+      if (sqlType == OraOopOracleQueries.getOracleType("BINARY_DOUBLE")) {
+        hiveType = "DOUBLE";
+      }
+    }
+
+    if (hiveType == null) {
+      LOG.warn(String.format("%s should be updated to cater for data-type: %d",
+          OraOopUtilities.getCurrentMethodName(), sqlType));
+    }
+
+    return hiveType;
+  }
+
+  @Override
+  public String toJavaType(int sqlType) {
+
+    String javaType = super.toJavaType(sqlType);
+
+    if (sqlType == OraOopOracleQueries.getOracleType("TIMESTAMP")) {
+      // Get the Oracle JDBC driver to convert this value to a string
+      // instead of the generic JDBC driver.
+      // If the generic JDBC driver is used, it will take into account the
+      // timezone of the client machine's locale. The problem with this is that
+      // timestamp data should not be associated with a timezone. In practice,
+      // this
+      // leads to problems, for example, the time '2010-10-03 02:01:00' being
+      // changed to '2010-10-03 03:01:00' if the client machine's locale is
+      // Melbourne.
+      // (This is in response to daylight saving starting in Melbourne on
+      // this date at 2am.)
+      javaType = timestampJavaType;
+    }
+
+    if (sqlType == OraOopOracleQueries.getOracleType("TIMESTAMPTZ")) {
+      // Returning "String" produces: "2010-08-08 09:00:00.0 +10:00"
+      // Returning "java.sql.Timestamp" produces: "2010-08-08 09:00:00.0"
+
+      // If we use "java.sql.Timestamp", the field's value will not
+      // contain the timezone when converted to a string and written to the HDFS
+      // CSV file.
+      // I.e. Get the Oracle JDBC driver to convert this value to a string
+      // instead of the generic JDBC driver...
+      javaType = timestampJavaType;
+    }
+
+    if (sqlType == OraOopOracleQueries.getOracleType("TIMESTAMPLTZ")) {
+      // Returning "String" produces:
+      // "2010-08-08 09:00:00.0 Australia/Melbourne"
+      // Returning "java.sql.Timestamp" produces: "2010-08-08 09:00:00.0"
+      javaType = timestampJavaType;
+    }
+
+    /*
+     * http://www.oracle.com/technology/sample_code/tech/java/sqlj_jdbc/files
+     * /oracle10g/ieee/Readme.html
+     *
+     * BINARY_DOUBLE is a 64-bit, double-precision floating-point number
+     * datatype. (IEEE 754) Each BINARY_DOUBLE value requires 9 bytes, including
+     * a length byte. A 64-bit double format number X is divided as sign s 1-bit
+     * exponent e 11-bits fraction f 52-bits
+     *
+     * BINARY_FLOAT is a 32-bit, single-precision floating-point number
+     * datatype. (IEEE 754) Each BINARY_FLOAT value requires 5 bytes, including
+     * a length byte. A 32-bit single format number X is divided as sign s 1-bit
+     * exponent e 8-bits fraction f 23-bits
+     */
+    if (sqlType == OraOopOracleQueries.getOracleType("BINARY_FLOAT")) {
+      // http://people.uncw.edu/tompkinsj/133/numbers/Reals.htm
+      javaType = "Float";
+    }
+
+    if (sqlType == OraOopOracleQueries.getOracleType("BINARY_DOUBLE")) {
+      // http://people.uncw.edu/tompkinsj/133/numbers/Reals.htm
+      javaType = "Double";
+    }
+
+    if (sqlType == OraOopOracleQueries.getOracleType("STRUCT")) {
+      // E.g. URITYPE
+      javaType = "String";
+    }
+
+    if (javaType == null) {
+
+      // For constant values, refer to:
+      // http://oracleadvisor.com/documentation/oracle/database/11.2/
+      //   appdev.112/e13995/constant-values.html#oracle_jdbc
+
+      if (sqlType == OraOopOracleQueries.getOracleType("BFILE")
+          || sqlType == OraOopOracleQueries.getOracleType("NCLOB")
+          || sqlType == OraOopOracleQueries.getOracleType("NCHAR")
+          || sqlType == OraOopOracleQueries.getOracleType("NVARCHAR")
+          || sqlType == OraOopOracleQueries.getOracleType("ROWID")
+          || sqlType == OraOopOracleQueries.getOracleType("INTERVALYM")
+          || sqlType == OraOopOracleQueries.getOracleType("INTERVALDS")
+          || sqlType == OraOopOracleQueries.getOracleType("OTHER")) {
+        javaType = "String";
+      }
+
+    }
+
+    if (javaType == null) {
+      LOG.warn(String.format("%s should be updated to cater for data-type: %d",
+          OraOopUtilities.getCurrentMethodName(), sqlType));
+    }
+
+    return javaType;
+  }
+
+  @Override
+  public String timestampToQueryString(Timestamp ts) {
+
+    return "TO_TIMESTAMP('" + ts + "', 'YYYY-MM-DD HH24:MI:SS.FF')";
+  }
+
+  public OracleTable getOracleTableContext() {
+
+    return OraOopUtilities.decodeOracleTableName(this.options.getUsername(),
+        this.options.getTableName(), this.options.getConf());
+  }
+
+  @Override
+  public Map<String, Integer> getColumnTypes(String tableName) {
+
+    if (this.columnTypesInOracleTable == null) {
+
+      Map<String, Integer> columnTypes = super.getColumnTypes(tableName);
+      this.columnTypesInOracleTable = new HashMap<String, Integer>();
+
+      List<String> colNames = getColumnNamesInOracleTable(tableName);
+
+      for (int idx = 0; idx < colNames.size(); idx++) {
+
+        String columnNameInTable = colNames.get(idx);
+        if (columnTypes.containsKey(columnNameInTable)) {
+
+          // Unescape the column names being returned...
+          int colType = columnTypes.get(columnNameInTable);
+          String key = unescapeOracleColumnName(columnNameInTable); // <- See
+                                                                    // notes at
+                                                                    // top about
+                                                                    // escaped
+                                                                    // column
+                                                                    // names
+          this.columnTypesInOracleTable.put(key, colType);
+        }
+      }
+    }
+
+    return this.columnTypesInOracleTable;
+  }
+
+  private boolean isEscaped(String name) {
+
+    return name.startsWith("\"") && name.endsWith("\"");
+  }
+
+  private String escapeOracleColumnName(String columnName) {
+    // See notes at top about escaped column names
+    if (isEscaped(columnName)) {
+      return columnName;
+    } else {
+      return "\"" + columnName + "\"";
+    }
+  }
+
+  @Override
+  public String escapeColName(String colName) {
+
+    return super.escapeColName(colName); // <- See notes at top about escaped
+                                         // column names
+  }
+
+  private String unescapeOracleColumnName(String columnName) {
+
+    if (isEscaped(columnName)) {
+      return columnName.substring(1, columnName.length() - 1);
+    } else {
+      return columnName;
+    }
+  }
+
+  private void logImportTableDetails(ImportJobContext context) {
+
+    Path outputDirectory = context.getDestination();
+    if (outputDirectory != null) {
+      LOG.debug("The output directory for the sqoop table import is : "
+          + outputDirectory.getName());
+    }
+
+    // Indicate whether we can load the class named: OraOopOraStats
+    showUserWhetherOraOopOraStatsIsAvailable(context.getOptions().getConf());
+  }
+
+  private void logExportTableDetails(ExportJobContext context) {
+
+    // Indicate whether we can load the class named: OraOopOraStats
+    showUserWhetherOraOopOraStatsIsAvailable(context.getOptions().getConf());
+
+    // Indicate what the update/merge columns are...
+    String[] updateKeyColumns =
+        OraOopUtilities.getExportUpdateKeyColumnNames(context.getOptions());
+    if (updateKeyColumns.length > 0) {
+      LOG.info(String.format(
+          "The column%s used to match rows in the HDFS file with rows in "
+              + "the Oracle table %s: %s", updateKeyColumns.length > 1 ? "s"
+              : "", updateKeyColumns.length > 1 ? "are" : "is", OraOopUtilities
+              .stringArrayToCSV(updateKeyColumns)));
+    }
+  }
+
+  private void showUserWhetherOraOopOraStatsIsAvailable(Configuration conf) {
+
+    if (OraOopUtilities.userWantsOracleSessionStatisticsReports(conf)) {
+
+      LOG.info(String.format("%s=true",
+          OraOopConstants.ORAOOP_REPORT_SESSION_STATISTICS));
+
+      // This will log a warning if it's unable to load the OraOopOraStats
+      // class...
+      OraOopUtilities.startSessionSnapshot(null);
+    }
+  }
+
+  @Override
+  protected String getCurTimestampQuery() {
+
+    return "SELECT SYSTIMESTAMP FROM DUAL";
+  }
+
+  @Override
+  protected void checkTableImportOptions(ImportJobContext context)
+      throws IOException, ImportException {
+
+    // Update the unit-test code if you modify this method.
+    super.checkTableImportOptions(context);
+  }
+
+  private void explainWhyExportClassCannotBeLoaded(NoClassDefFoundError ex,
+      String exportClassName) {
+
+    String msg =
+        String.format("Unable to load class %s.\n"
+            + "This is most likely caused by the Cloudera Shim Jar "
+            + "not being included in the Java Classpath.\n" + "Either:\n"
+            + "\tUse \"-libjars\" on the Sqoop command-line to "
+            + "include the Cloudera shim jar in the Java Classpath; or"
+            + "\n\tCopy the Cloudera shim jar into the Sqoop/lib "
+            + "directory so that it is automatically included in the "
+            + "Java Classpath; or\n"
+            + "\tObtain an updated version of Sqoop that addresses "
+            + "the Sqoop Jira \"SQOOP-127\".\n" + "\n"
+            + "The Java Classpath is:\n%s", exportClassName, OraOopUtilities
+            .getJavaClassPath());
+    LOG.fatal(msg, ex);
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopConstants.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopConstants.java b/src/java/org/apache/sqoop/manager/oracle/OraOopConstants.java
new file mode 100644
index 0000000..874ef02
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopConstants.java
@@ -0,0 +1,512 @@
+/**
+ * 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;
+
+/**
+ * Constants for OraOop.
+ */
+public final class OraOopConstants {
+  private OraOopConstants() {
+  }
+
+  public static final String ORAOOP_PRODUCT_NAME =
+      "Data Connector for Oracle and Hadoop";
+  public static final String ORAOOP_JAR_FILENAME = "oraoop.jar";
+
+  // Disables OraOop - falling back to the OracleManager inside Sqoop...
+  public static final String ORAOOP_DISABLED = "oraoop.disabled";
+
+  // Whether to log Oracle session statistics using Guy Harrison's jar file...
+  public static final String ORAOOP_REPORT_SESSION_STATISTICS =
+      "oraoop.report.session.statistics";
+
+  // Disables dynamic JDBC URL generation for each mapper...
+  public static final String ORAOOP_JDBC_URL_VERBATIM =
+      "oraoop.jdbc.url.verbatim";
+
+  // The name of the Oracle RAC service each mapper should connect to, via their
+  // dynamically generated JDBC URL...
+  public static final String ORAOOP_ORACLE_RAC_SERVICE_NAME =
+      "oraoop.oracle.rac.service.name";
+
+  // The log4j log-level for OraOop...
+  public static final String ORAOOP_LOGGING_LEVEL = "oraoop.logging.level";
+
+  // The file names for the configuration properties of OraOop...
+  public static final String ORAOOP_SITE_TEMPLATE_FILENAME =
+      "oraoop-site-template.xml";
+  public static final String ORAOOP_SITE_FILENAME = "oraoop-site.xml";
+
+  // A flag that indicates that the OraOop job has been cancelled.
+  // E.g. An Oracle DBA killed our Oracle session.
+  // public static final String ORAOOP_JOB_CANCELLED = "oraoop.job.cancelled";
+
+  // The SYSDATE from the Oracle database when this OraOop job was started.
+  // This is used to generate unique names for partitions and temporary tables
+  // that we create during the job...
+  public static final String ORAOOP_JOB_SYSDATE = "oraoop.job.sysdate";
+
+  // The properties are used internally by OraOop to indicate the schema and
+  // name of
+  // the table being imported/exported...
+  public static final String ORAOOP_TABLE_OWNER = "oraoop.table.owner";
+  public static final String ORAOOP_TABLE_NAME = "oraoop.table.name";
+
+  // Constants used to indicate the desired location of the WHERE clause within
+  // the SQL generated by the record-reader.
+  // E.g. A WHERE clause like "rownum <= 10" would want to be located so that
+  // it had an impact on the total number of rows returned by the split;
+  // as opposed to impacting the number of rows returned for each of the
+  // unioned data-chunks within each split.
+  public static final String ORAOOP_TABLE_IMPORT_WHERE_CLAUSE_LOCATION =
+      "oraoop.table.import.where.clause.location";
+
+  /**
+   * Location to place the WHERE clause.
+   */
+  public enum OraOopTableImportWhereClauseLocation {
+    SUBSPLIT, SPLIT
+  }
+
+  // The SQL statements to execute for each new Oracle session that is
+  // created...
+  public static final String ORAOOP_SESSION_INITIALIZATION_STATEMENTS =
+      "oraoop.oracle.session.initialization.statements";
+
+  // Reliably stores the number mappers requested for the sqoop map-reduce
+  // job...
+  public static final String ORAOOP_DESIRED_NUMBER_OF_MAPPERS =
+      "oraoop.desired.num.mappers";
+
+  // The minimum number of mappers required for OraOop to accept the import
+  // job...
+  public static final String ORAOOP_MIN_IMPORT_MAPPERS =
+      "oraoop.min.import.mappers";
+  public static final int MIN_NUM_IMPORT_MAPPERS_ACCEPTED_BY_ORAOOP = 2;
+
+  // The minimum number of mappers required for OraOop to accept the export
+  // job...
+  public static final String ORAOOP_MIN_EXPORT_MAPPERS =
+      "oraoop.min.export.mappers";
+  public static final int MIN_NUM_EXPORT_MAPPERS_ACCEPTED_BY_ORAOOP = 2;
+
+  // The query used to fetch oracle data chunks...
+  public static final String ORAOOP_ORACLE_DATA_CHUNKS_QUERY =
+      "oraoop.oracle.data.chunks.query";
+
+  // The minimum number of active instances in an Oracle RAC required for OraOop
+  // to use dynamically generated JDBC URLs...
+  public static final String ORAOOP_MIN_RAC_ACTIVE_INSTANCES =
+      "oraoop.min.rac.active.instances";
+  public static final int MIN_NUM_RAC_ACTIVE_INSTANCES_FOR_DYNAMIC_JDBC_URLS =
+      2;
+
+  // The name of the Oracle JDBC class...
+  public static final String ORACLE_JDBC_DRIVER_CLASS =
+      "oracle.jdbc.OracleDriver";
+
+  // How many rows to pre-fetch when executing Oracle queries...
+  public static final String ORACLE_ROW_FETCH_SIZE = "oracle.row.fetch.size";
+  public static final int ORACLE_ROW_FETCH_SIZE_DEFAULT = 5000;
+
+  // OraOop does not require a "--split-by" column to be defined...
+  public static final String TABLE_SPLIT_COLUMN_NOT_REQUIRED = "not-required";
+
+  // The name of the data_chunk_id column the OraOop appends to each (import)
+  // query...
+  public static final String COLUMN_NAME_DATA_CHUNK_ID = "data_chunk_id";
+
+  // The hint that will be used on the SELECT statement for import jobs
+  public static final String IMPORT_QUERY_HINT = "oraoop.import.hint";
+
+  // Pseudo-columns added to an partitioned export table (created by OraOop from
+  // a template table)
+  // to store the partition value and subpartition value. The partition value is
+  // the sysdate when
+  // the job was performed. The subpartition value is the mapper index...
+  public static final String COLUMN_NAME_EXPORT_PARTITION =
+      "ORAOOP_EXPORT_SYSDATE";
+  public static final String COLUMN_NAME_EXPORT_SUBPARTITION =
+      "ORAOOP_MAPPER_ID";
+  public static final String COLUMN_NAME_EXPORT_MAPPER_ROW =
+      "ORAOOP_MAPPER_ROW";
+
+  public static final String ORAOOP_EXPORT_PARTITION_DATE_VALUE =
+      "oraoop.export.partition.date.value";
+  public static final String ORAOOP_EXPORT_PARTITION_DATE_FORMAT =
+      "yyyy-mm-dd hh24:mi:ss";
+
+  // The string we want to pass to dbms_application_info.set_module() via the
+  // "module_name" parameter...
+  public static final String ORACLE_SESSION_MODULE_NAME = ORAOOP_PRODUCT_NAME;
+
+  // The name of the configuration property containing the string we want to
+  // pass to
+  // dbms_application_info.set_module() via the "action_name" parameter...
+  public static final String ORACLE_SESSION_ACTION_NAME =
+      "oraoop.oracle.session.module.action";
+
+  // Boolean whether to do a consistent read based off an SCN
+  public static final String ORAOOP_IMPORT_CONSISTENT_READ =
+      "oraoop.import.consistent.read";
+
+  // The SCN number to use for the consistent read - calculated automatically -
+  // cannot be overridden
+  public static final String ORAOOP_IMPORT_CONSISTENT_READ_SCN =
+      "oraoop.import.consistent.read.scn";
+
+  // The method that will be used to create data chunks - ROWID ranges or
+  // partitions
+  public static final String ORAOOP_ORACLE_DATA_CHUNK_METHOD =
+      "oraoop.chunk.method";
+
+  /**
+   * How should data be split up - by ROWID range, or by partition.
+   */
+  public enum OraOopOracleDataChunkMethod {
+    ROWID, PARTITION
+  }
+
+  // List of partitions to be imported, comma seperated list
+  public static final String ORAOOP_IMPORT_PARTITION_LIST =
+      "oraoop.import.partitions";
+
+  public static final OraOopOracleDataChunkMethod
+                          ORAOOP_ORACLE_DATA_CHUNK_METHOD_DEFAULT =
+                              OraOopOracleDataChunkMethod.ROWID;
+
+  // How to allocate data-chunks into splits...
+  public static final String ORAOOP_ORACLE_BLOCK_TO_SPLIT_ALLOCATION_METHOD =
+      "oraoop.block.allocation";
+
+  /**
+   * How splits should be allocated to the mappers.
+   */
+  public enum OraOopOracleBlockToSplitAllocationMethod {
+    ROUNDROBIN, SEQUENTIAL, RANDOM
+  }
+
+  // Whether to omit LOB and LONG columns during an import...
+  public static final String ORAOOP_IMPORT_OMIT_LOBS_AND_LONG =
+      "oraoop.import.omit.lobs.and.long";
+
+  // Identifies an existing Oracle table used to create a new table as the
+  // destination of a Sqoop export.
+  // Hence, use of this property implies that the "-table" does not exist in
+  // Oracle and OraOop should create it.
+  public static final String ORAOOP_EXPORT_CREATE_TABLE_TEMPLATE =
+      "oraoop.template.table";
+
+  // If the table already exists that we want to create, should we drop it?...
+  public static final String ORAOOP_EXPORT_CREATE_TABLE_DROP =
+      "oraoop.drop.table";
+
+  // If ORAOOP_EXPORT_CREATE_TABLE_TEMPLATE has been specified, then this flag
+  // indicates whether the created Oracle
+  // tables should have NOLOGGING...
+  public static final String ORAOOP_EXPORT_CREATE_TABLE_NO_LOGGING =
+      "oraoop.no.logging";
+
+  // If ORAOOP_EXPORT_CREATE_TABLE_TEMPLATE has been specified, then this flag
+  // indicates whether the created Oracle
+  // tables should be partitioned by job and mapper...
+  public static final String ORAOOP_EXPORT_CREATE_TABLE_PARTITIONED =
+      "oraoop.partitioned";
+
+  // Indicates (internally) the the export table we're dealling with has been
+  // paritioned by OraOop...
+  public static final String EXPORT_TABLE_HAS_ORAOOP_PARTITIONS =
+      "oraoop.export.table.has.oraoop.partitions";
+
+  // When using the Oracle hint... /* +APPEND_VALUES */ ...a commit must be
+  // performed after each batch insert.
+  // Therefore, the batches need to be quite large to avoid a performance
+  // penality (for the 'extra' commits).
+  // This is the minimum batch size to use under these conditions...
+  public static final String ORAOOP_MIN_APPEND_VALUES_BATCH_SIZE =
+      "oraoop.min.append.values.batch.size";
+  public static final int ORAOOP_MIN_APPEND_VALUES_BATCH_SIZE_DEFAULT = 5000;
+
+  // The version of the Oracle database we're connected to...
+  public static final String ORAOOP_ORACLE_DATABASE_VERSION_MAJOR =
+      "oraoop.oracle.database.version.major";
+  public static final String ORAOOP_ORACLE_DATABASE_VERSION_MINOR =
+      "oraoop.oracle.database.version.minor";
+
+  // When OraOop creates a table for a Sqoop export (from a template table) and
+  // the table contains partitions,
+  // this is the prefix of those partition names. (This also allows us to later
+  // identify partitions that OraOop
+  // created.)
+  public static final String EXPORT_TABLE_PARTITION_NAME_PREFIX = "ORAOOP_";
+
+  // When OraOop creates temporary tables for each mapper during a Sqoop export
+  // this is the prefix of table names...
+  public static final String EXPORT_MAPPER_TABLE_NAME_PREFIX = "ORAOOP_";
+
+  // The format string used to turn a DATE into a string for use within the
+  // names of Oracle objects
+  // that we create. For example, temporary tables, table partitions, table
+  // subpartitions...
+  public static final String ORACLE_OBJECT_NAME_DATE_TO_STRING_FORMAT_STRING =
+      "yyyymmdd_hh24miss";
+
+  // Indicates whether to perform a "merge" operation when performing a Sqoop
+  // export.
+  // If false, 'insert' statements will be used (i.e. no 'updates')...
+  public static final String ORAOOP_EXPORT_MERGE = "oraoop.export.merge";
+
+  // This property allows the user to enable parallelization during exports...
+  public static final String ORAOOP_EXPORT_PARALLEL =
+      "oraoop.export.oracle.parallelization.enabled";
+
+  // Flag used to indicate that the Oracle table contains at least one column of
+  // type BINARY_DOUBLE...
+  public static final String TABLE_CONTAINS_BINARY_DOUBLE_COLUMN =
+      "oraoop.table.contains.binary.double.column";
+  // Flag used to indicate that the Oracle table contains at least one column of
+  // type BINARY_FLOAT...
+  public static final String TABLE_CONTAINS_BINARY_FLOAT_COLUMN =
+      "oraoop.table.contains.binary.float.column";
+
+  // The storage clause to append to the end of any CREATE TABLE statements we
+  // execute for temporary Oracle tables...
+  public static final String ORAOOP_TEMPORARY_TABLE_STORAGE_CLAUSE =
+      "oraoop.temporary.table.storage.clause";
+
+  // The storage clause to append to the end of any CREATE TABLE statements we
+  // execute for permanent (export) Oracle tables...
+  public static final String ORAOOP_EXPORT_TABLE_STORAGE_CLAUSE =
+      "oraoop.table.storage.clause";
+
+  // Additional columns to include with the --update-key column...
+  public static final String ORAOOP_UPDATE_KEY_EXTRA_COLUMNS =
+      "oraoop.update.key.extra.columns";
+
+  // Should OraOop map Timestamps as java.sql.Timestamp as Sqoop does, or as
+  // String
+  public static final String ORAOOP_MAP_TIMESTAMP_AS_STRING =
+      "oraoop.timestamp.string";
+  public static final boolean ORAOOP_MAP_TIMESTAMP_AS_STRING_DEFAULT = true;
+
+  // This flag allows the user to force use of the APPEND_VALUES Oracle hint
+  // either ON, OFF or AUTO...
+  public static final String ORAOOP_ORACLE_APPEND_VALUES_HINT_USAGE =
+      "oraoop.oracle.append.values.hint.usage";
+
+  /**
+   * Whether to use the append values hint for exports.
+   */
+  public enum AppendValuesHintUsage {
+    AUTO, ON, OFF
+  }
+
+  // http://download.oracle.com/docs/cd/E11882_01/server.112/e17118/
+  //     sql_elements001.htm#i45441
+  public static final String SUPPORTED_IMPORT_ORACLE_DATA_TYPES_CLAUSE =
+      "(DATA_TYPE IN ("
+          +
+          // "'BFILE',"+
+          "'BINARY_DOUBLE',"
+          + "'BINARY_FLOAT',"
+          + "'BLOB',"
+          + "'CHAR',"
+          + "'CLOB',"
+          + "'DATE',"
+          + "'FLOAT',"
+          + "'LONG',"
+          +
+          // "'LONG RAW',"+
+          // "'MLSLABEL',"+
+          "'NCHAR',"
+          + "'NCLOB',"
+          + "'NUMBER',"
+          + "'NVARCHAR2',"
+          + "'RAW',"
+          + "'ROWID',"
+          +
+          // "'UNDEFINED',"+
+          "'URITYPE',"
+          +
+          // "'UROWID',"+ //<- SqlType = 1111 = "OTHER" Not supported as
+          // "AAAAACAADAAAAAEAAF" is being returned as "AAAAAAgADAAAA"
+          "'VARCHAR2'"
+          + // <- Columns declared as VARCHAR are listed as VARCHAR2 in
+            // dba_tabl_columns
+          // "'XMLTYPE',"+
+          ")" + " OR DATA_TYPE LIKE 'INTERVAL YEAR(%) TO MONTH'"
+          + " OR DATA_TYPE LIKE 'INTERVAL DAY(%) TO SECOND(%)'"
+          + " OR DATA_TYPE LIKE 'TIMESTAMP(%)'"
+          + " OR DATA_TYPE LIKE 'TIMESTAMP(%) WITH TIME ZONE'"
+          + " OR DATA_TYPE LIKE 'TIMESTAMP(%) WITH LOCAL TIME ZONE'" + ")";
+
+  public static final String SUPPORTED_EXPORT_ORACLE_DATA_TYPES_CLAUSE =
+      "(DATA_TYPE IN ("
+          +
+          // "'BFILE',"+
+          "'BINARY_DOUBLE',"
+          + "'BINARY_FLOAT',"
+          +
+          // "'BLOB',"+ //<- Jira: SQOOP-117 Sqoop cannot export LOB data
+          "'CHAR',"
+          +
+          // "'CLOB',"+ //<- Jira: SQOOP-117 Sqoop cannot export LOB data
+          "'DATE',"
+          + "'FLOAT',"
+          +
+          // "'LONG',"+ //<- "create table as select..." and
+          // "insert into table as select..." do not work when a long column
+          // exists.
+          // "'LONG RAW',"+
+          // "'MLSLABEL',"+
+          "'NCHAR',"
+          +
+          // "'NCLOB',"+ //<- Jira: SQOOP-117 Sqoop cannot export LOB data
+          "'NUMBER',"
+          + "'NVARCHAR2',"
+          +
+          // "'RAW',"+
+          "'ROWID',"
+          +
+          // "'UNDEFINED',"+
+          "'URITYPE',"
+          +
+          // "'UROWID',"+ //<- SqlType = 1111 = "OTHER" Not supported as
+          // "AAAAACAADAAAAAEAAF" is being returned as "AAAAAAgADAAAA"
+          "'VARCHAR2'"
+          + // <- Columns declared as VARCHAR are listed as VARCHAR2 in
+            // dba_tabl_columns
+          // "'XMLTYPE',"+
+          ")" + " OR DATA_TYPE LIKE 'INTERVAL YEAR(%) TO MONTH'"
+          + " OR DATA_TYPE LIKE 'INTERVAL DAY(%) TO SECOND(%)'"
+          + " OR DATA_TYPE LIKE 'TIMESTAMP(%)'"
+          + " OR DATA_TYPE LIKE 'TIMESTAMP(%) WITH TIME ZONE'"
+          + " OR DATA_TYPE LIKE 'TIMESTAMP(%) WITH LOCAL TIME ZONE'" + ")";
+
+  // public static final int[] SUPPORTED_ORACLE_DATA_TYPES = {
+  // oracle.jdbc.OracleTypes.BIT // -7;
+  // ,oracle.jdbc.OracleTypes.TINYINT // -6;
+  // ,oracle.jdbc.OracleTypes.SMALLINT // 5;
+  // ,oracle.jdbc.OracleTypes.INTEGER // 4;
+  // ,oracle.jdbc.OracleTypes.BIGINT // -5;
+  // ,oracle.jdbc.OracleTypes.FLOAT // 6;
+  // ,oracle.jdbc.OracleTypes.REAL // 7;
+  // ,oracle.jdbc.OracleTypes.DOUBLE // 8;
+  // ,oracle.jdbc.OracleTypes.NUMERIC // 2;
+  // ,oracle.jdbc.OracleTypes.DECIMAL // 3;
+  // ,oracle.jdbc.OracleTypes.CHAR // 1;
+  // ,oracle.jdbc.OracleTypes.VARCHAR // 12;
+  // ,oracle.jdbc.OracleTypes.LONGVARCHAR // -1;
+  // ,oracle.jdbc.OracleTypes.DATE // 91;
+  // ,oracle.jdbc.OracleTypes.TIME // 92;
+  // ,oracle.jdbc.OracleTypes.TIMESTAMP // 93;
+  // // ,oracle.jdbc.OracleTypes.TIMESTAMPNS // -100; //<- Deprecated
+  // ,oracle.jdbc.OracleTypes.TIMESTAMPTZ // -101;
+  // ,oracle.jdbc.OracleTypes.TIMESTAMPLTZ // -102;
+  // ,oracle.jdbc.OracleTypes.INTERVALYM // -103;
+  // ,oracle.jdbc.OracleTypes.INTERVALDS // -104;
+  // ,oracle.jdbc.OracleTypes.BINARY // -2;
+  // /// ,oracle.jdbc.OracleTypes.VARBINARY // -3;
+  // ,oracle.jdbc.OracleTypes.LONGVARBINARY // -4;
+  // ,oracle.jdbc.OracleTypes.ROWID // -8;
+  // ,oracle.jdbc.OracleTypes.CURSOR // -10;
+  // ,oracle.jdbc.OracleTypes.BLOB // 2004;
+  // ,oracle.jdbc.OracleTypes.CLOB // 2005;
+  // // ,oracle.jdbc.OracleTypes.BFILE // -13;
+  // // ,oracle.jdbc.OracleTypes.STRUCT // 2002;
+  // // ,oracle.jdbc.OracleTypes.ARRAY // 2003;
+  // ,oracle.jdbc.OracleTypes.REF // 2006;
+  // ,oracle.jdbc.OracleTypes.NCHAR // -15;
+  // ,oracle.jdbc.OracleTypes.NCLOB // 2011;
+  // ,oracle.jdbc.OracleTypes.NVARCHAR // -9;
+  // ,oracle.jdbc.OracleTypes.LONGNVARCHAR // -16;
+  // // ,oracle.jdbc.OracleTypes.SQLXML // 2009;
+  // // ,oracle.jdbc.OracleTypes.OPAQUE // 2007;
+  // // ,oracle.jdbc.OracleTypes.JAVA_STRUCT // 2008;
+  // // ,oracle.jdbc.OracleTypes.JAVA_OBJECT // 2000;
+  // // ,oracle.jdbc.OracleTypes.PLSQL_INDEX_TABLE // -14;
+  // ,oracle.jdbc.OracleTypes.BINARY_FLOAT // 100;
+  // ,oracle.jdbc.OracleTypes.BINARY_DOUBLE // 101;
+  // ,oracle.jdbc.OracleTypes.NULL // 0;
+  // ,oracle.jdbc.OracleTypes.NUMBER // 2;
+  // // ,oracle.jdbc.OracleTypes.RAW // -2;
+  // // ,oracle.jdbc.OracleTypes.OTHER // 1111;
+  // ,oracle.jdbc.OracleTypes.FIXED_CHAR // 999;
+  // // ,oracle.jdbc.OracleTypes.DATALINK // 70;
+  // ,oracle.jdbc.OracleTypes.BOOLEAN // 16;
+  // };
+
+  /**
+   * Constants for things belonging to sqoop...
+   */
+  public static final class Sqoop {
+    private Sqoop() {
+    }
+
+    /**
+     * What type of Sqoop tool is being run.
+     */
+    public enum Tool {
+      UNKNOWN, IMPORT, EXPORT
+    }
+
+    public static final String IMPORT_TOOL_NAME = "import";
+    public static final String MAX_MAPREDUCE_ATTEMPTS =
+        "mapred.map.max.attempts";
+  }
+
+/**
+ * Constants for things belonging to Oracle...
+ */
+  public static final class Oracle {
+    private Oracle() {
+    }
+
+    public static final int ROWID_EXTENDED_ROWID_TYPE = 1;
+    public static final int ROWID_MAX_ROW_NUMBER_PER_BLOCK = 32767;
+
+    // This is how you comment-out a line of SQL text in Oracle.
+    public static final String ORACLE_SQL_STATEMENT_COMMENT_TOKEN = "--";
+
+    public static final String OBJECT_TYPE_TABLE = "TABLE";
+
+    public static final String URITYPE = "URITYPE";
+
+    public static final int MAX_IDENTIFIER_LENGTH = 30; // <- Max length of an
+                                                        // Oracle name
+                                                        // (table-name,
+                                                        // partition-name etc.)
+
+    public static final String HINT_SYNTAX = "/*+ %s */ "; // Syntax for a hint
+                                                           // in Oracle
+  }
+
+  /**
+   * Logging constants.
+   */
+  public static class Logging {
+    /**
+     * Level of log to output.
+     */
+    public enum Level {
+      TRACE, DEBUG, INFO, WARN, ERROR, FATAL
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopDBInputSplit.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopDBInputSplit.java b/src/java/org/apache/sqoop/manager/oracle/OraOopDBInputSplit.java
new file mode 100644
index 0000000..93efa76
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopDBInputSplit.java
@@ -0,0 +1,195 @@
+/**
+ * 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.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.io.Text;
+import com.cloudera.sqoop.mapreduce.db.DBInputFormat;
+
+class OraOopDBInputSplit extends DBInputFormat.DBInputSplit {
+
+  private int splitId;
+  private double totalNumberOfBlocksInAllSplits;
+  private String splitLocation;
+  private List<OraOopOracleDataChunk> oracleDataChunks;
+
+  // NB: Update write(), readFields() and getDebugDetails() if you add fields
+  // here.
+
+  public OraOopDBInputSplit() {
+
+    this.splitId = -1;
+    this.splitLocation = "";
+    this.oracleDataChunks = new ArrayList<OraOopOracleDataChunk>();
+  }
+
+  public OraOopDBInputSplit(List<OraOopOracleDataChunk> dataChunks) {
+
+    setOracleDataChunks(dataChunks);
+  }
+
+  public void setOracleDataChunks(List<OraOopOracleDataChunk> dataChunks) {
+
+    this.oracleDataChunks = dataChunks;
+  }
+
+  public List<OraOopOracleDataChunk> getDataChunks() {
+
+    return this.oracleDataChunks;
+  }
+
+  public int getNumberOfDataChunks() {
+
+    if (this.getDataChunks() == null) {
+      return 0;
+    } else {
+      return this.getDataChunks().size();
+    }
+  }
+
+  @Override
+  public String[] getLocations() throws IOException {
+
+    if (this.splitLocation.isEmpty()) {
+      return new String[] {};
+    } else {
+      return new String[] { this.splitLocation };
+    }
+
+  }
+
+  /**
+   * @return The total number of blocks within the data-chunks of this split
+   */
+  @Override
+  public long getLength() {
+
+    return this.getTotalNumberOfBlocksInThisSplit();
+  }
+
+  public int getTotalNumberOfBlocksInThisSplit() {
+
+    if (this.getNumberOfDataChunks() == 0) {
+      return 0;
+    }
+
+    int result = 0;
+    for (OraOopOracleDataChunk dataChunk : this.getDataChunks()) {
+      result += dataChunk.getNumberOfBlocks();
+    }
+
+    return result;
+  }
+
+  public OraOopOracleDataChunk findDataChunkById(String id) {
+
+    for (OraOopOracleDataChunk dataChunk : this.getDataChunks()) {
+      if (dataChunk.getId().equals(id)) {
+        return dataChunk;
+      }
+    }
+    return null;
+  }
+
+  @Override
+  /** {@inheritDoc} */
+  public void write(DataOutput output) throws IOException {
+
+    output.writeInt(splitId);
+
+    if (this.oracleDataChunks == null) {
+      output.writeInt(0);
+    } else {
+      output.writeInt(this.oracleDataChunks.size());
+      for (OraOopOracleDataChunk dataChunk : this.oracleDataChunks) {
+        Text.writeString(output, dataChunk.getClass().getName());
+        dataChunk.write(output);
+      }
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  /** {@inheritDoc} */
+  public void readFields(DataInput input) throws IOException {
+
+    this.splitId = input.readInt();
+
+    int dataChunkCount = input.readInt();
+    if (dataChunkCount == 0) {
+      this.oracleDataChunks = null;
+    } else {
+      Class<? extends OraOopOracleDataChunk> dataChunkClass;
+      OraOopOracleDataChunk dataChunk;
+      this.oracleDataChunks =
+          new ArrayList<OraOopOracleDataChunk>(dataChunkCount);
+      for (int idx = 0; idx < dataChunkCount; idx++) {
+        try {
+          dataChunkClass =
+              (Class<? extends OraOopOracleDataChunk>) Class.forName(Text
+                  .readString(input));
+          dataChunk = dataChunkClass.newInstance();
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+        dataChunk.readFields(input);
+        this.oracleDataChunks.add(dataChunk);
+      }
+    }
+  }
+
+  public String getDebugDetails() {
+
+    StringBuilder result = new StringBuilder();
+
+    if (this.getNumberOfDataChunks() == 0) {
+      result.append(String.format(
+          "Split[%s] does not contain any Oracle data-chunks.", this.splitId));
+    } else {
+      result.append(String.format(
+          "Split[%s] includes the Oracle data-chunks:\n", this.splitId));
+      for (OraOopOracleDataChunk dataChunk : getDataChunks()) {
+        result.append(dataChunk.toString());
+      }
+    }
+    return result.toString();
+  }
+
+  protected int getSplitId() {
+    return this.splitId;
+  }
+
+  protected void setSplitId(int newSplitId) {
+    this.splitId = newSplitId;
+  }
+
+  protected void setSplitLocation(String newSplitLocation) {
+    this.splitLocation = newSplitLocation;
+  }
+
+  protected void setTotalNumberOfBlocksInAllSplits(
+      int newTotalNumberOfBlocksInAllSplits) {
+    this.totalNumberOfBlocksInAllSplits = newTotalNumberOfBlocksInAllSplits;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopDBRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopDBRecordReader.java b/src/java/org/apache/sqoop/manager/oracle/OraOopDBRecordReader.java
new file mode 100644
index 0000000..45a88ef
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopDBRecordReader.java
@@ -0,0 +1,468 @@
+/**
+ * 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.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.cloudera.sqoop.lib.SqoopRecord;
+import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
+import com.cloudera.sqoop.mapreduce.db.DBInputFormat;
+import com.cloudera.sqoop.mapreduce.db.DBInputFormat.DBInputSplit;
+import com.cloudera.sqoop.mapreduce.db.DataDrivenDBRecordReader;
+import org.apache.sqoop.manager.oracle.OraOopConstants.
+           OraOopTableImportWhereClauseLocation;
+import org.apache.sqoop.manager.oracle.OraOopUtilities.OraOopStatsReports;
+
+/*
+ * NOTES:
+ *
+ *   T is the output-type of this record reader.
+ *
+ *   getFieldNames() is overridden to insert an "data_chunk_id" column
+ *   containing the id (integer) of the Oracle data-chunk the data
+ *   was obtained from. This is used to calculate the "percentage complete"
+ *   for this mapper.
+ *
+ *   getSelectQuery() is overridden to inject the actual data_chunk_id number
+ *   into the query that is executed (for the data-chunk being processed).
+ *
+ *   This class extends DBRecordReader. Unfortunately, DBRecordReader does
+ *   not expose its results property (of type ResultSet), so we have to
+ *   override executeQuery() in order to obtain a reference to the data
+ *   obtained when the SQL generated by getSelectQuery() is executed.
+ */
+class OraOopDBRecordReader<T extends SqoopRecord> extends
+    DataDrivenDBRecordReader<T> {
+
+  private static final OraOopLog LOG = OraOopLogFactory
+      .getLog(OraOopDBRecordReader.class);
+
+  private OraOopDBInputSplit dbInputSplit; // <- The split this record-reader is
+                                           // working on.
+  private int numberOfBlocksInThisSplit; // <- The number of Oracle blocks in
+                                         // this Oracle data-chunk.
+  private int numberOfBlocksProcessedInThisSplit; // <- How many Oracle blocks
+                                                  // we've processed with this
+                                                  // record-reader.
+  private String currentDataChunkId; // <- The id of the current data-chunk
+                                     // being processed
+  private ResultSet results; // <- The ResultSet containing the data from the
+                             // query returned by getSelectQuery()
+  private int columnIndexDataChunkIdZeroBased = -1; // <- The zero-based column
+                                                    // index of the
+                                                    // data_chunk_id column.
+  private boolean progressCalculationErrorLogged; // <- Whether we've logged a
+                                                  // problem with the progress
+                                                  // calculation during
+                                                  // nextKeyValue().
+  private Object oraOopOraStats; // <- A reference to the Oracle statistics
+                                 // object that is being tracked for this Oracle
+                                 // session.
+  private boolean profilingEnabled; // <- Whether to collect profiling metrics
+  private long timeSpentInNextKeyValueInNanoSeconds; // <- Total time spent in
+                                                     // super.nextKeyValue()
+
+  public OraOopDBRecordReader(DBInputFormat.DBInputSplit split,
+      Class<T> inputClass, Configuration conf, Connection conn,
+      DBConfiguration dbConfig, String cond, String[] fields, String table)
+      throws SQLException {
+
+    super(split, inputClass, conf, conn, dbConfig, cond, fields, table,
+        "ORACLE-ORAOOP");
+
+    OraOopUtilities.enableDebugLoggingIfRequired(conf);
+
+    this.dbInputSplit = castSplit(split);
+
+    String thisOracleInstanceName =
+        OraOopOracleQueries.getCurrentOracleInstanceName(conn);
+    LOG.info(String.format(
+        "This record reader is connected to Oracle via the JDBC URL: \n"
+            + "\t\"%s\"\n" + "\tto the Oracle instance: \"%s\"", conn
+            .toString(), thisOracleInstanceName));
+
+    OracleConnectionFactory.initializeOracleConnection(conn, conf);
+
+    if (OraOopUtilities.userWantsOracleSessionStatisticsReports(conf)) {
+      this.oraOopOraStats = OraOopUtilities.startSessionSnapshot(conn);
+    }
+
+    this.numberOfBlocksInThisSplit =
+        this.dbInputSplit.getTotalNumberOfBlocksInThisSplit();
+    this.numberOfBlocksProcessedInThisSplit = 0;
+
+    this.profilingEnabled = conf.getBoolean("oraoop.profiling.enabled", false);
+  }
+
+  public static OraOopDBInputSplit castSplit(DBInputSplit split) {
+
+    // Check there's a split available...
+    if (split == null) {
+      throw new IllegalArgumentException("The DBInputSplit cannot be null.");
+    }
+
+    // Check that the split is the correct type...
+    Class<?> desiredSplitClass = OraOopDBInputSplit.class;
+    if (!(split.getClass() == desiredSplitClass)) {
+      String errMsg =
+          String.format("The type of Split available within %s "
+              + "should be an instance of class %s, "
+              + "but is actually an instance of class %s", OraOopUtilities
+              .getCurrentMethodName(), desiredSplitClass.getName(), split
+              .getClass().getName());
+      throw new RuntimeException(errMsg);
+    }
+
+    // TODO Cast this using desiredSplitClass, so we only need 1 line of code
+    // that
+    // identifies the type of the split class...
+    // inputSplit = (desiredSplitClass)this.getSplit();
+    return (OraOopDBInputSplit) split;
+  }
+
+  @Override
+  protected String[] getFieldNames() {
+
+    String[] fieldNames = super.getFieldNames();
+    ArrayList<String> result = new ArrayList<String>();
+
+    for (int idx = 0; idx < fieldNames.length; idx++) {
+      result.add(fieldNames[idx]);
+    }
+
+    result.add(OraOopConstants.COLUMN_NAME_DATA_CHUNK_ID);
+    this.columnIndexDataChunkIdZeroBased = result.size() - 1;
+
+    return result.toArray(new String[result.size()]);
+  }
+
+  @Override
+  protected String getSelectQuery() {
+
+    boolean consistentRead =
+        this.getDBConf().getConf().getBoolean(
+            OraOopConstants.ORAOOP_IMPORT_CONSISTENT_READ, false);
+    long consistentReadScn =
+        this.getDBConf().getConf().getLong(
+            OraOopConstants.ORAOOP_IMPORT_CONSISTENT_READ_SCN, 0L);
+    if (consistentRead && consistentReadScn == 0L) {
+      throw new RuntimeException("Could not get SCN for consistent read.");
+    }
+
+    StringBuilder query = new StringBuilder();
+
+    if (this.dbInputSplit.getDataChunks() == null) {
+      String errMsg =
+          String.format("The %s does not contain any data-chunks, within %s.",
+              this.dbInputSplit.getClass().getName(), OraOopUtilities
+                  .getCurrentMethodName());
+      throw new RuntimeException(errMsg);
+    }
+
+    OraOopConstants.OraOopTableImportWhereClauseLocation whereClauseLocation =
+        OraOopUtilities.getOraOopTableImportWhereClauseLocation(this
+            .getDBConf().getConf(),
+            OraOopConstants.OraOopTableImportWhereClauseLocation.SUBSPLIT);
+
+    OracleTable tableContext = getOracleTableContext();
+    OracleTableColumns tableColumns = null;
+    try {
+
+      Configuration conf = this.getDBConf().getConf();
+
+      tableColumns =
+          OraOopOracleQueries.getTableColumns(getConnection(), tableContext,
+              OraOopUtilities.omitLobAndLongColumnsDuringImport(conf),
+              OraOopUtilities.recallSqoopJobType(conf)
+              , true // <- onlyOraOopSupportedTypes
+              , true // <- omitOraOopPseudoColumns
+              );
+    } catch (SQLException ex) {
+      LOG.error(String.format(
+          "Unable to obtain the data-types of the columns in table %s.\n"
+              + "Error:\n%s", tableContext.toString(), ex.getMessage()));
+      throw new RuntimeException(ex);
+    }
+
+    int numberOfDataChunks = this.dbInputSplit.getNumberOfDataChunks();
+    for (int idx = 0; idx < numberOfDataChunks; idx++) {
+
+      OraOopOracleDataChunk dataChunk =
+          this.dbInputSplit.getDataChunks().get(idx);
+
+      if (idx > 0) {
+        query.append("UNION ALL \n");
+      }
+
+      query.append(getColumnNamesClause(tableColumns, dataChunk.getId())) // <-
+                                                                     // SELECT
+                                                                     // clause
+          .append("\n");
+
+      query.append(" FROM ").append(this.getTableName()).append(" ");
+
+      if (consistentRead) {
+        query.append("AS OF SCN ").append(consistentReadScn).append(" ");
+      }
+
+      query.append(getPartitionClauseForDataChunk(this.dbInputSplit, idx))
+          .append(" t").append("\n");
+
+      query.append(" WHERE (").append(
+          getWhereClauseForDataChunk(this.dbInputSplit, idx)).append(")\n");
+
+      // If the user wants the WHERE clause applied to each data-chunk...
+      if (whereClauseLocation
+              == OraOopTableImportWhereClauseLocation.SUBSPLIT) {
+        String conditions = this.getConditions();
+        if (conditions != null && conditions.length() > 0) {
+          query.append(" AND (").append(conditions).append(")\n");
+        }
+      }
+
+    }
+
+    // If the user wants the WHERE clause applied to the whole split...
+    if (whereClauseLocation == OraOopTableImportWhereClauseLocation.SPLIT) {
+      String conditions = this.getConditions();
+      if (conditions != null && conditions.length() > 0) {
+
+        // Insert a "select everything" line at the start of the SQL query...
+        query.insert(0, getColumnNamesClause(tableColumns, null) + " FROM (\n");
+
+        // ...and then apply the WHERE clause to all the UNIONed sub-queries...
+        query.append(")\n").append("WHERE\n").append(conditions).append("\n");
+      }
+    }
+
+    LOG.info("SELECT QUERY = \n" + query.toString());
+
+    return query.toString();
+  }
+
+  private String getColumnNamesClause(OracleTableColumns tableColumns,
+      String dataChunkId) {
+
+    StringBuilder result = new StringBuilder();
+
+    result.append("SELECT ");
+    result.append(OraOopUtilities.getImportHint(this.getDBConf().getConf()));
+
+    String[] fieldNames = this.getFieldNames();
+
+    int firstFieldIndex = 0;
+    int lastFieldIndex = fieldNames.length - 1;
+    for (int i = firstFieldIndex; i <= lastFieldIndex; i++) {
+      if (i > firstFieldIndex) {
+        result.append(",");
+      }
+      String fieldName = fieldNames[i];
+
+      OracleTableColumn oracleTableColumn =
+          tableColumns.findColumnByName(fieldName);
+      if (oracleTableColumn != null) {
+        if (oracleTableColumn.getDataType().equals(
+            OraOopConstants.Oracle.URITYPE)) {
+          fieldName = String.format("uritype.geturl(%s)", fieldName);
+        }
+      }
+
+      // If this field is the "data_chunk_id" that we inserted during
+      // getFields()
+      // then we need to insert the value of that data_chunk_id now...
+      if (i == this.columnIndexDataChunkIdZeroBased
+          && fieldName == OraOopConstants.COLUMN_NAME_DATA_CHUNK_ID) {
+        if (dataChunkId != null && !dataChunkId.isEmpty()) {
+          fieldName =
+              String.format("'%s' %s", dataChunkId,
+                  OraOopConstants.COLUMN_NAME_DATA_CHUNK_ID);
+        }
+      }
+
+      result.append(fieldName);
+    }
+    return result.toString();
+  }
+
+  private String getPartitionClauseForDataChunk(OraOopDBInputSplit split,
+      int dataChunkIndex) {
+    OraOopOracleDataChunk dataChunk = split.getDataChunks().get(dataChunkIndex);
+    return dataChunk.getPartitionClause();
+  }
+
+  private String getWhereClauseForDataChunk(OraOopDBInputSplit split,
+      int dataChunkIndex) {
+
+    OraOopOracleDataChunk dataChunk = split.getDataChunks().get(dataChunkIndex);
+    return dataChunk.getWhereClause();
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getPos() throws IOException {
+
+    // This split contains multiple data-chunks.
+    // Each data-chunk contains multiple blocks.
+    // Return the number of blocks that have been processed by this split...
+    return numberOfBlocksProcessedInThisSplit;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public float getProgress() throws IOException {
+
+    return numberOfBlocksProcessedInThisSplit
+        / (float) numberOfBlocksInThisSplit;
+  }
+
+  @Override
+  public boolean nextKeyValue() throws IOException {
+
+    boolean result = false;
+    try {
+
+      long startTime = 0;
+      if (this.profilingEnabled) {
+        startTime = System.nanoTime();
+      }
+
+      result = super.nextKeyValue();
+
+      if (this.profilingEnabled) {
+        this.timeSpentInNextKeyValueInNanoSeconds +=
+            System.nanoTime() - startTime;
+      }
+
+      // Keep track of which data-chunk we're processing, and therefore how many
+      // Oracle blocks we've processed. This can be used to calculate our
+      // "percentage complete"...
+      if (result && this.results != null) {
+
+        String thisDataChunkId = null;
+        try {
+          // ColumnIndexes are 1-based in jdbc...
+          thisDataChunkId =
+              this.results.getString(this.columnIndexDataChunkIdZeroBased + 1);
+        } catch (SQLException ex) {
+          if (!progressCalculationErrorLogged) {
+            // This prevents us from flooding the log with the same message
+            // thousands of times...
+            progressCalculationErrorLogged = true;
+
+            LOG.warn(String
+                .format(
+                  "Unable to obtain the value of the %s column in method %s.\n"
+                      + "\tthis.columnIndexDataChunkIdZeroBased = %d (NB: "
+                      + "jdbc field indexes are 1-based)\n\tAs a consequence, "
+                      + "progress for the record-reader cannot be calculated.\n"
+                      + "\tError=\n%s",
+                  OraOopConstants.COLUMN_NAME_DATA_CHUNK_ID, OraOopUtilities
+                      .getCurrentMethodName(),
+                  this.columnIndexDataChunkIdZeroBased, ex.getMessage()));
+          }
+        }
+
+        if (thisDataChunkId != null
+            && !thisDataChunkId.equals(this.currentDataChunkId)) {
+          if (this.currentDataChunkId != null
+              && !this.currentDataChunkId.isEmpty()) {
+            OraOopOracleDataChunk dataChunk =
+                this.dbInputSplit.findDataChunkById(thisDataChunkId);
+            if (dataChunk != null) {
+              this.numberOfBlocksProcessedInThisSplit +=
+                  dataChunk.getNumberOfBlocks();
+            }
+          }
+          this.currentDataChunkId = thisDataChunkId;
+        }
+      }
+    } catch (IOException ex) {
+      if (OraOopUtilities.oracleSessionHasBeenKilled(ex)) {
+        LOG.info("\n*********************************************************"
+            + "\nThe Oracle session in use has been killed by a 3rd party."
+            + "\n*********************************************************");
+      }
+      throw ex;
+    }
+
+    return result;
+  }
+
+  @Override
+  protected ResultSet executeQuery(String query) throws SQLException {
+
+    try {
+      this.results = super.executeQuery(query);
+      return this.results;
+    } catch (SQLException ex) {
+      LOG.error(String.format("Error in %s while executing the SQL query:\n"
+          + "%s\n\n" + "%s", OraOopUtilities.getCurrentMethodName(), query, ex
+          .getMessage()));
+      throw ex;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+
+    if (this.profilingEnabled) {
+      LOG.info(String.format(
+          "Time spent in super.nextKeyValue() = %s seconds.",
+          this.timeSpentInNextKeyValueInNanoSeconds / Math.pow(10, 9)));
+    }
+
+    if (OraOopUtilities.userWantsOracleSessionStatisticsReports(getDBConf()
+        .getConf())) {
+      OraOopStatsReports reports =
+          OraOopUtilities.stopSessionSnapshot(this.oraOopOraStats);
+      this.oraOopOraStats = null;
+
+      LOG.info(String.format("Oracle Statistics Report for OraOop:\n\n%s",
+          reports.getPerformanceReport()));
+
+      String fileName =
+          String.format("oracle-stats-csv-%d", this.dbInputSplit.getSplitId());
+      OraOopUtilities.writeOutputFile(this.getDBConf().getConf(), fileName,
+          reports.getCsvReport());
+
+      fileName =
+          String.format("oracle-stats-%d", this.dbInputSplit.getSplitId());
+      OraOopUtilities.writeOutputFile(this.getDBConf().getConf(), fileName,
+          reports.getPerformanceReport());
+    }
+
+    super.close();
+  }
+
+  public OracleTable getOracleTableContext() {
+
+    Configuration conf = this.getDBConf().getConf();
+    OracleTable result =
+        new OracleTable(conf.get(OraOopConstants.ORAOOP_TABLE_OWNER), conf
+            .get(OraOopConstants.ORAOOP_TABLE_NAME));
+    return result;
+  }
+
+}


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

Posted by ve...@apache.org.
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;
+  }
+}


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

Posted by ve...@apache.org.
http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopOracleQueries.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopOracleQueries.java b/src/java/org/apache/sqoop/manager/oracle/OraOopOracleQueries.java
new file mode 100644
index 0000000..7fd18a1
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopOracleQueries.java
@@ -0,0 +1,1687 @@
+/**
+ * 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.lang.reflect.Constructor;
+import java.lang.reflect.Method;
+import java.math.BigDecimal;
+import java.security.InvalidParameterException;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Contains the queries to get data dictionary information from Oracle database.
+ */
+public final class OraOopOracleQueries {
+
+  private static final OraOopLog LOG = OraOopLogFactory
+      .getLog(OraOopOracleQueries.class);
+
+  private static Class<?> oracleConnectionClass;
+  private static Class<?> oracleStatementClass;
+  private static Class<?> oracleResultSetClass;
+  private static Class<?> oracleTypesClass;
+  private static Class<?> oracleDateClass;
+  private static Method methSetLongAtName;
+  private static Method methSetBigDecimalAtName;
+  private static Method methSetStringAtName;
+  private static Method methSetTimestampAtName;
+  private static Method methSetBinaryDoubleAtName;
+  private static Method methSetObjectAtName;
+  private static Method methSetBinaryFloatAtName;
+  private static Method methSetIntAtName;
+
+  private static final Map<String, Integer> ORACLE_TYPES =
+      new HashMap<String, Integer>();
+
+  static {
+    try {
+      oracleStatementClass =
+          Class.forName("oracle.jdbc.OraclePreparedStatement");
+      methSetLongAtName =
+          oracleStatementClass.getMethod("setLongAtName", String.class,
+              long.class);
+      methSetBigDecimalAtName =
+          oracleStatementClass.getMethod("setBigDecimalAtName", String.class,
+              BigDecimal.class);
+      methSetStringAtName =
+          oracleStatementClass.getMethod("setStringAtName", String.class,
+              String.class);
+      methSetTimestampAtName =
+          oracleStatementClass.getMethod("setTimestampAtName", String.class,
+              Timestamp.class);
+      methSetBinaryDoubleAtName =
+          oracleStatementClass.getMethod("setBinaryDoubleAtName", String.class,
+              double.class);
+      methSetObjectAtName =
+          oracleStatementClass.getMethod("setObjectAtName", String.class,
+              Object.class);
+      methSetBinaryFloatAtName =
+          oracleStatementClass.getMethod("setBinaryFloatAtName", String.class,
+              float.class);
+      methSetIntAtName =
+          oracleStatementClass.getMethod("setIntAtName", String.class,
+              int.class);
+
+      oracleResultSetClass = Class.forName("oracle.jdbc.OracleResultSet");
+      oracleDateClass = Class.forName("oracle.sql.DATE");
+      oracleConnectionClass = Class.forName("oracle.jdbc.OracleConnection");
+      oracleTypesClass = Class.forName("oracle.jdbc.OracleTypes");
+    } catch (Exception e) {
+      throw new RuntimeException(
+          "Problem getting Oracle JDBC methods via reflection.", e);
+    }
+  }
+
+  private OraOopOracleQueries() {
+  }
+
+  protected static void setJdbcFetchSize(Connection connection,
+      org.apache.hadoop.conf.Configuration conf) {
+
+    int fetchSize =
+        conf.getInt(OraOopConstants.ORACLE_ROW_FETCH_SIZE,
+            OraOopConstants.ORACLE_ROW_FETCH_SIZE_DEFAULT);
+    try {
+      Method methSetPrefetch =
+          oracleConnectionClass.getMethod("setDefaultRowPrefetch", int.class);
+      methSetPrefetch.invoke(connection, fetchSize);
+
+      String msg =
+          "The Oracle connection has had its default row fetch size set to : "
+              + fetchSize;
+      if (fetchSize == OraOopConstants.ORACLE_ROW_FETCH_SIZE_DEFAULT) {
+        LOG.debug(msg);
+      } else {
+        LOG.info(msg);
+      }
+    } catch (Exception ex) {
+      LOG.warn(
+          String
+              .format(
+                  "Unable to configure the DefaultRowPrefetch of the "
+                + "Oracle connection in %s.",
+                  OraOopUtilities.getCurrentMethodName()), ex);
+    }
+
+  }
+
+  public static void setConnectionTimeZone(Connection connection,
+      Configuration conf) {
+    String timeZoneString = conf.get("oracle.sessionTimeZone", "GMT");
+    setConnectionTimeZone(connection, timeZoneString);
+  }
+
+  public static void setConnectionTimeZone(Connection connection,
+      String timeZone) {
+    TimeZone timeZoneObj = TimeZone.getTimeZone(timeZone);
+    try {
+      Method methSession =
+          oracleConnectionClass.getMethod("setSessionTimeZone", String.class);
+      Method methDefault =
+          oracleConnectionClass.getMethod("setDefaultTimeZone", TimeZone.class);
+      methSession.invoke(connection, timeZoneObj.getID());
+      methDefault.invoke(connection, timeZoneObj);
+      TimeZone.setDefault(timeZoneObj);
+      LOG.info("Session Time Zone set to " + timeZoneObj.getID());
+    } catch (Exception e) {
+      LOG.error("Error setting time zone: " + e.getMessage());
+    }
+  }
+
+  public static OracleTablePartitions getPartitions(Connection connection,
+      OracleTable table) throws SQLException {
+
+    OracleTablePartitions result = new OracleTablePartitions();
+
+    PreparedStatement statement =
+      connection
+        .prepareStatement("with"
+        + " partitions as"
+        + "   (select table_owner, table_name, partition_name"
+        +  "   from dba_tab_partitions"
+        + "   where"
+        + "   table_owner = ? and"
+        + "   table_name = ?),"
+        + " subpartitions as"
+        + "  (select table_owner, table_name, partition_name, subpartition_name"
+        + "  from dba_tab_subpartitions"
+        + "  where"
+        + "  table_owner = ? and"
+        + "  table_name = ?)"
+        + " select"
+        + "   partitions.partition_name,"
+        + "   subpartitions.subpartition_name"
+        + " from partitions left outer join subpartitions on"
+        + " (partitions.table_owner = subpartitions.table_owner"
+        + " and partitions.table_name = subpartitions.table_name"
+        + " and partitions.partition_name = subpartitions.partition_name)"
+        + " order by partition_name, subpartition_name");
+
+    statement.setString(1, table.getSchema());
+    statement.setString(2, table.getName());
+    statement.setString(3, table.getSchema());
+    statement.setString(4, table.getName());
+
+    ResultSet resultSet = statement.executeQuery();
+
+    OracleTablePartition partition = null;
+    while (resultSet.next()) {
+      String partitionName = resultSet.getString("partition_name");
+      String subPartitionName = resultSet.getString("subpartition_name");
+
+      if (subPartitionName != null &&  !("".equals(subPartitionName))) {
+        partition = new OracleTablePartition(subPartitionName, true);
+        result.add(partition);
+      } else {
+        if (partition == null || partition.isSubPartition()
+            || partition.getName() != partitionName) {
+          partition = new OracleTablePartition(partitionName, false);
+          result.add(partition);
+        }
+      }
+    }
+
+    resultSet.close();
+    statement.close();
+
+    return result;
+  }
+
+  public static int getOracleDataObjectNumber(Connection connection,
+      OracleTable table) throws SQLException {
+
+    PreparedStatement statement =
+        connection.prepareStatement("SELECT data_object_id "
+            + " FROM dba_objects" + " WHERE owner = ?" + " and object_name = ?"
+            + " and object_type = ?");
+    statement.setString(1, table.getSchema());
+    statement.setString(2, table.getName());
+    statement.setString(3, "TABLE");
+
+    ResultSet resultSet = statement.executeQuery();
+
+    resultSet.next();
+    int result = resultSet.getInt("data_object_id");
+
+    resultSet.close();
+    statement.close();
+
+    return result;
+  }
+
+  private static String getPartitionBindVars(List<String> partitionList) {
+    String result = "";
+    for (int i = 1; i <= partitionList.size(); i++) {
+      result += (i > 1) ? "," : "";
+      result += ":part" + i;
+    }
+    return result;
+  }
+
+  private static void bindPartitionBindVars(PreparedStatement statement,
+      List<String> partitionList) throws SQLException {
+    int i = 0;
+    for (String partition : partitionList) {
+      i++;
+      OraOopOracleQueries.setStringAtName(statement, "part" + i, partition);
+    }
+  }
+
+  public static List<OraOopOracleDataChunkPartition>
+      getOracleDataChunksPartition(Connection connection, OracleTable table,
+          List<String> partitionList) throws SQLException {
+    List<OraOopOracleDataChunkPartition> result =
+        new ArrayList<OraOopOracleDataChunkPartition>();
+    String sql =
+        "SELECT "
+          + "  pl.partition_name, "
+          + "  pl.is_subpartition, "
+          + "  s.blocks "
+          + "FROM "
+          + "  (SELECT tp.table_owner, "
+          + "    tp.table_name, "
+          + "    NVL(tsp.subpartition_name,tp.partition_name) partition_name, "
+          + "    nvl2(tsp.subpartition_name,1,0) is_subpartition "
+          + "  FROM dba_tab_partitions tp, "
+          + "    dba_tab_subpartitions tsp "
+          + "  WHERE tp.table_owner     = :table_owner"
+          + "  AND tp.table_name        = :table_name"
+          + "  AND tsp.table_owner(+)   =tp.table_owner "
+          + "  AND tsp.table_name(+)    =tp.table_name "
+          + "  AND tsp.partition_name(+)=tp.partition_name ";
+
+    if (partitionList != null && partitionList.size() > 0) {
+      sql +=
+          " AND tp.partition_name IN (" + getPartitionBindVars(partitionList)
+              + ") ";
+    }
+
+    sql +=
+        "  ) pl, " + "  dba_segments s "
+            + "WHERE s.owner       =pl.table_owner "
+            + "AND s.segment_name  =pl.table_name "
+            + "AND s.partition_name=pl.partition_name ";
+
+    PreparedStatement statement = connection.prepareStatement(sql);
+    OraOopOracleQueries.setStringAtName(statement, "table_owner", table
+        .getSchema());
+    OraOopOracleQueries.setStringAtName(statement, "table_name", table
+        .getName());
+
+    if (partitionList != null && partitionList.size() > 0) {
+      bindPartitionBindVars(statement, partitionList);
+    }
+
+    trace(String.format("%s SQL Query =\n%s", OraOopUtilities
+        .getCurrentMethodName(), sql.replace(":table_owner", table.getSchema())
+        .replace(":table_name", table.getName())));
+
+    ResultSet resultSet = statement.executeQuery();
+
+    while (resultSet.next()) {
+      OraOopOracleDataChunkPartition dataChunk =
+          new OraOopOracleDataChunkPartition(resultSet
+              .getString("partition_name"), resultSet
+              .getBoolean("is_subpartition"), resultSet.getInt("blocks"));
+      result.add(dataChunk);
+    }
+    resultSet.close();
+    statement.close();
+    return result;
+  }
+
+  public static List<OraOopOracleDataChunkExtent> getOracleDataChunksExtent(
+      Configuration conf, Connection connection, OracleTable table,
+      List<String> partitionList, int numberOfChunksPerOracleDataFile)
+      throws SQLException {
+
+    List<OraOopOracleDataChunkExtent> result =
+        new ArrayList<OraOopOracleDataChunkExtent>();
+
+    String sql =
+        "SELECT data_object_id, "
+          + "file_id, "
+          + "relative_fno, "
+          + "file_batch, "
+          + "MIN (start_block_id) start_block_id, "
+          + "MAX (end_block_id) end_block_id, "
+          + "SUM (blocks) blocks "
+          + "FROM (SELECT o.data_object_id, "
+          + "e.file_id, "
+          + "e.relative_fno, "
+          + "e.block_id start_block_id, "
+          + "e.block_id + e.blocks - 1 end_block_id, "
+          + "e.blocks, "
+          + "CEIL ( "
+          + "   SUM ( "
+          + "      e.blocks) "
+          + "   OVER (PARTITION BY o.data_object_id, e.file_id "
+          + "         ORDER BY e.block_id ASC) "
+          + "   / (SUM (e.blocks) "
+          + "         OVER (PARTITION BY o.data_object_id, e.file_id) "
+          + "      / :numchunks)) "
+          + "   file_batch "
+          + "FROM dba_extents e, dba_objects o, dba_tab_subpartitions tsp "
+          + "WHERE     o.owner = :owner "
+          + "AND o.object_name = :object_name "
+          + "AND e.owner = :owner "
+          + "AND e.segment_name = :object_name "
+          + "AND o.owner = e.owner "
+          + "AND o.object_name = e.segment_name "
+          + "AND (o.subobject_name = e.partition_name "
+          + "     OR (o.subobject_name IS NULL AND e.partition_name IS NULL)) "
+          + "AND o.owner = tsp.table_owner(+) "
+          + "AND o.object_name = tsp.table_name(+) "
+          + "AND o.subobject_name = tsp.subpartition_name(+) ";
+
+    if (partitionList != null && partitionList.size() > 0) {
+      sql +=
+          " AND case when o.object_type='TABLE SUBPARTITION' then "
+          + "tsp.partition_name else o.subobject_name end IN ("
+              + getPartitionBindVars(partitionList) + ") ";
+    }
+
+    sql +=
+        ") " + "GROUP BY data_object_id, " + "         file_id, "
+            + "         relative_fno, " + "         file_batch "
+            + "ORDER BY data_object_id, " + "         file_id, "
+            + "         relative_fno, " + "         file_batch";
+
+    sql = conf.get(OraOopConstants.ORAOOP_ORACLE_DATA_CHUNKS_QUERY, sql);
+
+    PreparedStatement statement = connection.prepareStatement(sql);
+    OraOopOracleQueries.setIntAtName(statement, "numchunks",
+        numberOfChunksPerOracleDataFile);
+    OraOopOracleQueries.setStringAtName(statement, "owner", table.getSchema());
+    OraOopOracleQueries.setStringAtName(statement, "object_name", table
+        .getName());
+
+    if (partitionList != null && partitionList.size() > 0) {
+      bindPartitionBindVars(statement, partitionList);
+    }
+
+    trace(String.format("%s SQL Query =\n%s", OraOopUtilities
+        .getCurrentMethodName(), sql.replace(":numchunks",
+        Integer.toString(numberOfChunksPerOracleDataFile)).replace(":owner",
+        table.getSchema()).replace(":object_name", table.getName())));
+
+    ResultSet resultSet = statement.executeQuery();
+
+    while (resultSet.next()) {
+      int fileId = resultSet.getInt("relative_fno");
+      int fileBatch = resultSet.getInt("file_batch");
+      String dataChunkId =
+          OraOopUtilities.generateDataChunkId(fileId, fileBatch);
+      OraOopOracleDataChunkExtent dataChunk =
+          new OraOopOracleDataChunkExtent(dataChunkId, resultSet
+              .getInt("data_object_id"), resultSet.getInt("relative_fno"),
+              resultSet.getInt("start_block_id"), resultSet
+                  .getInt("end_block_id"));
+      result.add(dataChunk);
+    }
+
+    resultSet.close();
+    statement.close();
+
+    return result;
+  }
+
+  private static void trace(String message) {
+
+    LOG.debug(message);
+  }
+
+  public static String getOracleObjectType(Connection connection,
+      OracleTable table) throws SQLException {
+
+    PreparedStatement statement =
+        connection.prepareStatement("SELECT object_type " + " FROM dba_objects"
+            + " WHERE owner = ?" + " and object_name = ?");
+    statement.setString(1, table.getSchema());
+    statement.setString(2, table.getName());
+
+    ResultSet resultSet = statement.executeQuery();
+
+    String result = null;
+    if (resultSet.next()) {
+      result = resultSet.getString("object_type");
+    }
+
+    resultSet.close();
+    statement.close();
+
+    return result;
+  }
+
+  public static OracleVersion getOracleVersion(Connection connection)
+      throws SQLException {
+
+    String sql =
+        "SELECT \n"
+      + "  v.banner, \n"
+      + "  rtrim(v.version)      full_version, \n"
+      + "  rtrim(v.version_bit) version_bit, \n"
+      + "  SUBSTR(v.version, 1, INSTR(v.version, '.', 1, 1)-1) major, \n"
+      + "  SUBSTR(v.version, INSTR(v.version, '.', 1, 1) + 1, "
+      + "  INSTR(v.version, '.', 1, 2) - INSTR(v.version, '.', 1, 1) - 1) "
+      + "    minor, \n"
+      + "  SUBSTR(v.version, INSTR(v.version, '.', 1, 2) + 1, "
+      + "  INSTR(v.version, '.', 1, 3) - INSTR(v.version, '.', 1, 2) - 1) "
+      + "    version, \n"
+      + "  SUBSTR(v.version, INSTR(v.version, '.', 1, 3) + 1, "
+      + "  INSTR(v.version, '.', 1, 4) - INSTR(v.version, '.', 1, 3) - 1) "
+      + "    patch, \n"
+      + "  DECODE(instr(v.banner, '64bit'), 0, 'False', 'True') isDb64bit, \n"
+      + "  DECODE(instr(b.banner, 'HPUX'), 0, 'False', 'True') isHPUX \n"
+      + "FROM (SELECT rownum row_num, \n"
+      + "   banner,\n"
+      + "   SUBSTR(SUBSTR(banner,INSTR(banner,'Release ')+8), 1) version_bit,\n"
+      + "   SUBSTR(SUBSTR(banner,INSTR(banner,'Release ')+8), 1,\n"
+      + "    INSTR(SUBSTR(banner,INSTR(banner,'Release ')+8),' ')) version\n"
+      + "FROM v$version\n" + "  WHERE banner LIKE 'Oracle%'\n"
+      + "     OR banner LIKE 'Personal Oracle%') v,\n" + "v$version b\n"
+      + "  WHERE v.row_num = 1\n" + "  and b.banner like 'TNS%'\n";
+
+    Statement statement = connection.createStatement();
+    ResultSet resultSet = statement.executeQuery(sql);
+    resultSet.next();
+    OracleVersion result =
+        new OracleVersion(resultSet.getInt("major"), resultSet.getInt("minor"),
+            resultSet.getInt("version"), resultSet.getInt("patch"), resultSet
+                .getString("banner"));
+
+    resultSet.close();
+    statement.close();
+
+    return result;
+  }
+
+  public static List<OracleTable> getTables(Connection connection)
+      throws SQLException {
+
+    return getTables(connection, null, null, TableNameQueryType.Equals);
+  }
+
+  private enum GetTablesOptions {
+    Owner, Table
+  }
+
+  private enum TableNameQueryType {
+    Equals, Like
+  }
+
+  public static List<OracleTable>
+      getTables(Connection connection, String owner) throws SQLException {
+
+    return getTables(connection, owner, null, TableNameQueryType.Equals);
+  }
+
+  public static OracleTable getTable(Connection connection, String owner,
+      String tableName) throws SQLException {
+
+    List<OracleTable> tables =
+        getTables(connection, owner, tableName, TableNameQueryType.Equals);
+    if (tables.size() > 0) {
+      return tables.get(0);
+    }
+
+    return null;
+  }
+
+  public static List<OracleTable> getTablesWithTableNameLike(
+      Connection connection, String owner, String tableNameLike)
+      throws SQLException {
+
+    return getTables(connection, owner, tableNameLike, TableNameQueryType.Like);
+  }
+
+  private static List<OracleTable> getTables(Connection connection,
+      String owner, String tableName, TableNameQueryType tableNameQueryType)
+      throws SQLException {
+
+    EnumSet<GetTablesOptions> options = EnumSet.noneOf(GetTablesOptions.class);
+
+    if (owner != null && !owner.isEmpty()) {
+      options.add(GetTablesOptions.Owner);
+    }
+
+    if (tableName != null && !tableName.isEmpty()) {
+      options.add(GetTablesOptions.Table);
+    }
+
+    String sql =
+        "SELECT owner, table_name " + " FROM dba_tables" + " %s %s %s %s "
+            + " ORDER BY owner, table_name";
+
+    String tableComparitor = null;
+    switch (tableNameQueryType) {
+      case Equals:
+        tableComparitor = "=";
+        break;
+      case Like:
+        tableComparitor = "LIKE";
+        break;
+      default:
+        throw new RuntimeException("Operator not implemented.");
+    }
+
+    sql =
+        String.format(sql, options.isEmpty() ? "" : "WHERE", options
+            .contains(GetTablesOptions.Owner) ? "owner = ?" : "", options
+            .containsAll(EnumSet.of(GetTablesOptions.Owner,
+                GetTablesOptions.Table)) ? "AND" : "", options
+            .contains(GetTablesOptions.Table) ? String.format(
+            "table_name %s ?", tableComparitor) : "");
+
+    PreparedStatement statement = connection.prepareStatement(sql);
+
+    if (options.containsAll(EnumSet.of(GetTablesOptions.Owner,
+        GetTablesOptions.Table))) {
+      statement.setString(1, owner);
+      statement.setString(2, tableName);
+    } else {
+      if (options.contains(GetTablesOptions.Owner)) {
+        statement.setString(1, owner);
+      } else if (options.contains(GetTablesOptions.Table)) {
+        statement.setString(1, tableName);
+      }
+    }
+
+    ResultSet resultSet = statement.executeQuery();
+
+    ArrayList<OracleTable> result = new ArrayList<OracleTable>();
+    while (resultSet.next()) {
+      result.add(new OracleTable(resultSet.getString("owner"), resultSet
+          .getString("table_name")));
+    }
+
+    resultSet.close();
+    statement.close();
+
+    return result;
+  }
+
+  public static List<String> getTableColumnNames(Connection connection,
+      OracleTable table) throws SQLException {
+
+    OracleTableColumns oracleTableColumns = getTableColumns(connection, table);
+    List<String> result = new ArrayList<String>(oracleTableColumns.size());
+
+    for (int idx = 0; idx < oracleTableColumns.size(); idx++) {
+      result.add(oracleTableColumns.get(idx).getName());
+    }
+
+    return result;
+  }
+
+  public static List<String> getTableColumnNames(Connection connection,
+      OracleTable table, boolean omitLobAndLongColumnsDuringImport,
+      OraOopConstants.Sqoop.Tool sqoopTool, boolean onlyOraOopSupportedTypes,
+      boolean omitOraOopPseudoColumns) throws SQLException {
+
+    OracleTableColumns oracleTableColumns =
+        getTableColumns(connection, table, omitLobAndLongColumnsDuringImport,
+            sqoopTool, onlyOraOopSupportedTypes, omitOraOopPseudoColumns);
+
+    List<String> result = new ArrayList<String>(oracleTableColumns.size());
+
+    for (int idx = 0; idx < oracleTableColumns.size(); idx++) {
+      result.add(oracleTableColumns.get(idx).getName());
+    }
+
+    return result;
+  }
+
+  private static OracleTableColumns getTableColumns(Connection connection,
+      OracleTable table, boolean omitLobColumns, String dataTypesClause,
+      HashSet<String> columnNamesToOmit) throws SQLException {
+
+    String sql =
+        "SELECT column_name, data_type " + " FROM dba_tab_columns"
+            + " WHERE owner = ?" + " and table_name = ?" + " %s"
+            + " ORDER BY column_id";
+
+    sql =
+        String.format(sql, dataTypesClause == null ? "" : " and "
+            + dataTypesClause);
+
+    LOG.debug(String.format("%s : sql = \n%s", OraOopUtilities
+        .getCurrentMethodName(), sql));
+
+    OracleTableColumns result = new OracleTableColumns();
+    PreparedStatement statement = connection.prepareStatement(sql);
+    statement.setString(1, getTableSchema(connection, table));
+    statement.setString(2, table.getName());
+
+    ResultSet resultSet = statement.executeQuery();
+
+    while (resultSet.next()) {
+
+      String columnName = resultSet.getString("column_name");
+
+      if (columnNamesToOmit != null) {
+        if (columnNamesToOmit.contains(columnName)) {
+          continue;
+        }
+      }
+
+      result.add(new OracleTableColumn(columnName, resultSet
+          .getString("data_type")));
+    }
+
+    resultSet.close();
+    statement.close();
+
+    // Now get the actual JDBC data-types for these columns...
+    StringBuilder columnList = new StringBuilder();
+    for (int idx = 0; idx < result.size(); idx++) {
+      if (idx > 0) {
+        columnList.append(",");
+      }
+      columnList.append(result.get(idx).getName());
+    }
+    sql =
+        String.format("SELECT %s FROM %s WHERE 0=1", columnList.toString(),
+            table.toString());
+    Statement statementDesc = connection.createStatement();
+    ResultSet resultSetDesc = statementDesc.executeQuery(sql);
+    ResultSetMetaData metaData = resultSetDesc.getMetaData();
+    for (int idx = 0; idx < metaData.getColumnCount(); idx++) {
+      result.get(idx).setOracleType(metaData.getColumnType(idx + 1)); // <- JDBC
+                                                                    // is
+                                                                    // 1-based
+    }
+    resultSetDesc.close();
+    statementDesc.close();
+
+    return result;
+  }
+
+  public static OracleTableColumns getTableColumns(Connection connection,
+      OracleTable table) throws SQLException {
+
+    return getTableColumns(connection, table, false, null // <- dataTypesClause
+        , null); // <-columnNamesToOmit
+  }
+
+  public static OracleTableColumns getTableColumns(Connection connection,
+      OracleTable table, boolean omitLobAndLongColumnsDuringImport,
+      OraOopConstants.Sqoop.Tool sqoopTool, boolean onlyOraOopSupportedTypes,
+      boolean omitOraOopPseudoColumns) throws SQLException {
+
+    String dataTypesClause = "";
+    HashSet<String> columnNamesToOmit = null;
+
+    if (onlyOraOopSupportedTypes) {
+
+      switch (sqoopTool) {
+
+        case UNKNOWN:
+          throw new InvalidParameterException(
+              "The sqoopTool parameter must not be \"UNKNOWN\".");
+
+        case IMPORT:
+          dataTypesClause =
+              OraOopConstants.SUPPORTED_IMPORT_ORACLE_DATA_TYPES_CLAUSE;
+
+          if (omitLobAndLongColumnsDuringImport) {
+            LOG.info("LOB and LONG columns are being omitted from the Import.");
+            dataTypesClause =
+                " DATA_TYPE not in ('BLOB', 'CLOB', 'NCLOB', 'LONG') and "
+                    + dataTypesClause;
+          }
+          break;
+
+        case EXPORT:
+          dataTypesClause =
+              OraOopConstants.SUPPORTED_EXPORT_ORACLE_DATA_TYPES_CLAUSE;
+          break;
+
+        default:
+          throw new InvalidParameterException("Sqoop Tool not implemented.");
+
+      }
+    }
+
+    if (omitOraOopPseudoColumns) {
+
+      switch (sqoopTool) {
+
+        case EXPORT:
+          if (columnNamesToOmit == null) {
+            columnNamesToOmit = new HashSet<String>();
+          }
+          columnNamesToOmit.add(OraOopConstants.COLUMN_NAME_EXPORT_PARTITION);
+          columnNamesToOmit
+              .add(OraOopConstants.COLUMN_NAME_EXPORT_SUBPARTITION);
+          columnNamesToOmit.add(OraOopConstants.COLUMN_NAME_EXPORT_MAPPER_ROW);
+          break;
+        default:
+          // Only applicable for export.
+          break;
+      }
+    }
+
+    return getTableColumns(connection, table,
+        omitLobAndLongColumnsDuringImport, dataTypesClause, columnNamesToOmit);
+  }
+
+  public static List<OracleActiveInstance> getOracleActiveInstances(
+      Connection connection) throws SQLException {
+
+    // Returns null if there are no rows in v$active_instances - which indicates
+    // this Oracle database is not a RAC.
+    ArrayList<OracleActiveInstance> result = null;
+
+    Statement statement = connection.createStatement();
+    ResultSet resultSet =
+        statement.executeQuery("select inst_name from v$active_instances ");
+
+    while (resultSet.next()) {
+      String instName = resultSet.getString("inst_name");
+      String[] nameFragments = instName.split(":");
+
+      if (nameFragments.length != 2) {
+        throw new SQLException(
+            "Parsing Error: The inst_name column of v$active_instances does "
+            + "not contain two values separated by a colon.");
+      }
+
+      String hostName = nameFragments[0].trim();
+      String instanceName = nameFragments[1].trim();
+
+      if (hostName.isEmpty()) {
+        throw new SQLException(
+            "Parsing Error: The inst_name column of v$active_instances does "
+            + "not include a host name.");
+      }
+
+      if (instanceName.isEmpty()) {
+        throw new SQLException(
+            "Parsing Error: The inst_name column of v$active_instances does "
+            + "not include an instance name.");
+      }
+
+      OracleActiveInstance instance = new OracleActiveInstance();
+      instance.setHostName(hostName);
+      instance.setInstanceName(instanceName);
+
+      if (result == null) {
+        result = new ArrayList<OracleActiveInstance>();
+      }
+
+      result.add(instance);
+    }
+
+    resultSet.close();
+    statement.close();
+    return result;
+  }
+
+  public static String getCurrentOracleInstanceName(Connection connection)
+      throws SQLException {
+
+    String result = "";
+
+    Statement statement = connection.createStatement();
+    ResultSet resultSet =
+        statement.executeQuery("select instance_name from v$instance");
+
+    if (resultSet.next()) {
+      result = resultSet.getString("instance_name");
+    }
+
+    resultSet.close();
+    statement.close();
+    return result;
+  }
+
+  public static Object getSysDate(Connection connection) throws SQLException {
+    Statement statement = connection.createStatement();
+    ResultSet resultSet = statement.executeQuery("select sysdate from dual");
+
+    resultSet.next();
+    try {
+      Method method = oracleResultSetClass.getMethod("getDATE", int.class);
+      return method.invoke(resultSet, 1);
+    } catch (Exception e) {
+      if (e.getCause() instanceof SQLException) {
+        throw (SQLException) e.getCause();
+      } else {
+        throw new RuntimeException("Could not get sysdate", e);
+      }
+    } finally {
+      resultSet.close();
+      statement.close();
+    }
+  }
+
+  public static String oraDATEToString(Object date, String format) {
+    try {
+      Method dateMethod =
+          oracleDateClass.getMethod("toText", String.class, String.class);
+      return (String) dateMethod.invoke(date, format, null);
+    } catch (Exception e) {
+      throw new RuntimeException(String.format(
+          "Unable to convert the oracle.sql.DATE value \"%s\" to text.", date
+              .toString()), e);
+    }
+  }
+
+  public static Object oraDATEFromString(String date, String format) {
+    try {
+      Method dateMethod =
+          oracleDateClass.getMethod("fromText", String.class, String.class,
+              String.class);
+      return dateMethod.invoke(null, date, format, null);
+    } catch (Exception e) {
+      throw new RuntimeException(String
+          .format(
+              "Unable to convert the String value \"%s\" to oracle.sql.DATE.",
+              date), e);
+    }
+  }
+
+  public static Date oraDATEToDate(Object date) {
+    try {
+      Method dateMethod = oracleDateClass.getMethod("dateValue");
+      return (Date) dateMethod.invoke(date);
+    } catch (Exception e) {
+      throw new RuntimeException("Could not get sysdate", e);
+    }
+  }
+
+  public static String getSysTimeStamp(Connection connection)
+      throws SQLException {
+
+    Statement statement = connection.createStatement();
+    ResultSet resultSet =
+        statement.executeQuery("select systimestamp from dual");
+
+    resultSet.next();
+
+    try {
+      Method method = oracleResultSetClass.getMethod("getTIMESTAMP", int.class);
+      Object timestamp = method.invoke(resultSet, 1);
+      return timestamp.toString();
+    } catch (Exception e) {
+      if (e.getCause() instanceof SQLException) {
+        throw (SQLException) e.getCause();
+      } else {
+        throw new RuntimeException("Could not get sysdate", e);
+      }
+    } finally {
+      resultSet.close();
+      statement.close();
+    }
+  }
+
+  public static boolean isTableAnIndexOrganizedTable(Connection connection,
+      OracleTable table) throws SQLException {
+
+    /*
+     * http://ss64.com/orad/DBA_TABLES.html IOT_TYPE: If index-only table,then
+     * IOT_TYPE is IOT or IOT_OVERFLOW or IOT_MAPPING else NULL
+     */
+
+    boolean result = false;
+
+    PreparedStatement statement =
+        connection.prepareStatement("select iot_type " + "from dba_tables "
+            + "where owner = ? " + "and table_name = ?");
+    statement.setString(1, table.getSchema());
+    statement.setString(2, table.getName());
+    ResultSet resultSet = statement.executeQuery();
+
+    if (resultSet.next()) {
+      String iotType = resultSet.getString("iot_type");
+      result = iotType != null && !iotType.isEmpty();
+    }
+
+    resultSet.close();
+    statement.close();
+
+    return result;
+  }
+
+  public static void dropTable(Connection connection, OracleTable table)
+      throws SQLException {
+
+    String sql = String.format("DROP TABLE %s", table.toString());
+
+    Statement statement = connection.createStatement();
+    try {
+      statement.execute(sql);
+    } catch (SQLException ex) {
+      if (ex.getErrorCode() != 942) { // ORA-00942: table or view does not exist
+        throw ex;
+      }
+    }
+    statement.close();
+  }
+
+  public static void
+      exchangeSubpartition(Connection connection, OracleTable table,
+          String subPartitionName, OracleTable subPartitionTable)
+          throws SQLException {
+
+    Statement statement = connection.createStatement();
+    String sql =
+        String.format("ALTER TABLE %s EXCHANGE SUBPARTITION %s WITH TABLE %s",
+            table.toString(), subPartitionName, subPartitionTable.toString());
+    statement.execute(sql);
+    statement.close();
+  }
+
+  public static void createExportTableFromTemplate(Connection connection,
+      OracleTable newTable, String tableStorageClause,
+      OracleTable templateTable, boolean noLogging) throws SQLException {
+
+    String sql =
+        String.format("CREATE TABLE %s \n" + "%s %s \n" + "AS \n"
+            + "(SELECT * FROM %s WHERE 0=1)", newTable.toString(),
+            noLogging ? "NOLOGGING" : "", tableStorageClause, templateTable
+                .toString());
+
+    Statement statement = connection.createStatement();
+    statement.execute(sql);
+    statement.close();
+  }
+
+  private static Object oraDATEAddJulianDays(Object date, int julianDay,
+      int julianSec) {
+    try {
+      Constructor<?> dateCon = oracleDateClass.getConstructor(byte[].class);
+      Method dateBytes = oracleDateClass.getMethod("toBytes");
+      Object result = dateCon.newInstance(dateBytes.invoke(date));
+      Method dateAdd =
+          oracleDateClass.getMethod("addJulianDays", int.class, int.class);
+      result = dateAdd.invoke(result, julianDay, julianSec);
+      return result;
+    } catch (Exception e) {
+      throw new RuntimeException("Could not add days to date.", e);
+    }
+  }
+
+  public static void createExportTableFromTemplateWithPartitioning(
+      Connection connection, OracleTable newTable, String tableStorageClause,
+      OracleTable templateTable, boolean noLogging, String partitionName,
+      Object jobDateTime, int numberOfMappers, String[] subPartitionNames)
+      throws SQLException {
+
+    String dateFormat = "yyyy-mm-dd hh24:mi:ss";
+
+    Object partitionBound =
+        OraOopOracleQueries.oraDATEAddJulianDays(jobDateTime, 0, 1);
+
+    String partitionBoundStr =
+        OraOopOracleQueries.oraDATEToString(partitionBound, dateFormat);
+
+    StringBuilder subPartitions = new StringBuilder();
+    for (int idx = 0; idx < numberOfMappers; idx++) {
+      if (idx > 0) {
+        subPartitions.append(",");
+      }
+
+      subPartitions.append(String.format(" SUBPARTITION %s VALUES (%d)",
+          subPartitionNames[idx], idx));
+    }
+
+    String sql =
+        String.format(
+            "CREATE TABLE %s \n" + "%s %s \n" + "PARTITION BY RANGE (%s) \n"
+                + "SUBPARTITION BY LIST (%s) \n" + "(PARTITION %s \n"
+                + "VALUES LESS THAN (to_date('%s', '%s')) \n" + "( %s ) \n"
+                + ") \n" + "AS \n"
+                + "(SELECT t.*, sysdate %s, 0 %s, 0 %s FROM %s t \n"
+                + "WHERE 0=1)", newTable.toString(), noLogging ? "NOLOGGING"
+                : "", tableStorageClause,
+            OraOopConstants.COLUMN_NAME_EXPORT_PARTITION,
+            OraOopConstants.COLUMN_NAME_EXPORT_SUBPARTITION, partitionName,
+            partitionBoundStr, dateFormat, subPartitions.toString(),
+            OraOopConstants.COLUMN_NAME_EXPORT_PARTITION,
+            OraOopConstants.COLUMN_NAME_EXPORT_SUBPARTITION,
+            OraOopConstants.COLUMN_NAME_EXPORT_MAPPER_ROW, templateTable
+                .toString());
+
+    LOG.debug(String.format("SQL generated by %s:\n%s", OraOopUtilities
+        .getCurrentMethodName(), sql));
+
+    try {
+
+      // Create the main export table...
+      PreparedStatement preparedStatement = connection.prepareStatement(sql);
+      preparedStatement.execute(sql);
+      preparedStatement.close();
+    } catch (SQLException ex) {
+      LOG.error(String
+          .format(
+              "The error \"%s\" was encountered when executing the following "
+              + "SQL statement:\n%s",
+              ex.getMessage(), sql));
+      throw ex;
+    }
+  }
+
+  public static void createExportTableForMapper(Connection connection,
+      OracleTable table, String tableStorageClause, OracleTable templateTable,
+      boolean addOraOopPartitionColumns) throws SQLException {
+
+    String sql = "";
+    try {
+
+      // Create the N tables to be used by the mappers...
+      Statement statement = connection.createStatement();
+      if (addOraOopPartitionColumns) {
+        sql =
+            String.format("CREATE TABLE %s \n" + "NOLOGGING %s \n" + "AS \n"
+                + "(SELECT t.*, SYSDATE %s, 0 %s, 0 %s FROM %s t WHERE 0=1)",
+                table.toString(), tableStorageClause,
+                OraOopConstants.COLUMN_NAME_EXPORT_PARTITION,
+                OraOopConstants.COLUMN_NAME_EXPORT_SUBPARTITION,
+                OraOopConstants.COLUMN_NAME_EXPORT_MAPPER_ROW, templateTable
+                    .toString());
+      } else {
+        sql =
+            String.format("CREATE TABLE %s \n" + "NOLOGGING %s \n" + "AS \n"
+                + "(SELECT * FROM %s WHERE 0=1)", table.toString(),
+                tableStorageClause, templateTable.toString());
+      }
+
+      LOG.info(String.format("SQL generated by %s:\n%s", OraOopUtilities
+          .getCurrentMethodName(), sql));
+
+      statement.execute(sql);
+      statement.close();
+    } catch (SQLException ex) {
+      LOG.error(String
+          .format(
+              "The error \"%s\" was encountered when executing the following "
+              + "SQL statement:\n%s",
+              ex.getMessage(), sql));
+      throw ex;
+    }
+  }
+
+  // public static void createExportTablesForMappers(Connection connection
+  // ,String[] mapperTableNames
+  // ,OracleTable templateTable
+  // ,boolean addOraOopPartitionColumns)
+  // throws SQLException {
+  //
+  // for(String tableName : mapperTableNames)
+  // createExportTableForMapper(connection, tableName, templateTable,
+  // addOraOopPartitionColumns);
+  // }
+
+  public static void createMoreExportTablePartitions(Connection connection,
+      OracleTable table, String partitionName, Object jobDateTime,
+      String[] subPartitionNames) throws SQLException {
+
+    String dateFormat = "yyyy-mm-dd hh24:mi:ss";
+
+    Object partitionBound =
+        OraOopOracleQueries.oraDATEAddJulianDays(jobDateTime, 0, 1);
+    String partitionBoundStr =
+        OraOopOracleQueries.oraDATEToString(partitionBound, dateFormat);
+
+    StringBuilder subPartitions = new StringBuilder();
+    for (int idx = 0; idx < subPartitionNames.length; idx++) {
+      if (idx > 0) {
+        subPartitions.append(",");
+      }
+
+      subPartitions.append(String.format(" SUBPARTITION %s VALUES (%d)",
+          subPartitionNames[idx], idx));
+    }
+
+    String sql =
+        String.format("ALTER TABLE %s " + "ADD PARTITION %s "
+            + "VALUES LESS THAN (to_date('%s', '%s'))" + "( %s ) ", table
+            .toString(), partitionName, partitionBoundStr, dateFormat,
+            subPartitions.toString());
+
+    LOG.debug(String.format("SQL generated by %s:\n%s", OraOopUtilities
+        .getCurrentMethodName(), sql));
+
+    try {
+      PreparedStatement preparedStatement = connection.prepareStatement(sql);
+      preparedStatement.execute(sql);
+      preparedStatement.close();
+
+    } catch (SQLException ex) {
+      LOG.error(String
+          .format(
+              "The error \"%s\" was encountered when executing the following "
+              + "SQL statement:\n%s",
+              ex.getMessage(), sql));
+      throw ex;
+    }
+  }
+
+  public static void mergeTable(Connection connection, OracleTable targetTable,
+      OracleTable sourceTable, String[] mergeColumnNames,
+      OracleTableColumns oracleTableColumns, Object oraOopSysDate,
+      int oraOopMapperId, boolean parallelizationEnabled) throws SQLException {
+
+    StringBuilder updateClause = new StringBuilder();
+    StringBuilder insertClause = new StringBuilder();
+    StringBuilder valuesClause = new StringBuilder();
+    for (int idx = 0; idx < oracleTableColumns.size(); idx++) {
+      OracleTableColumn oracleTableColumn = oracleTableColumns.get(idx);
+      String columnName = oracleTableColumn.getName();
+
+      if (insertClause.length() > 0) {
+        insertClause.append(",");
+      }
+      insertClause.append(String.format("target.%s", columnName));
+
+      if (valuesClause.length() > 0) {
+        valuesClause.append(",");
+      }
+      valuesClause.append(String.format("source.%s", columnName));
+
+      if (!OraOopUtilities.stringArrayContains(mergeColumnNames, columnName,
+          true)) {
+
+        // If we're performing a merge, then the table is not partitioned. (If
+        // the table
+        // was partitioned, we'd be deleting and then inserting rows.)
+        if (!columnName
+            .equalsIgnoreCase(OraOopConstants.COLUMN_NAME_EXPORT_PARTITION)
+            && !columnName
+            .equalsIgnoreCase(OraOopConstants.COLUMN_NAME_EXPORT_SUBPARTITION)
+            && !columnName
+            .equalsIgnoreCase(OraOopConstants.COLUMN_NAME_EXPORT_MAPPER_ROW)) {
+
+          if (updateClause.length() > 0) {
+            updateClause.append(",");
+          }
+          updateClause.append(String.format("target.%1$s = source.%1$s",
+              columnName));
+
+        }
+      }
+    }
+
+    String sourceClause = valuesClause.toString();
+
+    String sql =
+        String.format("MERGE %7$s INTO %1$s target \n"
+            + "USING (SELECT %8$s * FROM %2$s) source \n" + "  ON (%3$s) \n"
+            + "WHEN MATCHED THEN \n" + "  UPDATE SET %4$s \n"
+            + "WHEN NOT MATCHED THEN \n" + "  INSERT (%5$s) \n"
+            + "  VALUES (%6$s)", targetTable.toString(),
+            sourceTable.toString(),
+            generateUpdateKeyColumnsWhereClauseFragment(mergeColumnNames,
+                "target", "source"), updateClause.toString(), insertClause
+                .toString(), sourceClause,
+            parallelizationEnabled ? "/*+ append parallel(target) */" : "",
+            parallelizationEnabled ? "/*+parallel*/" : "");
+
+    LOG.info(String.format("Merge SQL statement:\n" + sql));
+
+    Statement statement = connection.createStatement();
+    ResultSet resultSet = statement.executeQuery(sql);
+    resultSet.close();
+    statement.close();
+  }
+
+  public static void updateTable(Connection connection,
+      OracleTable targetTable, OracleTable sourceTable,
+      String[] mergeColumnNames, OracleTableColumns oracleTableColumns,
+      Object oraOopSysDate, int oraOopMapperId, boolean parallelizationEnabled)
+      throws SQLException {
+
+    StringBuilder targetColumnsClause = new StringBuilder();
+    StringBuilder sourceColumnsClause = new StringBuilder();
+    for (int idx = 0; idx < oracleTableColumns.size(); idx++) {
+      OracleTableColumn oracleTableColumn = oracleTableColumns.get(idx);
+      String columnName = oracleTableColumn.getName();
+
+      if (targetColumnsClause.length() > 0) {
+        targetColumnsClause.append(",");
+      }
+      targetColumnsClause.append(String.format("a.%s", columnName));
+
+      if (sourceColumnsClause.length() > 0) {
+        sourceColumnsClause.append(",");
+      }
+      sourceColumnsClause.append(String.format("b.%s", columnName));
+    }
+
+    String sourceClause = sourceColumnsClause.toString();
+
+    sourceClause =
+        sourceClause.replaceAll(OraOopConstants.COLUMN_NAME_EXPORT_PARTITION,
+            String.format("to_date('%s', 'yyyy/mm/dd hh24:mi:ss')",
+                OraOopOracleQueries.oraDATEToString(oraOopSysDate,
+                    "yyyy/mm/dd hh24:mi:ss")));
+
+    sourceClause =
+        sourceClause.replaceAll(
+            OraOopConstants.COLUMN_NAME_EXPORT_SUBPARTITION, Integer
+                .toString(oraOopMapperId));
+
+    String sql =
+        String.format("UPDATE %5$s %1$s a \n" + "SET \n" + "(%2$s) \n"
+            + "= (SELECT \n" + "%3$s \n" + "FROM %4$s b \n" + "WHERE %6$s) \n"
+            + "WHERE EXISTS (SELECT null FROM %4$s c " + "WHERE %7$s)",
+            targetTable.toString(), targetColumnsClause.toString(),
+            sourceClause, sourceTable.toString(),
+            parallelizationEnabled ? "/*+ parallel */" : "",
+            generateUpdateKeyColumnsWhereClauseFragment(mergeColumnNames, "b",
+                "a"), generateUpdateKeyColumnsWhereClauseFragment(
+                mergeColumnNames, "c", "a"));
+
+    LOG.info(String.format("Update SQL statement:\n" + sql));
+
+    Statement statement = connection.createStatement();
+    int rowsAffected = statement.executeUpdate(sql);
+
+    LOG.info(String.format(
+        "The number of rows affected by the update SQL was: %d", rowsAffected));
+
+    statement.close();
+  }
+
+  /**
+   * Whether new rows should be included in changes table or not.
+   */
+  public enum CreateExportChangesTableOptions {
+    OnlyRowsThatDiffer, RowsThatDifferPlusNewRows
+  }
+
+  public static int createExportChangesTable(Connection connection,
+      OracleTable tableToCreate, String tableToCreateStorageClause,
+      OracleTable tableContainingUpdates, OracleTable tableToBeUpdated,
+      String[] joinColumnNames, CreateExportChangesTableOptions options,
+      boolean parallelizationEnabled) throws SQLException {
+
+    List<String> columnNames =
+        getTableColumnNames(connection, tableToBeUpdated
+            , false // <- omitLobAndLongColumnsDuringImport
+            , OraOopConstants.Sqoop.Tool.EXPORT
+            , true // <- onlyOraOopSupportedTypes
+            , false // <- omitOraOopPseudoColumns
+        );
+
+    StringBuilder columnClause = new StringBuilder(2 * columnNames.size());
+    for (int idx = 0; idx < columnNames.size(); idx++) {
+      if (idx > 0) {
+        columnClause.append(",");
+      }
+      columnClause.append("a." + columnNames.get(idx));
+    }
+
+    StringBuilder rowEqualityClause = new StringBuilder();
+    for (int idx = 0; idx < columnNames.size(); idx++) {
+      String columnName = columnNames.get(idx);
+
+      // We need to omit the OraOop pseudo columns from the SQL statement that
+      // compares the data in
+      // the two tables we're interested in. Otherwise, EVERY row will be
+      // considered to be changed,
+      // since the values in the pseudo columns will differ. (i.e.
+      // ORAOOP_EXPORT_SYSDATE will differ.)
+      if (columnName
+          .equalsIgnoreCase(OraOopConstants.COLUMN_NAME_EXPORT_PARTITION)
+          || columnName
+             .equalsIgnoreCase(OraOopConstants.COLUMN_NAME_EXPORT_SUBPARTITION)
+          || columnName
+             .equalsIgnoreCase(OraOopConstants.COLUMN_NAME_EXPORT_MAPPER_ROW)) {
+        continue;
+      }
+
+      if (idx > 0) {
+        rowEqualityClause.append("OR");
+      }
+
+      rowEqualityClause.append(String.format("(a.%1$s <> b.%1$s "
+          + "OR (a.%1$s IS NULL AND b.%1$s IS NOT NULL) "
+          + "OR (a.%1$s IS NOT NULL AND b.%1$s IS NULL))", columnName));
+    }
+
+    String sqlJoin = null;
+    switch (options) {
+
+      case OnlyRowsThatDiffer:
+        sqlJoin = "";
+        break;
+
+      case RowsThatDifferPlusNewRows:
+        sqlJoin = "(+)"; // <- An outer-join will cause the "new" rows to be
+                         // included
+        break;
+
+      default:
+        throw new RuntimeException(String.format(
+            "Update %s to cater for the option \"%s\".", OraOopUtilities
+                .getCurrentMethodName(), options.toString()));
+    }
+
+    String sql =
+        String.format("CREATE TABLE %1$s \n" + "NOLOGGING %8$s \n" + "%7$s \n"
+            + "AS \n " + "SELECT \n" + "%5$s \n" + "FROM %2$s a, %3$s b \n"
+            + "WHERE (%4$s) \n" + "AND ( \n" + "%6$s \n" + ")", tableToCreate
+            .toString(), tableContainingUpdates.toString(), tableToBeUpdated
+            .toString(), generateUpdateKeyColumnsWhereClauseFragment(
+            joinColumnNames, "a", "b", sqlJoin), columnClause.toString(),
+            rowEqualityClause.toString(), parallelizationEnabled ? "PARALLEL"
+                : "", tableToCreateStorageClause);
+
+    LOG.info(String.format("The SQL to create the changes-table is:\n%s", sql));
+
+    Statement statement = connection.createStatement();
+
+    long start = System.nanoTime();
+    statement.executeUpdate(sql);
+    double timeInSec = (System.nanoTime() - start) / Math.pow(10, 9);
+    LOG.info(String.format("Time spent creating change-table: %f sec.",
+        timeInSec));
+
+    String indexName = tableToCreate.toString().replaceAll("CHG", "IDX");
+    start = System.nanoTime();
+    statement.execute(String.format("CREATE INDEX %s ON %s (%s)", indexName,
+        tableToCreate.toString(), OraOopUtilities
+            .stringArrayToCSV(joinColumnNames)));
+    timeInSec = (System.nanoTime() - start) / Math.pow(10, 9);
+    LOG.info(String.format("Time spent creating change-table index: %f sec.",
+        timeInSec));
+
+    int changeTableRowCount = 0;
+
+    ResultSet resultSet =
+        statement.executeQuery(String.format("select count(*) from %s",
+            tableToCreate.toString()));
+    resultSet.next();
+    changeTableRowCount = resultSet.getInt(1);
+    LOG.info(String.format("The number of rows in the change-table is: %d",
+        changeTableRowCount));
+
+    statement.close();
+    return changeTableRowCount;
+  }
+
+  public static void deleteRowsFromTable(Connection connection,
+      OracleTable tableToDeleteRowsFrom,
+      OracleTable tableContainingRowsToDelete, String[] joinColumnNames,
+      boolean parallelizationEnabled) throws SQLException {
+
+    String sql =
+        String.format("DELETE %4$s FROM %1$s a \n" + "WHERE EXISTS ( \n"
+            + "SELECT null FROM %3$s b WHERE \n" + "%2$s)",
+            tableToDeleteRowsFrom.toString(),
+            generateUpdateKeyColumnsWhereClauseFragment(joinColumnNames, "a",
+                "b"), tableContainingRowsToDelete.toString(),
+            parallelizationEnabled ? "/*+ parallel */" : "");
+
+    LOG.info(String.format("The SQL to delete rows from a table:\n%s", sql));
+
+    Statement statement = connection.createStatement();
+    int rowsAffected = statement.executeUpdate(sql);
+
+    LOG.info(String.format(
+        "The number of rows affected by the delete SQL was: %d", rowsAffected));
+
+    statement.close();
+  }
+
+  public static void insertRowsIntoExportTable(Connection connection,
+      OracleTable tableToInsertRowsInto,
+      OracleTable tableContainingRowsToInsert, Object oraOopSysDate,
+      int oraOopMapperId, boolean parallelizationEnabled) throws SQLException {
+
+    List<String> columnNames =
+        getTableColumnNames(connection, tableToInsertRowsInto);
+
+    StringBuilder columnClause =
+        new StringBuilder(2 + (2 * columnNames.size()));
+    for (int idx = 0; idx < columnNames.size(); idx++) {
+      if (idx > 0) {
+        columnClause.append(",");
+      }
+      columnClause.append(columnNames.get(idx));
+    }
+
+    String columnsClause = columnClause.toString();
+
+    String sql =
+        String.format("insert %4$s \n" + "into %1$s \n" + "select \n"
+            + "%2$s \n" + "from %3$s", tableToInsertRowsInto.toString(),
+            columnsClause, tableContainingRowsToInsert.toString(),
+            parallelizationEnabled ? "/*+ append parallel */" : "");
+
+    LOG.info(String.format(
+        "The SQL to insert rows from one table into another:\n%s", sql));
+
+    Statement statement = connection.createStatement();
+    ResultSet resultSet = statement.executeQuery(sql);
+    resultSet.close();
+    statement.close();
+  }
+
+  public static boolean doesIndexOnColumnsExist(Connection connection,
+      OracleTable oracleTable, String[] columnNames) throws SQLException {
+
+    // Attempts to find an index on the table that *starts* with the N column
+    // names passed.
+    // These columns can be in any order.
+
+    String columnNamesInClause =
+        OraOopUtilities.stringArrayToCSV(columnNames, "'");
+
+    String sql =
+        String.format("SELECT b.index_name, \n"
+            + "  sum(case when b.column_name in (%1$s) then 1 end) num_cols \n"
+            + "FROM dba_indexes a, dba_ind_columns b \n" + "WHERE \n"
+            + "a.owner = b.index_owner \n"
+            + "AND a.index_name = b.index_name \n" + "AND b.table_owner = ? \n"
+            + "AND b.table_name = ? \n" + "AND a.status = 'VALID' \n"
+            + "AND b.column_position <= ? \n" + "GROUP BY b.index_name \n"
+            + "HAVING sum(case when b.column_name in (%1$s) then 1 end) = ?",
+            columnNamesInClause);
+
+    PreparedStatement statement = connection.prepareStatement(sql);
+    statement.setString(1, oracleTable.getSchema());
+    statement.setString(2, oracleTable.getName());
+    statement.setInt(3, columnNames.length);
+    statement.setInt(4, columnNames.length);
+
+    LOG.debug(String.format("SQL to find an index on the columns %s:\n%s",
+        columnNamesInClause, sql));
+
+    ResultSet resultSet = statement.executeQuery();
+
+    boolean result = false;
+    if (resultSet.next()) {
+      LOG.debug(String
+          .format(
+              "The table %s has an index named %s starting with the column(s) "
+              + "%s (in any order).",
+              oracleTable.toString(), resultSet.getString("index_name"),
+              columnNamesInClause));
+      result = true;
+    }
+
+    resultSet.close();
+    statement.close();
+
+    return result;
+  }
+
+  private static String generateUpdateKeyColumnsWhereClauseFragment(
+      String[] joinColumnNames, String prefix1, String prefix2) {
+
+    return generateUpdateKeyColumnsWhereClauseFragment(joinColumnNames,
+        prefix1, prefix2, "");
+  }
+
+  private static String generateUpdateKeyColumnsWhereClauseFragment(
+      String[] joinColumnNames, String prefix1, String prefix2,
+      String sqlJoinOperator) {
+
+    StringBuilder result = new StringBuilder();
+    for (int idx = 0; idx < joinColumnNames.length; idx++) {
+      String joinColumnName = joinColumnNames[idx];
+      if (idx > 0) {
+        result.append(" AND ");
+      }
+      result.append(String.format("%1$s.%3$s = %2$s.%3$s %4$s", prefix1,
+          prefix2, joinColumnName, sqlJoinOperator));
+    }
+    return result.toString();
+  }
+
+  public static String getCurrentSchema(Connection connection)
+      throws SQLException {
+    String sql = "SELECT SYS_CONTEXT('USERENV','CURRENT_SCHEMA') FROM DUAL";
+
+    PreparedStatement statement = connection.prepareStatement(sql);
+
+    ResultSet resultSet = statement.executeQuery();
+
+    resultSet.next();
+    String result = resultSet.getString(1);
+
+    resultSet.close();
+    statement.close();
+
+    LOG.info("Current schema is: " + result);
+
+    return result;
+  }
+
+  public static String getTableSchema(Connection connection, OracleTable table)
+      throws SQLException {
+    if (table.getSchema() == null || table.getSchema().isEmpty()) {
+      return getCurrentSchema(connection);
+    } else {
+      return table.getSchema();
+    }
+  }
+
+  public static long getCurrentScn(Connection connection) throws SQLException {
+    String sql = "SELECT current_scn FROM v$database";
+    PreparedStatement statement = connection.prepareStatement(sql);
+    ResultSet resultSet = statement.executeQuery();
+
+    resultSet.next();
+    long result = resultSet.getLong(1);
+    resultSet.close();
+    statement.close();
+
+    return result;
+  }
+
+  public static void setLongAtName(PreparedStatement statement,
+      String bindName, long bindValue) throws SQLException {
+    try {
+      methSetLongAtName.invoke(statement, bindName, bindValue);
+    } catch (Exception e) {
+      if (e.getCause() instanceof SQLException) {
+        throw (SQLException) e.getCause();
+      } else {
+        throw new RuntimeException("Could not set bind variable", e);
+      }
+    }
+  }
+
+  public static void setBigDecimalAtName(PreparedStatement statement,
+      String bindName, BigDecimal bindValue) throws SQLException {
+    try {
+      methSetBigDecimalAtName.invoke(statement, bindName, bindValue);
+    } catch (Exception e) {
+      if (e.getCause() instanceof SQLException) {
+        throw (SQLException) e.getCause();
+      } else {
+        throw new RuntimeException("Could not set bind variable", e);
+      }
+    }
+  }
+
+  public static void setStringAtName(PreparedStatement statement,
+      String bindName, String bindValue) throws SQLException {
+    try {
+      methSetStringAtName.invoke(statement, bindName, bindValue);
+    } catch (Exception e) {
+      if (e.getCause() instanceof SQLException) {
+        throw (SQLException) e.getCause();
+      } else {
+        throw new RuntimeException("Could not set bind variable", e);
+      }
+    }
+  }
+
+  public static void setTimestampAtName(PreparedStatement statement,
+      String bindName, Timestamp bindValue) throws SQLException {
+    try {
+      methSetTimestampAtName.invoke(statement, bindName, bindValue);
+    } catch (Exception e) {
+      if (e.getCause() instanceof SQLException) {
+        throw (SQLException) e.getCause();
+      } else {
+        throw new RuntimeException("Could not set bind variable", e);
+      }
+    }
+  }
+
+  public static void setBinaryDoubleAtName(PreparedStatement statement,
+      String bindName, double bindValue) throws SQLException {
+    try {
+      methSetBinaryDoubleAtName.invoke(statement, bindName, bindValue);
+    } catch (Exception e) {
+      if (e.getCause() instanceof SQLException) {
+        throw (SQLException) e.getCause();
+      } else {
+        throw new RuntimeException("Could not set bind variable", e);
+      }
+    }
+  }
+
+  public static void setObjectAtName(PreparedStatement statement,
+      String bindName, Object bindValue) throws SQLException {
+    try {
+      methSetObjectAtName.invoke(statement, bindName, bindValue);
+    } catch (Exception e) {
+      if (e.getCause() instanceof SQLException) {
+        throw (SQLException) e.getCause();
+      } else {
+        throw new RuntimeException("Could not set bind variable", e);
+      }
+    }
+  }
+
+  public static void setBinaryFloatAtName(PreparedStatement statement,
+      String bindName, float bindValue) throws SQLException {
+    try {
+      methSetBinaryFloatAtName.invoke(statement, bindName, bindValue);
+    } catch (Exception e) {
+      if (e.getCause() instanceof SQLException) {
+        throw (SQLException) e.getCause();
+      } else {
+        throw new RuntimeException("Could not set bind variable", e);
+      }
+    }
+  }
+
+  public static void setIntAtName(PreparedStatement statement, String bindName,
+      int bindValue) throws SQLException {
+    try {
+      methSetIntAtName.invoke(statement, bindName, bindValue);
+    } catch (Exception e) {
+      if (e.getCause() instanceof SQLException) {
+        throw (SQLException) e.getCause();
+      } else {
+        throw new RuntimeException("Could not set bind variable", e);
+      }
+    }
+  }
+
+  public static int getOracleType(String name) {
+    Integer result = ORACLE_TYPES.get(name);
+    if (result == null) {
+      synchronized (ORACLE_TYPES) {
+        try {
+          result = oracleTypesClass.getField(name).getInt(null);
+          ORACLE_TYPES.put(name, result);
+        } catch (Exception e) {
+          throw new RuntimeException("Invalid oracle type specified", e);
+        }
+      }
+    }
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopOutputFormatBase.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopOutputFormatBase.java b/src/java/org/apache/sqoop/manager/oracle/OraOopOutputFormatBase.java
new file mode 100644
index 0000000..7c4d1c5
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopOutputFormatBase.java
@@ -0,0 +1,713 @@
+/**
+ * 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.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import com.cloudera.sqoop.lib.SqoopRecord;
+import com.cloudera.sqoop.mapreduce.AsyncSqlOutputFormat;
+import com.cloudera.sqoop.mapreduce.ExportOutputFormat;
+import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
+
+abstract class OraOopOutputFormatBase<K extends SqoopRecord, V> extends
+    ExportOutputFormat<K, V> {
+
+  private static final OraOopLog LOG = OraOopLogFactory
+      .getLog(OraOopOutputFormatBase.class);
+
+  @Override
+  public void checkOutputSpecs(JobContext context) throws IOException,
+      InterruptedException {
+
+    super.checkOutputSpecs(context);
+
+    Configuration conf = context.getConfiguration();
+
+    // This code is now running on a Datanode in the Hadoop cluster, so we
+    // need to enable debug logging in this JVM...
+    OraOopUtilities.enableDebugLoggingIfRequired(conf);
+  }
+
+  protected int getMapperId(TaskAttemptContext context) {
+
+    return context.getTaskAttemptID().getTaskID().getId();
+  }
+
+  protected void applyMapperJdbcUrl(TaskAttemptContext context, int mapperId) {
+
+    Configuration conf = context.getConfiguration();
+
+    // Retrieve the JDBC URL that should be used by this mapper.
+    // We achieve this by modifying the JDBC URL property in the
+    // configuration, prior to the OraOopDBRecordWriter's (ancestral)
+    // constructor using the configuration to establish a connection
+    // to the database - via DBConfiguration.getConnection()...
+    String mapperJdbcUrlPropertyName =
+        OraOopUtilities.getMapperJdbcUrlPropertyName(mapperId, conf);
+
+    // Get this mapper's JDBC URL
+    String mapperJdbcUrl = conf.get(mapperJdbcUrlPropertyName, null);
+
+    LOG.debug(String.format("Mapper %d has a JDBC URL of: %s", mapperId,
+        mapperJdbcUrl == null ? "<null>" : mapperJdbcUrl));
+
+    if (mapperJdbcUrl != null) {
+      conf.set(DBConfiguration.URL_PROPERTY, mapperJdbcUrl);
+    }
+  }
+
+  protected boolean canUseOracleAppendValuesHint(TaskAttemptContext context) {
+
+    Configuration conf = context.getConfiguration();
+
+    // Should we use the APPEND_VALUES Oracle hint?...
+    // (Yes, if this is Oracle 11.2 or above)...
+    OracleVersion oracleVersion =
+        new OracleVersion(conf.getInt(
+            OraOopConstants.ORAOOP_ORACLE_DATABASE_VERSION_MAJOR, 0), conf
+            .getInt(OraOopConstants.ORAOOP_ORACLE_DATABASE_VERSION_MINOR, 0),
+            0, 0, "");
+
+    boolean result = oracleVersion.isGreaterThanOrEqualTo(11, 2, 0, 0);
+
+    // If there is a BINARY_DOUBLE or BINARY_FLOAT column, then we'll avoid
+    // using
+    // the APPEND_VALUES hint. If there is a NULL in the HDFS file, then we'll
+    // encounter
+    // "ORA-12838: cannot read/modify an object after modifying it in parallel"
+    // due to the JDBC driver issuing the INSERT statement twice to the database
+    // without a COMMIT in between (as was observed via WireShark).
+    // We're not sure why this happens - we just know how to avoid it.
+    if (result) {
+      boolean binaryDoubleColumnExists =
+          conf.getBoolean(OraOopConstants.TABLE_CONTAINS_BINARY_DOUBLE_COLUMN,
+              false);
+      boolean binaryFloatColumnExists =
+          conf.getBoolean(OraOopConstants.TABLE_CONTAINS_BINARY_FLOAT_COLUMN,
+              false);
+      if (binaryDoubleColumnExists || binaryFloatColumnExists) {
+        result = false;
+        LOG.info("The APPEND_VALUES Oracle hint will not be used for the "
+            + "INSERT SQL statement, as the Oracle table "
+            + "contains either a BINARY_DOUBLE or BINARY_FLOAT column.");
+      }
+    }
+
+    return result;
+  }
+
+  protected boolean allowUserToOverrideUseOfTheOracleAppendValuesHint(
+      TaskAttemptContext context, boolean useAppendValuesOracleHint) {
+
+    Configuration conf = context.getConfiguration();
+
+    boolean result = useAppendValuesOracleHint;
+
+    // Has the user forced the use of APPEND_VALUES either on or off?...
+    switch (OraOopUtilities.getOracleAppendValuesHintUsage(conf)) {
+
+      case OFF:
+        result = false;
+        LOG.debug(String
+            .format(
+                "Use of the APPEND_VALUES Oracle hint has been forced OFF. "
+                + "(It was %s to used).",
+                useAppendValuesOracleHint ? "going" : "not going"));
+        break;
+
+      case ON:
+        result = true;
+        LOG.debug(String
+            .format(
+                "Use of the APPEND_VALUES Oracle hint has been forced ON. "
+                + "(It was %s to used).",
+                useAppendValuesOracleHint ? "going" : "not going"));
+        break;
+
+      case AUTO:
+        LOG.debug(String.format("The APPEND_VALUES Oracle hint %s be used.",
+            result ? "will" : "will not"));
+        break;
+
+      default:
+        throw new RuntimeException("Invalid value for APPEND_VALUES.");
+    }
+    return result;
+  }
+
+  protected void updateBatchSizeInConfigurationToAllowOracleAppendValuesHint(
+      TaskAttemptContext context) {
+
+    Configuration conf = context.getConfiguration();
+
+    // If using APPEND_VALUES, check the batch size and commit frequency...
+    int originalBatchesPerCommit =
+        conf.getInt(AsyncSqlOutputFormat.STATEMENTS_PER_TRANSACTION_KEY, 0);
+    if (originalBatchesPerCommit != 1) {
+      conf.setInt(AsyncSqlOutputFormat.STATEMENTS_PER_TRANSACTION_KEY, 1);
+      LOG.info(String
+          .format(
+              "The number of batch-inserts to perform per commit has been "
+                  + "changed from %d to %d. This is in response "
+                  + "to the Oracle APPEND_VALUES hint being used.",
+              originalBatchesPerCommit, 1));
+    }
+
+    int originalBatchSize =
+        conf.getInt(AsyncSqlOutputFormat.RECORDS_PER_STATEMENT_KEY, 0);
+    int minAppendValuesBatchSize =
+        OraOopUtilities.getMinAppendValuesBatchSize(conf);
+    if (originalBatchSize < minAppendValuesBatchSize) {
+      conf.setInt(AsyncSqlOutputFormat.RECORDS_PER_STATEMENT_KEY,
+          minAppendValuesBatchSize);
+      LOG.info(String
+          .format(
+              "The number of rows per batch-insert has been changed from %d "
+                  + "to %d. This is in response "
+                  + "to the Oracle APPEND_VALUES hint being used.",
+              originalBatchSize, minAppendValuesBatchSize));
+    }
+  }
+
+  abstract class OraOopDBRecordWriterBase extends
+      ExportOutputFormat<K, V>.ExportRecordWriter<K, V> {
+
+    protected OracleTable oracleTable; // <- If exporting into a partitioned
+                                       // table, this table will be unique for
+                                       // this mapper
+    private OracleTableColumns oracleTableColumns; // <- The columns in the
+                                                   // table we're inserting rows
+                                                   // into
+    protected int mapperId; // <- The index of this Hadoop mapper
+    protected boolean tableHasMapperRowNumberColumn; // <- Whether the export
+                                                     // table contain the column
+                                                     // ORAOOP_MAPPER_ROW
+    protected long mapperRowNumber; // <- The 1-based row number being processed
+                                    // by this mapper. It's inserted into the
+                                    // "ORAOOP_MAPPER_ROW" column
+
+    public OraOopDBRecordWriterBase(TaskAttemptContext context, int mapperId)
+        throws ClassNotFoundException, SQLException {
+
+      super(context);
+      this.mapperId = mapperId;
+      this.mapperRowNumber = 1;
+
+      Configuration conf = context.getConfiguration();
+
+      // Log any info that might be useful to us...
+      logBatchSettings();
+
+      // Connect to Oracle...
+      Connection connection = this.getConnection();
+
+      String thisOracleInstanceName =
+          OraOopOracleQueries.getCurrentOracleInstanceName(connection);
+      LOG.info(String.format(
+          "This record writer is connected to Oracle via the JDBC URL: \n"
+              + "\t\"%s\"\n" + "\tto the Oracle instance: \"%s\"", connection
+              .toString(), thisOracleInstanceName));
+
+      // Initialize the Oracle session...
+      OracleConnectionFactory.initializeOracleConnection(connection, conf);
+      connection.setAutoCommit(false);
+    }
+
+    protected void setOracleTableColumns(
+        OracleTableColumns newOracleTableColumns) {
+
+      this.oracleTableColumns = newOracleTableColumns;
+      this.tableHasMapperRowNumberColumn =
+          this.oracleTableColumns.findColumnByName(
+              OraOopConstants.COLUMN_NAME_EXPORT_MAPPER_ROW) != null;
+    }
+
+    protected OracleTableColumns getOracleTableColumns() {
+
+      return this.oracleTableColumns;
+    }
+
+    protected void getExportTableAndColumns(TaskAttemptContext context)
+        throws SQLException {
+
+      Configuration conf = context.getConfiguration();
+
+      String schema =
+          context.getConfiguration().get(OraOopConstants.ORAOOP_TABLE_OWNER);
+      String localTableName =
+          context.getConfiguration().get(OraOopConstants.ORAOOP_TABLE_NAME);
+
+      if (schema == null || schema.isEmpty() || localTableName == null
+          || localTableName.isEmpty()) {
+        throw new RuntimeException(
+            "Unable to recall the schema and name of the Oracle table "
+            + "being exported.");
+      }
+
+      this.oracleTable = new OracleTable(schema, localTableName);
+
+      setOracleTableColumns(OraOopOracleQueries.getTableColumns(this
+          .getConnection(), this.oracleTable, OraOopUtilities
+          .omitLobAndLongColumnsDuringImport(conf), OraOopUtilities
+          .recallSqoopJobType(conf), true // <- onlyOraOopSupportedTypes
+          , false // <- omitOraOopPseudoColumns
+          ));
+    }
+
+    @Override
+    protected PreparedStatement getPreparedStatement(
+        List<SqoopRecord> userRecords) throws SQLException {
+
+      Connection connection = this.getConnection();
+
+      String sql = getBatchSqlStatement();
+      LOG.debug(String.format("Prepared Statement SQL:\n%s", sql));
+
+      PreparedStatement statement;
+
+      try {
+        // Synchronize on connection to ensure this does not conflict
+        // with the operations in the update thread.
+        synchronized (connection) {
+          statement = connection.prepareStatement(sql);
+        }
+
+        configurePreparedStatement(statement, userRecords);
+      } catch (Exception ex) {
+        if (ex instanceof SQLException) {
+          throw (SQLException) ex;
+        } else {
+          LOG.error(String.format("The following error occurred during %s",
+              OraOopUtilities.getCurrentMethodName()), ex);
+          throw new SQLException(ex);
+        }
+      }
+
+      return statement;
+    }
+
+    @Override
+    protected boolean isBatchExec() {
+
+      return true;
+    }
+
+    @Override
+    protected String getInsertStatement(int numRows) {
+
+      throw new UnsupportedOperationException(String.format(
+          "%s should not be called, as %s operates in batch mode.",
+          OraOopUtilities.getCurrentMethodName(), this.getClass().getName()));
+    }
+
+    protected String getBatchInsertSqlStatement(String oracleHint) {
+
+      // String[] columnNames = this.getColumnNames();
+      StringBuilder sqlNames = new StringBuilder();
+      StringBuilder sqlValues = new StringBuilder();
+
+      /*
+       * NOTE: "this.oracleTableColumns" may contain a different list of columns
+       * than "this.getColumnNames()". This is because: (1)
+       * "this.getColumnNames()" includes columns with data-types that are not
+       * supported by OraOop. (2) "this.oracleTableColumns" includes any
+       * pseudo-columns that we've added to the export table (and don't exist in
+       * the HDFS file being read). For example, if exporting to a partitioned
+       * table (that OraOop created), there are two pseudo-columns we added to
+       * the table to identify the export job and the mapper.
+       */
+
+      int colCount = 0;
+      for (int idx = 0; idx < this.oracleTableColumns.size(); idx++) {
+        OracleTableColumn oracleTableColumn = this.oracleTableColumns.get(idx);
+        String columnName = oracleTableColumn.getName();
+
+        // column names...
+        if (colCount > 0) {
+          sqlNames.append("\n,");
+        }
+        sqlNames.append(columnName);
+
+        // column values...
+        if (colCount > 0) {
+          sqlValues.append("\n,");
+        }
+
+        String pseudoColumnValue =
+            generateInsertValueForPseudoColumn(columnName);
+
+        String bindVarName = null;
+
+        if (pseudoColumnValue != null) {
+          bindVarName = pseudoColumnValue;
+        } else if (oracleTableColumn.getOracleType() == OraOopOracleQueries
+            .getOracleType("STRUCT")) {
+          if (oracleTableColumn.getDataType().equals(
+              OraOopConstants.Oracle.URITYPE)) {
+            bindVarName =
+                String.format("urifactory.getUri(%s)",
+                    columnNameToBindVariable(columnName));
+          }
+        } else if (getConf().getBoolean(
+            OraOopConstants.ORAOOP_MAP_TIMESTAMP_AS_STRING,
+            OraOopConstants.ORAOOP_MAP_TIMESTAMP_AS_STRING_DEFAULT)) {
+          if (oracleTableColumn.getOracleType() == OraOopOracleQueries
+              .getOracleType("DATE")) {
+            bindVarName =
+                String.format("to_date(%s, 'yyyy-mm-dd hh24:mi:ss')",
+                    columnNameToBindVariable(columnName));
+          } else if (oracleTableColumn.getOracleType() == OraOopOracleQueries
+              .getOracleType("TIMESTAMP")) {
+            bindVarName =
+                String.format("to_timestamp(%s, 'yyyy-mm-dd hh24:mi:ss.ff')",
+                    columnNameToBindVariable(columnName));
+          } else if (oracleTableColumn.getOracleType() == OraOopOracleQueries
+              .getOracleType("TIMESTAMPTZ")) {
+            bindVarName =
+                String.format(
+                    "to_timestamp_tz(%s, 'yyyy-mm-dd hh24:mi:ss.ff TZR')",
+                    columnNameToBindVariable(columnName));
+          } else if (oracleTableColumn.getOracleType() == OraOopOracleQueries
+              .getOracleType("TIMESTAMPLTZ")) {
+            bindVarName =
+                String.format(
+                    "to_timestamp_tz(%s, 'yyyy-mm-dd hh24:mi:ss.ff TZR')",
+                    columnNameToBindVariable(columnName));
+          }
+        }
+
+        if (bindVarName == null) {
+          bindVarName = columnNameToBindVariable(columnName);
+        }
+
+        sqlValues.append(bindVarName);
+
+        colCount++;
+      }
+
+      String sql =
+          String.format("insert %s into %s\n" + "(%s)\n" + "values\n"
+              + "(%s)\n", oracleHint, this.oracleTable.toString(), sqlNames
+              .toString(), sqlValues.toString());
+
+      LOG.info("Batch-Mode insert statement:\n" + sql);
+      return sql;
+    }
+
+    abstract void configurePreparedStatement(
+        PreparedStatement preparedStatement, List<SqoopRecord> userRecords)
+        throws SQLException;
+
+    private void setBindValueAtName(PreparedStatement statement,
+        String bindValueName, Object bindValue, OracleTableColumn column)
+        throws SQLException {
+      if (column.getOracleType()
+          == OraOopOracleQueries.getOracleType("NUMBER")) {
+        OraOopOracleQueries.setBigDecimalAtName(statement, bindValueName,
+            (BigDecimal) bindValue);
+      } else if (column.getOracleType() == OraOopOracleQueries
+          .getOracleType("VARCHAR")) {
+        OraOopOracleQueries.setStringAtName(statement, bindValueName,
+            (String) bindValue);
+      } else if (column.getOracleType() == OraOopOracleQueries
+          .getOracleType("TIMESTAMP")
+          || column.getOracleType() == OraOopOracleQueries
+              .getOracleType("TIMESTAMPTZ")
+          || column.getOracleType() == OraOopOracleQueries
+              .getOracleType("TIMESTAMPLTZ")) {
+        Object objValue = bindValue;
+        if (objValue instanceof Timestamp) {
+          Timestamp value = (Timestamp) objValue;
+          OraOopOracleQueries.setTimestampAtName(statement, bindValueName,
+              value);
+        } else {
+          String value = (String) objValue;
+
+          if (value == null || value.equalsIgnoreCase("null")) {
+            value = "";
+          }
+
+          OraOopOracleQueries.setStringAtName(statement, bindValueName, value);
+        }
+      } else if (column.getOracleType() == OraOopOracleQueries
+          .getOracleType("BINARY_DOUBLE")) {
+        Double value = (Double) bindValue;
+        if (value != null) {
+          OraOopOracleQueries.setBinaryDoubleAtName(statement, bindValueName,
+              value);
+        } else {
+          OraOopOracleQueries.setObjectAtName(statement, bindValueName, null);
+        }
+      } else if (column.getOracleType() == OraOopOracleQueries
+          .getOracleType("BINARY_FLOAT")) {
+        Float value = (Float) bindValue;
+        if (value != null) {
+          OraOopOracleQueries.setBinaryFloatAtName(statement, bindValueName,
+              value);
+        } else {
+          OraOopOracleQueries.setObjectAtName(statement, bindValueName, null);
+        }
+      } else if (column.getOracleType() == OraOopOracleQueries
+          .getOracleType("STRUCT")) { // <- E.g. URITYPE
+        if (column.getDataType().equals(OraOopConstants.Oracle.URITYPE)) {
+          String value = (String) bindValue;
+          OraOopOracleQueries.setStringAtName(statement, bindValueName, value);
+        } else {
+          String msg =
+              String.format(
+                  "%s needs to be updated to cope with the data-type: %s "
+                      + "where the Oracle data_type is \"%s\".",
+                  OraOopUtilities.getCurrentMethodName(), column.getDataType(),
+                  column.getOracleType());
+          LOG.error(msg);
+          throw new UnsupportedOperationException(msg);
+        }
+      } else {
+        // LOB data-types are currently not supported during
+        // a Sqoop Export.
+        // JIRA: SQOOP-117
+        // OraOopConstants.SUPPORTED_EXPORT_ORACLE_DATA_TYPES_CLAUSE
+        // will already have excluded all LOB columns.
+
+        // case oracle.jdbc.OracleTypes.CLOB:
+        // {
+        // oracle.sql.CLOB clob = new
+        // oracle.sql.CLOB(connection);
+        // Object value = fieldMap.get(colName);
+        // //clob.set
+        // statement.setCLOBAtName(bindValueName, clob);
+        // break;
+        // }
+        String msg =
+            String.format(
+                "%s may need to be updated to cope with the data-type: %s",
+                OraOopUtilities.getCurrentMethodName(), column.getOracleType());
+        LOG.debug(msg);
+
+        OraOopOracleQueries
+            .setObjectAtName(statement, bindValueName, bindValue);
+      }
+    }
+
+    protected void configurePreparedStatementColumns(
+        PreparedStatement statement, Map<String, Object> fieldMap)
+        throws SQLException {
+
+      String bindValueName;
+
+      if (this.tableHasMapperRowNumberColumn) {
+        bindValueName =
+            columnNameToBindVariable(
+                OraOopConstants.COLUMN_NAME_EXPORT_MAPPER_ROW).replaceFirst(
+                ":", "");
+        try {
+          OraOopOracleQueries.setLongAtName(statement, bindValueName,
+              this.mapperRowNumber);
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+        this.mapperRowNumber++;
+      }
+
+      Iterator<String> columnNameIterator = fieldMap.keySet().iterator();
+      while (columnNameIterator.hasNext()) {
+        String colName = columnNameIterator.next();
+        bindValueName = columnNameToBindVariable(colName).replaceFirst(":", "");
+
+        OracleTableColumn oracleTableColumn =
+            oracleTableColumns.findColumnByName(colName);
+        setBindValueAtName(statement, bindValueName, fieldMap.get(colName),
+            oracleTableColumn);
+      }
+      statement.addBatch();
+    }
+
+    abstract String getBatchSqlStatement();
+
+    protected String columnNameToBindVariable(String columnName) {
+
+      return ":" + columnName;
+    }
+
+    @Override
+    public void write(K key, V value) throws InterruptedException, IOException {
+
+      try {
+        super.write(key, value);
+      } catch (IOException ex) {
+        // This IOException may contain a SQLException that occurred
+        // during the batch insert...
+        showSqlBatchErrorDetails(ex);
+        throw ex;
+      }
+    }
+
+    @Override
+    public void close(TaskAttemptContext context) throws IOException,
+        InterruptedException {
+
+      try {
+        super.close(context);
+      } catch (IOException ex) {
+        // This IOException may contain a SQLException that occurred
+        // during the batch insert...
+        showSqlBatchErrorDetails(ex);
+        throw ex;
+      }
+    }
+
+    private void showSqlBatchErrorDetails(Exception exception) {
+
+      if (OraOopUtilities.oracleSessionHasBeenKilled(exception)) {
+        LOG.info("\n*********************************************************"
+            + "\nThe Oracle session in use has been killed by a 3rd party."
+            + "\n*********************************************************");
+      }
+
+      /*
+       * Unfortunately, BatchUpdateException.getUpdateCounts() only returns
+       * information about UPDATE statements (not INSERT) statements. Since
+       * we're only performing INSERT statements, there's no extra information
+       * we can provide to the user at this point.
+       */
+
+      // if(exception == null)
+      // return;
+      //
+      // if(exception instanceof BatchUpdateException) {
+      // BatchUpdateException ex = (BatchUpdateException)exception;
+      //
+      // int[] updateCounts = ex.getUpdateCounts();
+      // LOG.error("The number of successful updates was: " +
+      // updateCounts.length);
+      //
+      // // Recurse for chained exceptions...
+      // SQLException nextEx = ex.getNextException();
+      // while(nextEx != null) {
+      // showSqlBatchErrorDetails(nextEx);
+      // nextEx = nextEx.getNextException();
+      // }
+      // }
+      //
+      // // Recurse for nested exceptions...
+      // Throwable cause = exception.getCause();
+      // if(cause instanceof Exception)
+      // showSqlBatchErrorDetails((Exception)cause);
+
+    }
+
+    protected Object getJobSysDate(TaskAttemptContext context) {
+
+      Configuration conf = context.getConfiguration();
+      return OraOopUtilities.recallOracleDateTime(conf,
+          OraOopConstants.ORAOOP_JOB_SYSDATE);
+    }
+
+    protected OracleTable createUniqueMapperTable(TaskAttemptContext context)
+        throws SQLException {
+
+      // This mapper inserts data into a unique table before either:
+      // - exchanging it into a subpartition of the 'real' export table; or
+      // - merging it into the 'real' export table.
+
+      Configuration conf = context.getConfiguration();
+
+      Object sysDateTime = getJobSysDate(context);
+
+      String schema = conf.get(OraOopConstants.ORAOOP_TABLE_OWNER);
+      String localTableName = conf.get(OraOopConstants.ORAOOP_TABLE_NAME);
+
+      OracleTable templateTable = new OracleTable(schema, localTableName);
+
+      OracleTable mapperTable =
+          OraOopUtilities.generateExportTableMapperTableName(this.mapperId,
+              sysDateTime, null);
+
+      // If this mapper is being reattempted in response to a failure, we need
+      // to delete the
+      // temporary table created by the previous attempt...
+      OraOopOracleQueries.dropTable(this.getConnection(), mapperTable);
+
+      String temporaryTableStorageClause =
+          OraOopUtilities.getTemporaryTableStorageClause(conf);
+
+      OraOopOracleQueries.createExportTableForMapper(this.getConnection(),
+          mapperTable, temporaryTableStorageClause, templateTable
+          , false); // <- addOraOopPartitionColumns
+
+      LOG.debug(String.format("Created temporary mapper table %s", mapperTable
+          .toString()));
+
+      return mapperTable;
+    }
+
+    protected String generateInsertValueForPseudoColumn(String columnName) {
+
+      if (columnName
+          .equalsIgnoreCase(OraOopConstants.COLUMN_NAME_EXPORT_PARTITION)) {
+
+        String partitionValueStr =
+            this.getConf().get(
+                OraOopConstants.ORAOOP_EXPORT_PARTITION_DATE_VALUE, null);
+        if (partitionValueStr == null) {
+          throw new RuntimeException(
+              "Unable to recall the value of the partition date-time.");
+        }
+
+        return String.format("to_date('%s', '%s')", partitionValueStr,
+            OraOopConstants.ORAOOP_EXPORT_PARTITION_DATE_FORMAT);
+      }
+
+      if (columnName
+          .equalsIgnoreCase(OraOopConstants.COLUMN_NAME_EXPORT_SUBPARTITION)) {
+        return Integer.toString(this.mapperId);
+      }
+
+      return null;
+    }
+
+    protected void logBatchSettings() {
+
+      LOG.info(String.format("The number of rows per batch is: %d",
+          this.rowsPerStmt));
+
+      int stmtsPerTx =
+          this.getConf().getInt(
+              AsyncSqlOutputFormat.STATEMENTS_PER_TRANSACTION_KEY,
+              AsyncSqlOutputFormat.DEFAULT_STATEMENTS_PER_TRANSACTION);
+
+      LOG.info(String.format("The number of batches per commit is: %d",
+          stmtsPerTx));
+    }
+
+  }
+
+}


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

Posted by ve...@apache.org.
http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopOutputFormatInsert.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopOutputFormatInsert.java b/src/java/org/apache/sqoop/manager/oracle/OraOopOutputFormatInsert.java
new file mode 100644
index 0000000..d5eebf4
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopOutputFormatInsert.java
@@ -0,0 +1,263 @@
+/**
+ * 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.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import com.cloudera.sqoop.lib.SqoopRecord;
+
+/**
+ * Insert into an Oracle table based on emitted keys.
+ */
+public class OraOopOutputFormatInsert<K extends SqoopRecord, V> extends
+    OraOopOutputFormatBase<K, V> {
+
+  private static final OraOopLog LOG = OraOopLogFactory
+      .getLog(OraOopOutputFormatInsert.class);
+
+  /**
+   * Type of insert to use - direct or partition exchange load.
+   */
+  public enum InsertMode {
+    DirectInsert, ExchangePartition
+  }
+
+  @Override
+  public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
+      throws IOException {
+    OraOopUtilities.checkJavaSecurityEgd();
+    Configuration conf = context.getConfiguration();
+
+    int mapperId = this.getMapperId(context);
+    applyMapperJdbcUrl(context, mapperId);
+
+    // Is each mapper inserting rows into a unique table?...
+    InsertMode insertMode = OraOopUtilities.getExportInsertMode(conf);
+
+    // Should we use the APPEND_VALUES Oracle hint?...
+    boolean useAppendValuesOracleHint = false;
+    if (insertMode == InsertMode.ExchangePartition) {
+      // NB: "Direct inserts" cannot utilize APPEND_VALUES, otherwise Oracle
+      // will serialize
+      // the N mappers, causing a lot of lock contention.
+      useAppendValuesOracleHint = this.canUseOracleAppendValuesHint(context);
+    }
+
+    // Has the user forced the use of APPEND_VALUES either on or off?...
+    useAppendValuesOracleHint =
+        allowUserToOverrideUseOfTheOracleAppendValuesHint(context,
+            useAppendValuesOracleHint);
+
+    // If using APPEND_VALUES, check the batch size and commit frequency...
+    if (useAppendValuesOracleHint) {
+      updateBatchSizeInConfigurationToAllowOracleAppendValuesHint(context);
+    }
+
+    // Create the Record Writer...
+    OraOopDBRecordWriterInsert result = null;
+    try {
+      result =
+          new OraOopDBRecordWriterInsert(context, mapperId, insertMode,
+              useAppendValuesOracleHint);
+    } catch (NoClassDefFoundError ex) {
+      throw new IOException(String.format(
+          "Unable to create an instance of OraOopDBRecordWriterInsert.\n"
+              + "The classpath is:\n%s", OraOopUtilities.getJavaClassPath()),
+          ex);
+    } catch (Exception ex) {
+      throw new IOException(ex);
+    }
+
+    try {
+      result.getExportTableAndColumns(context);
+    } catch (SQLException ex) {
+      throw new IOException(ex);
+    }
+
+    return result;
+  }
+
+  /**
+   * Insert into an Oracle table based on emitted keys.
+   */
+  public class OraOopDBRecordWriterInsert extends OraOopDBRecordWriterBase {
+
+    private String sqlStatement; // <- The SQL used when inserting batches of
+                                 // rows into the Oracle table
+    private InsertMode insertMode; // <- The modus operandi of this class. i.e.
+                                   // Whether we insert into the Oracle table
+                                   // directly, or insert data into a separate
+                                   // table and then perform an EXCHANGE
+                                   // PARTITION statement.
+    private boolean useAppendValuesOracleHint; // <- Whether to use the
+                                               // " /*+APPEND_VALUES*/ " hint
+                                               // within the Oracle SQL
+                                               // statement we generate
+    private String subPartitionName; // <- The name of the subpartition in the
+                                     // "main table" that this mappers unique
+                                     // table will be exchanged with
+
+    public OraOopDBRecordWriterInsert(TaskAttemptContext context, int mapperId,
+        InsertMode insertMode, boolean useAppendValuesOracleHint)
+        throws ClassNotFoundException, SQLException {
+
+      super(context, mapperId);
+      this.insertMode = insertMode;
+      this.useAppendValuesOracleHint = useAppendValuesOracleHint;
+    }
+
+    @Override
+    protected void getExportTableAndColumns(TaskAttemptContext context)
+        throws SQLException {
+
+      Configuration conf = context.getConfiguration();
+
+      switch (this.insertMode) {
+
+        case DirectInsert:
+          super.getExportTableAndColumns(context);
+          break;
+
+        case ExchangePartition:
+          // This mapper inserts data into a unique table before exchanging it
+          // into
+          // a subpartition of the 'real' export table...
+
+          this.oracleTable = createUniqueMapperTable(context);
+          setOracleTableColumns(OraOopOracleQueries.getTableColumns(this
+              .getConnection(), this.oracleTable, OraOopUtilities
+              .omitLobAndLongColumnsDuringImport(conf), OraOopUtilities
+              .recallSqoopJobType(conf), true // <- onlyOraOopSupportedTypes
+              , false) // <- omitOraOopPseudoColumns
+          );
+
+          this.subPartitionName =
+              OraOopUtilities.generateExportTableSubPartitionName(
+                  this.mapperId, this.getJobSysDate(context), conf);
+
+          break;
+
+        default:
+          throw new RuntimeException(String.format(
+              "Update %s to cater for the insertMode \"%s\".", OraOopUtilities
+                  .getCurrentMethodName(), this.insertMode.toString()));
+      }
+
+    }
+
+    @Override
+    public void closeConnection(TaskAttemptContext context)
+        throws SQLException {
+
+      // If this mapper is inserting data into a unique table, we'll now
+      // move this data into the main export table...
+      if (this.insertMode == InsertMode.ExchangePartition) {
+
+        // Perform an "exchange subpartition" operation on the "main table"
+        // to convert this table into a subpartition of the "main table"...
+        exchangePartitionUniqueMapperTableDataIntoMainExportTable(context);
+
+        LOG.debug(String.format("Dropping temporary mapper table %s",
+            this.oracleTable.toString()));
+        OraOopOracleQueries.dropTable(this.getConnection(), this.oracleTable);
+      }
+
+      super.closeConnection(context);
+    }
+
+    private void exchangePartitionUniqueMapperTableDataIntoMainExportTable(
+        TaskAttemptContext context) throws SQLException {
+
+      String schema =
+          context.getConfiguration().get(OraOopConstants.ORAOOP_TABLE_OWNER);
+      String localTableName =
+          context.getConfiguration().get(OraOopConstants.ORAOOP_TABLE_NAME);
+      OracleTable mainTable = new OracleTable(schema, localTableName);
+
+      try {
+        long start = System.nanoTime();
+
+        OraOopOracleQueries.exchangeSubpartition(this.getConnection(),
+            mainTable, this.subPartitionName, this.oracleTable);
+
+        double timeInSec = (System.nanoTime() - start) / Math.pow(10, 9);
+        LOG.info(String
+            .format(
+                "Time spent performing an \"exchange subpartition with "
+                + "table\": %f sec.",
+                timeInSec));
+      } catch (SQLException ex) {
+        throw new SQLException(
+            String
+                .format(
+                    "Unable to perform an \"exchange subpartition\" operation "
+                        + "for the table %s, for the subpartition named "
+                        + "\"%s\" with the table named \"%s\".",
+                    mainTable.toString(), this.subPartitionName,
+                    this.oracleTable.toString()), ex);
+      }
+    }
+
+    @Override
+    protected String getBatchSqlStatement() {
+
+      if (sqlStatement == null) {
+        this.sqlStatement =
+            getBatchInsertSqlStatement(this.useAppendValuesOracleHint
+                ? "/*+APPEND_VALUES*/" : "");
+      }
+
+      return this.sqlStatement;
+    }
+
+    @Override
+    void configurePreparedStatement(PreparedStatement statement,
+        List<SqoopRecord> userRecords) throws SQLException {
+
+      Map<String, Object> fieldMap;
+      try {
+        for (SqoopRecord record : userRecords) {
+          fieldMap = record.getFieldMap();
+
+          configurePreparedStatementColumns(statement, fieldMap);
+        }
+
+      } catch (Exception ex) {
+        if (ex instanceof SQLException) {
+          throw (SQLException) ex;
+        } else {
+          LOG.error(String.format("The following error occurred during %s",
+              OraOopUtilities.getCurrentMethodName()), ex);
+          throw new SQLException(ex);
+        }
+      }
+
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopOutputFormatUpdate.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopOutputFormatUpdate.java b/src/java/org/apache/sqoop/manager/oracle/OraOopOutputFormatUpdate.java
new file mode 100644
index 0000000..a33768f
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopOutputFormatUpdate.java
@@ -0,0 +1,418 @@
+/**
+ * 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.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import com.cloudera.sqoop.lib.SqoopRecord;
+import org.apache.sqoop.manager.oracle.OraOopOracleQueries.
+           CreateExportChangesTableOptions;
+
+/**
+ * Update an Oracle table based on emitted keys.
+ */
+public class OraOopOutputFormatUpdate<K extends SqoopRecord, V> extends
+    OraOopOutputFormatBase<K, V> {
+
+  private static final OraOopLog LOG = OraOopLogFactory
+      .getLog(OraOopOutputFormatUpdate.class);
+
+  /**
+   * Type of export - straight update or merge (update-insert).
+   */
+  public enum UpdateMode {
+    Update, Merge
+  }
+
+  private enum ExportTableUpdateTechnique {
+    ReInsertUpdatedRows, ReInsertUpdatedRowsAndNewRows, UpdateSql, MergeSql
+  }
+
+  @Override
+  public RecordWriter<K, V> getRecordWriter(TaskAttemptContext context)
+      throws IOException {
+    OraOopUtilities.checkJavaSecurityEgd();
+    Configuration conf = context.getConfiguration();
+
+    // Get the unique JDBC URL to use for this mapper and update the
+    // configuration property
+    // so that the URL is actually used...
+    int mapperId = this.getMapperId(context);
+    applyMapperJdbcUrl(context, mapperId);
+
+    UpdateMode updateMode = OraOopUtilities.getExportUpdateMode(conf);
+
+    boolean useAppendValuesOracleHint = false;
+
+    if (updateMode == UpdateMode.Merge || updateMode == UpdateMode.Update) {
+      // Should we use the APPEND_VALUES Oracle hint?...
+      useAppendValuesOracleHint = this.canUseOracleAppendValuesHint(context);
+    }
+
+    // Has the user forced the use of APPEND_VALUES either on or off?...
+    useAppendValuesOracleHint =
+        allowUserToOverrideUseOfTheOracleAppendValuesHint(context,
+            useAppendValuesOracleHint);
+
+    // If using APPEND_VALUES, check the batch size and commit frequency...
+    if (useAppendValuesOracleHint) {
+      updateBatchSizeInConfigurationToAllowOracleAppendValuesHint(context);
+    }
+
+    // Create the Record Writer...
+    OraOopDBRecordWriterUpdate result = null;
+    try {
+      result =
+          new OraOopDBRecordWriterUpdate(context, mapperId, updateMode,
+              useAppendValuesOracleHint);
+    } catch (NoClassDefFoundError ex) {
+      throw new IOException(String.format(
+          "Unable to create an instance of OraOopDBRecordWriterUpdate.\n"
+              + "The classpath is:\n%s", OraOopUtilities.getJavaClassPath()),
+          ex);
+    } catch (Exception ex) {
+      throw new IOException(ex);
+    }
+
+    try {
+      result.getExportTableAndColumns(context);
+    } catch (SQLException ex) {
+      throw new IOException(ex);
+    }
+
+    return result;
+  }
+
+  /**
+   * Update an Oracle table based on emitted keys.
+   */
+  public class OraOopDBRecordWriterUpdate extends OraOopDBRecordWriterBase {
+
+    private String sqlStatement; // <- The SQL used when updating batches of
+                                 // rows into the Oracle table
+    private String[] updateColumnNames; // <- The name of the column(s) used to
+                                        // match a row in the HDFS file to a row
+                                        // in the Oracle table. i.e. What as
+                                        // specified in the "--update-key" sqoop
+                                        // argument.
+    private UpdateMode updateMode; // <- The modus operandi of this class. i.e.
+                                   // Whether we update the Oracle table
+                                   // directly, or insert data into a separate
+                                   // table and then apply a SQL MERGE
+                                   // statement.
+    private boolean useAppendValuesOracleHint; // <- Whether to use the
+                                               // " /*+APPEND_VALUES*/ " hint
+                                               // within the Oracle SQL
+                                               // statement we generate
+    private boolean tableHasOraOopPartitions; // <- Indicates whether the export
+                                              // table has partitions that were
+                                              // creted by OraOop
+    private long numberOfRowsSkipped; // <- The number of rows encountered
+                                      // during configurePreparedStatement()
+                                      // that had a NULL value for (one of) the
+                                      // update columns. This row was therefore
+                                      // skipped.
+
+    public OraOopDBRecordWriterUpdate(TaskAttemptContext context, int mapperId,
+        UpdateMode updateMode, boolean useAppendValuesOracleHint)
+        throws ClassNotFoundException, SQLException {
+
+      super(context, mapperId);
+
+      Configuration conf = context.getConfiguration();
+
+      this.updateColumnNames =
+          OraOopUtilities.getExportUpdateKeyColumnNames(conf);
+      this.useAppendValuesOracleHint = useAppendValuesOracleHint;
+      this.updateMode = updateMode;
+      this.tableHasOraOopPartitions =
+          conf.getBoolean(OraOopConstants.EXPORT_TABLE_HAS_ORAOOP_PARTITIONS,
+              false);
+    }
+
+    @Override
+    protected void getExportTableAndColumns(TaskAttemptContext context)
+        throws SQLException {
+
+      Configuration conf = context.getConfiguration();
+
+      this.oracleTable = createUniqueMapperTable(context);
+      setOracleTableColumns(OraOopOracleQueries.getTableColumns(this
+          .getConnection(), this.oracleTable, OraOopUtilities
+          .omitLobAndLongColumnsDuringImport(conf), OraOopUtilities
+          .recallSqoopJobType(conf), true // <- onlyOraOopSupportedTypes
+          , false) // <- omitOraOopPseudoColumns
+      );
+    }
+
+    @Override
+    public void closeConnection(TaskAttemptContext context)
+        throws SQLException {
+
+      try {
+
+        if (this.numberOfRowsSkipped > 0) {
+          LOG.warn(String.format(
+              "%d records were skipped due to a NULL value within one of the "
+            + "update-key column(s).\nHaving a NULL value prevents a record "
+            + "from being able to be matched to a row in the Oracle table.",
+                  this.numberOfRowsSkipped));
+        }
+
+        // Now update the "main" export table with data that was inserted into
+        // this mapper's table...
+        updateMainExportTableFromUniqueMapperTable(context,
+            this.updateColumnNames);
+
+        LOG.debug(String.format("Dropping temporary mapper table %s",
+            this.oracleTable.toString()));
+        OraOopOracleQueries.dropTable(this.getConnection(), this.oracleTable);
+      } finally {
+        super.closeConnection(context);
+      }
+    }
+
+    private ExportTableUpdateTechnique getExportTableUpdateTechnique() {
+
+      ExportTableUpdateTechnique result;
+
+      if (this.tableHasOraOopPartitions) {
+        switch (this.updateMode) {
+
+          case Update:
+            result = ExportTableUpdateTechnique.ReInsertUpdatedRows;
+            break;
+
+          case Merge:
+            result = ExportTableUpdateTechnique.ReInsertUpdatedRowsAndNewRows;
+            break;
+
+          default:
+            throw new RuntimeException(String.format(
+                "Update %s to cater for the updateMode \"%s\".",
+                OraOopUtilities.getCurrentMethodName(), this.updateMode
+                    .toString()));
+        }
+      } else {
+        switch (this.updateMode) {
+
+          case Update:
+            result = ExportTableUpdateTechnique.UpdateSql;
+            break;
+
+          case Merge:
+            result = ExportTableUpdateTechnique.MergeSql;
+            break;
+
+          default:
+            throw new RuntimeException(String.format(
+                "Update %s to cater for the updateMode \"%s\".",
+                OraOopUtilities.getCurrentMethodName(), this.updateMode
+                    .toString()));
+        }
+      }
+
+      return result;
+    }
+
+    private void updateMainExportTableFromUniqueMapperTable(
+        TaskAttemptContext context, String[] mergeColumnNames)
+        throws SQLException {
+
+      String schema =
+          context.getConfiguration().get(OraOopConstants.ORAOOP_TABLE_OWNER);
+      String localTableName =
+          context.getConfiguration().get(OraOopConstants.ORAOOP_TABLE_NAME);
+      OracleTable targetTable = new OracleTable(schema, localTableName);
+
+      Object sysDateTime = getJobSysDate(context);
+      OracleTable changesTable =
+          OraOopUtilities.generateExportTableMapperTableName(Integer
+              .toString(this.mapperId)
+              + "_CHG", sysDateTime, null);
+
+      OraOopOracleQueries.CreateExportChangesTableOptions changesTableOptions;
+      boolean parallelizationEnabled =
+          OraOopUtilities.enableOracleParallelProcessingDuringExport(context
+              .getConfiguration());
+
+      ExportTableUpdateTechnique exportTableUpdateTechnique =
+          getExportTableUpdateTechnique();
+      switch (exportTableUpdateTechnique) {
+
+        case ReInsertUpdatedRows:
+        case UpdateSql:
+          changesTableOptions =
+              CreateExportChangesTableOptions.OnlyRowsThatDiffer;
+          break;
+
+        case ReInsertUpdatedRowsAndNewRows:
+        case MergeSql:
+          changesTableOptions =
+              CreateExportChangesTableOptions.RowsThatDifferPlusNewRows;
+          break;
+
+        default:
+          throw new RuntimeException(String.format(
+              "Update %s to cater for the ExportTableUpdateTechnique \"%s\".",
+              OraOopUtilities.getCurrentMethodName(),
+              exportTableUpdateTechnique.toString()));
+      }
+
+      String temporaryTableStorageClause =
+          OraOopUtilities.getTemporaryTableStorageClause(context
+              .getConfiguration());
+
+      try {
+        int changeTableRowCount =
+            OraOopOracleQueries.createExportChangesTable(this.getConnection(),
+                changesTable, temporaryTableStorageClause, this.oracleTable,
+                targetTable, this.updateColumnNames, changesTableOptions,
+                parallelizationEnabled);
+
+        if (changeTableRowCount == 0) {
+          LOG.debug(String.format(
+              "The changes-table does not contain any rows. %s is now exiting.",
+                  OraOopUtilities.getCurrentMethodName()));
+          return;
+        }
+
+        switch (exportTableUpdateTechnique) {
+
+          case ReInsertUpdatedRows:
+          case ReInsertUpdatedRowsAndNewRows:
+
+            OraOopOracleQueries.deleteRowsFromTable(this.getConnection(),
+                targetTable, changesTable, this.updateColumnNames,
+                parallelizationEnabled);
+
+            OraOopOracleQueries.insertRowsIntoExportTable(this.getConnection(),
+                targetTable, changesTable, sysDateTime, this.mapperId,
+                parallelizationEnabled);
+            break;
+
+          case UpdateSql:
+
+            long start = System.nanoTime();
+
+            OraOopOracleQueries.updateTable(this.getConnection(), targetTable,
+                changesTable, this.updateColumnNames, this
+                    .getOracleTableColumns(), sysDateTime, this.mapperId,
+                parallelizationEnabled);
+
+            double timeInSec = (System.nanoTime() - start) / Math.pow(10, 9);
+            LOG.info(String.format("Time spent performing an update: %f sec.",
+                timeInSec));
+            break;
+
+          case MergeSql:
+
+            long mergeStart = System.nanoTime();
+
+            OraOopOracleQueries.mergeTable(this.getConnection(), targetTable,
+                changesTable, this.updateColumnNames, this
+                    .getOracleTableColumns(), sysDateTime, this.mapperId,
+                parallelizationEnabled);
+
+            double mergeTimeInSec = (System.nanoTime() - mergeStart)
+                / Math.pow(10, 9);
+            LOG.info(String.format("Time spent performing a merge: %f sec.",
+                mergeTimeInSec));
+
+            break;
+
+          default:
+            throw new RuntimeException(
+              String.format(
+                "Update %s to cater for the ExportTableUpdateTechnique \"%s\".",
+                        OraOopUtilities.getCurrentMethodName(),
+                        exportTableUpdateTechnique.toString()));
+        }
+
+        this.getConnection().commit();
+      } catch (SQLException ex) {
+        this.getConnection().rollback();
+        throw ex;
+      } finally {
+        OraOopOracleQueries.dropTable(this.getConnection(), changesTable);
+      }
+    }
+
+    @Override
+    protected String getBatchSqlStatement() {
+
+      if (sqlStatement == null) {
+        this.sqlStatement =
+            getBatchInsertSqlStatement(
+                this.useAppendValuesOracleHint ? "/*+APPEND_VALUES*/"
+                : "");
+      }
+
+      return this.sqlStatement;
+    }
+
+    @Override
+    void configurePreparedStatement(PreparedStatement statement,
+        List<SqoopRecord> userRecords) throws SQLException {
+
+      Map<String, Object> fieldMap;
+      try {
+        for (SqoopRecord record : userRecords) {
+          fieldMap = record.getFieldMap();
+
+          boolean updateKeyValueIsNull = false;
+          for (int idx = 0; idx < this.updateColumnNames.length; idx++) {
+            String updateColumnName = this.updateColumnNames[idx];
+            Object updateKeyValue = fieldMap.get(updateColumnName);
+            if (updateKeyValue == null) {
+              this.numberOfRowsSkipped++;
+              updateKeyValueIsNull = true;
+              break;
+            }
+          }
+
+          if (updateKeyValueIsNull) {
+            continue;
+          }
+
+          configurePreparedStatementColumns(statement, fieldMap);
+        }
+
+      } catch (Exception ex) {
+        if (ex instanceof SQLException) {
+          throw (SQLException) ex;
+        } else {
+          LOG.error(String.format("The following error occurred during %s",
+              OraOopUtilities.getCurrentMethodName()), ex);
+          throw new SQLException(ex);
+        }
+      }
+
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopUtilities.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopUtilities.java b/src/java/org/apache/sqoop/manager/oracle/OraOopUtilities.java
new file mode 100644
index 0000000..473a5ae
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopUtilities.java
@@ -0,0 +1,1461 @@
+/**
+ * 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.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.Category;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+
+import com.cloudera.sqoop.Sqoop;
+import com.cloudera.sqoop.SqoopOptions;
+import com.cloudera.sqoop.mapreduce.ExportJobBase;
+
+import org.apache.sqoop.manager.oracle.OraOopOutputFormatInsert.InsertMode;
+import org.apache.sqoop.manager.oracle.OraOopOutputFormatUpdate.UpdateMode;
+
+/**
+ * Utilities used by OraOop.
+ */
+public final class OraOopUtilities {
+
+  private OraOopUtilities() {
+  }
+
+  /**
+   * Used for testing purposes - can get OraOop to call a class to run a report
+   * on various performance metrics.
+   */
+  public static class OraOopStatsReports {
+    private String csvReport;
+    private String performanceReport;
+
+    public String getPerformanceReport() {
+      return performanceReport;
+    }
+
+    public void setPerformanceReport(String newPerformanceReport) {
+      this.performanceReport = newPerformanceReport;
+    }
+
+    public String getCsvReport() {
+      return csvReport;
+    }
+
+    public void setCsvReport(String newCsvReport) {
+      this.csvReport = newCsvReport;
+    }
+  }
+
+  protected static final OraOopLog LOG = OraOopLogFactory
+      .getLog(OraOopUtilities.class.getName());
+
+  public static List<String> copyStringList(List<String> list) {
+
+    List<String> result = new ArrayList<String>(list.size());
+    result.addAll(list);
+    return result;
+  }
+
+  public static OracleTable decodeOracleTableName(
+      String oracleConnectionUserName, String tableStr) {
+
+    String tableOwner;
+    String tableName;
+
+    // These are the possibilities for double-quote location...
+    // table
+    // "table"
+    // schema.table
+    // schema."table"
+    // "schema".table
+    // "schema"."table"
+    String[] tableStrings = tableStr.split("\"");
+
+    switch (tableStrings.length) {
+
+      case 1: // <- table or schema.table
+
+        tableStrings = tableStr.split("\\.");
+
+        switch (tableStrings.length) {
+
+          case 1: // <- No period
+            tableOwner = oracleConnectionUserName.toUpperCase();
+            tableName = tableStrings[0].toUpperCase();
+            break;
+          case 2: // <- 1 period
+            tableOwner = tableStrings[0].toUpperCase();
+            tableName = tableStrings[1].toUpperCase();
+            break;
+          default:
+            LOG.debug(String.format(
+                "Unable to decode the table name (displayed in "
+                    + "double quotes): \"%s\"", tableStr));
+            throw new RuntimeException(String.format(
+                "Unable to decode the table name: %s", tableStr));
+        }
+        break;
+
+      case 2: // <- "table" or schema."table"
+
+        if (tableStrings[0] == null || tableStrings[0].isEmpty()) {
+          tableOwner = oracleConnectionUserName.toUpperCase();
+        } else {
+          tableOwner = tableStrings[0].toUpperCase();
+          // Remove the "." from the end of the schema name...
+          if (tableOwner.endsWith(".")) {
+            tableOwner = tableOwner.substring(0, tableOwner.length() - 1);
+          }
+        }
+
+        tableName = tableStrings[1];
+        break;
+
+      case 3: // <- "schema".table
+
+        tableOwner = tableStrings[1];
+        tableName = tableStrings[2].toUpperCase();
+        // Remove the "." from the start of the table name...
+        if (tableName.startsWith(".")) {
+          tableName = tableName.substring(1, tableName.length());
+        }
+
+        break;
+
+      case 4: // <- "schema"."table"
+        tableOwner = tableStrings[1];
+        tableName = tableStrings[3];
+        break;
+
+      default:
+        LOG.debug(String.format(
+            "Unable to decode the table name (displayed in double "
+                + "quotes): \"%s\"", tableStr));
+        throw new RuntimeException(String.format(
+            "Unable to decode the table name: %s", tableStr));
+
+    }
+    OracleTable result = new OracleTable(tableOwner, tableName);
+    return result;
+  }
+
+  public static OracleTable decodeOracleTableName(
+      String oracleConnectionUserName, String tableStr,
+      org.apache.hadoop.conf.Configuration conf) {
+
+    OracleTable result = new OracleTable();
+
+    // Have we already determined the answer to this question?...
+    if (conf != null) {
+      String tableOwner = conf.get(OraOopConstants.ORAOOP_TABLE_OWNER);
+      String tableName = conf.get(OraOopConstants.ORAOOP_TABLE_NAME);
+      result = new OracleTable(tableOwner, tableName);
+    }
+
+    // If we couldn't look up the answer, then determine it now...
+    if (result.getSchema() == null || result.getName() == null) {
+
+      result = decodeOracleTableName(oracleConnectionUserName, tableStr);
+
+      LOG.debug(String.format(
+          "The Oracle table context has been derived from:\n"
+              + "\toracleConnectionUserName = %s\n" + "\ttableStr = %s\n"
+              + "\tas:\n" + "\towner : %s\n" + "\ttable : %s",
+          oracleConnectionUserName, tableStr, result.getSchema(), result
+              .getName()));
+
+      // Save the answer for next time...
+      if (conf != null) {
+        conf.set(OraOopConstants.ORAOOP_TABLE_OWNER, result.getSchema());
+        conf.set(OraOopConstants.ORAOOP_TABLE_NAME, result.getName());
+      }
+    }
+
+    return result;
+  }
+
+  public static boolean oracleJdbcUrlGenerationDisabled(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    return conf.getBoolean(OraOopConstants.ORAOOP_JDBC_URL_VERBATIM, false);
+  }
+
+  public static boolean userWantsOracleSessionStatisticsReports(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    return conf.getBoolean(OraOopConstants.ORAOOP_REPORT_SESSION_STATISTICS,
+        false);
+  }
+
+  public static boolean enableDebugLoggingIfRequired(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    boolean result = false;
+
+    try {
+
+      Level desiredOraOopLoggingLevel =
+          Level.toLevel(conf.get(OraOopConstants.ORAOOP_LOGGING_LEVEL),
+              Level.INFO);
+
+      Level sqoopLogLevel =
+          Logger.getLogger(Sqoop.class.getName()).getParent().getLevel();
+
+      if (desiredOraOopLoggingLevel == Level.DEBUG
+          || desiredOraOopLoggingLevel == Level.ALL
+          || sqoopLogLevel == Level.DEBUG || sqoopLogLevel == Level.ALL) {
+
+        Category oraOopLogger =
+            Logger.getLogger(OraOopManagerFactory.class.getName()).getParent();
+        oraOopLogger.setLevel(Level.DEBUG);
+        LOG.debug("Enabled OraOop debug logging.");
+        result = true;
+
+        conf.set(OraOopConstants.ORAOOP_LOGGING_LEVEL, Level.DEBUG.toString());
+      }
+    } catch (Exception ex) {
+      LOG.error(String.format(
+          "Unable to determine whether debug logging should be enabled.\n%s",
+          getFullExceptionMessage(ex)));
+    }
+
+    return result;
+  }
+
+  public static String generateDataChunkId(int fileId, int fileBatch) {
+    StringBuilder sb = new StringBuilder();
+    return sb.append(fileId).append("_").append(fileBatch).toString();
+  }
+
+  public static String getCurrentMethodName() {
+
+    StackTraceElement[] stackTraceElements = (new Throwable()).getStackTrace();
+    return String.format("%s()", stackTraceElements[1].getMethodName());
+  }
+
+  public static String[] getDuplicatedStringArrayValues(String[] list,
+      boolean ignoreCase) {
+
+    if (list == null) {
+      throw new IllegalArgumentException("The list argument cannot be null");
+    }
+
+    ArrayList<String> duplicates = new ArrayList<String>();
+
+    for (int idx1 = 0; idx1 < list.length - 1; idx1++) {
+      for (int idx2 = idx1 + 1; idx2 < list.length; idx2++) {
+        if (list[idx1].equals(list[idx2])) {
+          // If c is a duplicate of both a & b, don't add c to the list twice...
+          if (!duplicates.contains(list[idx2])) {
+            duplicates.add(list[idx2]);
+          }
+
+        } else if (ignoreCase && list[idx1].equalsIgnoreCase((list[idx2]))) {
+          // If c is a duplicate of both a & b, don't add c to the list twice...
+          if (stringListIndexOf(duplicates, list[idx2], ignoreCase) == -1) {
+            duplicates.add(list[idx2]);
+          }
+        }
+      }
+    }
+
+    return duplicates.toArray(new String[duplicates.size()]);
+  }
+
+  public static String getFullExceptionMessage(Exception ex) {
+
+    ByteArrayOutputStream arrayStream = new ByteArrayOutputStream();
+    PrintStream printStream = new PrintStream(arrayStream);
+    ex.printStackTrace(printStream);
+    return arrayStream.toString();
+  }
+
+  public static int getMinNumberOfImportMappersAcceptedByOraOop(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    return conf.getInt(OraOopConstants.ORAOOP_MIN_IMPORT_MAPPERS,
+        OraOopConstants.MIN_NUM_IMPORT_MAPPERS_ACCEPTED_BY_ORAOOP);
+  }
+
+  public static int getMinAppendValuesBatchSize(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    return conf.getInt(OraOopConstants.ORAOOP_MIN_APPEND_VALUES_BATCH_SIZE,
+        OraOopConstants.ORAOOP_MIN_APPEND_VALUES_BATCH_SIZE_DEFAULT);
+  }
+
+  public static int getMinNumberOfExportMappersAcceptedByOraOop(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    return conf.getInt(OraOopConstants.ORAOOP_MIN_EXPORT_MAPPERS,
+        OraOopConstants.MIN_NUM_EXPORT_MAPPERS_ACCEPTED_BY_ORAOOP);
+  }
+
+  public static int getMinNumberOfOracleRacActiveInstancesForDynamicJdbcUrlUse(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    return conf.getInt(OraOopConstants.ORAOOP_MIN_RAC_ACTIVE_INSTANCES,
+        OraOopConstants.MIN_NUM_RAC_ACTIVE_INSTANCES_FOR_DYNAMIC_JDBC_URLS);
+  }
+
+  public static int getNumberOfDataChunksPerOracleDataFile(
+      int desiredNumberOfMappers, org.apache.hadoop.conf.Configuration conf) {
+
+    final String MAPPER_MULTIPLIER = "oraoop.datachunk.mapper.multiplier";
+    final String RESULT_INCREMENT = "oraoop.datachunk.result.increment";
+
+    int numberToMultiplyMappersBy = conf.getInt(MAPPER_MULTIPLIER, 2);
+    int numberToIncrementResultBy = conf.getInt(RESULT_INCREMENT, 1);
+
+    // The number of chunks generated will *not* be a multiple of the number of
+    // splits,
+    // to ensure that each split doesn't always get data from the start of each
+    // data-file...
+    int numberOfDataChunksPerOracleDataFile =
+        (desiredNumberOfMappers * numberToMultiplyMappersBy)
+            + numberToIncrementResultBy;
+
+    LOG.debug(String.format("%s:\n" + "\t%s=%d\n" + "\t%s=%d\n"
+        + "\tdesiredNumberOfMappers=%d\n" + "\tresult=%d",
+        getCurrentMethodName(), MAPPER_MULTIPLIER, numberToMultiplyMappersBy,
+        RESULT_INCREMENT, numberToIncrementResultBy, desiredNumberOfMappers,
+        numberOfDataChunksPerOracleDataFile));
+
+    return numberOfDataChunksPerOracleDataFile;
+  }
+
+  public static OraOopConstants.OraOopOracleDataChunkMethod
+      getOraOopOracleDataChunkMethod(Configuration conf) {
+    if (conf == null) {
+      throw new IllegalArgumentException("The conf argument cannot be null");
+    }
+
+    String strMethod =
+        conf.get(OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD);
+    if (strMethod == null) {
+      return OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD_DEFAULT;
+    }
+
+    OraOopConstants.OraOopOracleDataChunkMethod result;
+
+    try {
+      strMethod = strMethod.toUpperCase().trim();
+      result = OraOopConstants.OraOopOracleDataChunkMethod.valueOf(strMethod);
+    } catch (IllegalArgumentException ex) {
+      result = OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD_DEFAULT;
+      LOG.error("An invalid value of \"" + strMethod
+          + "\" was specified for the \""
+          + OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD
+          + "\" configuration property value.\n" + "\tThe default value of "
+          + OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD_DEFAULT
+          + " will be used.");
+    }
+    return result;
+  }
+
+  public static
+      OraOopConstants.OraOopOracleBlockToSplitAllocationMethod
+      getOraOopOracleBlockToSplitAllocationMethod(
+          org.apache.hadoop.conf.Configuration conf,
+          OraOopConstants.OraOopOracleBlockToSplitAllocationMethod
+            defaultMethod) {
+
+    if (conf == null) {
+      throw new IllegalArgumentException("The conf argument cannot be null");
+    }
+
+    String strMethod =
+        conf.get(
+            OraOopConstants.ORAOOP_ORACLE_BLOCK_TO_SPLIT_ALLOCATION_METHOD);
+    if (strMethod == null) {
+      return defaultMethod;
+    }
+
+    OraOopConstants.OraOopOracleBlockToSplitAllocationMethod result;
+
+    try {
+      strMethod = strMethod.toUpperCase().trim();
+      result =
+          OraOopConstants.OraOopOracleBlockToSplitAllocationMethod
+              .valueOf(strMethod);
+    } catch (IllegalArgumentException ex) {
+      result = defaultMethod;
+
+      String errorMsg =
+          String
+              .format(
+                "An invalid value of \"%s\" was specified for the \"%s\" "
+                    + "configuration property value.\n"
+                    + "\tValid values are: %s\n"
+                    + "\tThe default value of %s will be used.",
+                strMethod,
+                OraOopConstants.ORAOOP_ORACLE_BLOCK_TO_SPLIT_ALLOCATION_METHOD,
+                getOraOopOracleBlockToSplitAllocationMethods(), defaultMethod
+                    .name());
+      LOG.error(errorMsg);
+    }
+
+    return result;
+  }
+
+  private static String getOraOopOracleBlockToSplitAllocationMethods() {
+
+    OraOopConstants.OraOopOracleBlockToSplitAllocationMethod[] values =
+        OraOopConstants.OraOopOracleBlockToSplitAllocationMethod.values();
+
+    StringBuilder result =
+        new StringBuilder((2 * values.length) - 1); // <- Include capacity
+                                                    //    for commas
+
+    for (int idx = 0; idx < values.length; idx++) {
+      OraOopConstants.OraOopOracleBlockToSplitAllocationMethod value =
+          values[idx];
+      if (idx > 0) {
+        result.append(" or ");
+      }
+      result.append(value.name());
+    }
+    return result.toString();
+  }
+
+  public static OraOopConstants.OraOopTableImportWhereClauseLocation
+      getOraOopTableImportWhereClauseLocation(
+        org.apache.hadoop.conf.Configuration conf,
+        OraOopConstants.OraOopTableImportWhereClauseLocation defaultLocation) {
+
+    if (conf == null) {
+      throw new IllegalArgumentException("The conf argument cannot be null");
+    }
+
+    String strLocation =
+        conf.get(OraOopConstants.ORAOOP_TABLE_IMPORT_WHERE_CLAUSE_LOCATION);
+    if (strLocation == null) {
+      return defaultLocation;
+    }
+
+    OraOopConstants.OraOopTableImportWhereClauseLocation result;
+
+    try {
+      strLocation = strLocation.toUpperCase().trim();
+      result =
+          OraOopConstants.OraOopTableImportWhereClauseLocation
+              .valueOf(strLocation);
+    } catch (IllegalArgumentException ex) {
+      result = defaultLocation;
+
+      String errorMsg =
+          String
+              .format(
+                  "An invalid value of \"%s\"was specified for the \"%s\" "
+                      + "configuration property value.\n"
+                      + "\tValid values are: %s\n"
+                      + "\tThe default value of %s will be used.", strLocation,
+                  OraOopConstants.ORAOOP_TABLE_IMPORT_WHERE_CLAUSE_LOCATION,
+                  getOraOopTableImportWhereClauseLocations(), defaultLocation
+                      .name());
+      LOG.error(errorMsg);
+    }
+
+    return result;
+  }
+
+  private static String getOraOopTableImportWhereClauseLocations() {
+
+    OraOopConstants.OraOopTableImportWhereClauseLocation[] locationValues =
+        OraOopConstants.OraOopTableImportWhereClauseLocation.values();
+
+    StringBuilder result =
+        new StringBuilder((2 * locationValues.length) - 1); // <- Include
+                                                            //    capacity for
+                                                            //    commas
+
+    for (int idx = 0; idx < locationValues.length; idx++) {
+      OraOopConstants.OraOopTableImportWhereClauseLocation locationValue =
+          locationValues[idx];
+      if (idx > 0) {
+        result.append(" or ");
+      }
+      result.append(locationValue.name());
+    }
+    return result.toString();
+  }
+
+  public static String getOutputDirectory(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    String workingDir = conf.get("mapred.working.dir");
+    String outputDir = conf.get("mapred.output.dir");
+
+    return workingDir + "/" + outputDir;
+  }
+
+  public static String padLeft(String s, int n) {
+    return StringUtils.leftPad(s, n);
+  }
+
+  public static String padRight(String s, int n) {
+    return StringUtils.rightPad(s, n);
+  }
+
+  public static String replaceConfigurationExpression(String str,
+      org.apache.hadoop.conf.Configuration conf) {
+
+    int startPos = str.indexOf('{');
+    int endPos = str.indexOf('}');
+
+    // Example:
+    // alter session set timezone = '{oracle.sessionTimeZone|GMT}';
+
+    if (startPos == -1 || endPos == -1) {
+      return str;
+    }
+
+    String configName = null;
+    String defaultValue = null;
+
+    String expression = str.substring(startPos + 1, endPos);
+    int defaultValuePos = expression.indexOf('|');
+    if (defaultValuePos == -1) {
+      // return expression;
+      configName = expression;
+    } else {
+      configName = expression.substring(0, defaultValuePos);
+      defaultValue = expression.substring(defaultValuePos + 1);
+    }
+
+    if (defaultValue == null) {
+      defaultValue = "";
+    }
+
+    String configValue = conf.get(configName);
+    if (configValue == null) {
+      configValue = defaultValue;
+    }
+
+    String result = str.replace(String.format("{%s}", expression), configValue);
+
+    LOG.debug(String.format("The expression:\n%s\nwas replaced with:\n%s", str,
+        result));
+
+    // Recurse to evaluate any other expressions...
+    result = replaceConfigurationExpression(result, conf);
+
+    return result;
+  }
+
+  public static boolean stackContainsClass(String className) {
+
+    StackTraceElement[] stackTraceElements = (new Throwable()).getStackTrace();
+    for (StackTraceElement stackTraceElement : stackTraceElements) {
+      if (stackTraceElement.getClassName().equalsIgnoreCase(className)) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  public static Object startSessionSnapshot(Connection connection) {
+
+    Object result = null;
+    try {
+
+      Class<?> oraOopOraStatsClass =
+          Class.forName("quest.com.oraOop.oracleStats.OraOopOraStats");
+      Method startSnapshotMethod =
+          oraOopOraStatsClass.getMethod("startSnapshot", Connection.class);
+      if (connection != null) {
+        result = startSnapshotMethod.invoke(null, connection);
+      }
+    } catch (ClassNotFoundException ex) {
+      throw new RuntimeException(ex);
+    } catch (NoSuchMethodException ex) {
+      throw new RuntimeException(ex);
+    } catch (InvocationTargetException ex) {
+      throw new RuntimeException(ex);
+    } catch (IllegalAccessException ex) {
+      throw new RuntimeException(ex);
+    }
+
+    return result;
+  }
+
+  public static OraOopStatsReports stopSessionSnapshot(Object oraOopOraStats) {
+
+    OraOopStatsReports result = new OraOopStatsReports();
+
+    if (oraOopOraStats == null) {
+      return result;
+    }
+
+    try {
+
+      Class<?> oraOopOraStatsClass =
+          Class.forName("quest.com.oraOop.oracleStats.OraOopOraStats");
+      Method finalizeSnapshotMethod =
+          oraOopOraStatsClass.getMethod("finalizeSnapshot", (Class<?>[]) null);
+      finalizeSnapshotMethod.invoke(oraOopOraStats, (Object[]) null);
+
+      Method performanceReportCsvMethod =
+          oraOopOraStatsClass.getMethod("getStatisticsCSV", (Class<?>[]) null);
+      result.setCsvReport((String) performanceReportCsvMethod.invoke(
+          oraOopOraStats, (Object[]) null));
+
+      Method performanceReportMethod =
+          oraOopOraStatsClass.getMethod("performanceReport", (Class<?>[]) null);
+      result.setPerformanceReport((String) performanceReportMethod.invoke(
+          oraOopOraStats, (Object[]) null));
+    } catch (ClassNotFoundException ex) {
+      throw new RuntimeException(ex);
+    } catch (NoSuchMethodException ex) {
+      throw new RuntimeException(ex);
+    } catch (InvocationTargetException ex) {
+      throw new RuntimeException(ex);
+    } catch (IllegalAccessException ex) {
+      throw new RuntimeException(ex);
+    }
+
+    return result;
+  }
+
+  public static boolean stringArrayContains(String[] list, String value,
+      boolean ignoreCase) {
+
+    return stringArrayIndexOf(list, value, ignoreCase) > -1;
+  }
+
+  public static int stringArrayIndexOf(String[] list, String value,
+      boolean ignoreCase) {
+
+    for (int idx = 0; idx < list.length; idx++) {
+      if (list[idx].equals(value)) {
+        return idx;
+      }
+      if (ignoreCase && list[idx].equalsIgnoreCase(value)) {
+        return idx;
+      }
+    }
+    return -1;
+  }
+
+  public static String stringArrayToCSV(String[] list) {
+
+    return stringArrayToCSV(list, "");
+  }
+
+  public static String
+      stringArrayToCSV(String[] list, String encloseValuesWith) {
+
+    StringBuilder result = new StringBuilder((list.length * 2) - 1);
+    for (int idx = 0; idx < list.length; idx++) {
+      if (idx > 0) {
+        result.append(",");
+      }
+      result
+          .append(String.format("%1$s%2$s%1$s", encloseValuesWith, list[idx]));
+    }
+    return result.toString();
+  }
+
+  public static int stringListIndexOf(List<String> list, String value,
+      boolean ignoreCase) {
+
+    for (int idx = 0; idx < list.size(); idx++) {
+      if (list.get(idx).equals(value)) {
+        return idx;
+      }
+      if (ignoreCase && list.get(idx).equalsIgnoreCase(value)) {
+        return idx;
+      }
+    }
+    return -1;
+  }
+
+  public static void writeOutputFile(org.apache.hadoop.conf.Configuration conf,
+      String fileName, String fileText) {
+
+    Path uniqueFileName = null;
+    try {
+      FileSystem fileSystem = FileSystem.get(conf);
+
+      // NOTE: This code is not thread-safe.
+      // i.e. A race-condition could still cause this code to 'fail'.
+
+      int suffix = 0;
+      String fileNameTemplate = fileName + "%s";
+      while (true) {
+        uniqueFileName =
+            new Path(getOutputDirectory(conf), String.format(fileNameTemplate,
+                suffix == 0 ? "" : String.format(" (%d)", suffix)));
+        if (!fileSystem.exists(uniqueFileName)) {
+          break;
+        }
+        suffix++;
+      }
+
+      FSDataOutputStream outputStream =
+          fileSystem.create(uniqueFileName, false);
+      if (fileText != null) {
+        outputStream.writeBytes(fileText);
+      }
+      outputStream.flush();
+      outputStream.close();
+    } catch (IOException ex) {
+      LOG.error(String.format("Error attempting to write the file %s\n" + "%s",
+          (uniqueFileName == null ? "null" : uniqueFileName.toUri()),
+          getFullExceptionMessage(ex)));
+    }
+  }
+
+  /**
+   * Class to wrap details about Oracle connection string.
+   */
+  public static class JdbcOracleThinConnection {
+    private String host;
+    private int port;
+    private String sid;
+    private String service;
+
+    public JdbcOracleThinConnection(String host, int port, String sid,
+        String service) {
+      this.host = host;
+      this.port = port;
+      this.sid = sid;
+      this.service = service;
+    }
+
+    @Override
+    public String toString() {
+
+      // Use the SID if it's available...
+      if (this.sid != null && !this.sid.isEmpty()) {
+        return String.format("jdbc:oracle:thin:@%s:%d:%s", this.host,
+            this.port, this.sid);
+      }
+
+      // Otherwise, use the SERVICE. Note that the service is prefixed by "/",
+      // not by ":"...
+      if (this.service != null && !this.service.isEmpty()) {
+        return String.format("jdbc:oracle:thin:@%s:%d/%s", this.host,
+            this.port, this.service);
+      }
+
+      throw new RuntimeException(
+          "Unable to generate a JDBC URL, as no SID or SERVICE has been "
+            + "provided.");
+
+    }
+
+    public String getHost() {
+      return host;
+    }
+
+    public int getPort() {
+      return port;
+    }
+
+    public String getSid() {
+      return sid;
+    }
+
+    public String getService() {
+      return service;
+    }
+  }
+
+  /**
+   * Thrown if the Oracle connection string cannot be parsed.
+   */
+  public static class JdbcOracleThinConnectionParsingError extends Exception {
+
+    private static final long serialVersionUID = 1559860600099354233L;
+
+    public JdbcOracleThinConnectionParsingError(String message) {
+
+      super(message);
+    }
+
+    public JdbcOracleThinConnectionParsingError(String message,
+                                                Throwable cause) {
+
+      super(message, cause);
+    }
+
+    public JdbcOracleThinConnectionParsingError(Throwable cause) {
+
+      super(cause);
+    }
+  }
+
+  public static String getOracleServiceName(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    return conf.get(OraOopConstants.ORAOOP_ORACLE_RAC_SERVICE_NAME, "");
+  }
+
+  public static String generateOracleSidJdbcUrl(String hostName, int port,
+      String sid) {
+
+    return String.format("jdbc:oracle:thin:@(DESCRIPTION=" + "(ADDRESS_LIST="
+        + "(ADDRESS=(PROTOCOL=TCP)(HOST=%s)(PORT=%d))" + ")"
+        + "(CONNECT_DATA=(SERVER=DEDICATED)(SID=%s))" + ")", hostName, port,
+        sid);
+  }
+
+  public static String generateOracleServiceNameJdbcUrl(String hostName,
+      int port, String serviceName) {
+
+    return String.format("jdbc:oracle:thin:@(DESCRIPTION=" + "(ADDRESS_LIST="
+        + "(ADDRESS=(PROTOCOL=TCP)(HOST=%s)(PORT=%d))" + ")"
+        + "(CONNECT_DATA=(SERVER=DEDICATED)(SERVICE_NAME=%s))" + ")", hostName,
+        port, serviceName);
+  }
+
+  public static String getMapperJdbcUrlPropertyName(int mapperId,
+      org.apache.hadoop.conf.Configuration conf) {
+
+    return String.format("oraoop.mapper.jdbc.url.%d", mapperId);
+  }
+
+  public static final String SQOOP_JOB_TYPE = "oraoop.sqoop.job.type";
+
+  public static void rememberSqoopJobType(OraOopConstants.Sqoop.Tool jobType,
+      org.apache.hadoop.conf.Configuration conf) {
+
+    conf.set(SQOOP_JOB_TYPE, jobType.name());
+  }
+
+  public static OraOopConstants.Sqoop.Tool recallSqoopJobType(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    String jobType = conf.get(SQOOP_JOB_TYPE);
+    if (jobType == null || jobType.isEmpty()) {
+      throw new RuntimeException(
+          "RecallSqoopJobType() cannot be called unless RememberSqoopJobType() "
+            + "has been used.");
+    }
+
+    OraOopConstants.Sqoop.Tool result =
+        OraOopConstants.Sqoop.Tool.valueOf(jobType);
+    return result;
+  }
+
+  public static boolean omitLobAndLongColumnsDuringImport(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    return conf.getBoolean(OraOopConstants.ORAOOP_IMPORT_OMIT_LOBS_AND_LONG,
+        false);
+  }
+
+  public static boolean oracleSessionHasBeenKilled(Exception exception) {
+
+    Throwable ex = exception;
+
+    while (ex != null) {
+      if (ex instanceof SQLException
+          && ((SQLException) ex).getErrorCode() == 28) { // ORA-00028: your
+                                                         // session has been
+                                                         // killed
+        return true;
+      }
+
+      ex = ex.getCause();
+    }
+
+    return false;
+  }
+
+  private static String
+      formatTimestampForOracleObjectName(Object oracleDateTime) {
+
+    // NOTE: Update decodeTimestampFromOracleObjectName() if you modify this
+    // method.
+
+    String jobTimeStr =
+        OraOopOracleQueries.oraDATEToString(oracleDateTime,
+            OraOopConstants.ORACLE_OBJECT_NAME_DATE_TO_STRING_FORMAT_STRING);
+
+    return jobTimeStr;
+
+    // E.g. 20101028_151000 (15 characters)
+  }
+
+  private static Object decodeTimestampFromOracleObjectName(
+      String oracleObjectNameTimestampFragment) {
+
+    String dateString = oracleObjectNameTimestampFragment;
+    String dateFormatString =
+        OraOopConstants.ORACLE_OBJECT_NAME_DATE_TO_STRING_FORMAT_STRING;
+
+    // return oracle.sql.DATE.fromText(oracleObjectNameTimestampFragment
+    // ,OraOopConstants.ORACLE_OBJECT_NAME_DATE_TO_STRING_FORMAT_STRING
+    // ,null);
+
+    /*
+     * Unfortunately, we don't seem to be able to reliably decode strings into
+     * DATE objects using Oracle. For example, the following string will cause
+     * Oracle to throw an "Invalid Oracle date" exception, due to the time
+     * portion starting with a zero...
+     * oracle.sql.DATE.fromText("20101123 091554", "yyyymmdd hh24miss", null);
+     *
+     * Therefore, we need to manually deconstruct the date string and insert
+     * some colons into the time so that Oracle can decode the string. (This is
+     * therefore an Oracle bug we're working around.)
+     */
+
+    try {
+      String year = oracleObjectNameTimestampFragment.substring(0, 4);
+      String month = oracleObjectNameTimestampFragment.substring(4, 6);
+      String day = oracleObjectNameTimestampFragment.substring(6, 8);
+      String hour = oracleObjectNameTimestampFragment.substring(9, 11);
+      String minute = oracleObjectNameTimestampFragment.substring(11, 13);
+      String second = oracleObjectNameTimestampFragment.substring(13, 15);
+      dateString =
+          String.format("%s/%s/%s %s:%s:%s", year, month, day, hour, minute,
+              second);
+      dateFormatString = "yyyy/mm/dd hh24:mi:ss";
+
+      return OraOopOracleQueries.oraDATEFromString(
+          dateString, dateFormatString);
+    } catch (Exception ex) {
+      LOG.debug(String.format(
+          "%s could not convert the string \"%s\" into a DATE via the format "
+              + "string \"%s\".\n" + "The error encountered was:\n%s",
+          getCurrentMethodName(), dateString, dateFormatString,
+          getFullExceptionMessage(ex)));
+
+      return null;
+    }
+  }
+
+  public static String createExportTablePartitionNameFromOracleTimestamp(
+      Object oracleDateTime) {
+
+    // Partition name can be up to 30 characters long and must start with a
+    // letter...
+    return OraOopConstants.EXPORT_TABLE_PARTITION_NAME_PREFIX
+        + formatTimestampForOracleObjectName(oracleDateTime);
+
+    // E.g. ORAOOP_20101028_151000 (22 characters)
+  }
+
+  public static String createExportTableNamePrefixFromOracleTimestamp(
+      Object oracleDateTime) {
+
+    // NOTE: Alter decodeExportTableNamePrefix() if you modify this method.
+
+    // Table name can be 30 characters long and must start with a letter...
+    return OraOopConstants.EXPORT_MAPPER_TABLE_NAME_PREFIX
+        + formatTimestampForOracleObjectName(oracleDateTime);
+    // G1.ORAOOP_20101028_152500 (22 characters) (This is just the prefix,
+    // append "_3" for mapper 4)
+  }
+
+  public static Object decodeExportTableNamePrefix(String tableNamePrefix) {
+
+    if (tableNamePrefix == null || tableNamePrefix.isEmpty()) {
+      return null;
+    }
+
+    if (!tableNamePrefix
+        .startsWith(OraOopConstants.EXPORT_MAPPER_TABLE_NAME_PREFIX)) {
+      return null;
+    }
+
+    String formattedTimestamp =
+        tableNamePrefix.substring(
+            OraOopConstants.EXPORT_MAPPER_TABLE_NAME_PREFIX.length(),
+            tableNamePrefix.length());
+
+    return decodeTimestampFromOracleObjectName(formattedTimestamp);
+  }
+
+  private static boolean userWantsToCreateExportTableFromTemplate(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    String exportTableTemplate =
+        conf.get(OraOopConstants.ORAOOP_EXPORT_CREATE_TABLE_TEMPLATE, "");
+    if (exportTableTemplate.isEmpty()) {
+      return false;
+    }
+
+    OraOopConstants.Sqoop.Tool tool = OraOopUtilities.recallSqoopJobType(conf);
+    switch (tool) {
+      case UNKNOWN:
+      case EXPORT:
+        return true;
+
+      default:
+        return false;
+    }
+  }
+
+  public static boolean enableOracleParallelProcessingDuringExport(
+      org.apache.hadoop.conf.Configuration conf) {
+    return conf.getBoolean(OraOopConstants.ORAOOP_EXPORT_PARALLEL, false);
+  }
+
+  public static boolean userWantsToCreatePartitionedExportTableFromTemplate(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    return userWantsToCreateExportTableFromTemplate(conf)
+        && conf.getBoolean(
+            OraOopConstants.ORAOOP_EXPORT_CREATE_TABLE_PARTITIONED, false);
+  }
+
+  public static boolean userWantsToCreateNonPartitionedExportTableFromTemplate(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    return userWantsToCreateExportTableFromTemplate(conf)
+        && !conf.getBoolean(
+            OraOopConstants.ORAOOP_EXPORT_CREATE_TABLE_PARTITIONED, false);
+  }
+
+  public static String generateExportTableSubPartitionName(int mapperId,
+      Object sysDateTime, org.apache.hadoop.conf.Configuration conf) {
+
+    String partitionName =
+        createExportTablePartitionNameFromOracleTimestamp(sysDateTime);
+
+    String subPartitionName = String.format("%s_MAP_%d" // <- Should allow for
+                                                        // 1,000 mappers before
+                                                        // exceeding 30
+                                                        // characters
+        , partitionName // <- Partition name is 22 characters
+        , mapperId);
+
+    // Check the length of the name...
+    if (subPartitionName.length()
+            > OraOopConstants.Oracle.MAX_IDENTIFIER_LENGTH) {
+      throw new RuntimeException(
+          String
+              .format(
+                  "The generated Oracle subpartition name \"%s\" is longer "
+                + "than %d characters.",
+                  subPartitionName,
+                  OraOopConstants.Oracle.MAX_IDENTIFIER_LENGTH));
+    }
+
+    return subPartitionName;
+  }
+
+  public static String[] generateExportTableSubPartitionNames(int numMappers,
+      Object sysDateTime, org.apache.hadoop.conf.Configuration conf) {
+
+    String[] result = new String[numMappers];
+    for (int idx = 0; idx < numMappers; idx++) {
+      result[idx] = generateExportTableSubPartitionName(idx, sysDateTime, conf);
+    }
+
+    return result;
+  }
+
+  public static OracleTable generateExportTableMapperTableName(int mapperId,
+      Object sysDateTime, String schema) {
+    //mapperId: should allow 10,000,000 mappers before it exceeds 30 characters.
+    return generateExportTableMapperTableName(Integer.toString(mapperId)
+        , sysDateTime, schema);
+  }
+
+  public static OracleTable generateExportTableMapperTableName(
+      String mapperSuffix, Object sysDateTime, String schema) {
+
+    // NOTE: Update decodeExportTableMapperTableName() if you alter this method.
+
+    // Generate a (22 character) prefix to use for the N tables that need to be
+    // created for the N mappers to insert into...
+    String mapperTableNamePrefix =
+        createExportTableNamePrefixFromOracleTimestamp(sysDateTime);
+
+    // Generate the name...
+    String tableName = String.format("%s_%s", mapperTableNamePrefix // <- 22
+                                                                    // chars
+        , mapperSuffix);
+
+    // Check the length of the name...
+    if (tableName.length() > OraOopConstants.Oracle.MAX_IDENTIFIER_LENGTH) {
+      throw new RuntimeException(
+          String
+              .format(
+                  "The generated Oracle table name \"%s\" is longer than "
+                + "%d characters.",
+                  tableName, OraOopConstants.Oracle.MAX_IDENTIFIER_LENGTH));
+    }
+
+    return new OracleTable(schema, tableName);
+  }
+
+  /**
+   * Class to wrap the table name to be used for the mapper.
+   */
+  public static class DecodedExportMapperTableName {
+    private String mapperId; // <- This is not an int, because it might be "CHG"
+                            // in the case of a "changes-table".
+    private Object tableDateTime;
+
+    public String getMapperId() {
+      return mapperId;
+    }
+
+    public void setMapperId(String newMapperId) {
+      this.mapperId = newMapperId;
+    }
+
+    public Object getTableDateTime() {
+      return tableDateTime;
+    }
+
+    public void setTableDateTime(Object newTableDateTime) {
+      this.tableDateTime = newTableDateTime;
+    }
+  }
+
+  public static DecodedExportMapperTableName decodeExportTableMapperTableName(
+      OracleTable oracleTable) {
+
+    DecodedExportMapperTableName result = null;
+    try {
+      int lastUnderScoreIndex = oracleTable.getName().lastIndexOf("_");
+      if (lastUnderScoreIndex == -1) {
+        return result;
+      }
+
+      String dateFragment =
+          oracleTable.getName().substring(0, lastUnderScoreIndex);
+      String mapperIdFragment =
+          oracleTable.getName().substring(lastUnderScoreIndex + 1,
+              oracleTable.getName().length());
+
+      Object sysDateTime = decodeExportTableNamePrefix(dateFragment);
+      if (sysDateTime != null) {
+        result = new DecodedExportMapperTableName();
+        result.setTableDateTime(sysDateTime);
+        result.setMapperId(mapperIdFragment);
+      }
+    } catch (Exception ex) {
+      LOG.debug(
+        String.format(
+         "Error when attempting to decode the export mapper-table name \"%s\".",
+                  oracleTable.toString()), ex);
+    }
+    return result;
+  }
+
+  public static void rememberOracleDateTime(
+      org.apache.hadoop.conf.Configuration conf, String propertyName,
+      String dateTime) {
+    conf.set(propertyName, dateTime);
+  }
+
+  public static Object recallOracleDateTime(
+      org.apache.hadoop.conf.Configuration conf, String propertyName) {
+
+    String dateTimeStr = conf.get(propertyName);
+    if (dateTimeStr == null || dateTimeStr.isEmpty()) {
+      throw new RuntimeException(String.format(
+          "Unable to recall the value of the property \"%s\".", propertyName));
+    }
+
+    return OraOopOracleQueries.oraDATEFromString(dateTimeStr,
+        "yyyy-mm-dd hh24:mi:ss");
+  }
+
+  public static UpdateMode getExportUpdateMode(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    // NB: The Sqoop code does not add the column specified in the
+    // "--update-key" argument
+    // as a configuration property value until quite late in the process. i.e.
+    // After the
+    // OraOopManagerFactory.accept() have been called.
+    // (It is available via sqoopOptions.getUpdateKeyCol() however.)
+    // Therefore, when calling this method you need to be confident that the
+    // export being
+    // performed is actually an "update" export and not an "import" one.
+
+    // String updateKeyCol =
+    // conf.get(ExportJobBase.SQOOP_EXPORT_UPDATE_COL_KEY);
+    // if(updateKeyCol == null ||
+    // updateKeyCol.isEmpty())
+    // throw new
+    // RuntimeException(String.format("This job is not an update-export. "+
+    // "i.e. %s has not been specified."
+    // ,ExportJobBase.SQOOP_EXPORT_UPDATE_COL_KEY));
+
+    UpdateMode updateMode = UpdateMode.Update;
+
+    boolean mergeData =
+        conf.getBoolean(OraOopConstants.ORAOOP_EXPORT_MERGE, false);
+    if (mergeData) {
+      updateMode = UpdateMode.Merge;
+    }
+
+    return updateMode;
+  }
+
+  public static InsertMode getExportInsertMode(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    InsertMode result = InsertMode.DirectInsert;
+
+    if (OraOopUtilities
+        .userWantsToCreatePartitionedExportTableFromTemplate(conf)
+        || conf.getBoolean(OraOopConstants.EXPORT_TABLE_HAS_ORAOOP_PARTITIONS,
+            false)) {
+      result = InsertMode.ExchangePartition;
+    }
+
+    return result;
+  }
+
+  public static String getJavaClassPath() {
+
+    return System.getProperty("java.class.path");
+  }
+
+  public static String replaceAll(String inputString, String textToReplace,
+      String replaceWith) {
+
+    String result = inputString.replaceAll(textToReplace, replaceWith);
+    if (!result.equals(inputString)) {
+      result = replaceAll(result, textToReplace, replaceWith);
+    }
+
+    return result;
+  }
+
+  public static String getTemporaryTableStorageClause(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    String result =
+        conf.get(OraOopConstants.ORAOOP_TEMPORARY_TABLE_STORAGE_CLAUSE, "");
+    if (result == null) {
+      result = "";
+    }
+    return result;
+  }
+
+  public static String getExportTableStorageClause(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    String result =
+        conf.get(OraOopConstants.ORAOOP_EXPORT_TABLE_STORAGE_CLAUSE, "");
+    if (result == null) {
+      result = "";
+    }
+    return result;
+  }
+
+  public static String[] getExportUpdateKeyColumnNames(SqoopOptions options) {
+
+    String updateKey = options.getUpdateKeyCol();
+    return getExtraExportUpdateKeyColumnNames(updateKey, options.getConf());
+  }
+
+  public static String[] getExportUpdateKeyColumnNames(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    String updateKey = conf.get(ExportJobBase.SQOOP_EXPORT_UPDATE_COL_KEY);
+    return getExtraExportUpdateKeyColumnNames(updateKey, conf);
+  }
+
+  /**
+   * Splits a string separated by commas - the elements can be optionally
+   * enclosed in quotes - this allows the elements to have commas in them.
+   *
+   * @param value
+   *          The String to be split
+   * @return A list of values
+   */
+  public static List<String> splitStringList(String value) {
+    List<String> result = new ArrayList<String>();
+    if (value != null && !value.isEmpty()) {
+      Pattern pattern = Pattern.compile("([^\",]*|\"[^\"]*\")(,|$)");
+      Matcher matcher = pattern.matcher(value);
+      while (matcher.find()) {
+        if (matcher.group(1) != null && !matcher.group(1).isEmpty()) {
+          result.add(matcher.group(1));
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Splits a string list separated by commas. If the element is not surrounded
+   * by quotes it will be return in upper case. If the element is enclosed in
+   * quotes it will be returned in the same case and special characters will be
+   * retained.
+   *
+   * @param value
+   *          The String to be split
+   * @return A list of values
+   */
+  public static List<String> splitOracleStringList(String value) {
+    List<String> result = new ArrayList<String>();
+    List<String> splitValue = splitStringList(value);
+    Pattern pattern = Pattern.compile("(\")([^\"]*)(\")");
+    for (String element : splitValue) {
+      Matcher matcher = pattern.matcher(element);
+      if (matcher.find()) {
+        result.add(matcher.group(2));
+      } else {
+        result.add(element.toUpperCase());
+      }
+    }
+    return result;
+  }
+
+  private static String[] getExtraExportUpdateKeyColumnNames(String updateKey,
+      org.apache.hadoop.conf.Configuration conf) {
+
+    if (updateKey == null) {
+      // This must be an "insert-export" if no --update-key has been specified!
+      return new String[0];
+    }
+
+    String extraKeys =
+        conf.get(OraOopConstants.ORAOOP_UPDATE_KEY_EXTRA_COLUMNS, "");
+    if (!extraKeys.isEmpty()) {
+      updateKey += "," + extraKeys;
+    }
+
+    String[] columnNames = updateKey.split(",");
+    for (int idx = 0; idx < columnNames.length; idx++) {
+      columnNames[idx] = columnNames[idx].trim();
+      if (!columnNames[idx].startsWith("\"")) {
+        columnNames[idx] = columnNames[idx].toUpperCase();
+      }
+
+    }
+    return columnNames;
+  }
+
+  public static OraOopConstants.AppendValuesHintUsage
+    getOracleAppendValuesHintUsage(org.apache.hadoop.conf.Configuration conf) {
+
+    if (conf == null) {
+      throw new IllegalArgumentException("The conf argument cannot be null");
+    }
+
+    String strUsage =
+        conf.get(OraOopConstants.ORAOOP_ORACLE_APPEND_VALUES_HINT_USAGE);
+    if (strUsage == null) {
+      return OraOopConstants.AppendValuesHintUsage.AUTO;
+    }
+
+    OraOopConstants.AppendValuesHintUsage result;
+
+    try {
+      strUsage = strUsage.toUpperCase().trim();
+      result = OraOopConstants.AppendValuesHintUsage.valueOf(strUsage);
+    } catch (IllegalArgumentException ex) {
+      result = OraOopConstants.AppendValuesHintUsage.AUTO;
+
+      String errorMsg =
+          String
+              .format(
+                  "An invalid value of \"%s\" was specified for the \"%s\" "
+                      + "configuration property value.\n"
+                      + "\tValid values are: %s\n"
+                      + "\tThe default value of %s will be used.", strUsage,
+                  OraOopConstants.ORAOOP_ORACLE_APPEND_VALUES_HINT_USAGE,
+                  getOraOopOracleAppendValuesHintUsageValues(),
+                  OraOopConstants.AppendValuesHintUsage.AUTO.name());
+      LOG.error(errorMsg);
+    }
+
+    return result;
+  }
+
+  private static String getOraOopOracleAppendValuesHintUsageValues() {
+
+    OraOopConstants.AppendValuesHintUsage[] values =
+        OraOopConstants.AppendValuesHintUsage.values();
+
+    StringBuilder result = new StringBuilder((2 * values.length) - 1); // <-
+                                                                     // Include
+                                                                     // capacity
+                                                                     // for
+                                                                     // commas
+
+    for (int idx = 0; idx < values.length; idx++) {
+      OraOopConstants.AppendValuesHintUsage value = values[idx];
+      if (idx > 0) {
+        result.append(" or ");
+      }
+      result.append(value.name());
+    }
+    return result.toString();
+  }
+
+  public static String getImportHint(
+                           org.apache.hadoop.conf.Configuration conf) {
+    String result = null;
+    result = conf.get(OraOopConstants.IMPORT_QUERY_HINT);
+    if (result == null || result.trim().isEmpty()) {
+      result = "";
+    } else {
+      result = String.format(OraOopConstants.Oracle.HINT_SYNTAX, result);
+    }
+    return result;
+  }
+
+  public static void appendJavaSecurityEgd(Configuration conf) {
+    String mapredJavaOpts = conf.get("mapred.child.java.opts");
+    if (mapredJavaOpts == null
+        || !mapredJavaOpts.contains("-Djava.security.egd")) {
+      StringBuilder newMapredJavaOpts =
+          new StringBuilder("-Djava.security.egd=file:///dev/urandom");
+      if (mapredJavaOpts != null && !mapredJavaOpts.isEmpty()) {
+        newMapredJavaOpts.append(" ").append(mapredJavaOpts);
+      }
+      String newMapredJavaOptsString = newMapredJavaOpts.toString();
+      conf.set("mapred.child.java.opts", newMapredJavaOptsString);
+      LOG.debug("Updated mapred.child.java.opts from \"" + mapredJavaOpts
+          + "\" to \"" + newMapredJavaOptsString + "\"");
+    }
+  }
+
+  public static void checkJavaSecurityEgd() {
+    String javaSecurityEgd = System.getProperty("java.security.egd");
+    if (!"file:///dev/urandom".equals(javaSecurityEgd)) {
+      LOG.warn("System property java.security.egd is not set to "
+             + "file:///dev/urandom - Oracle connections may time out.");
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OracleActiveInstance.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OracleActiveInstance.java b/src/java/org/apache/sqoop/manager/oracle/OracleActiveInstance.java
new file mode 100644
index 0000000..180da53
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OracleActiveInstance.java
@@ -0,0 +1,44 @@
+/**
+ * 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;
+
+/**
+ * Wraps data from v$active_instances.
+ */
+public class OracleActiveInstance {
+
+  private String instanceName;
+  private String hostName;
+
+  public String getInstanceName() {
+    return instanceName;
+  }
+
+  public void setInstanceName(String newInstanceName) {
+    this.instanceName = newInstanceName;
+  }
+
+  public String getHostName() {
+    return hostName;
+  }
+
+  public void setHostName(String newHostName) {
+    this.hostName = newHostName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OracleConnectionFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OracleConnectionFactory.java b/src/java/org/apache/sqoop/manager/oracle/OracleConnectionFactory.java
new file mode 100644
index 0000000..094576b
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OracleConnectionFactory.java
@@ -0,0 +1,217 @@
+/**
+ * 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.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Create and initialize connections to Oracle RDBMS.
+ */
+public class OracleConnectionFactory {
+
+  protected OracleConnectionFactory() {
+  }
+
+  protected static final OraOopLog LOG = OraOopLogFactory
+      .getLog(OracleConnectionFactory.class.getName());
+
+  public static Connection createOracleJdbcConnection(
+      String jdbcDriverClassName, String jdbcUrl, String username,
+      String password) throws SQLException {
+    Properties props = null;
+    return createOracleJdbcConnection(jdbcDriverClassName, jdbcUrl, username,
+        password, props);
+  }
+
+  public static Connection createOracleJdbcConnection(
+      String jdbcDriverClassName, String jdbcUrl, String username,
+      String password, Properties additionalProps) throws SQLException {
+
+    loadJdbcDriver(jdbcDriverClassName);
+    Connection connection =
+        createConnection(jdbcUrl, username, password, additionalProps);
+
+    // Only OraOopDBRecordReader will call initializeOracleConnection(), as
+    // we only need to initialize the session(s) prior to the mapper starting
+    // it's job.
+    // i.e. We don't need to initialize the sessions in order to get the
+    // table's data-files etc.
+
+    // initializeOracleConnection(connection, conf);
+
+    return connection;
+  }
+
+  private static void loadJdbcDriver(String jdbcDriverClassName) {
+
+    try {
+      Class.forName(jdbcDriverClassName);
+    } catch (ClassNotFoundException ex) {
+      String errorMsg =
+          "Unable to load the jdbc driver class : " + jdbcDriverClassName;
+      LOG.error(errorMsg);
+      throw new RuntimeException(errorMsg);
+    }
+  }
+
+  private static Connection createConnection(String jdbcUrl, String username,
+      String password, Properties additionalProps) throws SQLException {
+
+    Properties props = new Properties();
+    if (username != null) {
+      props.put("user", username);
+    }
+
+    if (password != null) {
+      props.put("password", password);
+    }
+
+    if (additionalProps != null && additionalProps.size() > 0) {
+      props.putAll(additionalProps);
+    }
+
+    try {
+      return DriverManager.getConnection(jdbcUrl, props);
+    } catch (SQLException ex) {
+      String errorMsg = String.format(
+        "Unable to obtain a JDBC connection to the URL \"%s\" as user \"%s\": ",
+        jdbcUrl, (username != null) ? username : "[null]");
+      LOG.error(errorMsg, ex);
+      throw ex;
+    }
+  }
+
+  public static void initializeOracleConnection(Connection connection,
+      org.apache.hadoop.conf.Configuration conf) throws SQLException {
+
+    connection.setTransactionIsolation(Connection.TRANSACTION_READ_COMMITTED);
+
+    OraOopOracleQueries.setConnectionTimeZone(connection, conf);
+
+    setSessionClientInfo(connection, conf);
+
+    OraOopOracleQueries.setJdbcFetchSize(connection, conf);
+
+    executeOraOopSessionInitializationStatements(connection, conf);
+  }
+
+  protected static void setSessionClientInfo(Connection connection,
+      org.apache.hadoop.conf.Configuration conf) {
+
+    String sql = "";
+    try {
+      sql =
+          "begin \n"
+              + "  dbms_application_info.set_module(module_name => "
+              + "'%s', action_name => '%s'); \n"
+              + "end;";
+
+      String oracleSessionActionName =
+          conf.get(OraOopConstants.ORACLE_SESSION_ACTION_NAME);
+
+      sql =
+          String.format(sql, OraOopConstants.ORACLE_SESSION_MODULE_NAME,
+              oracleSessionActionName);
+
+      Statement statement = connection.createStatement();
+      statement.execute(sql);
+      LOG.info("Initializing Oracle session with SQL :\n" + sql);
+    } catch (Exception ex) {
+      LOG.error(String.format("An error occurred while attempting to execute "
+          + "the following Oracle session-initialization statement:" + "\n%s"
+          + "\nError:" + "\n%s", sql, ex.getMessage()));
+    }
+  }
+
+  protected static void executeOraOopSessionInitializationStatements(
+      Connection connection, org.apache.hadoop.conf.Configuration conf) {
+
+    List<String> statements = parseOraOopSessionInitializationStatements(conf);
+
+    if (statements.size() == 0) {
+      LOG.warn(String
+          .format(
+              "No Oracle 'session initialization' statements were found to "
+              + "execute.\nCheck that your %s and/or %s files are correctly "
+                  + "installed in the ${SQOOP_HOME}/conf directory.",
+              OraOopConstants.ORAOOP_SITE_TEMPLATE_FILENAME,
+              OraOopConstants.ORAOOP_SITE_FILENAME));
+    } else {
+      for (String statement : statements) {
+        try {
+          connection.createStatement().execute(statement);
+          LOG.info("Initializing Oracle session with SQL : " + statement);
+        } catch (Exception ex) {
+          LOG.error(String.format(
+              "An error occurred while attempting to execute "
+                  + "the following Oracle session-initialization statement:"
+                  + "\n%s" + "\nError:" + "\n%s", statement, ex.getMessage()));
+        }
+      }
+    }
+  }
+
+  protected static List<String> parseOraOopSessionInitializationStatements(
+      org.apache.hadoop.conf.Configuration conf) {
+
+    ArrayList<String> result = new ArrayList<String>();
+
+    if (conf == null) {
+      throw new IllegalArgumentException(
+          "No configuration argument must be specified.");
+    }
+
+    String sessionInitializationStatements =
+        conf.get(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS);
+    if (sessionInitializationStatements != null
+        && !sessionInitializationStatements.isEmpty()) {
+      String[] initializationStatements =
+          sessionInitializationStatements.split(";");
+      for (String initializationStatement : initializationStatements) {
+        initializationStatement = initializationStatement.trim();
+        if (initializationStatement != null
+            && !initializationStatement.isEmpty()
+            && !initializationStatement
+                .startsWith(OraOopConstants.Oracle.
+                    ORACLE_SQL_STATEMENT_COMMENT_TOKEN)) {
+
+          LOG.debug(String
+              .format(
+                  "initializationStatement (quoted & pre-expression "
+                  + "evaluation) = \"%s\"",
+                  initializationStatement));
+
+          initializationStatement =
+              OraOopUtilities.replaceConfigurationExpression(
+                  initializationStatement, conf);
+
+          result.add(initializationStatement);
+        }
+      }
+    }
+    return result;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OracleTable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OracleTable.java b/src/java/org/apache/sqoop/manager/oracle/OracleTable.java
new file mode 100644
index 0000000..a0ecce1
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OracleTable.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.sqoop.manager.oracle;
+
+/**
+ * Contains details about an Oracle table.
+ */
+public class OracleTable {
+
+  private String schema;
+  private String name;
+
+  public String getSchema() {
+    return schema;
+  }
+
+  private void setSchema(String newSchema) {
+    this.schema = newSchema;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  private void setName(String newName) {
+    this.name = newName;
+  }
+
+  public OracleTable() {
+
+  }
+
+  public OracleTable(String schema, String name) {
+
+    setSchema(schema);
+    setName(name);
+  }
+
+  public OracleTable(String name) {
+    setName(name);
+  }
+
+  @Override
+  public String toString() {
+    String result =
+        (getSchema() == null || getSchema().isEmpty()) ? "" : "\""
+            + getSchema() + "\".";
+    result += "\"" + getName() + "\"";
+    return result;
+  }
+
+}


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

Posted by ve...@apache.org.
http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopDataDrivenDBInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopDataDrivenDBInputFormat.java b/src/java/org/apache/sqoop/manager/oracle/OraOopDataDrivenDBInputFormat.java
new file mode 100644
index 0000000..13f05d5
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopDataDrivenDBInputFormat.java
@@ -0,0 +1,359 @@
+/**
+ * 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.Connection;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+
+import com.cloudera.sqoop.lib.SqoopRecord;
+import com.cloudera.sqoop.mapreduce.db.DBConfiguration;
+import com.cloudera.sqoop.mapreduce.db.DataDrivenDBInputFormat;
+
+/**
+ * Reads data from Oracle table - data is divided between mappers based on ROWID
+ * split.
+ *
+ * @param <T> Output type of the record reader
+ */
+public class OraOopDataDrivenDBInputFormat<T extends SqoopRecord> extends
+    DataDrivenDBInputFormat<T> implements Configurable {
+
+  public static final OraOopLog LOG = OraOopLogFactory
+      .getLog(OraOopDataDrivenDBInputFormat.class.getName());
+
+  public OraOopDataDrivenDBInputFormat() {
+    super();
+    OraOopUtilities.checkJavaSecurityEgd();
+  }
+
+  @Override
+  public List<InputSplit> getSplits(JobContext jobContext) throws IOException {
+
+    int desiredNumberOfMappers = getDesiredNumberOfMappers(jobContext);
+
+    // Resolve the Oracle owner and name of the table we're importing...
+    OracleTable table = identifyOracleTableFromJobContext(jobContext);
+    List<String> partitionList = getPartitionList(jobContext);
+
+    // Get our Oracle connection...
+    Connection connection = getConnection();
+
+    List<InputSplit> splits = null;
+    try {
+      OracleConnectionFactory.initializeOracleConnection(connection, getConf());
+
+      // The number of chunks generated will *not* be a multiple of the number
+      // of splits,
+      // to ensure that each split doesn't always get data from the start of
+      // each data-file...
+      int numberOfChunksPerOracleDataFile = (desiredNumberOfMappers * 2) + 1;
+
+      // Get the Oracle data-chunks for the table...
+      List<? extends OraOopOracleDataChunk> dataChunks;
+      if (OraOopUtilities.getOraOopOracleDataChunkMethod(getConf()).equals(
+          OraOopConstants.OraOopOracleDataChunkMethod.PARTITION)) {
+        dataChunks =
+            OraOopOracleQueries.getOracleDataChunksPartition(connection, table,
+                partitionList);
+      } else {
+        dataChunks =
+            OraOopOracleQueries.getOracleDataChunksExtent(jobContext
+                .getConfiguration(), connection, table, partitionList,
+                numberOfChunksPerOracleDataFile);
+      }
+
+      if (dataChunks.size() == 0) {
+        String errMsg;
+        if (OraOopUtilities.getOraOopOracleDataChunkMethod(getConf()).equals(
+            OraOopConstants.OraOopOracleDataChunkMethod.PARTITION)) {
+          errMsg =
+              String
+                  .format(
+                      "The table %s does not contain any partitions and you "
+                      + "have specified to chunk the table by partitions.",
+                      table.getName());
+        } else {
+          errMsg =
+              String.format("The table %s does not contain any data.", table
+                  .getName());
+        }
+        LOG.fatal(errMsg);
+        throw new RuntimeException(errMsg);
+      } else {
+        OraOopConstants.OraOopOracleBlockToSplitAllocationMethod
+            blockAllocationMethod = OraOopUtilities
+                .getOraOopOracleBlockToSplitAllocationMethod(
+                    jobContext.getConfiguration(),
+                    OraOopConstants.
+                        OraOopOracleBlockToSplitAllocationMethod.ROUNDROBIN);
+
+        // Group the Oracle data-chunks into splits...
+        splits =
+            groupTableDataChunksIntoSplits(dataChunks, desiredNumberOfMappers,
+                blockAllocationMethod);
+
+        String oraoopLocations =
+            jobContext.getConfiguration().get("oraoop.locations", "");
+        String[] locations = oraoopLocations.split(",");
+        for (int idx = 0; idx < locations.length; idx++) {
+          if (idx < splits.size()) {
+            String location = locations[idx].trim();
+            if (!location.isEmpty()) {
+              ((OraOopDBInputSplit) splits.get(idx)).setSplitLocation(location);
+
+              LOG.info(String
+                  .format("Split[%d] has been assigned location \"%s\".", idx,
+                      location));
+            }
+          }
+        }
+
+      }
+    } catch (SQLException ex) {
+      throw new IOException(ex);
+    }
+
+    return splits;
+  }
+
+  @Override
+  protected RecordReader<LongWritable, T> createDBRecordReader(
+      DBInputSplit split, Configuration conf) throws IOException {
+
+    // This code is now running on a Datanode in the Hadoop cluster, so we need
+    // to enable debug logging in this JVM...
+    OraOopUtilities.enableDebugLoggingIfRequired(conf);
+
+    // Retrieve the JDBC URL that should be used by this mapper.
+    // We achieve this by modifying the JDBC URL property in the configuration,
+    // prior to the
+    // OraOopDBRecordReader (or its ancestors) using the configuration to
+    // establish a connection
+    // to the database - via DBConfiguration.getConnection()...
+    OraOopDBInputSplit oraOopSplit = OraOopDBRecordReader.castSplit(split);
+    int mapperId = oraOopSplit.getSplitId();
+    String mapperJdbcUrlPropertyName =
+        OraOopUtilities.getMapperJdbcUrlPropertyName(mapperId, conf);
+
+    // Get this mapper's JDBC URL
+    String mapperJdbcUrl = conf.get(mapperJdbcUrlPropertyName, null);
+    LOG.debug(String.format("Mapper %d has a JDBC URL of: %s", mapperId,
+        mapperJdbcUrl == null ? "<null>" : mapperJdbcUrl));
+
+    DBConfiguration dbConf = getDBConf();
+
+    if (mapperJdbcUrl != null) {
+      // Just changing the URL_PROPERTY in the conf object does not work - as
+      // dbConf.getConf()
+      // seems to refer to a separate instance of the configuration properties.
+      // Therefore, we
+      // need to update the URL_PROPERTY in dbConf so that we connect to the
+      // appropriate instance
+      // in the Oracle RAC. To help avoid confusion, we'll also update the
+      // URL_PROPERTY in the
+      // conf object to match...
+      dbConf.getConf().set(DBConfiguration.URL_PROPERTY, mapperJdbcUrl);
+      conf.set(DBConfiguration.URL_PROPERTY, mapperJdbcUrl);
+    }
+
+    @SuppressWarnings("unchecked")
+    Class<T> inputClass = (Class<T>) (dbConf.getInputClass());
+
+    try {
+      // Use Oracle-specific db reader
+
+      // this.getConnection() will return the connection created when the
+      // DBInputFormat ancestor
+      // was created. This connection will be based on the URL_PROPERTY that was
+      // current at that
+      // time. We've just changed the URL_PROPERTY (if this is an Oracle RAC)
+      // and therefore need
+      // to use dbConf.getConnection() so that a new connection is created using
+      // the current
+      // value of the URL_PROPERTY...
+
+      return new OraOopDBRecordReader<T>(split, inputClass, conf, dbConf
+          .getConnection(), dbConf, dbConf.getInputConditions(), dbConf
+          .getInputFieldNames(), dbConf.getInputTableName());
+    } catch (SQLException ex) {
+      throw new IOException(ex);
+    } catch (ClassNotFoundException ex) {
+      throw new IOException(ex);
+    }
+  }
+
+  private OracleTable identifyOracleTableFromJobContext(JobContext jobContext) {
+
+    OracleTable result = new OracleTable();
+
+    String dbUserName =
+        jobContext.getConfiguration().get(DBConfiguration.USERNAME_PROPERTY);
+    String tableName = getDBConf().getInputTableName();
+
+    result =
+        OraOopUtilities.decodeOracleTableName(dbUserName, tableName, jobContext
+            .getConfiguration());
+
+    return result;
+  }
+
+  private int getDesiredNumberOfMappers(JobContext jobContext) {
+
+    int desiredNumberOfMappers =
+        jobContext.getConfiguration().getInt(
+            OraOopConstants.ORAOOP_DESIRED_NUMBER_OF_MAPPERS, -1);
+
+    int minMappersAcceptedByOraOop =
+        OraOopUtilities.getMinNumberOfImportMappersAcceptedByOraOop(jobContext
+            .getConfiguration());
+
+    if (desiredNumberOfMappers < minMappersAcceptedByOraOop) {
+      LOG.warn(String.format("%s should not be used to perform a sqoop import "
+          + "when the number of mappers is %d\n "
+          + "i.e. OraOopManagerFactory.accept() should only appect jobs "
+          + "where the number of mappers is at least %d",
+          OraOopConstants.ORAOOP_PRODUCT_NAME, desiredNumberOfMappers,
+          minMappersAcceptedByOraOop));
+    }
+
+    return desiredNumberOfMappers;
+  }
+
+  private List<String> getPartitionList(JobContext jobContext) {
+    LOG.debug(OraOopConstants.ORAOOP_IMPORT_PARTITION_LIST
+        + " = "
+        + jobContext.getConfiguration().get(
+            OraOopConstants.ORAOOP_IMPORT_PARTITION_LIST));
+    List<String> result =
+        OraOopUtilities.splitOracleStringList(jobContext.getConfiguration()
+            .get(OraOopConstants.ORAOOP_IMPORT_PARTITION_LIST));
+    if (result != null && result.size() > 0) {
+      LOG.debug("Partition filter list: " + result.toString());
+    }
+    return result;
+  }
+
+  protected
+      List<InputSplit>
+      groupTableDataChunksIntoSplits(
+          List<? extends OraOopOracleDataChunk> dataChunks,
+          int desiredNumberOfSplits,
+          OraOopConstants.OraOopOracleBlockToSplitAllocationMethod
+              blockAllocationMethod) {
+
+    int numberOfDataChunks = dataChunks.size();
+    int actualNumberOfSplits =
+        Math.min(numberOfDataChunks, desiredNumberOfSplits);
+    int totalNumberOfBlocksInAllDataChunks = 0;
+    for (OraOopOracleDataChunk dataChunk : dataChunks) {
+      totalNumberOfBlocksInAllDataChunks += dataChunk.getNumberOfBlocks();
+    }
+
+    String debugMsg = String.format(
+        "The table being imported by sqoop has %d blocks "
+      + "that have been divided into %d chunks "
+      + "which will be processed in %d splits. "
+      + "The chunks will be allocated to the splits using the method : %s",
+        totalNumberOfBlocksInAllDataChunks, numberOfDataChunks,
+        actualNumberOfSplits, blockAllocationMethod.toString());
+    LOG.info(debugMsg);
+
+    List<InputSplit> splits = new ArrayList<InputSplit>(actualNumberOfSplits);
+
+    for (int i = 0; i < actualNumberOfSplits; i++) {
+      OraOopDBInputSplit split = new OraOopDBInputSplit();
+      split.setSplitId(i);
+      split.setTotalNumberOfBlocksInAllSplits(
+          totalNumberOfBlocksInAllDataChunks);
+      splits.add(split);
+    }
+
+    switch (blockAllocationMethod) {
+
+      case RANDOM:
+        // Randomize the order of the data chunks and then "fall through" into
+        // the ROUNDROBIN block below...
+        Collections.shuffle(dataChunks);
+
+        // NB: No "break;" statement here - we're intentionally falling into the
+        // ROUNDROBIN block below...
+
+      //$FALL-THROUGH$
+      case ROUNDROBIN:
+        int idxSplitRoundRobin = 0;
+        for (OraOopOracleDataChunk dataChunk : dataChunks) {
+
+          if (idxSplitRoundRobin >= splits.size()) {
+            idxSplitRoundRobin = 0;
+          }
+          OraOopDBInputSplit split =
+              (OraOopDBInputSplit) splits.get(idxSplitRoundRobin++);
+
+          split.getDataChunks().add(dataChunk);
+        }
+        break;
+
+      case SEQUENTIAL:
+        double dataChunksPerSplit = dataChunks.size() / (double) splits.size();
+        int dataChunksAllocatedToSplits = 0;
+
+        int idxSplitSeq = 0;
+        for (OraOopOracleDataChunk dataChunk : dataChunks) {
+
+          OraOopDBInputSplit split
+              = (OraOopDBInputSplit) splits.get(idxSplitSeq);
+          split.getDataChunks().add(dataChunk);
+
+          dataChunksAllocatedToSplits++;
+
+          if (dataChunksAllocatedToSplits
+                  >= (dataChunksPerSplit * (idxSplitSeq + 1))
+              && idxSplitSeq < splits.size()) {
+            idxSplitSeq++;
+          }
+        }
+        break;
+
+      default:
+        throw new RuntimeException("Block allocation method not implemented.");
+
+    }
+
+    if (LOG.isDebugEnabled()) {
+      for (int idx = 0; idx < splits.size(); idx++) {
+        LOG.debug("\n\t"
+            + ((OraOopDBInputSplit) splits.get(idx)).getDebugDetails());
+      }
+    }
+
+    return splits;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopGenerics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopGenerics.java b/src/java/org/apache/sqoop/manager/oracle/OraOopGenerics.java
new file mode 100644
index 0000000..7307fcb
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopGenerics.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;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Generic class to hold list of objects.
+ */
+public class OraOopGenerics {
+
+  /**
+   * Generic class to hold list of objects.
+   */
+  public static class ObjectList<T> {
+
+    private List<T> objects;
+
+    public ObjectList() {
+
+      this.objects = new ArrayList<T>();
+    }
+
+    public void add(T item) {
+
+      this.objects.add(item);
+    }
+
+    public int size() {
+
+      return this.objects.size();
+    }
+
+    public T get(int index) {
+
+      return this.objects.get(index);
+    }
+
+    public Iterator<T> iterator() {
+
+      return this.objects.iterator();
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopJdbcUrl.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopJdbcUrl.java b/src/java/org/apache/sqoop/manager/oracle/OraOopJdbcUrl.java
new file mode 100644
index 0000000..8502b79
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopJdbcUrl.java
@@ -0,0 +1,232 @@
+/**
+ * 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 org.apache.sqoop.manager.oracle.OraOopUtilities.JdbcOracleThinConnection;
+import org.apache.sqoop.manager.oracle.OraOopUtilities
+         .JdbcOracleThinConnectionParsingError;
+
+/**
+ * Parses the Oracle connection string.
+ */
+public class OraOopJdbcUrl {
+
+  private String jdbcConnectString;
+
+  public OraOopJdbcUrl(String jdbcConnectString) {
+
+    if (jdbcConnectString == null) {
+      throw new IllegalArgumentException(
+          "The jdbcConnectionString argument must not be null.");
+    }
+
+    if (jdbcConnectString.isEmpty()) {
+      throw new IllegalArgumentException(
+          "The jdbcConnectionString argument must not be empty.");
+    }
+
+    this.jdbcConnectString = jdbcConnectString;
+  }
+
+  public JdbcOracleThinConnection parseJdbcOracleThinConnectionString()
+      throws JdbcOracleThinConnectionParsingError {
+
+    /*
+     * http://wiki.oracle.com/page/JDBC
+     *
+     * There are different flavours of JDBC connections for Oracle, including:
+     * Thin E.g. jdbc:oracle:thin:@localhost.locadomain:1521:orcl
+     *
+     * A pure Java driver used on the client side that does not need an Oracle
+     * client installation. It is recommended that you use this driver unless
+     * you need support for non-TCP/IP networks because it provides for maximum
+     * portability and performance.
+     *
+     * Oracle Call Interface driver (OCI). E.g. jdbc:oracle:oci8:@orcl.world
+     * //<- "orcl.world" is a TNS entry
+     *
+     * This uses the Oracle client installation libraries and interfaces. If you
+     * want to support connection pooling or client side caching of requests,
+     * use this driver. You will also need this driver if you are using
+     * transparent application failover (TAF) from your application as well as
+     * strong authentication like Kerberos and PKI certificates.
+     *
+     * JDBC-ODBC bridge. E.g. jdbc:odbc:mydatabase //<- "mydatabase" is an ODBC
+     * data source.
+     *
+     * This uses the ODBC driver in Windows to connect to the database.
+     */
+
+    String hostName = null;
+    int port = 0;
+    String sid = null;
+    String service = null;
+
+    String jdbcUrl = this.jdbcConnectString.trim();
+
+    // If there are any parameters included at the end of the connection URL,
+    // let's remove them now...
+    int paramsIdx = jdbcUrl.indexOf("?");
+    if (paramsIdx > -1) {
+      jdbcUrl = jdbcUrl.substring(0, paramsIdx);
+    }
+
+    /*
+     * The format of an Oracle jdbc URL is one of:
+     * jdbc:oracle:<driver-type>:@<host>:<port>:<sid>
+     * jdbc:oracle:<driver-type>:@<host>:<port>/<service>
+     * jdbc:oracle:<driver-type>:@<host>:<port>/<service>?<parameters>
+     * jdbc:oracle:<driver-type>:@//<host>:<port>/<service>
+     * jdbc:oracle:<driver-type>:@//<host>:<port>/<service>?<parameters>
+     */
+
+    // Split the URL on its ":" characters...
+    String[] jdbcFragments = jdbcUrl.trim().split(":");
+
+    // Clean up each fragment of the URL...
+    for (int idx = 0; idx < jdbcFragments.length; idx++) {
+      jdbcFragments[idx] = jdbcFragments[idx].trim();
+    }
+
+    // Check we can proceed...
+    if (jdbcFragments.length < 5 || jdbcFragments.length > 6) {
+      throw new JdbcOracleThinConnectionParsingError(
+        String.format(
+          "There should be 5 or 6 colon-separated pieces of data in the JDBC "
+        + "URL, such as:\n\tjdbc:oracle:<driver-type>:@<host>:<port>:<sid>\n"
+        + "\tjdbc:oracle:<driver-type>:@<host>:<port>/<service>\n"
+        + "\tjdbc:oracle:<driver-type>:@<host>:<port>/<service>?<parameters>\n"
+        + "The JDBC URL specified was:\n"
+        + "%s\n"
+        + "which contains %d pieces of colon-separated data.",
+                  this.jdbcConnectString, jdbcFragments.length));
+    }
+
+    // jdbc
+    if (!jdbcFragments[0].equalsIgnoreCase("jdbc")) {
+      throw new JdbcOracleThinConnectionParsingError(
+          "The first item in the colon-separated JDBC URL must be \"jdbc\".");
+    }
+
+    // jdbc:oracle
+    if (!jdbcFragments[1].equalsIgnoreCase("oracle")) {
+      throw new JdbcOracleThinConnectionParsingError(
+        "The second item in the colon-separated JDBC URL must be \"oracle\".");
+    }
+
+    // jdbc:oracle:thin
+    if (!jdbcFragments[2].equalsIgnoreCase("thin")) {
+      throw new JdbcOracleThinConnectionParsingError(
+          String
+              .format(
+                  "The Oracle \"thin\" JDBC driver is not being used.\n"
+                      + "The third item in the colon-separated JDBC URL must "
+                      + "be \"thin\", not \"%s\".",
+                  jdbcFragments[2]));
+    }
+
+    // jdbc:oracle:thin:@<host>
+    hostName = jdbcFragments[3];
+    if (hostName.isEmpty() || hostName.equalsIgnoreCase("@")) {
+      throw new JdbcOracleThinConnectionParsingError(
+          "The fourth item in the colon-separated JDBC URL (the host name) "
+          + "must not be empty.");
+    }
+
+    if (!hostName.startsWith("@")) {
+      throw new JdbcOracleThinConnectionParsingError(
+          "The fourth item in the colon-separated JDBC URL (the host name) "
+          + "must a prefixed with the \"@\" character.");
+    }
+
+    String portStr = "";
+    switch (jdbcFragments.length) {
+      case 6:
+        // jdbc:oracle:<driver-type>:@<host>:<port>:<sid>
+        portStr = jdbcFragments[4];
+        sid = jdbcFragments[5];
+        break;
+
+      case 5:
+        // jdbc:oracle:<driver-type>:@<host>:<port>/<service>
+        String[] portAndService = jdbcFragments[4].split("/");
+        if (portAndService.length != 2) {
+          throw new JdbcOracleThinConnectionParsingError(
+              "The fifth colon-separated item in the JDBC URL "
+              + "(<port>/<service>) must contain two items "
+              + "separated by a \"/\".");
+        }
+        portStr = portAndService[0].trim();
+        service = portAndService[1].trim();
+        break;
+
+      default:
+        throw new JdbcOracleThinConnectionParsingError("Internal error parsing "
+            + "JDBC connection string.");
+    }
+
+    if (portStr.isEmpty()) {
+      throw new JdbcOracleThinConnectionParsingError(
+          "The fifth item in the colon-separated JDBC URL (the port) must not "
+          + "be empty.");
+    }
+
+    try {
+      port = Integer.parseInt(portStr);
+    } catch (NumberFormatException ex) {
+      throw new JdbcOracleThinConnectionParsingError(
+          String
+              .format(
+                  "The fifth item in the colon-separated JDBC URL (the port) "
+                  + "must be a valid number.\n"
+                      + "\"%s\" could not be parsed as an integer.", portStr));
+    }
+
+    if (port <= 0) {
+      throw new JdbcOracleThinConnectionParsingError(
+          String
+              .format(
+                  "The fifth item in the colon-separated JDBC URL (the port) "
+                  + "must be greater than zero.\n"
+                      + "\"%s\" was specified.", portStr));
+    }
+
+    if (sid == null && service == null) {
+      throw new JdbcOracleThinConnectionParsingError(
+        "The JDBC URL does not contain a SID or SERVICE. The URL should look "
+      + "like one of these:\n\tjdbc:oracle:<driver-type>:@<host>:<port>:<sid>\n"
+      + "\tjdbc:oracle:<driver-type>:@<host>:<port>/<service>\n"
+      + "\tjdbc:oracle:<driver-type>:@<host>:<port>/<service>?<parameters>\n"
+      + "\tjdbc:oracle:<driver-type>:@//<host>:<port>/<service>\n"
+      + "\tjdbc:oracle:<driver-type>:@<host>:<port>/<service>?<parameters>");
+    }
+
+    // Remove the "@" prefix of the hostname
+    JdbcOracleThinConnection result =
+        new JdbcOracleThinConnection(hostName.replaceFirst("^[@][/]{0,2}", "")
+            , port, sid, service);
+
+    return result;
+  }
+
+  public String getConnectionUrl() {
+    return this.jdbcConnectString;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopLog.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopLog.java b/src/java/org/apache/sqoop/manager/oracle/OraOopLog.java
new file mode 100644
index 0000000..5d5ee02
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopLog.java
@@ -0,0 +1,235 @@
+/**
+ * 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;
+
+/**
+ * Class to wrap commons logging to support caching entries.
+ */
+public class OraOopLog implements org.apache.commons.logging.Log,
+    OraOopLogFactory.OraOopLog2 {
+
+  private org.apache.commons.logging.Log log;
+  private StringBuilder cache;
+
+  public OraOopLog(org.apache.commons.logging.Log otherLog) {
+
+    this.log = otherLog;
+  }
+
+  @Override
+  public void debug(Object message) {
+
+    if (cacheLogEntry(message)) {
+      return;
+    }
+
+    log.debug(message);
+
+  }
+
+  @Override
+  public void debug(Object message, Throwable t) {
+
+    if (cacheLogEntry(message)) {
+      return;
+    }
+
+    log.debug(message, t);
+  }
+
+  @Override
+  public void error(Object message) {
+
+    if (cacheLogEntry(message)) {
+      return;
+    }
+
+    log.error(message);
+  }
+
+  @Override
+  public void error(Object message, Throwable t) {
+
+    if (cacheLogEntry(message)) {
+      return;
+    }
+
+    log.error(message, t);
+  }
+
+  @Override
+  public void fatal(Object message) {
+
+    if (cacheLogEntry(message)) {
+      return;
+    }
+
+    log.fatal(message);
+  }
+
+  @Override
+  public void fatal(Object message, Throwable t) {
+
+    if (cacheLogEntry(message)) {
+      return;
+    }
+
+    log.fatal(message, t);
+  }
+
+  @Override
+  public void info(Object message) {
+
+    if (cacheLogEntry(message)) {
+      return;
+    }
+
+    log.info(message);
+  }
+
+  @Override
+  public void info(Object message, Throwable t) {
+
+    if (cacheLogEntry(message)) {
+      return;
+    }
+
+    log.info(message, t);
+  }
+
+  @Override
+  public boolean isDebugEnabled() {
+
+    return log.isDebugEnabled();
+  }
+
+  @Override
+  public boolean isErrorEnabled() {
+
+    return log.isErrorEnabled();
+  }
+
+  @Override
+  public boolean isFatalEnabled() {
+
+    return log.isFatalEnabled();
+  }
+
+  @Override
+  public boolean isInfoEnabled() {
+
+    return log.isInfoEnabled();
+  }
+
+  @Override
+  public boolean isTraceEnabled() {
+
+    return log.isTraceEnabled();
+  }
+
+  @Override
+  public boolean isWarnEnabled() {
+
+    return log.isWarnEnabled();
+  }
+
+  @Override
+  public void trace(Object message) {
+
+    log.trace(message);
+    cacheLogEntry(message);
+  }
+
+  @Override
+  public void trace(Object message, Throwable t) {
+
+    if (cacheLogEntry(message)) {
+      return;
+    }
+
+    log.trace(message, t);
+  }
+
+  @Override
+  public void warn(Object message) {
+
+    if (cacheLogEntry(message)) {
+      return;
+    }
+
+    log.warn(message);
+  }
+
+  @Override
+  public void warn(Object message, Throwable t) {
+
+    if (cacheLogEntry(message)) {
+      return;
+    }
+
+    log.warn(message, t);
+  }
+
+  @Override
+  public boolean getCacheLogEntries() {
+
+    return (this.cache != null);
+  }
+
+  @Override
+  public String getLogEntries() {
+
+    if (this.cache != null) {
+      return this.cache.toString();
+    } else {
+      return "";
+    }
+  }
+
+  @Override
+  public void setCacheLogEntries(boolean value) {
+
+    if (getCacheLogEntries() && !value) {
+      this.cache = null;
+    } else if (!getCacheLogEntries() && value) {
+      this.cache = new StringBuilder();
+    }
+  }
+
+  @Override
+  public void clearCache() {
+
+    if (getCacheLogEntries()) {
+      this.cache = new StringBuilder();
+    }
+  }
+
+  private boolean cacheLogEntry(Object message) {
+
+    boolean result = getCacheLogEntries();
+
+    if (result && message != null) {
+      this.cache.append(message.toString());
+    }
+
+    return result;
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopLogFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopLogFactory.java b/src/java/org/apache/sqoop/manager/oracle/OraOopLogFactory.java
new file mode 100644
index 0000000..d278225
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopLogFactory.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;
+
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Wraps commons logging.
+ */
+public final class OraOopLogFactory {
+  private OraOopLogFactory() {
+  }
+
+  /**
+   * Interface for log entries including caching for test purposes.
+   */
+  public interface OraOopLog2 {
+
+    boolean getCacheLogEntries();
+
+    void setCacheLogEntries(boolean value);
+
+    String getLogEntries();
+
+    void clearCache();
+  }
+
+  public static OraOopLog getLog(Class<?> clazz) {
+
+    return OraOopLogFactory.getLog(clazz.getName());
+  }
+
+  public static OraOopLog getLog(String logName) {
+
+    return new OraOopLog(LogFactory.getLog(logName));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopLogMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopLogMessage.java b/src/java/org/apache/sqoop/manager/oracle/OraOopLogMessage.java
new file mode 100644
index 0000000..2acba55
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopLogMessage.java
@@ -0,0 +1,61 @@
+/**
+ * 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;
+
+/**
+ * Represents an individual log message.
+ */
+public class OraOopLogMessage {
+  private OraOopConstants.Logging.Level level;
+  private String message;
+
+  public OraOopLogMessage(OraOopConstants.Logging.Level level, String message) {
+
+    this.level = level;
+    this.message = message;
+  }
+
+  public void log(OraOopLog log) {
+
+    switch (this.level) {
+      case TRACE:
+        log.trace(this.message);
+        break;
+      case DEBUG:
+        log.debug(this.message);
+        break;
+      case INFO:
+        log.info(this.message);
+        break;
+      case WARN:
+        log.warn(this.message);
+        break;
+      case ERROR:
+        log.error(this.message);
+        break;
+      case FATAL:
+        log.fatal(this.message);
+        break;
+      default:
+        throw new RuntimeException(String.format(
+            "%s needs to be updated to cater for the message level \"%s\".",
+            OraOopUtilities.getCurrentMethodName(), this.level.toString()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopManagerFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopManagerFactory.java b/src/java/org/apache/sqoop/manager/oracle/OraOopManagerFactory.java
new file mode 100644
index 0000000..fcf4a4d
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopManagerFactory.java
@@ -0,0 +1,1126 @@
+/**
+ * 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.sql.Connection;
+import java.sql.SQLException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.cloudera.sqoop.SqoopOptions;
+import com.cloudera.sqoop.SqoopOptions.IncrementalMode;
+import com.cloudera.sqoop.manager.ConnManager;
+import com.cloudera.sqoop.manager.ManagerFactory;
+import com.cloudera.sqoop.metastore.JobData;
+import org.apache.sqoop.manager.oracle.OraOopOutputFormatUpdate.UpdateMode;
+import org.apache.sqoop.manager.oracle.OraOopUtilities.
+           JdbcOracleThinConnectionParsingError;
+
+/**
+ * OraOop manager - if OraOop cannot be used it should fall back to the default
+ * OracleManager.
+ *
+ * To increase the amount of heap memory available to the mappers:
+ *                                     -Dmapred.child.java.opts=-Xmx4000M
+ * To prevent failed mapper tasks from being reattempted:
+ *                                     -Dmapred.map.max.attempts=1
+ */
+public class OraOopManagerFactory extends ManagerFactory {
+
+  @SuppressWarnings("unused")
+  private static final OraOopLog ORAOOP_LOG = OraOopLogFactory
+      .getLog("org.apache.sqoop.manager.oracle");
+  private static final OraOopLog LOG = OraOopLogFactory
+      .getLog(OraOopManagerFactory.class.getName());
+
+  static {
+    Configuration
+        .addDefaultResource(OraOopConstants.ORAOOP_SITE_TEMPLATE_FILENAME);
+    Configuration.addDefaultResource(OraOopConstants.ORAOOP_SITE_FILENAME);
+  }
+
+  @Override
+  public ConnManager accept(JobData jobData) {
+
+    OraOopUtilities.enableDebugLoggingIfRequired(jobData.getSqoopOptions()
+        .getConf());
+
+    LOG.debug(String.format("%s can be called by Sqoop!",
+        OraOopConstants.ORAOOP_PRODUCT_NAME));
+
+    ConnManager result = null;
+
+    if (jobData != null) {
+
+      SqoopOptions sqoopOptions = jobData.getSqoopOptions();
+
+      String connectString = sqoopOptions.getConnectString();
+      if (connectString != null
+          && connectString.toLowerCase().trim().startsWith("jdbc:oracle")) {
+
+        if (!isOraOopEnabled(sqoopOptions)) {
+          return result;
+        }
+
+        OraOopConnManager oraOopConnManager = null;
+
+        OraOopConstants.Sqoop.Tool jobType = getSqoopJobType(jobData);
+        OraOopUtilities.rememberSqoopJobType(jobType, jobData.getSqoopOptions()
+            .getConf());
+
+        List<OraOopLogMessage> messagesToDisplayAfterWelcome =
+            new ArrayList<OraOopLogMessage>();
+
+        switch (jobType) {
+
+          case IMPORT:
+            if (isNumberOfImportMappersOkay(sqoopOptions)
+                && !isSqoopImportIncremental(jobData)
+                && isSqoopImportJobTableBased(sqoopOptions)) {
+
+              // At this stage, the Sqoop import job appears to be one we're
+              // interested in accepting. We now need to connect to
+              // the Oracle database to perform more tests...
+
+              oraOopConnManager = new OraOopConnManager(sqoopOptions);
+
+              try {
+                Connection connection = oraOopConnManager.getConnection();
+
+                if (isSqoopTableAnOracleTable(connection, sqoopOptions
+                    .getUsername(),
+                    oraOopConnManager.getOracleTableContext())) {
+
+                  // OraOop will not accept responsibility for an Index
+                  // Organized Table (IOT)...
+                  if (!isSqoopTableAnIndexOrganizedTable(connection,
+                      oraOopConnManager.getOracleTableContext())) {
+                    result = oraOopConnManager; // <- OraOop accepts
+                                                // responsibility for this Sqoop
+                                                // job!
+                  }
+                }
+              } catch (SQLException ex) {
+                throw new RuntimeException(String.format(
+                    "Unable to connect to the Oracle database at %s\n"
+                        + "Error:%s", sqoopOptions.getConnectString(), ex
+                        .getMessage()), ex);
+              }
+            }
+            break;
+
+          case EXPORT:
+            if (isNumberOfExportMappersOkay(sqoopOptions)) {
+
+              // At this stage, the Sqoop export job appears to be one we're
+              // interested in accepting. We now need to connect to
+              // the Oracle database to perform more tests...
+
+              oraOopConnManager = new OraOopConnManager(sqoopOptions);
+
+              Connection connection = null;
+              try {
+                connection = oraOopConnManager.getConnection();
+              } catch (SQLException ex) {
+                throw new RuntimeException(String.format(
+                    "Unable to connect to the Oracle database at %s\n"
+                        + "Error:%s", sqoopOptions.getConnectString(), ex
+                        .getMessage()), ex);
+              }
+
+              try {
+
+                createAnyRequiredOracleObjects(sqoopOptions, connection,
+                    oraOopConnManager, messagesToDisplayAfterWelcome);
+
+                if (isSqoopTableAnOracleTable(connection, sqoopOptions
+                    .getUsername(),
+                    oraOopConnManager.getOracleTableContext())) {
+
+                  result = oraOopConnManager; // <- OraOop accepts
+                                              // responsibility for this Sqoop
+                                              // job!
+                }
+
+              } catch (SQLException ex) {
+                LOG.error(OraOopUtilities.getFullExceptionMessage(ex));
+              }
+            }
+
+            break;
+          default:
+            // OraOop doesn't know how to handle other types of jobs - so won't
+            // accept them.
+            break;
+        }
+
+        // If OraOop has accepted this Sqoop job...
+        if (result != null) {
+
+          showUserTheOraOopWelcomeMessage();
+
+          for (OraOopLogMessage message : messagesToDisplayAfterWelcome) {
+            message.log(LOG);
+          }
+
+          // By the time we get into getSplits(), the number of mappers
+          // stored in the config can be either 4 or 1 - so it seems
+          // a bit unreliable. We'll use our own property name to ensure
+          // getSplits() gets the correct value...
+          sqoopOptions.getConf().setInt(
+              OraOopConstants.ORAOOP_DESIRED_NUMBER_OF_MAPPERS,
+              sqoopOptions.getNumMappers());
+
+          // Generate the "action" name that we'll assign to our Oracle sessions
+          // so that the user knows which Oracle sessions belong to OraOop...
+          sqoopOptions.getConf().set(
+              OraOopConstants.ORACLE_SESSION_ACTION_NAME,
+              getOracleSessionActionName(jobData));
+
+          OraOopUtilities.appendJavaSecurityEgd(sqoopOptions.getConf());
+
+          // Get the Oracle database version...
+          try {
+            OracleVersion oracleVersion =
+                OraOopOracleQueries.getOracleVersion(result.getConnection());
+            LOG.info(String.format("Oracle Database version: %s",
+                oracleVersion.getBanner()));
+            sqoopOptions.getConf().setInt(
+                OraOopConstants.ORAOOP_ORACLE_DATABASE_VERSION_MAJOR,
+                oracleVersion.getMajor());
+            sqoopOptions.getConf().setInt(
+                OraOopConstants.ORAOOP_ORACLE_DATABASE_VERSION_MINOR,
+                oracleVersion.getMinor());
+          } catch (SQLException ex) {
+            LOG.error("Unable to obtain the Oracle database version.", ex);
+          }
+
+          try {
+            if (sqoopOptions.getConf().getBoolean(
+                OraOopConstants.ORAOOP_IMPORT_CONSISTENT_READ, false)) {
+              long scn =
+                  sqoopOptions.getConf().getLong(
+                      OraOopConstants.ORAOOP_IMPORT_CONSISTENT_READ_SCN, 0);
+              if (scn == 0) {
+                scn = OraOopOracleQueries.getCurrentScn(result.getConnection());
+              }
+              sqoopOptions.getConf().setLong(
+                  OraOopConstants.ORAOOP_IMPORT_CONSISTENT_READ_SCN, scn);
+              LOG.info("Performing a consistent read using SCN: " + scn);
+            }
+          } catch (SQLException ex) {
+            throw new RuntimeException("Unable to determine SCN of database.",
+                ex);
+          }
+
+          // Generate the JDBC URLs to be used by each mapper...
+          setMapperConnectionDetails(oraOopConnManager, jobData);
+
+          // Show the user the Oracle command that can be used to kill this
+          // OraOop
+          // job via Oracle...
+          showUserTheOracleCommandToKillOraOop(sqoopOptions);
+        }
+
+      }
+    }
+
+    return result;
+  }
+
+  private void setMapperConnectionDetails(OraOopConnManager oraOopConnManager,
+      JobData jobData) {
+
+    // Ensure we have a connection to the database...
+    Connection connection = null;
+    try {
+      connection = oraOopConnManager.getConnection();
+    } catch (SQLException ex) {
+      throw new RuntimeException(String.format(
+          "Unable to connect to the Oracle database at %s\n" + "Error:%s",
+          jobData.getSqoopOptions().getConnectString(), ex.getMessage()));
+    }
+
+    // Query v$active_instances to get a list of all instances in the Oracle RAC
+    // (assuming this *could* be a RAC)...
+    List<OracleActiveInstance> activeInstances = null;
+    try {
+      activeInstances =
+          OraOopOracleQueries.getOracleActiveInstances(connection);
+    } catch (SQLException ex) {
+      throw new RuntimeException(
+          "An error was encountered when attempting to determine the "
+        + "configuration of the Oracle RAC.",
+        ex);
+    }
+
+    if (activeInstances == null) {
+      LOG.info("This Oracle database is not a RAC.");
+    } else {
+      LOG.info("This Oracle database is a RAC.");
+    }
+
+    // Is dynamic JDBC URL generation disabled?...
+    if (OraOopUtilities.oracleJdbcUrlGenerationDisabled(jobData
+        .getSqoopOptions().getConf())) {
+      LOG.info(String
+          .format(
+              "%s will not use dynamically generated JDBC URLs - this feature "
+            + "has been disabled.",
+            OraOopConstants.ORAOOP_PRODUCT_NAME));
+      return;
+    }
+
+    boolean generateRacBasedJdbcUrls = false;
+
+    // Decide whether this is a multi-instance RAC, and whether we need to do
+    // anything more...
+    if (activeInstances != null) {
+      generateRacBasedJdbcUrls = true;
+
+      if (activeInstances.size() < OraOopUtilities
+          .getMinNumberOfOracleRacActiveInstancesForDynamicJdbcUrlUse(jobData
+              .getSqoopOptions().getConf())) {
+        LOG.info(String.format(
+            "There are only %d active instances in the Oracle RAC. "
+          + "%s will not bother utilizing dynamically generated JDBC URLs.",
+          activeInstances.size(), OraOopConstants.ORAOOP_PRODUCT_NAME));
+        generateRacBasedJdbcUrls = false;
+      }
+    }
+
+    // E.g. jdbc:oracle:thin:@localhost.localdomain:1521:orcl
+    String jdbcConnectStr = jobData.getSqoopOptions().getConnectString();
+
+    // Parse the JDBC URL to obtain the port number for the TNS listener...
+    String jdbcHost = "";
+    int jdbcPort = 0;
+    String jdbcSid = "";
+    String jdbcService = "";
+    try {
+
+      OraOopJdbcUrl oraOopJdbcUrl = new OraOopJdbcUrl(jdbcConnectStr);
+      OraOopUtilities.JdbcOracleThinConnection jdbcConnection =
+          oraOopJdbcUrl.parseJdbcOracleThinConnectionString();
+      jdbcHost = jdbcConnection.getHost();
+      jdbcPort = jdbcConnection.getPort();
+      jdbcSid = jdbcConnection.getSid();
+      jdbcService = jdbcConnection.getService();
+    } catch (JdbcOracleThinConnectionParsingError ex) {
+      LOG.info(String.format(
+          "Unable to parse the JDBC connection URL \"%s\" as a connection "
+        + "that uses the Oracle 'thin' JDBC driver.\n"
+        + "This problem prevents %s from being able to dynamically generate "
+        + "JDBC URLs that specify 'dedicated server connections' or spread "
+        + "mapper sessions across multiple Oracle instances.\n"
+        + "If the JDBC driver-type is 'OCI' (instead of 'thin'), then "
+        + "load-balancing should be appropriately managed automatically.",
+              jdbcConnectStr, OraOopConstants.ORAOOP_PRODUCT_NAME, ex));
+      return;
+    }
+
+    if (generateRacBasedJdbcUrls) {
+
+      // Retrieve the Oracle service name to use when connecting to the RAC...
+      String oracleServiceName =
+          OraOopUtilities.getOracleServiceName(jobData.getSqoopOptions()
+              .getConf());
+
+      // Generate JDBC URLs for each of the mappers...
+      if (!oracleServiceName.isEmpty()) {
+        if (!generateRacJdbcConnectionUrlsByServiceName(jdbcHost, jdbcPort,
+            oracleServiceName, jobData)) {
+          throw new RuntimeException(String.format(
+              "Unable to connect to the Oracle database at %s "
+                  + "via the service name \"%s\".", jobData.getSqoopOptions()
+                  .getConnectString(), oracleServiceName));
+        }
+      } else {
+        generateJdbcConnectionUrlsByActiveInstance(activeInstances, jdbcPort,
+            jobData);
+      }
+    } else {
+      generateJdbcConnectionUrlsBySidOrService(jdbcHost, jdbcPort, jdbcSid,
+          jdbcService, jobData);
+    }
+
+  }
+
+  private void generateJdbcConnectionUrlsBySidOrService(String hostName,
+      int port, String sid, String serviceName, JobData jobData) {
+
+    String jdbcUrl = null;
+
+    if (sid != null && !sid.isEmpty()) {
+      jdbcUrl = OraOopUtilities.generateOracleSidJdbcUrl(hostName, port, sid);
+    } else {
+      jdbcUrl =
+          OraOopUtilities.generateOracleServiceNameJdbcUrl(hostName, port,
+              serviceName);
+    }
+
+    // Now store these connection strings in such a way that each mapper knows
+    // which one to use...
+    for (int idxMapper = 0; idxMapper < jobData.getSqoopOptions()
+        .getNumMappers(); idxMapper++) {
+      storeJdbcUrlForMapper(idxMapper, jdbcUrl, jobData);
+    }
+  }
+
+  private boolean generateRacJdbcConnectionUrlsByServiceName(String hostName,
+      int port, String serviceName, JobData jobData) {
+
+    boolean result = false;
+    String jdbcUrl =
+        OraOopUtilities.generateOracleServiceNameJdbcUrl(hostName, port,
+            serviceName);
+
+    if (testDynamicallyGeneratedOracleRacInstanceConnection(jdbcUrl, jobData
+        .getSqoopOptions().getUsername(), jobData.getSqoopOptions()
+        .getPassword(), jobData.getSqoopOptions().getConnectionParams()
+        , false // <- ShowInstanceSysTimestamp
+        , "" // <- instanceDescription
+    )) {
+
+      LOG.info(String.format(
+          "%s will load-balance sessions across the Oracle RAC instances "
+              + "by connecting each mapper to the Oracle Service \"%s\".",
+          OraOopConstants.ORAOOP_PRODUCT_NAME, serviceName));
+
+      // Now store these connection strings in such a way that each mapper knows
+      // which one to use...
+      for (int idxMapper = 0; idxMapper < jobData.getSqoopOptions()
+          .getNumMappers(); idxMapper++) {
+        storeJdbcUrlForMapper(idxMapper, jdbcUrl, jobData);
+      }
+      result = true;
+    }
+    return result;
+  }
+
+  private void
+      generateJdbcConnectionUrlsByActiveInstance(
+          List<OracleActiveInstance> activeInstances, int jdbcPort,
+          JobData jobData) {
+
+    // Generate JDBC URLs for each of the instances in the RAC...
+    ArrayList<OraOopUtilities.JdbcOracleThinConnection>
+        jdbcOracleActiveThinConnections =
+            new ArrayList<OraOopUtilities.JdbcOracleThinConnection>(
+                activeInstances.size());
+
+    for (OracleActiveInstance activeInstance : activeInstances) {
+
+      OraOopUtilities.JdbcOracleThinConnection
+          jdbcActiveInstanceThinConnection =
+              new OraOopUtilities.JdbcOracleThinConnection(
+                  activeInstance.getHostName(),
+                  jdbcPort, activeInstance.getInstanceName(), "");
+
+      if (testDynamicallyGeneratedOracleRacInstanceConnection(
+          jdbcActiveInstanceThinConnection.toString(), jobData
+              .getSqoopOptions().getUsername(), jobData.getSqoopOptions()
+              .getPassword(), jobData.getSqoopOptions().getConnectionParams(),
+          true, activeInstance.getInstanceName())) {
+        jdbcOracleActiveThinConnections.add(jdbcActiveInstanceThinConnection);
+      }
+    }
+
+    // If there are multiple JDBC URLs that work okay for the RAC, then we'll
+    // make use of them...
+    if (jdbcOracleActiveThinConnections.size() < OraOopUtilities
+        .getMinNumberOfOracleRacActiveInstancesForDynamicJdbcUrlUse(jobData
+            .getSqoopOptions().getConf())) {
+      LOG.info(String
+          .format(
+              "%s will not attempt to load-balance sessions across instances "
+            + "of an Oracle RAC - as multiple JDBC URLs to the "
+            + "Oracle RAC could not be dynamically generated.",
+              OraOopConstants.ORAOOP_PRODUCT_NAME));
+      return;
+    } else {
+      StringBuilder msg = new StringBuilder();
+      msg.append(String
+          .format(
+              "%s will load-balance sessions across the following instances of"
+            + "the Oracle RAC:\n",
+              OraOopConstants.ORAOOP_PRODUCT_NAME));
+
+      for (OraOopUtilities.JdbcOracleThinConnection thinConnection
+               : jdbcOracleActiveThinConnections) {
+        msg.append(String.format("\tInstance: %s \t URL: %s\n",
+            thinConnection.getSid(), thinConnection.toString()));
+      }
+      LOG.info(msg.toString());
+    }
+
+    // Now store these connection strings in such a way that each mapper knows
+    // which one to use...
+    int racInstanceIdx = 0;
+    OraOopUtilities.JdbcOracleThinConnection thinUrl;
+    for (int idxMapper = 0; idxMapper < jobData.getSqoopOptions()
+        .getNumMappers(); idxMapper++) {
+      if (racInstanceIdx > jdbcOracleActiveThinConnections.size() - 1) {
+        racInstanceIdx = 0;
+      }
+      thinUrl = jdbcOracleActiveThinConnections.get(racInstanceIdx);
+      racInstanceIdx++;
+      storeJdbcUrlForMapper(idxMapper, thinUrl.toString(), jobData);
+    }
+  }
+
+  private boolean testDynamicallyGeneratedOracleRacInstanceConnection(
+      String url, String userName, String password, Properties additionalProps,
+      boolean showInstanceSysTimestamp, String instanceDescription) {
+
+    boolean result = false;
+
+    // Test the connection...
+    try {
+      Connection testConnection =
+          OracleConnectionFactory.createOracleJdbcConnection(
+              OraOopConstants.ORACLE_JDBC_DRIVER_CLASS, url, userName,
+              password, additionalProps);
+
+      // Show the system time on each instance...
+      if (showInstanceSysTimestamp) {
+        LOG.info(String.format("\tDatabase time on %s is %s",
+            instanceDescription, OraOopOracleQueries
+                .getSysTimeStamp(testConnection)));
+      }
+
+      testConnection.close();
+      result = true;
+    } catch (SQLException ex) {
+      LOG.warn(
+          String
+              .format(
+                  "The dynamically generated JDBC URL \"%s\" was unable to "
+                  + "connect to an instance in the Oracle RAC.",
+                  url), ex);
+    }
+
+    return result;
+  }
+
+  private void storeJdbcUrlForMapper(int mapperIdx, String jdbcUrl,
+      JobData jobData) {
+
+    // Now store these connection strings in such a way that each mapper knows
+    // which one to use...
+    Configuration conf = jobData.getSqoopOptions().getConf();
+    String mapperJdbcUrlPropertyName =
+        OraOopUtilities.getMapperJdbcUrlPropertyName(mapperIdx, conf);
+    conf.set(mapperJdbcUrlPropertyName, jdbcUrl);
+  }
+
+  private boolean isOraOopEnabled(SqoopOptions sqoopOptions) {
+
+    String oraOopDisabled =
+        sqoopOptions.getConf().get(OraOopConstants.ORAOOP_DISABLED, "false")
+            .toLowerCase();
+    boolean oraOopIsDisabled =
+        oraOopDisabled.equalsIgnoreCase("true")
+            || oraOopDisabled.equalsIgnoreCase("yes")
+            || oraOopDisabled.equalsIgnoreCase("y")
+            || oraOopDisabled.equalsIgnoreCase("1");
+
+    oraOopIsDisabled = oraOopIsDisabled || !sqoopOptions.isDirect();
+
+    if (oraOopIsDisabled) {
+      LOG.info(String.format("%s is disabled.",
+          OraOopConstants.ORAOOP_PRODUCT_NAME));
+    }
+
+    return !oraOopIsDisabled;
+  }
+
+  private OraOopConstants.Sqoop.Tool getSqoopJobType(JobData jobData) {
+
+    OraOopConstants.Sqoop.Tool result = OraOopConstants.Sqoop.Tool.UNKNOWN;
+
+    String sqoopToolName = getSqoopToolName(jobData).toUpperCase().trim();
+    try {
+      result = OraOopConstants.Sqoop.Tool.valueOf(sqoopToolName);
+    } catch (IllegalArgumentException ex) {
+      LOG.debug(String.format(
+          "The Sqoop tool name \"%s\" is not supported by OraOop",
+          sqoopToolName), ex);
+    }
+    return result;
+  }
+
+  private boolean isNumberOfImportMappersOkay(SqoopOptions sqoopOptions) {
+
+    // Check whether there are enough mappers for OraOop to be of benefit...
+    boolean result =
+      (sqoopOptions.getNumMappers() >= OraOopUtilities
+          .getMinNumberOfImportMappersAcceptedByOraOop(sqoopOptions.getConf()));
+
+    if (!result) {
+      LOG.info(String.format(
+        "%s will not process this sqoop connection, as an insufficient number "
+            + "of mappers are being used.",
+        OraOopConstants.ORAOOP_PRODUCT_NAME));
+    }
+
+    return result;
+  }
+
+  private boolean isNumberOfExportMappersOkay(SqoopOptions sqoopOptions) {
+
+    // Check whether there are enough mappers for OraOop to be of benefit...
+    boolean result =
+        (sqoopOptions.getNumMappers() >= OraOopUtilities
+          .getMinNumberOfExportMappersAcceptedByOraOop(sqoopOptions.getConf()));
+
+    if (!result) {
+      LOG.info(String.format(
+        "%s will not process this sqoop connection, as an insufficient number "
+            + "of mappers are being used.",
+        OraOopConstants.ORAOOP_PRODUCT_NAME));
+    }
+
+    return result;
+  }
+
+  private boolean isSqoopImportJobTableBased(SqoopOptions sqoopOptions) {
+
+    String tableName = sqoopOptions.getTableName();
+    return (tableName != null && !tableName.isEmpty());
+  }
+
+  private boolean isSqoopTableAnOracleTable(Connection connection,
+      String connectionUserName, OracleTable tableContext) {
+
+    String oracleObjectType;
+
+    try {
+
+      // Find the table via dba_tables...
+      OracleTable oracleTable =
+          OraOopOracleQueries.getTable(connection, tableContext.getSchema(),
+              tableContext.getName());
+      if (oracleTable != null) {
+        return true;
+      }
+
+      // If we could not find the table via dba_tables, then try and determine
+      // what type of database object the
+      // user was referring to. Perhaps they've specified the name of a view?...
+      oracleObjectType =
+          OraOopOracleQueries.getOracleObjectType(connection, tableContext);
+
+      if (oracleObjectType == null) {
+        LOG.info(String.format(
+            "%1$s will not process this Sqoop connection, "
+          + "as the Oracle user %2$s does not own a table named %3$s.\n"
+          + "\tPlease prefix the table name with the owner.\n "
+          + "\tNote: You may need to double-quote the owner and/or table name."
+          + "\n\tE.g. sqoop ... --username %4$s --table %2$s.%3$s\n",
+          OraOopConstants.ORAOOP_PRODUCT_NAME, tableContext.getSchema(),
+          tableContext.getName(), connectionUserName));
+        return false;
+      }
+
+    } catch (SQLException ex) {
+      LOG.warn(String.format(
+        "Unable to determine the Oracle-type of the object named %s owned by "
+            + "%s.\nError:\n" + "%s", tableContext.getName(), tableContext
+            .getSchema(), ex.getMessage()));
+
+      // In the absence of conflicting information, let's assume the object is
+      // actually a table...
+      return true;
+    }
+
+    boolean result =
+        oracleObjectType
+            .equalsIgnoreCase(OraOopConstants.Oracle.OBJECT_TYPE_TABLE);
+
+    if (!result) {
+      LOG.info(String.format("%s will not process this sqoop connection, "
+          + "as %s is not an Oracle table, it's a %s.",
+          OraOopConstants.ORAOOP_PRODUCT_NAME, tableContext.toString(),
+          oracleObjectType));
+    }
+
+    return result;
+  }
+
+  private boolean isSqoopTableAnIndexOrganizedTable(Connection connection,
+      OracleTable tableContext) {
+
+    boolean result = false;
+    try {
+      result =
+          OraOopOracleQueries.isTableAnIndexOrganizedTable(connection,
+              tableContext);
+      if (result) {
+        LOG.info(String.format("%s will not process this Sqoop connection, "
+            + "as the Oracle table %s is an index-organized table.",
+            OraOopConstants.ORAOOP_PRODUCT_NAME, tableContext.toString()));
+      }
+      return result;
+    } catch (SQLException ex) {
+      LOG.warn(String.format(
+         "Unable to determine whether the Oracle table %s is an index-organized"
+       + " table.\nError:\n" + "%s", tableContext.toString(), ex.getMessage()));
+    }
+    return result;
+  }
+
+  private String getSqoopToolName(JobData jobData) {
+
+    return jobData.getSqoopTool().getToolName();
+  }
+
+  private String getOracleSessionActionName(JobData jobData) {
+
+    // This method has been written assuming that:
+    // (1) OraOop only processes Sqoop "import" and "export" jobs; and
+    // (2) a table will be used during the import/export (not a query).
+
+    if (getSqoopJobType(jobData) != OraOopConstants.Sqoop.Tool.IMPORT
+        && getSqoopJobType(jobData) != OraOopConstants.Sqoop.Tool.EXPORT) {
+      throw new UnsupportedOperationException(String.format(
+          "%s needs to be updated to cope " + "with Sqoop jobs of type %s.",
+          OraOopUtilities.getCurrentMethodName(), getSqoopToolName(jobData)));
+    }
+
+    String timeStr =
+        (new SimpleDateFormat("yyyyMMddHHmmsszzz")).format(new Date());
+
+    String result = String.format("%s %s", getSqoopToolName(jobData), timeStr);
+
+    // NOTE: The "action" column of v$session is only a 32 character column.
+    // Therefore we need to ensure that the string returned by this
+    // method does not exceed 32 characters...
+    if (result.length() > 32) {
+      result = result.substring(0, 32).trim();
+    }
+
+    return result;
+  }
+
+  private boolean isSqoopImportIncremental(JobData jobData) {
+
+    boolean result =
+        jobData.getSqoopOptions().getIncrementalMode() != IncrementalMode.None;
+
+    if (result) {
+      LOG.info(String.format("%1$s will not process this sqoop connection, "
+          + "as incremental mode is not supported by %1$s.",
+          OraOopConstants.ORAOOP_PRODUCT_NAME));
+    }
+
+    return result;
+  }
+
+  private void showUserTheOraOopWelcomeMessage() {
+
+    String msg1 =
+        String.format("Using %s", OraOopConstants.ORAOOP_PRODUCT_NAME);
+
+    int longestMessage = msg1.length();
+
+    msg1 = OraOopUtilities.padRight(msg1, longestMessage);
+
+    char[] asterisks = new char[longestMessage + 8];
+    Arrays.fill(asterisks, '*');
+
+    String msg =
+        String.format("\n" + "%1$s\n" + "*** %2$s ***\n" + "%1$s", new String(
+            asterisks), msg1);
+    LOG.info(msg);
+  }
+
+  private void showUserTheOracleCommandToKillOraOop(SqoopOptions sqoopOptions) {
+
+    int taskAttempts =
+        sqoopOptions.getConf().getInt(
+            OraOopConstants.Sqoop.MAX_MAPREDUCE_ATTEMPTS, -1);
+
+    // If killing the Oracle session if futile - because the job will be
+    // reattempted, then don't
+    // bother telling the user about this feature...
+    if (taskAttempts != 1) {
+      return;
+    }
+
+    String moduleName = OraOopConstants.ORACLE_SESSION_MODULE_NAME;
+    String actionName =
+        sqoopOptions.getConf().get(OraOopConstants.ORACLE_SESSION_ACTION_NAME);
+
+    String msg = String.format(
+        "\nNote: This %s job can be killed via Oracle by executing the "
+      + "following statement:\n\tbegin\n"
+      + "\t\tfor row in (select sid,serial# from v$session where module='%s' "
+      + "and action='%s') loop\n"
+      + "\t\t\texecute immediate 'alter system kill session ''' || row.sid || "
+      + "',' || row.serial# || '''';\n"
+      + "\t\tend loop;\n" + "\tend;",
+      OraOopConstants.ORAOOP_PRODUCT_NAME, moduleName, actionName);
+    LOG.info(msg);
+  }
+
+  private void createAnyRequiredOracleObjects(SqoopOptions sqoopOptions,
+    Connection connection, OraOopConnManager oraOopConnManager,
+    List<OraOopLogMessage> messagesToDisplayAfterWelcome) throws SQLException {
+
+    Configuration conf = sqoopOptions.getConf();
+
+    // The SYSDATE on the Oracle database will be used as the partition value
+    // for this export job...
+    Object sysDateTime = OraOopOracleQueries.getSysDate(connection);
+    String sysDateStr =
+      OraOopOracleQueries.oraDATEToString(sysDateTime, "yyyy-mm-dd hh24:mi:ss");
+    OraOopUtilities.rememberOracleDateTime(conf,
+        OraOopConstants.ORAOOP_JOB_SYSDATE, sysDateStr);
+
+    checkForOldOraOopTemporaryOracleTables(connection, sysDateTime,
+        OraOopOracleQueries.getCurrentSchema(connection),
+        messagesToDisplayAfterWelcome);
+
+    // Store the actual partition value, so the N mappers know what value to
+    // insert...
+    String partitionValue =
+        OraOopOracleQueries.oraDATEToString(sysDateTime,
+            OraOopConstants.ORAOOP_EXPORT_PARTITION_DATE_FORMAT);
+    conf.set(OraOopConstants.ORAOOP_EXPORT_PARTITION_DATE_VALUE,
+             partitionValue);
+
+    // Generate the (22 character) partition name...
+    String partitionName =
+        OraOopUtilities
+            .createExportTablePartitionNameFromOracleTimestamp(sysDateTime);
+
+    int numMappers = sqoopOptions.getNumMappers();
+
+    String exportTableTemplate =
+        conf.get(OraOopConstants.ORAOOP_EXPORT_CREATE_TABLE_TEMPLATE, "");
+    OracleTable templateTableContext =
+        OraOopUtilities.decodeOracleTableName(sqoopOptions.getUsername(),
+            exportTableTemplate);
+
+    boolean noLoggingOnNewTable =
+        conf.getBoolean(OraOopConstants.ORAOOP_EXPORT_CREATE_TABLE_NO_LOGGING,
+            false);
+
+    String updateKeyCol = sqoopOptions.getUpdateKeyCol();
+
+    /* =========================== */
+    /* VALIDATION OF INPUTS */
+    /* =========================== */
+
+    if (updateKeyCol == null || updateKeyCol.isEmpty()) {
+      // We're performing an "insert" export, not an "update" export.
+
+      // Check that the "oraoop.export.merge" property has not been specified,
+      // as this would be
+      // an invalid scenario...
+      if (OraOopUtilities.getExportUpdateMode(conf) == UpdateMode.Merge) {
+        throw new RuntimeException(String.format(
+            "\n\nThe option \"%s\" can only be used if \"%s\" is "
+                + "also used.\n", OraOopConstants.ORAOOP_EXPORT_MERGE,
+            "--update-key"));
+      }
+    }
+
+    if (OraOopUtilities
+        .userWantsToCreatePartitionedExportTableFromTemplate(conf)
+        || OraOopUtilities
+            .userWantsToCreateNonPartitionedExportTableFromTemplate(conf)) {
+
+      // OraOop will create the export table.
+
+      if (oraOopConnManager.getOracleTableContext().getName().length()
+              > OraOopConstants.Oracle.MAX_IDENTIFIER_LENGTH) {
+        String msg =
+            String.format(
+                "The Oracle table name \"%s\" is longer than %d characters.\n"
+              + "Oracle will not allow a table with this name to be created.",
+            oraOopConnManager.getOracleTableContext().getName(),
+            OraOopConstants.Oracle.MAX_IDENTIFIER_LENGTH);
+        throw new RuntimeException(msg);
+      }
+
+      if (updateKeyCol != null && !updateKeyCol.isEmpty()) {
+
+        // We're performing an "update" export, not an "insert" export.
+
+        // Check whether the user is attempting an "update" (i.e. a non-merge).
+        // If so, they're
+        // asking to only UPDATE rows in a (about to be created) (empty) table
+        // that contains no rows.
+        // This will be a waste of time, as we'd be attempting to perform UPDATE
+        // operations against a
+        // table with no rows in it...
+        UpdateMode updateMode = OraOopUtilities.getExportUpdateMode(conf);
+        if (updateMode == UpdateMode.Update) {
+          throw new RuntimeException(String.format(
+              "\n\nCombining the option \"%s\" with the option \"%s=false\" is "
+            + "nonsensical, as this would create an "
+            + "empty table and then perform "
+            + "a lot of work that results in a table containing no rows.\n",
+              OraOopConstants.ORAOOP_EXPORT_CREATE_TABLE_TEMPLATE,
+              OraOopConstants.ORAOOP_EXPORT_MERGE));
+        }
+      }
+
+      // Check that the specified template table actually exists and is a
+      // table...
+      String templateTableObjectType =
+          OraOopOracleQueries.getOracleObjectType(connection,
+              templateTableContext);
+      if (templateTableObjectType == null) {
+        throw new RuntimeException(String.format(
+            "The specified Oracle template table \"%s\" does not exist.",
+            templateTableContext.toString()));
+      }
+
+      if (!templateTableObjectType
+          .equalsIgnoreCase(OraOopConstants.Oracle.OBJECT_TYPE_TABLE)) {
+        throw new RuntimeException(
+          String.format(
+              "The specified Oracle template table \"%s\" is not an "
+            + "Oracle table, it's a %s.",
+              templateTableContext.toString(), templateTableObjectType));
+      }
+
+      if (conf.getBoolean(OraOopConstants.ORAOOP_EXPORT_CREATE_TABLE_DROP,
+          false)) {
+        OraOopOracleQueries.dropTable(connection, oraOopConnManager
+            .getOracleTableContext());
+      }
+
+      // Check that there is no existing database object with the same name of
+      // the table to be created...
+      String newTableObjectType =
+          OraOopOracleQueries.getOracleObjectType(connection, oraOopConnManager
+              .getOracleTableContext());
+      if (newTableObjectType != null) {
+        throw new RuntimeException(
+          String.format(
+              "%s cannot create a new Oracle table named %s as a \"%s\" "
+            + "with this name already exists.",
+            OraOopConstants.ORAOOP_PRODUCT_NAME, oraOopConnManager
+                .getOracleTableContext().toString(), newTableObjectType));
+      }
+    } else {
+      // The export table already exists.
+
+      if (updateKeyCol != null && !updateKeyCol.isEmpty()) {
+
+        // We're performing an "update" export, not an "insert" export.
+
+        // Check that there exists an index on the export table on the
+        // update-key column(s).
+        // Without such an index, this export may perform like a real dog...
+        String[] updateKeyColumns =
+            OraOopUtilities.getExportUpdateKeyColumnNames(sqoopOptions);
+        if (!OraOopOracleQueries.doesIndexOnColumnsExist(connection,
+            oraOopConnManager.getOracleTableContext(), updateKeyColumns)) {
+          String msg = String.format(
+              "\n**************************************************************"
+            + "***************************************************************"
+            + "\n\tThe table %1$s does not have a valid index on "
+            + "the column(s) %2$s.\n"
+            + "\tAs a consequence, this export may take a long time to "
+            + "complete.\n"
+            + "\tIf performance is unacceptable, consider reattempting this "
+            + "job after creating an index "
+            + "on this table via the SQL...\n"
+            + "\t\tcreate index <index_name> on %1$s(%2$s);\n"
+            + "****************************************************************"
+            + "*************************************************************",
+                      oraOopConnManager.getOracleTableContext().toString(),
+                      OraOopUtilities.stringArrayToCSV(updateKeyColumns));
+          messagesToDisplayAfterWelcome.add(new OraOopLogMessage(
+              OraOopConstants.Logging.Level.WARN, msg));
+        }
+      }
+    }
+
+    /* ================================= */
+    /* CREATE A PARTITIONED TABLE */
+    /* ================================= */
+    if (OraOopUtilities
+        .userWantsToCreatePartitionedExportTableFromTemplate(conf)) {
+
+      // Create a new Oracle table using the specified template...
+
+      String[] subPartitionNames =
+          OraOopUtilities.generateExportTableSubPartitionNames(numMappers,
+              sysDateTime, conf);
+      // Create the export table from a template table...
+      String tableStorageClause =
+          OraOopUtilities.getExportTableStorageClause(conf);
+
+      OraOopOracleQueries.createExportTableFromTemplateWithPartitioning(
+          connection, oraOopConnManager.getOracleTableContext(),
+          tableStorageClause, templateTableContext, noLoggingOnNewTable,
+          partitionName, sysDateTime, sqoopOptions.getNumMappers(),
+          subPartitionNames);
+      return;
+    }
+
+    /* ===================================== */
+    /* CREATE A NON-PARTITIONED TABLE */
+    /* ===================================== */
+    if (OraOopUtilities
+        .userWantsToCreateNonPartitionedExportTableFromTemplate(conf)) {
+
+      String tableStorageClause =
+          OraOopUtilities.getExportTableStorageClause(conf);
+
+      OraOopOracleQueries.createExportTableFromTemplate(connection,
+          oraOopConnManager.getOracleTableContext(), tableStorageClause,
+          templateTableContext, noLoggingOnNewTable);
+      return;
+    }
+
+    /* ===================================================== */
+    /* ADD ADDITIONAL PARTITIONS TO AN EXISTING TABLE */
+    /* ===================================================== */
+
+    // If the export table is partitioned, and the partitions were created by
+    // OraOop, then we need
+    // create additional partitions...
+
+    OracleTablePartitions tablePartitions =
+        OraOopOracleQueries.getPartitions(connection, oraOopConnManager
+            .getOracleTableContext());
+    // Find any partition name starting with "ORAOOP_"...
+    OracleTablePartition oraOopPartition =
+        tablePartitions.findPartitionByRegEx("^"
+            + OraOopConstants.EXPORT_TABLE_PARTITION_NAME_PREFIX);
+
+    if (tablePartitions.size() > 0 && oraOopPartition == null) {
+
+      for (int idx = 0; idx < tablePartitions.size(); idx++) {
+        messagesToDisplayAfterWelcome.add(new OraOopLogMessage(
+            OraOopConstants.Logging.Level.INFO, String.format(
+                "The Oracle table %s has a partition named \"%s\".",
+                oraOopConnManager.getOracleTableContext().toString(),
+                tablePartitions.get(idx).getName())));
+      }
+
+      messagesToDisplayAfterWelcome.add(new OraOopLogMessage(
+          OraOopConstants.Logging.Level.WARN, String.format(
+              "The Oracle table %s is partitioned.\n"
+                  + "These partitions were not created by %s.",
+              oraOopConnManager.getOracleTableContext().toString(),
+              OraOopConstants.ORAOOP_PRODUCT_NAME)));
+    }
+
+    if (oraOopPartition != null) {
+
+      // Indicate in the configuration what's happening...
+      conf.setBoolean(OraOopConstants.EXPORT_TABLE_HAS_ORAOOP_PARTITIONS, true);
+
+      messagesToDisplayAfterWelcome
+          .add(new OraOopLogMessage(
+              OraOopConstants.Logging.Level.INFO,
+              String
+                  .format(
+                      "The Oracle table %s is partitioned.\n"
+                          + "These partitions were created by %s, so "
+                          + "additional partitions will now be created.\n"
+                          + "The name of the new partition will be \"%s\".",
+                      oraOopConnManager.getOracleTableContext().toString(),
+                      OraOopConstants.ORAOOP_PRODUCT_NAME, partitionName)));
+
+      String[] subPartitionNames =
+          OraOopUtilities.generateExportTableSubPartitionNames(numMappers,
+              sysDateTime, conf);
+
+      // Add another partition (and N subpartitions) to this existing,
+      // partitioned export table...
+      OraOopOracleQueries.createMoreExportTablePartitions(connection,
+          oraOopConnManager.getOracleTableContext(), partitionName,
+          sysDateTime, subPartitionNames);
+
+      return;
+    }
+  }
+
+  private void checkForOldOraOopTemporaryOracleTables(Connection connection,
+      Object sysDateTime, String schema,
+      List<OraOopLogMessage> messagesToDisplayAfterWelcome) {
+
+    try {
+
+      StringBuilder message = new StringBuilder();
+      message
+        .append(String.format(
+          "The following tables appear to be old temporary tables created by "
+        + "%s that have not been deleted.\n"
+        + "They are probably left over from jobs that encountered an error and "
+        + "could not clean up after themselves.\n"
+        + "You might want to drop these Oracle tables in order to reclaim "
+        + "Oracle storage space:\n", OraOopConstants.ORAOOP_PRODUCT_NAME));
+      boolean showMessage = false;
+
+      String generatedTableName =
+          OraOopUtilities.generateExportTableMapperTableName(0, sysDateTime,
+              schema).getName();
+      generatedTableName = generatedTableName.replaceAll("[0-9]", "%");
+      generatedTableName =
+          OraOopUtilities.replaceAll(generatedTableName, "%%", "%");
+      Date sysDate = OraOopOracleQueries.oraDATEToDate(sysDateTime);
+
+      List<OracleTable> tables =
+          OraOopOracleQueries.getTablesWithTableNameLike(connection, schema,
+              generatedTableName);
+
+      for (OracleTable oracleTable : tables) {
+        OraOopUtilities.DecodedExportMapperTableName tableName =
+            OraOopUtilities.decodeExportTableMapperTableName(oracleTable);
+        if (tableName != null) {
+          Date tableDate =
+              OraOopOracleQueries.oraDATEToDate(tableName.getTableDateTime());
+          double daysApart =
+              (sysDate.getTime() - tableDate.getTime()) / (1000 * 60 * 60 * 24);
+          if (daysApart > 1.0) {
+            showMessage = true;
+            message.append(String.format("\t%s\n", oracleTable.toString()));
+          }
+        }
+      }
+
+      if (showMessage) {
+        messagesToDisplayAfterWelcome.add(new OraOopLogMessage(
+            OraOopConstants.Logging.Level.INFO, message.toString()));
+      }
+    } catch (Exception ex) {
+      messagesToDisplayAfterWelcome.add(new OraOopLogMessage(
+          OraOopConstants.Logging.Level.WARN, String.format(
+              "%s was unable to check for the existance of old "
+                  + "temporary Oracle tables.\n" + "Error:\n%s",
+              OraOopConstants.ORAOOP_PRODUCT_NAME, ex.toString())));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopOracleDataChunk.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopOracleDataChunk.java b/src/java/org/apache/sqoop/manager/oracle/OraOopOracleDataChunk.java
new file mode 100644
index 0000000..bc94abd
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopOracleDataChunk.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.sqoop.manager.oracle;
+
+import java.lang.reflect.Field;
+
+import org.apache.hadoop.io.Writable;
+
+/**
+ * How data should be split between mappers.
+ */
+public abstract class OraOopOracleDataChunk implements Writable {
+
+  private String id;
+
+  public abstract int getNumberOfBlocks();
+
+  public String getWhereClause() {
+    return "1=1";
+  }
+
+  public String getPartitionClause() {
+    return "";
+  }
+
+  @Override
+  public String toString() {
+
+    String result = super.toString();
+    for (Field field : this.getClass().getDeclaredFields()) {
+      try {
+        Object fieldValue = field.get(this);
+        result +=
+            String.format("\n\t%s = %s", field.getName(),
+                (fieldValue == null ? "null" : fieldValue.toString()));
+      } catch (IllegalAccessException ex) {
+        // Ignore this exception.
+      }
+    }
+
+    return result;
+  }
+
+  public String getId() {
+    return id;
+  }
+
+  public void setId(String newId) {
+    this.id = newId;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopOracleDataChunkExtent.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopOracleDataChunkExtent.java b/src/java/org/apache/sqoop/manager/oracle/OraOopOracleDataChunkExtent.java
new file mode 100644
index 0000000..5262d0e
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopOracleDataChunkExtent.java
@@ -0,0 +1,93 @@
+/**
+ * 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.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import org.apache.hadoop.io.Text;
+
+/**
+ * Data should be split by extent for ROWID scans.
+ */
+public class OraOopOracleDataChunkExtent extends OraOopOracleDataChunk {
+
+  private int oracleDataObjectId;
+  private int relativeDatafileNumber;
+  private int startBlockNumber;
+  private int finishBlockNumber;
+
+  OraOopOracleDataChunkExtent() {
+
+  }
+
+  OraOopOracleDataChunkExtent(String id, int oracleDataObjectId,
+      int relativeDatafileNumber, int startBlockNumber, int finishBlockNumber) {
+
+    this.setId(id);
+    this.oracleDataObjectId = oracleDataObjectId;
+    this.relativeDatafileNumber = relativeDatafileNumber;
+    this.startBlockNumber = startBlockNumber;
+    this.finishBlockNumber = finishBlockNumber;
+  }
+
+  @Override
+  public String getWhereClause() {
+    return String.format(
+        "(rowid >= dbms_rowid.rowid_create(%d, %d, %d, %d, %d)",
+        OraOopConstants.Oracle.ROWID_EXTENDED_ROWID_TYPE,
+        this.oracleDataObjectId, this.relativeDatafileNumber,
+        this.startBlockNumber, 0)
+        + String.format(
+            " AND rowid <= dbms_rowid.rowid_create(%d, %d, %d, %d, %d))",
+            OraOopConstants.Oracle.ROWID_EXTENDED_ROWID_TYPE,
+            this.oracleDataObjectId, this.relativeDatafileNumber,
+            this.finishBlockNumber,
+            OraOopConstants.Oracle.ROWID_MAX_ROW_NUMBER_PER_BLOCK);
+  }
+
+  @Override
+  public void write(DataOutput output) throws IOException {
+    Text.writeString(output, this.getId());
+    output.writeInt(this.oracleDataObjectId);
+    output.writeInt(this.relativeDatafileNumber);
+    output.writeInt(this.startBlockNumber);
+    output.writeInt(this.finishBlockNumber);
+  }
+
+  @Override
+  public void readFields(DataInput input) throws IOException {
+    this.setId(Text.readString(input));
+    this.oracleDataObjectId = input.readInt();
+    this.relativeDatafileNumber = input.readInt();
+    this.startBlockNumber = input.readInt();
+    this.finishBlockNumber = input.readInt();
+  }
+
+  @Override
+  public int getNumberOfBlocks() {
+
+    if (this.finishBlockNumber == 0 && this.startBlockNumber == 0) {
+      return 0;
+    } else {
+      return (this.finishBlockNumber - this.startBlockNumber) + 1;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OraOopOracleDataChunkPartition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OraOopOracleDataChunkPartition.java b/src/java/org/apache/sqoop/manager/oracle/OraOopOracleDataChunkPartition.java
new file mode 100644
index 0000000..fbd8a16
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OraOopOracleDataChunkPartition.java
@@ -0,0 +1,78 @@
+/**
+ * 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.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Text;
+
+/**
+ * Data should be split by partition.
+ */
+public class OraOopOracleDataChunkPartition extends OraOopOracleDataChunk {
+
+  private boolean isSubPartition;
+  private int blocks;
+
+  OraOopOracleDataChunkPartition() {
+
+  }
+
+  OraOopOracleDataChunkPartition(String partitionName, boolean isSubPartition,
+      int blocks) {
+    this.setId(partitionName);
+    this.isSubPartition = isSubPartition;
+    this.blocks = blocks;
+  }
+
+  @Override
+  public int getNumberOfBlocks() {
+    return this.blocks;
+  }
+
+  @Override
+  public void write(DataOutput output) throws IOException {
+    Text.writeString(output, this.getId());
+    output.writeBoolean(this.isSubPartition);
+    output.writeInt(this.blocks);
+  }
+
+  @Override
+  public void readFields(DataInput input) throws IOException {
+    this.setId(Text.readString(input));
+    this.isSubPartition = input.readBoolean();
+    this.blocks = input.readInt();
+  }
+
+  @Override
+  public String getPartitionClause() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(" ");
+    if (this.isSubPartition) {
+      sb.append("SUBPARTITION");
+    } else {
+      sb.append("PARTITION");
+    }
+    sb.append("(\"").append(this.getId()).append("\")");
+    return sb.toString();
+  }
+
+}


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

Posted by ve...@apache.org.
http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OracleTableColumn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OracleTableColumn.java b/src/java/org/apache/sqoop/manager/oracle/OracleTableColumn.java
new file mode 100644
index 0000000..3b28e0c
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OracleTableColumn.java
@@ -0,0 +1,59 @@
+/**
+ * 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;
+
+/**
+ * Contains details about a column in an Oracle table.
+ */
+public class OracleTableColumn {
+
+  private String name;
+  private String dataType; // <- i.e. The data_type from dba_tab_columns
+  private int oracleType;
+
+  public OracleTableColumn(String name, String dataType) {
+
+    this.setName(name);
+    this.setDataType(dataType);
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String newName) {
+    this.name = newName;
+  }
+
+  public String getDataType() {
+    return dataType;
+  }
+
+  public void setDataType(String newDataType) {
+    this.dataType = newDataType;
+  }
+
+  public int getOracleType() {
+    return oracleType;
+  }
+
+  public void setOracleType(int newOracleType) {
+    this.oracleType = newOracleType;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OracleTableColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OracleTableColumns.java b/src/java/org/apache/sqoop/manager/oracle/OracleTableColumns.java
new file mode 100644
index 0000000..ea6d549
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OracleTableColumns.java
@@ -0,0 +1,43 @@
+/**
+ * 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.Iterator;
+
+/**
+ * Contains a list of Oracle columns.
+ */
+public class OracleTableColumns extends
+    OraOopGenerics.ObjectList<OracleTableColumn> {
+
+  public OracleTableColumn findColumnByName(String columnName) {
+
+    OracleTableColumn result;
+
+    Iterator<OracleTableColumn> iterator = this.iterator();
+    while (iterator.hasNext()) {
+      result = iterator.next();
+      if (result.getName().equals(columnName)) {
+        return result;
+      }
+    }
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OracleTablePartition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OracleTablePartition.java b/src/java/org/apache/sqoop/manager/oracle/OracleTablePartition.java
new file mode 100644
index 0000000..2661010
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OracleTablePartition.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;
+
+/**
+ * Contains details about a partition for an Oracle table.
+ */
+public class OracleTablePartition {
+
+  private String name;
+  private boolean isSubPartition;
+
+  public OracleTablePartition(String name, boolean isSubPartition) {
+    this.setName(name);
+    this.setSubPartition(isSubPartition);
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String newName) {
+    this.name = newName;
+  }
+
+  public boolean isSubPartition() {
+    return isSubPartition;
+  }
+
+  public void setSubPartition(boolean newIsSubPartition) {
+    this.isSubPartition = newIsSubPartition;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OracleTablePartitions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OracleTablePartitions.java b/src/java/org/apache/sqoop/manager/oracle/OracleTablePartitions.java
new file mode 100644
index 0000000..5795187
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OracleTablePartitions.java
@@ -0,0 +1,62 @@
+/**
+ * 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.Iterator;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Contains a list of Oracle table partitions.
+ */
+public class OracleTablePartitions extends
+    OraOopGenerics.ObjectList<OracleTablePartition> {
+
+  public OracleTablePartition findPartitionByName(String partitionName) {
+
+    OracleTablePartition result;
+
+    Iterator<OracleTablePartition> iterator = this.iterator();
+    while (iterator.hasNext()) {
+      result = iterator.next();
+      if (result.getName().equals(partitionName)) {
+        return result;
+      }
+    }
+    return null;
+  }
+
+  public OracleTablePartition findPartitionByRegEx(String regEx) {
+
+    OracleTablePartition result;
+
+    Pattern pattern = Pattern.compile(regEx);
+
+    Iterator<OracleTablePartition> iterator = this.iterator();
+    while (iterator.hasNext()) {
+      result = iterator.next();
+      Matcher matcher = pattern.matcher(result.getName());
+      if (matcher.find()) {
+        return result;
+      }
+    }
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/java/org/apache/sqoop/manager/oracle/OracleVersion.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/manager/oracle/OracleVersion.java b/src/java/org/apache/sqoop/manager/oracle/OracleVersion.java
new file mode 100644
index 0000000..0d90065
--- /dev/null
+++ b/src/java/org/apache/sqoop/manager/oracle/OracleVersion.java
@@ -0,0 +1,84 @@
+/**
+ * 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;
+
+/**
+ * Represents an Oracle version and allows comparing of versions.
+ */
+public class OracleVersion {
+  private int major;
+  private int minor;
+  private int version;
+  private int patch;
+  private String banner;
+
+  public OracleVersion(int major, int minor, int version, int patch,
+      String banner) {
+
+    this.major = major;
+    this.minor = minor;
+    this.version = version;
+    this.patch = patch;
+    this.banner = banner;
+  }
+
+  public boolean isGreaterThanOrEqualTo(int otherMajor, int otherMinor,
+      int otherVersion, int otherPatch) {
+
+    if (this.major > otherMajor) {
+      return true;
+    }
+
+    if (this.major == otherMajor && this.minor > otherMinor) {
+      return true;
+    }
+
+    if (this.major == otherMajor && this.minor == otherMinor
+        && this.version > otherVersion) {
+      return true;
+    }
+
+    if (this.major == otherMajor && this.minor == otherMinor
+        && this.version == otherVersion && this.patch >= otherPatch) {
+      return true;
+    }
+
+    return false;
+  }
+
+  public int getMajor() {
+    return major;
+  }
+
+  public int getMinor() {
+    return minor;
+  }
+
+  public int getVersion() {
+    return version;
+  }
+
+  public int getPatch() {
+    return patch;
+  }
+
+  public String getBanner() {
+    return banner;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/com/cloudera/sqoop/manager/OracleUtils.java
----------------------------------------------------------------------
diff --git a/src/test/com/cloudera/sqoop/manager/OracleUtils.java b/src/test/com/cloudera/sqoop/manager/OracleUtils.java
index 70e2925..d9e31b8 100644
--- a/src/test/com/cloudera/sqoop/manager/OracleUtils.java
+++ b/src/test/com/cloudera/sqoop/manager/OracleUtils.java
@@ -46,6 +46,16 @@ public final class OracleUtils {
   public static final String ORACLE_SECONDARY_USER_NAME = "SQOOPTEST2";
   public static final String ORACLE_SECONDARY_USER_PASS = "ABCDEF";
 
+  public static final String ORACLE_INVALID_USER_NAME = "invalidusr";
+  public static final String SYSTEMTEST_TABLE_NAME = "oraoop_test";
+  public static final int SYSTEMTEST_NUM_ROWS = 100;
+  public static final int INTEGRATIONTEST_NUM_ROWS = 10000;
+  // Number of mappers if wanting to override default setting
+  public static final int NUM_MAPPERS = 0;
+  // Oracle degree of parallelism to use when creating table.
+  // If 0 we will calculate a recommended value
+  public static final int ORACLE_PARALLEL_DEGREE = 0;
+
   private OracleUtils() { }
 
   public static void setOracleAuth(SqoopOptions options) {

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/com/cloudera/sqoop/testutil/BaseSqoopTestCase.java
----------------------------------------------------------------------
diff --git a/src/test/com/cloudera/sqoop/testutil/BaseSqoopTestCase.java b/src/test/com/cloudera/sqoop/testutil/BaseSqoopTestCase.java
index 793c23e..a94ab90 100644
--- a/src/test/com/cloudera/sqoop/testutil/BaseSqoopTestCase.java
+++ b/src/test/com/cloudera/sqoop/testutil/BaseSqoopTestCase.java
@@ -220,6 +220,8 @@ public abstract class BaseSqoopTestCase extends TestCase {
       manager = testServer.getManager();
     } else {
       Configuration conf = getConf();
+      //Need to disable OraOop for existing tests
+      conf.set("oraoop.disabled", "true");
       SqoopOptions opts = getSqoopOptions(conf);
       opts.setConnectString(getConnectString());
       opts.setTableName(getTableName());

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/com/cloudera/sqoop/testutil/ExportJobTestCase.java
----------------------------------------------------------------------
diff --git a/src/test/com/cloudera/sqoop/testutil/ExportJobTestCase.java b/src/test/com/cloudera/sqoop/testutil/ExportJobTestCase.java
index 4421f0c..9a6e8da 100644
--- a/src/test/com/cloudera/sqoop/testutil/ExportJobTestCase.java
+++ b/src/test/com/cloudera/sqoop/testutil/ExportJobTestCase.java
@@ -317,6 +317,8 @@ public abstract class ExportJobTestCase extends BaseSqoopTestCase {
     try {
       ExportTool exporter = new ExportTool();
       Configuration conf = getConf();
+      //Need to disable OraOop for existing tests
+      conf.set("oraoop.disabled", "true");
       SqoopOptions opts = getSqoopOptions(conf);
       Sqoop sqoop = new Sqoop(exporter, conf, opts);
       ret = Sqoop.runSqoop(sqoop, argv);

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/com/cloudera/sqoop/testutil/ImportJobTestCase.java
----------------------------------------------------------------------
diff --git a/src/test/com/cloudera/sqoop/testutil/ImportJobTestCase.java b/src/test/com/cloudera/sqoop/testutil/ImportJobTestCase.java
index 9fc8eba..d5b7f8f 100644
--- a/src/test/com/cloudera/sqoop/testutil/ImportJobTestCase.java
+++ b/src/test/com/cloudera/sqoop/testutil/ImportJobTestCase.java
@@ -115,6 +115,8 @@ public abstract class ImportJobTestCase extends BaseSqoopTestCase {
     removeTableDir();
 
     Configuration conf = getConf();
+    //Need to disable OraOop for existing tests
+    conf.set("oraoop.disabled", "true");
     SqoopOptions opts = getSqoopOptions(conf);
 
     // run the tool through the normal entry-point.
@@ -210,6 +212,8 @@ public abstract class ImportJobTestCase extends BaseSqoopTestCase {
     int ret;
     try {
       Configuration conf = getConf();
+      //Need to disable OraOop for existing tests
+      conf.set("oraoop.disabled", "true");
       SqoopOptions opts = getSqoopOptions(conf);
       Sqoop sqoop = new Sqoop(tool, conf, opts);
       ret = Sqoop.runSqoop(sqoop, argv);

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/oraoop/create_users.sql
----------------------------------------------------------------------
diff --git a/src/test/oraoop/create_users.sql b/src/test/oraoop/create_users.sql
new file mode 100644
index 0000000..b9ccbb9
--- /dev/null
+++ b/src/test/oraoop/create_users.sql
@@ -0,0 +1,49 @@
+create user sqooptest identified by 12345 default tablespace users;
+alter user sqooptest quota unlimited on users;
+grant create session to sqooptest;
+grant create procedure to sqooptest;
+grant alter session to sqooptest;
+grant select on v_$instance to sqooptest;
+grant select on dba_tables to sqooptest;
+grant select on dba_tab_columns to sqooptest;
+grant select on dba_objects to sqooptest;
+grant select on dba_extents to sqooptest;
+grant select on dba_segments to sqooptest;
+grant select on v_$database to sqooptest;
+grant select on v_$parameter to sqooptest;
+grant select on v_$session to sqooptest;
+grant select on v_$sql to sqooptest;
+grant create table to sqooptest;
+grant select on dba_tab_partitions to sqooptest;
+grant select on dba_tab_subpartitions to sqooptest;
+grant select on dba_indexes to sqooptest;
+grant select on dba_ind_columns to sqooptest;
+grant select any table to sqooptest;
+grant create any table to sqooptest;
+grant insert any table to sqooptest;
+grant alter any table to sqooptest;
+
+create user sqooptest2 identified by ABCDEF default tablespace users;
+alter user sqooptest2 quota unlimited on users;
+grant create session to sqooptest2;
+grant create procedure to sqooptest2;
+grant alter session to sqooptest2;
+grant select on v_$instance to sqooptest2;
+grant select on dba_tables to sqooptest2;
+grant select on dba_tab_columns to sqooptest2;
+grant select on dba_objects to sqooptest2;
+grant select on dba_extents to sqooptest2;
+grant select on dba_segments to sqooptest2;
+grant select on v_$database to sqooptest2;
+grant select on v_$parameter to sqooptest2;
+grant select on v_$session to sqooptest2;
+grant select on v_$sql to sqooptest2;
+grant create table to sqooptest2;
+grant select on dba_tab_partitions to sqooptest2;
+grant select on dba_tab_subpartitions to sqooptest2;
+grant select on dba_indexes to sqooptest2;
+grant select on dba_ind_columns to sqooptest2;
+grant select any table to sqooptest2;
+grant create any table to sqooptest2;
+grant insert any table to sqooptest2;
+grant alter any table to sqooptest2;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/oraoop/pkg_tst_product_gen.pbk
----------------------------------------------------------------------
diff --git a/src/test/oraoop/pkg_tst_product_gen.pbk b/src/test/oraoop/pkg_tst_product_gen.pbk
new file mode 100644
index 0000000..0bc7df7
--- /dev/null
+++ b/src/test/oraoop/pkg_tst_product_gen.pbk
@@ -0,0 +1,126 @@
+CREATE OR REPLACE
+PACKAGE BODY         "PKG_ODG_$TABLE_NAME"
+AS
+
+/**
+ * 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.
+ */
+
+   FUNCTION fnc_gen_data (i_parameters typ_rc_parameters)
+      RETURN typ_oraoop_datagen_tab
+      PIPELINED
+      PARALLEL_ENABLE(PARTITION i_parameters BY ANY)
+   IS
+      l_parameters   typ_parameters;
+      l_ret_rec      typ_oraoop_datagen;
+   BEGIN
+      LOOP
+         FETCH i_parameters INTO l_parameters;
+
+         EXIT WHEN i_parameters%NOTFOUND;
+
+         FOR id IN l_parameters.start_range .. l_parameters.end_range
+         LOOP
+            $DATA_EXPRESSION_LIST
+            PIPE ROW (l_ret_rec);
+         END LOOP;
+      END LOOP;
+
+      RETURN;
+   END fnc_gen_data;
+
+   PROCEDURE prc_create_table
+   IS
+   BEGIN
+      EXECUTE IMMEDIATE '
+         CREATE TABLE "$TABLE_NAME"
+         (
+            $COLUMN_LIST
+         )
+         NOLOGGING
+         PARALLEL
+         $PARTITION_CLAUSE';
+   END;
+
+   PROCEDURE prc_insert_data (i_degree NUMBER)
+   IS
+   BEGIN
+      EXECUTE IMMEDIATE 'ALTER SESSION ENABLE PARALLEL DML';
+
+      EXECUTE IMMEDIATE
+         'INSERT /*+append parallel(' || i_degree || ') */
+               INTO  "$TABLE_NAME"
+            SELECT /*+parallel(' || i_degree || ') */
+                  *
+              FROM TABLE (
+                      "PKG_ODG_$TABLE_NAME".
+                       fnc_gen_data (
+                         CURSOR (SELECT /*+parallel(' || i_degree || ') */
+                                       start_range, end_range FROM "ODGP_$TABLE_NAME")))';
+      COMMIT;
+   END;
+
+   PROCEDURE prc_create_parameters_table (i_degree            NUMBER,
+                                          i_rows_per_slave    NUMBER)
+   IS
+      l_sql           VARCHAR2 (32767) := 'CREATE TABLE "ODGP_$TABLE_NAME"
+                                           (
+                                             START_RANGE NUMBER
+                                           , END_RANGE NUMBER
+                                           )
+                                           PARTITION BY RANGE (START_RANGE)  ';
+      l_start_range   NUMBER;
+      l_end_range     NUMBER;
+   BEGIN
+      FOR i IN 1 .. i_degree
+      LOOP
+         l_sql :=
+               l_sql
+            || CASE WHEN i = 1 THEN '(' ELSE ',' END
+            || ' PARTITION "ODGP_$TABLE_NAME_P'
+            || i
+            || '" VALUES LESS THAN ('
+            || i_rows_per_slave * i
+            || ') NOLOGGING ';
+      END LOOP;
+
+      l_sql := l_sql || ')';
+
+      EXECUTE IMMEDIATE l_sql;
+
+      FOR i IN 1 .. i_degree
+      LOOP
+         l_start_range := (i_rows_per_slave * (i - 1)) + 1;
+         l_end_range := (i_rows_per_slave * i);
+
+         EXECUTE IMMEDIATE 'INSERT INTO "ODGP_$TABLE_NAME" VALUES (:start_range, :end_range)'
+            USING l_start_range, l_end_range;
+      END LOOP;
+
+      COMMIT;
+   END;
+
+   PROCEDURE prc_load_table (i_degree NUMBER, i_rows_per_slave NUMBER)
+   IS
+   BEGIN
+      prc_create_table;
+      prc_create_parameters_table (i_degree, i_rows_per_slave);
+      prc_insert_data (i_degree);
+
+      EXECUTE IMMEDIATE 'DROP TABLE "ODGP_$TABLE_NAME"';
+   END;
+END;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/oraoop/pkg_tst_product_gen.psk
----------------------------------------------------------------------
diff --git a/src/test/oraoop/pkg_tst_product_gen.psk b/src/test/oraoop/pkg_tst_product_gen.psk
new file mode 100644
index 0000000..77052a5
--- /dev/null
+++ b/src/test/oraoop/pkg_tst_product_gen.psk
@@ -0,0 +1,45 @@
+CREATE OR REPLACE
+PACKAGE         "PKG_ODG_$TABLE_NAME"
+AS
+
+/**
+ * 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.
+ */
+
+   TYPE typ_oraoop_datagen IS RECORD
+   (
+      $COLUMN_LIST
+   );
+
+   TYPE typ_oraoop_datagen_tab IS TABLE OF typ_oraoop_datagen;
+
+   TYPE typ_parameters IS RECORD
+   (
+      start_range   NUMBER,
+      end_range     NUMBER
+   );
+
+   TYPE typ_rc_parameters IS REF CURSOR
+      RETURN typ_parameters;
+
+   FUNCTION fnc_gen_data (i_parameters typ_rc_parameters)
+      RETURN typ_oraoop_datagen_tab
+      PIPELINED
+      PARALLEL_ENABLE(PARTITION i_parameters BY ANY);
+
+   PROCEDURE prc_load_table (i_degree NUMBER, i_rows_per_slave NUMBER);
+END;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/oraoop/table_tst_product.xml
----------------------------------------------------------------------
diff --git a/src/test/oraoop/table_tst_product.xml b/src/test/oraoop/table_tst_product.xml
new file mode 100644
index 0000000..76c1a10
--- /dev/null
+++ b/src/test/oraoop/table_tst_product.xml
@@ -0,0 +1,90 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<table>
+  <name>TST_PRODUCT</name>
+  <columns>
+    <column>
+      <name>product_id</name>
+      <dataType>INTEGER</dataType>
+      <dataExpression>id</dataExpression>
+    </column>
+    <column>
+      <name>supplier_code</name>
+      <dataType>VARCHAR2 (30)</dataType>
+      <dataExpression>TO_CHAR (id - MOD (id, 5000),'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')</dataExpression>
+    </column>
+    <column>
+      <name>product_code</name>
+      <dataType>VARCHAR2 (30)</dataType>
+      <dataExpression>TO_CHAR (MOD (id, 100000), 'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')</dataExpression>
+    </column>
+    <column>
+      <name>product_descr</name>
+      <dataType>VARCHAR2 (255)</dataType>
+      <dataExpression>DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 100)))</dataExpression>
+    </column>
+    <column>
+      <name>product_long_descr</name>
+      <dataType>VARCHAR2 (4000)</dataType>
+      <dataExpression>DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 200)))</dataExpression>
+    </column>
+    <column>
+      <name>product_cost_price</name>
+      <dataType>NUMBER</dataType>
+      <dataExpression>ROUND (DBMS_RANDOM.VALUE (0, 100000), 2)</dataExpression>
+    </column>
+    <column>
+      <name>sell_from_date</name>
+      <dataType>DATE</dataType>
+      <dataExpression>TRUNC (SYSDATE + DBMS_RANDOM.VALUE (-365, 365))</dataExpression>
+    </column>
+    <column>
+      <name>sell_price</name>
+      <dataType>NUMBER</dataType>
+      <dataExpression>ROUND (DBMS_RANDOM.VALUE (0, 200000), 2)</dataExpression>
+    </column>
+    <column>
+      <name>create_user</name>
+      <dataType>VARCHAR2 (30)</dataType>
+      <dataExpression>DBMS_RANDOM.string ('U', 30)</dataExpression>
+    </column>
+    <column>
+      <name>create_time</name>
+      <dataType>TIMESTAMP</dataType>
+      <dataExpression>TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')</dataExpression>
+    </column>
+    <column>
+      <name>last_update_user</name>
+      <dataType>VARCHAR2 (30)</dataType>
+      <dataExpression>DBMS_RANDOM.string ('U', 30)</dataExpression>
+    </column>
+    <column>
+      <name>last_update_time</name>
+      <dataType>TIMESTAMP</dataType>
+      <dataExpression>TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')</dataExpression>
+    </column>
+  </columns>
+  <primaryKeyColumns>
+    <primaryKeyColumn>product_id</primaryKeyColumn>
+  </primaryKeyColumns>
+  <uniqueKeyColumns>
+    <uniqueKeyColumn>supplier_code</uniqueKeyColumn>
+    <uniqueKeyColumn>product_code</uniqueKeyColumn>
+  </uniqueKeyColumns>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/oraoop/table_tst_product_part.xml
----------------------------------------------------------------------
diff --git a/src/test/oraoop/table_tst_product_part.xml b/src/test/oraoop/table_tst_product_part.xml
new file mode 100644
index 0000000..6014476
--- /dev/null
+++ b/src/test/oraoop/table_tst_product_part.xml
@@ -0,0 +1,103 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<table>
+  <name>TST_PRODUCT_PART</name>
+  <columns>
+    <column>
+      <name>product_id</name>
+      <dataType>INTEGER</dataType>
+      <dataExpression>id</dataExpression>
+    </column>
+    <column>
+      <name>supplier_code</name>
+      <dataType>VARCHAR2 (30)</dataType>
+      <dataExpression>TO_CHAR (id - MOD (id, 5000),'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')</dataExpression>
+    </column>
+    <column>
+      <name>product_code</name>
+      <dataType>VARCHAR2 (30)</dataType>
+      <dataExpression>TO_CHAR (MOD (id, 100000), 'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')</dataExpression>
+    </column>
+    <column>
+      <name>product_descr</name>
+      <dataType>VARCHAR2 (255)</dataType>
+      <dataExpression>DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 100)))</dataExpression>
+    </column>
+    <column>
+      <name>product_long_descr</name>
+      <dataType>VARCHAR2 (4000)</dataType>
+      <dataExpression>DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 200)))</dataExpression>
+    </column>
+    <column>
+      <name>product_cost_price</name>
+      <dataType>NUMBER</dataType>
+      <dataExpression>ROUND (DBMS_RANDOM.VALUE (0, 100000), 2)</dataExpression>
+    </column>
+    <column>
+      <name>sell_from_date</name>
+      <dataType>DATE</dataType>
+      <dataExpression>TRUNC (SYSDATE + DBMS_RANDOM.VALUE (-365, 365))</dataExpression>
+    </column>
+    <column>
+      <name>sell_price</name>
+      <dataType>NUMBER</dataType>
+      <dataExpression>ROUND (DBMS_RANDOM.VALUE (0, 200000), 2)</dataExpression>
+    </column>
+    <column>
+      <name>create_user</name>
+      <dataType>VARCHAR2 (30)</dataType>
+      <dataExpression>DBMS_RANDOM.string ('U', 30)</dataExpression>
+    </column>
+    <column>
+      <name>create_time</name>
+      <dataType>TIMESTAMP</dataType>
+      <dataExpression>TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')</dataExpression>
+    </column>
+    <column>
+      <name>last_update_user</name>
+      <dataType>VARCHAR2 (30)</dataType>
+      <dataExpression>DBMS_RANDOM.string ('U', 30)</dataExpression>
+    </column>
+    <column>
+      <name>last_update_time</name>
+      <dataType>TIMESTAMP</dataType>
+      <dataExpression>TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')</dataExpression>
+    </column>
+  </columns>
+  <primaryKeyColumns>
+    <primaryKeyColumn>product_id</primaryKeyColumn>
+  </primaryKeyColumns>
+  <uniqueKeyColumns>
+    <uniqueKeyColumn>supplier_code</uniqueKeyColumn>
+    <uniqueKeyColumn>product_code</uniqueKeyColumn>
+  </uniqueKeyColumns>
+  <partitionClause>
+    PARTITION BY RANGE(sell_from_date)
+    (
+      PARTITION tst_product_part_1 values less than (to_date(''' || to_char(add_months(sysdate,-9),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
+      PARTITION tst_product_part_2 values less than (to_date(''' || to_char(add_months(sysdate,-6),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
+      PARTITION tst_product_part_3 values less than (to_date(''' || to_char(add_months(sysdate,-3),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
+      PARTITION tst_product_part_4 values less than (to_date(''' || to_char(sysdate,'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
+      PARTITION tst_product_part_5 values less than (to_date(''' || to_char(add_months(sysdate,3),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
+      PARTITION "tst_product_pa#rt_6" values less than (to_date(''' || to_char(add_months(sysdate,6),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
+      PARTITION "tst_product_part_7" values less than (to_date(''' || to_char(add_months(sysdate,9),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
+      PARTITION tst_product_part_max values less than (MAXVALUE)
+    )
+  </partitionClause>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/oraoop/table_tst_product_special_chars.xml
----------------------------------------------------------------------
diff --git a/src/test/oraoop/table_tst_product_special_chars.xml b/src/test/oraoop/table_tst_product_special_chars.xml
new file mode 100644
index 0000000..721f808
--- /dev/null
+++ b/src/test/oraoop/table_tst_product_special_chars.xml
@@ -0,0 +1,90 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<table>
+  <name>T5+_Pr#duct</name>
+  <columns>
+    <column>
+      <name>product_id</name>
+      <dataType>INTEGER</dataType>
+      <dataExpression>id</dataExpression>
+    </column>
+    <column>
+      <name>supplier_code</name>
+      <dataType>VARCHAR2 (30)</dataType>
+      <dataExpression>TO_CHAR (id - MOD (id, 5000),'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')</dataExpression>
+    </column>
+    <column>
+      <name>product_code</name>
+      <dataType>VARCHAR2 (30)</dataType>
+      <dataExpression>TO_CHAR (MOD (id, 100000), 'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')</dataExpression>
+    </column>
+    <column>
+      <name>product_descr</name>
+      <dataType>VARCHAR2 (255)</dataType>
+      <dataExpression>DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 100)))</dataExpression>
+    </column>
+    <column>
+      <name>product_long_descr</name>
+      <dataType>VARCHAR2 (4000)</dataType>
+      <dataExpression>DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 200)))</dataExpression>
+    </column>
+    <column>
+      <name>product_cost_price</name>
+      <dataType>NUMBER</dataType>
+      <dataExpression>ROUND (DBMS_RANDOM.VALUE (0, 100000), 2)</dataExpression>
+    </column>
+    <column>
+      <name>sell_from_date</name>
+      <dataType>DATE</dataType>
+      <dataExpression>TRUNC (SYSDATE + DBMS_RANDOM.VALUE (-365, 365))</dataExpression>
+    </column>
+    <column>
+      <name>sell_price</name>
+      <dataType>NUMBER</dataType>
+      <dataExpression>ROUND (DBMS_RANDOM.VALUE (0, 200000), 2)</dataExpression>
+    </column>
+    <column>
+      <name>create_user</name>
+      <dataType>VARCHAR2 (30)</dataType>
+      <dataExpression>DBMS_RANDOM.string ('U', 30)</dataExpression>
+    </column>
+    <column>
+      <name>create_time</name>
+      <dataType>TIMESTAMP</dataType>
+      <dataExpression>TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')</dataExpression>
+    </column>
+    <column>
+      <name>last_update_user</name>
+      <dataType>VARCHAR2 (30)</dataType>
+      <dataExpression>DBMS_RANDOM.string ('U', 30)</dataExpression>
+    </column>
+    <column>
+      <name>last_update_time</name>
+      <dataType>TIMESTAMP</dataType>
+      <dataExpression>TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')</dataExpression>
+    </column>
+  </columns>
+  <primaryKeyColumns>
+    <primaryKeyColumn>product_id</primaryKeyColumn>
+  </primaryKeyColumns>
+  <uniqueKeyColumns>
+    <uniqueKeyColumn>supplier_code</uniqueKeyColumn>
+    <uniqueKeyColumn>product_code</uniqueKeyColumn>
+  </uniqueKeyColumns>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/oraoop/table_tst_product_subpart.xml
----------------------------------------------------------------------
diff --git a/src/test/oraoop/table_tst_product_subpart.xml b/src/test/oraoop/table_tst_product_subpart.xml
new file mode 100644
index 0000000..4883244
--- /dev/null
+++ b/src/test/oraoop/table_tst_product_subpart.xml
@@ -0,0 +1,105 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<table>
+  <name>TST_PRODUCT_SUBPART</name>
+  <columns>
+    <column>
+      <name>product_id</name>
+      <dataType>INTEGER</dataType>
+      <dataExpression>id</dataExpression>
+    </column>
+    <column>
+      <name>supplier_code</name>
+      <dataType>VARCHAR2 (30)</dataType>
+      <dataExpression>TO_CHAR (id - MOD (id, 5000),'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')</dataExpression>
+    </column>
+    <column>
+      <name>product_code</name>
+      <dataType>VARCHAR2 (30)</dataType>
+      <dataExpression>TO_CHAR (MOD (id, 100000), 'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')</dataExpression>
+    </column>
+    <column>
+      <name>product_descr</name>
+      <dataType>VARCHAR2 (255)</dataType>
+      <dataExpression>DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 100)))</dataExpression>
+    </column>
+    <column>
+      <name>product_long_descr</name>
+      <dataType>VARCHAR2 (4000)</dataType>
+      <dataExpression>DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 200)))</dataExpression>
+    </column>
+    <column>
+      <name>product_cost_price</name>
+      <dataType>NUMBER</dataType>
+      <dataExpression>ROUND (DBMS_RANDOM.VALUE (0, 100000), 2)</dataExpression>
+    </column>
+    <column>
+      <name>sell_from_date</name>
+      <dataType>DATE</dataType>
+      <dataExpression>TRUNC (SYSDATE + DBMS_RANDOM.VALUE (-365, 365))</dataExpression>
+    </column>
+    <column>
+      <name>sell_price</name>
+      <dataType>NUMBER</dataType>
+      <dataExpression>ROUND (DBMS_RANDOM.VALUE (0, 200000), 2)</dataExpression>
+    </column>
+    <column>
+      <name>create_user</name>
+      <dataType>VARCHAR2 (30)</dataType>
+      <dataExpression>DBMS_RANDOM.string ('U', 30)</dataExpression>
+    </column>
+    <column>
+      <name>create_time</name>
+      <dataType>TIMESTAMP</dataType>
+      <dataExpression>TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')</dataExpression>
+    </column>
+    <column>
+      <name>last_update_user</name>
+      <dataType>VARCHAR2 (30)</dataType>
+      <dataExpression>DBMS_RANDOM.string ('U', 30)</dataExpression>
+    </column>
+    <column>
+      <name>last_update_time</name>
+      <dataType>TIMESTAMP</dataType>
+      <dataExpression>TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')</dataExpression>
+    </column>
+  </columns>
+  <primaryKeyColumns>
+    <primaryKeyColumn>product_id</primaryKeyColumn>
+  </primaryKeyColumns>
+  <uniqueKeyColumns>
+    <uniqueKeyColumn>supplier_code</uniqueKeyColumn>
+    <uniqueKeyColumn>product_code</uniqueKeyColumn>
+  </uniqueKeyColumns>
+  <partitionClause>
+    PARTITION BY RANGE(sell_from_date)
+    SUBPARTITION BY HASH(supplier_code)
+    SUBPARTITIONS 2
+    (
+      PARTITION tst_product_part_1 values less than (to_date(''' || to_char(add_months(sysdate,-9),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
+      PARTITION tst_product_part_2 values less than (to_date(''' || to_char(add_months(sysdate,-6),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
+      PARTITION tst_product_part_3 values less than (to_date(''' || to_char(add_months(sysdate,-3),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
+      PARTITION tst_product_part_4 values less than (to_date(''' || to_char(sysdate,'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
+      PARTITION tst_product_part_5 values less than (to_date(''' || to_char(add_months(sysdate,3),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
+      PARTITION tst_product_part_6 values less than (to_date(''' || to_char(add_months(sysdate,6),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
+      PARTITION tst_product_part_7 values less than (to_date(''' || to_char(add_months(sysdate,9),'DD/MM/YYYY') || ''',''DD/MM/YYYY'')),
+      PARTITION tst_product_part_max values less than (MAXVALUE)
+    )
+  </partitionClause>
+</table>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/ExportTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/ExportTest.java b/src/test/org/apache/sqoop/manager/oracle/ExportTest.java
new file mode 100644
index 0000000..80b6536
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/ExportTest.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.sqoop.manager.oracle;
+
+import junit.framework.Assert;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test exporting data into Oracle.
+ */
+public class ExportTest extends OraOopTestCase {
+
+  private static final ExportTest TEST_CASE = new ExportTest();
+
+  @BeforeClass
+  public static void setUpHdfsData() throws Exception {
+    // Copy the TST_PRODUCT table into HDFS which can be used for the export
+    // tests
+    TEST_CASE.setSqoopTargetDirectory(TEST_CASE.getSqoopTargetDirectory()
+        + "tst_product");
+    TEST_CASE.createTable("table_tst_product.xml");
+
+    int retCode =
+        TEST_CASE.runImport("tst_product", TEST_CASE.getSqoopConf(), false);
+    Assert.assertEquals("Return code should be 0", 0, retCode);
+  }
+
+  @Test
+  public void testProductExport() throws Exception {
+    int retCode =
+        TEST_CASE.runExportFromTemplateTable("tst_product", "tst_product_exp");
+    Assert.assertEquals("Return code should be 0", 0, retCode);
+  }
+
+  @Test
+  public void testProductExportMixedCaseTableName() throws Exception {
+    int retCode =
+        TEST_CASE.runExportFromTemplateTable("tst_product",
+            "\"\"T5+_Pr#duct_Exp\"\"");
+    Assert.assertEquals("Return code should be 0", 0, retCode);
+  }
+
+  @AfterClass
+  public static void cleanUpHdfsData() throws Exception {
+    TEST_CASE.cleanupFolders();
+    TEST_CASE.closeTestEnvConnection();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/ImportTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/ImportTest.java b/src/test/org/apache/sqoop/manager/oracle/ImportTest.java
new file mode 100644
index 0000000..d914e3f
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/ImportTest.java
@@ -0,0 +1,241 @@
+/**
+ * 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.apache.sqoop.manager.oracle.OraOopConstants.
+           OraOopOracleDataChunkMethod;
+import org.junit.Test;
+
+/**
+ * Test import data from Oracle.
+ */
+public class ImportTest extends OraOopTestCase {
+
+  @Test
+  public void testProductImport() throws Exception {
+    setSqoopTargetDirectory(getSqoopTargetDirectory() + "tst_product");
+    createTable("table_tst_product.xml");
+
+    try {
+      int retCode = runImport("tst_product", getSqoopConf(), false);
+      Assert.assertEquals("Return code should be 0", 0, retCode);
+
+    } finally {
+      cleanupFolders();
+      closeTestEnvConnection();
+    }
+  }
+
+  @Test
+  public void testProductPartImport() throws Exception {
+    setSqoopTargetDirectory(getSqoopTargetDirectory() + "tst_product_part");
+    createTable("table_tst_product_part.xml");
+
+    try {
+      int retCode = runImport("tst_product_part", getSqoopConf(), false);
+      Assert.assertEquals("Return code should be 0", 0, retCode);
+
+    } finally {
+      cleanupFolders();
+      closeTestEnvConnection();
+    }
+  }
+
+  @Test
+  public void testProductPartImportPartitionChunk() throws Exception {
+    setSqoopTargetDirectory(getSqoopTargetDirectory() + "tst_product_part");
+    createTable("table_tst_product_part.xml");
+
+    Configuration sqoopConf = getSqoopConf();
+    sqoopConf.set(OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD,
+        OraOopConstants.OraOopOracleDataChunkMethod.PARTITION.toString());
+
+    try {
+      int retCode = runImport("tst_product_part", sqoopConf, false);
+      Assert.assertEquals("Return code should be 0", 0, retCode);
+
+    } finally {
+      cleanupFolders();
+      closeTestEnvConnection();
+    }
+  }
+
+  @Test
+  public void testProductPartImportSubset() throws Exception {
+    setSqoopTargetDirectory(getSqoopTargetDirectory() + "tst_product_part");
+    createTable("table_tst_product_part.xml");
+
+    Configuration sqoopConf = getSqoopConf();
+    sqoopConf.set(OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD,
+        OraOopOracleDataChunkMethod.ROWID.toString());
+    sqoopConf.set(OraOopConstants.ORAOOP_IMPORT_PARTITION_LIST,
+        "tst_product_part_1,tst_product_part_2,\"tst_product_pa#rt_6\"");
+
+    try {
+      int retCode = runImport("tst_product_part", sqoopConf, false);
+      Assert.assertEquals("Return code should be 0", 0, retCode);
+
+    } finally {
+      cleanupFolders();
+      closeTestEnvConnection();
+    }
+  }
+
+  @Test
+  public void testProductPartImportSubsetPartitionChunk() throws Exception {
+    setSqoopTargetDirectory(getSqoopTargetDirectory() + "tst_product_part");
+    createTable("table_tst_product_part.xml");
+
+    Configuration sqoopConf = getSqoopConf();
+    sqoopConf.set(OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD,
+        OraOopOracleDataChunkMethod.PARTITION.toString());
+    sqoopConf
+        .set(
+            OraOopConstants.ORAOOP_IMPORT_PARTITION_LIST,
+            "tst_product_part_1,tst_product_part_2,"
+           +"tst_product_part_3,\"tst_product_pa#rt_6\"");
+
+    try {
+      int retCode = runImport("tst_product_part", sqoopConf, false);
+      Assert.assertEquals("Return code should be 0", 0, retCode);
+
+    } finally {
+      cleanupFolders();
+      closeTestEnvConnection();
+    }
+  }
+
+  @Test
+  public void testProductSubPartImport() throws Exception {
+    setSqoopTargetDirectory(getSqoopTargetDirectory() + "tst_product_subpart");
+    createTable("table_tst_product_subpart.xml");
+
+    try {
+      int retCode = runImport("tst_product_subpart", getSqoopConf(), false);
+      Assert.assertEquals("Return code should be 0", 0, retCode);
+
+    } finally {
+      cleanupFolders();
+      closeTestEnvConnection();
+    }
+  }
+
+  @Test
+  public void testProductSubPartImportPartitionChunk() throws Exception {
+    setSqoopTargetDirectory(getSqoopTargetDirectory() + "tst_product_subpart");
+    createTable("table_tst_product_subpart.xml");
+
+    Configuration sqoopConf = getSqoopConf();
+    sqoopConf.set(OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD,
+        OraOopConstants.OraOopOracleDataChunkMethod.PARTITION.toString());
+
+    try {
+      int retCode = runImport("tst_product_subpart", sqoopConf, false);
+      Assert.assertEquals("Return code should be 0", 0, retCode);
+
+    } finally {
+      cleanupFolders();
+      closeTestEnvConnection();
+    }
+  }
+
+  @Test
+  public void testProductSubPartImportSubset() throws Exception {
+    setSqoopTargetDirectory(getSqoopTargetDirectory() + "tst_product_subpart");
+    createTable("table_tst_product_subpart.xml");
+
+    Configuration sqoopConf = getSqoopConf();
+    sqoopConf.set(OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD,
+        OraOopOracleDataChunkMethod.ROWID.toString());
+    sqoopConf
+        .set(OraOopConstants.ORAOOP_IMPORT_PARTITION_LIST,
+            "TST_PRODUCT_PART_1,TST_PRODUCT_PART_2,"
+           +"TST_PRODUCT_PART_3,TST_PRODUCT_PART_4");
+
+    try {
+      int retCode = runImport("tst_product_subpart", sqoopConf, false);
+      Assert.assertEquals("Return code should be 0", 0, retCode);
+
+    } finally {
+      cleanupFolders();
+      closeTestEnvConnection();
+    }
+  }
+
+  @Test
+  public void testProductSubPartImportSubsetPartitionChunk() throws Exception {
+    setSqoopTargetDirectory(getSqoopTargetDirectory() + "tst_product_subpart");
+    createTable("table_tst_product_subpart.xml");
+
+    Configuration sqoopConf = getSqoopConf();
+    sqoopConf.set(OraOopConstants.ORAOOP_ORACLE_DATA_CHUNK_METHOD,
+        OraOopConstants.OraOopOracleDataChunkMethod.PARTITION.toString());
+    sqoopConf.set(OraOopConstants.ORAOOP_IMPORT_PARTITION_LIST,
+        "TST_PRODUCT_PART_1,TST_PRODUCT_PART_2,TST_PRODUCT_PART_3");
+
+    try {
+      int retCode = runImport("tst_product_subpart", sqoopConf, false);
+      Assert.assertEquals("Return code should be 0", 0, retCode);
+
+    } finally {
+      cleanupFolders();
+      closeTestEnvConnection();
+    }
+  }
+
+  @Test
+  public void testProductImportConsistentRead() throws Exception {
+    setSqoopTargetDirectory(getSqoopTargetDirectory() + "tst_product");
+    createTable("table_tst_product.xml");
+
+    // Make sure Oracle SCN has updated since creating table
+    Thread.sleep(10000);
+
+    Configuration sqoopConf = getSqoopConf();
+    sqoopConf.setBoolean(OraOopConstants.ORAOOP_IMPORT_CONSISTENT_READ, true);
+
+    try {
+      int retCode = runImport("tst_product", sqoopConf, false);
+      Assert.assertEquals("Return code should be 0", 0, retCode);
+
+    } finally {
+      cleanupFolders();
+      closeTestEnvConnection();
+    }
+  }
+
+  @Test
+  public void testProductImportMixedCaseTableName() throws Exception {
+    setSqoopTargetDirectory(getSqoopTargetDirectory() + "T5+_Pr#duct");
+    createTable("table_tst_product_special_chars.xml");
+
+    try {
+      int retCode = runImport("\"\"T5+_Pr#duct\"\"", getSqoopConf(), 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/OraOopOracleQueriesTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/OraOopOracleQueriesTest.java b/src/test/org/apache/sqoop/manager/oracle/OraOopOracleQueriesTest.java
new file mode 100644
index 0000000..09314da
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/OraOopOracleQueriesTest.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;
+
+import org.junit.Assert;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+
+import org.junit.Test;
+
+import com.cloudera.sqoop.manager.OracleUtils;
+
+/**
+ * Test Oracle queries against Oracle database.
+ */
+public class OraOopOracleQueriesTest extends OraOopTestCase {
+
+  @Test
+  public void testGetCurrentSchema() throws Exception {
+    Connection conn = getTestEnvConnection();
+    try {
+      String schema = OraOopOracleQueries.getCurrentSchema(conn);
+      Assert.assertEquals(OracleUtils.ORACLE_USER_NAME.toUpperCase(), schema
+          .toUpperCase());
+
+      PreparedStatement stmt =
+          conn.prepareStatement("ALTER SESSION SET CURRENT_SCHEMA=SYS");
+      stmt.execute();
+
+      schema = OraOopOracleQueries.getCurrentSchema(conn);
+      Assert.assertEquals("SYS", schema);
+    } finally {
+      closeTestEnvConnection();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/OraOopTestCase.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/OraOopTestCase.java b/src/test/org/apache/sqoop/manager/oracle/OraOopTestCase.java
new file mode 100644
index 0000000..6f7d070
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/OraOopTestCase.java
@@ -0,0 +1,321 @@
+/**
+ * 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.StringWriter;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.Layout;
+import org.apache.log4j.Logger;
+import org.apache.log4j.PatternLayout;
+import org.apache.log4j.WriterAppender;
+
+import com.cloudera.sqoop.Sqoop;
+import com.cloudera.sqoop.manager.OracleUtils;
+
+import org.apache.sqoop.manager.oracle.util.HadoopFiles;
+import org.apache.sqoop.manager.oracle.util.OracleData;
+
+/**
+ * Base test case for OraOop to handle common functions.
+ */
+public abstract class OraOopTestCase {
+
+  private static final OraOopLog LOG = OraOopLogFactory.getLog(
+      OraOopTestCase.class.getName());
+
+  private String sqoopGenLibDirectory = System.getProperty("user.dir")
+      + "/target/tmp/lib";
+  private String sqoopGenSrcDirectory = System.getProperty("user.dir")
+      + "/target/tmp/src";
+  private String sqoopTargetDirectory = "target/tmp/";
+  private String sqoopGenClassName = "org.apache.sqoop.gen.OraOopTestClass";
+
+  private Connection conn;
+
+  protected ClassLoader classLoader;
+  {
+    classLoader = Thread.currentThread().getContextClassLoader();
+    if (classLoader == null) {
+      classLoader = OraOopTestCase.class.getClassLoader();
+    }
+  }
+
+  static {
+    Configuration
+        .addDefaultResource(OraOopConstants.ORAOOP_SITE_TEMPLATE_FILENAME);
+    Configuration.addDefaultResource(OraOopConstants.ORAOOP_SITE_FILENAME);
+  }
+
+  protected String getSqoopTargetDirectory() {
+    return sqoopTargetDirectory;
+  }
+
+  protected void setSqoopTargetDirectory(String newSqoopTargetDirectory) {
+    this.sqoopTargetDirectory = newSqoopTargetDirectory;
+  }
+
+  protected String getSqoopGenLibDirectory() {
+    return sqoopGenLibDirectory;
+  }
+
+  protected String getSqoopGenSrcDirectory() {
+    return sqoopGenSrcDirectory;
+  }
+
+  protected String getSqoopGenClassName() {
+    return sqoopGenClassName;
+  }
+
+  protected Connection getTestEnvConnection() throws SQLException {
+    if (this.conn == null) {
+      this.conn =
+          DriverManager.getConnection(OracleUtils.CONNECT_STRING,
+              OracleUtils.ORACLE_USER_NAME, OracleUtils.ORACLE_USER_PASS);
+    }
+    return this.conn;
+  }
+
+  protected void closeTestEnvConnection() {
+    try {
+      if (this.conn != null) {
+        this.conn.close();
+      }
+    } catch (SQLException e) {
+      // Tried to close connection but failed - continue anyway
+    }
+    this.conn = null;
+  }
+
+  protected void createTable(String fileName) {
+    try {
+      Connection localConn = getTestEnvConnection();
+      int parallelProcesses = OracleData.getParallelProcesses(localConn);
+      int rowsPerSlave =
+          OracleUtils.INTEGRATIONTEST_NUM_ROWS / parallelProcesses;
+      try {
+        long startTime = System.currentTimeMillis();
+        OracleData.createTable(localConn, fileName, parallelProcesses,
+            rowsPerSlave);
+        LOG.debug("Created and loaded table in "
+            + ((System.currentTimeMillis() - startTime) / 1000) + " seconds.");
+      } catch (SQLException e) {
+        if (e.getErrorCode() == 955) {
+          LOG.debug("Table already exists - using existing data");
+        } else {
+          throw new RuntimeException(e);
+        }
+      }
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  protected int countTable(String inputTableName, List<String> partitionList) {
+    String tableName = inputTableName;
+    if (tableName.startsWith("\"\"") && tableName.endsWith("\"\"")) {
+      // Table names may be double quoted to work around Sqoop issue
+      tableName = tableName.substring(1, tableName.length() - 1);
+    }
+    String sql = null;
+    int numRows = 0;
+    if (partitionList != null && partitionList.size() > 0) {
+      sql = "SELECT sum(cnt) FROM (";
+      int i = 0;
+      for (String partition : partitionList) {
+        i++;
+        if (i > 1) {
+          sql += " UNION ALL ";
+        }
+        sql +=
+            "SELECT count(*) cnt FROM " + tableName + " PARTITION(\""
+                + partition + "\")";
+      }
+      sql += ")";
+    } else {
+      sql = "SELECT count(*) FROM " + tableName;
+    }
+    try {
+      PreparedStatement stmt =
+          this.getTestEnvConnection().prepareStatement(sql);
+      stmt.execute();
+      ResultSet results = stmt.getResultSet();
+      results.next();
+      numRows = results.getInt(1);
+    } catch (SQLException e) {
+      throw new RuntimeException("Could not count number of rows in table "
+          + tableName, e);
+    }
+    return numRows;
+  }
+
+  protected Configuration getSqoopConf() {
+    Configuration sqoopConf = new Configuration();
+    return sqoopConf;
+  }
+
+  protected int runImport(String tableName, Configuration sqoopConf,
+      boolean sequenceFile) {
+    Logger rootLogger = Logger.getRootLogger();
+    rootLogger.removeAllAppenders();
+    StringWriter stringWriter = new StringWriter();
+    Layout layout = new PatternLayout("%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n");
+    WriterAppender writerAppender = new WriterAppender(layout, stringWriter);
+    rootLogger.addAppender(writerAppender);
+
+    List<String> sqoopArgs = new ArrayList<String>();
+
+    sqoopArgs.add("import");
+
+    if (sequenceFile) {
+      sqoopArgs.add("--as-sequencefile");
+    }
+
+    sqoopArgs.add("--connect");
+    sqoopArgs.add(OracleUtils.CONNECT_STRING);
+
+    sqoopArgs.add("--username");
+    sqoopArgs.add(OracleUtils.ORACLE_USER_NAME);
+
+    sqoopArgs.add("--password");
+    sqoopArgs.add(OracleUtils.ORACLE_USER_PASS);
+
+    sqoopArgs.add("--table");
+    sqoopArgs.add(tableName);
+
+    sqoopArgs.add("--target-dir");
+    sqoopArgs.add(this.sqoopTargetDirectory);
+
+    sqoopArgs.add("--class-name");
+    sqoopArgs.add(getSqoopGenClassName());
+
+    sqoopArgs.add("--bindir");
+    sqoopArgs.add(this.sqoopGenLibDirectory);
+
+    sqoopArgs.add("--outdir");
+    sqoopArgs.add(this.sqoopGenSrcDirectory);
+
+    if (OracleUtils.NUM_MAPPERS != 0) {
+      sqoopArgs.add("--num-mappers");
+      sqoopArgs.add(Integer.toString(OracleUtils.NUM_MAPPERS));
+    }
+
+    int rowsInTable =
+        countTable(tableName, OraOopUtilities.splitOracleStringList(sqoopConf
+            .get(OraOopConstants.ORAOOP_IMPORT_PARTITION_LIST)));
+
+    int retCode =
+        Sqoop.runTool(sqoopArgs.toArray(new String[sqoopArgs.size()]),
+            sqoopConf);
+    int rowsImported = 0;
+    if (retCode == 0) {
+      String logString = stringWriter.toString();
+      Pattern pattern =
+          Pattern.compile(
+              "(INFO mapreduce.ImportJobBase: Retrieved )([0-9]+)( records.)");
+      Matcher matcher = pattern.matcher(logString);
+      while (matcher.find()) {
+        rowsImported = Integer.parseInt(matcher.group(2));
+      }
+    }
+    if (retCode != 0 || rowsInTable != rowsImported) {
+      System.out.println(stringWriter.toString());
+    }
+    Assert.assertEquals("Incorrect number of rows imported", rowsInTable,
+        rowsImported);
+    return retCode;
+  }
+
+  protected int runExportFromTemplateTable(String templateTableName,
+      String tableName) {
+    List<String> sqoopArgs = new ArrayList<String>();
+
+    sqoopArgs.add("export");
+
+    sqoopArgs.add("--connect");
+    sqoopArgs.add(OracleUtils.CONNECT_STRING);
+
+    sqoopArgs.add("--username");
+    sqoopArgs.add(OracleUtils.ORACLE_USER_NAME);
+
+    sqoopArgs.add("--password");
+    sqoopArgs.add(OracleUtils.ORACLE_USER_PASS);
+
+    sqoopArgs.add("--table");
+    sqoopArgs.add(tableName);
+
+    sqoopArgs.add("--export-dir");
+    sqoopArgs.add(this.sqoopTargetDirectory);
+
+    sqoopArgs.add("--class-name");
+    sqoopArgs.add(getSqoopGenClassName());
+
+    sqoopArgs.add("--bindir");
+    sqoopArgs.add(this.sqoopGenLibDirectory);
+
+    sqoopArgs.add("--outdir");
+    sqoopArgs.add(this.sqoopGenSrcDirectory);
+
+    Configuration sqoopConf = getSqoopConf();
+
+    sqoopConf.set("oraoop.template.table", templateTableName);
+    sqoopConf.setBoolean("oraoop.drop.table", true);
+    sqoopConf.setBoolean("oraoop.nologging", true);
+    sqoopConf.setBoolean("oraoop.partitioned", false);
+
+    return Sqoop.runTool(sqoopArgs.toArray(new String[sqoopArgs.size()]),
+        sqoopConf);
+  }
+
+  protected int runCompareTables(Connection connection, String table1,
+      String table2) throws SQLException {
+    PreparedStatement stmt;
+    stmt = connection.prepareStatement(
+        "select count(*) from (select * from (select * from "
+                + table1
+                + " minus select * from "
+                + table2
+                + ") union all select * from (select * from "
+                + table2
+                + " minus select * from " + table1 + "))");
+    ResultSet results = stmt.executeQuery();
+    results.next();
+    int numDifferences = results.getInt(1);
+    return numDifferences;
+  }
+
+  protected void cleanupFolders() throws Exception {
+    HadoopFiles.delete(new Path(getSqoopTargetDirectory()), true);
+    HadoopFiles.delete(new Path(getSqoopGenSrcDirectory()), true);
+    HadoopFiles.delete(new Path(getSqoopGenLibDirectory()), true);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/OraOopTestConstants.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/OraOopTestConstants.java b/src/test/org/apache/sqoop/manager/oracle/OraOopTestConstants.java
new file mode 100644
index 0000000..b2634f3
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/OraOopTestConstants.java
@@ -0,0 +1,62 @@
+/**
+ * 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;
+
+/**
+ * Constants for testing OraOop.
+ */
+public final class OraOopTestConstants {
+  private OraOopTestConstants() {
+  }
+
+  public static final String SQL_TABLE =
+      "WITH sqltable AS "
+    + "    ( "
+    + "       SELECT executions, rows_processed, fetches, "
+    + "              ROUND (rows_processed / executions, 2) AS rows_per_exec, "
+    + "              ROUND (rows_processed / fetches, 2) AS rows_per_fetch, "
+    + "              ROUND (LEAST (  ROUND (rows_processed / fetches, 2) "
+    + "                            / LEAST (rows_processed / executions, 10), "
+    + "                            1 "
+    + "                           ), "
+    + "                     2 "
+    + "                    ) batch_efficiency, "
+    + "              sql_text, u.username parsing_schema_name, buffer_gets, "
+    + "              disk_reads, cpu_time/1000 cpu_time, elapsed_time/1000"
+    + "               elapsed_time, hash_value sql_id, child_number "
+    + "         FROM v$sql s join all_users u on (u.user_id=s.parsing_user_id) "
+    + "        WHERE fetches > 0 AND executions > 0 AND rows_processed > 0 "
+    + "          AND parsing_schema_id <> 0 AND sql_text like "
+    + "                                                 'select%dba_objects' )"
+    + "SELECT   sql_id, child_number, array_wastage, "
+    + "         rows_processed, fetches, rows_per_exec, "
+    + "        rows_per_fetch, parsing_schema_name, buffer_gets, disk_reads, "
+    + "        cpu_time, elapsed_time, sql_text,executions "
+    + "   FROM (SELECT sql_id, "
+    + "                child_number, "
+    + "                rows_processed * (1 - batch_efficiency) array_wastage, "
+    + "                rows_processed, " + "                fetches, "
+    + "                rows_per_exec, "
+    + "                rows_per_fetch, " + "                sql_text, "
+    + "                parsing_schema_name, "
+    + "                buffer_gets, " + "                disk_reads, "
+    + "                cpu_time, " + "                elapsed_time, "
+    + "                executions " + "           FROM sqltable) ";
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/6bfaa9d6/src/test/org/apache/sqoop/manager/oracle/OracleConnectionFactoryTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/OracleConnectionFactoryTest.java b/src/test/org/apache/sqoop/manager/oracle/OracleConnectionFactoryTest.java
new file mode 100644
index 0000000..9e6931b
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/OracleConnectionFactoryTest.java
@@ -0,0 +1,520 @@
+/**
+ * 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.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.cloudera.sqoop.manager.OracleUtils;
+
+/**
+ * Test OracleConnectionFactory class including initialization statements.
+ */
+public class OracleConnectionFactoryTest extends OraOopTestCase {
+
+  /**
+   * This is just to expose methods in OracleConnectionFactory.
+   */
+  public class Exposer extends OracleConnectionFactory {
+
+  }
+
+  @Test
+  public void testSetJdbcFetchSize() {
+    setAndCheckJdbcFetchSize(45);
+    setAndCheckJdbcFetchSize(2000);
+  }
+
+  private void setAndCheckJdbcFetchSize(int jdbcFetchSize) {
+
+    try {
+      Connection conn = getConnection();
+
+      String uniqueJunk =
+          (new SimpleDateFormat("yyyyMMddHHmmsszzz")).format(new Date())
+              + jdbcFetchSize;
+
+      org.apache.hadoop.conf.Configuration conf = new Configuration();
+      conf.setInt(OraOopConstants.ORACLE_ROW_FETCH_SIZE, jdbcFetchSize);
+
+      // 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 =
+          OracleConnectionFactory.LOG.getCacheLogEntries();
+      OracleConnectionFactory.LOG.setCacheLogEntries(true);
+
+      OraOopOracleQueries.setJdbcFetchSize(conn, conf);
+
+      OracleConnectionFactory.LOG.setCacheLogEntries(logIsBeingCached);
+
+      String uniqueSql =
+          String.format("select /*%s*/ * from dba_objects", uniqueJunk);
+      // Usually dba_objects will have a lot of rows
+      ResultSet resultSet1 = conn.createStatement().executeQuery(uniqueSql);
+      while (resultSet1.next()) {
+        // Nothing to do
+        continue;
+      }
+
+      ResultSet resultSet2 =
+          conn.createStatement().executeQuery(OraOopTestConstants.SQL_TABLE);
+      boolean sqlFound = false;
+      double rowsPerFetch = 0;
+      while (resultSet2.next()) {
+        String sqlText = resultSet2.getString("SQL_TEXT");
+        if (sqlText.contains(uniqueJunk)) {
+          sqlFound = true;
+          rowsPerFetch = resultSet2.getDouble("ROWS_PER_FETCH");
+          break;
+        }
+      }
+
+      if (!sqlFound) {
+        Assert
+            .fail("Unable to find the performance metrics for the SQL "
+                + "statement being used to check the JDBC fetch size.");
+      }
+
+      if (rowsPerFetch < jdbcFetchSize * 0.95
+          || rowsPerFetch > jdbcFetchSize * 1.05) {
+        Assert
+            .fail(String
+                .format(
+                    "The measured JDBC fetch size is not within 5%% of what we "
+                    + "expected. Expected=%s rows/fetch, actual=%s rows/fetch",
+                    jdbcFetchSize, rowsPerFetch));
+      }
+
+    } catch (SQLException ex) {
+      Assert.fail(ex.getMessage());
+    }
+  }
+
+  @Test
+  public void testCreateOracleJdbcConnectionBadUserName() {
+
+    try {
+
+      // Prevent createOracleJdbcConnection() from logging a problem with the
+      // bad username we're about to use. Otherwise, the junit output will be
+      // polluted with messages about things that aren't actually a problem...
+      boolean logIsBeingCached =
+          OracleConnectionFactory.LOG.getCacheLogEntries();
+      OracleConnectionFactory.LOG.setCacheLogEntries(true);
+
+      OracleConnectionFactory.createOracleJdbcConnection(
+          OraOopConstants.ORACLE_JDBC_DRIVER_CLASS, OracleUtils.CONNECT_STRING,
+          OracleUtils.ORACLE_INVALID_USER_NAME, OracleUtils.ORACLE_USER_PASS);
+
+      OracleConnectionFactory.LOG.setCacheLogEntries(logIsBeingCached);
+
+      Assert
+          .fail("OracleConnectionFactory should have thrown an exception in "
+              + "response to a rubbish user name.");
+
+    } catch (SQLException ex) {
+      assertEquals(ex.getErrorCode(), 1017); // <- ORA-01017 invalid
+                                             // username/password; logon denied.
+    }
+  }
+
+  @Test
+  public void testCreateOracleJdbcConnectionBadPassword() {
+
+    try {
+      // Prevent createOracleJdbcConnection() from logging a problem with the
+      // bad username we're about to use. Otherwise, the junit output will be
+      // polluted with messages about things that aren't actually a problem...
+      boolean logIsBeingCached =
+          OracleConnectionFactory.LOG.getCacheLogEntries();
+      OracleConnectionFactory.LOG.setCacheLogEntries(true);
+
+      OracleConnectionFactory.createOracleJdbcConnection(
+          OraOopConstants.ORACLE_JDBC_DRIVER_CLASS, OracleUtils.CONNECT_STRING,
+          OracleUtils.ORACLE_USER_NAME, "a" + OracleUtils.ORACLE_USER_PASS);
+
+      OracleConnectionFactory.LOG.setCacheLogEntries(logIsBeingCached);
+
+      Assert
+          .fail("OracleConnectionFactory should have thrown an exception in "
+              + "response to a rubbish password.");
+
+    } catch (SQLException ex) {
+      assertEquals(ex.getErrorCode(), 1017); // <- ORA-01017 invalid
+                                             // username/password; logon denied.
+    }
+  }
+
+  @Test
+  public void testCreateOracleJdbcConnectionOk() {
+
+    try {
+      Connection conn = getConnection();
+
+      assertEquals(
+          "The connection to the Oracle database does not appear to be valid.",
+          true, conn.isValid(15));
+
+      ResultSet resultSet =
+          conn.createStatement().executeQuery(
+              "select instance_name from v$instance");
+      if (!resultSet.next() || resultSet.getString(1).isEmpty()) {
+        Assert.fail("Got blank instance name from v$instance");
+      }
+    } catch (SQLException ex) {
+      Assert.fail(ex.getMessage());
+    }
+  }
+
+  @Test
+  public void testExecuteOraOopSessionInitializationStatements() {
+
+    // Exposer.LOG = null;
+    // protected static final Log LOG =
+    // LogFactory.getLog(OracleConnectionFactory.class.getName());
+
+    OraOopLogFactory.OraOopLog2 oraoopLog = Exposer.LOG;
+
+    oraoopLog.setCacheLogEntries(true);
+
+    // Check that the default session-initialization statements are reflected in
+    // the log...
+    oraoopLog.clearCache();
+    checkExecuteOraOopSessionInitializationStatements(null);
+    checkLogContainsText(oraoopLog,
+        "Initializing Oracle session with SQL : alter session disable "
+        + "parallel query");
+    checkLogContainsText(
+        oraoopLog,
+        "Initializing Oracle session with SQL : alter session set "
+        + "\"_serial_direct_read\"=true");
+
+    // Check that the absence of session-initialization statements is reflected
+    // in the log...
+    oraoopLog.clearCache();
+    checkExecuteOraOopSessionInitializationStatements("");
+    checkLogContainsText(oraoopLog,
+        "No Oracle 'session initialization' statements were found to execute");
+
+    // This should do nothing (i.e. not throw an exception)...
+    checkExecuteOraOopSessionInitializationStatements(";");
+
+    // This should throw an exception, as Oracle won't know what to do with
+    // this...
+    oraoopLog.clearCache();
+    checkExecuteOraOopSessionInitializationStatements("loremipsum");
+    checkLogContainsText(oraoopLog, "loremipsum");
+    checkLogContainsText(oraoopLog, "ORA-00900: invalid SQL statement");
+
+    Connection conn = getConnection();
+    try {
+
+      // Try a session-initialization statement that creates a table...
+      dropTable(conn, OracleUtils.SYSTEMTEST_TABLE_NAME);
+      checkExecuteOraOopSessionInitializationStatements("create table "
+          + OracleUtils.SYSTEMTEST_TABLE_NAME + " (col1 varchar2(1))");
+      if (!doesTableExist(conn, OracleUtils.SYSTEMTEST_TABLE_NAME)) {
+        Assert.fail("The session-initialization statement to create the table "
+            + OracleUtils.SYSTEMTEST_TABLE_NAME + " did not work.");
+      }
+
+      // Try a sequence of a few statements...
+      dropTable(conn, OracleUtils.SYSTEMTEST_TABLE_NAME);
+      checkExecuteOraOopSessionInitializationStatements("create table "
+          + OracleUtils.SYSTEMTEST_TABLE_NAME + " (col1 number);insert into "
+          + OracleUtils.SYSTEMTEST_TABLE_NAME + " values (1) ; --update "
+          + OracleUtils.SYSTEMTEST_TABLE_NAME + " set col1 = col1 + 1; update "
+          + OracleUtils.SYSTEMTEST_TABLE_NAME
+          + " set col1 = col1 + 1; commit ;;");
+
+      ResultSet resultSet =
+          conn.createStatement().executeQuery(
+              "select col1 from " + OracleUtils.SYSTEMTEST_TABLE_NAME);
+      resultSet.next();
+      int actualValue = resultSet.getInt("col1");
+      if (actualValue != 2) {
+        Assert.fail("The table " + OracleUtils.SYSTEMTEST_TABLE_NAME
+            + " does not contain the data we expected.");
+      }
+
+      dropTable(conn, OracleUtils.SYSTEMTEST_TABLE_NAME);
+
+    } catch (Exception ex) {
+      Assert.fail(ex.getMessage());
+    }
+  }
+
+  @Test
+  public void testParseOraOopSessionInitializationStatements() {
+
+    List<String> statements = null;
+
+    try {
+      statements =
+          OracleConnectionFactory
+              .parseOraOopSessionInitializationStatements(null);
+      Assert.fail("An IllegalArgumentException should have been thrown.");
+    } catch (IllegalArgumentException ex) {
+      // This is what we wanted.
+    }
+
+    org.apache.hadoop.conf.Configuration conf = new Configuration();
+
+    statements =
+        OracleConnectionFactory
+            .parseOraOopSessionInitializationStatements(conf);
+    Assert.assertTrue(statements.size() > 0);
+
+    conf.set(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS, "");
+    statements =
+        OracleConnectionFactory
+            .parseOraOopSessionInitializationStatements(conf);
+    Assert.assertEquals(0, statements.size());
+
+    conf.set(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS, ";");
+    statements =
+        OracleConnectionFactory
+            .parseOraOopSessionInitializationStatements(conf);
+    Assert.assertEquals(0, statements.size());
+
+    conf.set(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS,
+        ";--;\t--");
+    statements =
+        OracleConnectionFactory
+            .parseOraOopSessionInitializationStatements(conf);
+    Assert.assertEquals(0, statements.size());
+
+    conf.set(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS
+        , "\ta");
+    statements =
+        OracleConnectionFactory
+            .parseOraOopSessionInitializationStatements(conf);
+    Assert.assertEquals(1, statements.size());
+    if (!statements.get(0).equalsIgnoreCase("a")) {
+      Assert.fail("Expected a session initialization statement of \"a\"");
+    }
+
+    conf.set(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS,
+        "a;b;--c;d;");
+    statements =
+        OracleConnectionFactory
+            .parseOraOopSessionInitializationStatements(conf);
+    Assert.assertEquals(3, statements.size());
+    if (!statements.get(0).equalsIgnoreCase("a")) {
+      Assert.fail("Expected a session initialization statement of \"a\"");
+    }
+    if (!statements.get(1).equalsIgnoreCase("b")) {
+      Assert.fail("Expected a session initialization statement of \"b\"");
+    }
+    if (!statements.get(2).equalsIgnoreCase("d")) {
+      Assert.fail("Expected a session initialization statement of \"d\"");
+    }
+
+    // Expressions without default values...
+    conf.set(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS,
+        "set a={expr1};b={expr2}/{expr3};");
+    conf.set("expr1", "1");
+    conf.set("expr2", "2");
+    conf.set("expr3", "3");
+    statements =
+        OracleConnectionFactory
+            .parseOraOopSessionInitializationStatements(conf);
+    Assert.assertEquals(2, statements.size());
+    String actual = statements.get(0);
+    String expected = "set a=1";
+    if (!actual.equalsIgnoreCase(expected)) {
+      Assert.fail(String.format(
+        "Expected a session initialization statement of \"%s\", but got \"%s\"."
+                  , expected, actual));
+    }
+    actual = statements.get(1);
+    expected = "b=2/3";
+    if (!actual.equalsIgnoreCase(expected)) {
+      Assert.fail(String.format(
+        "Expected a session initialization statement of \"%s\", but got \"%s\"."
+                  , expected, actual));
+    }
+
+    // Expressions with default values...
+    conf.set(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS,
+        "set c={expr3|66};d={expr4|15}/{expr5|90};");
+    conf.set("expr3", "20");
+    // conf.set("expr4", "21");
+    // conf.set("expr5", "23");
+    statements =
+        OracleConnectionFactory
+            .parseOraOopSessionInitializationStatements(conf);
+    Assert.assertEquals(2, statements.size());
+    actual = statements.get(0);
+    expected = "set c=20";
+    if (!actual.equalsIgnoreCase(expected)) {
+      Assert.fail(String.format(
+        "Expected a session initialization statement of \"%s\", but got \"%s\"."
+                  , expected, actual));
+    }
+    actual = statements.get(1);
+    expected = "d=15/90";
+    if (!actual.equalsIgnoreCase(expected)) {
+      Assert.fail(String.format(
+        "Expected a session initialization statement of \"%s\", but got \"%s\"."
+                  , expected, actual));
+    }
+
+  }
+
+  private void dropTable(Connection conn, String tableName) {
+
+    try {
+      conn.createStatement().executeQuery("drop table " + tableName);
+
+      if (doesTableExist(conn, tableName)) {
+        Assert.fail("Unable to drop the table " + tableName);
+      }
+    } catch (SQLException ex) {
+      if (ex.getErrorCode() != 942) { // <- Table or view does not exist
+        Assert.fail(ex.getMessage());
+      }
+    }
+  }
+
+  private boolean doesTableExist(Connection conn, String tableName) {
+
+    boolean result = false;
+    try {
+      List<OracleTable> tables = OraOopOracleQueries.getTables(conn);
+
+      for (int idx = 0; idx < tables.size(); idx++) {
+        if (tables.get(idx).getName().equalsIgnoreCase(tableName)) {
+          result = true;
+          break;
+        }
+      }
+    } catch (SQLException ex) {
+      Assert.fail(ex.getMessage());
+    }
+    return result;
+  }
+
+  private void checkLogContainsText(OraOopLogFactory.OraOopLog2 oraoopLog,
+      String text) {
+
+    if (!oraoopLog.getLogEntries().toLowerCase().contains(text.toLowerCase())) {
+      Assert.fail(
+          "The LOG does not contain the following text (when it should):\n\t"
+              + text);
+    }
+  }
+
+  private void checkExecuteOraOopSessionInitializationStatements(
+      String statements) {
+
+    Connection conn = getConnection();
+
+    org.apache.hadoop.conf.Configuration conf = new Configuration();
+    if (statements != null) {
+      conf.set(OraOopConstants.ORAOOP_SESSION_INITIALIZATION_STATEMENTS,
+          statements);
+    }
+
+    Exposer.executeOraOopSessionInitializationStatements(conn, conf);
+  }
+
+  @Test
+  public void testSetSessionClientInfo() {
+
+    Connection conn = getConnection();
+
+    org.apache.hadoop.conf.Configuration conf = new Configuration();
+
+    String moduleName = OraOopConstants.ORACLE_SESSION_MODULE_NAME;
+    String actionName =
+        (new SimpleDateFormat("yyyyMMddHHmmsszzz")).format(new Date());
+
+    conf.set(OraOopConstants.ORACLE_SESSION_ACTION_NAME, actionName);
+
+    try {
+      PreparedStatement statement =
+          conn.prepareStatement("select process, module, action "
+              + "from v$session " + "where module = ? and action = ?");
+      statement.setString(1, moduleName);
+      statement.setString(2, actionName);
+
+      // Check no session have this action name - because we haven't applied to
+      // our session yet...
+      ResultSet resultSet = statement.executeQuery();
+      if (resultSet.next()) {
+        Assert
+            .fail("There should be no Oracle sessions with an action name of "
+                + actionName);
+      }
+
+      // Apply this action name to our session...
+      OracleConnectionFactory.setSessionClientInfo(conn, conf);
+
+      // Now check there is a session with our action name...
+      int sessionFoundCount = 0;
+      resultSet = statement.executeQuery();
+      while (resultSet.next()) {
+        sessionFoundCount++;
+      }
+
+      if (sessionFoundCount < 1) {
+        Assert
+            .fail("Unable to locate an Oracle session with the expected module "
+                + "and action.");
+      }
+
+      if (sessionFoundCount > 1) {
+        Assert
+            .fail("Multiple sessions were found with the expected module and "
+                + "action - we only expected to find one.");
+      }
+    } catch (SQLException ex) {
+      Assert.fail(ex.getMessage());
+    }
+
+  }
+
+  private Connection getConnection() {
+
+    try {
+      return OracleConnectionFactory.createOracleJdbcConnection(
+          OraOopConstants.ORACLE_JDBC_DRIVER_CLASS, OracleUtils.CONNECT_STRING,
+          OracleUtils.ORACLE_USER_NAME, OracleUtils.ORACLE_USER_PASS);
+    } catch (SQLException ex) {
+      Assert.fail(ex.getMessage());
+    }
+    return null;
+  }
+
+}