diff --git a/COMPILING.txt b/COMPILING.txt
index ddbed30a..ae284119 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
diff --git a/build.xml b/build.xml
index 2dc99a87..ec5d2fad 100644
--- a/build.xml
+++ b/build.xml
@@ -827,6 +827,11 @@
+
+
+
+
+
+
+
+
+
+
+
+
+ oraoop.oracle.session.initialization.statements
+ 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';
+
+ 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.
+
+
+
+
+ mapred.map.tasks.speculative.execution
+ false
+ Speculative execution is disabled to prevent redundant load on the Oracle database.
+
+
+
+
+ oraoop.import.hint
+ NO_INDEX(t)
+ 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.
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/src/java/org/apache/sqoop/ConnFactory.java b/src/java/org/apache/sqoop/ConnFactory.java
index 61d33075..22765250 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.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 ConnFactory(Configuration conf) {
// 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 ConnFactory(Configuration conf) {
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 {
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 00000000..302849cc
--- /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 columnNamesInOracleTable = null;
+ private Map 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 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 getSelectedColumnNamesInOracleTable(String tableName) {
+
+ List 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 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 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 getColumnTypes(String tableName) {
+
+ if (this.columnTypesInOracleTable == null) {
+
+ Map columnTypes = super.getColumnTypes(tableName);
+ this.columnTypesInOracleTable = new HashMap();
+
+ List 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);
+ }
+}
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 00000000..874ef02a
--- /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
+ }
+ }
+
+}
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 00000000..93efa769
--- /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 oracleDataChunks;
+
+ // NB: Update write(), readFields() and getDebugDetails() if you add fields
+ // here.
+
+ public OraOopDBInputSplit() {
+
+ this.splitId = -1;
+ this.splitLocation = "";
+ this.oracleDataChunks = new ArrayList();
+ }
+
+ public OraOopDBInputSplit(List dataChunks) {
+
+ setOracleDataChunks(dataChunks);
+ }
+
+ public void setOracleDataChunks(List dataChunks) {
+
+ this.oracleDataChunks = dataChunks;
+ }
+
+ public List 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(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;
+ }
+
+}
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 00000000..45a88ef2
--- /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 extends
+ DataDrivenDBRecordReader {
+
+ 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 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 result = new ArrayList();
+
+ 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;
+ }
+
+}
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 00000000..13f05d52
--- /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 Output type of the record reader
+ */
+public class OraOopDataDrivenDBInputFormat extends
+ DataDrivenDBInputFormat implements Configurable {
+
+ public static final OraOopLog LOG = OraOopLogFactory
+ .getLog(OraOopDataDrivenDBInputFormat.class.getName());
+
+ public OraOopDataDrivenDBInputFormat() {
+ super();
+ OraOopUtilities.checkJavaSecurityEgd();
+ }
+
+ @Override
+ public List 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 partitionList = getPartitionList(jobContext);
+
+ // Get our Oracle connection...
+ Connection connection = getConnection();
+
+ List 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 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 ? "" : 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 inputClass = (Class) (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(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 getPartitionList(JobContext jobContext) {
+ LOG.debug(OraOopConstants.ORAOOP_IMPORT_PARTITION_LIST
+ + " = "
+ + jobContext.getConfiguration().get(
+ OraOopConstants.ORAOOP_IMPORT_PARTITION_LIST));
+ List 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
+ 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 splits = new ArrayList(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;
+ }
+
+}
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 00000000..7307fcba
--- /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 {
+
+ private List objects;
+
+ public ObjectList() {
+
+ this.objects = new ArrayList();
+ }
+
+ 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 iterator() {
+
+ return this.objects.iterator();
+ }
+
+ }
+
+}
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 00000000..8502b796
--- /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::@::
+ * jdbc:oracle::@:/
+ * jdbc:oracle::@:/?
+ * jdbc:oracle::@//:/
+ * jdbc:oracle::@//:/?
+ */
+
+ // 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::@::\n"
+ + "\tjdbc:oracle::@:/\n"
+ + "\tjdbc:oracle::@:/?\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:@
+ 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::@::
+ portStr = jdbcFragments[4];
+ sid = jdbcFragments[5];
+ break;
+
+ case 5:
+ // jdbc:oracle::@:/
+ String[] portAndService = jdbcFragments[4].split("/");
+ if (portAndService.length != 2) {
+ throw new JdbcOracleThinConnectionParsingError(
+ "The fifth colon-separated item in the JDBC URL "
+ + "(/) 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::@::\n"
+ + "\tjdbc:oracle::@:/\n"
+ + "\tjdbc:oracle::@:/?\n"
+ + "\tjdbc:oracle::@//:/\n"
+ + "\tjdbc:oracle::@:/?");
+ }
+
+ // 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;
+ }
+
+}
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 00000000..5d5ee026
--- /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;
+ }
+
+}
+
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 00000000..d278225f
--- /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));
+ }
+
+}
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 00000000..2acba557
--- /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()));
+ }
+ }
+}
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 00000000..fcf4a4d8
--- /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 messagesToDisplayAfterWelcome =
+ new ArrayList();
+
+ 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 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 activeInstances, int jdbcPort,
+ JobData jobData) {
+
+ // Generate JDBC URLs for each of the instances in the RAC...
+ ArrayList
+ jdbcOracleActiveThinConnections =
+ new ArrayList(
+ 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 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 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 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 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())));
+ }
+ }
+}
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 00000000..bc94abd8
--- /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;
+ }
+
+}
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 00000000..5262d0ef
--- /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;
+ }
+ }
+
+}
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 00000000..fbd8a16b
--- /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();
+ }
+
+}
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 00000000..7fd18a18
--- /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 ORACLE_TYPES =
+ new HashMap();
+
+ 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 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 partitionList) throws SQLException {
+ int i = 0;
+ for (String partition : partitionList) {
+ i++;
+ OraOopOracleQueries.setStringAtName(statement, "part" + i, partition);
+ }
+ }
+
+ public static List
+ getOracleDataChunksPartition(Connection connection, OracleTable table,
+ List partitionList) throws SQLException {
+ List result =
+ new ArrayList();
+ 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 getOracleDataChunksExtent(
+ Configuration conf, Connection connection, OracleTable table,
+ List partitionList, int numberOfChunksPerOracleDataFile)
+ throws SQLException {
+
+ List result =
+ new ArrayList();
+
+ 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 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
+ 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 tables =
+ getTables(connection, owner, tableName, TableNameQueryType.Equals);
+ if (tables.size() > 0) {
+ return tables.get(0);
+ }
+
+ return null;
+ }
+
+ public static List getTablesWithTableNameLike(
+ Connection connection, String owner, String tableNameLike)
+ throws SQLException {
+
+ return getTables(connection, owner, tableNameLike, TableNameQueryType.Like);
+ }
+
+ private static List getTables(Connection connection,
+ String owner, String tableName, TableNameQueryType tableNameQueryType)
+ throws SQLException {
+
+ EnumSet 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 result = new ArrayList();
+ while (resultSet.next()) {
+ result.add(new OracleTable(resultSet.getString("owner"), resultSet
+ .getString("table_name")));
+ }
+
+ resultSet.close();
+ statement.close();
+
+ return result;
+ }
+
+ public static List getTableColumnNames(Connection connection,
+ OracleTable table) throws SQLException {
+
+ OracleTableColumns oracleTableColumns = getTableColumns(connection, table);
+ List result = new ArrayList(oracleTableColumns.size());
+
+ for (int idx = 0; idx < oracleTableColumns.size(); idx++) {
+ result.add(oracleTableColumns.get(idx).getName());
+ }
+
+ return result;
+ }
+
+ public static List 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 result = new ArrayList(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 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 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();
+ }
+ 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 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 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();
+ }
+
+ 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 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 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;
+ }
+}
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 00000000..7c4d1c5b
--- /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 extends
+ ExportOutputFormat {
+
+ 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 ? "" : 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.ExportRecordWriter {
+
+ 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 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 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 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 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));
+ }
+
+ }
+
+}
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 00000000..d5eebf4c
--- /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 extends
+ OraOopOutputFormatBase {
+
+ 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 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 userRecords) throws SQLException {
+
+ Map 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);
+ }
+ }
+
+ }
+
+ }
+
+}
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 00000000..a33768fc
--- /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 extends
+ OraOopOutputFormatBase {
+
+ 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 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 userRecords) throws SQLException {
+
+ Map 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);
+ }
+ }
+
+ }
+ }
+
+}
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 00000000..473a5ae9
--- /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 copyStringList(List list) {
+
+ List result = new ArrayList(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 duplicates = new ArrayList();
+
+ 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 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 splitStringList(String value) {
+ List result = new ArrayList();
+ 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 splitOracleStringList(String value) {
+ List result = new ArrayList();
+ List 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.");
+ }
+ }
+
+}
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 00000000..180da53a
--- /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;
+ }
+}
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 00000000..094576b0
--- /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 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 parseOraOopSessionInitializationStatements(
+ org.apache.hadoop.conf.Configuration conf) {
+
+ ArrayList result = new ArrayList();
+
+ 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;
+ }
+
+}
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 00000000..a0ecce13
--- /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;
+ }
+
+}
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 00000000..3b28e0cf
--- /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;
+ }
+}
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 00000000..ea6d549f
--- /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 {
+
+ public OracleTableColumn findColumnByName(String columnName) {
+
+ OracleTableColumn result;
+
+ Iterator iterator = this.iterator();
+ while (iterator.hasNext()) {
+ result = iterator.next();
+ if (result.getName().equals(columnName)) {
+ return result;
+ }
+ }
+ return null;
+ }
+
+}
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 00000000..26610109
--- /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;
+ }
+
+}
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 00000000..5795187c
--- /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 {
+
+ public OracleTablePartition findPartitionByName(String partitionName) {
+
+ OracleTablePartition result;
+
+ Iterator 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 iterator = this.iterator();
+ while (iterator.hasNext()) {
+ result = iterator.next();
+ Matcher matcher = pattern.matcher(result.getName());
+ if (matcher.find()) {
+ return result;
+ }
+ }
+ return null;
+ }
+
+}
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 00000000..0d900653
--- /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;
+ }
+}
diff --git a/src/test/com/cloudera/sqoop/manager/OracleUtils.java b/src/test/com/cloudera/sqoop/manager/OracleUtils.java
index 70e2925b..d9e31b8b 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) {
diff --git a/src/test/com/cloudera/sqoop/testutil/BaseSqoopTestCase.java b/src/test/com/cloudera/sqoop/testutil/BaseSqoopTestCase.java
index 793c23e9..a94ab909 100644
--- a/src/test/com/cloudera/sqoop/testutil/BaseSqoopTestCase.java
+++ b/src/test/com/cloudera/sqoop/testutil/BaseSqoopTestCase.java
@@ -220,6 +220,8 @@ public void setUp() {
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());
diff --git a/src/test/com/cloudera/sqoop/testutil/ExportJobTestCase.java b/src/test/com/cloudera/sqoop/testutil/ExportJobTestCase.java
index 4421f0c7..9a6e8daa 100644
--- a/src/test/com/cloudera/sqoop/testutil/ExportJobTestCase.java
+++ b/src/test/com/cloudera/sqoop/testutil/ExportJobTestCase.java
@@ -317,6 +317,8 @@ protected List runExport(String [] argv) throws IOException {
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);
diff --git a/src/test/com/cloudera/sqoop/testutil/ImportJobTestCase.java b/src/test/com/cloudera/sqoop/testutil/ImportJobTestCase.java
index 9fc8eba2..d5b7f8f6 100644
--- a/src/test/com/cloudera/sqoop/testutil/ImportJobTestCase.java
+++ b/src/test/com/cloudera/sqoop/testutil/ImportJobTestCase.java
@@ -115,6 +115,8 @@ protected void verifyImport(String expectedVal, String [] importCols) {
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 @@ protected void runImport(SqoopTool tool, String [] argv) throws IOException {
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);
diff --git a/src/test/oraoop/create_users.sql b/src/test/oraoop/create_users.sql
new file mode 100644
index 00000000..b9ccbb9b
--- /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;
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 00000000..0bc7df7d
--- /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
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 00000000..77052a5f
--- /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
diff --git a/src/test/oraoop/table_tst_product.xml b/src/test/oraoop/table_tst_product.xml
new file mode 100644
index 00000000..76c1a102
--- /dev/null
+++ b/src/test/oraoop/table_tst_product.xml
@@ -0,0 +1,90 @@
+
+
+
+ TST_PRODUCT
+
+
+ product_id
+ INTEGER
+ id
+
+
+ supplier_code
+ VARCHAR2 (30)
+ TO_CHAR (id - MOD (id, 5000),'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')
+
+
+ product_code
+ VARCHAR2 (30)
+ TO_CHAR (MOD (id, 100000), 'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')
+
+
+ product_descr
+ VARCHAR2 (255)
+ DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 100)))
+
+
+ product_long_descr
+ VARCHAR2 (4000)
+ DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 200)))
+
+
+ product_cost_price
+ NUMBER
+ ROUND (DBMS_RANDOM.VALUE (0, 100000), 2)
+
+
+ sell_from_date
+ DATE
+ TRUNC (SYSDATE + DBMS_RANDOM.VALUE (-365, 365))
+
+
+ sell_price
+ NUMBER
+ ROUND (DBMS_RANDOM.VALUE (0, 200000), 2)
+
+
+ create_user
+ VARCHAR2 (30)
+ DBMS_RANDOM.string ('U', 30)
+
+
+ create_time
+ TIMESTAMP
+ TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')
+
+
+ last_update_user
+ VARCHAR2 (30)
+ DBMS_RANDOM.string ('U', 30)
+
+
+ last_update_time
+ TIMESTAMP
+ TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')
+
+
+
+ product_id
+
+
+ supplier_code
+ product_code
+
+
\ No newline at end of file
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 00000000..6014476b
--- /dev/null
+++ b/src/test/oraoop/table_tst_product_part.xml
@@ -0,0 +1,103 @@
+
+
+
+ TST_PRODUCT_PART
+
+
+ product_id
+ INTEGER
+ id
+
+
+ supplier_code
+ VARCHAR2 (30)
+ TO_CHAR (id - MOD (id, 5000),'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')
+
+
+ product_code
+ VARCHAR2 (30)
+ TO_CHAR (MOD (id, 100000), 'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')
+
+
+ product_descr
+ VARCHAR2 (255)
+ DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 100)))
+
+
+ product_long_descr
+ VARCHAR2 (4000)
+ DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 200)))
+
+
+ product_cost_price
+ NUMBER
+ ROUND (DBMS_RANDOM.VALUE (0, 100000), 2)
+
+
+ sell_from_date
+ DATE
+ TRUNC (SYSDATE + DBMS_RANDOM.VALUE (-365, 365))
+
+
+ sell_price
+ NUMBER
+ ROUND (DBMS_RANDOM.VALUE (0, 200000), 2)
+
+
+ create_user
+ VARCHAR2 (30)
+ DBMS_RANDOM.string ('U', 30)
+
+
+ create_time
+ TIMESTAMP
+ TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')
+
+
+ last_update_user
+ VARCHAR2 (30)
+ DBMS_RANDOM.string ('U', 30)
+
+
+ last_update_time
+ TIMESTAMP
+ TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')
+
+
+
+ product_id
+
+
+ supplier_code
+ product_code
+
+
+ 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)
+ )
+
+
\ No newline at end of file
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 00000000..721f8089
--- /dev/null
+++ b/src/test/oraoop/table_tst_product_special_chars.xml
@@ -0,0 +1,90 @@
+
+
+
+ T5+_Pr#duct
+
+
+ product_id
+ INTEGER
+ id
+
+
+ supplier_code
+ VARCHAR2 (30)
+ TO_CHAR (id - MOD (id, 5000),'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')
+
+
+ product_code
+ VARCHAR2 (30)
+ TO_CHAR (MOD (id, 100000), 'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')
+
+
+ product_descr
+ VARCHAR2 (255)
+ DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 100)))
+
+
+ product_long_descr
+ VARCHAR2 (4000)
+ DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 200)))
+
+
+ product_cost_price
+ NUMBER
+ ROUND (DBMS_RANDOM.VALUE (0, 100000), 2)
+
+
+ sell_from_date
+ DATE
+ TRUNC (SYSDATE + DBMS_RANDOM.VALUE (-365, 365))
+
+
+ sell_price
+ NUMBER
+ ROUND (DBMS_RANDOM.VALUE (0, 200000), 2)
+
+
+ create_user
+ VARCHAR2 (30)
+ DBMS_RANDOM.string ('U', 30)
+
+
+ create_time
+ TIMESTAMP
+ TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')
+
+
+ last_update_user
+ VARCHAR2 (30)
+ DBMS_RANDOM.string ('U', 30)
+
+
+ last_update_time
+ TIMESTAMP
+ TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')
+
+
+
+ product_id
+
+
+ supplier_code
+ product_code
+
+
\ No newline at end of file
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 00000000..4883244b
--- /dev/null
+++ b/src/test/oraoop/table_tst_product_subpart.xml
@@ -0,0 +1,105 @@
+
+
+
+ TST_PRODUCT_SUBPART
+
+
+ product_id
+ INTEGER
+ id
+
+
+ supplier_code
+ VARCHAR2 (30)
+ TO_CHAR (id - MOD (id, 5000),'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')
+
+
+ product_code
+ VARCHAR2 (30)
+ TO_CHAR (MOD (id, 100000), 'FMXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX')
+
+
+ product_descr
+ VARCHAR2 (255)
+ DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 100)))
+
+
+ product_long_descr
+ VARCHAR2 (4000)
+ DBMS_RANDOM.string ('x', ROUND (DBMS_RANDOM.VALUE (1, 200)))
+
+
+ product_cost_price
+ NUMBER
+ ROUND (DBMS_RANDOM.VALUE (0, 100000), 2)
+
+
+ sell_from_date
+ DATE
+ TRUNC (SYSDATE + DBMS_RANDOM.VALUE (-365, 365))
+
+
+ sell_price
+ NUMBER
+ ROUND (DBMS_RANDOM.VALUE (0, 200000), 2)
+
+
+ create_user
+ VARCHAR2 (30)
+ DBMS_RANDOM.string ('U', 30)
+
+
+ create_time
+ TIMESTAMP
+ TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')
+
+
+ last_update_user
+ VARCHAR2 (30)
+ DBMS_RANDOM.string ('U', 30)
+
+
+ last_update_time
+ TIMESTAMP
+ TO_TIMESTAMP (TO_CHAR (SYSDATE + DBMS_RANDOM.VALUE (-730, 0),'YYYYMMDDHH24MISS') || '.' || TRUNC (TO_CHAR (DBMS_RANDOM.VALUE * 999999999)), 'YYYYMMDDHH24MISSXFF')
+
+
+
+ product_id
+
+
+ supplier_code
+ product_code
+
+
+ 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)
+ )
+
+
\ No newline at end of file
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 00000000..80b65362
--- /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();
+ }
+
+}
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 00000000..d914e3f6
--- /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();
+ }
+ }
+
+}
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 00000000..09314daf
--- /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();
+ }
+ }
+
+}
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 00000000..6f7d070a
--- /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 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 sqoopArgs = new ArrayList();
+
+ 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 sqoopArgs = new ArrayList();
+
+ 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);
+ }
+
+}
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 00000000..b2634f36
--- /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) ";
+
+}
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 00000000..9e6931bb
--- /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 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 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;
+ }
+
+}
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 00000000..0f85854b
--- /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) 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 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();
+ }
+ }
+}
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 00000000..7d3abfd2
--- /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 extends
+ OraOopDataDrivenDBInputFormat {
+
+ @Override
+ public
+ List
+ groupTableDataChunksIntoSplits(
+ List extends OraOopOracleDataChunk> dataChunks,
+ int desiredNumberOfSplits,
+ OraOopConstants.OraOopOracleBlockToSplitAllocationMethod
+ blockAllocationMethod) {
+
+ return super.groupTableDataChunksIntoSplits(dataChunks,
+ desiredNumberOfSplits, blockAllocationMethod);
+ }
+
+ }
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void testgroupTableDataChunksIntoSplits() {
+
+ List dataChunks =
+ new ArrayList();
+
+ 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 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));
+ }
+ }
+
+}
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 00000000..2cd77e63
--- /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);
+
+ }
+
+}
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 00000000..93592afa
--- /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 result = null;
+ List expected = null;
+
+ expected = new ArrayList();
+ expected.add("abcde");
+ expected.add("ghijklm");
+ result = OraOopUtilities.splitStringList("abcde,ghijklm");
+ Assert.assertEquals(expected, result);
+
+ expected = new ArrayList();
+ expected.add("\"abcde\"");
+ expected.add("\"ghijklm\"");
+ result = OraOopUtilities.splitStringList("\"abcde\",\"ghijklm\"");
+ Assert.assertEquals(expected, result);
+
+ expected = new ArrayList();
+ expected.add("abcde");
+ expected.add("\"ghijklm\"");
+ result = OraOopUtilities.splitStringList("abcde,\"ghijklm\"");
+ Assert.assertEquals(expected, result);
+
+ expected = new ArrayList();
+ expected.add("\"abcde\"");
+ expected.add("ghijklm");
+ result = OraOopUtilities.splitStringList("\"abcde\",ghijklm");
+ Assert.assertEquals(expected, result);
+
+ expected = new ArrayList();
+ expected.add("\"ab,cde\"");
+ expected.add("ghijklm");
+ result = OraOopUtilities.splitStringList("\"ab,cde\",ghijklm");
+ Assert.assertEquals(expected, result);
+
+ expected = new ArrayList();
+ expected.add("abcde");
+ expected.add("\"ghi,jklm\"");
+ result = OraOopUtilities.splitStringList("abcde,\"ghi,jklm\"");
+ Assert.assertEquals(expected, result);
+
+ expected = new ArrayList();
+ expected.add("\"ab,cde\"");
+ expected.add("\"ghi,jklm\"");
+ result = OraOopUtilities.splitStringList("\"ab,cde\",\"ghi,jklm\"");
+ Assert.assertEquals(expected, result);
+
+ expected = new ArrayList();
+ 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();
+ 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 result = null;
+ List expected = null;
+
+ expected = new ArrayList();
+ 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();
+ 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();
+ 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);
+ }
+}
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 00000000..854d826b
--- /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());
+ }
+
+}
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 00000000..f3385959
--- /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();
+ }
+ }
+
+}
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 00000000..5a7c698b
--- /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 {
+ 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;
+ }
+}
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 00000000..27fe34e5
--- /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 {
+ @Override
+ public Double next() {
+ return Double.longBitsToDouble(rng.nextLong());
+ }
+
+}
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 00000000..2e1be293
--- /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 {
+ @Override
+ public Float next() {
+ return Float.intBitsToFloat(rng.nextInt());
+ }
+
+}
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 00000000..6279f152
--- /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 {
+ 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);
+ }
+ }
+
+}
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 00000000..5b105e60
--- /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 {
+ 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;
+ }
+
+}
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 00000000..a0539096
--- /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 {
+ 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);
+ }
+
+}
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 00000000..f345d8de
--- /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 {
+ 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;
+ }
+
+}
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 00000000..6592036f
--- /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);
+ }
+
+}
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 00000000..a762643b
--- /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 {
+ 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;
+ }
+}
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 00000000..af572535
--- /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 {
+ 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);
+ }
+}
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 00000000..b3abe333
--- /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 {
+ 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);
+ }
+
+}
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 00000000..d8c5eb81
--- /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
+ * The type that will be generated
+ */
+public abstract class OraOopTestDataGenerator {
+ 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();
+}
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 00000000..c94560bc
--- /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;
+ }
+}
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 00000000..871d3173
--- /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 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 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 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);
+ }
+
+}
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 00000000..12568380
--- /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();
+ }
+
+}
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 00000000..5a8c42c1
--- /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 columnList =
+ new ArrayList();
+ private List primaryKeyColumns = new ArrayList();
+ private List uniqueKeyColumns = new ArrayList();
+ private String partitionClause;
+
+ public List getUniqueKeyColumns() {
+ return uniqueKeyColumns;
+ }
+
+ public void setUniqueKeyColumns(List newUniqueKeyColumns) {
+ this.uniqueKeyColumns = newUniqueKeyColumns;
+ }
+
+ public List getPrimaryKeyColumns() {
+ return primaryKeyColumns;
+ }
+
+ public void setPrimaryKeyColumns(List newPrimaryKeyColumns) {
+ this.primaryKeyColumns = newPrimaryKeyColumns;
+ }
+
+ public List